Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Feature/add error handling for bqio #30081

Merged
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 2 additions & 2 deletions sdks/java/io/google-cloud-platform/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -210,8 +210,8 @@ task integrationTest(type: Test, dependsOn: processTestResources) {

include '**/*IT.class'
exclude '**/BigQueryIOReadIT.class'
exclude '**/BigQueryIOStorageQueryIT.class'
exclude '**/BigQueryIOStorageReadIT.class'
// exclude '**/BigQueryIOStorageQueryIT.class'
// exclude '**/BigQueryIOStorageReadIT.class'
exclude '**/BigQueryIOStorageWriteIT.class'
exclude '**/BigQueryToTableIT.class'

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -113,7 +113,9 @@
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.DoFn.Element;
import org.apache.beam.sdk.transforms.DoFn.MultiOutputReceiver;
import org.apache.beam.sdk.transforms.DoFn.ProcessElement;
import org.apache.beam.sdk.transforms.MapElements;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
Expand Down Expand Up @@ -1452,28 +1454,65 @@ private PCollection<T> expandForDirectRead(
PBegin input, Coder<T> outputCoder, Schema beamSchema, BigQueryOptions bqOptions) {
ValueProvider<TableReference> tableProvider = getTableProvider();
Pipeline p = input.getPipeline();
if (tableProvider != null && getBadRecordRouter() instanceof ThrowingBadRecordRouter) {
// No job ID is required. Read directly from BigQuery storage.
PCollection<T> rows =
p.apply(
org.apache.beam.sdk.io.Read.from(
BigQueryStorageTableSource.create(
tableProvider,
getFormat(),
getSelectedFields(),
getRowRestriction(),
getParseFn(),
outputCoder,
getBigQueryServices(),
getProjectionPushdownApplied())));
if (beamSchema != null) {
rows.setSchema(
beamSchema,
getTypeDescriptor(),
getToBeamRowFn().apply(beamSchema),
getFromBeamRowFn().apply(beamSchema));
if (tableProvider != null) {
// ThrowingBadRecordRouter is the default value, and is what is used if the user hasn't
// specified any particular error handling.
if (getBadRecordRouter() instanceof ThrowingBadRecordRouter) {
// No job ID is required. Read directly from BigQuery storage.
PCollection<T> rows =
p.apply(
org.apache.beam.sdk.io.Read.from(
BigQueryStorageTableSource.create(
tableProvider,
getFormat(),
getSelectedFields(),
getRowRestriction(),
getParseFn(),
outputCoder,
getBigQueryServices(),
getProjectionPushdownApplied())));
if (beamSchema != null) {
rows.setSchema(
beamSchema,
getTypeDescriptor(),
getToBeamRowFn().apply(beamSchema),
getFromBeamRowFn().apply(beamSchema));
}
return rows;
} else {
// We need to manually execute the table source, so as to be able to capture exceptions
// to pipe to error handling
BigQueryStorageTableSource<T> source =
BigQueryStorageTableSource.create(
tableProvider,
getFormat(),
getSelectedFields(),
getRowRestriction(),
getParseFn(),
outputCoder,
getBigQueryServices(),
getProjectionPushdownApplied());
List<? extends BoundedSource<T>> sources;
try {
sources = source.split(0, input.getPipeline().getOptions());
johnjcasey marked this conversation as resolved.
Show resolved Hide resolved
} catch (Exception e) {
throw new RuntimeException("Unable to split TableSource", e);
}
TupleTag<T> rowTag = new TupleTag<>();
PCollectionTuple resultTuple =
p.apply(Create.of(sources))
.apply(
"Read Storage Table Source",
ParDo.of(new ReadTableSource<T>(rowTag, getParseFn(), getBadRecordRouter()))
.withOutputTags(rowTag, TupleTagList.of(BAD_RECORD_TAG)));
getBadRecordErrorHandler()
.addErrorCollection(
resultTuple
.get(BAD_RECORD_TAG)
.setCoder(BadRecord.getCoder(input.getPipeline())));

return resultTuple.get(rowTag).setCoder(outputCoder);
}
return rows;
}

checkArgument(
Expand Down Expand Up @@ -1613,6 +1652,52 @@ void cleanup(ContextContainer c) throws Exception {
return rows.apply(new PassThroughThenCleanup<>(cleanupOperation, jobIdTokenView));
}

private static class ReadTableSource<T> extends DoFn<BoundedSource<T>, T> {

private final TupleTag<T> rowTag;

private final SerializableFunction<SchemaAndRecord, T> parseFn;

private final BadRecordRouter badRecordRouter;

public ReadTableSource(
TupleTag<T> rowTag,
SerializableFunction<SchemaAndRecord, T> parseFn,
BadRecordRouter badRecordRouter) {
this.rowTag = rowTag;
this.parseFn = parseFn;
this.badRecordRouter = badRecordRouter;
}

@ProcessElement
public void processElement(
@Element BoundedSource<T> boundedSource,
MultiOutputReceiver outputReceiver,
PipelineOptions options)
throws Exception {
ErrorHandlingParseFn<T> errorHandlingParseFn = new ErrorHandlingParseFn<T>(parseFn);
BoundedSource<T> sourceWithErrorHandlingParseFn;
if (boundedSource instanceof BigQueryStorageStreamSource) {
sourceWithErrorHandlingParseFn =
((BigQueryStorageStreamSource<T>) boundedSource).fromExisting(errorHandlingParseFn);
} else if (boundedSource instanceof BigQueryStorageStreamBundleSource) {
sourceWithErrorHandlingParseFn =
((BigQueryStorageStreamBundleSource<T>) boundedSource)
.fromExisting(errorHandlingParseFn);
} else {
throw new RuntimeException(
"Bounded Source is not BigQueryStorageStreamSource or BigQueryStorageStreamBundleSource, unable to read");
}
readSource(
options,
rowTag,
outputReceiver,
sourceWithErrorHandlingParseFn,
errorHandlingParseFn,
badRecordRouter);
}
}

private PCollectionTuple createTupleForDirectRead(
PCollection<String> jobIdTokenCollection,
Coder<T> outputCoder,
Expand Down Expand Up @@ -1749,10 +1834,11 @@ public void processElement(ProcessContext c) throws Exception {
return tuple;
}

private class ErrorHandlingParseFn implements SerializableFunction<SchemaAndRecord, T> {
private static class ErrorHandlingParseFn<T>
implements SerializableFunction<SchemaAndRecord, T> {
private final SerializableFunction<SchemaAndRecord, T> parseFn;

private SchemaAndRecord schemaAndRecord = null;
private transient SchemaAndRecord schemaAndRecord = null;

private ErrorHandlingParseFn(SerializableFunction<SchemaAndRecord, T> parseFn) {
this.parseFn = parseFn;
Expand Down Expand Up @@ -1803,8 +1889,8 @@ public void processElement(
c.sideInput(tableSchemaView), TableSchema.class);
ReadStream readStream = c.element();

ErrorHandlingParseFn errorHandlingParseFn =
new ErrorHandlingParseFn(getParseFn());
ErrorHandlingParseFn<T> errorHandlingParseFn =
new ErrorHandlingParseFn<T>(getParseFn());

BigQueryStorageStreamSource<T> streamSource =
BigQueryStorageStreamSource.create(
Expand All @@ -1815,12 +1901,13 @@ public void processElement(
outputCoder,
getBigQueryServices());

readStreamSource(
readSource(
c.getPipelineOptions(),
rowTag,
outputReceiver,
streamSource,
errorHandlingParseFn);
errorHandlingParseFn,
getBadRecordRouter());
}
})
.withSideInputs(readSessionView, tableSchemaView)
Expand Down Expand Up @@ -1857,8 +1944,8 @@ public void processElement(
c.sideInput(tableSchemaView), TableSchema.class);
List<ReadStream> streamBundle = c.element();

ErrorHandlingParseFn errorHandlingParseFn =
new ErrorHandlingParseFn(getParseFn());
ErrorHandlingParseFn<T> errorHandlingParseFn =
new ErrorHandlingParseFn<T>(getParseFn());

BigQueryStorageStreamBundleSource<T> streamSource =
BigQueryStorageStreamBundleSource.create(
Expand All @@ -1870,12 +1957,13 @@ public void processElement(
getBigQueryServices(),
1L);

readStreamSource(
readSource(
c.getPipelineOptions(),
rowTag,
outputReceiver,
streamSource,
errorHandlingParseFn);
errorHandlingParseFn,
getBadRecordRouter());
}
})
.withSideInputs(readSessionView, tableSchemaView)
Expand All @@ -1888,12 +1976,13 @@ public void processElement(
return resultTuple.get(rowTag).setCoder(outputCoder);
}

public void readStreamSource(
public static <T> void readSource(
PipelineOptions options,
TupleTag<T> rowTag,
MultiOutputReceiver outputReceiver,
BoundedSource<T> streamSource,
ErrorHandlingParseFn errorHandlingParseFn)
ErrorHandlingParseFn<T> errorHandlingParseFn,
BadRecordRouter badRecordRouter)
throws Exception {
// Read all the data from the stream. In the event that this work
// item fails and is rescheduled, the same rows will be returned in
Expand All @@ -1908,13 +1997,12 @@ public void readStreamSource(
}
} catch (ParseException e) {
GenericRecord record = errorHandlingParseFn.getSchemaAndRecord().getRecord();
getBadRecordRouter()
.route(
outputReceiver,
record,
AvroCoder.of(record.getSchema()),
(Exception) e.getCause(),
"Unable to parse record reading from BigQuery");
badRecordRouter.route(
outputReceiver,
record,
AvroCoder.of(record.getSchema()),
(Exception) e.getCause(),
"Unable to parse record reading from BigQuery");
}

while (true) {
Expand All @@ -1926,13 +2014,12 @@ public void readStreamSource(
}
} catch (ParseException e) {
GenericRecord record = errorHandlingParseFn.getSchemaAndRecord().getRecord();
getBadRecordRouter()
.route(
outputReceiver,
record,
AvroCoder.of(record.getSchema()),
(Exception) e.getCause(),
"Unable to parse record reading from BigQuery");
badRecordRouter.route(
outputReceiver,
record,
AvroCoder.of(record.getSchema()),
(Exception) e.getCause(),
"Unable to parse record reading from BigQuery");
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -110,6 +110,18 @@ public BigQueryStorageStreamBundleSource<T> fromExisting(List<ReadStream> newStr
getMinBundleSize());
}

public BigQueryStorageStreamBundleSource<T> fromExisting(
SerializableFunction<SchemaAndRecord, T> parseFn) {
return new BigQueryStorageStreamBundleSource<>(
readSession,
streamBundle,
jsonTableSchema,
parseFn,
outputCoder,
bqServices,
getMinBundleSize());
}

private final ReadSession readSession;
private final List<ReadStream> streamBundle;
private final String jsonTableSchema;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,12 @@ public BigQueryStorageStreamSource<T> fromExisting(ReadStream newReadStream) {
readSession, newReadStream, jsonTableSchema, parseFn, outputCoder, bqServices);
}

public BigQueryStorageStreamSource<T> fromExisting(
SerializableFunction<SchemaAndRecord, T> parseFn) {
return new BigQueryStorageStreamSource<>(
readSession, readStream, jsonTableSchema, parseFn, outputCoder, bqServices);
}

private final ReadSession readSession;
private final ReadStream readStream;
private final String jsonTableSchema;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -183,8 +183,7 @@ public void processElement(
try {
tableRow = messageConverter.toTableRow(element.getValue());
} catch (Exception e) {
badRecordRouter.route(
o, element, elementCoder, e, "Unable to convert value to StorageWriteApiPayload");
badRecordRouter.route(o, element, elementCoder, e, "Unable to convert value to TableRow");
return;
}
o.get(failedWritesTag)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -248,11 +248,15 @@ public void processElement(
} catch (BigQueryRowSerializationException e) {
try {
badRecordRouter.route(
outputReceiver, element, coder, e, "Unable to Write BQ Record to File");
outputReceiver,
element,
coder,
e,
"Unable to Write BQ Record to File because serialization to TableRow failed");
} catch (Exception e2) {
cleanupWriter(writer, e2);
}
} catch (IOException e) {
} catch (Exception e) {
cleanupWriter(writer, e);
}
}
Expand Down
Loading
Loading