diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AppendClientInfo.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AppendClientInfo.java index 4761c8074283..e9adc8097604 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AppendClientInfo.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AppendClientInfo.java @@ -33,6 +33,7 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.util.Preconditions; /** * Container class used by {@link StorageApiWritesShardedRecords} and {@link @@ -138,6 +139,11 @@ public void close() { } } + @Memoized + public byte[] getTableSchemaHash() { + return TableRowToStorageApiProto.tableSchemaHash(getTableSchema()); + } + boolean hasSchemaChanged(TableSchema updatedTableSchema) { return updatedTableSchema.hashCode() != getTableSchema().hashCode(); } @@ -145,15 +151,17 @@ boolean hasSchemaChanged(TableSchema updatedTableSchema) { public ByteString encodeUnknownFields(TableRow unknown, boolean ignoreUnknownValues) throws TableRowToStorageApiProto.SchemaConversionException { Message msg = - TableRowToStorageApiProto.messageFromTableRow( - getSchemaInformation(), - getDescriptorIgnoreRequired(), - unknown, - ignoreUnknownValues, - true, - null, - null, - null); + Preconditions.checkArgumentNotNull( + TableRowToStorageApiProto.messageFromTableRow( + getSchemaInformation(), + getDescriptorIgnoreRequired(), + unknown, + ignoreUnknownValues, + true, + null, + null, + null, + TableRowToStorageApiProto.ErrorCollector.DONT_COLLECT)); return msg.toByteString(); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 5c035cca08ea..6e3b18ab6ecf 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -3246,8 +3246,12 @@ public Write withWriteDisposition(WriteDisposition writeDisposition) { /** * Allows the schema of the destination table to be updated as a side effect of the write. * - *

This configuration applies only when writing to BigQuery with {@link Method#FILE_LOADS} as + *

This configuration applies only when writing to BigQuery with {@link Method#FILE_LOADS}, + * {@link Method#STORAGE_WRITE_API", or {@link Method#STORAGE_API_AT_LEAST_ONCE} as * method. + *

If using with storage-write API, new fields (except for nested messages) will always be created with type + * STRING. + * TODO: Followon PR will add support for a user-supplied type mapping. */ public Write withSchemaUpdateOptions(Set schemaUpdateOptions) { checkArgument(schemaUpdateOptions != null, "schemaUpdateOptions can not be null"); @@ -4202,9 +4206,19 @@ private WriteResult continueExpandTyped( } return input.apply(batchLoads); } else if (method == Method.STORAGE_WRITE_API || method == Method.STORAGE_API_AT_LEAST_ONCE) { + boolean useSchemaUpdate = + getSchemaUpdateOptions() != null && !getSchemaUpdateOptions().isEmpty(); + if (useSchemaUpdate) { + checkArgument( + !getAutoSchemaUpdate(), + "Schema update options are not supported when using auto schema update"); + checkArgument(!getIgnoreUnknownValues()); + } BigQueryOptions bqOptions = input.getPipeline().getOptions().as(BigQueryOptions.class); StorageApiDynamicDestinations storageApiDynamicDestinations; if (getUseBeamSchema()) { + checkArgument( + !useSchemaUpdate, "SchemaUpdateOptions are not supported when using Beam schemas"); // This ensures that the Beam rows are directly translated into protos for Storage API // writes, with no // need to round trip through JSON TableRow objects. @@ -4216,6 +4230,9 @@ private WriteResult continueExpandTyped( getFormatRecordOnFailureFunction(), getRowMutationInformationFn() != null); } else if (getWriteProtosClass() != null && getDirectWriteProtos()) { + checkArgument( + !useSchemaUpdate, "SchemaUpdateOptions are not supported when writing protos"); + // We could support both of these by falling back to // StorageApiDynamicDestinationsTableRow. This // would defeat the optimization (we would be forced to create a new dynamic proto message @@ -4233,6 +4250,8 @@ private WriteResult continueExpandTyped( !getIgnoreUnknownValues(), "ignoreUnknownValues not supported when using writeProtos." + " Try setting withDirectWriteProtos(false)"); + checkArgument(!useSchemaUpdate); + storageApiDynamicDestinations = (StorageApiDynamicDestinations) new StorageApiDynamicDestinationsProto( @@ -4240,6 +4259,9 @@ private WriteResult continueExpandTyped( getWriteProtosClass(), getFormatRecordOnFailureFunction()); } else if (getAvroRowWriterFactory() != null) { + checkArgument( + !useSchemaUpdate, "SchemaUpdateOptions are not supported when writing avros"); + // we can configure the avro to storage write api proto converter for this // assuming the format function returns an Avro GenericRecord // and there is a schema defined @@ -4248,7 +4270,6 @@ private WriteResult continueExpandTyped( || getDynamicDestinations() != null || getSchemaFromView() != null, "A schema must be provided for avro rows to be used with StorageWrite API."); - RowWriterFactory.AvroRowWriterFactory recordWriterFactory = (RowWriterFactory.AvroRowWriterFactory) @@ -4275,7 +4296,10 @@ private WriteResult continueExpandTyped( getRowMutationInformationFn() != null, getCreateDisposition(), getIgnoreUnknownValues(), - getAutoSchemaUpdate()); + getAutoSchemaUpdate(), + getSchemaUpdateOptions() == null + ? Collections.emptySet() + : getSchemaUpdateOptions()); } int numShards = getStorageApiNumStreams(bqOptions); @@ -4287,6 +4311,7 @@ private WriteResult continueExpandTyped( StorageApiLoads storageApiLoads = new StorageApiLoads<>( destinationCoder, + elementCoder, storageApiDynamicDestinations, getRowMutationInformationFn(), getCreateDisposition(), @@ -4304,7 +4329,8 @@ private WriteResult continueExpandTyped( getDefaultMissingValueInterpretation(), getBigLakeConfiguration(), getBadRecordRouter(), - getBadRecordErrorHandler()); + getBadRecordErrorHandler(), + !getSchemaUpdateOptions().isEmpty()); return input.apply("StorageApiLoads", storageApiLoads); } else { throw new RuntimeException("Unexpected write method " + method); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java index e02a150a4de6..2f16a64b0d76 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryOptions.java @@ -245,4 +245,12 @@ public interface BigQueryOptions Boolean getGroupFilesFileLoad(); void setGroupFilesFileLoad(Boolean value); + + @Hidden + @Description( + "The number of parallelization to use for buffering elements when upgrading table schemas.") + @Default.Integer(50) + Integer getSchemaUpgradeBufferingShards(); + + void setSchemaUpgradeBufferingShards(Integer value); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java index 8dbc47359b70..78e714a7ccd6 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java @@ -204,6 +204,10 @@ long insertAll( /** Patch BigQuery {@link Table} description. */ Table patchTableDescription(TableReference tableReference, @Nullable String tableDescription) throws IOException, InterruptedException; + + Table patchTableSchema( + TableReference tableReference, com.google.api.services.bigquery.model.TableSchema newSchema) + throws IOException, InterruptedException; } /** An interface to get, create and flush Cloud BigQuery STORAGE API write streams. */ diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java index 60593da55093..2a9cc7649c21 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java @@ -1459,6 +1459,31 @@ public Table patchTableDescription( ALWAYS_RETRY); } + @Override + public Table patchTableSchema( + TableReference tableReference, com.google.api.services.bigquery.model.TableSchema newSchema) + throws IOException, InterruptedException { + Table newTable = new Table(); + newTable.setSchema(newSchema); + + Tables.Patch request = + client + .tables() + .patch( + tableReference.getProjectId(), + tableReference.getDatasetId(), + tableReference.getTableId(), + newTable); + return executeWithRetries( + request, + String.format( + "Unable to patch table: %s, aborting after %d retries.", + tableReference, MAX_RPC_RETRIES), + Sleeper.DEFAULT, + createDefaultBackoff(), + DONT_RETRY_INVALID_ARG_OR_PRECONDITION); + } + @Override public void close() throws Exception { // Nothing to close @@ -1664,6 +1689,11 @@ public void close() throws Exception { return !errorExtractor.itemNotFound(input); }; + static final SerializableFunction DONT_RETRY_INVALID_ARG_OR_PRECONDITION = + input -> { + ApiErrorExtractor errorExtractor = new ApiErrorExtractor(); + return !errorExtractor.preconditionNotMet(input) && !errorExtractor.badRequest(input); + }; static final SerializableFunction ALWAYS_RETRY = input -> true; @VisibleForTesting diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetrics.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetrics.java index 1ead98a4feea..bd887c0e1f97 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetrics.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetrics.java @@ -46,6 +46,7 @@ public class BigQuerySinkMetrics { public static final String METRICS_NAMESPACE = "BigQuerySink"; + // Status codes // Status codes public static final String UNKNOWN = Status.Code.UNKNOWN.toString(); public static final String OK = Status.Code.OK.toString(); @@ -63,7 +64,9 @@ public enum RpcMethod { STREAMING_INSERTS, APPEND_ROWS, FLUSH_ROWS, - FINALIZE_STREAM + FINALIZE_STREAM, + PATCH_TABLE, + OPEN_WRITE_STREAM } // Status of a BigQuery row from the AppendRows RPC call. diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ConvertMessagesDoFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ConvertMessagesDoFn.java new file mode 100644 index 000000000000..f84a3150b3bf --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ConvertMessagesDoFn.java @@ -0,0 +1,296 @@ +/* + * 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.gcp.bigquery; + +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.bigquery.storage.v1.TableSchema; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.errorhandling.BadRecordRouter; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.util.Preconditions; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Instant; + +/** + * DoFn that interacts with the StorageApiDynamicDestinations instance to convert messages to + * StorageApiWritePaylod. Messages that fail to convert are routed to the dead-letter PCollection. + * If schemaUpdateOptions are set, then messages that fail to convert due to missing columns are + * routed to a buffering transform that holds them until the table's schema has been updated. + */ +public class ConvertMessagesDoFn + extends DoFn, KV> { + private final StorageApiDynamicDestinations dynamicDestinations; + private TwoLevelMessageConverterCache messageConverters; + private final BigQueryServices bqServices; + private final TupleTag failedWritesTag; + private final TupleTag> successfulWritesTag; + private final TupleTag> patchTableSchemaTag; + private final TupleTag> retryElementsWaitingForSchemaTag; + private final @Nullable SerializableFunction rowMutationFn; + private final BadRecordRouter badRecordRouter; + private final Coder> elementCoder; + private final Map errorCollectors = Maps.newHashMap(); + private final boolean hasSchemaUpdateOptions; + private transient BigQueryServices.@Nullable DatasetService datasetServiceInternal = null; + private transient BigQueryServices.@Nullable WriteStreamService writeStreamServiceInternal = null; + + static final class BufferedCollectorInformation { + TableRowToStorageApiProto.@Nullable ErrorCollector collector = null; + final TableSchema schema; + Instant timestamp; + + public BufferedCollectorInformation(TableSchema schema) { + this.schema = schema; + this.timestamp = BoundedWindow.TIMESTAMP_MAX_VALUE; + } + + void addCollector(TableRowToStorageApiProto.ErrorCollector collector, Instant ts) + throws TableRowToStorageApiProto.SchemaConversionException { + if (this.collector == null) { + this.collector = collector; + } else { + this.collector.mergeInto(collector); + } + if (ts.isBefore(this.timestamp)) { + this.timestamp = ts; + } + } + } + + ConvertMessagesDoFn( + StorageApiDynamicDestinations dynamicDestinations, + BigQueryServices bqServices, + String operationName, + TupleTag failedWritesTag, + TupleTag> successfulWritesTag, + TupleTag> patchTableSchemaTag, + TupleTag> retryElementsWaitingForSchemaTag, + @Nullable SerializableFunction rowMutationFn, + BadRecordRouter badRecordRouter, + Coder> elementCoder, + boolean hasSchemaUpdateOptions) { + this.dynamicDestinations = dynamicDestinations; + this.messageConverters = new TwoLevelMessageConverterCache<>(operationName); + this.bqServices = bqServices; + this.failedWritesTag = failedWritesTag; + this.successfulWritesTag = successfulWritesTag; + this.patchTableSchemaTag = patchTableSchemaTag; + this.retryElementsWaitingForSchemaTag = retryElementsWaitingForSchemaTag; + this.rowMutationFn = rowMutationFn; + this.badRecordRouter = badRecordRouter; + this.elementCoder = elementCoder; + this.hasSchemaUpdateOptions = hasSchemaUpdateOptions; + } + + BigQueryServices.DatasetService getDatasetService(PipelineOptions pipelineOptions) + throws IOException { + if (datasetServiceInternal == null) { + datasetServiceInternal = + bqServices.getDatasetService(pipelineOptions.as(BigQueryOptions.class)); + } + return datasetServiceInternal; + } + + BigQueryServices.WriteStreamService getWriteStreamService(PipelineOptions pipelineOptions) + throws IOException { + if (writeStreamServiceInternal == null) { + writeStreamServiceInternal = + bqServices.getWriteStreamService(pipelineOptions.as(BigQueryOptions.class)); + } + return writeStreamServiceInternal; + } + + StorageApiDynamicDestinations getDynamicDestinations() { + return dynamicDestinations; + } + + TwoLevelMessageConverterCache getMessageConverters() { + return messageConverters; + } + + @Teardown + public void onTeardown() { + try { + if (datasetServiceInternal != null) { + datasetServiceInternal.close(); + datasetServiceInternal = null; + } + if (writeStreamServiceInternal != null) { + writeStreamServiceInternal.close(); + writeStreamServiceInternal = null; + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @StartBundle + public void startBundle() { + this.errorCollectors.clear(); + } + + @ProcessElement + public void processElement( + ProcessContext c, + PipelineOptions pipelineOptions, + @Element KV element, + @Timestamp Instant timestamp, + MultiOutputReceiver o) + throws Exception { + DestinationT destination = element.getKey(); + + dynamicDestinations.setSideInputAccessorFromProcessContext(c); + // Should we do this across the entire bundle instead? Unfortunately that doesn't work because + // we can't access + // side inputs in finishBundle. + StorageApiDynamicDestinations.MessageConverter messageConverter = + messageConverters.get( + destination, + dynamicDestinations, + pipelineOptions, + getDatasetService(pipelineOptions), + getWriteStreamService(pipelineOptions)); + TableRowToStorageApiProto.ErrorCollector errorCollector = + hasSchemaUpdateOptions + ? UpgradeTableSchema.newErrorCollector() + : TableRowToStorageApiProto.ErrorCollector.DONT_COLLECT; + Iterable>> unProcessed = + handleProcessElements( + messageConverter, + ImmutableList.of(TimestampedValue.of(element, timestamp)), + o, + errorCollector); + if (!errorCollector.isEmpty()) { + org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState( + hasSchemaUpdateOptions); + + // Track all errors. Generate schema-update message in finishBundle. + BufferedCollectorInformation bufferedCollectorInformation = + errorCollectors.computeIfAbsent( + destination, + d -> new BufferedCollectorInformation(messageConverter.getTableSchema())); + bufferedCollectorInformation.addCollector(errorCollector, timestamp); + + // Forward the message to the buffering stage to wait for the schema to be updated. + unProcessed.forEach( + tv -> + o.get(retryElementsWaitingForSchemaTag) + .outputWithTimestamp(tv.getValue(), tv.getTimestamp())); + } + } + + @FinishBundle + public void finishBundle(FinishBundleContext c) throws Exception { + if (!errorCollectors.isEmpty()) { + for (Map.Entry e : errorCollectors.entrySet()) { + if (e.getValue().collector != null) { + c.output( + patchTableSchemaTag, + KV.of( + e.getKey(), + UpgradeTableSchema.getIncrementalSchema( + e.getValue().collector, e.getValue().schema)), + e.getValue().timestamp, + GlobalWindow.INSTANCE); + } + } + } + errorCollectors.clear(); + } + + public Iterable>> handleProcessElements( + StorageApiDynamicDestinations.MessageConverter messageConverter, + Iterable>> values, + MultiOutputReceiver o, + TableRowToStorageApiProto.ErrorCollector errorCollector) + throws Exception { + List>> unprocessedElements = Lists.newArrayList(); + + for (TimestampedValue> value : values) { + DestinationT destination = value.getValue().getKey(); + ElementT element = value.getValue().getValue(); + Instant timestamp = value.getTimestamp(); + + RowMutationInformation rowMutationInformation = null; + if (rowMutationFn != null) { + rowMutationInformation = Preconditions.checkStateNotNull(rowMutationFn).apply(element); + } + try { + StorageApiWritePayload payload = + messageConverter + .toMessage(element, rowMutationInformation, errorCollector) + .withTimestamp(timestamp); + if (errorCollector.isEmpty()) { + o.get(successfulWritesTag).outputWithTimestamp(KV.of(destination, payload), timestamp); + } else { + // This should only happen if the user sets schemaUpdateOptions. + unprocessedElements.add(value); + } + } catch (TableRowToStorageApiProto.SchemaConversionException conversionException) { + // If the user did not set schemaUpdateOptions or if we encounter an error we cannot address + // via schema update, + // the error is caught here. + TableRow failsafeTableRow; + try { + failsafeTableRow = messageConverter.toFailsafeTableRow(element); + } catch (Exception e) { + badRecordRouter.route( + o, + KV.of(destination, element), + elementCoder, + e, + "Unable to convert value to TableRow"); + continue; + } + TableReference tableReference = null; + TableDestination tableDestination = dynamicDestinations.getTable(destination); + if (tableDestination != null) { + tableReference = tableDestination.getTableReference(); + } + o.get(failedWritesTag) + .outputWithTimestamp( + new BigQueryStorageApiInsertError( + failsafeTableRow, conversionException.toString(), tableReference), + timestamp); + } catch (Exception e) { + badRecordRouter.route( + o, + KV.of(destination, element), + elementCoder, + e, + "Unable to convert value to StorageWriteApiPayload"); + } + } + return unprocessedElements; + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/MergeSchemaCombineFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/MergeSchemaCombineFn.java new file mode 100644 index 000000000000..ef5448c759c5 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/MergeSchemaCombineFn.java @@ -0,0 +1,76 @@ +/* + * 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.gcp.bigquery; + +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; + +import com.google.cloud.bigquery.storage.v1.TableSchema; +import java.util.Iterator; +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.transforms.Combine; + +/** A CombineFn to merge TableSchemas. */ +public class MergeSchemaCombineFn extends Combine.CombineFn { + @Override + public TableSchema createAccumulator() { + return TableSchema.newBuilder().build(); + } + + @Override + public TableSchema addInput(TableSchema accumulator, TableSchema input) { + if (input.equals(accumulator)) { + return accumulator; + } + return UpgradeTableSchema.mergeSchemas(accumulator, input); + } + + @Override + public Coder getAccumulatorCoder( + CoderRegistry registry, Coder inputCoder) { + return inputCoder; + } + + @Override + public Coder getDefaultOutputCoder( + CoderRegistry registry, Coder inputCoder) throws CannotProvideCoderException { + return inputCoder; + } + + @Override + public TableSchema mergeAccumulators(Iterable accumulators) { + checkNotNull(accumulators, "accumulators must be non-null"); + + Iterator iter = accumulators.iterator(); + if (!iter.hasNext()) { + return createAccumulator(); + } + + TableSchema merged = iter.next(); + while (iter.hasNext()) { + merged = addInput(merged, iter.next()); + } + return merged; + } + + @Override + public TableSchema extractOutput(TableSchema accumulator) { + return accumulator; + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PatchTableSchemaDoFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PatchTableSchemaDoFn.java new file mode 100644 index 000000000000..a4abb59d0969 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PatchTableSchemaDoFn.java @@ -0,0 +1,159 @@ +/* + * 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.gcp.bigquery; + +import com.google.cloud.bigquery.storage.v1.TableSchema; +import com.google.cloud.hadoop.util.ApiErrorExtractor; +import java.io.IOException; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.util.BackOff; +import org.apache.beam.sdk.util.BackOffUtils; +import org.apache.beam.sdk.util.FluentBackoff; +import org.apache.beam.sdk.util.Sleeper; +import org.apache.beam.sdk.values.KV; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Duration; + +/** + * This DoFn is responsible for updating a BigQuery's table schema. The input is a TableSchema + * containing only the schema delta (new fields, relaxed fields). It outputs elements for all + * updated tables, which act as notifcations to the buffering stage that the elements can be + * retried. + */ +public class PatchTableSchemaDoFn + extends DoFn, KV> { + private final BigQueryServices bqServices; + private final StorageApiDynamicDestinations dynamicDestinations; + private TwoLevelMessageConverterCache messageConverters; + private transient BigQueryServices.@Nullable DatasetService datasetServiceInternal = null; + private transient BigQueryServices.@Nullable WriteStreamService writeStreamServiceInternal = null; + + PatchTableSchemaDoFn( + String operationName, + BigQueryServices bqServices, + StorageApiDynamicDestinations dynamicDestinations) { + this.messageConverters = new TwoLevelMessageConverterCache<>(operationName); + this.bqServices = bqServices; + this.dynamicDestinations = dynamicDestinations; + } + + private BigQueryServices.DatasetService getDatasetService(PipelineOptions pipelineOptions) + throws IOException { + if (datasetServiceInternal == null) { + datasetServiceInternal = + bqServices.getDatasetService(pipelineOptions.as(BigQueryOptions.class)); + } + return datasetServiceInternal; + } + + private BigQueryServices.WriteStreamService getWriteStreamService(PipelineOptions pipelineOptions) + throws IOException { + if (writeStreamServiceInternal == null) { + writeStreamServiceInternal = + bqServices.getWriteStreamService(pipelineOptions.as(BigQueryOptions.class)); + } + return writeStreamServiceInternal; + } + + @Teardown + public void onTeardown() { + try { + if (datasetServiceInternal != null) { + datasetServiceInternal.close(); + datasetServiceInternal = null; + } + if (writeStreamServiceInternal != null) { + writeStreamServiceInternal.close(); + writeStreamServiceInternal = null; + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @ProcessElement + public void processElement( + @Element KV element, + OutputReceiver> o, + ProcessContext context, + PipelineOptions pipelineOptions) + throws Exception { + dynamicDestinations.setSideInputAccessorFromProcessContext(context); + DestinationT destination = element.getKey(); + TableSchema tableSchemaDiff = element.getValue(); + + StorageApiDynamicDestinations.MessageConverter messageConverter = + messageConverters.get( + destination, + dynamicDestinations, + pipelineOptions, + getDatasetService(pipelineOptions), + getWriteStreamService(pipelineOptions)); + messageConverter.updateSchemaFromTable(); + + while (true) { + TableSchema baseSchema = messageConverter.getTableSchema(); + TableSchema updatedSchema = UpgradeTableSchema.mergeSchemas(baseSchema, tableSchemaDiff); + // Check first to see if the schema still needs updating. + if (baseSchema.equals(updatedSchema)) { + return; + } + + BackOff backoff = + FluentBackoff.DEFAULT + .withInitialBackoff(Duration.standardSeconds(1)) + .withMaxBackoff(Duration.standardMinutes(1)) + .withMaxRetries(500) + .withThrottledTimeCounter( + BigQuerySinkMetrics.throttledTimeCounter( + BigQuerySinkMetrics.RpcMethod.PATCH_TABLE)) + .backoff(); + + boolean schemaOutOfDate = false; + Exception lastException = null; + do { + try { + getDatasetService(pipelineOptions) + .patchTableSchema( + dynamicDestinations.getTable(destination).getTableReference(), + TableRowToStorageApiProto.protoSchemaToTableSchema(updatedSchema)); + // Indicate that we've patched this schema. + o.output(KV.of(destination, null)); + return; + } catch (IOException e) { + ApiErrorExtractor errorExtractor = new ApiErrorExtractor(); + if (errorExtractor.preconditionNotMet(e) || errorExtractor.badRequest(e)) { + schemaOutOfDate = true; + break; + } else { + lastException = e; + } + } + } while (BackOffUtils.next(Sleeper.DEFAULT, backoff)); + if (schemaOutOfDate) { + // This could be due to an out-of-date schema. + messageConverter.updateSchemaFromTable(); + } else { + // We ran out of retries. + throw new RuntimeException("Failed to patch table schema.", lastException); + } + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/RetryManager.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/RetryManager.java index 1a7202de0a56..388f525872e4 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/RetryManager.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/RetryManager.java @@ -296,6 +296,14 @@ void run(boolean await) throws Exception { } } + int getRemainingOperationCount() { + return operations.size(); + } + + Iterable getRemainingContexts() { + return operations.stream().map(o -> o.context).collect(Collectors.toList()); + } + void await() throws Exception { while (!this.operations.isEmpty()) { Operation operation = this.operations.element(); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/SchemaUpdateHoldingFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/SchemaUpdateHoldingFn.java new file mode 100644 index 000000000000..5e5dfae0441d --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/SchemaUpdateHoldingFn.java @@ -0,0 +1,256 @@ +/* + * 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.gcp.bigquery; + +import com.google.api.client.util.BackOff; +import com.google.api.client.util.BackOffUtils; +import com.google.api.client.util.ExponentialBackOff; +import java.util.List; +import java.util.concurrent.TimeUnit; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.TimeDomain; +import org.apache.beam.sdk.state.Timer; +import org.apache.beam.sdk.state.TimerSpec; +import org.apache.beam.sdk.state.TimerSpecs; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.ShardedKey; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Duration; +import org.joda.time.Instant; + +/** + * This is a stateful DoFn that buffers elements that triggered table schema update. Once the table + * schema has been updated, this reprocesses the messages and allows them to continue on through the + * sink. + */ +public class SchemaUpdateHoldingFn + extends DoFn< + KV, @Nullable ElementT>, + KV> { + private static final Duration POLL_DURATION = Duration.standardSeconds(1); + + @StateId("bufferedElements") + private final StateSpec>> bufferedSpec; + + @StateId("minBufferedTimestamp") + private final StateSpec> minBufferedTsSpec; + + @StateId("timerTimestamp") + private final StateSpec> timerTsSpec; + + @TimerId("pollTimer") + private final TimerSpec pollTimerSpec = TimerSpecs.timer(TimeDomain.PROCESSING_TIME); + + private final ConvertMessagesDoFn convertMessagesDoFn; + + public SchemaUpdateHoldingFn( + Coder elementCoder, + ConvertMessagesDoFn convertMessagesDoFn) { + this.convertMessagesDoFn = convertMessagesDoFn; + this.bufferedSpec = StateSpecs.bag(TimestampedValue.TimestampedValueCoder.of(elementCoder)); + this.timerTsSpec = StateSpecs.value(); + + Combine.BinaryCombineLongFn minCombineFn = + new Combine.BinaryCombineLongFn() { + @Override + public long identity() { + return BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis(); + } + + @Override + public long apply(long left, long right) { + return Math.min(left, right); + } + }; + this.minBufferedTsSpec = StateSpecs.combining(minCombineFn); + } + + @StartBundle + public void startBundle() { + convertMessagesDoFn.startBundle(); + ; + } + + @Teardown + public void onTeardown() { + convertMessagesDoFn.onTeardown(); + } + + @ProcessElement + public void processElement( + @Element KV, @Nullable ElementT> element, + @Timestamp Instant timestamp, + @StateId("bufferedElements") BagState> bag, + @StateId("minBufferedTimestamp") CombiningState minBufferedTimestamp, + @StateId("timerTimestamp") ValueState timerTs, + @TimerId("pollTimer") Timer pollTimer, + ProcessContext context, + BoundedWindow window, + MultiOutputReceiver o) + throws Exception { + convertMessagesDoFn.getDynamicDestinations().setSideInputAccessorFromProcessContext(context); + + minBufferedTimestamp.readLater(); + timerTs.readLater(); + ElementT value = element.getValue(); + Instant newTimerTs = null; + if (value != null) { + // Buffer the element. + bag.add(TimestampedValue.of(value, timestamp)); + minBufferedTimestamp.add(timestamp.getMillis()); + Long currentTimerTs = timerTs.read(); + // We always have to reset the timer to update the output timestamp, however if there already + // is a timer then + // we keep the current expiration. + newTimerTs = + currentTimerTs == null + ? pollTimer.getCurrentRelativeTime().plus(POLL_DURATION) + : Instant.ofEpochMilli(currentTimerTs); + } else { + // This means that the table schema was recently updated. Try to flush the pending elements. + if (tryFlushBuffer( + element.getKey().getKey(), context.getPipelineOptions(), bag, minBufferedTimestamp, o)) { + // Nothing left in buffer. clear timer. + pollTimer.clear(); + timerTs.clear(); + } else { + // We just scanned the buffer, so bump the timer to the next poll duration. + newTimerTs = pollTimer.getCurrentRelativeTime().plus(POLL_DURATION); + } + } + if (newTimerTs != null) { + pollTimer + .withOutputTimestamp(Instant.ofEpochMilli(minBufferedTimestamp.read())) + .set(newTimerTs); + timerTs.write(newTimerTs.getMillis()); + } + } + + @Override + public Duration getAllowedTimestampSkew() { + // This is safe because a watermark hold will always be set using timer.withOutputTimestamp. + return Duration.millis(Long.MAX_VALUE); + } + + @OnTimer("pollTimer") + public void onPollTimer( + @Key ShardedKey key, + PipelineOptions pipelineOptions, + @StateId("bufferedElements") BagState> bag, + @StateId("minBufferedTimestamp") CombiningState minBufferedTimestamp, + @StateId("timerTimestamp") ValueState timerTs, + @TimerId("pollTimer") Timer pollTimer, + BoundedWindow window, + MultiOutputReceiver o) + throws Exception { + if (tryFlushBuffer(key.getKey(), pipelineOptions, bag, minBufferedTimestamp, o)) { + timerTs.clear(); + } else { + // We still have buffered elements. Make sure that the polling timer keeps looping. + Instant newTimerTs = pollTimer.getCurrentRelativeTime().plus(POLL_DURATION); + pollTimer + .withOutputTimestamp(Instant.ofEpochMilli(minBufferedTimestamp.read())) + .set(newTimerTs); + timerTs.write(newTimerTs.getMillis()); + } + } + + @OnWindowExpiration + public void onWindowExpiration( + @Key ShardedKey key, + PipelineOptions pipelineOptions, + @StateId("bufferedElements") BagState> bag, + @StateId("minBufferedTimestamp") CombiningState minBufferedTimestamp, + MultiOutputReceiver o) + throws Exception { + // This can happen on test completion or drain. We can't set any more timers in window + // expiration, so we just have to loop until the schema is updated. + BackOff backoff = + new ExponentialBackOff.Builder() + .setMaxElapsedTimeMillis((int) TimeUnit.SECONDS.toMillis(10)) + .build(); + do { + if (tryFlushBuffer(key.getKey(), pipelineOptions, bag, minBufferedTimestamp, o)) { + return; + } + } while (BackOffUtils.next(com.google.api.client.util.Sleeper.DEFAULT, backoff)); + throw new RuntimeException("Failed to flush elements on window expiration!"); + } + + // Returns true if the buffer is completely flushed. + public boolean tryFlushBuffer( + DestinationT destination, + PipelineOptions pipelineOptions, + @StateId("bufferedElements") BagState> bag, + @StateId("minBufferedTimestamp") CombiningState minBufferedTimestamp, + MultiOutputReceiver o) + throws Exception { + // Force an update of the MessageConverter schema. + StorageApiDynamicDestinations.MessageConverter messageConverter = + convertMessagesDoFn + .getMessageConverters() + .get( + destination, + convertMessagesDoFn.getDynamicDestinations(), + pipelineOptions, + convertMessagesDoFn.getDatasetService(pipelineOptions), + convertMessagesDoFn.getWriteStreamService(pipelineOptions)); + messageConverter.updateSchemaFromTable(); + + List> stillWaiting = Lists.newArrayList(); + minBufferedTimestamp.clear(); + + Iterable>> kvBagElements = + Iterables.transform( + bag.read(), + e -> TimestampedValue.of(KV.of(destination, e.getValue()), e.getTimestamp())); + + TableRowToStorageApiProto.ErrorCollector errorCollector = + UpgradeTableSchema.newErrorCollector(); + Iterable>> unProcessed = + convertMessagesDoFn.handleProcessElements( + messageConverter, kvBagElements, o, errorCollector); + if (!errorCollector.isEmpty()) { + // Collect all elements that still fail to convert. + unProcessed.forEach( + tv -> { + stillWaiting.add(TimestampedValue.of(tv.getValue().getValue(), tv.getTimestamp())); + minBufferedTimestamp.add(tv.getTimestamp().getMillis()); + }); + } + + // Add the remaining elements back into the bag. + bag.clear(); + stillWaiting.forEach(bag::add); + + return stillWaiting.isEmpty(); + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/SplittingIterable.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/SplittingIterable.java index 41cee0157706..26eb6d55d208 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/SplittingIterable.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/SplittingIterable.java @@ -17,16 +17,20 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; +import static org.apache.beam.sdk.io.gcp.bigquery.UpgradeTableSchema.isPayloadSchemaOutOfDate; + import com.google.api.services.bigquery.model.TableRow; import com.google.auto.value.AutoValue; import com.google.cloud.bigquery.storage.v1.ProtoRows; import com.google.protobuf.ByteString; +import com.google.protobuf.Descriptors; import java.io.IOException; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; import java.util.function.BiConsumer; import java.util.function.Function; +import org.apache.beam.sdk.util.ThrowingSupplier; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -47,6 +51,8 @@ abstract static class Value { abstract List getTimestamps(); abstract List<@Nullable TableRow> getFailsafeTableRows(); + + abstract boolean getSchemaMismatchSeen(); } interface ConcatFields { @@ -60,6 +66,8 @@ ByteString concat(ByteString bytes, TableRow tableRows) private final ConcatFields concatProtoAndTableRow; private final Function protoToTableRow; private final BiConsumer, String> failedRowsConsumer; + private final ThrowingSupplier getCurrentTableSchemaHash; + private final ThrowingSupplier getCurrentTableSchemaDescriptor; private final boolean autoUpdateSchema; private final Instant elementsTimestamp; @@ -69,6 +77,8 @@ public SplittingIterable( ConcatFields concatProtoAndTableRow, Function protoToTableRow, BiConsumer, String> failedRowsConsumer, + ThrowingSupplier getCurrentTableSchemaHash, + ThrowingSupplier getCurrentTableSchemaDescriptor, boolean autoUpdateSchema, Instant elementsTimestamp) { this.underlying = underlying; @@ -76,6 +86,8 @@ public SplittingIterable( this.concatProtoAndTableRow = concatProtoAndTableRow; this.protoToTableRow = protoToTableRow; this.failedRowsConsumer = failedRowsConsumer; + this.getCurrentTableSchemaHash = getCurrentTableSchemaHash; + this.getCurrentTableSchemaDescriptor = getCurrentTableSchemaDescriptor; this.autoUpdateSchema = autoUpdateSchema; this.elementsTimestamp = elementsTimestamp; } @@ -101,24 +113,31 @@ public Value next() { List<@Nullable TableRow> failsafeRows = Lists.newArrayList(); ProtoRows.Builder inserts = ProtoRows.newBuilder(); long bytesSize = 0; - while (underlyingIterator.hasNext()) { - // Make sure that we don't exceed the split-size length over multiple elements. A single - // element can exceed - // the split threshold, but in that case it should be the only element returned. - if ((bytesSize + underlyingIterator.peek().getPayload().length > splitSize) - && inserts.getSerializedRowsCount() > 0) { - break; - } - StorageApiWritePayload payload = underlyingIterator.next(); - ByteString byteString = ByteString.copyFrom(payload.getPayload()); - @Nullable TableRow failsafeTableRow = null; - try { - failsafeTableRow = payload.getFailsafeTableRow(); - } catch (IOException e) { - // Do nothing, table row will be generated later from row bytes - } - if (autoUpdateSchema) { + + boolean schemaMismatchSeen = false; + try { + while (underlyingIterator.hasNext()) { + // Make sure that we don't exceed the split-size length over multiple elements. A single + // element can exceed + // the split threshold, but in that case it should be the only element returned. + if ((bytesSize + underlyingIterator.peek().getPayload().length > splitSize) + && inserts.getSerializedRowsCount() > 0) { + break; + } + StorageApiWritePayload payload = underlyingIterator.next(); + schemaMismatchSeen = + schemaMismatchSeen + || isPayloadSchemaOutOfDate( + payload, getCurrentTableSchemaHash, getCurrentTableSchemaDescriptor); + + ByteString byteString = ByteString.copyFrom(payload.getPayload()); + @Nullable TableRow failsafeTableRow = null; try { + failsafeTableRow = payload.getFailsafeTableRow(); + } catch (IOException e) { + // Do nothing, table row will be generated later from row bytes + } + if (autoUpdateSchema) { @Nullable TableRow unknownFields = payload.getUnknownFields(); if (unknownFields != null && !unknownFields.isEmpty()) { // Protocol buffer serialization format supports concatenation. We serialize any new @@ -150,20 +169,21 @@ public Value next() { continue; } } - } catch (Exception e) { - throw new RuntimeException(e); } + inserts.addSerializedRows(byteString); + Instant timestamp = payload.getTimestamp(); + if (timestamp == null) { + timestamp = elementsTimestamp; + } + timestamps.add(timestamp); + failsafeRows.add(failsafeTableRow); + bytesSize += byteString.size(); } - inserts.addSerializedRows(byteString); - Instant timestamp = payload.getTimestamp(); - if (timestamp == null) { - timestamp = elementsTimestamp; - } - timestamps.add(timestamp); - failsafeRows.add(failsafeTableRow); - bytesSize += byteString.size(); + } catch (Exception e) { + throw new RuntimeException(e); } - return new AutoValue_SplittingIterable_Value(inserts.build(), timestamps, failsafeRows); + return new AutoValue_SplittingIterable_Value( + inserts.build(), timestamps, failsafeRows, schemaMismatchSeen); } }; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiConvertMessages.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiConvertMessages.java index e62429cf0f30..58d981db94ec 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiConvertMessages.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiConvertMessages.java @@ -19,29 +19,43 @@ import static org.apache.beam.sdk.transforms.errorhandling.BadRecordRouter.BAD_RECORD_TAG; -import com.google.api.services.bigquery.model.TableReference; -import com.google.api.services.bigquery.model.TableRow; -import java.io.IOException; +import com.google.cloud.bigquery.storage.v1.TableSchema; +import java.nio.ByteBuffer; +import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; -import org.apache.beam.sdk.io.gcp.bigquery.StorageApiDynamicDestinations.MessageConverter; -import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; +import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.FlatMapElements; +import org.apache.beam.sdk.transforms.Flatten; 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.SimpleFunction; import org.apache.beam.sdk.transforms.errorhandling.BadRecord; import org.apache.beam.sdk.transforms.errorhandling.BadRecordRouter; -import org.apache.beam.sdk.util.Preconditions; +import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; +import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.Repeatedly; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.ShardedKey; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionList; 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.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.primitives.UnsignedInteger; import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; -import org.joda.time.Instant; +import org.joda.time.Duration; /** * A transform that converts messages to protocol buffers in preparation for writing to BigQuery. @@ -52,11 +66,16 @@ public class StorageApiConvertMessages private final BigQueryServices bqServices; private final TupleTag failedWritesTag; private final TupleTag> successfulWritesTag; + private final TupleTag> patchTableSchemaTag; + private final TupleTag> elementsWaitingForSchemaTag; private final Coder errorCoder; private final Coder> successCoder; + private final Coder elementCoder; + private final Coder destinationCoder; private final @Nullable SerializableFunction rowMutationFn; private final BadRecordRouter badRecordRouter; + private final boolean hasSchemaUpdateOptions; public StorageApiConvertMessages( StorageApiDynamicDestinations dynamicDestinations, @@ -65,141 +84,199 @@ public StorageApiConvertMessages( TupleTag> successfulWritesTag, Coder errorCoder, Coder> successCoder, + Coder elementCoder, + Coder destinationCoder, @Nullable SerializableFunction rowMutationFn, - BadRecordRouter badRecordRouter) { + BadRecordRouter badRecordRouter, + boolean hasSchemaUpdateOptions) { this.dynamicDestinations = dynamicDestinations; this.bqServices = bqServices; this.failedWritesTag = failedWritesTag; this.successfulWritesTag = successfulWritesTag; + this.patchTableSchemaTag = new TupleTag<>("PatchTableSchema"); + this.elementsWaitingForSchemaTag = new TupleTag<>("elementHolder"); this.errorCoder = errorCoder; this.successCoder = successCoder; + this.elementCoder = elementCoder; + this.destinationCoder = destinationCoder; this.rowMutationFn = rowMutationFn; this.badRecordRouter = badRecordRouter; + this.hasSchemaUpdateOptions = hasSchemaUpdateOptions; } @Override public PCollectionTuple expand(PCollection> input) { String operationName = input.getName() + "/" + getName(); + // This code currently assumes that the input is in the global window. + Preconditions.checkState(input.getWindowingStrategy().getWindowFn() instanceof GlobalWindows); + + @SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) + }) + ConvertMessagesDoFn convertMessagesDoFn = + new ConvertMessagesDoFn<>( + dynamicDestinations, + bqServices, + operationName, + failedWritesTag, + successfulWritesTag, + patchTableSchemaTag, + elementsWaitingForSchemaTag, + rowMutationFn, + badRecordRouter, + input.getCoder(), + hasSchemaUpdateOptions); PCollectionTuple result = input.apply( "Convert to message", - ParDo.of( - new ConvertMessagesDoFn<>( - dynamicDestinations, - bqServices, - operationName, - failedWritesTag, - successfulWritesTag, - rowMutationFn, - badRecordRouter, - input.getCoder())) + ParDo.of(convertMessagesDoFn) .withOutputTags( successfulWritesTag, - TupleTagList.of(ImmutableList.of(failedWritesTag, BAD_RECORD_TAG))) + TupleTagList.of( + ImmutableList.of( + failedWritesTag, + BAD_RECORD_TAG, + patchTableSchemaTag, + elementsWaitingForSchemaTag))) .withSideInputs(dynamicDestinations.getSideInputs())); result.get(successfulWritesTag).setCoder(successCoder); result.get(failedWritesTag).setCoder(errorCoder); result.get(BAD_RECORD_TAG).setCoder(BadRecord.getCoder(input.getPipeline())); - return result; + result + .get(patchTableSchemaTag) + .setCoder(KvCoder.of(destinationCoder, ProtoCoder.of(TableSchema.class))); + result.get(elementsWaitingForSchemaTag).setCoder(KvCoder.of(destinationCoder, elementCoder)); + + final int numShards = + input + .getPipeline() + .getOptions() + .as(BigQueryOptions.class) + .getSchemaUpgradeBufferingShards(); + + // Throttle the stream to the patch-table function so that only a single update per table per + // two seconds gets processed (to match quotas). The combiner merges incremental schemas, so we + // won't miss any updates. + PCollection, ElementT>> tablesPatched = + result + .get(patchTableSchemaTag) + .apply( + "rewindow", + Window.>configure() + .triggering( + Repeatedly.forever( + AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(Duration.standardSeconds(2)))) + .discardingFiredPanes()) + .apply("merge schemas", Combine.perKey(new MergeSchemaCombineFn())) + .setCoder(KvCoder.of(destinationCoder, ProtoCoder.of(TableSchema.class))) + .apply( + "Patch table schema", + ParDo.of( + new PatchTableSchemaDoFn<>(operationName, bqServices, dynamicDestinations))) + .setCoder(KvCoder.of(destinationCoder, NullableCoder.of(elementCoder))) + // We need to make sure that all shards of the buffering transform are notified. + .apply( + "fanout to all shards", + FlatMapElements.via( + new SimpleFunction< + KV, + Iterable, ElementT>>>() { + @Override + public Iterable, ElementT>> apply( + KV elem) { + return IntStream.range(0, numShards) + .mapToObj( + i -> + KV.of( + StorageApiConvertMessages.AssignShardFn.getShardedKey( + elem.getKey(), i, numShards), + elem.getValue())) + .collect(Collectors.toList()); + } + })) + .setCoder( + KvCoder.of(ShardedKey.Coder.of(destinationCoder), NullableCoder.of(elementCoder))) + .apply( + Window., ElementT>>configure() + .triggering(DefaultTrigger.of())); + + // Any elements that are waiting for a schema update are sent to this stateful DoFn to be + // buffered. + // Note: we currently do not provide the DynamicDestinations object access to the side input in + // this path. + // This is because side inputs are not currently available from timer callbacks. Since side + // inputs are generally + // used for getSchema and in this case we read the schema from the table, this is unlikely to be + // a problem. + PCollection, ElementT>> shardedWaitingElements = + result + .get(elementsWaitingForSchemaTag) + // TODO: Consider using GroupIntoBatchs.withShardingKey to get auto sharding here + // instead of fixed sharding. + .apply("assignShard", ParDo.of(new AssignShardFn<>(numShards))) + .setCoder( + KvCoder.of(ShardedKey.Coder.of(destinationCoder), NullableCoder.of(elementCoder))); + + PCollectionList, ElementT>> waitingElementsList = + PCollectionList.of(shardedWaitingElements).and(tablesPatched); + PCollectionTuple retryResult = + waitingElementsList + .apply("Buffered flatten", Flatten.pCollections()) + .apply( + "bufferElements", + ParDo.of(new SchemaUpdateHoldingFn<>(elementCoder, convertMessagesDoFn)) + .withOutputTags( + successfulWritesTag, + TupleTagList.of(ImmutableList.of(failedWritesTag, BAD_RECORD_TAG)))); + retryResult.get(successfulWritesTag).setCoder(successCoder); + retryResult.get(failedWritesTag).setCoder(errorCoder); + retryResult.get(BAD_RECORD_TAG).setCoder(BadRecord.getCoder(input.getPipeline())); + + // Flatten successes and failures from both the regular transform and the retry transform. + PCollection> allSuccesses = + PCollectionList.of(result.get(successfulWritesTag)) + .and(retryResult.get(successfulWritesTag)) + .apply("flattenSuccesses", Flatten.pCollections()); + PCollection allFailures = + PCollectionList.of(result.get(failedWritesTag)) + .and(retryResult.get(failedWritesTag)) + .apply("flattenFailures", Flatten.pCollections()); + PCollection allBadRecords = + PCollectionList.of(result.get(BAD_RECORD_TAG)) + .and(retryResult.get(BAD_RECORD_TAG)) + .apply("flattenBadRecords", Flatten.pCollections()); + return PCollectionTuple.of(successfulWritesTag, allSuccesses) + .and(failedWritesTag, allFailures) + .and(BAD_RECORD_TAG, allBadRecords); } - public static class ConvertMessagesDoFn - extends DoFn, KV> { - private final StorageApiDynamicDestinations dynamicDestinations; - private TwoLevelMessageConverterCache messageConverters; - private final BigQueryServices bqServices; - private final TupleTag failedWritesTag; - private final TupleTag> successfulWritesTag; - private final @Nullable SerializableFunction rowMutationFn; - private final BadRecordRouter badRecordRouter; - Coder> elementCoder; - private transient @Nullable DatasetService datasetServiceInternal = null; - - ConvertMessagesDoFn( - StorageApiDynamicDestinations dynamicDestinations, - BigQueryServices bqServices, - String operationName, - TupleTag failedWritesTag, - TupleTag> successfulWritesTag, - @Nullable SerializableFunction rowMutationFn, - BadRecordRouter badRecordRouter, - Coder> elementCoder) { - this.dynamicDestinations = dynamicDestinations; - this.messageConverters = new TwoLevelMessageConverterCache<>(operationName); - this.bqServices = bqServices; - this.failedWritesTag = failedWritesTag; - this.successfulWritesTag = successfulWritesTag; - this.rowMutationFn = rowMutationFn; - this.badRecordRouter = badRecordRouter; - this.elementCoder = elementCoder; - } + static class AssignShardFn extends DoFn, KV, V>> { + private int shard; + private final int numBuckets; - private DatasetService getDatasetService(PipelineOptions pipelineOptions) throws IOException { - if (datasetServiceInternal == null) { - datasetServiceInternal = - bqServices.getDatasetService(pipelineOptions.as(BigQueryOptions.class)); - } - return datasetServiceInternal; + public AssignShardFn(int numBuckets) { + this.numBuckets = numBuckets; } - @Teardown - public void onTeardown() { - try { - if (datasetServiceInternal != null) { - datasetServiceInternal.close(); - datasetServiceInternal = null; - } - } catch (Exception e) { - throw new RuntimeException(e); - } + @Setup + public void setup() { + shard = ThreadLocalRandom.current().nextInt(); } @ProcessElement - public void processElement( - ProcessContext c, - PipelineOptions pipelineOptions, - @Element KV element, - @Timestamp Instant timestamp, - MultiOutputReceiver o) - throws Exception { - dynamicDestinations.setSideInputAccessorFromProcessContext(c); - MessageConverter messageConverter = - messageConverters.get( - element.getKey(), dynamicDestinations, getDatasetService(pipelineOptions)); - - RowMutationInformation rowMutationInformation = null; - if (rowMutationFn != null) { - rowMutationInformation = - Preconditions.checkStateNotNull(rowMutationFn).apply(element.getValue()); - } - try { - StorageApiWritePayload payload = - messageConverter - .toMessage(element.getValue(), rowMutationInformation) - .withTimestamp(timestamp); - o.get(successfulWritesTag).output(KV.of(element.getKey(), payload)); - } catch (TableRowToStorageApiProto.SchemaConversionException conversionException) { - TableRow failsafeTableRow; - try { - failsafeTableRow = messageConverter.toFailsafeTableRow(element.getValue()); - } catch (Exception e) { - badRecordRouter.route(o, element, elementCoder, e, "Unable to convert value to TableRow"); - return; - } - TableReference tableReference = null; - TableDestination tableDestination = dynamicDestinations.getTable(element.getKey()); - if (tableDestination != null) { - tableReference = tableDestination.getTableReference(); - } - o.get(failedWritesTag) - .output( - new BigQueryStorageApiInsertError( - failsafeTableRow, conversionException.toString(), tableReference)); - } catch (Exception e) { - badRecordRouter.route( - o, element, elementCoder, e, "Unable to convert value to StorageWriteApiPayload"); - } + public void processElement(@Element KV element, OutputReceiver, V>> r) { + ++shard; + r.output(KV.of(getShardedKey(element.getKey(), shard, numBuckets), element.getValue())); + } + + static ShardedKey getShardedKey( + K key, int shard, int numBuckets) { + UnsignedInteger unsignedNumBuckets = UnsignedInteger.fromIntBits(numBuckets); + ByteBuffer buffer = ByteBuffer.allocate(Integer.BYTES); + buffer.putInt(UnsignedInteger.fromIntBits(shard).mod(unsignedNumBuckets).intValue()); + return ShardedKey.of(key, buffer.array()); } } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinations.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinations.java index 87667ef2cb17..d6981858a4b6 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinations.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinations.java @@ -19,8 +19,10 @@ import com.google.api.services.bigquery.model.TableRow; import com.google.protobuf.DescriptorProtos; +import java.io.IOException; import javax.annotation.Nullable; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; /** Base dynamicDestinations class used by the Storage API sink. */ @@ -32,9 +34,14 @@ public interface MessageConverter { DescriptorProtos.DescriptorProto getDescriptor(boolean includeCdcColumns) throws Exception; StorageApiWritePayload toMessage( - T element, @Nullable RowMutationInformation rowMutationInformation) throws Exception; + T element, + @Nullable RowMutationInformation rowMutationInformation, + TableRowToStorageApiProto.ErrorCollector collectedExceptions) + throws Exception; TableRow toFailsafeTableRow(T element); + + void updateSchemaFromTable() throws IOException, InterruptedException; } StorageApiDynamicDestinations(DynamicDestinations inner) { @@ -42,7 +49,11 @@ StorageApiWritePayload toMessage( } public abstract MessageConverter getMessageConverter( - DestinationT destination, DatasetService datasetService) throws Exception; + DestinationT destination, + PipelineOptions options, + DatasetService datasetService, + BigQueryServices.WriteStreamService writeStreamService) + throws Exception; @Override void setSideInputAccessorFromProcessContext(DoFn.ProcessContext context) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsBeamRow.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsBeamRow.java index 21abde7d256c..401395030542 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsBeamRow.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsBeamRow.java @@ -22,7 +22,9 @@ import com.google.protobuf.DescriptorProtos; import com.google.protobuf.Descriptors.Descriptor; import com.google.protobuf.Message; +import java.io.IOException; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.SerializableBiFunction; import org.apache.beam.sdk.transforms.SerializableFunction; @@ -59,7 +61,11 @@ class StorageApiDynamicDestinationsBeamRow getMessageConverter( - DestinationT destination, DatasetService datasetService) throws Exception { + DestinationT destination, + PipelineOptions pipelineOptions, + DatasetService datasetService, + BigQueryServices.WriteStreamService writeStreamService) + throws Exception { return new BeamRowConverter(); } @@ -79,6 +85,9 @@ class BeamRowConverter implements MessageConverter { } } + @Override + public void updateSchemaFromTable() throws IOException, InterruptedException {} + @Override public TableSchema getTableSchema() { return tableSchema; @@ -92,7 +101,10 @@ public DescriptorProtos.DescriptorProto getDescriptor(boolean includeCdcColumns) @Override @SuppressWarnings("nullness") public StorageApiWritePayload toMessage( - T element, @Nullable RowMutationInformation rowMutationInformation) throws Exception { + T element, + @Nullable RowMutationInformation rowMutationInformation, + TableRowToStorageApiProto.ErrorCollector collectedExceptions) + throws Exception { String changeType = null; String changeSequenceNum = null; Descriptor descriptorToUse = descriptor; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsGenericRecord.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsGenericRecord.java index 56b4be4a1a1f..6d7b679fc9b6 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsGenericRecord.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsGenericRecord.java @@ -22,9 +22,11 @@ import com.google.protobuf.DescriptorProtos; import com.google.protobuf.Descriptors.Descriptor; import com.google.protobuf.Message; +import java.io.IOException; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.SerializableFunction; import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; @@ -54,7 +56,11 @@ class StorageApiDynamicDestinationsGenericRecord getMessageConverter( - DestinationT destination, DatasetService datasetService) throws Exception { + DestinationT destination, + PipelineOptions pipelineOptions, + DatasetService datasetService, + BigQueryServices.WriteStreamService writeStreamService) + throws Exception { return new GenericRecordConverter(destination); } @@ -69,6 +75,7 @@ class GenericRecordConverter implements MessageConverter { avroSchema = schemaFactory.apply(getSchema(destination)); protoTableSchema = AvroGenericRecordToStorageApiProto.protoTableSchemaFromAvroSchema(avroSchema); + descriptor = TableRowToStorageApiProto.getDescriptorFromTableSchema(protoTableSchema, true, false); if (usesCdc) { @@ -79,10 +86,16 @@ class GenericRecordConverter implements MessageConverter { } } + @Override + public void updateSchemaFromTable() throws IOException, InterruptedException {} + @Override @SuppressWarnings("nullness") public StorageApiWritePayload toMessage( - T element, @Nullable RowMutationInformation rowMutationInformation) throws Exception { + T element, + @Nullable RowMutationInformation rowMutationInformation, + TableRowToStorageApiProto.ErrorCollector collectedExceptions) + throws Exception { String changeType = null; String changeSequenceNum = null; Descriptor descriptorToUse = descriptor; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsProto.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsProto.java index 544c1dc28e53..2a1d8f3be6f2 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsProto.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsProto.java @@ -24,9 +24,11 @@ import com.google.protobuf.Descriptors; import com.google.protobuf.DynamicMessage; import com.google.protobuf.Message; +import java.io.IOException; import java.lang.reflect.InvocationTargetException; import javax.annotation.Nullable; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Predicates; import org.checkerframework.checker.nullness.qual.NonNull; @@ -57,7 +59,11 @@ class StorageApiDynamicDestinationsProto getMessageConverter( - DestinationT destination, DatasetService datasetService) throws Exception { + DestinationT destination, + PipelineOptions pipelineOptions, + DatasetService datasetService, + BigQueryServices.WriteStreamService writeStreamService) + throws Exception { return new Converter( TableRowToStorageApiProto.schemaToProtoTableSchema( Preconditions.checkStateNotNull(getSchema(destination)))); @@ -77,6 +83,9 @@ public TableSchema getTableSchema() { return tableSchema; } + @Override + public void updateSchemaFromTable() throws IOException, InterruptedException {} + public TableRowToStorageApiProto.SchemaInformation getSchemaInformation() { if (this.schemaInformation == null) { this.schemaInformation = @@ -95,7 +104,10 @@ public DescriptorProtos.DescriptorProto getDescriptor(boolean includeCdcColumns) @Override public StorageApiWritePayload toMessage( - T element, @Nullable RowMutationInformation rowMutationInformation) throws Exception { + T element, + @Nullable RowMutationInformation rowMutationInformation, + TableRowToStorageApiProto.ErrorCollector collectedExceptions) + throws Exception { // NB: What makes this path efficient is that the storage API directly understands protos, so // we can forward // the through directly. This means that we don't currently support ignoreUnknownValues or diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsTableRow.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsTableRow.java index 2438515b8770..1710d32689c9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsTableRow.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsTableRow.java @@ -23,24 +23,32 @@ import com.google.protobuf.DescriptorProtos; import com.google.protobuf.Descriptors.Descriptor; import com.google.protobuf.Message; +import java.io.IOException; +import java.util.Set; import java.util.concurrent.ExecutionException; -import javax.annotation.Nullable; +import java.util.concurrent.atomic.AtomicReference; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Suppliers; import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; public class StorageApiDynamicDestinationsTableRow extends StorageApiDynamicDestinations { private final BigQueryIO.TableRowFormatFunction formatFunction; - private final @Nullable BigQueryIO.TableRowFormatFunction formatRecordOnFailureFunction; + private final BigQueryIO.@Nullable TableRowFormatFunction formatRecordOnFailureFunction; private final boolean usesCdc; private final CreateDisposition createDisposition; private final boolean ignoreUnknownValues; private final boolean autoSchemaUpdates; + private final Set schemaUpdateOptions; private static final TableSchemaCache SCHEMA_CACHE = new TableSchemaCache(Duration.standardSeconds(1)); @@ -51,11 +59,12 @@ public class StorageApiDynamicDestinationsTableRow inner, BigQueryIO.TableRowFormatFunction formatFunction, - @Nullable BigQueryIO.TableRowFormatFunction formatRecordOnFailureFunction, + BigQueryIO.@Nullable TableRowFormatFunction formatRecordOnFailureFunction, boolean usesCdc, CreateDisposition createDisposition, boolean ignoreUnknownValues, - boolean autoSchemaUpdates) { + boolean autoSchemaUpdates, + Set schemaUpdateOptions) { super(inner); this.formatFunction = formatFunction; this.formatRecordOnFailureFunction = formatRecordOnFailureFunction; @@ -63,6 +72,7 @@ public class StorageApiDynamicDestinationsTableRow getMessageConverter( - DestinationT destination, DatasetService datasetService) throws Exception { - return new TableRowConverter(destination, datasetService); + DestinationT destination, + PipelineOptions options, + DatasetService datasetService, + BigQueryServices.WriteStreamService writeStreamService) { + SerializableFunction<@Nullable TableSchema, TableRowConverter> getConverter = + tableSchema -> { + try { + return new TableRowConverter(destination, datasetService, tableSchema); + } catch (Exception e) { + throw new RuntimeException(e); + } + }; + + return schemaUpdateOptions.isEmpty() + ? getConverter.apply(getSchema(destination)) + : new SchemaUpgradingTableRowConverter( + getConverter, options, datasetService, writeStreamService); + } + + // This is a wrapper class used when schemaUpdateOptions are set. + class SchemaUpgradingTableRowConverter implements MessageConverter { + private final SerializableFunction<@Nullable TableSchema, TableRowConverter> getConverter; + private final DatasetService datasetService; + private final BigQueryServices.WriteStreamService writeStreamService; + private final BigQueryOptions bigQueryOptions; + private AtomicReference delegate = new AtomicReference<>(); + + SchemaUpgradingTableRowConverter( + SerializableFunction<@Nullable TableSchema, TableRowConverter> getConverter, + PipelineOptions options, + DatasetService datasetService, + BigQueryServices.WriteStreamService writeStreamService) { + this.getConverter = getConverter; + this.datasetService = datasetService; + this.writeStreamService = writeStreamService; + this.bigQueryOptions = options.as(BigQueryOptions.class); + // Pass in null - force us to look up the actual table schema. + this.delegate.set(getConverter.apply(null)); + } + + @Override + public com.google.cloud.bigquery.storage.v1.TableSchema getTableSchema() { + return delegate.get().getTableSchema(); + } + + @Override + public DescriptorProtos.DescriptorProto getDescriptor(boolean includeCdcColumns) + throws Exception { + return delegate.get().getDescriptor(includeCdcColumns); + } + + @Override + public StorageApiWritePayload toMessage( + T element, + @Nullable RowMutationInformation rowMutationInformation, + TableRowToStorageApiProto.ErrorCollector collectedExceptions) + throws Exception { + TableRowConverter converter = delegate.get(); + StorageApiWritePayload payload = + converter.toMessage(element, rowMutationInformation, collectedExceptions); + // Set the schema hash on the payload so the next transform knows whether it has an + // out-of-date schema. + payload = payload.toBuilder().setSchemaHash(converter.getSchemaHash()).build(); + + return payload; + } + + @Override + public void updateSchemaFromTable() throws IOException, InterruptedException { + SCHEMA_CACHE.refreshSchema( + delegate.get().tableReference, datasetService, writeStreamService, bigQueryOptions); + // Recycle the internal MessageConverter so that we pick up the new schema from the cache. + this.delegate.set(getConverter.apply(null)); + } + + @Override + public TableRow toFailsafeTableRow(T element) { + return delegate.get().toFailsafeTableRow(element); + } } class TableRowConverter implements MessageConverter { + final TableReference tableReference; final @Nullable TableSchema tableSchema; final com.google.cloud.bigquery.storage.v1.TableSchema protoTableSchema; + final Supplier getSchemaHash; final TableRowToStorageApiProto.SchemaInformation schemaInformation; final Descriptor descriptor; final @Nullable Descriptor cdcDescriptor; TableRowConverter( - TableSchema tableSchema, - TableRowToStorageApiProto.SchemaInformation schemaInformation, - Descriptor descriptor) { - this.tableSchema = tableSchema; - this.protoTableSchema = TableRowToStorageApiProto.schemaToProtoTableSchema(tableSchema); - this.schemaInformation = schemaInformation; - this.descriptor = descriptor; - this.cdcDescriptor = null; - } - - TableRowConverter(DestinationT destination, DatasetService datasetService) throws Exception { - TableSchema localTableSchema = getSchema(destination); - TableReference tableReference = getTable(destination).getTableReference(); + DestinationT destination, + DatasetService datasetService, + @Nullable TableSchema localTableSchema) + throws Exception { + this.tableReference = getTable(destination).getTableReference(); if (localTableSchema == null) { // If the table already exists, then try and fetch the schema from the existing // table. @@ -125,6 +206,8 @@ class TableRowConverter implements MessageConverter { } this.tableSchema = localTableSchema; this.protoTableSchema = TableRowToStorageApiProto.schemaToProtoTableSchema(tableSchema); + this.getSchemaHash = + Suppliers.memoize(() -> TableRowToStorageApiProto.tableSchemaHash(this.protoTableSchema)); schemaInformation = TableRowToStorageApiProto.SchemaInformation.fromTableSchema(protoTableSchema); // If autoSchemaUpdates == true, then generate a descriptor where all the fields are optional. @@ -141,11 +224,18 @@ class TableRowConverter implements MessageConverter { } } + @Override + public void updateSchemaFromTable() throws IOException, InterruptedException {} + @Override public com.google.cloud.bigquery.storage.v1.TableSchema getTableSchema() { return protoTableSchema; } + byte[] getSchemaHash() { + return getSchemaHash.get(); + } + @Override public DescriptorProtos.DescriptorProto getDescriptor(boolean includeCdcColumns) throws Exception { @@ -163,7 +253,10 @@ public TableRow toFailsafeTableRow(T element) { @Override public StorageApiWritePayload toMessage( - T element, @Nullable RowMutationInformation rowMutationInformation) throws Exception { + T element, + @Nullable RowMutationInformation rowMutationInformation, + TableRowToStorageApiProto.ErrorCollector collectedExceptions) + throws Exception { TableRow tableRow = formatFunction.apply(schemaInformation, element); String changeType = null; @@ -179,6 +272,7 @@ public StorageApiWritePayload toMessage( boolean ignoreUnknown = ignoreUnknownValues || autoSchemaUpdates; @Nullable TableRow unknownFields = autoSchemaUpdates ? new TableRow() : null; boolean allowMissingFields = autoSchemaUpdates; + @Nullable Message msg = TableRowToStorageApiProto.messageFromTableRow( schemaInformation, @@ -188,9 +282,10 @@ public StorageApiWritePayload toMessage( allowMissingFields, unknownFields, changeType, - changeSequenceNum); + changeSequenceNum, + collectedExceptions); return StorageApiWritePayload.of( - msg.toByteArray(), + msg == null ? new byte[0] : msg.toByteArray(), unknownFields, formatRecordOnFailureFunction != null ? toFailsafeTableRow(element) : null); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiLoads.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiLoads.java index fcf67a8062ac..007bba5c6cdf 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiLoads.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiLoads.java @@ -63,6 +63,7 @@ public class StorageApiLoads @Nullable TupleTag successfulWrittenRowsTag; Predicate successfulRowsPredicate; private final Coder destinationCoder; + private final Coder elementCoder; private final StorageApiDynamicDestinations dynamicDestinations; private final @Nullable SerializableFunction rowUpdateFn; @@ -83,9 +84,11 @@ public class StorageApiLoads private final BadRecordRouter badRecordRouter; private final ErrorHandler badRecordErrorHandler; + private final boolean hasSchemaUpdateOptions; public StorageApiLoads( Coder destinationCoder, + Coder elementCoder, StorageApiDynamicDestinations dynamicDestinations, @Nullable SerializableFunction rowUpdateFn, CreateDisposition createDisposition, @@ -103,8 +106,10 @@ public StorageApiLoads( AppendRowsRequest.MissingValueInterpretation defaultMissingValueInterpretation, Map bigLakeConfiguration, BadRecordRouter badRecordRouter, - ErrorHandler badRecordErrorHandler) { + ErrorHandler badRecordErrorHandler, + boolean hasSchemaUpdateOptions) { this.destinationCoder = destinationCoder; + this.elementCoder = elementCoder; this.dynamicDestinations = dynamicDestinations; this.rowUpdateFn = rowUpdateFn; this.createDisposition = createDisposition; @@ -125,6 +130,7 @@ public StorageApiLoads( this.bigLakeConfiguration = bigLakeConfiguration; this.badRecordRouter = badRecordRouter; this.badRecordErrorHandler = badRecordErrorHandler; + this.hasSchemaUpdateOptions = hasSchemaUpdateOptions; } public TupleTag getFailedRowsTag() { @@ -171,8 +177,11 @@ public WriteResult expandInconsistent( successfulConvertedRowsTag, BigQueryStorageApiInsertErrorCoder.of(), successCoder, + elementCoder, + destinationCoder, rowUpdateFn, - badRecordRouter)); + badRecordRouter, + hasSchemaUpdateOptions)); PCollectionTuple writeRecordsResult = convertMessagesResult .get(successfulConvertedRowsTag) @@ -235,8 +244,11 @@ public WriteResult expandTriggered( successfulConvertedRowsTag, BigQueryStorageApiInsertErrorCoder.of(), successCoder, + elementCoder, + destinationCoder, rowUpdateFn, - badRecordRouter)); + badRecordRouter, + hasSchemaUpdateOptions)); PCollection, Iterable>> groupedRecords; @@ -358,8 +370,11 @@ public WriteResult expandUntriggered( successfulConvertedRowsTag, BigQueryStorageApiInsertErrorCoder.of(), successCoder, + elementCoder, + destinationCoder, rowUpdateFn, - badRecordRouter)); + badRecordRouter, + hasSchemaUpdateOptions)); PCollection> successfulConvertedRows = convertMessagesResult.get(successfulConvertedRowsTag); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritePayload.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritePayload.java index f0fce11b2d32..3f00ed67a8a5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritePayload.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritePayload.java @@ -42,6 +42,9 @@ public abstract class StorageApiWritePayload { @SuppressWarnings("mutable") public abstract @Nullable byte[] getFailsafeTableRowPayload(); + @SuppressWarnings("mutable") + public abstract @Nullable byte[] getSchemaHash(); + @AutoValue.Builder public abstract static class Builder { public abstract Builder setPayload(byte[] value); @@ -52,6 +55,8 @@ public abstract static class Builder { public abstract Builder setTimestamp(@Nullable Instant value); + public abstract Builder setSchemaHash(@Nullable byte[] value); + public abstract StorageApiWritePayload build(); } @@ -82,6 +87,10 @@ public StorageApiWritePayload withTimestamp(Instant instant) { return toBuilder().setTimestamp(instant).build(); } + public StorageApiWritePayload withSchemaHash(byte[] schemaHash) { + return toBuilder().setSchemaHash(schemaHash).build(); + } + public @Memoized @Nullable TableRow getUnknownFields() throws IOException { @Nullable byte[] fields = getUnknownFieldsPayload(); if (fields == null) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java index bb37bd4c1735..8a84125d71d8 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; +import static org.apache.beam.sdk.io.gcp.bigquery.UpgradeTableSchema.isPayloadSchemaOutOfDate; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; @@ -74,7 +75,11 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.BackOff; +import org.apache.beam.sdk.util.BackOffUtils; +import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.Preconditions; +import org.apache.beam.sdk.util.Sleeper; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.OutputBuilder; import org.apache.beam.sdk.values.PCollection; @@ -296,9 +301,8 @@ class DestinationState { private String streamName = ""; private @Nullable AppendClientInfo appendClientInfo = null; private long currentOffset = 0; - private List pendingMessages; + private List pendingMessages; private List pendingTimestamps; - private List<@Nullable TableRow> pendingFailsafeTableRows; private transient @Nullable WriteStreamService maybeWriteStreamService; private final Counter recordsAppended = Metrics.counter(WriteRecordsDoFn.class, "recordsAppended"); @@ -313,8 +317,7 @@ class DestinationState { private final Callable tryCreateTable; private final boolean useDefaultStream; - private TableSchema initialTableSchema; - private DescriptorProtos.DescriptorProto initialDescriptor; + private final MessageConverter messageConverter; private Instant nextCacheTickle = Instant.MAX; private final int clientNumber; private final boolean usingMultiplexing; @@ -340,11 +343,9 @@ public DestinationState( this.shortTableUrn = shortTableUrn; this.pendingMessages = Lists.newArrayList(); this.pendingTimestamps = Lists.newArrayList(); - this.pendingFailsafeTableRows = Lists.newArrayList(); this.maybeWriteStreamService = writeStreamService; this.useDefaultStream = useDefaultStream; - this.initialTableSchema = messageConverter.getTableSchema(); - this.initialDescriptor = messageConverter.getDescriptor(includeCdcColumns); + this.messageConverter = messageConverter; this.clientNumber = new Random().nextInt(streamAppendClientCount); this.usingMultiplexing = usingMultiplexing; this.maxRequestSize = maxRequestSize; @@ -471,7 +472,8 @@ SchemaAndDescriptor getCurrentTableSchema(String stream, @Nullable TableSchema u updatedSchema, true, includeCdcColumns)); } - AtomicReference currentSchema = new AtomicReference<>(initialTableSchema); + AtomicReference currentSchema = + new AtomicReference<>(messageConverter.getTableSchema()); AtomicBoolean updated = new AtomicBoolean(); CreateTableHelpers.createTableWrapper( () -> { @@ -482,7 +484,8 @@ SchemaAndDescriptor getCurrentTableSchema(String stream, @Nullable TableSchema u .getWriteStreamSchema(streamName); if (streamSchema != null) { Optional newSchema = - TableSchemaUpdateUtils.getUpdatedSchema(initialTableSchema, streamSchema); + TableSchemaUpdateUtils.getUpdatedSchema( + messageConverter.getTableSchema(), streamSchema); if (newSchema.isPresent()) { currentSchema.set(newSchema.get()); updated.set(true); @@ -503,7 +506,7 @@ SchemaAndDescriptor getCurrentTableSchema(String stream, @Nullable TableSchema u updated.get() ? TableRowToStorageApiProto.descriptorSchemaFromTableSchema( currentSchema.get(), true, includeCdcColumns) - : initialDescriptor; + : messageConverter.getDescriptor(includeCdcColumns); return new SchemaAndDescriptor(currentSchema.get(), descriptor); } @@ -577,8 +580,8 @@ void addMessage( OutputReceiver failedRowsReceiver) throws Exception { maybeTickleCache(); - ByteString payloadBytes = ByteString.copyFrom(payload.getPayload()); - @Nullable TableRow failsafeTableRow = payload.getFailsafeTableRow(); + + @Nullable ByteString mergedPayloadBytes = null; if (autoUpdateSchema) { if (appendClientInfo == null) { appendClientInfo = getAppendClientInfo(true, null); @@ -587,16 +590,17 @@ void addMessage( if (unknownFields != null && !unknownFields.isEmpty()) { // check if unknownFields contains repeated struct, merge // otherwise use concat + mergedPayloadBytes = ByteString.copyFrom(payload.getPayload()); try { - payloadBytes = + mergedPayloadBytes = Preconditions.checkStateNotNull(appendClientInfo) - .mergeNewFields(payloadBytes, unknownFields, ignoreUnknownValues); + .mergeNewFields(mergedPayloadBytes, unknownFields, ignoreUnknownValues); } catch (TableRowToStorageApiProto.SchemaConversionException e) { @Nullable TableRow tableRow = payload.getFailsafeTableRow(); if (tableRow == null) { tableRow = checkNotNull(appendClientInfo) - .toTableRow(payloadBytes, Predicates.alwaysTrue()); + .toTableRow(mergedPayloadBytes, Predicates.alwaysTrue()); } // TODO(24926, reuvenlax): We need to merge the unknown fields in! Currently we only // execute this @@ -615,8 +619,15 @@ void addMessage( } } } - pendingMessages.add(payloadBytes); - pendingFailsafeTableRows.add(failsafeTableRow); + byte[] byteArray = + mergedPayloadBytes != null ? mergedPayloadBytes.toByteArray() : payload.getPayload(); + StorageApiWritePayload pending = + StorageApiWritePayload.of(byteArray, null, payload.getFailsafeTableRow()); + byte[] schemaHash = payload.getSchemaHash(); + if (schemaHash != null) { + pending = pending.withSchemaHash(schemaHash); + } + pendingMessages.add(pending); org.joda.time.Instant timestamp = payload.getTimestamp(); pendingTimestamps.add(timestamp != null ? timestamp : elementTs); @@ -632,13 +643,51 @@ long flush( } final ProtoRows.Builder insertsBuilder = ProtoRows.newBuilder(); - insertsBuilder.addAllSerializedRows(pendingMessages); + List<@Nullable TableRow> failsafeTableRows = Lists.newArrayList(); + boolean schemaMismatchSeen; + BackOff backoff = + FluentBackoff.DEFAULT + .withInitialBackoff(Duration.standardSeconds(1)) + .withMaxBackoff(Duration.standardMinutes(1)) + .withMaxRetries(500) + .withThrottledTimeCounter( + BigQuerySinkMetrics.throttledTimeCounter( + BigQuerySinkMetrics.RpcMethod.OPEN_WRITE_STREAM)) + .backoff(); + do { + insertsBuilder.clear(); + failsafeTableRows.clear(); + schemaMismatchSeen = false; + for (StorageApiWritePayload payload : pendingMessages) { + schemaMismatchSeen = + isPayloadSchemaOutOfDate( + payload, + () -> getAppendClientInfo(true, null).getTableSchemaHash(), + () -> + TableRowToStorageApiProto.wrapDescriptorProto( + messageConverter.getDescriptor(includeCdcColumns))); + if (schemaMismatchSeen) { + break; + } + + insertsBuilder.addSerializedRows(ByteString.copyFrom(payload.getPayload())); + failsafeTableRows.add(payload.getFailsafeTableRow()); + } + if (schemaMismatchSeen) { + LOG.info("Schema out of date: refreshing table schema for {}.", tableUrn); + // Refresh our view of the schema and try again.. + this.messageConverter.updateSchemaFromTable(); + invalidateWriteStream(); + this.appendClientInfo = + Preconditions.checkStateNotNull( + getAppendClientInfo( + false, null /* read updated schema from messageConverter */)); + } + } while (schemaMismatchSeen && BackOffUtils.next(Sleeper.DEFAULT, backoff)); + pendingMessages.clear(); final ProtoRows inserts = insertsBuilder.build(); List insertTimestamps = pendingTimestamps; - List<@Nullable TableRow> failsafeTableRows = pendingFailsafeTableRows; - pendingTimestamps = Lists.newArrayList(); - pendingFailsafeTableRows = Lists.newArrayList(); // Handle the case where the request is too large. if (inserts.getSerializedSize() >= maxRequestSize) { @@ -743,7 +792,7 @@ long flush( if (failedRow == null) { ByteString protoBytes = failedContext.protoRows.getSerializedRows(failedIndex); - AppendClientInfo aci = Preconditions.checkStateNotNull(appendClientInfo); + AppendClientInfo aci = Preconditions.checkStateNotNull(this.appendClientInfo); failedRow = TableRowToStorageApiProto.tableRowFromMessage( aci.getSchemaInformation(), @@ -847,11 +896,40 @@ long flush( + failedContext.offset); } + // Schema mismatched exceptions can happen if the table was recently updated. Since + // vortex caches schemas + // we might see the new schema before vortex does. In this case, we simply need to + // retry. + Exceptions.@Nullable StorageException storageException = + (error == null) ? null : Exceptions.toStorageException(error); + boolean schemaMismatchError = + (storageException instanceof Exceptions.SchemaMismatchedException); + if (!schemaMismatchError && error != null) { + // There's no special error code for missing required fields, and that can also + // happen due to vortex + // being delayed at seeing a new schema. We're forced to parse the description to + // determine that this + // has happened. + Status status = Status.fromThrowable(error); + if (status.getCode() == Status.Code.INVALID_ARGUMENT) { + String description = status.getDescription(); + schemaMismatchError = + description != null && description.contains("incompatible fields"); + } + } + if (schemaMismatchError) { + LOG.info( + "Vortex failed stream open due to incompatible fields. This is likely because the BigTable " + + "schema was recently updated and Vortex hasn't noticed yet, so retrying. error {}", + Preconditions.checkStateNotNull(error).toString()); + } + boolean hasPersistentErrors = failedContext.getError() instanceof Exceptions.StreamFinalizedException || statusCode.equals(Status.Code.INVALID_ARGUMENT) || statusCode.equals(Status.Code.NOT_FOUND) || statusCode.equals(Status.Code.FAILED_PRECONDITION); + hasPersistentErrors = hasPersistentErrors && !schemaMismatchError; if (hasPersistentErrors) { throw new RuntimeException( String.format( @@ -948,7 +1026,7 @@ void postFlush() { if (updatedTableSchemaReturned != null) { Optional updatedTableSchema = TableSchemaUpdateUtils.getUpdatedSchema( - this.initialTableSchema, updatedTableSchemaReturned); + this.messageConverter.getTableSchema(), updatedTableSchemaReturned); if (updatedTableSchema.isPresent()) { invalidateWriteStream(); appendClientInfo = @@ -1045,7 +1123,6 @@ void flushAll( long numRowsWritten = 0; for (DestinationState destinationState : Preconditions.checkStateNotNull(destinations).values()) { - RetryManager retryManager = new RetryManager<>( Duration.standardSeconds(1), @@ -1130,7 +1207,13 @@ DestinationState createDestinationState( MessageConverter messageConverter; try { - messageConverter = messageConverters.get(destination, dynamicDestinations, datasetService); + messageConverter = + messageConverters.get( + destination, + dynamicDestinations, + c.getPipelineOptions(), + datasetService, + writeStreamService); return new DestinationState( tableDestination1, tableDestination1.getTableUrn(bigQueryOptions), diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java index 1f99bf1a3690..3c2e40e03363 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java @@ -23,6 +23,7 @@ import com.google.api.core.ApiFutures; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; +import com.google.auto.value.AutoValue; import com.google.cloud.bigquery.storage.v1.AppendRowsRequest; import com.google.cloud.bigquery.storage.v1.AppendRowsResponse; import com.google.cloud.bigquery.storage.v1.Exceptions; @@ -37,6 +38,7 @@ import java.io.IOException; import java.time.Instant; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; @@ -86,11 +88,16 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Repeatedly; import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.BackOff; +import org.apache.beam.sdk.util.BackOffUtils; +import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.util.ShardedKey; +import org.apache.beam.sdk.util.Sleeper; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.sdk.values.TypeDescriptor; @@ -144,45 +151,6 @@ public class StorageApiWritesShardedRecords> flushTag = new TupleTag<>("flushTag"); private static final ExecutorService closeWriterExecutor = Executors.newCachedThreadPool(); - // Context passed into RetryManager for each call. - class AppendRowsContext extends RetryManager.Operation.Context { - final ShardedKey key; - String streamName = ""; - @Nullable StreamAppendClient client = null; - long offset = -1; - long numRows = 0; - long tryIteration = 0; - ProtoRows protoRows; - - List timestamps; - List<@Nullable TableRow> failsafeTableRows; - - AppendRowsContext( - ShardedKey key, - ProtoRows protoRows, - List timestamps, - List<@Nullable TableRow> failsafeTableRows) { - this.key = key; - this.protoRows = protoRows; - this.timestamps = timestamps; - this.failsafeTableRows = failsafeTableRows; - } - - @Override - public String toString() { - return "Context: key=" - + key - + " streamName=" - + streamName - + " offset=" - + offset - + " numRows=" - + numRows - + " tryIteration: " - + tryIteration; - } - }; - private static final Cache>, AppendClientInfo> APPEND_CLIENTS = CacheBuilder.newBuilder() .expireAfterAccess(5, TimeUnit.MINUTES) @@ -309,6 +277,74 @@ public PCollectionTuple expand( return writeRecordsResult; } + // Context passed into RetryManager for each call. + static class AppendRowsContext + extends RetryManager.Operation.Context { + final ShardedKey key; + String streamName = ""; + @Nullable StreamAppendClient client = null; + long offset = -1; + long numRows = 0; + long tryIteration = 0; + ProtoRows protoRows; + + List timestamps; + List<@Nullable TableRow> failsafeTableRows; + + AppendRowsContext( + ShardedKey key, + ProtoRows protoRows, + List timestamps, + List<@Nullable TableRow> failsafeTableRows) { + this.key = key; + this.protoRows = protoRows; + this.timestamps = timestamps; + this.failsafeTableRows = failsafeTableRows; + } + + @Override + public String toString() { + return "Context: key=" + + key + + " streamName=" + + streamName + + " offset=" + + offset + + " numRows=" + + numRows + + " tryIteration: " + + tryIteration; + } + }; + + @AutoValue + abstract static class CreateRetryManagerResult { + abstract @Nullable RetryManager> + getRetryManager(); + + abstract boolean getSchemaMismatchSeen(); + + abstract List> getFailedRows(); + + abstract int getRecordsAppended(); + + abstract List getHistogramValues(); + + static CreateRetryManagerResult schemaMismatch() { + return new AutoValue_StorageApiWritesShardedRecords_CreateRetryManagerResult<>( + null, true, Collections.emptyList(), 0, Collections.emptyList()); + } + + static CreateRetryManagerResult of( + RetryManager> retryManager, + List> failedRows, + int recordsAppended, + List histogram) { + return new AutoValue_StorageApiWritesShardedRecords_CreateRetryManagerResult<>( + retryManager, false, failedRows, recordsAppended, histogram); + } + } + class WriteRecordsDoFn extends DoFn< KV, Iterable>, KV> { @@ -451,6 +487,73 @@ public void onTeardown() { } } + private CreateRetryManagerResult createRetryManager( + ShardedKey key, + Iterable messages, + Function, ApiFuture> runOperation, + Function>, RetryType> onError, + Consumer> onSuccess, + AppendClientInfo appendClientInfo, + TableReference tableReference) + throws Exception { + RetryManager> retryManager = + new RetryManager<>( + Duration.standardSeconds(1), + Duration.standardSeconds(20), + maxRetries, + BigQuerySinkMetrics.throttledTimeCounter(BigQuerySinkMetrics.RpcMethod.APPEND_ROWS)); + + List> failedRows = Lists.newArrayList(); + int recordsAppended = 0; + List histogramUpdates = Lists.newArrayList(); + for (SplittingIterable.Value splitValue : messages) { + if (splitValue.getSchemaMismatchSeen()) { + return CreateRetryManagerResult.schemaMismatch(); + } + // Handle the case of a row that is too large. + if (splitValue.getProtoRows().getSerializedSize() >= maxRequestSize) { + if (splitValue.getProtoRows().getSerializedRowsCount() > 1) { + // TODO(reuvenlax): Is it worth trying to handle this case by splitting the protoRows? + // Given that we split + // the ProtoRows iterable at 2MB and the max request size is 10MB, this scenario seems + // nearly impossible. + LOG.error( + "A request containing more than one row is over the request size limit of {}. This is unexpected. All rows in the request will be sent to the failed-rows PCollection.", + maxRequestSize); + } + for (int i = 0; i < splitValue.getProtoRows().getSerializedRowsCount(); ++i) { + org.joda.time.Instant timestamp = splitValue.getTimestamps().get(i); + TableRow failedRow = splitValue.getFailsafeTableRows().get(i); + if (failedRow == null) { + ByteString rowBytes = splitValue.getProtoRows().getSerializedRows(i); + failedRow = appendClientInfo.toTableRow(rowBytes, Predicates.alwaysTrue()); + } + failedRows.add( + TimestampedValue.of( + new BigQueryStorageApiInsertError( + failedRow, + "Row payload too large. Maximum size " + maxRequestSize, + tableReference), + timestamp)); + } + } else { + // RetryManager + AppendRowsContext context = + new AppendRowsContext<>( + key, + splitValue.getProtoRows(), + splitValue.getTimestamps(), + splitValue.getFailsafeTableRows()); + retryManager.addOperation(runOperation, onError, onSuccess, context); + recordsAppended += splitValue.getProtoRows().getSerializedRowsCount(); + histogramUpdates.add(context.protoRows.getSerializedRowsCount()); + } + } + + return CreateRetryManagerResult.of( + retryManager, failedRows, recordsAppended, histogramUpdates); + } + @ProcessElement public void process( ProcessContext c, @@ -516,6 +619,14 @@ public void process( () -> getOrCreateStream( tableId, streamName, streamOffset, idleTimer, writeStreamService, tryCreateTable); + + StorageApiDynamicDestinations.MessageConverter messageConverter = + messageConverters.get( + element.getKey().getKey(), + dynamicDestinations, + pipelineOptions, + datasetService, + writeStreamService); Callable getAppendClientInfo = () -> { @Nullable TableSchema tableSchema; @@ -531,11 +642,8 @@ public void process( } else { // Start off with the base schema. As we get notified of schema updates, we // will update the descriptor. - StorageApiDynamicDestinations.MessageConverter converter = - messageConverters.get( - element.getKey().getKey(), dynamicDestinations, datasetService); - tableSchema = converter.getTableSchema(); - descriptor = converter.getDescriptor(false); + tableSchema = messageConverter.getTableSchema(); + descriptor = messageConverter.getDescriptor(false); if (autoUpdateSchema) { // A StreamWriter ignores table schema updates that happen prior to its creation. @@ -611,35 +719,9 @@ public void process( } } - // Each ProtoRows object contains at most 1MB of rows. - // TODO: Push messageFromTableRow up to top level. That we we cans skip TableRow entirely if - // already proto or already schema. - Iterable messages = - new SplittingIterable( - element.getValue(), - splitSize, - (bytes, tableRow) -> - appendClientInfo.get().mergeNewFields(bytes, tableRow, ignoreUnknownValues), - bytes -> appendClientInfo.get().toTableRow(bytes, Predicates.alwaysTrue()), - (failedRow, errorMessage) -> { - o.get(failedRowsTag) - .outputWithTimestamp( - new BigQueryStorageApiInsertError( - failedRow.getValue(), errorMessage, tableReference), - failedRow.getTimestamp()); - rowsSentToFailedRowsCollection.inc(); - BigQuerySinkMetrics.appendRowsRowStatusCounter( - BigQuerySinkMetrics.RowStatus.FAILED, - BigQuerySinkMetrics.PAYLOAD_TOO_LARGE, - shortTableId) - .inc(1); - }, - autoUpdateSchema, - elementTs); - // Initialize stream names and offsets for all contexts. This will be called initially, but // will also be called if we roll over to a new stream on a retry. - BiConsumer, Boolean> initializeContexts = + BiConsumer>, Boolean> initializeContexts = (contexts, isFailure) -> { try { if (isFailure) { @@ -659,7 +741,7 @@ public void process( appendClientInfo.get().getStreamAppendClient()); String streamNameRead = Preconditions.checkArgumentNotNull(streamName.read()); long currentOffset = Preconditions.checkArgumentNotNull(streamOffset.read()); - for (AppendRowsContext context : contexts) { + for (AppendRowsContext context : contexts) { context.streamName = streamNameRead; streamAppendClient.pin(); context.client = appendClientInfo.get().getStreamAppendClient(); @@ -673,13 +755,13 @@ public void process( } }; - Consumer> clearClients = - contexts -> { + Consumer>> clearClients = + (contexts) -> { APPEND_CLIENTS.invalidate(messageConverters.getAppendClientKey(element.getKey())); appendClientInfo.set(appendClientInfo.get().withNoAppendClient()); APPEND_CLIENTS.put( messageConverters.getAppendClientKey(element.getKey()), appendClientInfo.get()); - for (AppendRowsContext context : contexts) { + for (AppendRowsContext context : contexts) { if (context.client != null) { // Unpin in a different thread, as it may execute a blocking close. runAsyncIgnoreFailure(closeWriterExecutor, context.client::unpin); @@ -688,7 +770,7 @@ public void process( } }; - Function> runOperation = + Function, ApiFuture> runOperation = context -> { if (context.protoRows.getSerializedRowsCount() == 0) { // This might happen if all rows in a batch failed and were sent to the failed-rows @@ -711,10 +793,10 @@ public void process( } }; - Function, RetryType> onError = + Function>, RetryType> onError = failedContexts -> { // The first context is always the one that fails. - AppendRowsContext failedContext = + AppendRowsContext failedContext = Preconditions.checkStateNotNull(Iterables.getFirst(failedContexts, null)); BigQuerySinkMetrics.reportFailedRPCMetrics( failedContext, BigQuerySinkMetrics.RpcMethod.APPEND_ROWS, shortTableId); @@ -776,7 +858,7 @@ public void process( // Since we removed rows, we need to update the insert offsets for all remaining rows. long offset = failedContext.offset; - for (AppendRowsContext context : failedContexts) { + for (AppendRowsContext context : failedContexts) { context.offset = offset; offset += context.protoRows.getSerializedRowsCount(); } @@ -785,6 +867,7 @@ public void process( } Throwable error = Preconditions.checkStateNotNull(failedContext.getError()); + Status.Code statusCode = Status.fromThrowable(error).getCode(); // This means that the offset we have stored does not match the current end of @@ -814,8 +897,12 @@ public void process( } if (!quotaError) { - // This forces us to close and reopen all gRPC connections to Storage API on error, - // which empirically fixes random stuckness issues. + // For known errors (offset mismatch, not found) we must reestablish + // the streams. + // However we've seen that doing this fixes random stuckness issues by reestablishing + // gRPC connections, + // so we close the clients for all non-quota errors. + clearClients.accept(failedContexts); } appendFailures.inc(); @@ -824,6 +911,36 @@ public void process( BigQuerySinkMetrics.RowStatus.RETRIED, errorCode, shortTableId) .inc(retriedRows); + // Schema mismatched exceptions can happen if the table was recently updated. Since + // vortex caches schemas + // we might see the new schema before vortex does. In this case, we simply need to + // retry. + Exceptions.@Nullable StorageException storageException = + Exceptions.toStorageException(error); + boolean schemaMismatchError = + (storageException instanceof Exceptions.SchemaMismatchedException); + if (!schemaMismatchError) { + // There's no special error code for missing required fields, and that can also happen + // due to vortex + // being delayed at seeing a new schema. We're forced to parse the description to + // determine that this has happened. + // TODO: Vortex team to introduce a special storage error code for this, so we don't + // have to parse + // descriptions. + Status status = Status.fromThrowable(error); + if (status.getCode() == Code.INVALID_ARGUMENT) { + String description = status.getDescription(); + schemaMismatchError = + description != null && description.contains("incompatible fields"); + } + } + if (schemaMismatchError) { + LOG.info( + "Vortex failed stream open due to incompatible fields. This is likely because the BigTable " + + "schema was recently updated and Vortex hasn't noticed yet, so retrying. error {}", + Preconditions.checkStateNotNull(error).toString()); + } + boolean explicitStreamFinalized = failedContext.getError() instanceof StreamFinalizedException; // This implies that the stream doesn't exist or has already been finalized. In this @@ -833,6 +950,8 @@ public void process( || statusCode.equals(Code.INVALID_ARGUMENT) || statusCode.equals(Code.NOT_FOUND) || statusCode.equals(Code.FAILED_PRECONDITION); + streamDoesNotExist = streamDoesNotExist && !schemaMismatchError; + if (offsetMismatch || streamDoesNotExist) { appendOffsetFailures.inc(); LOG.warn( @@ -855,7 +974,7 @@ public void process( return RetryType.RETRY_ALL_OPERATIONS; }; - Consumer onSuccess = + Consumer> onSuccess = context -> { AppendRowsResponse response = Preconditions.checkStateNotNull(context.getResult()); o.get(flushTag) @@ -884,64 +1003,99 @@ public void process( } } }; - Instant now = Instant.now(); - List contexts = Lists.newArrayList(); - RetryManager retryManager = - new RetryManager<>( - Duration.standardSeconds(1), - Duration.standardSeconds(20), - maxRetries, - BigQuerySinkMetrics.throttledTimeCounter(BigQuerySinkMetrics.RpcMethod.APPEND_ROWS)); - int numAppends = 0; - for (SplittingIterable.Value splitValue : messages) { - // Handle the case of a row that is too large. - if (splitValue.getProtoRows().getSerializedSize() >= maxRequestSize) { - if (splitValue.getProtoRows().getSerializedRowsCount() > 1) { - // TODO(reuvenlax): Is it worth trying to handle this case by splitting the protoRows? - // Given that we split - // the ProtoRows iterable at 2MB and the max request size is 10MB, this scenario seems - // nearly impossible. - LOG.error( - "A request containing more than one row is over the request size limit of {}. This is unexpected. All rows in the request will be sent to the failed-rows PCollection.", - maxRequestSize); - } - for (int i = 0; i < splitValue.getProtoRows().getSerializedRowsCount(); ++i) { - org.joda.time.Instant timestamp = splitValue.getTimestamps().get(i); - TableRow failedRow = splitValue.getFailsafeTableRows().get(i); - if (failedRow == null) { - ByteString rowBytes = splitValue.getProtoRows().getSerializedRows(i); - failedRow = appendClientInfo.get().toTableRow(rowBytes, Predicates.alwaysTrue()); - } - o.get(failedRowsTag) - .outputWithTimestamp( - new BigQueryStorageApiInsertError( - failedRow, - "Row payload too large. Maximum size " + maxRequestSize, - tableReference), - timestamp); - } - int numRowsFailed = splitValue.getProtoRows().getSerializedRowsCount(); - rowsSentToFailedRowsCollection.inc(numRowsFailed); - BigQuerySinkMetrics.appendRowsRowStatusCounter( - BigQuerySinkMetrics.RowStatus.FAILED, - BigQuerySinkMetrics.PAYLOAD_TOO_LARGE, - shortTableId) - .inc(numRowsFailed); - } else { - ++numAppends; - // RetryManager - AppendRowsContext context = - new AppendRowsContext( - element.getKey(), - splitValue.getProtoRows(), - splitValue.getTimestamps(), - splitValue.getFailsafeTableRows()); - contexts.add(context); - retryManager.addOperation(runOperation, onError, onSuccess, context); - recordsAppended.inc(splitValue.getProtoRows().getSerializedRowsCount()); - appendSizeDistribution.update(context.protoRows.getSerializedRowsCount()); + + BackOff backoff = + FluentBackoff.DEFAULT + .withInitialBackoff(Duration.standardSeconds(1)) + .withMaxBackoff(Duration.standardMinutes(1)) + .withMaxRetries(500) + .withThrottledTimeCounter( + BigQuerySinkMetrics.throttledTimeCounter( + BigQuerySinkMetrics.RpcMethod.OPEN_WRITE_STREAM)) + .backoff(); + CreateRetryManagerResult createRetryManagerResult; + do { + // Each ProtoRows object contains at most 1MB of rows. + // TODO: Push messageFromTableRow up to top level. That we we cans skip TableRow entirely if + // already proto or already schema. + Iterable messages = + new SplittingIterable( + element.getValue(), + splitSize, + // Unknown field merger + (bytes, tableRow) -> + appendClientInfo.get().mergeNewFields(bytes, tableRow, ignoreUnknownValues), + // Convert back to TableRow + bytes -> appendClientInfo.get().toTableRow(bytes, Predicates.alwaysTrue()), + // Failed rows consumer + (failedRow, errorMessage) -> { + o.get(failedRowsTag) + .outputWithTimestamp( + new BigQueryStorageApiInsertError( + failedRow.getValue(), errorMessage, tableReference), + failedRow.getTimestamp()); + rowsSentToFailedRowsCollection.inc(); + BigQuerySinkMetrics.appendRowsRowStatusCounter( + BigQuerySinkMetrics.RowStatus.FAILED, + BigQuerySinkMetrics.PAYLOAD_TOO_LARGE, + shortTableId) + .inc(1); + }, + // Get the currently-known TableSchema hash + () -> appendClientInfo.get().getTableSchemaHash(), + () -> + TableRowToStorageApiProto.wrapDescriptorProto( + messageConverter.getDescriptor(false)), + autoUpdateSchema, + elementTs); + + createRetryManagerResult = + createRetryManager( + element.getKey(), + messages, + runOperation, + onError, + onSuccess, + appendClientInfo.get(), + tableReference); + if (createRetryManagerResult.getSchemaMismatchSeen()) { + // TODO: The call to updateSchemaFromTable will throttle the DoFn (both because of the RPC + // call and because + // the cache has a delay on refresh). We should update throttling counters here as well. + LOG.info("Schema out of date: refreshing table schema for {}", tableId); + // Force the message converter to get the schema again from the table. + messageConverter.updateSchemaFromTable(); + // Close all RPC clients that were opened with the old descriptor. Clear the cache, + // forcing us to create a new append client with the updated descriptor. + APPEND_CLIENTS.invalidate(messageConverters.getAppendClientKey(element.getKey())); + appendClientInfo.set( + APPEND_CLIENTS.get( + messageConverters.getAppendClientKey(element.getKey()), getAppendClientInfo)); } - } + } while (createRetryManagerResult.getSchemaMismatchSeen() + && BackOffUtils.next(Sleeper.DEFAULT, backoff)); + + // Output any rows that failed along they way. + createRetryManagerResult + .getFailedRows() + .forEach( + tv -> o.get(failedRowsTag).outputWithTimestamp(tv.getValue(), tv.getTimestamp())); + rowsSentToFailedRowsCollection.inc(createRetryManagerResult.getFailedRows().size()); + BigQuerySinkMetrics.appendRowsRowStatusCounter( + BigQuerySinkMetrics.RowStatus.FAILED, + BigQuerySinkMetrics.PAYLOAD_TOO_LARGE, + shortTableId) + .inc(createRetryManagerResult.getFailedRows().size()); + + recordsAppended.inc(createRetryManagerResult.getRecordsAppended()); + createRetryManagerResult.getHistogramValues().forEach(appendSizeDistribution::update); + + Instant now = Instant.now(); + + RetryManager> retryManager = + Preconditions.checkStateNotNull(createRetryManagerResult.getRetryManager()); + int numAppends = retryManager.getRemainingOperationCount(); + Iterable> contexts = retryManager.getRemainingContexts(); if (numAppends > 0) { initializeContexts.accept(contexts, false); @@ -949,7 +1103,7 @@ public void process( retryManager.run(true); } finally { // Make sure that all pins are removed. - for (AppendRowsContext context : contexts) { + for (AppendRowsContext context : contexts) { if (context.client != null) { runAsyncIgnoreFailure(closeWriterExecutor, context.client::unpin); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProto.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProto.java index a3776b595d85..c3844e831480 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProto.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProto.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.gcp.bigquery; import static java.util.stream.Collectors.toList; +import static java.util.stream.Collectors.toSet; import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.DATETIME_SPACE_FORMATTER; import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.TIMESTAMP_FORMATTER; @@ -80,6 +81,9 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.hash.HashCode; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.hash.HashFunction; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.hash.Hashing; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.BaseEncoding; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Days; @@ -89,6 +93,45 @@ * with the Storage write API. */ public class TableRowToStorageApiProto { + + public static class ErrorCollector { + private final List exceptions = Lists.newArrayList(); + private final Predicate shouldCollect; + + public static final ErrorCollector DONT_COLLECT = new ErrorCollector(Predicates.alwaysFalse()); + + public ErrorCollector(Predicate shouldCollect) { + this.shouldCollect = shouldCollect; + } + + // Returns true if the exception was collected. + void collect(SchemaConversionException exception) throws SchemaConversionException { + if (shouldCollect.test(exception)) { + exceptions.add(exception); + } else { + throw exception; + } + } + + public void mergeInto(ErrorCollector other) throws SchemaConversionException { + for (SchemaConversionException e : other.exceptions) { + collect(e); + } + } + + List getExceptions() { + return exceptions; + } + + void clear() { + exceptions.clear(); + } + + boolean isEmpty() { + return exceptions.isEmpty(); + } + } + abstract static class SchemaConversionException extends Exception { SchemaConversionException(String msg) { super(msg); @@ -100,8 +143,36 @@ abstract static class SchemaConversionException extends Exception { } public static class SchemaTooNarrowException extends SchemaConversionException { - SchemaTooNarrowException(String msg) { + private final String missingField; + private final boolean isRepeated; + private final boolean isStruct; + + SchemaTooNarrowException( + String missingField, String msg, boolean isRepeated, boolean isStruct) { super(msg); + this.missingField = missingField; + this.isRepeated = isRepeated; + this.isStruct = isStruct; + } + + SchemaTooNarrowException( + String missingField, String msg, Exception e, boolean isRepeated, boolean isStruct) { + super(msg, e); + this.missingField = missingField; + this.isRepeated = isRepeated; + this.isStruct = isStruct; + } + + public String getMissingField() { + return missingField; + } + + public boolean isRepeated() { + return isRepeated; + } + + public boolean isStruct() { + return isStruct; } } @@ -115,6 +186,24 @@ public static class SchemaDoesntMatchException extends SchemaConversionException } } + public static class SchemaMissingRequiredFieldException extends SchemaConversionException { + private final Set missingFields; + + SchemaMissingRequiredFieldException(Set missingFields) { + super("Missing required fields: " + missingFields); + this.missingFields = missingFields; + } + + SchemaMissingRequiredFieldException(Set missingFields, Exception e) { + super("Missing required fields: " + missingFields + ". Exception: " + e, e); + this.missingFields = missingFields; + } + + public Set getMissingFields() { + return missingFields; + } + } + public static class SingleValueConversionException extends SchemaConversionException { SingleValueConversionException( Object sourceValue, TableFieldSchema.Type type, String fullName, Exception e) { @@ -213,6 +302,11 @@ public interface ThrowingBiFunction { OutputT apply(FirstInputT t, SecondInputT u) throws SchemaConversionException; } + @FunctionalInterface + public interface ThrowingBiConsumer { + void accept(FirstInputT t, SecondInputT u) throws SchemaConversionException; + } + static final DecimalFormat DECIMAL_FORMAT = new DecimalFormat("0.0###############", DecimalFormatSymbols.getInstance(Locale.ROOT)); @@ -255,7 +349,7 @@ public interface ThrowingBiFunction { }) .put( TableFieldSchema.Type.DOUBLE, - (schemaInformation, value) -> { + (fullName, value) -> { if (value instanceof String) { return Double.valueOf((String) value); } else if (value instanceof Number) { @@ -265,7 +359,7 @@ public interface ThrowingBiFunction { }) .put( TableFieldSchema.Type.BOOL, - (schemaInformation, value) -> { + (fullName, value) -> { if (value instanceof String) { return Boolean.valueOf((String) value); } else if (value instanceof Boolean) { @@ -275,7 +369,7 @@ public interface ThrowingBiFunction { }) .put( TableFieldSchema.Type.BYTES, - (schemaInformation, value) -> { + (fullName, value) -> { if (value instanceof String) { return ByteString.copyFrom(BaseEncoding.base64().decode((String) value)); } else if (value instanceof byte[]) { @@ -287,7 +381,7 @@ public interface ThrowingBiFunction { }) .put( TableFieldSchema.Type.TIMESTAMP, - (schemaInformation, value) -> { + (fullName, value) -> { if (value instanceof String) { try { // '2011-12-03T10:15:30Z', '2011-12-03 10:15:30+05:00' @@ -329,7 +423,7 @@ public interface ThrowingBiFunction { }) .put( TableFieldSchema.Type.DATE, - (schemaInformation, value) -> { + (fullName, value) -> { if (value instanceof String) { return ((Long) LocalDate.parse((String) value).toEpochDay()).intValue(); } else if (value instanceof LocalDate) { @@ -346,7 +440,7 @@ public interface ThrowingBiFunction { }) .put( TableFieldSchema.Type.NUMERIC, - (schemaInformation, value) -> { + (fullName, value) -> { if (value instanceof String) { return BigDecimalByteStringEncoder.encodeToNumericByteString( new BigDecimal((String) value)); @@ -366,7 +460,7 @@ public interface ThrowingBiFunction { }) .put( TableFieldSchema.Type.BIGNUMERIC, - (schemaInformation, value) -> { + (fullName, value) -> { if (value instanceof String) { return BigDecimalByteStringEncoder.encodeToBigNumericByteString( new BigDecimal((String) value)); @@ -386,7 +480,7 @@ public interface ThrowingBiFunction { }) .put( TableFieldSchema.Type.DATETIME, - (schemaInformation, value) -> { + (fullName, value) -> { if (value instanceof String) { try { // '2011-12-03T10:15:30' @@ -409,7 +503,7 @@ public interface ThrowingBiFunction { }) .put( TableFieldSchema.Type.TIME, - (schemaInformation, value) -> { + (fullName, value) -> { if (value instanceof String) { return CivilTimeEncoder.encodePacked64TimeMicros( LocalTime.parse((String) value)); @@ -425,18 +519,38 @@ public interface ThrowingBiFunction { }) .put( TableFieldSchema.Type.STRING, - (schemaInformation, value) -> - Preconditions.checkArgumentNotNull(value).toString()) + (fullName, value) -> Preconditions.checkArgumentNotNull(value).toString()) .put( TableFieldSchema.Type.JSON, - (schemaInformation, value) -> - Preconditions.checkArgumentNotNull(value).toString()) + (fullName, value) -> Preconditions.checkArgumentNotNull(value).toString()) .put( TableFieldSchema.Type.GEOGRAPHY, - (schemaInformation, value) -> - Preconditions.checkArgumentNotNull(value).toString()) + (fullName, value) -> Preconditions.checkArgumentNotNull(value).toString()) .build(); + static final HashFunction SCHEMA_HASH_FUNCTION = Hashing.goodFastHash(32); + + public static byte[] tableSchemaHash(TableSchema tableSchema) { + return tableSchemaHash("", tableSchema.getFieldsList()).asBytes(); + } + + public static HashCode tableSchemaHash(String prefix, List fields) { + List hashCodes = Lists.newArrayList(); + for (TableFieldSchema tableFieldSchema : fields) { + String name = + prefix.isEmpty() + ? tableFieldSchema.getName() + : String.join(".", prefix, tableFieldSchema.getName()); + hashCodes.add(SCHEMA_HASH_FUNCTION.hashString(name.toLowerCase(), StandardCharsets.UTF_8)); + hashCodes.add(SCHEMA_HASH_FUNCTION.hashInt(tableFieldSchema.getMode().getNumber())); + + if (tableFieldSchema.getType().equals(TableFieldSchema.Type.STRUCT)) { + hashCodes.add(tableSchemaHash(name, tableFieldSchema.getFieldsList())); + } + } + return Hashing.combineOrdered(hashCodes); + } + public static TableFieldSchema.Mode modeToProtoMode( @Nullable String defaultValueExpression, String mode) { TableFieldSchema.Mode resultMode = @@ -585,12 +699,25 @@ private SchemaInformation( } } + // Returns a SchemaInformation that descends from this one. Does not modify the current + // SchemaInformation - + // the new SchemaInformation is traversable upwards only. + public SchemaInformation createDescendent(TableFieldSchema tableFieldSchema) { + return new SchemaInformation( + tableFieldSchema, Iterables.concat(this.parentSchemas, ImmutableList.of(this))); + } + public String getFullName() { - String prefix = - StreamSupport.stream(parentSchemas.spliterator(), false) - .map(SchemaInformation::getName) - .collect(Collectors.joining(".")); - return prefix.isEmpty() ? getName() : prefix + "." + getName(); + if (!Iterables.isEmpty(parentSchemas)) { + String prefix = + StreamSupport.stream(parentSchemas.spliterator(), false) + .skip(1) + .map(SchemaInformation::getName) + .collect(Collectors.joining(".")); + return prefix.isEmpty() ? getName() : prefix + "." + getName(); + } else { + return ""; + } } public String getName() { @@ -712,24 +839,45 @@ public static Descriptor wrapDescriptorProto(DescriptorProto descriptorProto) return Iterables.getOnlyElement(fileDescriptor.getMessageTypes()); } - public static DynamicMessage messageFromMap( + public static @Nullable DynamicMessage messageFromMap( SchemaInformation schemaInformation, - Descriptor descriptor, + @Nullable Descriptor descriptor, AbstractMap map, boolean ignoreUnknownValues, boolean allowMissingRequiredFields, @Nullable TableRow unknownFields, @Nullable String changeType, - @Nullable String changeSequenceNum) + @Nullable String changeSequenceNum, + ErrorCollector collectedExceptions) throws SchemaConversionException { - DynamicMessage.Builder builder = DynamicMessage.newBuilder(descriptor); + DynamicMessage.@Nullable Builder builder = null; + if (descriptor != null) { + builder = DynamicMessage.newBuilder(descriptor); + } + + @Nullable Set requiredFieldsRemaining = null; + if (!allowMissingRequiredFields) { + requiredFieldsRemaining = + schemaInformation.subFields.stream() + .filter(s -> !s.isNullable() && !s.isRepeated()) + .map(SchemaInformation::getName) + .map(String::toLowerCase) + .collect(toSet()); + } for (final Map.Entry entry : map.entrySet()) { String key = entry.getKey().toLowerCase(); + if (requiredFieldsRemaining != null) { + requiredFieldsRemaining.remove(key); + } + String protoFieldName = BigQuerySchemaUtil.isProtoCompatible(key) ? key : BigQuerySchemaUtil.generatePlaceholderFieldName(key); - @Nullable FieldDescriptor fieldDescriptor = descriptor.findFieldByName(protoFieldName); + @Nullable + FieldDescriptor fieldDescriptor = + (descriptor == null) ? null : descriptor.findFieldByName(protoFieldName); + if (fieldDescriptor == null) { if (unknownFields != null) { unknownFields.set(key, entry.getValue()); @@ -737,11 +885,70 @@ public static DynamicMessage messageFromMap( if (ignoreUnknownValues) { continue; } else { - throw new SchemaTooNarrowException( - "TableRow contained unexpected field with name " - + entry.getKey() - + " not found in schema for " - + schemaInformation.getFullName()); + String prefix = schemaInformation.getFullName(); + String fieldName = prefix.isEmpty() ? key : String.join(".", prefix, key); + + boolean isRepeated = entry.getValue() instanceof List; + Predicate isStruct = + o -> { + if (o instanceof AbstractMap) { + return true; + } + if (o instanceof List) { + List list = (List) o; + if (!list.isEmpty()) { + if (list.get(0) instanceof AbstractMap) { + return true; + } + } + } + return false; + }; + + SchemaConversionException exception = + new SchemaTooNarrowException( + fieldName, + "TableRow contained unexpected field with name " + + entry.getKey() + + " not found in schema for " + + schemaInformation.getFullName(), + isRepeated, + isStruct.test(entry.getValue())); + collectedExceptions.collect(exception); // Throws if not collected. + + // Recursively search for missing nested fields. + ThrowingBiConsumer possiblyHandleNestedStruct = + (name, o) -> { + if (o != null && isStruct.test(o)) { + TableFieldSchema recursiveFieldSchema = + TableFieldSchema.newBuilder() + .setType(TableFieldSchema.Type.STRUCT) + .setName(name.toLowerCase()) + .build(); + messageValueFromFieldValue( + schemaInformation.createDescendent(recursiveFieldSchema), + fieldDescriptor, + o, + ignoreUnknownValues, + allowMissingRequiredFields, + () -> null, + collectedExceptions); + } + }; + // If the unknown field is a recursive type, we must continue recursing to see if there + // are further + // unknown fields added. + if (isRepeated) { + List list = (List) entry.getValue(); + if (list != null) { + for (Object o : list) { + possiblyHandleNestedStruct.accept(entry.getKey(), o); + } + } + } else { + possiblyHandleNestedStruct.accept(entry.getKey(), entry.getValue()); + } + continue; } } @@ -771,9 +978,10 @@ public static DynamicMessage messageFromMap( entry.getValue(), ignoreUnknownValues, allowMissingRequiredFields, - getNestedUnknown); + getNestedUnknown, + collectedExceptions); if (value != null) { - builder.setField(fieldDescriptor, value); + Preconditions.checkArgumentNotNull(builder).setField(fieldDescriptor, value); } // For STRUCT fields, we add a placeholder to unknownFields using the getNestedUnknown // supplier (in case we encounter unknown nested fields). If the placeholder comes out @@ -800,18 +1008,34 @@ public static DynamicMessage messageFromMap( } } if (changeType != null) { - builder.setField( - Preconditions.checkStateNotNull( - descriptor.findFieldByName(StorageApiCDC.CHANGE_TYPE_COLUMN)), - changeType); - builder.setField( - Preconditions.checkStateNotNull( - descriptor.findFieldByName(StorageApiCDC.CHANGE_SQN_COLUMN)), - Preconditions.checkStateNotNull(changeSequenceNum)); + Preconditions.checkArgumentNotNull(builder) + .setField( + Preconditions.checkStateNotNull( + Preconditions.checkArgumentNotNull(descriptor) + .findFieldByName(StorageApiCDC.CHANGE_TYPE_COLUMN)), + changeType); + Preconditions.checkArgumentNotNull(builder) + .setField( + Preconditions.checkStateNotNull( + Preconditions.checkArgumentNotNull(descriptor) + .findFieldByName(StorageApiCDC.CHANGE_SQN_COLUMN)), + Preconditions.checkStateNotNull(changeSequenceNum)); + } + + if (requiredFieldsRemaining != null && !requiredFieldsRemaining.isEmpty()) { + String prefix = schemaInformation.getFullName(); + Set missingFields = + requiredFieldsRemaining.stream() + .map(key -> prefix.isEmpty() ? key : String.join(".", prefix, key)) + .collect(Collectors.toSet()); + SchemaConversionException e = new SchemaMissingRequiredFieldException(missingFields); + collectedExceptions.collect(e); // Throws if not collected. + } + if (!collectedExceptions.isEmpty()) { + return null; } - try { - return builder.build(); + return Preconditions.checkArgumentNotNull(builder).build(); } catch (Exception e) { throw new SchemaDoesntMatchException( "Couldn't convert schema for " + schemaInformation.getFullName(), e); @@ -820,18 +1044,19 @@ public static DynamicMessage messageFromMap( /** * Forwards {@code changeSequenceNum} to {@link #messageFromTableRow(SchemaInformation, - * Descriptor, TableRow, boolean, boolean, TableRow, String, String)} via {@link + * Descriptor, TableRow, boolean, boolean, TableRow, String, String, ErrorCollector)} via {@link * Long#toHexString}. */ - public static DynamicMessage messageFromTableRow( + public static @Nullable DynamicMessage messageFromTableRow( SchemaInformation schemaInformation, - Descriptor descriptor, + @Nullable Descriptor descriptor, TableRow tableRow, boolean ignoreUnknownValues, boolean allowMissingRequiredFields, final @Nullable TableRow unknownFields, @Nullable String changeType, - long changeSequenceNum) + long changeSequenceNum, + ErrorCollector collectedExceptions) throws SchemaConversionException { return messageFromTableRow( schemaInformation, @@ -841,7 +1066,8 @@ public static DynamicMessage messageFromTableRow( allowMissingRequiredFields, unknownFields, changeType, - Long.toHexString(changeSequenceNum)); + Long.toHexString(changeSequenceNum), + collectedExceptions); } /** @@ -849,18 +1075,25 @@ public static DynamicMessage messageFromTableRow( * using the BigQuery Storage API. */ @SuppressWarnings("nullness") - public static DynamicMessage messageFromTableRow( + public static @Nullable DynamicMessage messageFromTableRow( SchemaInformation schemaInformation, - Descriptor descriptor, + @Nullable Descriptor descriptor, TableRow tableRow, boolean ignoreUnknownValues, boolean allowMissingRequiredFields, final @Nullable TableRow unknownFields, @Nullable String changeType, - @Nullable String changeSequenceNum) + @Nullable String changeSequenceNum, + ErrorCollector collectedExceptions) throws SchemaConversionException { @Nullable Object fValue = tableRow.get("f"); if (fValue instanceof List) { + if (descriptor == null) { + // This only happens if we are recursively finding unknwon field names. We don't support + // this for list cells + // (all we have is field position in that case) so just bail out. + return null; + } List> cells = (List>) fValue; DynamicMessage.Builder builder = DynamicMessage.newBuilder(descriptor); int cellsToProcess = cells.size(); @@ -868,9 +1101,17 @@ public static DynamicMessage messageFromTableRow( if (ignoreUnknownValues) { cellsToProcess = descriptor.getFields().size(); } else { - throw new SchemaTooNarrowException( - "TableRow contained too many fields and ignoreUnknownValues not set in " - + schemaInformation.getName()); + SchemaConversionException exception = + new SchemaTooNarrowException( + // In this scenario the row does not contain field names, so we can't record the + // name of the new column. + "", + "TableRow contained too many fields and ignoreUnknownValues not set in " + + schemaInformation.getName(), + false, + false); + collectedExceptions.collect(exception); // Throws if not collected. + cellsToProcess = descriptor.getFields().size(); } } @@ -911,7 +1152,8 @@ public static DynamicMessage messageFromTableRow( cell.get("v"), ignoreUnknownValues, allowMissingRequiredFields, - getNestedUnknown); + getNestedUnknown, + collectedExceptions); if (value != null) { builder.setField(fieldDescriptor, value); } @@ -942,6 +1184,9 @@ public static DynamicMessage messageFromTableRow( } } + if (!collectedExceptions.isEmpty()) { + return null; + } try { return builder.build(); } catch (Exception e) { @@ -957,7 +1202,8 @@ public static DynamicMessage messageFromTableRow( allowMissingRequiredFields, unknownFields, changeType, - changeSequenceNum); + changeSequenceNum, + collectedExceptions); } } @@ -1228,15 +1474,17 @@ public static ByteString mergeNewFields( throw new RuntimeException(e); } ByteString unknownFieldsProto = - messageFromTableRow( - schemaInformation, - descriptorIgnoreRequired, - unknownFields, - ignoreUnknownValues, - true, - null, - null, - null) + Preconditions.checkArgumentNotNull( + messageFromTableRow( + schemaInformation, + descriptorIgnoreRequired, + unknownFields, + ignoreUnknownValues, + true, + null, + null, + null, + ErrorCollector.DONT_COLLECT)) .toByteString(); return tableRowProto.concat(unknownFieldsProto); } @@ -1277,39 +1525,49 @@ public static ByteString mergeNewFields( } TableRow merged = TableRowToStorageApiProto.mergeNewFields(original, unknownFields); DynamicMessage dynamicMessage = - TableRowToStorageApiProto.messageFromTableRow( - schemaInformation, - descriptor, - merged, - ignoreUnknownValues, - false, - null, - cdcType, - sequence); + Preconditions.checkArgumentNotNull( + TableRowToStorageApiProto.messageFromTableRow( + schemaInformation, + descriptor, + merged, + ignoreUnknownValues, + false, + null, + cdcType, + sequence, + ErrorCollector.DONT_COLLECT)); return dynamicMessage.toByteString(); } private static @Nullable Object messageValueFromFieldValue( SchemaInformation schemaInformation, - FieldDescriptor fieldDescriptor, + @Nullable + FieldDescriptor + fieldDescriptor, // Null in the case of recursively finding missing fields. @Nullable Object bqValue, boolean ignoreUnknownValues, boolean allowMissingRequiredFields, - Supplier<@Nullable TableRow> getUnknownNestedFields) + Supplier<@Nullable TableRow> getUnknownNestedFields, + ErrorCollector collectedExceptions) throws SchemaConversionException { + boolean isOptional = fieldDescriptor == null || fieldDescriptor.isOptional(); + boolean isRepeated = fieldDescriptor != null && fieldDescriptor.isRepeated(); if (bqValue == null) { - if (fieldDescriptor.isOptional() || allowMissingRequiredFields) { + if (isOptional || allowMissingRequiredFields) { return null; - } else if (fieldDescriptor.isRepeated()) { + } else if (isRepeated) { return Collections.emptyList(); } else { - // TODO: Allow expanding this! - throw new SchemaDoesntMatchException( - "Received null value for non-nullable field " + schemaInformation.getFullName()); + SchemaConversionException exception = + new SchemaMissingRequiredFieldException( + ImmutableSet.of(schemaInformation.getFullName())); + collectedExceptions.collect(exception); // Throws if not collected. + return null; } } - if (fieldDescriptor.isRepeated()) { - List listValue = (List) bqValue; + if (isRepeated) { + List<@Nullable Object> listValue = + Preconditions.checkArgumentNotNull((List<@Nullable Object>) bqValue); List<@Nullable Object> protoList = Lists.newArrayListWithCapacity(listValue.size()); for (@Nullable Object o : listValue) { if (o != null) { // repeated field cannot contain null. @@ -1320,7 +1578,8 @@ public static ByteString mergeNewFields( o, ignoreUnknownValues, allowMissingRequiredFields, - getUnknownNestedFields)); + getUnknownNestedFields, + collectedExceptions)); } } return protoList; @@ -1331,17 +1590,19 @@ public static ByteString mergeNewFields( Preconditions.checkStateNotNull(bqValue), ignoreUnknownValues, allowMissingRequiredFields, - getUnknownNestedFields); + getUnknownNestedFields, + collectedExceptions); } @VisibleForTesting static @Nullable Object singularFieldToProtoValue( SchemaInformation schemaInformation, - FieldDescriptor fieldDescriptor, + @Nullable FieldDescriptor fieldDescriptor, Object value, boolean ignoreUnknownValues, boolean allowMissingRequiredFields, - Supplier<@Nullable TableRow> getUnknownNestedFields) + Supplier<@Nullable TableRow> getUnknownNestedFields, + ErrorCollector collectedExceptions) throws SchemaConversionException { @Nullable Object converted = null; if (schemaInformation.getType() == TableFieldSchema.Type.STRUCT) { @@ -1350,26 +1611,31 @@ public static ByteString mergeNewFields( converted = messageFromTableRow( schemaInformation, - fieldDescriptor.getMessageType(), + fieldDescriptor == null ? null : fieldDescriptor.getMessageType(), tableRow, ignoreUnknownValues, allowMissingRequiredFields, getUnknownNestedFields.get(), null, - null); + null, + collectedExceptions); } else if (value instanceof AbstractMap) { // This will handle nested rows. AbstractMap map = ((AbstractMap) value); converted = messageFromMap( schemaInformation, - fieldDescriptor.getMessageType(), + fieldDescriptor == null ? null : fieldDescriptor.getMessageType(), map, ignoreUnknownValues, allowMissingRequiredFields, getUnknownNestedFields.get(), null, - null); + null, + collectedExceptions); + if (!collectedExceptions.isEmpty()) { + return null; + } } } else if (schemaInformation.getType() == TableFieldSchema.Type.TIMESTAMP && schemaInformation.getTimestampPrecision() == PICOSECOND_PRECISION) { @@ -1394,11 +1660,11 @@ public static ByteString mergeNewFields( "Unsupported timestamp value type: " + value.getClass().getName()); } + Descriptor messageType = Preconditions.checkArgumentNotNull(fieldDescriptor).getMessageType(); converted = - DynamicMessage.newBuilder(fieldDescriptor.getMessageType()) - .setField(fieldDescriptor.getMessageType().findFieldByName("seconds"), seconds) - .setField( - fieldDescriptor.getMessageType().findFieldByName("picoseconds"), picoseconds) + DynamicMessage.newBuilder(messageType) + .setField(messageType.findFieldByName("seconds"), seconds) + .setField(messageType.findFieldByName("picoseconds"), picoseconds) .build(); } else { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableSchemaCache.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableSchemaCache.java index 1e80a91824ad..554a67e119dc 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableSchemaCache.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableSchemaCache.java @@ -30,13 +30,14 @@ import java.util.concurrent.Executors; import java.util.function.Consumer; import java.util.function.Supplier; -import javax.annotation.Nullable; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; +import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Monitor; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Monitor.Guard; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.MoreExecutors; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; @@ -59,10 +60,19 @@ static SchemaHolder of(TableSchema tableSchema, int version) { abstract static class Refresh { abstract DatasetService getDatasetService(); + abstract BigQueryServices.WriteStreamService getWriteStreamService(); + + abstract BigQueryOptions getOptions(); + abstract int getTargetVersion(); - static Refresh of(DatasetService datasetService, int targetVersion) { - return new AutoValue_TableSchemaCache_Refresh(datasetService, targetVersion); + static Refresh of( + DatasetService datasetService, + BigQueryServices.WriteStreamService writeStreamService, + BigQueryOptions options, + int targetVersion) { + return new AutoValue_TableSchemaCache_Refresh( + datasetService, writeStreamService, options, targetVersion); } } @@ -156,19 +166,18 @@ private static String tableKey(TableReference tableReference) { return BigQueryHelpers.stripPartitionDecorator(BigQueryHelpers.toTableSpec(tableReference)); } - @Nullable - public TableSchema getSchema(TableReference tableReference, DatasetService datasetService) { + public @Nullable TableSchema getSchema( + TableReference tableReference, DatasetService datasetService) { Optional schemaHolder; // We don't use computeIfAbsent here, as we want to avoid calling into datasetService (which can - // be an RPC - // with the monitor locked). + // be an RPC with the monitor locked). final String key = tableKey(tableReference); schemaHolder = runUnderMonitor(() -> Optional.ofNullable(cachedSchemas.get(key))); if (!schemaHolder.isPresent()) { // Not initialized. Query the new schema with the monitor released and then update the cache. try { // requesting the BASIC view will prevent BQ backend to run calculations - // related with storage stats that are not needed here + // related with storage stats that are not needed here. @Nullable Table table = datasetService.getTable( @@ -189,8 +198,8 @@ public TableSchema getSchema(TableReference tableReference, DatasetService datas * Registers schema for a table if one is not already present. If a schema is already in the * cache, returns the existing schema, otherwise returns null. */ - @Nullable - public TableSchema putSchemaIfAbsent(TableReference tableReference, TableSchema tableSchema) { + public @Nullable TableSchema putSchemaIfAbsent( + TableReference tableReference, TableSchema tableSchema) { final String key = tableKey(tableReference); Optional existing = runUnderMonitor( @@ -200,7 +209,11 @@ public TableSchema putSchemaIfAbsent(TableReference tableReference, TableSchema return existing.map(SchemaHolder::getTableSchema).orElse(null); } - public void refreshSchema(TableReference tableReference, DatasetService datasetService) { + public void refreshSchema( + TableReference tableReference, + DatasetService datasetService, + BigQueryServices.WriteStreamService writeStreamService, + BigQueryOptions options) { int targetVersion = runUnderMonitor( () -> { @@ -211,9 +224,12 @@ public void refreshSchema(TableReference tableReference, DatasetService datasetS String key = tableKey(tableReference); @Nullable SchemaHolder schemaHolder = cachedSchemas.get(key); int nextVersion = schemaHolder != null ? schemaHolder.getVersion() + 1 : 0; - tablesToRefresh.put(key, Refresh.of(datasetService, nextVersion)); + @Nullable + Refresh existing = + tablesToRefresh.putIfAbsent( + key, Refresh.of(datasetService, writeStreamService, options, nextVersion)); // Wait at least until the next version. - return nextVersion; + return (existing == null) ? nextVersion : existing.getTargetVersion(); }); waitForRefresh(tableReference, targetVersion); } @@ -267,19 +283,31 @@ public void refreshThread() { } // Query all the tables for their schema. - final Map schemas = refreshAll(localTablesToRefresh); + final Map schemas = refreshAll(localTablesToRefresh); runUnderMonitor( () -> { // Update the cache schemas. - for (Map.Entry entry : schemas.entrySet()) { + for (Map.Entry entry : schemas.entrySet()) { SchemaHolder schemaHolder = cachedSchemas.get(entry.getKey()); if (schemaHolder == null) { throw new RuntimeException("Unexpected null schema for " + entry.getKey()); } - SchemaHolder newSchema = - SchemaHolder.of(entry.getValue(), schemaHolder.getVersion() + 1); - cachedSchemas.put(entry.getKey(), newSchema); + + if (entry.getValue() == null) { + // There was an error fetching the schema. Reschedule it. + Refresh oldRefresh = + Preconditions.checkStateNotNull(localTablesToRefresh.get(entry.getKey())); + Refresh existingRefresh = this.tablesToRefresh.get(entry.getKey()); + if (existingRefresh == null + || oldRefresh.getTargetVersion() > existingRefresh.getTargetVersion()) { + this.tablesToRefresh.put(entry.getKey(), oldRefresh); + } + } else { + SchemaHolder newSchema = + SchemaHolder.of(entry.getValue(), schemaHolder.getVersion() + 1); + cachedSchemas.put(entry.getKey(), newSchema); + } } }); @@ -299,22 +327,51 @@ public void refreshThread() { this.refreshExecutor.submit(this::refreshThread); } - private Map refreshAll(Map tables) + private @Nullable TableSchema optimizedGetSchema( + String tableSpec, + DatasetService datasetService, + BigQueryServices.WriteStreamService writeStreamService, + BigQueryOptions options) throws IOException, InterruptedException { - Map schemas = Maps.newHashMapWithExpectedSize(tables.size()); - for (Map.Entry entry : tables.entrySet()) { - TableReference tableReference = BigQueryHelpers.parseTableSpec(entry.getKey()); + // Instead of querying BT metadata for the table schema, we first query Vortex. Vortex has much + // higher quotas for + // querying, however the result may be slightly out of date (usually seconds). + TableDestination tableDestination = new TableDestination(tableSpec, null); + String defaultStreamName = + BigQueryHelpers.stripPartitionDecorator( + tableDestination.getTableUrn(options) + "/streams/_default"); + com.google.cloud.bigquery.storage.v1.TableSchema schema = + writeStreamService.getWriteStreamSchema(defaultStreamName); + if (schema != null) { + return TableRowToStorageApiProto.protoSchemaToTableSchema(schema); + } else { + TableReference tableReference = BigQueryHelpers.parseTableSpec(tableSpec); Table table = - entry - .getValue() - .getDatasetService() - .getTable( - tableReference, Collections.emptyList(), DatasetService.TableMetadataView.BASIC); - if (table == null) { - throw new RuntimeException("Did not get value for table " + tableReference); + datasetService.getTable( + tableReference, Collections.emptyList(), DatasetService.TableMetadataView.BASIC); + return (table == null) ? null : table.getSchema(); + } + } + + private Map refreshAll(Map tables) + throws IOException, InterruptedException { + Map schemas = Maps.newHashMapWithExpectedSize(tables.size()); + for (Map.Entry entry : tables.entrySet()) { + Refresh refresh = entry.getValue(); + @Nullable + TableSchema tableSchema = + optimizedGetSchema( + entry.getKey(), + refresh.getDatasetService(), + refresh.getWriteStreamService(), + refresh.getOptions()); + if (tableSchema == null) { + LOG.info("Did not get a value for table {}", entry.getKey()); + schemas.put(entry.getKey(), null); + } else { + LOG.info("Refreshed BigQuery schema for {}", entry.getKey()); + schemas.put(entry.getKey(), tableSchema); } - LOG.info("Refreshed BigQuery schema for {}", entry.getKey()); - schemas.put(entry.getKey(), table.getSchema()); } return schemas; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TwoLevelMessageConverterCache.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TwoLevelMessageConverterCache.java index 0ce7c7573c9c..8d8ff22b521c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TwoLevelMessageConverterCache.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TwoLevelMessageConverterCache.java @@ -20,6 +20,7 @@ import java.io.Serializable; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; import org.apache.beam.sdk.io.gcp.bigquery.StorageApiDynamicDestinations.MessageConverter; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.ShardedKey; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache; @@ -61,7 +62,9 @@ static void clear() { public MessageConverter get( DestinationT destination, StorageApiDynamicDestinations dynamicDestinations, - DatasetService datasetService) + PipelineOptions pipelineOptions, + DatasetService datasetService, + BigQueryServices.WriteStreamService writeStreamService) throws Exception { // Lookup first in the local cache, and fall back to the static cache if necessary. return localMessageConverters.get( @@ -70,7 +73,9 @@ public MessageConverter get( (MessageConverter) CACHED_MESSAGE_CONVERTERS.get( KV.of(operationName, destination), - () -> dynamicDestinations.getMessageConverter(destination, datasetService))); + () -> + dynamicDestinations.getMessageConverter( + destination, pipelineOptions, datasetService, writeStreamService))); } public KV> getAppendClientKey(ShardedKey shardedKey) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/UpgradeTableSchema.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/UpgradeTableSchema.java new file mode 100644 index 000000000000..120492d2ebdd --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/UpgradeTableSchema.java @@ -0,0 +1,308 @@ +/* + * 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.gcp.bigquery; +/* + * 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. + */ + +import com.google.cloud.bigquery.storage.v1.TableFieldSchema; +import com.google.cloud.bigquery.storage.v1.TableSchema; +import com.google.protobuf.Descriptors; +import com.google.protobuf.DynamicMessage; +import com.google.protobuf.Message; +import com.google.protobuf.UnknownFieldSet; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.beam.sdk.util.ThrowingSupplier; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.hash.HashCode; +import org.checkerframework.checker.nullness.qual.Nullable; + +public class UpgradeTableSchema { + public static TableRowToStorageApiProto.ErrorCollector newErrorCollector() { + return new TableRowToStorageApiProto.ErrorCollector( + e -> + (e instanceof TableRowToStorageApiProto.SchemaTooNarrowException + && !((TableRowToStorageApiProto.SchemaTooNarrowException) e) + .getMissingField() + .isEmpty()) + || e instanceof TableRowToStorageApiProto.SchemaMissingRequiredFieldException); + } + + public static TableSchema getIncrementalSchema( + TableRowToStorageApiProto.ErrorCollector errorCollector, TableSchema oldSchema) + throws TableRowToStorageApiProto.SchemaDoesntMatchException { + // This isn't the most efficient, especially if we have deeply-nested schemas. However we don't + // expect to be + // upgrading schemas very regularly - if we do then we have other problems! + Map> newFields = Maps.newHashMap(); + Map> relaxedFields = Maps.newHashMap(); + + for (TableRowToStorageApiProto.SchemaConversionException schemaConversionException : + errorCollector.getExceptions()) { + if (schemaConversionException instanceof TableRowToStorageApiProto.SchemaTooNarrowException) { + TableRowToStorageApiProto.SchemaTooNarrowException e = + (TableRowToStorageApiProto.SchemaTooNarrowException) schemaConversionException; + List components = Arrays.asList(e.getMissingField().toLowerCase().split("\\.")); + String prefix = String.join(".", components.subList(0, components.size() - 1)); + String name = components.get(components.size() - 1); + TableFieldSchema.Mode mode = + e.isRepeated() ? TableFieldSchema.Mode.REPEATED : TableFieldSchema.Mode.NULLABLE; + TableFieldSchema.Type type = + e.isStruct() ? TableFieldSchema.Type.STRUCT : TableFieldSchema.Type.STRING; + @Nullable + TableFieldSchema oldValue = + newFields + .computeIfAbsent(prefix, p -> Maps.newLinkedHashMap()) + .put( + name, + TableFieldSchema.newBuilder() + .setName(name) + .setMode(mode) + .setType(type) + .build()); + if (oldValue != null) { + // Duplicates are ok because we might run this over an entire bundle. However we must + // ensure that they are compatible. + if (!oldValue.getType().equals(type)) { + throw new TableRowToStorageApiProto.SchemaDoesntMatchException( + "Inconsistent types seen for field: " + e.getMissingField()); + } + } + } else if (schemaConversionException + instanceof TableRowToStorageApiProto.SchemaMissingRequiredFieldException) { + ((TableRowToStorageApiProto.SchemaMissingRequiredFieldException) schemaConversionException) + .getMissingFields() + .forEach( + f -> { + List components = Arrays.asList(f.toLowerCase().split("\\.")); + String prefix = String.join(".", components.subList(0, components.size() - 1)); + String name = components.get(components.size() - 1); + relaxedFields.computeIfAbsent(prefix, p -> Sets.newHashSet()).add(name); + }); + } else { + throw new RuntimeException( + "Unexpected error " + schemaConversionException, schemaConversionException); + } + } + return TableSchema.newBuilder() + .addAllFields( + getIncrementalSchemaHelper(newFields, relaxedFields, oldSchema.getFieldsList(), "")) + .build(); + } + + private static List getIncrementalSchemaHelper( + Map> newFields, + Map> relaxedFields, + List tableFields, + String prefix) { + List fields = Lists.newArrayList(); + + Set fieldsToRelax = relaxedFields.getOrDefault(prefix, Collections.emptySet()); + // Add existing fields in the same order. + for (TableFieldSchema fieldSchema : tableFields) { + String fieldName = fieldSchema.getName().toLowerCase(); + TableFieldSchema.Builder clonedField = null; + if (fieldsToRelax.contains(fieldName)) { + // Since we're only generating the incremental schema, existing fields are only examined if + // they change - + // i.e. they are relaxed. + clonedField = fieldSchema.toBuilder(); + clonedField.setMode(TableFieldSchema.Mode.NULLABLE); + } + if (fieldSchema.getType().equals(TableFieldSchema.Type.STRUCT)) { + // Recursively walk the schema, looking for more field relaxations. + String newPrefix = prefix.isEmpty() ? fieldName : String.join(".", prefix, fieldName); + List newSubfields = + getIncrementalSchemaHelper( + newFields, relaxedFields, fieldSchema.getFieldsList(), newPrefix); + if (!newSubfields.isEmpty()) { + if (clonedField == null) { + clonedField = fieldSchema.toBuilder(); + } + clonedField.clearFields(); + clonedField.addAllFields(newSubfields); + } + } + if (clonedField != null) { + fields.add(clonedField.build()); + } + } + + LinkedHashMap fieldsToAdd = + newFields.getOrDefault(prefix, new LinkedHashMap<>()); + for (Map.Entry entry : fieldsToAdd.entrySet()) { + TableFieldSchema.Builder field = entry.getValue().toBuilder(); + // We rely on the exception telling us intermediate struct fields. + if (field.getType().equals(TableFieldSchema.Type.STRUCT)) { + String fieldName = field.getName().toLowerCase(); + String newPrefix = prefix.isEmpty() ? fieldName : String.join(".", prefix, fieldName); + field.addAllFields( + getIncrementalSchemaHelper( + newFields, relaxedFields, Collections.emptyList(), newPrefix)); + } + fields.add(field.build()); + } + + return fields; + } + + // Merge two schemas. schema1 is considered the primary schema, and will control what order + // overlapping fields + // are created in the final schema. + public static TableSchema mergeSchemas(TableSchema schema1, TableSchema schema2) { + List mergedFields = + mergeFields(schema1.getFieldsList(), schema2.getFieldsList()); + return TableSchema.newBuilder().addAllFields(mergedFields).build(); + } + + private static List mergeFields( + List fields1, List fields2) { + // Use LinkedHashMap to preserve the order of fields + Map mergedFieldsMap = Maps.newLinkedHashMap(); + + // Add all fields from schema 1. + fields1.forEach(f -> mergedFieldsMap.put(f.getName().toLowerCase(), f)); + + // Merge or append fields from schema 2 + for (TableFieldSchema f2 : fields2) { + String lowerName = f2.getName().toLowerCase(); + mergedFieldsMap.compute(lowerName, (k, v) -> v == null ? f2 : mergeField(v, f2)); + } + return Lists.newArrayList(mergedFieldsMap.values()); + } + + private static TableFieldSchema mergeField(TableFieldSchema f1, TableFieldSchema f2) { + if (!f1.getType().equals(f2.getType())) { + throw new IllegalArgumentException( + String.format( + "Conflicting field types for field '%s': %s vs %s", + f1.getName(), f1.getType(), f2.getType())); + } + + TableFieldSchema.Builder builder = f1.toBuilder().mergeFrom(f2); + builder.clearFields(); + + // Handle mode weakening (REPEATED > NULLABLE > REQUIRED) + TableFieldSchema.Mode mode1 = + f1.getMode() == TableFieldSchema.Mode.MODE_UNSPECIFIED + ? TableFieldSchema.Mode.NULLABLE + : f1.getMode(); + TableFieldSchema.Mode mode2 = + f2.getMode() == TableFieldSchema.Mode.MODE_UNSPECIFIED + ? TableFieldSchema.Mode.NULLABLE + : f2.getMode(); + + if (mode1 == TableFieldSchema.Mode.REPEATED || mode2 == TableFieldSchema.Mode.REPEATED) { + // Any field can be relaxed into a repeated field. + builder.setMode(TableFieldSchema.Mode.REPEATED); + } else if (mode1 == TableFieldSchema.Mode.NULLABLE || mode2 == TableFieldSchema.Mode.NULLABLE) { + // Any field can be relaxed into a nullable field. + builder.setMode(TableFieldSchema.Mode.NULLABLE); + } else { + builder.setMode(TableFieldSchema.Mode.REQUIRED); + } + + // Recursively merge nested fields if the type is STRUCT (Record) + if (f1.getType() == TableFieldSchema.Type.STRUCT) { + builder.addAllFields(mergeFields(f1.getFieldsList(), f2.getFieldsList())); + } + + return builder.build(); + } + + public static boolean isPayloadSchemaOutOfDate( + StorageApiWritePayload payload, + ThrowingSupplier schemaHash, + ThrowingSupplier schemaDescriptor) + throws Exception { + byte @Nullable [] payloadSchemaHash = payload.getSchemaHash(); + if (payloadSchemaHash != null) { + // Schema hash is only included in the payload if schema update options are set. + HashCode lhs = HashCode.fromBytes(payloadSchemaHash); + HashCode rhs = HashCode.fromBytes(schemaHash.get()); + if (!lhs.equals(rhs)) { + DynamicMessage msg = + DynamicMessage.newBuilder(schemaDescriptor.get()) + .mergeFrom(payload.getPayload()) + .buildPartial(); + return !msg.isInitialized() || hasUnknownFields(msg); + } + } + return false; + } + + private static boolean hasUnknownFields(Message message) { + if (message == null) { + return false; + } + + // 1. Check if the current message level has any unknown fields + UnknownFieldSet unknownFields = message.getUnknownFields(); + if (unknownFields != null && !unknownFields.asMap().isEmpty()) { + return true; + } + + // 2. Iterate through all populated fields in the current message + for (Map.Entry entry : message.getAllFields().entrySet()) { + Descriptors.FieldDescriptor fieldDescriptor = entry.getKey(); + Object value = entry.getValue(); + + // We only care about nested messages, as scalar types don't have sub-fields + if (fieldDescriptor.getJavaType() == Descriptors.FieldDescriptor.JavaType.MESSAGE) { + if (fieldDescriptor.isRepeated()) { + // For repeated message fields, recursively check each element in the list + Iterable repeatedMessages = (Iterable) value; + for (Object element : repeatedMessages) { + if (element != null && hasUnknownFields((Message) element)) { + return true; + } + } + } else { + // For singular message fields, recursively check the nested message + if (value != null && hasUnknownFields((Message) value)) { + return true; + } + } + } + } + + // If we reach here, neither this message nor its descendants have unknown fields + return false; + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java index 8d263a286f3e..c6872cfa181d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java @@ -29,6 +29,7 @@ import com.google.api.services.bigquery.model.DatasetReference; import com.google.api.services.bigquery.model.Table; import com.google.api.services.bigquery.model.TableDataInsertAllResponse; +import com.google.api.services.bigquery.model.TableFieldSchema; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; @@ -39,9 +40,11 @@ import com.google.cloud.bigquery.storage.v1.FinalizeWriteStreamResponse; import com.google.cloud.bigquery.storage.v1.FlushRowsResponse; import com.google.cloud.bigquery.storage.v1.ProtoRows; +import com.google.cloud.bigquery.storage.v1.StorageError; import com.google.cloud.bigquery.storage.v1.WriteStream; import com.google.errorprone.annotations.FormatMethod; import com.google.errorprone.annotations.FormatString; +import com.google.protobuf.Any; import com.google.protobuf.ByteString; import com.google.protobuf.DescriptorProtos; import com.google.protobuf.Descriptors; @@ -52,6 +55,7 @@ import io.grpc.Status; import java.io.IOException; import java.io.Serializable; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -59,6 +63,8 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers; @@ -97,6 +103,20 @@ public class FakeDatasetService implements DatasetService, WriteStreamService, S @Override public void close() throws Exception {} + private static Exceptions.StorageException getStorageException( + String streamName, StorageError.StorageErrorCode code, String errorMessage) { + StorageError storageError = + StorageError.newBuilder() + .setEntity(streamName) + .setCode(code) + .setErrorMessage(errorMessage) + .build(); + com.google.rpc.Status status = + com.google.rpc.Status.newBuilder().addDetails(Any.pack(storageError)).build(); + return org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull( + Exceptions.toStorageException(status, null)); + } + static class Stream { static class Entry { enum UpdateType { @@ -144,10 +164,11 @@ TableSchema getUpdatedSchema() { } WriteStream toWriteStream() { + TableSchema tableSchema = updatedSchema != null ? updatedSchema : currentSchema; return WriteStream.newBuilder() .setName(streamName) .setType(type) - .setTableSchema(TableRowToStorageApiProto.schemaToProtoTableSchema(currentSchema)) + .setTableSchema(TableRowToStorageApiProto.schemaToProtoTableSchema(tableSchema)) .build(); } @@ -156,23 +177,28 @@ long finalizeStream() { return stream.size(); } - void appendRows(long position, List rowsToAppend) { + @Nullable + Exceptions.StorageException appendRows(long position, List rowsToAppend) { if (finalized) { - throw new RuntimeException("Stream already finalized."); + return getStorageException( + streamName, StorageError.StorageErrorCode.STREAM_FINALIZED, "Stream finalized"); } + if (position != -1 && position != stream.size()) { - throw new RuntimeException( - "Bad append: " - + position - + " + for stream " - + streamName - + " expected " - + stream.size()); + String errorMessage = + String.format("expected offset %d, received %d", stream.size(), position); + StorageError.StorageErrorCode code = + position > stream.size() + ? StorageError.StorageErrorCode.OFFSET_OUT_OF_RANGE + : StorageError.StorageErrorCode.OFFSET_ALREADY_EXISTS; + return getStorageException(streamName, code, errorMessage); } + stream.addAll(rowsToAppend); if (type == WriteStream.Type.COMMITTED) { rowsToAppend.forEach(this::applyEntry); } + return null; } void flush(long position) { @@ -268,6 +294,12 @@ public Table getTableImpl( } } + public List getAllRows(TableReference tableReference) + throws InterruptedException, IOException { + return getAllRows( + tableReference.getProjectId(), tableReference.getDatasetId(), tableReference.getTableId()); + } + public List getAllRows(String projectId, String datasetId, String tableId) throws InterruptedException, IOException { synchronized (FakeDatasetService.class) { @@ -378,7 +410,7 @@ public void updateTableSchema(TableReference tableReference, TableSchema tableSc if (tableContainer == null) { throwNotFound("Tried to get a table %s, but no such table existed", tableReference); } - // TODO: Only allow "legal" schema changes. + checkSchemaChanges(tableContainer.table.getSchema().getFields(), tableSchema.getFields()); tableContainer.table.setSchema(tableSchema); for (Stream stream : writeStreams.values()) { @@ -389,6 +421,100 @@ public void updateTableSchema(TableReference tableReference, TableSchema tableSc } } + private void checkSchemaChanges( + List oldSchema, List newSchema) { + List oldSchemaProtos = + oldSchema.stream() + .map(TableRowToStorageApiProto::tableFieldToProtoTableField) + .collect(Collectors.toList()); + + List newSchemaProtos = + newSchema.stream() + .map(TableRowToStorageApiProto::tableFieldToProtoTableField) + .collect(Collectors.toList()); + checkSchemaChangesProtos(oldSchemaProtos, newSchemaProtos); + } + + private void checkSchemaChangesProtos( + List oldSchemaProtos, + List newSchemaProtos) { + // Convert new schema to a map for easy name-based lookup + Map newFields = + newSchemaProtos.stream() + .collect(Collectors.toMap(t -> t.getName().toLowerCase(), Function.identity())); + Map newFieldIndices = + IntStream.range(0, newSchemaProtos.size()) + .boxed() + .collect(Collectors.toMap(i -> newSchemaProtos.get(i).getName().toLowerCase(), i -> i)); + + Map oldFields = + oldSchemaProtos.stream() + .collect(Collectors.toMap(t -> t.getName().toLowerCase(), Function.identity())); + + for (com.google.cloud.bigquery.storage.v1.TableFieldSchema oldField : oldSchemaProtos) { + com.google.cloud.bigquery.storage.v1.TableFieldSchema newField = + newFields.get(oldField.getName().toLowerCase()); + // 1. Check that no fields were removed + Preconditions.checkArgument(newField != null, "Cannot remove field %s", oldField.getName()); + + // 2. Check that the types match + Preconditions.checkArgument( + oldField.getType().equals(newField.getType()), + "Field type cannot change for field %s: %s to %s", + oldField.getName(), + oldField.getType(), + newField.getType()); + + com.google.cloud.bigquery.storage.v1.TableFieldSchema.Mode oldMode = oldField.getMode(); + com.google.cloud.bigquery.storage.v1.TableFieldSchema.Mode newMode = newField.getMode(); + + // 3. Check that the mode only changes if relaxing from REQUIRED to NULLABLE + if (oldMode.equals(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Mode.REQUIRED)) { + Preconditions.checkArgument( + newMode.equals(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Mode.REQUIRED) + || newMode.equals( + com.google.cloud.bigquery.storage.v1.TableFieldSchema.Mode.NULLABLE), + "Cannot change mode of %s from REQUIRED to %s", + oldField.getName(), + newMode); + } else { + Preconditions.checkArgument( + oldMode.equals(newMode), + "Cannot change mode of %s from %s to %s", + oldField.getName(), + oldMode, + newMode); + } + + // 4. Recursively check nested schema fields + if (oldField.getFieldsCount() > 0) { + Preconditions.checkArgument( + newField.getFieldsCount() >= oldField.getFieldsCount(), "Cannot remove nested fields"); + checkSchemaChangesProtos(oldField.getFieldsList(), newField.getFieldsList()); + } + } + + for (com.google.cloud.bigquery.storage.v1.TableFieldSchema newField : newSchemaProtos) { + if (oldFields.containsKey(newField.getName().toLowerCase())) { + // We've already checked this above. + continue; + } + + int newIndex = newFieldIndices.get(newField.getName().toLowerCase()); + Preconditions.checkArgument( + newIndex >= oldFields.size(), "New fields can only be added at the end of a schema."); + Preconditions.checkArgument( + !newField + .getMode() + .equals(com.google.cloud.bigquery.storage.v1.TableFieldSchema.Mode.REQUIRED), + "New field cannot be required"); + if (newField.getFieldsCount() > 0) { + // Recurse only on the new fields. + checkSchemaChangesProtos(Collections.emptyList(), newField.getFieldsList()); + } + } + } + public void setPrimaryKey(TableReference tableReference, List columns) throws IOException { validateWholeTableReference(tableReference); @@ -565,6 +691,20 @@ public Table patchTableDescription( } } + @Override + public Table patchTableSchema(TableReference tableReference, TableSchema newSchema) + throws IOException, InterruptedException { + updateTableSchema(tableReference, newSchema); + synchronized (FakeDatasetService.class) { + TableContainer tableContainer = + getTableContainer( + tableReference.getProjectId(), + tableReference.getDatasetId(), + tableReference.getTableId()); + return tableContainer.getTable(); + } + } + @Override public WriteStream createWriteStream(String tableUrn, WriteStream.Type type) throws InterruptedException { @@ -617,30 +757,49 @@ public StreamAppendClient getStreamAppendClient( private boolean usedForInsert = false; private boolean usedForUpdate = false; - { - this.protoDescriptor = TableRowToStorageApiProto.wrapDescriptorProto(descriptor); - + @Nullable + Exceptions.StorageException tryInitialize() throws Exception { synchronized (FakeDatasetService.class) { - Stream stream = writeStreams.get(streamName); - if (stream == null) { - // TODO(relax): Return the exact error that BigQuery returns. - throw new ApiException(null, GrpcStatusCode.of(Status.Code.NOT_FOUND), false); + if (this.protoDescriptor == null) { + this.protoDescriptor = TableRowToStorageApiProto.wrapDescriptorProto(descriptor); + + Stream stream = writeStreams.get(streamName); + if (stream == null) { + return getStorageException( + streamName, StorageError.StorageErrorCode.STREAM_NOT_FOUND, "Stream not found"); + } + + // TODO: we should validate the descriptor against the table schema and ensure it + // matches. + + currentSchema = stream.tableContainer.getTable().getSchema(); + schemaInformation = + TableRowToStorageApiProto.SchemaInformation.fromTableSchema( + TableRowToStorageApiProto.schemaToProtoTableSchema(currentSchema)); } - currentSchema = stream.tableContainer.getTable().getSchema(); - schemaInformation = - TableRowToStorageApiProto.SchemaInformation.fromTableSchema( - TableRowToStorageApiProto.schemaToProtoTableSchema(currentSchema)); } + return null; } @Override public ApiFuture appendRows(long offset, ProtoRows rows) throws Exception { + // The BigQuery client returns stream-open errors when the first append is called, so we + // duplicate that here. + Exceptions.StorageException storageException = tryInitialize(); + if (storageException != null) { + return ApiFutures.immediateFailedFuture(storageException); + } + AppendRowsResponse.Builder responseBuilder = AppendRowsResponse.newBuilder(); synchronized (FakeDatasetService.class) { Stream stream = writeStreams.get(streamName); if (stream == null) { - throw new RuntimeException("No such stream: " + streamName); + return ApiFutures.immediateFailedFuture( + getStorageException( + streamName, + StorageError.StorageErrorCode.STREAM_NOT_FOUND, + "Stream not found")); } List streamEntries = Lists.newArrayListWithExpectedSize(rows.getSerializedRowsCount()); @@ -648,9 +807,6 @@ public ApiFuture appendRows(long offset, ProtoRows rows) for (int i = 0; i < rows.getSerializedRowsCount(); ++i) { ByteString bytes = rows.getSerializedRows(i); DynamicMessage msg = DynamicMessage.parseFrom(protoDescriptor, bytes); - if (msg.getUnknownFields() != null && !msg.getUnknownFields().asMap().isEmpty()) { - throw new RuntimeException("Unknown fields set in append! " + msg.getUnknownFields()); - } TableRow tableRow = TableRowToStorageApiProto.tableRowFromMessage( schemaInformation, @@ -695,7 +851,10 @@ public ApiFuture appendRows(long offset, ProtoRows rows) stream.streamName, rowIndexToErrorMessage)); } - stream.appendRows(offset, streamEntries); + @Nullable Exceptions.StorageException failure = stream.appendRows(offset, streamEntries); + if (failure != null) { + return ApiFutures.immediateFailedFuture(failure); + } if (stream.getUpdatedSchema() != null) { com.google.cloud.bigquery.storage.v1.TableSchema newSchema = TableRowToStorageApiProto.schemaToProtoTableSchema(stream.getUpdatedSchema()); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java index e5ad761993e7..94037e056476 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java @@ -91,6 +91,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ThreadLocalRandom; import java.util.function.Function; +import java.util.function.IntFunction; import java.util.function.LongFunction; import java.util.function.Predicate; import java.util.regex.Matcher; @@ -191,6 +192,7 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ArrayListMultimap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; @@ -2513,6 +2515,180 @@ public void updateTableSchemaTest(boolean useSet) throws Exception { Iterables.concat(expectedDroppedValues, expectedFullValues), TableRow.class))); } + @Test + public void testAutoPatchTableSchemaTest() throws Exception { + assumeTrue(useStreaming); + assumeTrue(useStorageApi); + + // Make sure that GroupIntoBatches does not buffer data. + p.getOptions().as(BigQueryOptions.class).setStorageApiAppendThresholdBytes(1); + p.getOptions().as(BigQueryOptions.class).setNumStorageWriteApiStreams(1); + p.getOptions().as(BigQueryOptions.class).setSchemaUpgradeBufferingShards(2); + + BigQueryIO.Write.Method method = + useStorageApiApproximate ? Method.STORAGE_API_AT_LEAST_ONCE : Method.STORAGE_WRITE_API; + p.enableAbandonedNodeEnforcement(false); + + TableReference tableRef = BigQueryHelpers.parseTableSpec("project-id:dataset-id.table"); + TableSchema tableSchema = + new TableSchema() + .setFields( + ImmutableList.of( + new TableFieldSchema().setName("number").setType("INT64"), + new TableFieldSchema().setName("name").setType("STRING"), + new TableFieldSchema().setName("req").setType("STRING").setMode("REQUIRED"))); + fakeDatasetService.createTable(new Table().setTableReference(tableRef).setSchema(tableSchema)); + + final int stride = 5; + Function getUpdatedSchema = + currentStride -> { + TableSchema tableSchemaUpdated = new TableSchema(); + tableSchemaUpdated.setFields( + Lists.newArrayList( + new TableFieldSchema().setName("number").setType("INT64").setMode("NULLABLE"), + new TableFieldSchema().setName("name").setType("STRING").setMode("NULLABLE"), + new TableFieldSchema().setName("req").setType("STRING").setMode("NULLABLE"), + new TableFieldSchema().setName("new1").setType("STRING").setMode("NULLABLE"), + new TableFieldSchema().setName("new2").setType("STRING").setMode("NULLABLE"))); + + if (currentStride >= 2) { + List nestedFields = + Lists.newArrayList( + new TableFieldSchema() + .setName("nested_field1") + .setType("STRING") + .setMode("NULLABLE"), + new TableFieldSchema() + .setName("nested_field2") + .setType("STRING") + .setMode("NULLABLE")); + if (currentStride >= 3) { + List doubleNestedFields = + Lists.newArrayList( + new TableFieldSchema() + .setName("double_nested_field1") + .setType("STRING") + .setMode("NULLABLE")); + nestedFields.add( + new TableFieldSchema() + .setName("double_nested") + .setType("STRUCT") + .setMode("NULLABLE") + .setFields(doubleNestedFields)); + + List repeatedNestedFields = + Lists.newArrayList( + new TableFieldSchema() + .setName("repeated_nested_field1") + .setType("STRING") + .setMode("NULLABLE"), + new TableFieldSchema() + .setName("repeated_nested_field2") + .setType("STRING") + .setMode("NULLABLE")); + + nestedFields.add( + new TableFieldSchema() + .setName("repeated_nested") + .setType("STRUCT") + .setMode("REPEATED") + .setFields(repeatedNestedFields)); + } + tableSchemaUpdated + .getFields() + .add( + new TableFieldSchema() + .setName("nested") + .setType("STRUCT") + .setMode("NULLABLE") + .setFields(nestedFields)); + } + return tableSchemaUpdated; + }; + + IntFunction getRow = + (IntFunction & Serializable) + (int i) -> { + TableRow row = new TableRow().set("name", "name" + i).set("number", Long.toString(i)); + if (i < stride) { + row = row.set("req", "foo"); + } else { + row = row.set("new1", "blah" + i); + row = row.set("new2", "baz" + i); + + if (i >= 2 * stride) { + TableRow nested = + new TableRow() + .set("nested_field1", "nested1" + i) + .set("nested_field2", "nested2" + i); + + if (i >= 3 * stride) { + TableRow doubleNested = + new TableRow().set("double_nested_field1", "double_nested1" + i); + nested = nested.set("double_nested", doubleNested); + + // Add a repeated struct to ensure that we capture this code path as well. + TableRow repeatedNested1 = + new TableRow().set("repeated_nested_field1", "repeated_nested1" + i); + TableRow repeatedNested2 = + new TableRow().set("repeated_nested_field2", "repeated_nested2" + i); + nested = + nested.set( + "repeated_nested", ImmutableList.of(repeatedNested1, repeatedNested2)); + } + row.set("nested", nested); + } + } + return row; + }; + + TestStream.Builder testStream = + TestStream.create(TableRowJsonCoder.of()).advanceWatermarkTo(new Instant(0)); + List expectedRows = Lists.newArrayList(); + for (int i = 0; i < 20; i += stride) { + for (int j = i; j < i + stride; ++j) { + TableRow tableRow = getRow.apply(j); + expectedRows.add(tableRow); + testStream = testStream.addElements(tableRow); + } + if (i > 0 && (i % 5) == 0) { + for (int n = 0; n < 5; ++n) { + testStream = testStream.advanceProcessingTime(Duration.standardSeconds(2)); + } + } + } + for (int i = 0; i < 5; ++i) { + testStream = testStream.advanceProcessingTime(Duration.standardSeconds(2)); + } + + PCollection tableRows = p.apply(testStream.advanceWatermarkToInfinity()); + + BigQueryIO.Write write = + BigQueryIO.writeTableRows() + .to(tableRef) + .withMethod(method) + .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_NEVER) + .withSchemaUpdateOptions( + ImmutableSet.of( + SchemaUpdateOption.ALLOW_FIELD_ADDITION, + SchemaUpdateOption.ALLOW_FIELD_RELAXATION)) + .withTestServices(fakeBqServices) + .withoutValidation(); + if (method == Method.STORAGE_WRITE_API) { + write = + write + .withTriggeringFrequency(Duration.standardSeconds(1)) + .withNumStorageWriteApiStreams(2); + } + tableRows.apply(write); + + p.run(); + assertEquals( + getUpdatedSchema.apply(Integer.MAX_VALUE), + fakeDatasetService.getTable(tableRef).getSchema()); + assertThat(fakeDatasetService.getAllRows(tableRef), containsInAnyOrder(expectedRows.toArray())); + } + TableRow filterUnknownValues(TableRow row, List tableSchemaFields) { Map schemaTypes = tableSchemaFields.stream() diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDataTriggeredSchemaUpdateIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDataTriggeredSchemaUpdateIT.java new file mode 100644 index 000000000000..738f1ab2416d --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDataTriggeredSchemaUpdateIT.java @@ -0,0 +1,338 @@ +/* + * 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.gcp.bigquery; + +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects.firstNonNull; +import static org.junit.Assert.assertEquals; + +import com.google.api.services.bigquery.model.Table; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.auto.value.AutoValue; +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; +import org.apache.beam.sdk.io.gcp.testing.BigqueryClient; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.WithKeys; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.joda.time.Duration; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestName; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Integration test for BigQueryIO Storage Write API auto schema upgrade triggered by data. + * + *

Uses a Stateful DoFn to sequence elements ensuring base schema elements are written before + * evolved schema elements, avoiding race conditions in distributed execution. + */ +@RunWith(JUnit4.class) +public class StorageApiDataTriggeredSchemaUpdateIT { + private static final Logger LOG = + LoggerFactory.getLogger(StorageApiDataTriggeredSchemaUpdateIT.class); + + private static final BigqueryClient BQ_CLIENT = + new BigqueryClient("StorageApiDataTriggeredSchemaUpdateIT"); + private static final String PROJECT = + TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject(); + private static final String BIG_QUERY_DATASET_ID = + "storage_api_data_triggered_schema_update_" + System.nanoTime(); + + private static String bigQueryLocation; + + @Rule public TestName testName = new TestName(); + + @BeforeClass + public static void setUpTestEnvironment() throws IOException, InterruptedException { + bigQueryLocation = + TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class).getBigQueryLocation(); + BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID, null, bigQueryLocation); + } + + @AfterClass + public static void cleanUp() { + LOG.info("Cleaning up dataset {} and tables.", BIG_QUERY_DATASET_ID); + BQ_CLIENT.deleteDataset(PROJECT, BIG_QUERY_DATASET_ID); + } + + private String createTable(TableSchema tableSchema) throws IOException, InterruptedException { + String tableId = testName.getMethodName().replace("[", "_").replace("]", "_"); + BQ_CLIENT.deleteTable(PROJECT, BIG_QUERY_DATASET_ID, tableId); + BQ_CLIENT.createNewTable( + PROJECT, + BIG_QUERY_DATASET_ID, + new Table() + .setSchema(tableSchema) + .setTableReference( + new TableReference() + .setTableId(tableId) + .setDatasetId(BIG_QUERY_DATASET_ID) + .setProjectId(PROJECT))); + return tableId; + } + + static class SequenceRowsDoFn extends DoFn, TableRow> { + private static final String COUNTER = "counter"; + + @StateId(COUNTER) + @SuppressWarnings("unused") + private final StateSpec> counterSpec = StateSpecs.value(); + + private final int stride; + + public SequenceRowsDoFn(int stride) { + this.stride = stride; + } + + @ProcessElement + public void processElement(ProcessContext c, @StateId(COUNTER) ValueState counter) { + int current = firstNonNull(counter.read(), 0); + c.output(getRow(current)); + counter.write(++current); + } + + TableRow getRow(int i) { + TableRow row = new TableRow().set("name", "name" + i).set("number", Long.toString(i)); + if (i < stride) { + row = row.set("req", "foo"); + } else { + row = row.set("new1", "blah" + i); + row = row.set("new2", "baz" + i); + + if (i >= 2 * stride) { + TableRow nested = + new TableRow() + .set("nested_field1", "nested1" + i) + .set("nested_field2", "nested2" + i); + + if (i >= 3 * stride) { + TableRow doubleNested = + new TableRow().set("double_nested_field1", "double_nested1" + i); + nested = nested.set("double_nested", doubleNested); + + // Add a repeated struct to ensure that we capture this code path as well. + TableRow repeatedNested1 = + new TableRow().set("repeated_nested_field1", "repeated_nested1" + i); + TableRow repeatedNested2 = + new TableRow().set("repeated_nested_field2", "repeated_nested2" + i); + nested = + nested.set("repeated_nested", ImmutableList.of(repeatedNested1, repeatedNested2)); + } + row.set("nested", nested); + } + } + return row; + }; + } + + @Test + public void testDataTriggeredSchemaUpgradeExactlyOnce() throws Exception { + runTest(Write.Method.STORAGE_WRITE_API); + } + + @Test + public void testDataTriggeredSchemaUpgradeAtLeastOnce() throws Exception { + runTest(Write.Method.STORAGE_API_AT_LEAST_ONCE); + } + + private void runTest(Write.Method method) throws Exception { + Pipeline p = Pipeline.create(TestPipeline.testingPipelineOptions()); + p.getOptions().as(BigQueryOptions.class).setSchemaUpgradeBufferingShards(1); + + TableSchema baseSchema = + new TableSchema() + .setFields( + ImmutableList.of( + new TableFieldSchema().setName("number").setType("INT64"), + new TableFieldSchema().setName("name").setType("STRING"), + new TableFieldSchema().setName("req").setType("STRING").setMode("REQUIRED"))); + TableSchema evolvedSchema = + new TableSchema() + .setFields( + ImmutableList.of( + new TableFieldSchema().setName("number").setType("INT64").setMode("NULLABLE"), + new TableFieldSchema().setName("name").setType("STRING").setMode("NULLABLE"), + new TableFieldSchema().setName("req").setType("STRING").setMode("NULLABLE"), + new TableFieldSchema().setName("new1").setType("STRING").setMode("NULLABLE"), + new TableFieldSchema().setName("new2").setType("STRING").setMode("NULLABLE"), + new TableFieldSchema() + .setName("nested") + .setType("STRUCT") + .setMode("NULLABLE") + .setFields( + ImmutableList.of( + new TableFieldSchema() + .setName("nested_field1") + .setType("STRING") + .setMode("NULLABLE"), + new TableFieldSchema() + .setName("nested_field2") + .setType("STRING") + .setMode("NULLABLE"), + new TableFieldSchema() + .setName("double_nested") + .setType("STRUCT") + .setMode("NULLABLE") + .setFields( + ImmutableList.of( + new TableFieldSchema() + .setName("double_nested_field1") + .setType("STRING") + .setMode("NULLABLE"))), + new TableFieldSchema() + .setName("repeated_nested") + .setType("STRUCT") + .setMode("REPEATED") + .setFields( + ImmutableList.of( + new TableFieldSchema() + .setName("repeated_nested_field1") + .setType("STRING") + .setMode("NULLABLE"), + new TableFieldSchema() + .setName("repeated_nested_field2") + .setType("STRING") + .setMode("NULLABLE"))))))); + + String tableId = createTable(baseSchema); + String tableSpec = PROJECT + ":" + BIG_QUERY_DATASET_ID + "." + tableId; + + List dummyInputs = IntStream.range(0, 20).boxed().collect(Collectors.toList()); + + BigQueryIO.Write write = + BigQueryIO.writeTableRows() + .to(tableSpec) + .withMethod(method) + .withSchemaUpdateOptions( + ImmutableSet.of( + Write.SchemaUpdateOption.ALLOW_FIELD_ADDITION, + Write.SchemaUpdateOption.ALLOW_FIELD_RELAXATION)) + .withCreateDisposition(CreateDisposition.CREATE_NEVER) + .withWriteDisposition(WriteDisposition.WRITE_APPEND); + if (method == Write.Method.STORAGE_WRITE_API) { + write = + write + .withTriggeringFrequency(Duration.standardSeconds(1)) + .withNumStorageWriteApiStreams(2); + } + + p.apply("Create Dummy Inputs", Create.of(dummyInputs)) + .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED) + .apply("Add a dummy key", WithKeys.of(1)) + .apply("Sequence Rows", ParDo.of(new SequenceRowsDoFn(5))) + .apply("Stream to BigQuery", write); + + p.run().waitUntilFinish(); + + List response = + BQ_CLIENT.queryUnflattened( + String.format("SELECT * FROM [%s] ORDER BY number", tableSpec), + PROJECT, + true, + false, + bigQueryLocation); + System.err.println("FULL RESULT " + response); + + // Verification + verifyTableSchemaUpdated(tableSpec, evolvedSchema); + List verifications = + ImmutableList.of( + new AutoValue_StorageApiDataTriggeredSchemaUpdateIT_VerificationInfo("", 20), + new AutoValue_StorageApiDataTriggeredSchemaUpdateIT_VerificationInfo("req IS NULL", 15), + new AutoValue_StorageApiDataTriggeredSchemaUpdateIT_VerificationInfo( + "new1 IS NOT NULL", 15), + new AutoValue_StorageApiDataTriggeredSchemaUpdateIT_VerificationInfo( + "new2 IS NOT NULL", 15), + new AutoValue_StorageApiDataTriggeredSchemaUpdateIT_VerificationInfo( + "nested.nested_field1 IS NOT NULL", 10), + new AutoValue_StorageApiDataTriggeredSchemaUpdateIT_VerificationInfo( + "nested.nested_field2 IS NOT NULL", 10), + new AutoValue_StorageApiDataTriggeredSchemaUpdateIT_VerificationInfo( + "nested.double_nested.double_nested_field1 IS NOT NULL", 5), + new AutoValue_StorageApiDataTriggeredSchemaUpdateIT_VerificationInfo( + "nested.repeated_nested.repeated_nested_field1 IS NOT NULL", 5), + new AutoValue_StorageApiDataTriggeredSchemaUpdateIT_VerificationInfo( + "nested.repeated_nested.repeated_nested_field2 IS NOT NULL", 5)); + verifyDataWritten(tableSpec, verifications); + } + + private void verifyTableSchemaUpdated(String tableSpec, TableSchema evolvedSchema) + throws IOException, InterruptedException { + Table table = + BQ_CLIENT.getTableResource( + PROJECT, + BIG_QUERY_DATASET_ID, + Iterables.getLast( + org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter.on('.') + .split(tableSpec))); + assertEquals( + TableRowToStorageApiProto.schemaToProtoTableSchema(evolvedSchema), + TableRowToStorageApiProto.schemaToProtoTableSchema(table.getSchema())); + } + + @AutoValue + abstract static class VerificationInfo { + abstract String getFilter(); + + abstract int getExpectedCount(); + } + + private void verifyDataWritten(String tableSpec, List verifications) + throws IOException, InterruptedException { + for (VerificationInfo verification : verifications) { + String format = + verification.getFilter().isEmpty() + ? String.format("SELECT COUNT(1) as total FROM [%s]", tableSpec) + : String.format( + "SELECT COUNT(1) as total FROM [%s] WHERE %s", + tableSpec, verification.getFilter()); + TableRow totalCountResponse = + Iterables.getOnlyElement( + BQ_CLIENT.queryUnflattened(format, PROJECT, true, false, bigQueryLocation)); + assertEquals( + "Unexpected result for query " + format, + verification.getExpectedCount(), + Integer.parseInt((String) totalCountResponse.get("total"))); + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoTest.java index 27f7b2cf5249..0469e8674fa0 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoTest.java @@ -1614,7 +1614,15 @@ public void testMessageFromTableRow() throws Exception { TableRowToStorageApiProto.SchemaInformation.fromTableSchema(NESTED_TABLE_SCHEMA); DynamicMessage msg = TableRowToStorageApiProto.messageFromTableRow( - schemaInformation, descriptor, tableRow, false, false, null, null, -1); + schemaInformation, + descriptor, + tableRow, + false, + false, + null, + null, + -1, + TableRowToStorageApiProto.ErrorCollector.DONT_COLLECT); assertEquals(4, msg.getAllFields().size()); @@ -1642,7 +1650,15 @@ public void testTableRowFromMessageNoF() throws Exception { TableRowToStorageApiProto.SchemaInformation.fromTableSchema(NESTED_TABLE_SCHEMA_NO_F); DynamicMessage msg = TableRowToStorageApiProto.messageFromTableRow( - schemaInformation, descriptor, tableRow, false, false, null, null, -1); + schemaInformation, + descriptor, + tableRow, + false, + false, + null, + null, + -1, + TableRowToStorageApiProto.ErrorCollector.DONT_COLLECT); TableRow recovered = TableRowToStorageApiProto.tableRowFromMessage( @@ -1690,7 +1706,15 @@ public void testTableRowFromMessageWithF() throws Exception { TableRowToStorageApiProto.SchemaInformation.fromTableSchema(nestedSchema); DynamicMessage msg = TableRowToStorageApiProto.messageFromTableRow( - schemaInformation, descriptor, tableRow, false, false, null, null, -1); + schemaInformation, + descriptor, + tableRow, + false, + false, + null, + null, + -1, + TableRowToStorageApiProto.ErrorCollector.DONT_COLLECT); TableRow recovered = TableRowToStorageApiProto.tableRowFromMessage( schemaInformation, msg, true, Predicates.alwaysTrue()); @@ -1722,7 +1746,15 @@ public void testTableRowFromMessageWithNestedArrayF() throws Exception { TableRowToStorageApiProto.SchemaInformation.fromTableSchema(nestedSchema); DynamicMessage msg = TableRowToStorageApiProto.messageFromTableRow( - schemaInformation, descriptor, tableRow, false, false, null, null, -1); + schemaInformation, + descriptor, + tableRow, + false, + false, + null, + null, + -1, + TableRowToStorageApiProto.ErrorCollector.DONT_COLLECT); TableRow recovered = TableRowToStorageApiProto.tableRowFromMessage( schemaInformation, msg, true, Predicates.alwaysTrue()); @@ -1738,7 +1770,15 @@ public void testMessageWithFFromTableRow() throws Exception { TableRowToStorageApiProto.SchemaInformation.fromTableSchema(BASE_TABLE_SCHEMA); DynamicMessage msg = TableRowToStorageApiProto.messageFromTableRow( - schemaInformation, descriptor, BASE_TABLE_ROW, false, false, null, null, -1); + schemaInformation, + descriptor, + BASE_TABLE_ROW, + false, + false, + null, + null, + -1, + TableRowToStorageApiProto.ErrorCollector.DONT_COLLECT); assertBaseRecord(msg, true); } @@ -1782,7 +1822,15 @@ public void testRepeatedDescriptorFromTableSchema() throws Exception { TableRowToStorageApiProto.SchemaInformation.fromTableSchema(REPEATED_MESSAGE_SCHEMA); DynamicMessage msg = TableRowToStorageApiProto.messageFromTableRow( - schemaInformation, descriptor, repeatedRow, false, false, null, null, -1); + schemaInformation, + descriptor, + repeatedRow, + false, + false, + null, + null, + -1, + TableRowToStorageApiProto.ErrorCollector.DONT_COLLECT); assertEquals(4, msg.getAllFields().size()); Map fieldDescriptors = @@ -1828,7 +1876,15 @@ public void testNullRepeatedDescriptorFromTableSchema() throws Exception { TableRowToStorageApiProto.SchemaInformation.fromTableSchema(REPEATED_MESSAGE_SCHEMA); DynamicMessage msg = TableRowToStorageApiProto.messageFromTableRow( - schemaInformation, descriptor, repeatedRow, false, false, null, null, -1); + schemaInformation, + descriptor, + repeatedRow, + false, + false, + null, + null, + -1, + TableRowToStorageApiProto.ErrorCollector.DONT_COLLECT); Map fieldDescriptors = descriptor.getFields().stream() @@ -1882,7 +1938,7 @@ public void testIntegerTypeConversion() throws DescriptorValidationException { try { Object converted = TableRowToStorageApiProto.singularFieldToProtoValue( - fieldSchema, fieldDescriptor, sourceValue, false, false, () -> null); + fieldSchema, fieldDescriptor, sourceValue, false, false, () -> null, null); assertEquals(expectedConvertedValue, converted); } catch (SchemaConversionException e) { fail( @@ -1928,7 +1984,7 @@ public void testIntegerTypeConversion() throws DescriptorValidationException { String expectedError = (String) invalidValue[1]; try { TableRowToStorageApiProto.singularFieldToProtoValue( - fieldSchema, fieldDescriptor, sourceValue, false, false, () -> null); + fieldSchema, fieldDescriptor, sourceValue, false, false, () -> null, null); fail( "Expected to throw an exception converting " + sourceValue @@ -1954,7 +2010,15 @@ public void testRejectUnknownField() throws Exception { thrown.expect(TableRowToStorageApiProto.SchemaConversionException.class); TableRowToStorageApiProto.messageFromTableRow( - schemaInformation, descriptor, row, false, false, null, null, -1); + schemaInformation, + descriptor, + row, + false, + false, + null, + null, + -1, + TableRowToStorageApiProto.ErrorCollector.DONT_COLLECT); } @Test @@ -1971,7 +2035,15 @@ public void testRejectUnknownFieldF() throws Exception { thrown.expect(TableRowToStorageApiProto.SchemaConversionException.class); TableRowToStorageApiProto.messageFromTableRow( - schemaInformation, descriptor, row, false, false, null, null, -1); + schemaInformation, + descriptor, + row, + false, + false, + null, + null, + -1, + TableRowToStorageApiProto.ErrorCollector.DONT_COLLECT); } @Test @@ -1989,7 +2061,15 @@ public void testRejectUnknownNestedField() throws Exception { thrown.expect(TableRowToStorageApiProto.SchemaConversionException.class); TableRowToStorageApiProto.messageFromTableRow( - schemaInformation, descriptor, topRow, false, false, null, null, -1); + schemaInformation, + descriptor, + topRow, + false, + false, + null, + null, + -1, + TableRowToStorageApiProto.ErrorCollector.DONT_COLLECT); } @Test @@ -2009,7 +2089,15 @@ public void testRejectUnknownNestedFieldF() throws Exception { thrown.expect(TableRowToStorageApiProto.SchemaConversionException.class); TableRowToStorageApiProto.messageFromTableRow( - schemaInformation, descriptor, topRow, false, false, null, null, -1); + schemaInformation, + descriptor, + topRow, + false, + false, + null, + null, + -1, + TableRowToStorageApiProto.ErrorCollector.DONT_COLLECT); } @Test @@ -2025,7 +2113,15 @@ public void testIgnoreUnknownField() throws Exception { TableRow ignored = new TableRow(); TableRowToStorageApiProto.messageFromTableRow( - schemaInformation, descriptor, row, true, false, ignored, null, -1); + schemaInformation, + descriptor, + row, + true, + false, + ignored, + null, + -1, + TableRowToStorageApiProto.ErrorCollector.DONT_COLLECT); assertEquals(1, ignored.size()); assertEquals("foobar", ignored.get("unknown")); } @@ -2044,7 +2140,15 @@ public void testIgnoreUnknownFieldF() throws Exception { TableRow ignored = new TableRow(); TableRowToStorageApiProto.messageFromTableRow( - schemaInformation, descriptor, row, true, false, ignored, null, -1); + schemaInformation, + descriptor, + row, + true, + false, + ignored, + null, + -1, + TableRowToStorageApiProto.ErrorCollector.DONT_COLLECT); assertEquals(BASE_TABLE_ROW.getF().size() + 1, ignored.getF().size()); assertEquals("foobar", ignored.getF().get(BASE_TABLE_ROW.getF().size()).getV()); } @@ -2075,7 +2179,15 @@ public void testIgnoreUnknownNestedField() throws Exception { TableRow unknown = new TableRow(); TableRowToStorageApiProto.messageFromTableRow( - schemaInformation, descriptor, topRow, true, false, unknown, null, -1); + schemaInformation, + descriptor, + topRow, + true, + false, + unknown, + null, + -1, + TableRowToStorageApiProto.ErrorCollector.DONT_COLLECT); assertEquals(3, unknown.size()); assertEquals("foobar", unknown.get("unknowntop")); assertEquals(1, ((TableRow) unknown.get("nestedvalue1")).size()); @@ -2122,7 +2234,15 @@ public void testIgnoreUnknownRepeatedNestedField() throws Exception { TableRow unknown = new TableRow(); TableRowToStorageApiProto.messageFromTableRow( - schemaInformation, descriptor, repeatedRow, true, false, unknown, null, -1); + schemaInformation, + descriptor, + repeatedRow, + true, + false, + unknown, + null, + -1, + TableRowToStorageApiProto.ErrorCollector.DONT_COLLECT); System.out.println(unknown); // unkown at top level assertEquals(2, unknown.size()); @@ -2174,7 +2294,15 @@ public void testIgnoreUnknownRepeatedNestedFieldWithNoUnknowns() throws Exceptio TableRow unknown = new TableRow(); DynamicMessage msg = TableRowToStorageApiProto.messageFromTableRow( - schemaInformation, descriptor, tableRow, true, false, unknown, null, -1); + schemaInformation, + descriptor, + tableRow, + true, + false, + unknown, + null, + -1, + TableRowToStorageApiProto.ErrorCollector.DONT_COLLECT); assertEquals(2, msg.getAllFields().size()); assertTrue(unknown.isEmpty()); } @@ -2213,7 +2341,15 @@ public void testIgnoreUnknownRepeatedNestedFieldWithUnknownInRepeatedField() thr TableRow unknown = new TableRow(); DynamicMessage msg = TableRowToStorageApiProto.messageFromTableRow( - schemaInformation, descriptor, tableRow, true, false, unknown, null, -1); + schemaInformation, + descriptor, + tableRow, + true, + false, + unknown, + null, + -1, + TableRowToStorageApiProto.ErrorCollector.DONT_COLLECT); assertEquals(2, msg.getAllFields().size()); assertFalse(unknown.isEmpty()); assertEquals(2, ((List) unknown.get("repeated1")).size()); @@ -2257,7 +2393,15 @@ public void testMergeUnknownRepeatedNestedFieldWithUnknownInRepeatedField() thro TableRow unknown = new TableRow(); DynamicMessage msg = TableRowToStorageApiProto.messageFromTableRow( - schemaInformation, descriptor, tableRow, true, false, unknown, null, -1); + schemaInformation, + descriptor, + tableRow, + true, + false, + unknown, + null, + -1, + TableRowToStorageApiProto.ErrorCollector.DONT_COLLECT); assertTrue( ((TableRow) ((List) unknown.get("repeated1")).get(0)).isEmpty()); // empty tablerow @@ -2330,7 +2474,15 @@ public void testMergeUnknownRepeatedNestedFieldWithUnknownInRepeatedFieldWhenSch TableRow unknown = new TableRow(); DynamicMessage msg = TableRowToStorageApiProto.messageFromTableRow( - schemaInformation, descriptor, tableRow, true, false, unknown, null, -1); + schemaInformation, + descriptor, + tableRow, + true, + false, + unknown, + null, + -1, + TableRowToStorageApiProto.ErrorCollector.DONT_COLLECT); assertTrue( ((TableRow) ((List) unknown.get("repeated1")).get(0)).isEmpty()); // empty tablerow @@ -2379,7 +2531,15 @@ public void testCdcFields() throws Exception { TableRowToStorageApiProto.SchemaInformation.fromTableSchema(NESTED_TABLE_SCHEMA); DynamicMessage msg = TableRowToStorageApiProto.messageFromTableRow( - schemaInformation, descriptor, tableRow, false, false, null, "UPDATE", 42); + schemaInformation, + descriptor, + tableRow, + false, + false, + null, + "UPDATE", + 42, + TableRowToStorageApiProto.ErrorCollector.DONT_COLLECT); assertEquals(6, msg.getAllFields().size()); Map fieldDescriptors =