From 014f8afcd90b3ff4e32df296e3813317b9975c04 Mon Sep 17 00:00:00 2001 From: Reuven Lax Date: Tue, 23 Mar 2021 09:19:52 -0700 Subject: [PATCH 1/2] Resubmit Storage API sink with noop failing test removed. --- .../beam/gradle/BeamModulePlugin.groovy | 5 +- .../BigQueryStorageAPIStreamingIT.java | 132 +++++ .../main/resources/beam/spotbugs-filter.xml | 3 + .../gcp/auth/GcpCredentialFactory.java | 2 + .../io/google-cloud-platform/build.gradle | 3 +- .../beam/sdk/io/gcp/bigquery/BatchLoads.java | 9 +- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 118 +++- .../sdk/io/gcp/bigquery/BigQueryOptions.java | 19 + .../sdk/io/gcp/bigquery/BigQueryServices.java | 49 ++ .../io/gcp/bigquery/BigQueryServicesImpl.java | 142 ++++- .../sdk/io/gcp/bigquery/BigQueryUtils.java | 4 +- .../sdk/io/gcp/bigquery/RetryManager.java | 283 ++++++++++ .../StorageApiDynamicDestinations.java | 78 +++ .../StorageApiDynamicDestinationsBeamRow.java | 65 +++ ...StorageApiDynamicDestinationsTableRow.java | 71 +++ .../StorageApiFinalizeWritesDoFn.java | 171 ++++++ .../StorageApiFlushAndFinalizeDoFn.java | 207 +++++++ .../sdk/io/gcp/bigquery/StorageApiLoads.java | 157 ++++++ .../StorageApiWriteUnshardedRecords.java | 323 +++++++++++ .../StorageApiWritesShardedRecords.java | 524 ++++++++++++++++++ .../sdk/io/gcp/bigquery/TableDestination.java | 9 + .../bigquery/TableRowToStorageApiProto.java | 9 +- .../TwoLevelMessageConverterCache.java | 68 +++ .../io/gcp/testing/FakeDatasetService.java | 171 +++++- .../beam/sdk/io/gcp/GcpApiSurfaceTest.java | 2 + .../io/gcp/bigquery/BigQueryIOWriteTest.java | 279 +++++++--- .../bigquery/BigQueryServicesImplTest.java | 52 +- .../sdk/io/gcp/bigquery/BigQueryUtilTest.java | 4 +- .../sdk/io/gcp/bigquery/RetryManagerTest.java | 212 +++++++ 29 files changed, 3030 insertions(+), 141 deletions(-) create mode 100644 examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryStorageAPIStreamingIT.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/RetryManager.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinations.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsBeamRow.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsTableRow.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFinalizeWritesDoFn.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFlushAndFinalizeDoFn.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiLoads.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TwoLevelMessageConverterCache.java create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/RetryManagerTest.java diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index b2a5288467d3..350c1ae61914 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -424,6 +424,7 @@ class BeamModulePlugin implements Plugin { def classgraph_version = "4.8.65" def errorprone_version = "2.3.4" def google_clients_version = "1.31.0" + def google_auth_version = "0.19.0" def google_cloud_bigdataoss_version = "2.1.6" def google_cloud_pubsublite_version = "0.7.0" def google_code_gson_version = "2.8.6" @@ -518,7 +519,7 @@ class BeamModulePlugin implements Plugin { google_auth_library_credentials : "com.google.auth:google-auth-library-credentials", // google_cloud_platform_libraries_bom sets version google_auth_library_oauth2_http : "com.google.auth:google-auth-library-oauth2-http", // google_cloud_platform_libraries_bom sets version google_cloud_bigquery : "com.google.cloud:google-cloud-bigquery", // google_cloud_platform_libraries_bom sets version - google_cloud_bigquery_storage : "com.google.cloud:google-cloud-bigquerystorage:1.8.5", + google_cloud_bigquery_storage : "com.google.cloud:google-cloud-bigquerystorage:1.12.0", google_cloud_bigtable_client_core : "com.google.cloud.bigtable:bigtable-client-core:1.16.0", google_cloud_bigtable_emulator : "com.google.cloud:google-cloud-bigtable-emulator:0.125.2", google_cloud_core : "com.google.cloud:google-cloud-core", // google_cloud_platform_libraries_bom sets version @@ -602,8 +603,8 @@ class BeamModulePlugin implements Plugin { protobuf_java : "com.google.protobuf:protobuf-java:$protobuf_version", protobuf_java_util : "com.google.protobuf:protobuf-java-util:$protobuf_version", proto_google_cloud_bigquery_storage_v1 : "com.google.api.grpc:proto-google-cloud-bigquerystorage-v1", // google_cloud_platform_libraries_bom sets version - proto_google_cloud_bigquerybeta2_storage_v1 : "com.google.api.grpc:proto-google-cloud-bigquerystorage-v1beta2", // google_cloud_platform_libraries_bom sets version proto_google_cloud_bigtable_admin_v2 : "com.google.api.grpc:proto-google-cloud-bigtable-admin-v2", // google_cloud_platform_libraries_bom sets version + proto_google_cloud_bigquery_storage_v1beta2 : "com.google.api.grpc:proto-google-cloud-bigquerystorage-v1beta2", // google_cloud_platform_libraries_bom sets version proto_google_cloud_bigtable_v2 : "com.google.api.grpc:proto-google-cloud-bigtable-v2", // google_cloud_platform_libraries_bom sets version proto_google_cloud_datacatalog_v1beta1 : "com.google.api.grpc:proto-google-cloud-datacatalog-v1beta1", // google_cloud_platform_libraries_bom sets version proto_google_cloud_datastore_v1 : "com.google.api.grpc:proto-google-cloud-datastore-v1", // google_cloud_platform_libraries_bom sets version diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryStorageAPIStreamingIT.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryStorageAPIStreamingIT.java new file mode 100644 index 000000000000..ad68904efc82 --- /dev/null +++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryStorageAPIStreamingIT.java @@ -0,0 +1,132 @@ +/* + * 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.examples.cookbook; + +import com.google.auto.value.AutoValue; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.concurrent.ThreadLocalRandom; +import javax.annotation.Nullable; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.io.GenerateSequence; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.TestPipelineOptions; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.joda.time.Duration; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class BigQueryStorageAPIStreamingIT { + @DefaultSchema(AutoValueSchema.class) + @AutoValue + public abstract static class Value { + public abstract long getNumber(); + + @Nullable + public abstract ByteBuffer getPayload(); + } + + public interface Options extends TestPipelineOptions { + @Description("The number of records per second to generate.") + @Default.Integer(10000) + Integer getRecordsPerSecond(); + + void setRecordsPerSecond(Integer recordsPerSecond); + + @Description("The size of the records to write in bytes.") + @Default.Integer(1024) + Integer getPayloadSizeBytes(); + + void setPayloadSizeBytes(Integer payloadSizeBytes); + + @Description("Parallelism used for Storage API writes.") + @Default.Integer(5) + Integer getNumShards(); + + void setNumShards(Integer numShards); + + @Description("Frequency to trigger appends. Each shard triggers independently.") + @Default.Integer(5) + Integer getTriggerFrequencySec(); + + void setTriggerFrequencySec(Integer triggerFrequencySec); + + @Description("The table to write to.") + String getTargetTable(); + + void setTargetTable(String table); + } + + @BeforeClass + public static void setUp() { + PipelineOptionsFactory.register(Options.class); + } + + @Test + public void testStorageAPIStreaming() throws Exception { + Options options = TestPipeline.testingPipelineOptions().as(Options.class); + Pipeline p = Pipeline.create(options); + final int payloadSizeBytes = options.getPayloadSizeBytes(); + + // Generate input. + PCollection values = + p.apply( + GenerateSequence.from(1) + .to(1000000) + .withRate(options.getRecordsPerSecond(), Duration.standardSeconds(1))) + .apply( + MapElements.into(TypeDescriptor.of(Value.class)) + .via( + l -> { + byte[] payload = "".getBytes(StandardCharsets.UTF_8); + if (payloadSizeBytes > 0) { + payload = new byte[payloadSizeBytes]; + ThreadLocalRandom.current().nextBytes(payload); + } + return new AutoValue_BigQueryStorageAPIStreamingIT_Value( + l, ByteBuffer.wrap(payload)); + })); + + // Write results using Vortex. + values.apply( + "writeVortex", + BigQueryIO.write() + .useBeamSchema() + .to(options.getTargetTable()) + .withMethod(Write.Method.STORAGE_WRITE_API) + .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) + .withWriteDisposition(WriteDisposition.WRITE_APPEND) + .withNumStorageWriteApiStreams(options.getNumShards()) + .withTriggeringFrequency(Duration.standardSeconds(options.getTriggerFrequencySec()))); + + p.run(); + } +} diff --git a/sdks/java/build-tools/src/main/resources/beam/spotbugs-filter.xml b/sdks/java/build-tools/src/main/resources/beam/spotbugs-filter.xml index 939b58c036a6..71f21720fbdd 100644 --- a/sdks/java/build-tools/src/main/resources/beam/spotbugs-filter.xml +++ b/sdks/java/build-tools/src/main/resources/beam/spotbugs-filter.xml @@ -72,4 +72,7 @@ + + + diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/auth/GcpCredentialFactory.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/auth/GcpCredentialFactory.java index 0931dd4485d8..e7193da1c6b1 100644 --- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/auth/GcpCredentialFactory.java +++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/auth/GcpCredentialFactory.java @@ -46,6 +46,8 @@ public class GcpCredentialFactory implements CredentialFactory { "https://www.googleapis.com/auth/devstorage.full_control", "https://www.googleapis.com/auth/userinfo.email", "https://www.googleapis.com/auth/datastore", + "https://www.googleapis.com/auth/bigquery", + "https://www.googleapis.com/auth/bigquery.insertdata", "https://www.googleapis.com/auth/pubsub"); private static final GcpCredentialFactory INSTANCE = new GcpCredentialFactory(); diff --git a/sdks/java/io/google-cloud-platform/build.gradle b/sdks/java/io/google-cloud-platform/build.gradle index e0704c53016d..cdf00f299800 100644 --- a/sdks/java/io/google-cloud-platform/build.gradle +++ b/sdks/java/io/google-cloud-platform/build.gradle @@ -105,8 +105,7 @@ dependencies { compile library.java.netty_tcnative_boringssl_static permitUnusedDeclared library.java.netty_tcnative_boringssl_static // BEAM-11761 compile library.java.proto_google_cloud_bigquery_storage_v1 - compile library.java.proto_google_cloud_bigquerybeta2_storage_v1 - permitUnusedDeclared library.java.proto_google_cloud_bigquerybeta2_storage_v1 // BEAM-11761 + compile library.java.proto_google_cloud_bigquery_storage_v1beta2 compile library.java.proto_google_cloud_bigtable_admin_v2 compile library.java.proto_google_cloud_bigtable_v2 compile library.java.proto_google_cloud_datastore_v1 diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java index 1372e3f565cd..0d4f7b7d80ec 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java @@ -356,12 +356,13 @@ private WriteResult expandTriggered(PCollection> inpu tempTables // Now that the load job has happened, we want the rename to happen immediately. .apply( + "Window Into Global Windows", Window.>into(new GlobalWindows()) .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(1)))) - .apply(WithKeys.of((Void) null)) + .apply("Add Void Key", WithKeys.of((Void) null)) .setCoder(KvCoder.of(VoidCoder.of(), tempTables.getCoder())) - .apply(GroupByKey.create()) - .apply(Values.create()) + .apply("GroupByKey", GroupByKey.create()) + .apply("Extract Values", Values.create()) .apply( "WriteRenameTriggered", ParDo.of( @@ -464,7 +465,7 @@ public void process(ProcessContext c) { // Generate the temporary-file prefix. private PCollectionView createTempFilePrefixView( Pipeline p, final PCollectionView jobIdView) { - return p.apply(Create.of("")) + return p.apply("Create dummy value", Create.of("")) .apply( "GetTempFilePrefix", ParDo.of( 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 d8f3b951531b..6493ee4c267d 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 @@ -1664,6 +1664,7 @@ public static Write write() { .setWriteDisposition(Write.WriteDisposition.WRITE_EMPTY) .setSchemaUpdateOptions(Collections.emptySet()) .setNumFileShards(0) + .setNumStorageWriteApiStreams(0) .setMethod(Write.Method.DEFAULT) .setExtendedErrorInfo(false) .setSkipInvalidRows(false) @@ -1724,7 +1725,9 @@ public enum Method { * href="https://cloud.google.com/bigquery/streaming-data-into-bigquery">Streaming Data into * BigQuery. */ - STREAMING_INSERTS + STREAMING_INSERTS, + /** Use the new, experimental Storage Write API. */ + STORAGE_WRITE_API } abstract @Nullable ValueProvider getJsonTableRef(); @@ -1771,6 +1774,8 @@ public enum Method { abstract int getNumFileShards(); + abstract int getNumStorageWriteApiStreams(); + abstract int getMaxFilesPerPartition(); abstract long getMaxBytesPerPartition(); @@ -1853,6 +1858,8 @@ abstract Builder setAvroSchemaFactory( abstract Builder setNumFileShards(int numFileShards); + abstract Builder setNumStorageWriteApiStreams(int numStorageApiStreams); + abstract Builder setMaxFilesPerPartition(int maxFilesPerPartition); abstract Builder setMaxBytesPerPartition(long maxBytesPerPartition); @@ -2285,6 +2292,19 @@ public Write withNumFileShards(int numFileShards) { return toBuilder().setNumFileShards(numFileShards).build(); } + /** + * Control how many parallel streams are used when using Storage API writes. Applicable only + * when also setting {@link #withTriggeringFrequency}. To let runner determine the sharding at + * runtime, set {@link #withAutoSharding()} instead. + */ + public Write withNumStorageWriteApiStreams(int numStorageWriteApiStreams) { + checkArgument( + numStorageWriteApiStreams > 0, + "numStorageWriteApiStreams must be > 0, but was: %s", + numStorageWriteApiStreams); + return toBuilder().setNumStorageWriteApiStreams(numStorageWriteApiStreams).build(); + } + /** * Provides a custom location on GCS for storing temporary files to be loaded via BigQuery batch * load jobs. See "Usage with templates" in {@link BigQueryIO} documentation for discussion. @@ -2455,6 +2475,9 @@ private Method resolveMethod(PCollection input) { if (getMethod() != Method.DEFAULT) { return getMethod(); } + if (input.getPipeline().getOptions().as(BigQueryOptions.class).getUseStorageWriteApi()) { + return Method.STORAGE_WRITE_API; + } // By default, when writing an Unbounded PCollection, we use StreamingInserts and // BigQuery's streaming import API. return (input.isBounded() == IsBounded.UNBOUNDED) @@ -2462,6 +2485,23 @@ private Method resolveMethod(PCollection input) { : Method.FILE_LOADS; } + private Duration getStorageApiTriggeringFrequency(BigQueryOptions options) { + if (getTriggeringFrequency() != null) { + return getTriggeringFrequency(); + } + if (options.getStorageWriteApiTriggeringFrequencySec() != null) { + return Duration.standardSeconds(options.getStorageWriteApiTriggeringFrequencySec()); + } + return null; + } + + private int getStorageApiNumStreams(BigQueryOptions options) { + if (getNumStorageWriteApiStreams() != 0) { + return getNumStorageWriteApiStreams(); + } + return options.getNumStorageWriteApiStreams(); + } + @Override public WriteResult expand(PCollection input) { // We must have a destination to write to! @@ -2492,16 +2532,22 @@ public WriteResult expand(PCollection input) { "No more than one of jsonSchema, schemaFromView, or dynamicDestinations may be set"); Method method = resolveMethod(input); - if (input.isBounded() == IsBounded.UNBOUNDED && method == Method.FILE_LOADS) { + if (input.isBounded() == IsBounded.UNBOUNDED + && (method == Method.FILE_LOADS || method == Method.STORAGE_WRITE_API)) { + BigQueryOptions bqOptions = input.getPipeline().getOptions().as(BigQueryOptions.class); + Duration triggeringFrequency = + (method == Method.STORAGE_WRITE_API) + ? getStorageApiTriggeringFrequency(bqOptions) + : getTriggeringFrequency(); checkArgument( - getTriggeringFrequency() != null, - "When writing an unbounded PCollection via FILE_LOADS, " + triggeringFrequency != null, + "When writing an unbounded PCollection via FILE_LOADS or STORAGE_API_WRITES, " + "triggering frequency must be specified"); } else { checkArgument( getTriggeringFrequency() == null && getNumFileShards() == 0, "Triggering frequency or number of file shards can be specified only when writing " - + "an unbounded PCollection via FILE_LOADS, but: the collection was %s " + + "an unbounded PCollection via FILE_LOADS or STORAGE_API_WRITES, but: the collection was %s " + "and the method was %s", input.isBounded(), method); @@ -2519,6 +2565,9 @@ public WriteResult expand(PCollection input) { if (input.isBounded() == IsBounded.BOUNDED) { checkArgument(!getAutoSharding(), "Auto-sharding is only applicable to unbounded input."); } + if (method == Method.STORAGE_WRITE_API) { + checkArgument(!getAutoSharding(), "Auto sharding not yet available for Storage API writes"); + } if (getJsonTimePartitioning() != null) { checkArgument( @@ -2613,7 +2662,7 @@ private WriteResult expandTyped( "CreateDisposition is CREATE_IF_NEEDED, however no schema was provided."); } - Coder destinationCoder = null; + Coder destinationCoder; try { destinationCoder = dynamicDestinations.getDestinationCoderWithDefault( @@ -2664,27 +2713,33 @@ private WriteResult expandTyped( rowWriterFactory = RowWriterFactory.tableRows(formatFunction, formatRecordOnFailureFunction); } + PCollection> rowsWithDestination = input .apply( "PrepareWrite", new PrepareWrite<>(dynamicDestinations, SerializableFunctions.identity())) .setCoder(KvCoder.of(destinationCoder, input.getCoder())); + return continueExpandTyped( rowsWithDestination, input.getCoder(), + getUseBeamSchema() ? input.getSchema() : null, + getUseBeamSchema() ? input.getToRowFunction() : null, destinationCoder, dynamicDestinations, rowWriterFactory, method); } - private WriteResult continueExpandTyped( - PCollection> input, - Coder elementCoder, + private WriteResult continueExpandTyped( + PCollection> input, + Coder elementCoder, + @Nullable Schema elementSchema, + @Nullable SerializableFunction elementToRowFunction, Coder destinationCoder, DynamicDestinations dynamicDestinations, - RowWriterFactory rowWriterFactory, + RowWriterFactory rowWriterFactory, Method method) { if (method == Method.STREAMING_INSERTS) { checkArgument( @@ -2700,10 +2755,9 @@ private WriteResult continueExpandTyped( getSchemaUpdateOptions() == null || getSchemaUpdateOptions().isEmpty(), "SchemaUpdateOptions are not supported when method == STREAMING_INSERTS"); - RowWriterFactory.TableRowWriterFactory tableRowWriterFactory = - (RowWriterFactory.TableRowWriterFactory) rowWriterFactory; - - StreamingInserts streamingInserts = + RowWriterFactory.TableRowWriterFactory tableRowWriterFactory = + (RowWriterFactory.TableRowWriterFactory) rowWriterFactory; + StreamingInserts streamingInserts = new StreamingInserts<>( getCreateDisposition(), dynamicDestinations, @@ -2719,7 +2773,7 @@ private WriteResult continueExpandTyped( .withAutoSharding(getAutoSharding()) .withKmsKey(getKmsKey()); return input.apply(streamingInserts); - } else { + } else if (method == Method.FILE_LOADS) { checkArgument( getFailedInsertRetryPolicy() == null, "Record-insert retry policies are not supported when using BigQuery load jobs."); @@ -2730,7 +2784,7 @@ private WriteResult continueExpandTyped( "useAvroLogicalTypes can only be set with Avro output."); } - BatchLoads batchLoads = + BatchLoads batchLoads = new BatchLoads<>( getWriteDisposition(), getCreateDisposition(), @@ -2770,6 +2824,38 @@ private WriteResult continueExpandTyped( batchLoads.setNumFileShards(getNumFileShards()); } return input.apply(batchLoads); + } else if (method == Method.STORAGE_WRITE_API) { + StorageApiDynamicDestinations storageApiDynamicDestinations; + if (getUseBeamSchema()) { + // This ensures that the Beam rows are directly translated into protos for Sorage API + // writes, with no + // need to round trip through JSON TableRow objects. + storageApiDynamicDestinations = + new StorageApiDynamicDestinationsBeamRow( + dynamicDestinations, elementSchema, elementToRowFunction); + } else { + RowWriterFactory.TableRowWriterFactory tableRowWriterFactory = + (RowWriterFactory.TableRowWriterFactory) rowWriterFactory; + // Fallback behavior: convert to JSON TableRows and convert those into Beam TableRows. + storageApiDynamicDestinations = + new StorageApiDynamicDestinationsTableRow<>( + dynamicDestinations, tableRowWriterFactory.getToRowFn()); + } + + BigQueryOptions bqOptions = input.getPipeline().getOptions().as(BigQueryOptions.class); + StorageApiLoads storageApiLoads = + new StorageApiLoads( + destinationCoder, + elementCoder, + storageApiDynamicDestinations, + getCreateDisposition(), + getKmsKey(), + getStorageApiTriggeringFrequency(bqOptions), + getBigQueryServices(), + getStorageApiNumStreams(bqOptions)); + 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 877f4db8606c..383e79e023ce 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 @@ -78,4 +78,23 @@ public interface BigQueryOptions Integer getBqStreamingApiLoggingFrequencySec(); void setBqStreamingApiLoggingFrequencySec(Integer value); + + @Description("If set, then BigQueryIO.Write will default to using the Storage Write API.") + @Default.Boolean(false) + Boolean getUseStorageWriteApi(); + + void setUseStorageWriteApi(Boolean value); + + @Description( + "If set, then BigQueryIO.Write will default to using this number of Storage Write API streams.") + @Default.Integer(0) + Integer getNumStorageWriteApiStreams(); + + void setNumStorageWriteApiStreams(Integer value); + + @Description( + "If set, then BigQueryIO.Write will default to triggering the Storage Write API writes this often.") + Integer getStorageWriteApiTriggeringFrequencySec(); + + void setStorageWriteApiTriggeringFrequencySec(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 7064631bfb2b..37bb6bfcf057 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 @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; +import com.google.api.core.ApiFuture; import com.google.api.services.bigquery.model.Dataset; import com.google.api.services.bigquery.model.Job; import com.google.api.services.bigquery.model.JobConfigurationExtract; @@ -34,6 +35,13 @@ import com.google.cloud.bigquery.storage.v1.ReadSession; import com.google.cloud.bigquery.storage.v1.SplitReadStreamRequest; import com.google.cloud.bigquery.storage.v1.SplitReadStreamResponse; +import com.google.cloud.bigquery.storage.v1beta2.AppendRowsResponse; +import com.google.cloud.bigquery.storage.v1beta2.BatchCommitWriteStreamsResponse; +import com.google.cloud.bigquery.storage.v1beta2.FinalizeWriteStreamResponse; +import com.google.cloud.bigquery.storage.v1beta2.FlushRowsResponse; +import com.google.cloud.bigquery.storage.v1beta2.ProtoRows; +import com.google.cloud.bigquery.storage.v1beta2.WriteStream; +import com.google.protobuf.Descriptors.Descriptor; import java.io.IOException; import java.io.Serializable; import java.util.List; @@ -164,6 +172,47 @@ long insertAll( /** Patch BigQuery {@link Table} description. */ Table patchTableDescription(TableReference tableReference, @Nullable String tableDescription) throws IOException, InterruptedException; + + /** Create a Write Stream for use with the the Storage Write API. */ + WriteStream createWriteStream(String tableUrn, WriteStream.Type type) + throws IOException, InterruptedException; + + /** + * Create an append client for a given Storage API write stream. The stream must be created + * first. + */ + StreamAppendClient getStreamAppendClient(String streamName) throws Exception; + + /** Flush a given stream up to the given offset. The stream must have type BUFFERED. */ + ApiFuture flush(String streamName, long flushOffset) + throws IOException, InterruptedException; + + /** + * Finalize a write stream. After finalization, no more records can be appended to the stream. + */ + ApiFuture finalizeWriteStream(String streamName); + + /** Commit write streams of type PENDING. The streams must be finalized before committing. */ + ApiFuture commitWriteStreams( + String tableUrn, Iterable writeStreamNames); + } + + /** An interface for appending records to a Storage API write stream. */ + interface StreamAppendClient extends AutoCloseable { + /** Append rows to a Storage API write stream at the given offset. */ + ApiFuture appendRows(long offset, ProtoRows rows, Descriptor descriptor) + throws Exception; + + /** + * Pin this object. If close() is called before all pins are removed, the underlying resources + * will not be freed until all pins are removed. + */ + void pin(); + + /** + * Unpin this object. If the object has been closed, this will release any underlying resources. + */ + void unpin() throws Exception; } /** 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 eb95c8b10409..8f7a0977044c 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 @@ -27,6 +27,7 @@ import com.google.api.client.util.BackOffUtils; import com.google.api.client.util.ExponentialBackOff; import com.google.api.client.util.Sleeper; +import com.google.api.core.ApiFuture; import com.google.api.gax.core.FixedCredentialsProvider; import com.google.api.gax.rpc.FixedHeaderProvider; import com.google.api.gax.rpc.HeaderProvider; @@ -62,8 +63,25 @@ import com.google.cloud.bigquery.storage.v1.ReadSession; import com.google.cloud.bigquery.storage.v1.SplitReadStreamRequest; import com.google.cloud.bigquery.storage.v1.SplitReadStreamResponse; +import com.google.cloud.bigquery.storage.v1beta2.AppendRowsRequest; +import com.google.cloud.bigquery.storage.v1beta2.AppendRowsResponse; +import com.google.cloud.bigquery.storage.v1beta2.BatchCommitWriteStreamsRequest; +import com.google.cloud.bigquery.storage.v1beta2.BatchCommitWriteStreamsResponse; +import com.google.cloud.bigquery.storage.v1beta2.BigQueryWriteClient; +import com.google.cloud.bigquery.storage.v1beta2.BigQueryWriteSettings; +import com.google.cloud.bigquery.storage.v1beta2.CreateWriteStreamRequest; +import com.google.cloud.bigquery.storage.v1beta2.FinalizeWriteStreamRequest; +import com.google.cloud.bigquery.storage.v1beta2.FinalizeWriteStreamResponse; +import com.google.cloud.bigquery.storage.v1beta2.FlushRowsRequest; +import com.google.cloud.bigquery.storage.v1beta2.FlushRowsResponse; +import com.google.cloud.bigquery.storage.v1beta2.ProtoRows; +import com.google.cloud.bigquery.storage.v1beta2.ProtoSchema; +import com.google.cloud.bigquery.storage.v1beta2.StreamWriterV2; +import com.google.cloud.bigquery.storage.v1beta2.WriteStream; import com.google.cloud.hadoop.util.ApiErrorExtractor; import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer; +import com.google.protobuf.Descriptors.Descriptor; +import com.google.protobuf.Int64Value; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -84,6 +102,7 @@ import org.apache.beam.runners.core.metrics.MonitoringInfoConstants; import org.apache.beam.runners.core.metrics.MonitoringInfoMetricName; import org.apache.beam.sdk.extensions.gcp.auth.NullCredentialInitializer; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.extensions.gcp.options.GcsOptions; import org.apache.beam.sdk.extensions.gcp.util.BackOffAdapter; import org.apache.beam.sdk.extensions.gcp.util.CustomHttpErrors; @@ -99,6 +118,7 @@ import org.apache.beam.sdk.values.FailsafeValueInSingleWindow; import org.apache.beam.sdk.values.ValueInSingleWindow; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; @@ -438,6 +458,7 @@ static class DatasetServiceImpl implements DatasetService { private final ApiErrorExtractor errorExtractor; private final Bigquery client; + @Nullable private final BigQueryWriteClient newWriteClient; private final PipelineOptions options; private final long maxRowsPerBatch; private final long maxRowBatchSize; @@ -466,10 +487,12 @@ static class DatasetServiceImpl implements DatasetService { protected static final String CANONICAL_STATUS_UNKNOWN = "unknown"; @VisibleForTesting - DatasetServiceImpl(Bigquery client, PipelineOptions options) { + DatasetServiceImpl( + Bigquery client, @Nullable BigQueryWriteClient newWriteClient, PipelineOptions options) { BigQueryOptions bqOptions = options.as(BigQueryOptions.class); this.errorExtractor = new ApiErrorExtractor(); this.client = client; + this.newWriteClient = newWriteClient; this.options = options; this.maxRowsPerBatch = bqOptions.getMaxStreamingRowsToBatch(); this.maxRowBatchSize = bqOptions.getMaxStreamingBatchSize(); @@ -477,10 +500,15 @@ static class DatasetServiceImpl implements DatasetService { } @VisibleForTesting - DatasetServiceImpl(Bigquery client, PipelineOptions options, long maxRowsPerBatch) { + DatasetServiceImpl( + Bigquery client, + BigQueryWriteClient newWriteClient, + PipelineOptions options, + long maxRowsPerBatch) { BigQueryOptions bqOptions = options.as(BigQueryOptions.class); this.errorExtractor = new ApiErrorExtractor(); this.client = client; + this.newWriteClient = newWriteClient; this.options = options; this.maxRowsPerBatch = maxRowsPerBatch; this.maxRowBatchSize = bqOptions.getMaxStreamingBatchSize(); @@ -490,6 +518,7 @@ static class DatasetServiceImpl implements DatasetService { private DatasetServiceImpl(BigQueryOptions bqOptions) { this.errorExtractor = new ApiErrorExtractor(); this.client = newBigQueryClient(bqOptions).build(); + this.newWriteClient = newBigQueryWriteClient(bqOptions); this.options = bqOptions; this.maxRowsPerBatch = bqOptions.getMaxStreamingRowsToBatch(); this.maxRowBatchSize = bqOptions.getMaxStreamingBatchSize(); @@ -1054,6 +1083,104 @@ public Table patchTableDescription( createDefaultBackoff(), ALWAYS_RETRY); } + + @Override + public WriteStream createWriteStream(String tableUrn, WriteStream.Type type) + throws IOException { + return newWriteClient.createWriteStream( + CreateWriteStreamRequest.newBuilder() + .setParent(tableUrn) + .setWriteStream(WriteStream.newBuilder().setType(type).build()) + .build()); + } + + @Override + public StreamAppendClient getStreamAppendClient(String streamName) throws Exception { + StreamWriterV2 streamWriter = StreamWriterV2.newBuilder(streamName).build(); + return new StreamAppendClient() { + private int pins = 0; + private boolean closed = false; + + @Override + public void close() throws Exception { + boolean closeWriter; + synchronized (this) { + Preconditions.checkState(!closed); + closed = true; + closeWriter = (pins == 0); + } + if (closeWriter) { + streamWriter.close(); + } + } + + @Override + public void pin() { + synchronized (this) { + Preconditions.checkState(!closed); + ++pins; + } + } + + @Override + public void unpin() throws Exception { + boolean closeWriter; + synchronized (this) { + Preconditions.checkState(pins > 0); + --pins; + closeWriter = (pins == 0) && closed; + } + if (closeWriter) { + streamWriter.close(); + } + } + + @Override + public ApiFuture appendRows( + long offset, ProtoRows rows, Descriptor descriptor) throws Exception { + final AppendRowsRequest.ProtoData data = + AppendRowsRequest.ProtoData.newBuilder() + .setWriterSchema( + ProtoSchema.newBuilder().setProtoDescriptor(descriptor.toProto()).build()) + .setRows(rows) + .build(); + AppendRowsRequest.Builder appendRequestBuilder = + AppendRowsRequest.newBuilder().setProtoRows(data).setWriteStream(streamName); + if (offset >= 0) { + appendRequestBuilder = appendRequestBuilder.setOffset(Int64Value.of(offset)); + } + return streamWriter.append(appendRequestBuilder.build()); + } + }; + } + + @Override + public ApiFuture flush(String streamName, long flushOffset) + throws IOException, InterruptedException { + Int64Value offset = Int64Value.newBuilder().setValue(flushOffset).build(); + FlushRowsRequest request = + FlushRowsRequest.newBuilder().setWriteStream(streamName).setOffset(offset).build(); + return newWriteClient.flushRowsCallable().futureCall(request); + } + + @Override + public ApiFuture finalizeWriteStream(String streamName) { + return newWriteClient + .finalizeWriteStreamCallable() + .futureCall(FinalizeWriteStreamRequest.newBuilder().setName(streamName).build()); + } + + @Override + public ApiFuture commitWriteStreams( + String tableUrn, Iterable writeStreamNames) { + return newWriteClient + .batchCommitWriteStreamsCallable() + .futureCall( + BatchCommitWriteStreamsRequest.newBuilder() + .setParent(tableUrn) + .addAllWriteStreams(writeStreamNames) + .build()); + } } static final SerializableFunction DONT_RETRY_NOT_FOUND = @@ -1123,6 +1250,17 @@ private static Bigquery.Builder newBigQueryClient(BigQueryOptions options) { .setGoogleClientRequestInitializer(options.getGoogleApiTrace()); } + private static BigQueryWriteClient newBigQueryWriteClient(BigQueryOptions options) { + try { + return BigQueryWriteClient.create( + BigQueryWriteSettings.newBuilder() + .setCredentialsProvider(() -> options.as(GcpOptions.class).getGcpCredential()) + .build()); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + public static CustomHttpErrors createBigQueryClientCustomErrors() { CustomHttpErrors.Builder builder = new CustomHttpErrors.Builder(); // 403 errors, to list tables, matching this URL: diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java index 4ce40b6ab7db..d701fc3af48a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java @@ -110,7 +110,7 @@ public abstract static class Builder { public abstract static class SchemaConversionOptions implements Serializable { /** - * Controls whether to use the map or row FieldType for a TableSchema field that appears to + * /** Controls whether to use the map or row FieldType for a TableSchema field that appears to * represent a map (it is an array of structs containing only {@code key} and {@code value} * fields). */ @@ -558,7 +558,7 @@ public static TableRow toTableRow(Row row) { } // fall through default: - return fieldValue; + return fieldValue.toString(); } } 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 new file mode 100644 index 000000000000..16ae19712aa3 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/RetryManager.java @@ -0,0 +1,283 @@ +/* + * 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.core.ApiFuture; +import com.google.api.core.ApiFutureCallback; +import com.google.api.core.ApiFutures; +import java.util.Queue; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.stream.Collectors; +import javax.annotation.Nullable; +import org.apache.beam.sdk.io.gcp.bigquery.RetryManager.Operation.Context; +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.vendor.guava.v26_0_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Queues; +import org.joda.time.Duration; + +/** + * Retry manager used by Storage API operations. This class manages a sequence of operations (e.g. + * sequential appends to a stream) and retries of those operations. If any one operation fails, then + * all subsequent operations are expected to fail true and will alll be retried. + */ +class RetryManager> { + private Queue> operations; + private final BackOff backoff; + private final ExecutorService executor; + + // Enum returned by onError indicating whether errors should be retried. + enum RetryType { + // The in-flight operations will not be retried. + DONT_RETRY, + // All operations will be retried. + RETRY_ALL_OPERATIONS + }; + + static class WrappedFailure extends Throwable { + @Nullable private final Object result; + + public WrappedFailure(@Nullable Object result) { + this.result = result; + } + + @Nullable + Object getResult() { + return result; + } + } + + RetryManager(Duration initialBackoff, Duration maxBackoff, int maxRetries) { + this.operations = Queues.newArrayDeque(); + backoff = + FluentBackoff.DEFAULT + .withInitialBackoff(initialBackoff) + .withMaxBackoff(maxBackoff) + .withMaxRetries(maxRetries) + .backoff(); + this.executor = Executors.newCachedThreadPool(); + } + + static class Operation> { + static class Context { + private @Nullable Throwable error = null; + private @Nullable ResultT result = null; + + public void setError(@Nullable Throwable error) { + this.error = error; + } + + public @Nullable Throwable getError() { + return error; + } + + public void setResult(@Nullable ResultT result) { + this.result = result; + } + + public @Nullable ResultT getResult() { + return result; + } + } + + private final Function> runOperation; + private final Function, RetryType> onError; + private final Consumer onSuccess; + private final Function hasSucceeded; + @Nullable private ApiFuture future = null; + @Nullable private Callback callback = null; + @Nullable ContextT context = null; + + public Operation( + Function> runOperation, + Function, RetryType> onError, + Consumer onSuccess, + Function hasSucceeded, + ContextT context) { + this.runOperation = runOperation; + this.onError = onError; + this.onSuccess = onSuccess; + this.hasSucceeded = hasSucceeded; + this.context = context; + } + + @SuppressWarnings({"nullness"}) + void run(Executor executor) { + this.future = runOperation.apply(context); + this.callback = new Callback<>(hasSucceeded); + ApiFutures.addCallback(future, callback, executor); + } + + @SuppressWarnings({"nullness"}) + boolean await() throws Exception { + callback.await(); + return callback.getFailed(); + } + } + + private static class Callback implements ApiFutureCallback { + private final CountDownLatch waiter; + private final Function hasSucceeded; + @Nullable private Throwable failure = null; + boolean failed = false; + + Callback(Function hasSucceeded) { + this.waiter = new CountDownLatch(1); + this.hasSucceeded = hasSucceeded; + } + + void await() throws InterruptedException { + waiter.await(); + } + + boolean await(long timeoutSec) throws InterruptedException { + return waiter.await(timeoutSec, TimeUnit.SECONDS); + } + + @Override + public void onFailure(Throwable t) { + synchronized (this) { + failure = t; + failed = true; + } + waiter.countDown(); + } + + @Override + public void onSuccess(ResultT result) { + synchronized (this) { + if (hasSucceeded.apply(result)) { + failure = null; + } else { + failure = new WrappedFailure(result); + failed = true; + } + } + waiter.countDown(); + } + + @Nullable + Throwable getFailure() { + synchronized (this) { + return failure; + } + } + + boolean getFailed() { + synchronized (this) { + return failed; + } + } + } + + void addOperation( + Function> runOperation, + Function, RetryType> onError, + Consumer onSuccess, + ContextT context) + throws Exception { + addOperation(runOperation, onError, onSuccess, r -> true, context); + } + + void addOperation( + Function> runOperation, + Function, RetryType> onError, + Consumer onSuccess, + Function hasSucceeded, + ContextT context) + throws Exception { + addOperation(new Operation<>(runOperation, onError, onSuccess, hasSucceeded, context)); + } + + void addAndRunOperation( + Function> runOperation, + Function, RetryType> onError, + Consumer onSuccess, + ContextT context) + throws Exception { + addAndRunOperation(new Operation<>(runOperation, onError, onSuccess, r -> true, context)); + } + + void addAndRunOperation( + Function> runOperation, + Function, RetryType> onError, + Consumer onSuccess, + Function hasSucceeded, + ContextT context) + throws Exception { + addAndRunOperation(new Operation<>(runOperation, onError, onSuccess, hasSucceeded, context)); + } + + void addOperation(Operation operation) { + operations.add(operation); + } + + void addAndRunOperation(Operation operation) { + operation.run(executor); + operations.add(operation); + } + + void run(boolean await) throws Exception { + for (Operation operation : operations) { + operation.run(executor); + } + if (await) { + await(); + } + } + + @SuppressWarnings({"nullness"}) + void await() throws Exception { + while (!this.operations.isEmpty()) { + Operation operation = this.operations.element(); + boolean failed = operation.await(); + if (failed) { + Throwable failure = operation.callback.getFailure(); + operation.context.setError(failure); + RetryType retryType = + operation.onError.apply( + operations.stream().map(o -> o.context).collect(Collectors.toList())); + if (retryType == RetryType.DONT_RETRY) { + operations.clear(); + } else { + Preconditions.checkState(RetryType.RETRY_ALL_OPERATIONS == retryType); + if (!BackOffUtils.next(Sleeper.DEFAULT, backoff)) { + throw new RuntimeException(failure); + } + for (Operation awaitOperation : operations) { + awaitOperation.await(); + } + // Run all the operations again. + run(false); + } + } else { + operation.context.setResult(operation.future.get()); + operation.onSuccess.accept(operation.context); + operations.remove(); + } + } + } +} 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 new file mode 100644 index 000000000000..65e14c602962 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinations.java @@ -0,0 +1,78 @@ +/* + * 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.TableSchema; +import com.google.protobuf.Descriptors.Descriptor; +import com.google.protobuf.Message; +import java.util.List; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.ValueInSingleWindow; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** Base dynamicDestinations class used by the Storage API sink. */ +abstract class StorageApiDynamicDestinations + extends DynamicDestinations { + public interface MessageConverter { + Descriptor getSchemaDescriptor(); + + Message toMessage(T element); + } + + private DynamicDestinations inner; + + StorageApiDynamicDestinations(DynamicDestinations inner) { + this.inner = inner; + } + + public abstract MessageConverter getMessageConverter(DestinationT destination) + throws Exception; + + @Override + public DestinationT getDestination(ValueInSingleWindow element) { + return inner.getDestination(element); + } + + @Override + public @Nullable Coder getDestinationCoder() { + return inner.getDestinationCoder(); + } + + @Override + public TableDestination getTable(DestinationT destination) { + return inner.getTable(destination); + } + + @Override + public TableSchema getSchema(DestinationT destination) { + return inner.getSchema(destination); + } + + @Override + public List> getSideInputs() { + return inner.getSideInputs(); + } + + @Override + void setSideInputAccessorFromProcessContext(DoFn.ProcessContext context) { + super.setSideInputAccessorFromProcessContext(context); + inner.setSideInputAccessorFromProcessContext(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 new file mode 100644 index 000000000000..40814a0a359d --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsBeamRow.java @@ -0,0 +1,65 @@ +/* + * 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.protobuf.Descriptors.Descriptor; +import com.google.protobuf.Message; +import java.time.Duration; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.Cache; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheBuilder; + +@SuppressWarnings({"nullness"}) +/** Storage API DynamicDestinations used when the input is a Beam Row. */ +class StorageApiDynamicDestinationsBeamRow + extends StorageApiDynamicDestinations { + private final Schema schema; + private final SerializableFunction toRow; + private final Cache destinationDescriptorCache = + CacheBuilder.newBuilder().expireAfterAccess(Duration.ofMinutes(15)).build(); + + StorageApiDynamicDestinationsBeamRow( + DynamicDestinations inner, + Schema schema, + SerializableFunction toRow) { + super(inner); + this.schema = schema; + this.toRow = toRow; + } + + @Override + public MessageConverter getMessageConverter(DestinationT destination) throws Exception { + return new MessageConverter() { + Descriptor descriptor = + destinationDescriptorCache.get( + destination, () -> BeamRowToStorageApiProto.getDescriptorFromSchema(schema)); + + @Override + public Descriptor getSchemaDescriptor() { + return descriptor; + } + + @Override + public Message toMessage(T element) { + return BeamRowToStorageApiProto.messageFromBeamRow(descriptor, toRow.apply(element)); + } + }; + } +} 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 new file mode 100644 index 000000000000..0204488079e4 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsTableRow.java @@ -0,0 +1,71 @@ +/* + * 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.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.protobuf.Descriptors.Descriptor; +import com.google.protobuf.Message; +import java.time.Duration; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.Cache; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheBuilder; + +@SuppressWarnings({"nullness"}) +public class StorageApiDynamicDestinationsTableRow + extends StorageApiDynamicDestinations { + private final SerializableFunction formatFunction; + + // TODO: Make static! Or at least optimize the constant schema case. + private final Cache destinationDescriptorCache = + CacheBuilder.newBuilder().expireAfterAccess(Duration.ofMinutes(15)).build(); + + StorageApiDynamicDestinationsTableRow( + DynamicDestinations inner, + SerializableFunction formatFunction) { + super(inner); + this.formatFunction = formatFunction; + } + + @Override + public MessageConverter getMessageConverter(DestinationT destination) throws Exception { + final TableSchema tableSchema = getSchema(destination); + if (tableSchema == null) { + throw new RuntimeException( + "Schema must be set when writing TableRows using Storage API. Use " + + "BigQueryIO.Write.withSchema to set the schema."); + } + return new MessageConverter() { + Descriptor descriptor = + destinationDescriptorCache.get( + destination, + () -> TableRowToStorageApiProto.getDescriptorFromTableSchema(tableSchema)); + + @Override + public Descriptor getSchemaDescriptor() { + return descriptor; + } + + @Override + public Message toMessage(T element) { + return TableRowToStorageApiProto.messageFromTableRow( + descriptor, formatFunction.apply(element)); + } + }; + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFinalizeWritesDoFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFinalizeWritesDoFn.java new file mode 100644 index 000000000000..b3b820075dcf --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFinalizeWritesDoFn.java @@ -0,0 +1,171 @@ +/* + * 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.v1beta2.BatchCommitWriteStreamsResponse; +import com.google.cloud.bigquery.storage.v1beta2.FinalizeWriteStreamResponse; +import com.google.cloud.bigquery.storage.v1beta2.StorageError; +import com.google.cloud.bigquery.storage.v1beta2.StorageError.StorageErrorCode; +import java.io.IOException; +import java.util.Collection; +import java.util.Map; +import java.util.Set; +import javax.annotation.Nullable; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; +import org.apache.beam.sdk.io.gcp.bigquery.RetryManager.Operation.Context; +import org.apache.beam.sdk.io.gcp.bigquery.RetryManager.RetryType; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Sets; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** This DoFn finalizes and commits Storage API streams. */ +class StorageApiFinalizeWritesDoFn extends DoFn, Void> { + private static final Logger LOG = LoggerFactory.getLogger(StorageApiFinalizeWritesDoFn.class); + + private final Counter finalizeOperationsSent = + Metrics.counter(StorageApiFinalizeWritesDoFn.class, "finalizeOperationsSent"); + private final Counter finalizeOperationsSucceeded = + Metrics.counter(StorageApiFinalizeWritesDoFn.class, "finalizeOperationsSucceeded"); + private final Counter finalizeOperationsFailed = + Metrics.counter(StorageApiFinalizeWritesDoFn.class, "finalizeOperationsFailed"); + private final Counter batchCommitOperationsSent = + Metrics.counter(StorageApiFinalizeWritesDoFn.class, "batchCommitOperationsSent"); + private final Counter batchCommitOperationsSucceeded = + Metrics.counter(StorageApiFinalizeWritesDoFn.class, "batchCommitOperationsSucceeded"); + private final Counter batchCommitOperationsFailed = + Metrics.counter(StorageApiFinalizeWritesDoFn.class, "batchCommitOperationsFailed"); + + private Map> commitStreams; + private final BigQueryServices bqServices; + @Nullable private DatasetService datasetService; + + public StorageApiFinalizeWritesDoFn(BigQueryServices bqServices) { + this.bqServices = bqServices; + this.commitStreams = Maps.newHashMap(); + this.datasetService = null; + } + + private DatasetService getDatasetService(PipelineOptions pipelineOptions) throws IOException { + if (datasetService == null) { + datasetService = bqServices.getDatasetService(pipelineOptions.as(BigQueryOptions.class)); + } + return datasetService; + } + + @StartBundle + public void startBundle() throws IOException { + commitStreams = Maps.newHashMap(); + } + + @ProcessElement + @SuppressWarnings({"nullness"}) + public void process(PipelineOptions pipelineOptions, @Element KV element) + throws Exception { + String tableId = element.getKey(); + String streamId = element.getValue(); + DatasetService datasetService = getDatasetService(pipelineOptions); + + RetryManager> retryManager = + new RetryManager<>(Duration.standardSeconds(1), Duration.standardMinutes(1), 3); + retryManager.addOperation( + c -> { + finalizeOperationsSent.inc(); + return datasetService.finalizeWriteStream(streamId); + }, + contexts -> { + LOG.error( + "Finalize of stream " + + streamId + + " failed with " + + Iterables.getFirst(contexts, null).getError()); + finalizeOperationsFailed.inc(); + return RetryType.RETRY_ALL_OPERATIONS; + }, + c -> { + LOG.info("Finalize of stream " + streamId + " finished with " + c.getResult()); + finalizeOperationsSucceeded.inc(); + commitStreams.computeIfAbsent(tableId, d -> Lists.newArrayList()).add(streamId); + }, + new Context<>()); + retryManager.run(true); + } + + @FinishBundle + @SuppressWarnings({"nullness"}) + public void finishBundle(PipelineOptions pipelineOptions) throws Exception { + DatasetService datasetService = getDatasetService(pipelineOptions); + for (Map.Entry> entry : commitStreams.entrySet()) { + final String tableId = entry.getKey(); + final Collection streamNames = entry.getValue(); + final Set alreadyCommittedStreams = Sets.newHashSet(); + RetryManager> + retryManager = + new RetryManager<>(Duration.standardSeconds(1), Duration.standardMinutes(1), 3); + retryManager.addOperation( + c -> { + Iterable streamsToCommit = + Iterables.filter(streamNames, s -> !alreadyCommittedStreams.contains(s)); + batchCommitOperationsSent.inc(); + return datasetService.commitWriteStreams(tableId, streamsToCommit); + }, + contexts -> { + LOG.error( + "BatchCommit failed. tableId " + + tableId + + " streamNames " + + streamNames + + " error: " + + Iterables.getFirst(contexts, null).getError()); + batchCommitOperationsFailed.inc(); + return RetryType.RETRY_ALL_OPERATIONS; + }, + c -> { + LOG.info("BatchCommit succeeded for tableId " + tableId + " response " + c.getResult()); + batchCommitOperationsSucceeded.inc(); + }, + response -> { + if (!response.hasCommitTime()) { + for (StorageError storageError : response.getStreamErrorsList()) { + if (storageError.getCode() == StorageErrorCode.STREAM_ALREADY_COMMITTED) { + // Make sure that we don't retry any streams that are already committed. + alreadyCommittedStreams.add(storageError.getEntity()); + } + } + Iterable streamsToCommit = + Iterables.filter(streamNames, s -> !alreadyCommittedStreams.contains(s)); + // If there are no more streams left to commit, then report this operation as having + // succeeded. Otherwise, + // retry. + return Iterables.isEmpty(streamsToCommit); + } + return true; + }, + new Context<>()); + retryManager.run(true); + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFlushAndFinalizeDoFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFlushAndFinalizeDoFn.java new file mode 100644 index 000000000000..a3676ba05322 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiFlushAndFinalizeDoFn.java @@ -0,0 +1,207 @@ +/* + * 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.gax.rpc.ApiException; +import com.google.api.gax.rpc.StatusCode.Code; +import com.google.cloud.bigquery.storage.v1beta2.FinalizeWriteStreamResponse; +import com.google.cloud.bigquery.storage.v1beta2.FlushRowsResponse; +import java.io.IOException; +import java.io.Serializable; +import java.time.Instant; +import java.util.Objects; +import javax.annotation.Nullable; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; +import org.apache.beam.sdk.io.gcp.bigquery.RetryManager.Operation.Context; +import org.apache.beam.sdk.io.gcp.bigquery.RetryManager.RetryType; +import org.apache.beam.sdk.io.gcp.bigquery.StorageApiFlushAndFinalizeDoFn.Operation; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Distribution; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.schemas.JavaFieldSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaCreate; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** This DoFn flushes and optionally (if requested) finalizes Storage API streams. */ +public class StorageApiFlushAndFinalizeDoFn extends DoFn, Void> { + private static final Logger LOG = LoggerFactory.getLogger(StorageApiFlushAndFinalizeDoFn.class); + + private final BigQueryServices bqServices; + @Nullable private DatasetService datasetService = null; + private final Counter flushOperationsSent = + Metrics.counter(StorageApiFlushAndFinalizeDoFn.class, "flushOperationsSent"); + private final Counter flushOperationsSucceeded = + Metrics.counter(StorageApiFlushAndFinalizeDoFn.class, "flushOperationsSucceeded"); + private final Counter flushOperationsFailed = + Metrics.counter(StorageApiFlushAndFinalizeDoFn.class, "flushOperationsFailed"); + private final Counter flushOperationsAlreadyExists = + Metrics.counter(StorageApiFlushAndFinalizeDoFn.class, "flushOperationsAlreadyExists"); + private final Counter flushOperationsInvalidArgument = + Metrics.counter(StorageApiFlushAndFinalizeDoFn.class, "flushOperationsInvalidArgument"); + private final Distribution flushLatencyDistribution = + Metrics.distribution(StorageApiFlushAndFinalizeDoFn.class, "flushOperationLatencyMs"); + private final Counter finalizeOperationsSent = + Metrics.counter(StorageApiFlushAndFinalizeDoFn.class, "finalizeOperationsSent"); + private final Counter finalizeOperationsSucceeded = + Metrics.counter(StorageApiFlushAndFinalizeDoFn.class, "finalizeOperationsSucceeded"); + private final Counter finalizeOperationsFailed = + Metrics.counter(StorageApiFlushAndFinalizeDoFn.class, "finalizeOperationsFailed"); + + @DefaultSchema(JavaFieldSchema.class) + static class Operation implements Comparable, Serializable { + final long flushOffset; + final boolean finalizeStream; + + @SchemaCreate + public Operation(long flushOffset, boolean finalizeStream) { + this.flushOffset = flushOffset; + this.finalizeStream = finalizeStream; + } + + @Override + public boolean equals(@Nullable Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Operation operation = (Operation) o; + return flushOffset == operation.flushOffset && finalizeStream == operation.finalizeStream; + } + + @Override + public int hashCode() { + return Objects.hash(flushOffset, finalizeStream); + } + + @Override + public int compareTo(Operation other) { + int compValue = Long.compare(this.flushOffset, other.flushOffset); + if (compValue == 0) { + compValue = Boolean.compare(this.finalizeStream, other.finalizeStream); + } + return compValue; + } + } + + public StorageApiFlushAndFinalizeDoFn(BigQueryServices bqServices) { + this.bqServices = bqServices; + } + + private DatasetService getDatasetService(PipelineOptions pipelineOptions) throws IOException { + if (datasetService == null) { + datasetService = bqServices.getDatasetService(pipelineOptions.as(BigQueryOptions.class)); + } + return datasetService; + } + + @SuppressWarnings({"nullness"}) + @ProcessElement + public void process(PipelineOptions pipelineOptions, @Element KV element) + throws Exception { + final String streamId = element.getKey(); + final Operation operation = element.getValue(); + final DatasetService datasetService = getDatasetService(pipelineOptions); + // Flush the stream. If the flush offset < 0, that means we only need to finalize. + long offset = operation.flushOffset; + if (offset >= 0) { + Instant now = Instant.now(); + RetryManager> retryManager = + new RetryManager<>(Duration.standardSeconds(1), Duration.standardMinutes(1), 3); + retryManager.addOperation( + // runOperation + c -> { + try { + flushOperationsSent.inc(); + return datasetService.flush(streamId, offset); + } catch (Exception e) { + throw new RuntimeException(e); + } + }, + // onError + contexts -> { + Throwable error = Iterables.getFirst(contexts, null).getError(); + LOG.warn( + "Flush of stream " + streamId + " to offset " + offset + " failed with " + error); + flushOperationsFailed.inc(); + if (error instanceof ApiException) { + Code statusCode = ((ApiException) error).getStatusCode().getCode(); + if (statusCode.equals(Code.ALREADY_EXISTS)) { + flushOperationsAlreadyExists.inc(); + // Implies that we have already flushed up to this point, so don't retry. + return RetryType.DONT_RETRY; + } + if (statusCode.equals(Code.INVALID_ARGUMENT)) { + flushOperationsInvalidArgument.inc(); + // Implies that the stream has already been finalized. + // TODO: Storage API should provide a more-specific way of identifying this failure. + return RetryType.DONT_RETRY; + } + } + return RetryType.RETRY_ALL_OPERATIONS; + }, + // onSuccess + c -> { + flushOperationsSucceeded.inc(); + }, + new Context<>()); + retryManager.run(true); + java.time.Duration timeElapsed = java.time.Duration.between(now, Instant.now()); + flushLatencyDistribution.update(timeElapsed.toMillis()); + } + + // Finalize the stream. No need to commit the stream, since we are only dealing with BUFFERED + // streams here that have + // already been flushed. Note that in the case of errors upstream, we will leave an unflushed + // tail in the stream. + // This is by design - those records will be retried on a new stream, so we don't want to flush + // them in this stream + // or we would end up with duplicates. + if (operation.finalizeStream) { + RetryManager> retryManager = + new RetryManager<>(Duration.standardSeconds(1), Duration.standardMinutes(1), 3); + retryManager.addOperation( + c -> { + finalizeOperationsSent.inc(); + return datasetService.finalizeWriteStream(streamId); + }, + contexts -> { + LOG.warn( + "Finalize of stream " + + streamId + + " failed with " + + Iterables.getFirst(contexts, null).getError()); + finalizeOperationsFailed.inc(); + return RetryType.RETRY_ALL_OPERATIONS; + }, + r -> { + finalizeOperationsSucceeded.inc(); + }, + new Context<>()); + retryManager.run(true); + } + } +} 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 new file mode 100644 index 000000000000..fd3b638fff9c --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiLoads.java @@ -0,0 +1,157 @@ +/* + * 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.TableRow; +import java.nio.ByteBuffer; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.windowing.AfterFirst; +import org.apache.beam.sdk.transforms.windowing.AfterPane; +import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; +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.TupleTag; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** This {@link PTransform} manages loads into BigQuery using the Storage API. */ +public class StorageApiLoads + extends PTransform>, WriteResult> { + private static final Logger LOG = LoggerFactory.getLogger(StorageApiLoads.class); + static final int FILE_TRIGGERING_RECORD_COUNT = 100; + + private final Coder destinationCoder; + private final Coder elementCoder; + private final StorageApiDynamicDestinations dynamicDestinations; + private final CreateDisposition createDisposition; + private final String kmsKey; + private final Duration triggeringFrequency; + private final BigQueryServices bqServices; + private final int numShards; + + public StorageApiLoads( + Coder destinationCoder, + Coder elementCoder, + StorageApiDynamicDestinations dynamicDestinations, + CreateDisposition createDisposition, + String kmsKey, + Duration triggeringFrequency, + BigQueryServices bqServices, + int numShards) { + this.destinationCoder = destinationCoder; + this.elementCoder = elementCoder; + this.dynamicDestinations = dynamicDestinations; + this.createDisposition = createDisposition; + this.kmsKey = kmsKey; + this.triggeringFrequency = triggeringFrequency; + this.bqServices = bqServices; + this.numShards = numShards; + } + + @Override + public WriteResult expand(PCollection> input) { + return triggeringFrequency != null ? expandTriggered(input) : expandUntriggered(input); + } + + public WriteResult expandTriggered(PCollection> input) { + // Handle triggered, low-latency loads into BigQuery. + PCollection> inputInGlobalWindow = + input.apply( + "rewindowIntoGlobal", + Window.>into(new GlobalWindows()) + .triggering( + Repeatedly.forever( + AfterFirst.of( + AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(triggeringFrequency), + AfterPane.elementCountAtLeast(FILE_TRIGGERING_RECORD_COUNT)))) + .discardingFiredPanes()); + + // First shard all the records. + // TODO(reuvenlax): Add autosharding support so that users don't have to pick a shard count. + PCollection, ElementT>> shardedRecords = + inputInGlobalWindow + .apply( + "AddShard", + ParDo.of( + new DoFn, KV, ElementT>>() { + int shardNumber; + + @Setup + public void setup() { + shardNumber = ThreadLocalRandom.current().nextInt(numShards); + } + + @ProcessElement + public void processElement( + @Element KV element, + OutputReceiver, ElementT>> o) { + DestinationT destination = element.getKey(); + ByteBuffer buffer = ByteBuffer.allocate(Integer.BYTES); + buffer.putInt(++shardNumber % numShards); + o.output( + KV.of(ShardedKey.of(destination, buffer.array()), element.getValue())); + } + })) + .setCoder(KvCoder.of(ShardedKey.Coder.of(destinationCoder), elementCoder)); + + PCollection, Iterable>> groupedRecords = + shardedRecords.apply("GroupIntoShards", GroupByKey.create()); + + groupedRecords.apply( + "StorageApiWriteSharded", + new StorageApiWritesShardedRecords<>( + dynamicDestinations, createDisposition, kmsKey, bqServices, destinationCoder)); + + return writeResult(input.getPipeline()); + } + + public WriteResult expandUntriggered(PCollection> input) { + PCollection> inputInGlobalWindow = + input.apply( + "rewindowIntoGlobal", Window.>into(new GlobalWindows())); + inputInGlobalWindow.apply( + "StorageApiWriteUnsharded", + new StorageApiWriteUnshardedRecords<>( + dynamicDestinations, createDisposition, kmsKey, bqServices, destinationCoder)); + return writeResult(input.getPipeline()); + } + + private WriteResult writeResult(Pipeline p) { + // TODO(reuvenlax): Support per-record failures if schema doesn't match or if the record is too + // large. + PCollection empty = + p.apply("CreateEmptyFailedInserts", Create.empty(TypeDescriptor.of(TableRow.class))); + return WriteResult.in(p, new TupleTag<>("failedInserts"), empty); + } +} 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 new file mode 100644 index 000000000000..3e17768cda19 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java @@ -0,0 +1,323 @@ +/* + * 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.v26_0_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.bigquery.storage.v1beta2.AppendRowsResponse; +import com.google.cloud.bigquery.storage.v1beta2.ProtoRows; +import com.google.cloud.bigquery.storage.v1beta2.WriteStream.Type; +import com.google.protobuf.ByteString; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +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.io.gcp.bigquery.BigQueryServices.StreamAppendClient; +import org.apache.beam.sdk.io.gcp.bigquery.RetryManager.Operation.Context; +import org.apache.beam.sdk.io.gcp.bigquery.RetryManager.RetryType; +import org.apache.beam.sdk.io.gcp.bigquery.StorageApiDynamicDestinations.MessageConverter; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Reshuffle; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Supplier; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"nullness"}) +/** + * Write records to the Storage API using a standard batch approach. PENDING streams are used, which + * do not become visible until they are finalized and committed. Each input bundle to the DoFn + * creates a stream per output table, appends all records in the bundle to the stream, and schedules + * a finalize/commit operation at the end. + */ +public class StorageApiWriteUnshardedRecords + extends PTransform>, PCollection> { + private static final Logger LOG = LoggerFactory.getLogger(StorageApiWriteUnshardedRecords.class); + + private final StorageApiDynamicDestinations dynamicDestinations; + private final CreateDisposition createDisposition; + private final String kmsKey; + private final BigQueryServices bqServices; + private final Coder destinationCoder; + @Nullable private DatasetService datasetService = null; + + public StorageApiWriteUnshardedRecords( + StorageApiDynamicDestinations dynamicDestinations, + CreateDisposition createDisposition, + String kmsKey, + BigQueryServices bqServices, + Coder destinationCoder) { + this.dynamicDestinations = dynamicDestinations; + this.createDisposition = createDisposition; + this.kmsKey = kmsKey; + this.bqServices = bqServices; + this.destinationCoder = destinationCoder; + } + + private void initializeDatasetService(PipelineOptions pipelineOptions) { + if (datasetService == null) { + datasetService = bqServices.getDatasetService(pipelineOptions.as(BigQueryOptions.class)); + } + } + + @Override + public PCollection expand(PCollection> input) { + String operationName = input.getName() + "/" + getName(); + return input + .apply( + "Write Records", + ParDo.of(new WriteRecordsDoFn(operationName)) + .withSideInputs(dynamicDestinations.getSideInputs())) + .setCoder(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())) + // Calling Reshuffle makes the output stable - once this completes, the append operations + // will not retry. + // TODO(reuvenlax): This should use RequiresStableInput instead. + .apply("Reshuffle", Reshuffle.of()) + .apply("Finalize writes", ParDo.of(new StorageApiFinalizeWritesDoFn(bqServices))); + } + + private static final ExecutorService closeWriterExecutor = Executors.newCachedThreadPool(); + // Run a closure asynchronously, ignoring failures. + private interface ThrowingRunnable { + void run() throws Exception; + } + + @SuppressWarnings("FutureReturnValueIgnored") + private static void runAsyncIgnoreFailure(ExecutorService executor, ThrowingRunnable task) { + executor.submit( + () -> { + try { + task.run(); + } catch (Exception e) { + // + } + }); + } + + class WriteRecordsDoFn extends DoFn, KV> { + class DestinationState { + private final String tableUrn; + private final MessageConverter messageConverter; + private String streamName = ""; + private @Nullable StreamAppendClient streamAppendClient = null; + private long currentOffset = 0; + private List pendingMessages; + @Nullable private DatasetService datasetService; + + public DestinationState( + String tableUrn, + MessageConverter messageConverter, + DatasetService datasetService) { + this.tableUrn = tableUrn; + this.messageConverter = messageConverter; + this.pendingMessages = Lists.newArrayList(); + this.datasetService = datasetService; + } + + void close() { + if (streamAppendClient != null) { + try { + streamAppendClient.close(); + streamAppendClient = null; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } + + StreamAppendClient getWriteStream() { + try { + if (streamAppendClient == null) { + this.streamName = + Preconditions.checkNotNull(datasetService) + .createWriteStream(tableUrn, Type.PENDING) + .getName(); + this.streamAppendClient = + Preconditions.checkNotNull(datasetService).getStreamAppendClient(streamName); + this.currentOffset = 0; + } + return streamAppendClient; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + void invalidateWriteStream() { + try { + runAsyncIgnoreFailure(closeWriterExecutor, streamAppendClient::close); + streamAppendClient = null; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + void addMessage(ElementT element) throws Exception { + ByteString message = messageConverter.toMessage(element).toByteString(); + pendingMessages.add(message); + if (shouldFlush()) { + flush(); + } + } + + boolean shouldFlush() { + // TODO: look at byte size too? + return pendingMessages.size() > 100; + } + + @SuppressWarnings({"nullness"}) + void flush() throws Exception { + if (pendingMessages.isEmpty()) { + return; + } + final ProtoRows.Builder inserts = ProtoRows.newBuilder(); + for (ByteString m : pendingMessages) { + inserts.addSerializedRows(m); + } + + ProtoRows protoRows = inserts.build(); + pendingMessages.clear(); + + RetryManager> retryManager = + new RetryManager<>(Duration.standardSeconds(1), Duration.standardMinutes(1), 5); + retryManager.addOperation( + c -> { + try { + long offset = currentOffset; + currentOffset += inserts.getSerializedRowsCount(); + return getWriteStream() + .appendRows(offset, protoRows, messageConverter.getSchemaDescriptor()); + } catch (Exception e) { + throw new RuntimeException(e); + } + }, + contexts -> { + LOG.info( + "Append to stream " + + streamName + + " failed with error " + + Iterables.getFirst(contexts, null).getError()); + invalidateWriteStream(); + return RetryType.RETRY_ALL_OPERATIONS; + }, + response -> { + LOG.info("Append to stream " + streamName + " succeded."); + }, + new Context<>()); + // TODO: Do we have to wait on every append? + retryManager.run(true); + } + } + + private Map destinations = Maps.newHashMap(); + private final TwoLevelMessageConverterCache messageConverters; + + WriteRecordsDoFn(String operationName) { + this.messageConverters = new TwoLevelMessageConverterCache<>(operationName); + } + + @StartBundle + public void startBundle() throws IOException { + destinations = Maps.newHashMap(); + } + + DestinationState createDestinationState(ProcessContext c, DestinationT destination) { + TableDestination tableDestination1 = dynamicDestinations.getTable(destination); + checkArgument( + tableDestination1 != null, + "DynamicDestinations.getTable() may not return null, " + + "but %s returned null for destination %s", + dynamicDestinations, + destination); + Supplier schemaSupplier = () -> dynamicDestinations.getSchema(destination); + TableDestination createdTable = + CreateTableHelpers.possiblyCreateTable( + c, + tableDestination1, + schemaSupplier, + createDisposition, + destinationCoder, + kmsKey, + bqServices); + + MessageConverter messageConverter; + try { + messageConverter = messageConverters.get(destination, dynamicDestinations); + } catch (Exception e) { + throw new RuntimeException(e); + } + return new DestinationState(createdTable.getTableUrn(), messageConverter, datasetService); + } + + @ProcessElement + public void process( + ProcessContext c, + PipelineOptions pipelineOptions, + @Element KV element) + throws Exception { + initializeDatasetService(pipelineOptions); + dynamicDestinations.setSideInputAccessorFromProcessContext(c); + DestinationState state = + destinations.computeIfAbsent(element.getKey(), k -> createDestinationState(c, k)); + + if (state.shouldFlush()) { + // Too much memory being used. Flush the state and wait for it to drain out. + // TODO(reuvenlax): Consider waiting for memory usage to drop instead of waiting for all the + // appends to finish. + state.flush(); + } + state.addMessage(element.getValue()); + } + + @FinishBundle + public void finishBundle(FinishBundleContext context) throws Exception { + for (DestinationState state : destinations.values()) { + state.flush(); + context.output( + KV.of(state.tableUrn, state.streamName), + BoundedWindow.TIMESTAMP_MAX_VALUE.minus(1), + GlobalWindow.INSTANCE); + } + } + + @Teardown + public void teardown() { + for (DestinationState state : destinations.values()) { + state.close(); + } + } + } +} 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 new file mode 100644 index 000000000000..ca465dca282b --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java @@ -0,0 +1,524 @@ +/* + * 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.v26_0_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.api.core.ApiFuture; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.cloud.bigquery.storage.v1beta2.AppendRowsResponse; +import com.google.cloud.bigquery.storage.v1beta2.ProtoRows; +import com.google.cloud.bigquery.storage.v1beta2.WriteStream.Type; +import com.google.protobuf.ByteString; +import com.google.protobuf.Descriptors.Descriptor; +import com.google.protobuf.Message; +import io.grpc.Status; +import io.grpc.Status.Code; +import java.io.IOException; +import java.time.Instant; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.function.Function; +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +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.io.gcp.bigquery.BigQueryServices.StreamAppendClient; +import org.apache.beam.sdk.io.gcp.bigquery.RetryManager.RetryType; +import org.apache.beam.sdk.io.gcp.bigquery.StorageApiDynamicDestinations.MessageConverter; +import org.apache.beam.sdk.io.gcp.bigquery.StorageApiFlushAndFinalizeDoFn.Operation; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Distribution; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.schemas.NoSuchSchemaException; +import org.apache.beam.sdk.schemas.SchemaCoder; +import org.apache.beam.sdk.schemas.SchemaRegistry; +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.transforms.Combine; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.Max; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; +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.TypeDescriptor; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Supplier; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.Cache; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheBuilder; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.RemovalNotification; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** A transform to write sharded records to BigQuery using the Storage API. */ +@SuppressWarnings("FutureReturnValueIgnored") +public class StorageApiWritesShardedRecords + extends PTransform< + PCollection, Iterable>>, PCollection> { + private static final Logger LOG = LoggerFactory.getLogger(StorageApiWritesShardedRecords.class); + + private final StorageApiDynamicDestinations dynamicDestinations; + private final CreateDisposition createDisposition; + private final String kmsKey; + private final BigQueryServices bqServices; + private final Coder destinationCoder; + @Nullable private DatasetService datasetServiceInternal = null; + private static final ExecutorService closeWriterExecutor = Executors.newCachedThreadPool(); + + private static final Cache APPEND_CLIENTS = + CacheBuilder.newBuilder() + .expireAfterAccess(15, TimeUnit.MINUTES) + .removalListener( + (RemovalNotification removal) -> { + @Nullable final StreamAppendClient streamAppendClient = removal.getValue(); + // Close the writer in a different thread so as not to block the main one. + runAsyncIgnoreFailure(closeWriterExecutor, streamAppendClient::close); + }) + .build(); + + private DatasetService getDatasetService(PipelineOptions pipelineOptions) throws IOException { + if (datasetServiceInternal == null) { + datasetServiceInternal = + bqServices.getDatasetService(pipelineOptions.as(BigQueryOptions.class)); + } + return datasetServiceInternal; + } + + // Run a closure asynchronously, ignoring failures. + private interface ThrowingRunnable { + void run() throws Exception; + } + + private static void runAsyncIgnoreFailure(ExecutorService executor, ThrowingRunnable task) { + executor.submit( + () -> { + try { + task.run(); + } catch (Exception e) { + // + } + }); + } + + public StorageApiWritesShardedRecords( + StorageApiDynamicDestinations dynamicDestinations, + CreateDisposition createDisposition, + String kmsKey, + BigQueryServices bqServices, + Coder destinationCoder) { + this.dynamicDestinations = dynamicDestinations; + this.createDisposition = createDisposition; + this.kmsKey = kmsKey; + this.bqServices = bqServices; + this.destinationCoder = destinationCoder; + } + + @Override + public PCollection expand( + PCollection, Iterable>> input) { + String operationName = input.getName() + "/" + getName(); + // Append records to the Storage API streams. + PCollection> written = + input.apply( + "Write Records", + ParDo.of(new WriteRecordsDoFn(operationName)) + .withSideInputs(dynamicDestinations.getSideInputs())); + + SchemaCoder operationCoder; + try { + SchemaRegistry schemaRegistry = input.getPipeline().getSchemaRegistry(); + operationCoder = + SchemaCoder.of( + schemaRegistry.getSchema(Operation.class), + TypeDescriptor.of(Operation.class), + schemaRegistry.getToRowFunction(Operation.class), + schemaRegistry.getFromRowFunction(Operation.class)); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + + // Send all successful writes to be flushed. + return written + .setCoder(KvCoder.of(StringUtf8Coder.of(), operationCoder)) + .apply( + Window.>configure() + .triggering( + Repeatedly.forever( + AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(Duration.standardSeconds(1)))) + .discardingFiredPanes()) + .apply("maxFlushPosition", Combine.perKey(Max.naturalOrder(new Operation(-1, false)))) + .apply( + "Flush and finalize writes", ParDo.of(new StorageApiFlushAndFinalizeDoFn(bqServices))); + } + + /** + * Takes in an iterable and batches the results into multiple ProtoRows objects. The splitSize + * parameter controls how many rows are batched into a single ProtoRows object before we move on + * to the next one. + */ + static class SplittingIterable implements Iterable { + private final Iterable underlying; + private final long splitSize; + + public SplittingIterable(Iterable underlying, long splitSize) { + this.underlying = underlying; + this.splitSize = splitSize; + } + + @Override + public Iterator iterator() { + return new Iterator() { + final Iterator underlyingIterator = underlying.iterator(); + + @Override + public boolean hasNext() { + return underlyingIterator.hasNext(); + } + + @Override + public ProtoRows next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + ProtoRows.Builder inserts = ProtoRows.newBuilder(); + long bytesSize = 0; + while (underlyingIterator.hasNext()) { + ByteString byteString = underlyingIterator.next().toByteString(); + inserts.addSerializedRows(byteString); + bytesSize += byteString.size(); + if (bytesSize > splitSize) { + break; + } + } + return inserts.build(); + } + }; + } + } + + class WriteRecordsDoFn + extends DoFn, Iterable>, KV> { + private final Counter recordsAppended = + Metrics.counter(WriteRecordsDoFn.class, "recordsAppended"); + private final Counter streamsCreated = + Metrics.counter(WriteRecordsDoFn.class, "streamsCreated"); + private final Counter appendFailures = + Metrics.counter(WriteRecordsDoFn.class, "appendFailures"); + private final Counter appendOffsetFailures = + Metrics.counter(WriteRecordsDoFn.class, "appendOffsetFailures"); + private final Counter flushesScheduled = + Metrics.counter(WriteRecordsDoFn.class, "flushesScheduled"); + private final Distribution appendLatencyDistribution = + Metrics.distribution(WriteRecordsDoFn.class, "appendLatencyDistributionMs"); + private final Distribution appendSizeDistribution = + Metrics.distribution(WriteRecordsDoFn.class, "appendSizeDistribution"); + private final Distribution appendSplitDistribution = + Metrics.distribution(WriteRecordsDoFn.class, "appendSplitDistribution"); + + private Map destinations = Maps.newHashMap(); + + private TwoLevelMessageConverterCache messageConverters; + + // Stores the current stream for this key. + @StateId("streamName") + private final StateSpec> streamNameSpec = StateSpecs.value(); + + // Stores the current stream offset. + @StateId("streamOffset") + private final StateSpec> streamOffsetSpec = StateSpecs.value(); + + public WriteRecordsDoFn(String operationName) { + this.messageConverters = new TwoLevelMessageConverterCache<>(operationName); + } + + @StartBundle + public void startBundle() throws IOException { + destinations = Maps.newHashMap(); + } + + // Get the current stream for this key. If there is no current stream, create one and store the + // stream name in + // persistent state. + @SuppressWarnings({"nullness"}) + String getOrCreateStream( + String tableId, + ValueState streamName, + ValueState streamOffset, + DatasetService datasetService) + throws IOException, InterruptedException { + String stream = streamName.read(); + if (Strings.isNullOrEmpty(stream)) { + // In a buffered stream, data is only visible up to the offset to which it was flushed. + stream = datasetService.createWriteStream(tableId, Type.BUFFERED).getName(); + streamName.write(stream); + streamOffset.write(0L); + streamsCreated.inc(); + } + return stream; + } + + @SuppressWarnings({"nullness"}) + @ProcessElement + public void process( + ProcessContext c, + final PipelineOptions pipelineOptions, + @Element KV, Iterable> element, + final @AlwaysFetched @StateId("streamName") ValueState streamName, + final @AlwaysFetched @StateId("streamOffset") ValueState streamOffset, + final OutputReceiver> o) + throws Exception { + dynamicDestinations.setSideInputAccessorFromProcessContext(c); + TableDestination tableDestination = + destinations.computeIfAbsent( + element.getKey().getKey(), + dest -> { + TableDestination tableDestination1 = dynamicDestinations.getTable(dest); + checkArgument( + tableDestination1 != null, + "DynamicDestinations.getTable() may not return null, " + + "but %s returned null for destination %s", + dynamicDestinations, + dest); + Supplier schemaSupplier = () -> dynamicDestinations.getSchema(dest); + return CreateTableHelpers.possiblyCreateTable( + c, + tableDestination1, + schemaSupplier, + createDisposition, + destinationCoder, + kmsKey, + bqServices); + }); + final String tableId = tableDestination.getTableUrn(); + final DatasetService datasetService = getDatasetService(pipelineOptions); + MessageConverter messageConverter = + messageConverters.get(element.getKey().getKey(), dynamicDestinations); + Descriptor descriptor = messageConverter.getSchemaDescriptor(); + // 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. + final long oneMb = 1024 * 1024; + Iterable messages = + new SplittingIterable<>( + Iterables.transform(element.getValue(), e -> messageConverter.toMessage(e)), oneMb); + + class AppendRowsContext extends RetryManager.Operation.Context { + final ShardedKey key; + String streamName = ""; + StreamAppendClient client = null; + long offset = -1; + long numRows = 0; + long tryIteration = 0; + + AppendRowsContext(ShardedKey key) { + this.key = key; + } + + @Override + public String toString() { + return "Context: key=" + + key + + " streamName=" + + streamName + + " offset=" + + offset + + " numRows=" + + numRows + + " tryIteration: " + + tryIteration; + } + }; + + // 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 = + (contexts, isFailure) -> { + try { + if (isFailure) { + // Clear the stream name, forcing a new one to be created. + streamName.write(""); + } + String stream = getOrCreateStream(tableId, streamName, streamOffset, datasetService); + StreamAppendClient appendClient = + APPEND_CLIENTS.get(stream, () -> datasetService.getStreamAppendClient(stream)); + for (AppendRowsContext context : contexts) { + context.streamName = stream; + appendClient.pin(); + context.client = appendClient; + context.offset = streamOffset.read(); + ++context.tryIteration; + streamOffset.write(context.offset + context.numRows); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + }; + + Consumer> clearClients = + contexts -> { + APPEND_CLIENTS.invalidate(streamName.read()); + 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); + context.client = null; + } + } + }; + + Instant now = Instant.now(); + List contexts = Lists.newArrayList(); + RetryManager retryManager = + new RetryManager<>(Duration.standardSeconds(1), Duration.standardMinutes(1), 1000); + int numSplits = 0; + for (ProtoRows protoRows : messages) { + ++numSplits; + Function> run = + context -> { + try { + StreamAppendClient appendClient = + APPEND_CLIENTS.get( + context.streamName, + () -> datasetService.getStreamAppendClient(context.streamName)); + return appendClient.appendRows(context.offset, protoRows, descriptor); + } catch (Exception e) { + throw new RuntimeException(e); + } + }; + + // RetryManager + Function, RetryType> onError = + failedContexts -> { + // The first context is always the one that fails. + AppendRowsContext failedContext = + Preconditions.checkNotNull(Iterables.getFirst(failedContexts, null)); + Status.Code statusCode = Status.fromThrowable(failedContext.getError()).getCode(); + // Invalidate the StreamWriter and force a new one to be created. + LOG.error( + "Got error " + failedContext.getError() + " closing " + failedContext.streamName); + clearClients.accept(contexts); + appendFailures.inc(); + if (statusCode.equals(Code.OUT_OF_RANGE) || statusCode.equals(Code.ALREADY_EXISTS)) { + appendOffsetFailures.inc(); + LOG.warn( + "Append to " + + failedContext + + " failed with " + + failedContext.getError() + + " Will retry with a new stream"); + // This means that the offset we have stored does not match the current end of + // the stream in the Storage API. Usually this happens because a crash or a bundle + // failure + // happened after an append but before the worker could checkpoint it's + // state. The records that were appended in a failed bundle will be retried, + // meaning that the unflushed tail of the stream must be discarded to prevent + // duplicates. + + // Finalize the stream and clear streamName so a new stream will be created. + o.output( + KV.of(failedContext.streamName, new Operation(failedContext.offset - 1, true))); + // Reinitialize all contexts with the new stream and new offsets. + initializeContexts.accept(failedContexts, true); + + // Offset failures imply that all subsequent parallel appends will also fail. + // Retry them all. + return RetryType.RETRY_ALL_OPERATIONS; + } + + return RetryType.RETRY_ALL_OPERATIONS; + }; + + Consumer onSuccess = + context -> { + o.output( + KV.of( + context.streamName, + new Operation(context.offset + context.numRows - 1, false))); + flushesScheduled.inc(protoRows.getSerializedRowsCount()); + }; + + AppendRowsContext context = new AppendRowsContext(element.getKey()); + context.numRows = protoRows.getSerializedRowsCount(); + contexts.add(context); + retryManager.addOperation(run, onError, onSuccess, context); + recordsAppended.inc(protoRows.getSerializedRowsCount()); + appendSizeDistribution.update(context.numRows); + } + initializeContexts.accept(contexts, false); + + try { + retryManager.run(true); + } finally { + // Make sure that all pins are removed. + for (AppendRowsContext context : contexts) { + if (context.client != null) { + runAsyncIgnoreFailure(closeWriterExecutor, context.client::unpin); + } + } + } + appendSplitDistribution.update(numSplits); + + java.time.Duration timeElapsed = java.time.Duration.between(now, Instant.now()); + appendLatencyDistribution.update(timeElapsed.toMillis()); + } + + @OnWindowExpiration + public void onWindowExpiration( + @AlwaysFetched @StateId("streamName") ValueState streamName, + @AlwaysFetched @StateId("streamOffset") ValueState streamOffset, + OutputReceiver> o) { + // Window is done - usually because the pipeline has been drained. Make sure to clean up + // streams so that they are not leaked. + String stream = MoreObjects.firstNonNull(streamName.read(), null); + + if (!Strings.isNullOrEmpty(stream)) { + // Finalize the stream + long nextOffset = MoreObjects.firstNonNull(streamOffset.read(), 0L); + o.output(KV.of(stream, new Operation(nextOffset - 1, true))); + // Make sure that the stream object is closed. + APPEND_CLIENTS.invalidate(stream); + } + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java index f8513502c5f7..f872a15581e3 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java @@ -119,10 +119,19 @@ public TableDestination withTableReference(TableReference tableReference) { tableReference, tableDescription, jsonTimePartitioning, jsonClustering); } + /** Return the tablespec in [project:].dataset.tableid format. */ public String getTableSpec() { return tableSpec; } + /** Return the tablespec in projects/>project>/datasets/datset/tables/table format. */ + public String getTableUrn() { + TableReference table = getTableReference(); + return String.format( + "projects/%s/datasets/%s/tables/%s", + table.getProjectId(), table.getDatasetId(), table.getTableId()); + } + public TableReference getTableReference() { return BigQueryHelpers.parseTableSpec(tableSpec); } 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 873832fe3ddb..710d47dd76e1 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 @@ -92,8 +92,13 @@ public static Descriptor getDescriptorFromTableSchema(TableSchema jsonSchema) public static DynamicMessage messageFromTableRow(Descriptor descriptor, TableRow tableRow) { DynamicMessage.Builder builder = DynamicMessage.newBuilder(descriptor); for (Map.Entry entry : tableRow.entrySet()) { + @Nullable FieldDescriptor fieldDescriptor = descriptor.findFieldByName(entry.getKey().toLowerCase()); - Object value = messageValueFromFieldValue(fieldDescriptor, entry.getValue()); + if (fieldDescriptor == null) { + throw new RuntimeException( + "TableRow contained unexpected field with name " + entry.getKey()); + } + @Nullable Object value = messageValueFromFieldValue(fieldDescriptor, entry.getValue()); if (value != null) { builder.setField(fieldDescriptor, value); } @@ -298,7 +303,7 @@ public static Object jsonValueFromMessageValue( case ENUM: throw new RuntimeException("Enumerations not supported"); default: - return fieldValue; + return fieldValue.toString(); } } } 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 new file mode 100644 index 000000000000..3e928ff29655 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TwoLevelMessageConverterCache.java @@ -0,0 +1,68 @@ +/* + * 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 java.io.Serializable; +import org.apache.beam.sdk.io.gcp.bigquery.StorageApiDynamicDestinations.MessageConverter; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.Cache; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheBuilder; + +/** + * A cache for {@link MessageConverter} objects. + * + *

There is an instance-level cache which we try to use to avoid hashing the entire operation + * name. However since this object is stored in DoFns and many DoFns share the same + * MessageConverters, we also store a static cache keyed by operation name. + */ +class TwoLevelMessageConverterCache implements Serializable { + final String operationName; + + TwoLevelMessageConverterCache(String operationName) { + this.operationName = operationName; + } + + // Cache MessageConverters since creating them can be expensive. Cache is keyed by transform name + // and the destination. + @SuppressWarnings({"nullness"}) + private static final Cache, MessageConverter> CACHED_MESSAGE_CONVERTERS = + CacheBuilder.newBuilder().expireAfterAccess(java.time.Duration.ofMinutes(15)).build(); + + // Keep an instance-level cache of MessageConverter objects. This allows us to avoid hashing the + // entire operation name + // on every element. Since there will be multiple DoFn instances (and they may periodically be + // recreated), we + // still need the static cache to allow reuse. + @SuppressWarnings({"nullness"}) + private final Cache> localMessageConverters = + CacheBuilder.newBuilder().expireAfterAccess(java.time.Duration.ofMinutes(15)).build(); + + public MessageConverter get( + DestinationT destination, + StorageApiDynamicDestinations dynamicDestinations) + throws Exception { + // Lookup first in the local cache, and fall back to the static cache if necessary. + return localMessageConverters.get( + destination, + () -> + (MessageConverter) + CACHED_MESSAGE_CONVERTERS.get( + KV.of(operationName, destination), + () -> dynamicDestinations.getMessageConverter(destination))); + } +} 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 bfd6418f2165..b24bf1642223 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 @@ -21,33 +21,50 @@ import com.google.api.client.googleapis.json.GoogleJsonResponseException; import com.google.api.client.http.HttpHeaders; +import com.google.api.core.ApiFuture; +import com.google.api.core.ApiFutures; import com.google.api.services.bigquery.model.Dataset; 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.TableReference; import com.google.api.services.bigquery.model.TableRow; +import com.google.cloud.bigquery.storage.v1beta2.AppendRowsResponse; +import com.google.cloud.bigquery.storage.v1beta2.BatchCommitWriteStreamsResponse; +import com.google.cloud.bigquery.storage.v1beta2.FinalizeWriteStreamResponse; +import com.google.cloud.bigquery.storage.v1beta2.FlushRowsResponse; +import com.google.cloud.bigquery.storage.v1beta2.ProtoRows; +import com.google.cloud.bigquery.storage.v1beta2.WriteStream; +import com.google.cloud.bigquery.storage.v1beta2.WriteStream.Type; +import com.google.protobuf.ByteString; +import com.google.protobuf.Descriptors.Descriptor; +import com.google.protobuf.DynamicMessage; +import com.google.protobuf.Timestamp; import java.io.IOException; import java.io.Serializable; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; import java.util.regex.Pattern; +import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.StreamAppendClient; import org.apache.beam.sdk.io.gcp.bigquery.ErrorContainer; import org.apache.beam.sdk.io.gcp.bigquery.InsertRetryPolicy; import org.apache.beam.sdk.io.gcp.bigquery.InsertRetryPolicy.Context; +import org.apache.beam.sdk.io.gcp.bigquery.TableRowToStorageApiProto; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.values.FailsafeValueInSingleWindow; import org.apache.beam.sdk.values.ValueInSingleWindow; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.HashBasedTable; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps; -import org.checkerframework.checker.nullness.qual.Nullable; /** A fake dataset service that can be serialized, for use in testReadFromTable. */ @Internal @@ -61,6 +78,58 @@ public class FakeDatasetService implements DatasetService, Serializable { String, String, Map> tables; + static Map writeStreams; + + static class Stream { + final List stream; + final TableContainer tableContainer; + final Type type; + long nextFlushPosition; + boolean finalized; + + Stream(TableContainer tableContainer, Type type) { + this.stream = Lists.newArrayList(); + this.tableContainer = tableContainer; + this.type = type; + this.finalized = false; + this.nextFlushPosition = 0; + } + + long finalizeStream() { + this.finalized = true; + return stream.size(); + } + + void appendRows(long position, List rowsToAppend) { + if (finalized) { + throw new RuntimeException("Stream already finalized."); + } + if (position != -1 && position != stream.size()) { + throw new RuntimeException("Bad append: " + position); + } + stream.addAll(rowsToAppend); + } + + void flush(long position) { + Preconditions.checkState(type == Type.BUFFERED); + Preconditions.checkState(!finalized); + if (position >= stream.size()) { + throw new RuntimeException(""); + } + for (; nextFlushPosition <= position; ++nextFlushPosition) { + tableContainer.addRow(stream.get((int) nextFlushPosition), ""); + } + } + + void commit() { + if (!finalized) { + throw new RuntimeException("Can't commit unfinalized stream."); + } + Preconditions.checkState(type == Type.PENDING); + stream.forEach(tr -> tableContainer.addRow(tr, null)); + } + } + Map> insertErrors = Maps.newHashMap(); // The counter for the number of insertions performed. @@ -69,6 +138,7 @@ public class FakeDatasetService implements DatasetService, Serializable { public static void setUp() { tables = HashBasedTable.create(); insertCount = new AtomicInteger(0); + writeStreams = Maps.newHashMap(); FakeJobService.setUp(); } @@ -92,6 +162,7 @@ public Table getTable(TableReference tableRef, @Nullable List selectedFi tableRef.getProjectId(), tableRef.getDatasetId()); } TableContainer tableContainer = dataset.get(tableRef.getTableId()); + return tableContainer == null ? null : tableContainer.getTable(); } } @@ -99,6 +170,7 @@ public Table getTable(TableReference tableRef, @Nullable List selectedFi public List getAllRows(String projectId, String datasetId, String tableId) throws InterruptedException, IOException { synchronized (tables) { + TableContainer tableContainer = getTableContainer(projectId, datasetId, tableId); return getTableContainer(projectId, datasetId, tableId).getRows(); } } @@ -322,6 +394,103 @@ public Table patchTableDescription( } } + @Override + public WriteStream createWriteStream(String tableUrn, Type type) + throws IOException, InterruptedException { + if (type != Type.PENDING && type != Type.BUFFERED) { + throw new RuntimeException("We only support PENDING or BUFFERED streams."); + } + TableReference tableReference = + BigQueryHelpers.parseTableUrn(BigQueryHelpers.stripPartitionDecorator(tableUrn)); + synchronized (tables) { + TableContainer tableContainer = + getTableContainer( + tableReference.getProjectId(), + tableReference.getDatasetId(), + tableReference.getTableId()); + String streamName = UUID.randomUUID().toString(); + writeStreams.put(streamName, new Stream(tableContainer, type)); + return WriteStream.newBuilder().setName(streamName).build(); + } + } + + @Override + public StreamAppendClient getStreamAppendClient(String streamName) { + return new StreamAppendClient() { + @Override + public ApiFuture appendRows( + long offset, ProtoRows rows, Descriptor descriptor) throws Exception { + synchronized (tables) { + Stream stream = writeStreams.get(streamName); + if (stream == null) { + throw new RuntimeException("No such stream: " + streamName); + } + List tableRows = + Lists.newArrayListWithExpectedSize(rows.getSerializedRowsCount()); + for (ByteString bytes : rows.getSerializedRowsList()) { + tableRows.add( + TableRowToStorageApiProto.tableRowFromMessage( + DynamicMessage.parseFrom(descriptor, bytes))); + } + stream.appendRows(offset, tableRows); + } + return ApiFutures.immediateFuture(AppendRowsResponse.newBuilder().build()); + } + + @Override + public void close() throws Exception {} + + @Override + public void pin() {} + + @Override + public void unpin() throws Exception {} + }; + } + + @Override + public ApiFuture flush(String streamName, long offset) { + synchronized (tables) { + Stream stream = writeStreams.get(streamName); + if (stream == null) { + throw new RuntimeException("No such stream: " + streamName); + } + stream.flush(offset); + } + return ApiFutures.immediateFuture(FlushRowsResponse.newBuilder().build()); + } + + @Override + public ApiFuture finalizeWriteStream(String streamName) { + synchronized (tables) { + Stream stream = writeStreams.get(streamName); + if (stream == null) { + throw new RuntimeException("No such stream: " + streamName); + } + long numRows = stream.finalizeStream(); + return ApiFutures.immediateFuture( + FinalizeWriteStreamResponse.newBuilder().setRowCount(numRows).build()); + } + } + + @Override + public ApiFuture commitWriteStreams( + String tableUrn, Iterable writeStreamNames) { + synchronized (tables) { + for (String streamName : writeStreamNames) { + Stream stream = writeStreams.get(streamName); + if (stream == null) { + throw new RuntimeException("No such stream: " + streamName); + } + stream.commit(); + } + } + return ApiFutures.immediateFuture( + BatchCommitWriteStreamsResponse.newBuilder() + .setCommitTime(Timestamp.newBuilder().build()) + .build()); + } + /** * Cause a given {@link TableRow} object to fail when it's inserted. The errors link the list will * be returned on subsequent retries, and the insert will succeed when the errors run out. diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java index e2673f72d9b1..685bee3a84d9 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java @@ -65,6 +65,7 @@ public void testGcpApiSurface() throws Exception { classesInPackage("com.google.auth"), classesInPackage("com.google.bigtable.v2"), classesInPackage("com.google.cloud.bigquery.storage.v1"), + classesInPackage("com.google.cloud.bigquery.storage.v1beta2"), classesInPackage("com.google.cloud.bigtable.config"), classesInPackage("com.google.spanner.v1"), classesInPackage("com.google.pubsub.v1"), @@ -89,6 +90,7 @@ public void testGcpApiSurface() throws Exception { classesInPackage("com.google.cloud.spanner"), classesInPackage("com.google.datastore.v1"), classesInPackage("com.google.protobuf"), + classesInPackage("com.google.rpc"), classesInPackage("com.google.type"), classesInPackage("com.fasterxml.jackson.annotation"), classesInPackage("com.fasterxml.jackson.core"), 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 8a0dc2a52d33..59a5bc85b6a5 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 @@ -135,16 +135,36 @@ import org.junit.rules.TestRule; import org.junit.runner.Description; import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameter; +import org.junit.runners.Parameterized.Parameters; import org.junit.runners.model.Statement; /** Tests for {@link BigQueryIO#write}. */ -@RunWith(JUnit4.class) +@RunWith(Parameterized.class) +@SuppressWarnings({ + "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) +}) public class BigQueryIOWriteTest implements Serializable { private transient PipelineOptions options; private transient TemporaryFolder testFolder = new TemporaryFolder(); private transient TestPipeline p; + @Parameters + public static Iterable data() { + return ImmutableList.of( + new Object[] {false, false}, + new Object[] {false, true}, + new Object[] {true, false}, + new Object[] {true, true}); + } + + @Parameter(0) + public boolean useStorageApi; + + @Parameter(1) + public boolean useStreaming; + @Rule public final transient TestRule folderThenPipeline = new TestRule() { @@ -159,10 +179,16 @@ public Statement apply(final Statement base, final Description description) { @Override public void evaluate() throws Throwable { options = TestPipeline.testingPipelineOptions(); - options.as(BigQueryOptions.class).setProject("project-id"); - options - .as(BigQueryOptions.class) - .setTempLocation(testFolder.getRoot().getAbsolutePath()); + BigQueryOptions bqOptions = options.as(BigQueryOptions.class); + bqOptions.setProject("project-id"); + bqOptions.setTempLocation(testFolder.getRoot().getAbsolutePath()); + if (useStorageApi) { + bqOptions.setUseStorageWriteApi(true); + if (useStreaming) { + bqOptions.setNumStorageWriteApiStreams(2); + bqOptions.setStorageWriteApiTriggeringFrequencySec(1); + } + } p = TestPipeline.fromOptions(options); p.apply(base, description).evaluate(); } @@ -199,6 +225,9 @@ abstract static class StringLongDestinations extends DynamicDestinations(arg -> arg))); - if (streaming) { + if (useStreaming) { users = users.setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); } @@ -398,11 +422,8 @@ void testTimePartitioningClustering( new TableSchema() .setFields( ImmutableList.of( - new TableFieldSchema() - .setName("date") - .setType("DATE") - .setName("number") - .setType("INTEGER"))); + new TableFieldSchema().setName("date").setType("DATE"), + new TableFieldSchema().setName("number").setType("INTEGER"))); Write writeTransform = BigQueryIO.writeTableRows() @@ -443,27 +464,38 @@ void testClustering(BigQueryIO.Write.Method insertMethod) throws Exception { } @Test - public void testTimePartitioningStreamingInserts() throws Exception { - testTimePartitioning(BigQueryIO.Write.Method.STREAMING_INSERTS); - } - - @Test - public void testTimePartitioningBatchLoads() throws Exception { - testTimePartitioning(BigQueryIO.Write.Method.FILE_LOADS); + public void testTimePartitioning() throws Exception { + BigQueryIO.Write.Method method; + if (useStorageApi) { + method = Method.STORAGE_WRITE_API; + } else if (useStreaming) { + method = Method.STREAMING_INSERTS; + } else { + method = Method.FILE_LOADS; + } + testTimePartitioning(method); } @Test - public void testClusteringStreamingInserts() throws Exception { - testClustering(BigQueryIO.Write.Method.STREAMING_INSERTS); + public void testTimePartitioningStorageApi() throws Exception { + if (!useStorageApi) { + return; + } + testTimePartitioning(Method.STORAGE_WRITE_API); } @Test - public void testClusteringBatchLoads() throws Exception { - testClustering(BigQueryIO.Write.Method.FILE_LOADS); + public void testClusteringStorageApi() throws Exception { + if (useStorageApi) { + testClustering(Method.STORAGE_WRITE_API); + } } @Test(expected = IllegalArgumentException.class) public void testClusteringThrowsWithoutPartitioning() throws Exception { + if (useStorageApi || !useStreaming) { + throw new IllegalArgumentException(); + } p.enableAbandonedNodeEnforcement(false); testTimePartitioningClustering(Method.STREAMING_INSERTS, false, true); } @@ -479,11 +511,12 @@ public void testClusteringTableFunction() throws Exception { new TableSchema() .setFields( ImmutableList.of( - new TableFieldSchema() - .setName("date") - .setType("DATE") - .setName("number") - .setType("INTEGER"))); + new TableFieldSchema().setName("date").setType("DATE"), + new TableFieldSchema().setName("number").setType("INTEGER"))); + + // withMethod overrides the pipeline option, so we need to explicitly request + // STORAGE_API_WRITES. + BigQueryIO.Write.Method method = useStorageApi ? Method.STORAGE_WRITE_API : Method.FILE_LOADS; p.apply(Create.of(row1, row2)) .apply( BigQueryIO.writeTableRows() @@ -498,7 +531,7 @@ public void testClusteringTableFunction() throws Exception { new Clustering().setFields(ImmutableList.of("date"))); }) .withTestServices(fakeBqServices) - .withMethod(BigQueryIO.Write.Method.FILE_LOADS) + .withMethod(method) .withSchema(schema) .withClustering() .withoutValidation()); @@ -513,6 +546,9 @@ public void testClusteringTableFunction() throws Exception { @Test public void testTriggeredFileLoads() throws Exception { + if (useStorageApi || !useStreaming) { + return; + } List elements = Lists.newArrayList(); for (int i = 0; i < 30; ++i) { elements.add(new TableRow().set("number", i)); @@ -530,6 +566,7 @@ public void testTriggeredFileLoads() throws Exception { elements.get(20), Iterables.toArray(elements.subList(21, 30), TableRow.class)) .advanceWatermarkToInfinity(); + BigQueryIO.Write.Method method = Method.FILE_LOADS; p.apply(testStream) .apply( BigQueryIO.writeTableRows() @@ -542,7 +579,7 @@ public void testTriggeredFileLoads() throws Exception { .withTestServices(fakeBqServices) .withTriggeringFrequency(Duration.standardSeconds(30)) .withNumFileShards(2) - .withMethod(BigQueryIO.Write.Method.FILE_LOADS) + .withMethod(method) .withoutValidation()); p.run(); @@ -552,6 +589,9 @@ public void testTriggeredFileLoads() throws Exception { } public void testTriggeredFileLoadsWithTempTables(String tableRef) throws Exception { + if (useStorageApi || !useStreaming) { + return; + } List elements = Lists.newArrayList(); for (int i = 0; i < 30; ++i) { elements.add(new TableRow().set("number", i)); @@ -569,6 +609,7 @@ public void testTriggeredFileLoadsWithTempTables(String tableRef) throws Excepti elements.get(20), Iterables.toArray(elements.subList(21, 30), TableRow.class)) .advanceWatermarkToInfinity(); + BigQueryIO.Write.Method method = Method.FILE_LOADS; p.apply(testStream) .apply( BigQueryIO.writeTableRows() @@ -583,7 +624,7 @@ public void testTriggeredFileLoadsWithTempTables(String tableRef) throws Excepti .withNumFileShards(2) .withMaxBytesPerPartition(1) .withMaxFilesPerPartition(1) - .withMethod(BigQueryIO.Write.Method.FILE_LOADS) + .withMethod(method) .withoutValidation()); p.run(); @@ -604,6 +645,10 @@ public void testTriggeredFileLoadsWithTempTablesDefaultProject() throws Exceptio @Test public void testTriggeredFileLoadsWithAutoSharding() throws Exception { + if (useStorageApi || !useStreaming) { + // This test does not make sense for the storage API. + return; + } List elements = Lists.newArrayList(); for (int i = 0; i < 30; ++i) { elements.add(new TableRow().set("number", i)); @@ -672,6 +717,9 @@ public void testTriggeredFileLoadsWithAutoSharding() throws Exception { @Test public void testFailuresNoRetryPolicy() throws Exception { + if (useStorageApi || !useStreaming) { + return; + } TableRow row1 = new TableRow().set("name", "a").set("number", "1"); TableRow row2 = new TableRow().set("name", "b").set("number", "2"); TableRow row3 = new TableRow().set("name", "c").set("number", "3"); @@ -708,6 +756,9 @@ public void testFailuresNoRetryPolicy() throws Exception { @Test public void testRetryPolicy() throws Exception { + if (useStorageApi || !useStreaming) { + return; + } TableRow row1 = new TableRow().set("name", "a").set("number", "1"); TableRow row2 = new TableRow().set("name", "b").set("number", "2"); TableRow row3 = new TableRow().set("name", "c").set("number", "3"); @@ -777,6 +828,9 @@ public void testWrite() throws Exception { @Test public void testWriteWithoutInsertId() throws Exception { + if (useStorageApi || !useStreaming) { + return; + } TableRow row1 = new TableRow().set("name", "a").set("number", 1); TableRow row2 = new TableRow().set("name", "b").set("number", 2); TableRow row3 = new TableRow().set("name", "c").set("number", 3); @@ -827,6 +881,9 @@ public static InputRecord create( @Test public void testWriteAvro() throws Exception { + if (useStorageApi || useStreaming) { + return; + } p.apply( Create.of( InputRecord.create("test", 1, 1.0, Instant.parse("2019-01-01T00:00:00Z")), @@ -875,6 +932,9 @@ public void testWriteAvro() throws Exception { @Test public void testWriteAvroWithCustomWriter() throws Exception { + if (useStorageApi || useStreaming) { + return; + } SerializableFunction, GenericRecord> formatFunction = r -> { GenericRecord rec = new GenericData.Record(r.getSchema()); @@ -940,10 +1000,16 @@ public void testStreamingWrite() throws Exception { @Test public void testStreamingWriteWithAutoSharding() throws Exception { + if (useStorageApi) { + return; + } streamingWrite(true); } private void streamingWrite(boolean autoSharding) throws Exception { + if (!useStreaming) { + return; + } BigQueryIO.Write write = BigQueryIO.writeTableRows() .to("project-id:dataset-id.table-id") @@ -961,22 +1027,22 @@ private void streamingWrite(boolean autoSharding) throws Exception { } p.apply( Create.of( - new TableRow().set("name", "a").set("number", 1), - new TableRow().set("name", "b").set("number", 2), - new TableRow().set("name", "c").set("number", 3), - new TableRow().set("name", "d").set("number", 4)) + new TableRow().set("name", "a").set("number", "1"), + new TableRow().set("name", "b").set("number", "2"), + new TableRow().set("name", "c").set("number", "3"), + new TableRow().set("name", "d").set("number", "4")) .withCoder(TableRowJsonCoder.of())) .setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED) - .apply(write); + .apply("WriteToBQ", write); p.run(); assertThat( fakeDatasetService.getAllRows("project-id", "dataset-id", "table-id"), containsInAnyOrder( - new TableRow().set("name", "a").set("number", 1), - new TableRow().set("name", "b").set("number", 2), - new TableRow().set("name", "c").set("number", 3), - new TableRow().set("name", "d").set("number", 4))); + new TableRow().set("name", "a").set("number", "1"), + new TableRow().set("name", "b").set("number", "2"), + new TableRow().set("name", "c").set("number", "3"), + new TableRow().set("name", "d").set("number", "4"))); } @DefaultSchema(JavaFieldSchema.class) @@ -993,6 +1059,9 @@ static class SchemaPojo { @Test public void testSchemaWriteLoads() throws Exception { + // withMethod overrides the pipeline option, so we need to explicitly request + // STORAGE_API_WRITES. + BigQueryIO.Write.Method method = useStorageApi ? Method.STORAGE_WRITE_API : Method.FILE_LOADS; p.apply( Create.of( new SchemaPojo("a", 1), @@ -1003,12 +1072,14 @@ public void testSchemaWriteLoads() throws Exception { BigQueryIO.write() .to("project-id:dataset-id.table-id") .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) - .withMethod(Method.FILE_LOADS) + .withMethod(method) .useBeamSchema() .withTestServices(fakeBqServices) .withoutValidation()); p.run(); + System.err.println( + "Wrote: " + fakeDatasetService.getAllRows("project-id", "dataset-id", "table-id")); assertThat( fakeDatasetService.getAllRows("project-id", "dataset-id", "table-id"), containsInAnyOrder( @@ -1020,6 +1091,9 @@ public void testSchemaWriteLoads() throws Exception { @Test public void testSchemaWriteStreams() throws Exception { + if (useStorageApi || !useStreaming) { + return; + } p.apply( Create.of( new SchemaPojo("a", 1), @@ -1155,16 +1229,7 @@ public void verifyDeterministic() {} } @Test - public void testStreamingWriteWithDynamicTables() throws Exception { - testWriteWithDynamicTables(true); - } - - @Test - public void testBatchWriteWithDynamicTables() throws Exception { - testWriteWithDynamicTables(false); - } - - public void testWriteWithDynamicTables(boolean streaming) throws Exception { + public void testWriteWithDynamicTables() throws Exception { List inserts = new ArrayList<>(); for (int i = 0; i < 10; i++) { inserts.add(i); @@ -1203,7 +1268,7 @@ public void testWriteWithDynamicTables(boolean streaming) throws Exception { }; PCollection input = p.apply("CreateSource", Create.of(inserts)); - if (streaming) { + if (useStreaming) { input = input.setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); } @@ -1215,7 +1280,9 @@ public void testWriteWithDynamicTables(boolean streaming) throws Exception { .apply( BigQueryIO.write() .to(tableFunction) - .withFormatFunction(i -> new TableRow().set("name", "number" + i).set("number", i)) + .withFormatFunction( + i -> + new TableRow().set("name", "number" + i).set("number", Integer.toString(i))) .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) .withSchemaFromView(schemasView) .withTestServices(fakeBqServices) @@ -1242,13 +1309,20 @@ public void testWriteWithDynamicTables(boolean streaming) throws Exception { assertThat( fakeDatasetService.getAllRows("project-id", "dataset-id", tableId), containsInAnyOrder( - new TableRow().set("name", String.format("number%d", i)).set("number", i), - new TableRow().set("name", String.format("number%d", i + 5)).set("number", i + 5))); + new TableRow() + .set("name", String.format("number%d", i)) + .set("number", Integer.toString(i)), + new TableRow() + .set("name", String.format("number%d", i + 5)) + .set("number", Integer.toString(i + 5)))); } } @Test public void testWriteUnknown() throws Exception { + if (useStorageApi) { + return; + } p.apply( Create.of( new TableRow().set("name", "a").set("number", 1), @@ -1269,6 +1343,9 @@ public void testWriteUnknown() throws Exception { @Test public void testWriteFailedJobs() throws Exception { + if (useStorageApi) { + return; + } p.apply( Create.of( new TableRow().set("name", "a").set("number", 1), @@ -1292,10 +1369,16 @@ public void testWriteFailedJobs() throws Exception { @Test public void testWriteWithMissingSchemaFromView() throws Exception { + // Because no messages PCollectionView> view = p.apply("Create schema view", Create.of(KV.of("foo", "bar"), KV.of("bar", "boo"))) .apply(View.asMap()); - p.apply(Create.empty(TableRowJsonCoder.of())) + p.apply( + Create.of( + new TableRow().set("name", "a").set("number", 1), + new TableRow().set("name", "b").set("number", 2), + new TableRow().set("name", "c").set("number", 3)) + .withCoder(TableRowJsonCoder.of())) .apply( BigQueryIO.writeTableRows() .to("dataset-id.table-id") @@ -1405,13 +1488,14 @@ public void testBuildWriteDisplayData() { assertThat(displayData, hasDisplayItem("validation", false)); } - private void testWriteValidatesDataset(boolean unbounded) throws Exception { + @Test + public void testWriteValidatesDataset() throws Exception { TableReference tableRef = new TableReference(); tableRef.setDatasetId("somedataset"); tableRef.setTableId("sometable"); PCollection tableRows; - if (unbounded) { + if (useStreaming) { tableRows = p.apply(GenerateSequence.from(0)) .apply( @@ -1441,18 +1525,11 @@ public TableRow apply(Long input) { p.run(); } - @Test - public void testWriteValidatesDatasetBatch() throws Exception { - testWriteValidatesDataset(false); - } - - @Test - public void testWriteValidatesDatasetStreaming() throws Exception { - testWriteValidatesDataset(true); - } - @Test public void testCreateNeverWithStreaming() throws Exception { + if (!useStreaming) { + return; + } p.enableAbandonedNodeEnforcement(false); TableReference tableRef = new TableReference(); @@ -1513,6 +1590,9 @@ public void testWriteValidateFailsNoFormatFunction() { @Test public void testWriteValidateFailsBothFormatFunctions() { + if (useStorageApi) { + return; + } p.enableAbandonedNodeEnforcement(false); thrown.expect(IllegalArgumentException.class); @@ -1530,6 +1610,9 @@ public void testWriteValidateFailsBothFormatFunctions() { @Test public void testWriteValidateFailsWithBeamSchemaAndAvroFormatFunction() { + if (useStorageApi) { + return; + } p.enableAbandonedNodeEnforcement(false); thrown.expect(IllegalArgumentException.class); @@ -1545,6 +1628,9 @@ public void testWriteValidateFailsWithBeamSchemaAndAvroFormatFunction() { @Test public void testWriteValidateFailsWithAvroFormatAndStreamingInserts() { + if (!useStreaming && !useStorageApi) { + return; + } p.enableAbandonedNodeEnforcement(false); thrown.expect(IllegalArgumentException.class); @@ -1561,6 +1647,9 @@ public void testWriteValidateFailsWithAvroFormatAndStreamingInserts() { @Test public void testWriteValidateFailsWithBatchAutoSharding() { + if (useStorageApi) { + return; + } p.enableAbandonedNodeEnforcement(false); thrown.expect(IllegalArgumentException.class); @@ -1968,16 +2057,22 @@ public void testWriteToTableDecorator() throws Exception { TableRow row1 = new TableRow().set("name", "a").set("number", "1"); TableRow row2 = new TableRow().set("name", "b").set("number", "2"); + // withMethod overrides the pipeline option, so we need to explicitly requiest + // STORAGE_API_WRITES. + BigQueryIO.Write.Method method = + useStorageApi ? Method.STORAGE_WRITE_API : Method.STREAMING_INSERTS; TableSchema schema = new TableSchema() .setFields( - ImmutableList.of(new TableFieldSchema().setName("number").setType("INTEGER"))); + ImmutableList.of( + new TableFieldSchema().setName("name").setType("STRING"), + new TableFieldSchema().setName("number").setType("INTEGER"))); p.apply(Create.of(row1, row2)) .apply( BigQueryIO.writeTableRows() .to("project-id:dataset-id.table-id$20171127") .withTestServices(fakeBqServices) - .withMethod(BigQueryIO.Write.Method.STREAMING_INSERTS) + .withMethod(method) .withSchema(schema) .withoutValidation()); p.run(); @@ -1985,6 +2080,9 @@ public void testWriteToTableDecorator() throws Exception { @Test public void testExtendedErrorRetrieval() throws Exception { + if (useStorageApi) { + return; + } TableRow row1 = new TableRow().set("name", "a").set("number", "1"); TableRow row2 = new TableRow().set("name", "b").set("number", "2"); TableRow row3 = new TableRow().set("name", "c").set("number", "3"); @@ -2037,6 +2135,9 @@ public void testExtendedErrorRetrieval() throws Exception { @Test public void testWrongErrorConfigs() { + if (useStorageApi) { + return; + } p.enableAutoRunIfMissing(true); TableRow row1 = new TableRow().set("name", "a").set("number", "1"); @@ -2107,7 +2208,8 @@ void schemaUpdateOptionsTest( .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND) .withSchemaUpdateOptions(schemaUpdateOptions); - p.apply(Create.of(row)).apply(writeTransform); + p.apply("Create" + insertMethod, Create.of(row)) + .apply("Write" + insertMethod, writeTransform); p.run(); List expectedOptions = @@ -2139,6 +2241,9 @@ public void testWriteFileSchemaUpdateOptionAll() throws Exception { @Test public void testSchemaUpdateOptionsFailsStreamingInserts() throws Exception { + if (!useStreaming && !useStorageApi) { + return; + } Set options = EnumSet.of(SchemaUpdateOption.ALLOW_FIELD_ADDITION); p.enableAbandonedNodeEnforcement(false); thrown.expect(IllegalArgumentException.class); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java index 17dd474db438..4877d8665220 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java @@ -339,7 +339,8 @@ public void testGetTableSucceeds() throws Exception { .thenReturn(toStream(testTable)); BigQueryServicesImpl.DatasetServiceImpl datasetService = - new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + new BigQueryServicesImpl.DatasetServiceImpl( + bigquery, null, PipelineOptionsFactory.create()); Table table = datasetService.getTable(tableRef, null, BackOff.ZERO_BACKOFF, Sleeper.DEFAULT); @@ -355,7 +356,8 @@ public void testGetTableNotFound() throws IOException, InterruptedException { when(response.getStatusCode()).thenReturn(404); BigQueryServicesImpl.DatasetServiceImpl datasetService = - new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + new BigQueryServicesImpl.DatasetServiceImpl( + bigquery, null, PipelineOptionsFactory.create()); TableReference tableRef = new TableReference() @@ -385,7 +387,8 @@ public void testGetTableThrows() throws Exception { thrown.expectMessage(String.format("Unable to get table: %s", tableRef.getTableId())); BigQueryServicesImpl.DatasetServiceImpl datasetService = - new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + new BigQueryServicesImpl.DatasetServiceImpl( + bigquery, null, PipelineOptionsFactory.create()); datasetService.getTable(tableRef, null, BackOff.STOP_BACKOFF, Sleeper.DEFAULT); } @@ -407,7 +410,8 @@ public void testIsTableEmptySucceeds() throws Exception { .thenReturn(toStream(testDataList)); BigQueryServicesImpl.DatasetServiceImpl datasetService = - new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + new BigQueryServicesImpl.DatasetServiceImpl( + bigquery, null, PipelineOptionsFactory.create()); assertFalse(datasetService.isTableEmpty(tableRef, BackOff.ZERO_BACKOFF, Sleeper.DEFAULT)); @@ -422,7 +426,8 @@ public void testIsTableEmptyNoRetryForNotFound() throws IOException, Interrupted when(response.getStatusCode()).thenReturn(404); BigQueryServicesImpl.DatasetServiceImpl datasetService = - new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + new BigQueryServicesImpl.DatasetServiceImpl( + bigquery, null, PipelineOptionsFactory.create()); TableReference tableRef = new TableReference() @@ -454,7 +459,8 @@ public void testIsTableEmptyThrows() throws Exception { .setTableId("tableId"); BigQueryServicesImpl.DatasetServiceImpl datasetService = - new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + new BigQueryServicesImpl.DatasetServiceImpl( + bigquery, null, PipelineOptionsFactory.create()); thrown.expect(IOException.class); thrown.expectMessage(String.format("Unable to list table data: %s", tableRef.getTableId())); @@ -517,7 +523,7 @@ public void testInsertRateLimitRetry() throws Exception { .thenReturn(toStream(new TableDataInsertAllResponse())); DatasetServiceImpl dataService = - new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + new DatasetServiceImpl(bigquery, null, PipelineOptionsFactory.create()); dataService.insertAll( ref, rows, @@ -553,7 +559,7 @@ public void testInsertQuotaExceededRetry() throws Exception { .thenReturn(toStream(new TableDataInsertAllResponse())); DatasetServiceImpl dataService = - new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + new DatasetServiceImpl(bigquery, null, PipelineOptionsFactory.create()); dataService.insertAll( ref, rows, @@ -599,7 +605,7 @@ public void testInsertStoppedRetry() throws Exception { thrown.expectMessage("quotaExceeded"); DatasetServiceImpl dataService = - new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + new DatasetServiceImpl(bigquery, null, PipelineOptionsFactory.create()); dataService.insertAll( ref, rows, @@ -650,7 +656,7 @@ public void testInsertRetrySelectRows() throws Exception { .thenReturn(toStream(allRowsSucceeded)); DatasetServiceImpl dataService = - new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + new DatasetServiceImpl(bigquery, null, PipelineOptionsFactory.create()); dataService.insertAll( ref, rows, @@ -694,7 +700,7 @@ public void testInsertFailsGracefully() throws Exception { .thenAnswer(invocation -> toStream(row0Failed)); DatasetServiceImpl dataService = - new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + new DatasetServiceImpl(bigquery, null, PipelineOptionsFactory.create()); // Expect it to fail. try { @@ -745,7 +751,7 @@ public void testFailInsertOtherRetry() throws Exception { .thenReturn(toStream(errorWithReasonAndStatus("actually forbidden", 403))) .thenReturn(toStream(new TableDataInsertAllResponse())); DatasetServiceImpl dataService = - new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + new DatasetServiceImpl(bigquery, null, PipelineOptionsFactory.create()); thrown.expect(RuntimeException.class); thrown.expectMessage("actually forbidden"); try { @@ -817,7 +823,7 @@ public void testInsertRetryPolicy() throws InterruptedException, IOException { .thenReturn(toStream(allRowsSucceeded)); DatasetServiceImpl dataService = - new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + new DatasetServiceImpl(bigquery, null, PipelineOptionsFactory.create()); List> failedInserts = Lists.newArrayList(); dataService.insertAll( @@ -859,7 +865,7 @@ public void testSkipInvalidRowsIgnoreUnknownIgnoreInsertIdsValuesStreaming() .thenReturn(toStream(allRowsSucceeded)); DatasetServiceImpl dataService = - new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + new DatasetServiceImpl(bigquery, null, PipelineOptionsFactory.create()); // First, test with all flags disabled dataService.insertAll( @@ -935,7 +941,7 @@ private static GoogleJsonErrorContainer errorWithReasonAndStatus(String reason, @Test public void testGetErrorInfo() throws IOException { DatasetServiceImpl dataService = - new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + new DatasetServiceImpl(bigquery, null, PipelineOptionsFactory.create()); ErrorInfo info = new ErrorInfo(); List infoList = new ArrayList<>(); infoList.add(info); @@ -959,7 +965,8 @@ public void testCreateTableSucceeds() throws IOException { when(response.getContent()).thenReturn(toStream(testTable)); BigQueryServicesImpl.DatasetServiceImpl services = - new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + new BigQueryServicesImpl.DatasetServiceImpl( + bigquery, null, PipelineOptionsFactory.create()); Table ret = services.tryCreateTable( testTable, new RetryBoundedBackOff(0, BackOff.ZERO_BACKOFF), Sleeper.DEFAULT); @@ -987,7 +994,8 @@ public void testCreateTableDoesNotRetry() throws IOException { thrown.expectMessage("actually forbidden"); BigQueryServicesImpl.DatasetServiceImpl services = - new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + new BigQueryServicesImpl.DatasetServiceImpl( + bigquery, null, PipelineOptionsFactory.create()); try { services.tryCreateTable( testTable, new RetryBoundedBackOff(3, BackOff.ZERO_BACKOFF), Sleeper.DEFAULT); @@ -1016,7 +1024,8 @@ public void testCreateTableSucceedsAlreadyExists() throws IOException { when(response.getStatusCode()).thenReturn(409); // 409 means already exists BigQueryServicesImpl.DatasetServiceImpl services = - new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + new BigQueryServicesImpl.DatasetServiceImpl( + bigquery, null, PipelineOptionsFactory.create()); Table ret = services.tryCreateTable( testTable, new RetryBoundedBackOff(0, BackOff.ZERO_BACKOFF), Sleeper.DEFAULT); @@ -1042,7 +1051,8 @@ public void testCreateTableRetry() throws IOException { .thenReturn(toStream(testTable)); BigQueryServicesImpl.DatasetServiceImpl services = - new BigQueryServicesImpl.DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + new BigQueryServicesImpl.DatasetServiceImpl( + bigquery, null, PipelineOptionsFactory.create()); Table ret = services.tryCreateTable( testTable, new RetryBoundedBackOff(3, BackOff.ZERO_BACKOFF), Sleeper.DEFAULT); @@ -1087,7 +1097,7 @@ public void testSimpleErrorRetrieval() throws InterruptedException, IOException when(response.getContent()).thenReturn(toStream(failures)); DatasetServiceImpl dataService = - new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + new DatasetServiceImpl(bigquery, null, PipelineOptionsFactory.create()); List> failedInserts = Lists.newArrayList(); dataService.insertAll( @@ -1143,7 +1153,7 @@ public void testExtendedErrorRetrieval() throws InterruptedException, IOExceptio when(response.getContent()).thenReturn(toStream(failures)); DatasetServiceImpl dataService = - new DatasetServiceImpl(bigquery, PipelineOptionsFactory.create()); + new DatasetServiceImpl(bigquery, null, PipelineOptionsFactory.create()); List> failedInserts = Lists.newArrayList(); dataService.insertAll( diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java index b433254834e7..56268128c508 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java @@ -174,7 +174,7 @@ public void testTableGet() throws InterruptedException, IOException { onTableList(dataList); BigQueryServicesImpl.DatasetServiceImpl services = - new BigQueryServicesImpl.DatasetServiceImpl(mockClient, options); + new BigQueryServicesImpl.DatasetServiceImpl(mockClient, null, options); services.getTable( new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table")); @@ -194,7 +194,7 @@ public void testInsertAll() throws Exception { onInsertAll(errorsIndices); TableReference ref = BigQueryHelpers.parseTableSpec("project:dataset.table"); - DatasetServiceImpl datasetService = new DatasetServiceImpl(mockClient, options, 5); + DatasetServiceImpl datasetService = new DatasetServiceImpl(mockClient, null, options, 5); List> rows = new ArrayList<>(); List ids = new ArrayList<>(); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/RetryManagerTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/RetryManagerTest.java new file mode 100644 index 000000000000..6e8a5746b0b2 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/RetryManagerTest.java @@ -0,0 +1,212 @@ +/* + * 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.junit.Assert.assertEquals; + +import com.google.api.core.ApiFutures; +import java.util.List; +import java.util.Map; +import org.apache.beam.sdk.io.gcp.bigquery.RetryManager.Operation; +import org.apache.beam.sdk.io.gcp.bigquery.RetryManager.Operation.Context; +import org.apache.beam.sdk.io.gcp.bigquery.RetryManager.RetryType; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps; +import org.joda.time.Duration; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link RetryManager}. */ +@RunWith(JUnit4.class) +public class RetryManagerTest { + static class Context extends Operation.Context { + int numStarted = 0; + int numSucceeded = 0; + int numFailed = 0; + } + + @Test + public void testNoFailures() throws Exception { + List contexts = Lists.newArrayList(); + RetryManager retryManager = + new RetryManager<>(Duration.millis(1), Duration.millis(1), 5); + for (int i = 0; i < 5; ++i) { + Context context = new Context(); + contexts.add(context); + retryManager.addOperation( + c -> { + ++c.numStarted; + return ApiFutures.immediateFuture("yes"); + }, + cs -> { + cs.forEach(c -> ++c.numFailed); + return RetryType.DONT_RETRY; + }, + c -> ++c.numSucceeded, + context); + } + contexts.forEach( + c -> { + assertEquals(0, c.numStarted); + assertEquals(0, c.numSucceeded); + assertEquals(0, c.numFailed); + }); + retryManager.run(true); + contexts.forEach( + c -> { + assertEquals(1, c.numStarted); + assertEquals(1, c.numSucceeded); + assertEquals(0, c.numFailed); + }); + } + + @Test + public void testRetryInOrder() throws Exception { + Map contexts = Maps.newHashMap(); + Map expectedStarts = Maps.newHashMap(); + Map expectedFailures = Maps.newHashMap(); + + RetryManager retryManager = + new RetryManager<>(Duration.millis(1), Duration.millis(1), 50); + for (int i = 0; i < 5; ++i) { + final int index = i; + String value = "yes " + i; + Context context = new Context(); + contexts.put(value, context); + expectedStarts.put(value, i + 2); + expectedFailures.put(value, i + 1); + retryManager.addOperation( + c -> { + // Make sure that each operation fails on its own. Failing a previous operation + // automatically + // fails all subsequent operations. + if (c.numStarted <= index) { + ++c.numStarted; + RuntimeException e = new RuntimeException("foo"); + return ApiFutures.immediateFailedFuture(e); + } else { + ++c.numStarted; + return ApiFutures.immediateFuture(value); + } + }, + cs -> { + cs.forEach(c -> ++c.numFailed); + return RetryType.RETRY_ALL_OPERATIONS; + }, + c -> ++c.numSucceeded, + context); + } + contexts + .values() + .forEach( + c -> { + assertEquals(0, c.numStarted); + assertEquals(0, c.numSucceeded); + assertEquals(0, c.numFailed); + }); + retryManager.run(true); + contexts + .entrySet() + .forEach( + e -> { + assertEquals((int) expectedStarts.get(e.getKey()), e.getValue().numStarted); + assertEquals(1, e.getValue().numSucceeded); + assertEquals((int) expectedFailures.get(e.getKey()), e.getValue().numFailed); + }); + } + + @Test + public void testDontRetry() throws Exception { + List contexts = Lists.newArrayList(); + + RetryManager retryManager = + new RetryManager<>(Duration.millis(1), Duration.millis(1), 50); + for (int i = 0; i < 5; ++i) { + Context context = new Context(); + contexts.add(context); + String value = "yes " + i; + retryManager.addOperation( + c -> { + if (c.numStarted == 0) { + ++c.numStarted; + RuntimeException e = new RuntimeException("foo"); + return ApiFutures.immediateFailedFuture(e); + } else { + ++c.numStarted; + return ApiFutures.immediateFuture(value); + } + }, + cs -> { + cs.forEach(c -> ++c.numFailed); + return RetryType.DONT_RETRY; + }, + c -> ++c.numSucceeded, + context); + } + contexts.forEach( + c -> { + assertEquals(0, c.numStarted); + assertEquals(0, c.numSucceeded); + assertEquals(0, c.numFailed); + }); + retryManager.run(true); + contexts.forEach( + c -> { + assertEquals(1, c.numStarted); + assertEquals(0, c.numSucceeded); + assertEquals(1, c.numFailed); + }); + } + + @Test + public void testHasSucceeded() throws Exception { + List contexts = Lists.newArrayList(); + RetryManager retryManager = + new RetryManager<>(Duration.millis(1), Duration.millis(1), 5); + for (int i = 0; i < 5; ++i) { + Context context = new Context(); + contexts.add(context); + retryManager.addOperation( + c -> { + ++c.numStarted; + return ApiFutures.immediateFuture("yes"); + }, + cs -> { + cs.forEach(c -> ++c.numFailed); + return RetryType.DONT_RETRY; + }, + c -> ++c.numSucceeded, + c -> false, + context); + } + contexts.forEach( + c -> { + assertEquals(0, c.numStarted); + assertEquals(0, c.numSucceeded); + assertEquals(0, c.numFailed); + }); + retryManager.run(true); + contexts.forEach( + c -> { + assertEquals(1, c.numStarted); + assertEquals(0, c.numSucceeded); + assertEquals(1, c.numFailed); + }); + } +} From 60230dd8dfe27c1642301519ebf26ace46b70659 Mon Sep 17 00:00:00 2001 From: Reuven Lax Date: Tue, 23 Mar 2021 09:22:20 -0700 Subject: [PATCH 2/2] Remove bogus test --- .../BigQueryStorageAPIStreamingIT.java | 132 ------------------ 1 file changed, 132 deletions(-) delete mode 100644 examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryStorageAPIStreamingIT.java diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryStorageAPIStreamingIT.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryStorageAPIStreamingIT.java deleted file mode 100644 index ad68904efc82..000000000000 --- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryStorageAPIStreamingIT.java +++ /dev/null @@ -1,132 +0,0 @@ -/* - * 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.examples.cookbook; - -import com.google.auto.value.AutoValue; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.util.concurrent.ThreadLocalRandom; -import javax.annotation.Nullable; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.GenerateSequence; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; -import org.apache.beam.sdk.options.Default; -import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.schemas.AutoValueSchema; -import org.apache.beam.sdk.schemas.annotations.DefaultSchema; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.testing.TestPipelineOptions; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TypeDescriptor; -import org.joda.time.Duration; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -@RunWith(JUnit4.class) -public class BigQueryStorageAPIStreamingIT { - @DefaultSchema(AutoValueSchema.class) - @AutoValue - public abstract static class Value { - public abstract long getNumber(); - - @Nullable - public abstract ByteBuffer getPayload(); - } - - public interface Options extends TestPipelineOptions { - @Description("The number of records per second to generate.") - @Default.Integer(10000) - Integer getRecordsPerSecond(); - - void setRecordsPerSecond(Integer recordsPerSecond); - - @Description("The size of the records to write in bytes.") - @Default.Integer(1024) - Integer getPayloadSizeBytes(); - - void setPayloadSizeBytes(Integer payloadSizeBytes); - - @Description("Parallelism used for Storage API writes.") - @Default.Integer(5) - Integer getNumShards(); - - void setNumShards(Integer numShards); - - @Description("Frequency to trigger appends. Each shard triggers independently.") - @Default.Integer(5) - Integer getTriggerFrequencySec(); - - void setTriggerFrequencySec(Integer triggerFrequencySec); - - @Description("The table to write to.") - String getTargetTable(); - - void setTargetTable(String table); - } - - @BeforeClass - public static void setUp() { - PipelineOptionsFactory.register(Options.class); - } - - @Test - public void testStorageAPIStreaming() throws Exception { - Options options = TestPipeline.testingPipelineOptions().as(Options.class); - Pipeline p = Pipeline.create(options); - final int payloadSizeBytes = options.getPayloadSizeBytes(); - - // Generate input. - PCollection values = - p.apply( - GenerateSequence.from(1) - .to(1000000) - .withRate(options.getRecordsPerSecond(), Duration.standardSeconds(1))) - .apply( - MapElements.into(TypeDescriptor.of(Value.class)) - .via( - l -> { - byte[] payload = "".getBytes(StandardCharsets.UTF_8); - if (payloadSizeBytes > 0) { - payload = new byte[payloadSizeBytes]; - ThreadLocalRandom.current().nextBytes(payload); - } - return new AutoValue_BigQueryStorageAPIStreamingIT_Value( - l, ByteBuffer.wrap(payload)); - })); - - // Write results using Vortex. - values.apply( - "writeVortex", - BigQueryIO.write() - .useBeamSchema() - .to(options.getTargetTable()) - .withMethod(Write.Method.STORAGE_WRITE_API) - .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) - .withWriteDisposition(WriteDisposition.WRITE_APPEND) - .withNumStorageWriteApiStreams(options.getNumShards()) - .withTriggeringFrequency(Duration.standardSeconds(options.getTriggerFrequencySec()))); - - p.run(); - } -}