Skip to content

Commit

Permalink
Adds extended errors info to WriteResult
Browse files Browse the repository at this point in the history
  • Loading branch information
Carlos Alonso committed Jul 20, 2018
1 parent 0001afa commit fe00b20
Showing 1 changed file with 34 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@
*/
package org.apache.beam.sdk.io.gcp.bigquery;

import static com.google.common.base.Preconditions.checkArgument;

import com.google.api.services.bigquery.model.TableRow;
import com.google.common.collect.ImmutableMap;
import java.util.Map;
Expand All @@ -33,29 +35,58 @@ public final class WriteResult implements POutput {
private final Pipeline pipeline;
private final TupleTag<TableRow> failedInsertsTag;
private final PCollection<TableRow> failedInserts;
private final TupleTag<BigQueryInsertError> failedInsertsWithErrTag;
private final PCollection<BigQueryInsertError> failedInsertsWithErr;

/** Creates a {@link WriteResult} in the given {@link Pipeline}. */
static WriteResult in(
Pipeline pipeline, TupleTag<TableRow> failedInsertsTag, PCollection<TableRow> failedInserts) {
return new WriteResult(pipeline, failedInsertsTag, failedInserts);
return new WriteResult(pipeline, failedInsertsTag, failedInserts, null, null);
}

static WriteResult withExtendedErrors(
Pipeline pipeline,
TupleTag<BigQueryInsertError> failedInsertsTag,
PCollection<BigQueryInsertError> failedInserts) {
return new WriteResult(pipeline, null, null, failedInsertsTag, failedInserts);
}

@Override
public Map<TupleTag<?>, PValue> expand() {
return ImmutableMap.of(failedInsertsTag, failedInserts);
if (failedInsertsTag != null) {
return ImmutableMap.of(failedInsertsTag, failedInserts);
} else {
return ImmutableMap.of(failedInsertsWithErrTag, failedInsertsWithErr);
}
}

private WriteResult(
Pipeline pipeline, TupleTag<TableRow> failedInsertsTag, PCollection<TableRow> failedInserts) {
Pipeline pipeline,
TupleTag<TableRow> failedInsertsTag,
PCollection<TableRow> failedInserts,
TupleTag<BigQueryInsertError> failedInsertsWithErrTag,
PCollection<BigQueryInsertError> failedInsertsWithErr) {
this.pipeline = pipeline;
this.failedInsertsTag = failedInsertsTag;
this.failedInserts = failedInserts;
this.failedInsertsWithErrTag = failedInsertsWithErrTag;
this.failedInsertsWithErr = failedInsertsWithErr;
}

public PCollection<TableRow> getFailedInserts() {
checkArgument(failedInsertsTag != null,
"Cannot use getFailedInserts as this WriteResult uses extended errors"
+ " information. Use getFailedInsertsWithErr instead");
return failedInserts;
}

public PCollection<BigQueryInsertError> getFailedInsertsWithErr() {
checkArgument(failedInsertsWithErrTag != null,
"Cannot use getFailedInsertsWithErr as this WriteResult does not use"
+ " extended errors. Use getFailedInserts instead");
return failedInsertsWithErr;
}

@Override
public Pipeline getPipeline() {
return pipeline;
Expand Down

0 comments on commit fe00b20

Please sign in to comment.