Skip to content

Commit

Permalink
[CsvIO] update error and result handling. (#32023)
Browse files Browse the repository at this point in the history
Co-authored-by: Lahari Guduru <[email protected]>
  • Loading branch information
francisohara24 and lahariguduru authored Jul 30, 2024
1 parent 570f2f8 commit c624e02
Show file tree
Hide file tree
Showing 8 changed files with 332 additions and 91 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -24,23 +24,14 @@
import java.util.Optional;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.transforms.errorhandling.BadRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.Row;
import org.apache.commons.csv.CSVFormat;

/** Stores parameters needed for CSV record parsing. */
@AutoValue
abstract class CsvIOParseConfiguration<T> implements Serializable {

/** A Dead Letter Queue that returns potential errors with {@link BadRecord}. */
final PTransform<PCollection<BadRecord>, PCollection<BadRecord>> errorHandlerTransform =
new BadRecordOutput();

static <T> Builder<T> builder() {
return new AutoValue_CsvIOParseConfiguration.Builder<>();
}
Expand Down Expand Up @@ -84,20 +75,4 @@ final CsvIOParseConfiguration<T> build() {
return autoBuild();
}
}

private static class BadRecordOutput
extends PTransform<PCollection<BadRecord>, PCollection<BadRecord>> {

@Override
public PCollection<BadRecord> expand(PCollection<BadRecord> input) {
return input.apply(ParDo.of(new BadRecordTransformFn()));
}

private static class BadRecordTransformFn extends DoFn<BadRecord, BadRecord> {
@ProcessElement
public void process(@Element BadRecord input, OutputReceiver<BadRecord> receiver) {
receiver.output(input);
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,18 @@
*/
package org.apache.beam.sdk.io.csv;

import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;

import com.google.auto.value.AutoValue;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.schemas.AutoValueSchema;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.schemas.SchemaCoder;
import org.apache.beam.sdk.schemas.SchemaProvider;
import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.values.Row;
import org.apache.beam.sdk.values.TypeDescriptor;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.joda.time.Instant;

Expand All @@ -36,6 +45,21 @@ static Builder builder() {
return new AutoValue_CsvIOParseError.Builder();
}

private static final SchemaProvider SCHEMA_PROVIDER = new AutoValueSchema();

private static final TypeDescriptor<CsvIOParseError> TYPE =
TypeDescriptor.of(CsvIOParseError.class);

private static final Schema SCHEMA = checkStateNotNull(SCHEMA_PROVIDER.schemaFor(TYPE));

private static final SerializableFunction<CsvIOParseError, Row> TO_ROW_FN =
checkStateNotNull(SCHEMA_PROVIDER.toRowFunction(TYPE));

private static final SerializableFunction<Row, CsvIOParseError> FROM_ROW_FN =
checkStateNotNull(SCHEMA_PROVIDER.fromRowFunction(TYPE));

static final Coder<CsvIOParseError> CODER = SchemaCoder.of(SCHEMA, TYPE, TO_ROW_FN, FROM_ROW_FN);

/** The caught {@link Exception#getMessage()}. */
public abstract String getMessage();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,8 @@
*/
package org.apache.beam.sdk.io.csv;

import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.commons.csv.CSVRecord;
Expand All @@ -31,11 +30,17 @@
// TODO(https://github.com/apache/beam/issues/31873): implement class after all dependencies are
// completed.
class CsvIOParseKV<T>
extends PTransform<PCollection<KV<String, Iterable<String>>>, PCollection<T>> {
extends PTransform<PCollection<KV<String, Iterable<String>>>, CsvIOParseResult<T>> {

private final Coder<T> outputCoder;

private CsvIOParseKV(Coder<T> outputCoder) {
this.outputCoder = outputCoder;
}

// TODO(https://github.com/apache/beam/issues/31873): implement method.
@Override
public PCollection<T> expand(PCollection<KV<String, Iterable<String>>> input) {
return input.apply(ParDo.of(new DoFn<KV<String, Iterable<String>>, T>() {}));
public CsvIOParseResult<T> expand(PCollection<KV<String, Iterable<String>>> input) {
return CsvIOParseResult.empty(input.getPipeline(), outputCoder);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,100 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.sdk.io.csv;

import java.util.Map;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.beam.sdk.values.PInput;
import org.apache.beam.sdk.values.POutput;
import org.apache.beam.sdk.values.PValue;
import org.apache.beam.sdk.values.TupleTag;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;

/**
* The {@link T} and {@link CsvIOParseError} {@link PCollection} results of parsing CSV records. Use
* {@link #getOutput()} and {@link #getErrors()} to apply these results in a pipeline.
*/
public class CsvIOParseResult<T> implements POutput {

static <T> CsvIOParseResult<T> of(
TupleTag<T> outputTag,
Coder<T> outputCoder,
TupleTag<CsvIOParseError> errorTag,
PCollectionTuple pct) {
return new CsvIOParseResult<>(outputTag, outputCoder, errorTag, pct);
}

static <T> CsvIOParseResult<T> empty(Pipeline pipeline, Coder<T> outputCoder) {
return new CsvIOParseResult<>(
new TupleTag<T>() {},
outputCoder,
new TupleTag<CsvIOParseError>() {},
PCollectionTuple.empty(pipeline));
}

private final Pipeline pipeline;
private final TupleTag<T> outputTag;
private final PCollection<T> output;
private final TupleTag<CsvIOParseError> errorTag;
private final PCollection<CsvIOParseError> errors;

private CsvIOParseResult(
TupleTag<T> outputTag,
Coder<T> outputCoder,
TupleTag<CsvIOParseError> errorTag,
PCollectionTuple pct) {
this.outputTag = outputTag;
this.errorTag = errorTag;
this.pipeline = pct.getPipeline();
this.output = pct.get(outputTag).setCoder(outputCoder);
this.errors = pct.get(errorTag).setCoder(CsvIOParseError.CODER);
}

/** The {@link T} {@link PCollection} as a result of successfully parsing CSV records. */
public PCollection<T> getOutput() {
return output;
}

/**
* The {@link CsvIOParseError} {@link PCollection} as a result of errors associated with parsing
* CSV records.
*/
public PCollection<CsvIOParseError> getErrors() {
return errors;
}

@Override
public Pipeline getPipeline() {
return pipeline;
}

@Override
public Map<TupleTag<?>, PValue> expand() {
return ImmutableMap.of(
outputTag, output,
errorTag, errors);
}

@Override
public void finishSpecifyingOutput(
String transformName, PInput input, PTransform<?, ?> transform) {}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,20 +20,26 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.beam.sdk.values.Row;
import org.apache.beam.sdk.values.TupleTag;
import org.apache.beam.sdk.values.TupleTagList;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Throwables;
import org.joda.time.Instant;

/**
* {@link CsvIORecordToObjects} is a class that takes an input of {@link PCollection<List<String>>}
* and outputs custom type {@link PCollection<T>}.
*/
class CsvIORecordToObjects<T> extends PTransform<PCollection<List<String>>, PCollection<T>> {
class CsvIORecordToObjects<T> extends PTransform<PCollection<List<String>>, CsvIOParseResult<T>> {

/** The expected {@link Schema} of the target type. */
private final Schema schema;
Expand All @@ -44,6 +50,10 @@ class CsvIORecordToObjects<T> extends PTransform<PCollection<List<String>>, PCol
/** A {@link Map} of {@link Schema.Field}s to their expected positions within the CSV record. */
private final Map<Integer, Schema.Field> indexToFieldMap;

private final TupleTag<T> outputTag = new TupleTag<T>() {};

private final TupleTag<CsvIOParseError> errorTag = new TupleTag<CsvIOParseError>() {};

/**
* A {@link SerializableFunction} that converts from {@link Row} to {@link Schema} mapped custom
* type.
Expand All @@ -63,23 +73,40 @@ class CsvIORecordToObjects<T> extends PTransform<PCollection<List<String>>, PCol
}

@Override
public PCollection<T> expand(PCollection<List<String>> input) {
return input.apply(ParDo.of(new RecordToObjectsFn())).setCoder(coder);
public CsvIOParseResult<T> expand(PCollection<List<String>> input) {
PCollectionTuple pct =
input.apply(
RecordToObjectsFn.class.getSimpleName(),
ParDo.of(new RecordToObjectsFn()).withOutputTags(outputTag, TupleTagList.of(errorTag)));

return CsvIOParseResult.of(outputTag, coder, errorTag, pct);
}

private class RecordToObjectsFn extends DoFn<List<String>, T> {
@ProcessElement
public void process(@Element List<String> record, OutputReceiver<T> receiver) {
public void process(@Element List<String> record, MultiOutputReceiver receiver) {
Map<String, Object> fieldNamesToValues = new HashMap<>();
for (Map.Entry<Integer, Schema.Field> entry : indexToFieldMap.entrySet()) {
Schema.Field field = entry.getValue();
int index = entry.getKey();
String cell = record.get(index);
Object value = parseCell(cell, field);
fieldNamesToValues.put(field.getName(), value);
try {
for (Map.Entry<Integer, Schema.Field> entry : indexToFieldMap.entrySet()) {
Schema.Field field = entry.getValue();
int index = entry.getKey();
String cell = record.get(index);
Object value = parseCell(cell, field);
fieldNamesToValues.put(field.getName(), value);
}
Row row = Row.withSchema(schema).withFieldValues(fieldNamesToValues).build();
receiver.get(outputTag).output(fromRowFn.apply(row));
} catch (RuntimeException e) {
receiver
.get(errorTag)
.output(
CsvIOParseError.builder()
.setCsvRecord(record.toString())
.setMessage(Optional.ofNullable(e.getMessage()).orElse(""))
.setStackTrace(Throwables.getStackTraceAsString(e))
.setObservedTimestamp(Instant.now())
.build());
}
Row row = Row.withSchema(schema).withFieldValues(fieldNamesToValues).build();
receiver.output(fromRowFn.apply(row));
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,26 +20,37 @@
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import org.apache.beam.sdk.coders.ListCoder;
import org.apache.beam.sdk.coders.NullableCoder;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.beam.sdk.values.TupleTag;
import org.apache.beam.sdk.values.TupleTagList;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Throwables;
import org.apache.commons.csv.CSVFormat;
import org.apache.commons.csv.CSVParser;
import org.apache.commons.csv.CSVRecord;
import org.joda.time.Instant;

/**
* {@link CsvIOStringToCsvRecord} is a class that takes a {@link PCollection<String>} input and
* outputs a {@link PCollection<CSVRecord>} with potential {@link PCollection<CsvIOParseError>} for
* targeted error detection.
*/
final class CsvIOStringToCsvRecord
extends PTransform<PCollection<String>, PCollection<List<String>>> {
extends PTransform<PCollection<String>, CsvIOParseResult<List<String>>> {

private final CSVFormat csvFormat;

private final TupleTag<List<String>> outputTag = new TupleTag<List<String>>() {};

private final TupleTag<CsvIOParseError> errorTag = new TupleTag<CsvIOParseError>() {};

CsvIOStringToCsvRecord(CSVFormat csvFormat) {
this.csvFormat = csvFormat;
}
Expand All @@ -49,24 +60,40 @@ final class CsvIOStringToCsvRecord
* to Row or custom type.
*/
@Override
public PCollection<List<String>> expand(PCollection<String> input) {
return input
.apply(ParDo.of(new ProcessLineToRecordFn()))
.setCoder(ListCoder.of(NullableCoder.of(StringUtf8Coder.of())));
public CsvIOParseResult<List<String>> expand(PCollection<String> input) {
PCollectionTuple pct =
input.apply(
ProcessLineToRecordFn.class.getSimpleName(),
ParDo.of(new ProcessLineToRecordFn())
.withOutputTags(outputTag, TupleTagList.of(errorTag)));

return CsvIOParseResult.of(
outputTag, ListCoder.of(NullableCoder.of(StringUtf8Coder.of())), errorTag, pct);
}

/** Processes each line in order to convert it to a {@link CSVRecord}. */
private class ProcessLineToRecordFn extends DoFn<String, List<String>> {
private final String headerLine = headerLine(csvFormat);

@ProcessElement
public void process(@Element String line, OutputReceiver<List<String>> receiver)
throws IOException {
public void process(@Element String line, MultiOutputReceiver receiver) {
if (headerLine.equals(line)) {
return;
}
for (CSVRecord record : CSVParser.parse(line, csvFormat).getRecords()) {
receiver.output(csvRecordtoList(record));
try (CSVParser csvParser = CSVParser.parse(line, csvFormat)) {
for (CSVRecord record : csvParser.getRecords()) {
receiver.get(outputTag).output(csvRecordtoList(record));
}
} catch (IOException e) {
receiver
.get(errorTag)
.output(
CsvIOParseError.builder()
.setCsvRecord(line)
.setMessage(Optional.ofNullable(e.getMessage()).orElse(""))
.setObservedTimestamp(Instant.now())
.setStackTrace(Throwables.getStackTraceAsString(e))
.build());
}
}
}
Expand Down
Loading

0 comments on commit c624e02

Please sign in to comment.