From 597523f1d19bdc61f411d7a2c6a17e735e899ab9 Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Sun, 25 Sep 2022 19:32:45 -0400 Subject: [PATCH 01/23] chore: migrate BigtableIO to use Veneer --- .../beam/gradle/BeamModulePlugin.groovy | 2 +- sdks/java/extensions/sql/build.gradle | 2 +- .../bigtable/BigtableClientWrapper.java | 115 --- .../bigtable/BigtableTableFlatTest.java | 1 + .../provider/bigtable/BigtableTableIT.java | 1 + .../bigtable/BigtableTableTestUtils.java | 1 + .../bigtable/BigtableTableWithRowsTest.java | 1 + .../io/google-cloud-platform/build.gradle | 6 +- .../gcp/bigtable/BigtableClientWrapper.java | 94 +++ .../sdk/io/gcp/bigtable/BigtableConfig.java | 11 + .../BigtableHBaseVeneeringSettings.java | 758 ++++++++++++++++++ .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 25 +- .../sdk/io/gcp/bigtable/BigtableService.java | 4 +- .../io/gcp/bigtable/BigtableServiceImpl.java | 526 ++++++++---- .../VendoredListenableFutureAdapter.java | 10 +- .../beam/sdk/io/gcp/GcpApiSurfaceTest.java | 4 - .../sdk/io/gcp/bigtable/BigtableIOTest.java | 27 +- .../gcp/bigtable/BigtableServiceImplTest.java | 506 ++++++++---- .../sdk/io/gcp/bigtable/BigtableWriteIT.java | 103 +-- settings.gradle.kts | 1 + 20 files changed, 1633 insertions(+), 565 deletions(-) delete mode 100644 sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableClientWrapper.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableClientWrapper.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableHBaseVeneeringSettings.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 6fb508b610ca..e766a873c429 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -596,7 +596,7 @@ class BeamModulePlugin implements Plugin { 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", // google_cloud_platform_libraries_bom sets version google_cloud_bigtable : "com.google.cloud:google-cloud-bigtable", // google_cloud_platform_libraries_bom sets version - google_cloud_bigtable_client_core : "com.google.cloud.bigtable:bigtable-client-core:1.26.3", + google_cloud_bigtable_client_core_config : "com.google.cloud.bigtable:bigtable-client-core-config:1.28.0", google_cloud_bigtable_emulator : "com.google.cloud:google-cloud-bigtable-emulator:0.137.1", google_cloud_core : "com.google.cloud:google-cloud-core", // google_cloud_platform_libraries_bom sets version google_cloud_core_grpc : "com.google.cloud:google-cloud-core-grpc", // google_cloud_platform_libraries_bom sets version diff --git a/sdks/java/extensions/sql/build.gradle b/sdks/java/extensions/sql/build.gradle index 4f9efec51789..e307efe4e7f8 100644 --- a/sdks/java/extensions/sql/build.gradle +++ b/sdks/java/extensions/sql/build.gradle @@ -116,7 +116,7 @@ dependencies { testImplementation library.java.quickcheck_core testImplementation library.java.testcontainers_kafka testImplementation library.java.google_cloud_bigtable - testImplementation library.java.google_cloud_bigtable_client_core + testImplementation library.java.google_cloud_bigtable_client_core_config testImplementation library.java.google_cloud_bigtable_emulator testImplementation library.java.proto_google_cloud_bigtable_admin_v2 testImplementation library.java.proto_google_cloud_datastore_v1 diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableClientWrapper.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableClientWrapper.java deleted file mode 100644 index 6a8b343b18f6..000000000000 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableClientWrapper.java +++ /dev/null @@ -1,115 +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.sdk.extensions.sql.meta.provider.bigtable; - -import static org.apache.beam.sdk.io.gcp.bigtable.RowUtils.byteString; -import static org.apache.beam.sdk.io.gcp.bigtable.RowUtils.byteStringUtf8; - -import com.google.auth.Credentials; -import com.google.bigtable.admin.v2.ColumnFamily; -import com.google.bigtable.admin.v2.DeleteTableRequest; -import com.google.bigtable.admin.v2.Table; -import com.google.bigtable.v2.MutateRowRequest; -import com.google.bigtable.v2.Mutation; -import com.google.cloud.bigtable.config.BigtableOptions; -import com.google.cloud.bigtable.config.CredentialOptions; -import com.google.cloud.bigtable.grpc.BigtableDataClient; -import com.google.cloud.bigtable.grpc.BigtableSession; -import com.google.cloud.bigtable.grpc.BigtableTableAdminClient; -import java.io.IOException; -import java.io.Serializable; -import org.checkerframework.checker.nullness.qual.Nullable; - -class BigtableClientWrapper implements Serializable { - private final BigtableTableAdminClient tableAdminClient; - private final BigtableDataClient dataClient; - private final BigtableSession session; - private final BigtableOptions bigtableOptions; - - BigtableClientWrapper( - String project, - String instanceId, - @Nullable Integer emulatorPort, - @Nullable Credentials gcpCredentials) - throws IOException { - BigtableOptions.Builder optionsBuilder = - BigtableOptions.builder() - .setProjectId(project) - .setInstanceId(instanceId) - .setUserAgent("apache-beam-test"); - if (emulatorPort != null) { - optionsBuilder.enableEmulator("localhost", emulatorPort); - } - if (gcpCredentials != null) { - optionsBuilder.setCredentialOptions(CredentialOptions.credential(gcpCredentials)); - } - bigtableOptions = optionsBuilder.build(); - - session = new BigtableSession(bigtableOptions); - tableAdminClient = session.getTableAdminClient(); - dataClient = session.getDataClient(); - } - - void writeRow( - String key, - String table, - String familyColumn, - String columnQualifier, - byte[] value, - long timestampMicros) { - Mutation.SetCell setCell = - Mutation.SetCell.newBuilder() - .setFamilyName(familyColumn) - .setColumnQualifier(byteStringUtf8(columnQualifier)) - .setValue(byteString(value)) - .setTimestampMicros(timestampMicros) - .build(); - Mutation mutation = Mutation.newBuilder().setSetCell(setCell).build(); - MutateRowRequest mutateRowRequest = - MutateRowRequest.newBuilder() - .setRowKey(byteStringUtf8(key)) - .setTableName(bigtableOptions.getInstanceName().toTableNameStr(table)) - .addMutations(mutation) - .build(); - dataClient.mutateRow(mutateRowRequest); - } - - void createTable(String tableName, String familyName) { - Table.Builder tableBuilder = Table.newBuilder(); - tableBuilder.putColumnFamilies(familyName, ColumnFamily.newBuilder().build()); - - String instanceName = bigtableOptions.getInstanceName().toString(); - com.google.bigtable.admin.v2.CreateTableRequest.Builder createTableRequestBuilder = - com.google.bigtable.admin.v2.CreateTableRequest.newBuilder() - .setParent(instanceName) - .setTableId(tableName) - .setTable(tableBuilder.build()); - tableAdminClient.createTable(createTableRequestBuilder.build()); - } - - void deleteTable(String tableId) { - final String tableName = bigtableOptions.getInstanceName().toTableNameStr(tableId); - DeleteTableRequest.Builder deleteTableRequestBuilder = - DeleteTableRequest.newBuilder().setName(tableName); - tableAdminClient.deleteTable(deleteTableRequestBuilder.build()); - } - - void closeSession() throws IOException { - session.close(); - } -} diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableFlatTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableFlatTest.java index 08082e85e0e1..101990857f3a 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableFlatTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableFlatTest.java @@ -44,6 +44,7 @@ import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils; import org.apache.beam.sdk.extensions.sql.meta.Table; import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore; +import org.apache.beam.sdk.io.gcp.bigtable.BigtableClientWrapper; import org.apache.beam.sdk.io.gcp.bigtable.BigtableIO; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.testing.PAssert; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableIT.java index d62292a3f6ab..3abad4b21bf3 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableIT.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableIT.java @@ -32,6 +32,7 @@ import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils; +import org.apache.beam.sdk.io.gcp.bigtable.BigtableClientWrapper; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptionsFactory; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableTestUtils.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableTestUtils.java index ef66fb89affa..b812f880dc5e 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableTestUtils.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableTestUtils.java @@ -36,6 +36,7 @@ import com.google.bigtable.v2.Column; import com.google.bigtable.v2.Family; import java.util.List; +import org.apache.beam.sdk.io.gcp.bigtable.BigtableClientWrapper; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableWithRowsTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableWithRowsTest.java index 4b60eb13c894..7327a1cc8579 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableWithRowsTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableWithRowsTest.java @@ -38,6 +38,7 @@ import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils; import org.apache.beam.sdk.extensions.sql.meta.Table; import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore; +import org.apache.beam.sdk.io.gcp.bigtable.BigtableClientWrapper; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.MapElements; diff --git a/sdks/java/io/google-cloud-platform/build.gradle b/sdks/java/io/google-cloud-platform/build.gradle index a0785194d19d..a497126c204b 100644 --- a/sdks/java/io/google-cloud-platform/build.gradle +++ b/sdks/java/io/google-cloud-platform/build.gradle @@ -64,10 +64,7 @@ dependencies { implementation library.java.google_auth_library_credentials implementation library.java.google_auth_library_oauth2_http implementation library.java.google_cloud_bigquery_storage - implementation(library.java.google_cloud_bigtable_client_core) { - exclude group: 'io.grpc', module: 'grpc-core' // Use Beam's version - exclude group: 'io.grpc', module: 'grpc-grpclb' - } + implementation(library.java.google_cloud_bigtable_client_core_config) // google_cloud_bigtable_client_core declares old google-cloud-bigtable for // Java7 compatibility. The old google-cloud-bigtable is not compatible with // newer version of GAX. Declaring newer google-cloud-bigtable so that Beam @@ -132,7 +129,6 @@ dependencies { implementation library.java.netty_tcnative_boringssl_static permitUnusedDeclared library.java.netty_tcnative_boringssl_static // BEAM-11761 implementation library.java.proto_google_cloud_bigquery_storage_v1 - implementation library.java.proto_google_cloud_bigtable_admin_v2 implementation library.java.proto_google_cloud_bigtable_v2 implementation library.java.proto_google_cloud_datastore_v1 implementation library.java.proto_google_cloud_firestore_v1 diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableClientWrapper.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableClientWrapper.java new file mode 100644 index 000000000000..85a8498f2ecc --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableClientWrapper.java @@ -0,0 +1,94 @@ +/* + * 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.bigtable; + +import static org.apache.beam.sdk.io.gcp.bigtable.RowUtils.byteString; +import static org.apache.beam.sdk.io.gcp.bigtable.RowUtils.byteStringUtf8; + +import com.google.auth.Credentials; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; +import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; +import com.google.cloud.bigtable.config.BigtableOptions; +import com.google.cloud.bigtable.config.CredentialOptions; +import com.google.cloud.bigtable.data.v2.BigtableDataClient; +import com.google.cloud.bigtable.data.v2.models.RowMutation; +import java.io.IOException; +import java.io.Serializable; +import org.apache.beam.sdk.annotations.Internal; +import org.checkerframework.checker.nullness.qual.Nullable; + +@Internal +public class BigtableClientWrapper implements Serializable { + private final BigtableTableAdminClient tableAdminClient; + private final BigtableDataClient dataClient; + + private final BigtableOptions bigtableOptions; + + public BigtableClientWrapper( + String project, + String instanceId, + @Nullable Integer emulatorPort, + @Nullable Credentials gcpCredentials) + throws IOException { + BigtableOptions.Builder optionsBuilder = + BigtableOptions.builder() + .setProjectId(project) + .setInstanceId(instanceId) + .setUserAgent("apache-beam-test"); + if (emulatorPort != null) { + optionsBuilder.enableEmulator("localhost", emulatorPort); + } + if (gcpCredentials != null) { + optionsBuilder.setCredentialOptions(CredentialOptions.credential(gcpCredentials)); + } + bigtableOptions = optionsBuilder.build(); + + BigtableHBaseVeneeringSettings settings = + BigtableHBaseVeneeringSettings.create(bigtableOptions); + tableAdminClient = BigtableTableAdminClient.create(settings.getTableAdminSettings()); + dataClient = BigtableDataClient.create(settings.getDataSettings()); + } + + public void writeRow( + String key, + String table, + String familyColumn, + String columnQualifier, + byte[] value, + long timestampMicros) { + RowMutation rowMutation = + RowMutation.create(table, key) + .setCell( + familyColumn, byteStringUtf8(columnQualifier), timestampMicros, byteString(value)); + dataClient.mutateRow(rowMutation); + } + + public void createTable(String tableName, String familyName) { + CreateTableRequest createTableRequest = CreateTableRequest.of(tableName).addFamily(familyName); + tableAdminClient.createTable(createTableRequest); + } + + public void deleteTable(String tableId) { + tableAdminClient.deleteTable(tableId); + } + + public void closeSession() throws IOException { + dataClient.close(); + tableAdminClient.close(); + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java index db3ac3de6e9a..7aebad4679bc 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java @@ -48,6 +48,9 @@ abstract class BigtableConfig implements Serializable { /** Returns the table being read from. */ abstract @Nullable ValueProvider getTableId(); + /** Returns the app profile id of this workload. */ + abstract @Nullable ValueProvider getAppProfileId(); + /** * Returns the Google Cloud Bigtable instance being written to, and other parameters. * @@ -84,6 +87,8 @@ abstract static class Builder { abstract Builder setTableId(ValueProvider tableId); + abstract Builder setAppProfileId(ValueProvider appProfileId); + /** @deprecated will be replaced by bigtable options configurator. */ @Deprecated abstract Builder setBigtableOptions(BigtableOptions options); @@ -115,6 +120,11 @@ BigtableConfig withTableId(ValueProvider tableId) { return toBuilder().setTableId(tableId).build(); } + BigtableConfig withAppProfileId(@Nullable ValueProvider appProfileId) { + checkArgument(appProfileId != null, "App profile id can not be null"); + return toBuilder().setAppProfileId(appProfileId).build(); + } + /** @deprecated will be replaced by bigtable options configurator. */ @Deprecated BigtableConfig withBigtableOptions(BigtableOptions options) { @@ -250,6 +260,7 @@ public final String toString() { .add("projectId", getProjectId()) .add("instanceId", getInstanceId()) .add("tableId", getTableId()) + .add("appProfileId", getAppProfileId()) .add( "bigtableOptionsConfigurator", getBigtableOptionsConfigurator() == null diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableHBaseVeneeringSettings.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableHBaseVeneeringSettings.java new file mode 100644 index 000000000000..8f0063d1f5fb --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableHBaseVeneeringSettings.java @@ -0,0 +1,758 @@ +/* + * 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.bigtable; + +import com.google.api.core.ApiFunction; +import com.google.api.gax.batching.BatchingSettings; +import com.google.api.gax.batching.FlowControlSettings; +import com.google.api.gax.core.FixedCredentialsProvider; +import com.google.api.gax.core.NoCredentialsProvider; +import com.google.api.gax.grpc.InstantiatingGrpcChannelProvider; +import com.google.api.gax.rpc.FixedHeaderProvider; +import com.google.api.gax.rpc.ServerStreamingCallSettings; +import com.google.api.gax.rpc.StatusCode; +import com.google.api.gax.rpc.StubSettings; +import com.google.api.gax.rpc.UnaryCallSettings; +import com.google.auth.Credentials; +import com.google.auth.oauth2.GoogleCredentials; +import com.google.auth.oauth2.ServiceAccountCredentials; +import com.google.auth.oauth2.ServiceAccountJwtAccessCredentials; +import com.google.cloud.bigtable.Version; +import com.google.cloud.bigtable.admin.v2.BigtableInstanceAdminSettings; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminSettings; +import com.google.cloud.bigtable.config.BigtableOptions; +import com.google.cloud.bigtable.config.CredentialOptions; +import com.google.cloud.bigtable.data.v2.BigtableDataSettings; +import com.google.cloud.bigtable.data.v2.models.Query; +import com.google.cloud.bigtable.data.v2.models.Row; +import com.google.cloud.bigtable.data.v2.stub.BigtableBatchingCallSettings; +import com.google.cloud.bigtable.data.v2.stub.BigtableBulkReadRowsCallSettings; +import io.grpc.CallOptions; +import io.grpc.Channel; +import io.grpc.ClientCall; +import io.grpc.ClientInterceptor; +import io.grpc.Deadline; +import io.grpc.ManagedChannelBuilder; +import io.grpc.MethodDescriptor; +import io.grpc.Status; +import io.grpc.internal.GrpcUtil; +import java.io.FileInputStream; +import java.io.IOException; +import java.security.GeneralSecurityException; +import java.security.KeyStore; +import java.security.PrivateKey; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import javax.annotation.Nonnull; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Optional; +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.Lists; +import org.checkerframework.checker.initialization.qual.UnderInitialization; +import org.threeten.bp.Duration; + +/** Helper class to translate {@link BigtableOptions} to Bigtable Veneer settings. */ +class BigtableHBaseVeneeringSettings { + private static final String DEFAULT_BIGTABLE_BATCH_DATA_ENDPOINT = + "batch-bigtable.googleapis.com:443"; + private static final Duration DEFAULT_UNARY_ATTEMPT_TIMEOUTS = Duration.ofSeconds(20); + private static final Duration DEFAULT_BULK_MUTATE_ATTEMPT_TIMEOUTS = Duration.ofMinutes(6); + + private final BigtableDataSettings dataSettings; + private final BigtableTableAdminSettings tableAdminSettings; + private final BigtableInstanceAdminSettings instanceAdminSettings; + + private final BigtableIOOperationTimeouts clientTimeouts; + + static BigtableHBaseVeneeringSettings create(@Nonnull BigtableOptions options) + throws IOException { + return new BigtableHBaseVeneeringSettings(options); + } + + private BigtableHBaseVeneeringSettings(@Nonnull BigtableOptions options) throws IOException { + // Build configs for veneer + this.clientTimeouts = buildCallSettings(options); + + this.dataSettings = buildBigtableDataSettings(clientTimeouts, options); + this.tableAdminSettings = buildBigtableTableAdminSettings(options); + this.instanceAdminSettings = buildBigtableInstanceAdminSettings(options); + } + + // ************** Getters ************** + /** Utility to convert {@link BigtableOptions} to {@link BigtableDataSettings}. */ + BigtableDataSettings getDataSettings() { + return dataSettings; + } + + /** Utility to convert {@link BigtableOptions} to {@link BigtableTableAdminSettings}. */ + BigtableTableAdminSettings getTableAdminSettings() { + return tableAdminSettings; + } + + BigtableIOOperationTimeouts getOperationTimeouts() { + return clientTimeouts; + } + + /** Utility to convert {@link BigtableOptions} to {@link BigtableInstanceAdminSettings}. */ + BigtableInstanceAdminSettings getInstanceAdminSettings() { + return instanceAdminSettings; + } + + // ************** Private Helpers ************** + private BigtableDataSettings buildBigtableDataSettings( + @UnderInitialization BigtableHBaseVeneeringSettings this, + BigtableIOOperationTimeouts clientTimeouts, + BigtableOptions options) + throws IOException { + BigtableDataSettings.Builder dataBuilder; + + // Configure the Data connection + dataBuilder = BigtableDataSettings.newBuilder(); + if (options.useBatch()) { + configureConnection( + dataBuilder.stubSettings(), DEFAULT_BIGTABLE_BATCH_DATA_ENDPOINT, options); + } else { + configureConnection( + dataBuilder.stubSettings(), options.getDataHost() + ":" + options.getPort(), options); + } + configureCredentialProvider(dataBuilder.stubSettings(), options); + configureHeaderProvider(dataBuilder.stubSettings(), options); + + // Configure the target + dataBuilder.setProjectId(options.getProjectId()).setInstanceId(options.getInstanceId()); + if (options.getAppProfileId() != null) { + dataBuilder.setAppProfileId(options.getAppProfileId()); + } + + // Configure RPCs - this happens in multiple parts: + // - retry settings are configured here + // - timeouts are split into multiple places: + // - timeouts for retries are configured here + // - if USE_TIMEOUTS is explicitly disabled, then an interceptor is added to force all + // deadlines to 6 minutes + configureConnectionCallTimeouts(dataBuilder.stubSettings(), clientTimeouts); + + // Complex RPC method settings + configureBulkMutationSettings( + dataBuilder.stubSettings().bulkMutateRowsSettings(), + clientTimeouts.getBulkMutateTimeouts(), + options); + configureBulkReadRowsSettings( + dataBuilder.stubSettings().bulkReadRowsSettings(), + clientTimeouts.getBulkReadRowsTimeouts(), + options); + configureReadRowsSettings( + dataBuilder.stubSettings().readRowsSettings(), + clientTimeouts.getBulkReadRowsTimeouts(), + options); + + // RPC methods - simple + configureNonRetryableCallSettings( + dataBuilder.stubSettings().checkAndMutateRowSettings(), clientTimeouts.getUnaryTimeouts()); + configureNonRetryableCallSettings( + dataBuilder.stubSettings().readModifyWriteRowSettings(), clientTimeouts.getUnaryTimeouts()); + + configureRetryableCallSettings( + dataBuilder.stubSettings().mutateRowSettings(), clientTimeouts.getUnaryTimeouts(), options); + configureRetryableCallSettings( + dataBuilder.stubSettings().readRowSettings(), clientTimeouts.getUnaryTimeouts(), options); + configureRetryableCallSettings( + dataBuilder.stubSettings().sampleRowKeysSettings(), + clientTimeouts.getUnaryTimeouts(), + options); + + return dataBuilder.build(); + } + + private BigtableTableAdminSettings buildBigtableTableAdminSettings( + @UnderInitialization BigtableHBaseVeneeringSettings this, BigtableOptions options) + throws IOException { + BigtableTableAdminSettings.Builder adminBuilder; + + // Configure connection + adminBuilder = BigtableTableAdminSettings.newBuilder(); + configureConnection( + adminBuilder.stubSettings(), options.getAdminHost() + ":" + options.getPort(), options); + configureCredentialProvider(adminBuilder.stubSettings(), options); + + configureHeaderProvider(adminBuilder.stubSettings(), options); + + adminBuilder.setProjectId(options.getProjectId()).setInstanceId(options.getInstanceId()); + + // timeout/retry settings don't apply to admin operations + // v1 used to use RetryOptions for: + // - createTable + // - getTable + // - listTables + // - deleteTable + // - modifyColumnFamilies + // - dropRowRange + // However data latencies are very different from data latencies and end users shouldn't need to + // change the defaults + // if it turns out that the timeout & retry behavior needs to be configurable, we will expose + // separate settings + + return adminBuilder.build(); + } + + private BigtableInstanceAdminSettings buildBigtableInstanceAdminSettings( + @UnderInitialization BigtableHBaseVeneeringSettings this, BigtableOptions options) + throws IOException { + BigtableInstanceAdminSettings.Builder adminBuilder; + + // Configure connection + adminBuilder = BigtableInstanceAdminSettings.newBuilder(); + configureConnection( + adminBuilder.stubSettings(), options.getAdminHost() + ":" + options.getPort(), options); + configureCredentialProvider(adminBuilder.stubSettings(), options); + + configureHeaderProvider(adminBuilder.stubSettings(), options); + + adminBuilder.setProjectId(options.getProjectId()); + + return adminBuilder.build(); + } + + @SuppressWarnings("rawtypes") + private void configureConnection( + @UnderInitialization BigtableHBaseVeneeringSettings this, + StubSettings.Builder stubSettings, + String endpoint, + BigtableOptions options) { + final InstantiatingGrpcChannelProvider.Builder channelProvider = + ((InstantiatingGrpcChannelProvider) stubSettings.getTransportChannelProvider()).toBuilder(); + + stubSettings.setEndpoint(endpoint); + + if (options.usePlaintextNegotiation()) { + // Make sure to avoid clobbering the old Configurator + @SuppressWarnings("rawtypes") + final ApiFunction prevConfigurator = + channelProvider.getChannelConfigurator(); + //noinspection rawtypes + channelProvider.setChannelConfigurator( + new ApiFunction() { + @Override + public ManagedChannelBuilder apply(ManagedChannelBuilder channelBuilder) { + if (prevConfigurator != null) { + channelBuilder = prevConfigurator.apply(channelBuilder); + } + return channelBuilder.usePlaintext(); + } + }); + } + + channelProvider.setPoolSize(options.getChannelCount()); + + stubSettings.setTransportChannelProvider(channelProvider.build()); + } + + private void configureHeaderProvider( + @UnderInitialization BigtableHBaseVeneeringSettings this, + StubSettings.Builder stubSettings, + BigtableOptions options) { + + ImmutableMap.Builder headersBuilder = ImmutableMap.builder(); + List userAgentParts = Lists.newArrayList(); + userAgentParts.add("bigtable-" + Version.VERSION); + userAgentParts.add("jdk-" + System.getProperty("java.specification.version")); + + String customUserAgent = options.getUserAgent(); + if (customUserAgent != null) { + userAgentParts.add(customUserAgent); + } + + String userAgent = Joiner.on(",").join(userAgentParts); + headersBuilder.put(GrpcUtil.USER_AGENT_KEY.name(), userAgent); + + String tracingCookie = options.getTracingCookie(); + if (tracingCookie != null) { + headersBuilder.put("cookie", tracingCookie); + } + + stubSettings.setHeaderProvider(FixedHeaderProvider.create(headersBuilder.build())); + } + + @SuppressWarnings("rawtypes") + private void configureConnectionCallTimeouts( + @UnderInitialization BigtableHBaseVeneeringSettings this, + StubSettings.Builder stubSettings, + BigtableIOOperationTimeouts clientTimeouts) { + // Only patch settings when timeouts are disabled + if (clientTimeouts.getUseTimeouts()) { + return; + } + InstantiatingGrpcChannelProvider.Builder channelProvider = + ((InstantiatingGrpcChannelProvider) stubSettings.getTransportChannelProvider()).toBuilder(); + + final ApiFunction prevConfigurator = + channelProvider.getChannelConfigurator(); + + channelProvider.setChannelConfigurator( + new ApiFunction() { + @Override + public ManagedChannelBuilder apply(ManagedChannelBuilder managedChannelBuilder) { + if (prevConfigurator != null) { + managedChannelBuilder = prevConfigurator.apply(managedChannelBuilder); + } + return managedChannelBuilder.intercept(new NoTimeoutsInterceptor()); + } + }); + stubSettings.setTransportChannelProvider(channelProvider.build()); + } + + private void configureBulkMutationSettings( + @UnderInitialization BigtableHBaseVeneeringSettings this, + BigtableBatchingCallSettings.Builder builder, + OperationTimeouts operationTimeouts, + BigtableOptions options) { + BatchingSettings.Builder batchingSettingsBuilder = builder.getBatchingSettings().toBuilder(); + + // Start configure retries & timeouts + configureRetryableCallSettings(builder, operationTimeouts, options); + // End configure retries & timeouts + + // Start configure flush triggers + long autoFlushMs = options.getBulkOptions().getAutoflushMs(); + if (autoFlushMs <= 0) { + // setDelayThreshold(null) will cause build error. Ignore autoFlushMs + // if it's disabled and use the default value of 1 second instead until + // we fix the implementation in gax BatcherImpl to annotate with + // @Nonnull or check for 0 duration. + batchingSettingsBuilder.setDelayThreshold(Duration.ofSeconds(1)); + } else { + batchingSettingsBuilder.setDelayThreshold(Duration.ofMillis(autoFlushMs)); + } + + batchingSettingsBuilder.setElementCountThreshold( + Long.valueOf(options.getBulkOptions().getBulkMaxRowKeyCount())); + + batchingSettingsBuilder.setRequestByteThreshold( + Long.valueOf(options.getBulkOptions().getBulkMaxRequestSize())); + // End configure flush triggers + + // Start configure flow control + FlowControlSettings.Builder flowControl = + builder.getBatchingSettings().getFlowControlSettings().toBuilder(); + + // Approximate max inflight rpcs in terms of outstanding elements + int maxInflightRpcCount = options.getBulkOptions().getMaxInflightRpcs(); + + Long bulkMaxRowKeyCount = builder.getBatchingSettings().getElementCountThreshold(); + if (bulkMaxRowKeyCount == null) { + // Using Object.requireNotNull will cause build error "incompatible argument for + // parameter arg0 of requireNonNull" + throw new IllegalStateException("elementCountThreshold can't be null"); + } + Long maxInflightElements = maxInflightRpcCount * bulkMaxRowKeyCount; + flowControl.setMaxOutstandingElementCount(maxInflightElements); + + flowControl.setMaxOutstandingRequestBytes(options.getBulkOptions().getMaxMemory()); + + batchingSettingsBuilder.setFlowControlSettings(flowControl.build()); + // End configure flow control + + builder.setBatchingSettings(batchingSettingsBuilder.build()); + + if (options.getBulkOptions().isEnableBulkMutationThrottling()) { + long latencyMs = options.getBulkOptions().getBulkMutationRpcTargetMs(); + builder.enableLatencyBasedThrottling(latencyMs); + } + } + + private void configureBulkReadRowsSettings( + @UnderInitialization BigtableHBaseVeneeringSettings this, + BigtableBulkReadRowsCallSettings.Builder builder, + OperationTimeouts operationTimeouts, + BigtableOptions options) { + BatchingSettings.Builder bulkReadBatchingBuilder = builder.getBatchingSettings().toBuilder(); + + // Start configure retries & timeouts + configureRetryableCallSettings(builder, operationTimeouts, options); + // End configure retries & timeouts + + // Start config batch settings + long maxRowKeyCount = options.getBulkOptions().getBulkMaxRowKeyCount(); + bulkReadBatchingBuilder.setElementCountThreshold(maxRowKeyCount); + builder.setBatchingSettings(bulkReadBatchingBuilder.build()); + // End config batch settings + + // NOTE: autoflush, flow control settings are not currently exposed + } + + private void configureReadRowsSettings( + @UnderInitialization BigtableHBaseVeneeringSettings this, + ServerStreamingCallSettings.Builder readRowsSettings, + OperationTimeouts operationTimeouts, + BigtableOptions options) { + + // Configure retries + // NOTE: that similar but not the same as unary retry settings: per attempt timeouts don't + // exist, instead we use READ_PARTIAL_ROW_TIMEOUT_MS as the intra-row timeout + if (!options.getRetryOptions().enableRetries()) { + // user explicitly disabled retries, treat it as a non-idempotent method + readRowsSettings.setRetryableCodes(Collections.emptySet()); + } else { + // apply user retry settings + readRowsSettings.setRetryableCodes( + extractRetryCodesFromConfig(readRowsSettings.getRetryableCodes(), options)); + + // Configure backoff + long initialElapsedBackoffMs = options.getRetryOptions().getInitialBackoffMillis(); + readRowsSettings + .retrySettings() + .setInitialRetryDelay(Duration.ofMillis(initialElapsedBackoffMs)); + + if (initialElapsedBackoffMs + > readRowsSettings.retrySettings().getMaxRetryDelay().toMillis()) { + readRowsSettings + .retrySettings() + .setMaxRetryDelay(Duration.ofMillis(initialElapsedBackoffMs)); + } + + readRowsSettings + .retrySettings() + .setMaxAttempts(options.getRetryOptions().getMaxScanTimeoutRetries()); + } + + // Per response timeouts (note: gax maps rpcTimeouts to response timeouts for streaming rpcs) + if (operationTimeouts.getResponseTimeout().isPresent()) { + readRowsSettings + .retrySettings() + .setInitialRpcTimeout(operationTimeouts.getResponseTimeout().get()) + .setMaxRpcTimeout(operationTimeouts.getResponseTimeout().get()); + } + + // attempt timeout is configured in BigtableServiceImpl + + // overall timeout + if (operationTimeouts.getOperationTimeout().isPresent()) { + readRowsSettings + .retrySettings() + .setTotalTimeout(operationTimeouts.getOperationTimeout().get()); + } + } + + private void configureRetryableCallSettings( + @UnderInitialization BigtableHBaseVeneeringSettings this, + UnaryCallSettings.Builder unaryCallSettings, + OperationTimeouts operationTimeouts, + BigtableOptions options) { + + if (!options.getRetryOptions().enableRetries()) { + // user explicitly disabled retries, treat it as a non-idempotent method + configureNonRetryableCallSettings(unaryCallSettings, operationTimeouts); + return; + } + + // apply user user retry settings + unaryCallSettings.setRetryableCodes( + extractRetryCodesFromConfig(unaryCallSettings.getRetryableCodes(), options)); + + // Configure backoff + long initialBackoffMs = options.getRetryOptions().getInitialBackoffMillis(); + unaryCallSettings.retrySettings().setInitialRetryDelay(Duration.ofMillis(initialBackoffMs)); + + if (initialBackoffMs > unaryCallSettings.retrySettings().getMaxRetryDelay().toMillis()) { + unaryCallSettings.retrySettings().setMaxRetryDelay(Duration.ofMillis(initialBackoffMs)); + } + + // Configure overall timeout + if (operationTimeouts.getOperationTimeout().isPresent()) { + unaryCallSettings + .retrySettings() + .setTotalTimeout(operationTimeouts.getOperationTimeout().get()); + } + + // Configure attempt timeout - if the user hasn't explicitly configured it, then fallback to + // overall timeout to match previous behavior + Optional effectiveAttemptTimeout = + operationTimeouts.getAttemptTimeout().or(operationTimeouts.getOperationTimeout()); + if (effectiveAttemptTimeout.isPresent()) { + unaryCallSettings.retrySettings().setInitialRpcTimeout(effectiveAttemptTimeout.get()); + unaryCallSettings.retrySettings().setMaxRpcTimeout(effectiveAttemptTimeout.get()); + } + } + + private void configureNonRetryableCallSettings( + @UnderInitialization BigtableHBaseVeneeringSettings this, + UnaryCallSettings.Builder unaryCallSettings, + OperationTimeouts operationTimeouts) { + unaryCallSettings.setRetryableCodes(Collections.emptySet()); + + // NOTE: attempt timeouts are not configured for non-retriable rpcs + if (operationTimeouts.getOperationTimeout().isPresent()) { + unaryCallSettings + .retrySettings() + .setLogicalTimeout(operationTimeouts.getOperationTimeout().get()); + } + } + + private BigtableIOOperationTimeouts buildCallSettings( + @UnderInitialization BigtableHBaseVeneeringSettings this, BigtableOptions options) { + boolean useTimeouts = options.getCallOptionsConfig().isUseTimeout(); + + Optional bulkMutateOverallTimeout = + Optional.of(Duration.ofMillis(options.getCallOptionsConfig().getMutateRpcTimeoutMs())); + OperationTimeouts bulkMutateTimeouts = + new OperationTimeouts( + Optional.absent(), + extracBulkMutateAttemptTimeout(options), + bulkMutateOverallTimeout); + + Optional bulkReadPartialRowTimeout = + Optional.of(Duration.ofMillis(options.getRetryOptions().getReadPartialRowTimeoutMillis())); + Optional bulkReadRowsOverallTimeout = + Optional.of(Duration.ofMillis(options.getCallOptionsConfig().getReadStreamRpcTimeoutMs())); + OperationTimeouts bulkReadTimeouts = + new OperationTimeouts( + bulkReadPartialRowTimeout, + extractBulkReadRowsAttemptTimeout(options), + bulkReadRowsOverallTimeout); + + OperationTimeouts unaryTimeouts = + new OperationTimeouts( + Optional.absent(), + extractUnaryAttemptTimeout(options), + Optional.of(Duration.ofMillis(options.getCallOptionsConfig().getShortRpcTimeoutMs()))); + + return new BigtableIOOperationTimeouts( + useTimeouts, unaryTimeouts, bulkReadTimeouts, bulkMutateTimeouts); + } + + private Optional extractUnaryAttemptTimeout( + @UnderInitialization BigtableHBaseVeneeringSettings this, BigtableOptions options) { + + if (!options.getCallOptionsConfig().getShortRpcAttemptTimeoutMs().isPresent()) { + return Optional.of(DEFAULT_UNARY_ATTEMPT_TIMEOUTS); + } + long attemptTimeout = options.getCallOptionsConfig().getShortRpcAttemptTimeoutMs().get(); + return Optional.of(Duration.ofMillis(attemptTimeout)); + } + + private Optional extracBulkMutateAttemptTimeout( + @UnderInitialization BigtableHBaseVeneeringSettings this, BigtableOptions options) { + + if (!options.getCallOptionsConfig().getMutateRpcAttemptTimeoutMs().isPresent()) { + return Optional.of(DEFAULT_BULK_MUTATE_ATTEMPT_TIMEOUTS); + } + long attemptTimeout = options.getCallOptionsConfig().getMutateRpcAttemptTimeoutMs().get(); + return Optional.of(Duration.ofMillis(attemptTimeout)); + } + + private Optional extractBulkReadRowsAttemptTimeout( + @UnderInitialization BigtableHBaseVeneeringSettings this, BigtableOptions options) { + + if (!options.getCallOptionsConfig().getReadStreamRpcAttemptTimeoutMs().isPresent()) { + return Optional.absent(); + } + long attemptTimeout = options.getCallOptionsConfig().getReadStreamRpcAttemptTimeoutMs().get(); + return Optional.of(Duration.ofMillis(attemptTimeout)); + } + + private Set extractRetryCodesFromConfig( + @UnderInitialization BigtableHBaseVeneeringSettings this, + Set defaultCodes, + BigtableOptions options) { + + Set codes = new HashSet<>(defaultCodes); + + for (Status.Code code : options.getRetryOptions().getRetryableStatusCodes()) { + codes.add(StatusCode.Code.valueOf(code.name())); + } + + if (options.getRetryOptions().retryOnDeadlineExceeded()) { + codes.add(StatusCode.Code.DEADLINE_EXCEEDED); + } + + return codes; + } + + private void configureCredentialProvider( + @UnderInitialization BigtableHBaseVeneeringSettings this, + StubSettings.Builder stubSettings, + BigtableOptions options) + throws IOException { + CredentialOptions credentialOptions = options.getCredentialOptions(); + switch (credentialOptions.getCredentialType()) { + case DefaultCredentials: + GoogleCredentials credentials = GoogleCredentials.getApplicationDefault(); + + if (credentials instanceof ServiceAccountCredentials) { + stubSettings.setCredentialsProvider( + FixedCredentialsProvider.create( + getJwtToken((ServiceAccountCredentials) credentials))); + } + return; + case P12: + CredentialOptions.P12CredentialOptions p12Options = + (CredentialOptions.P12CredentialOptions) credentialOptions; + stubSettings.setCredentialsProvider( + FixedCredentialsProvider.create( + getCredentialFromPrivateKeyServiceAccount( + p12Options.getServiceAccount(), p12Options.getKeyFile()))); + return; + case SuppliedCredentials: + stubSettings.setCredentialsProvider( + FixedCredentialsProvider.create( + ((CredentialOptions.UserSuppliedCredentialOptions) credentialOptions) + .getCredential())); + return; + case SuppliedJson: + CredentialOptions.JsonCredentialsOptions jsonCredentialsOptions = + (CredentialOptions.JsonCredentialsOptions) credentialOptions; + synchronized (jsonCredentialsOptions) { + if (jsonCredentialsOptions.getCachedCredentials() == null) { + jsonCredentialsOptions.setCachedCredentails( + GoogleCredentials.fromStream(jsonCredentialsOptions.getInputStream())); + } + stubSettings.setCredentialsProvider( + FixedCredentialsProvider.create(jsonCredentialsOptions.getCachedCredentials())); + } + return; + case None: + stubSettings.setCredentialsProvider(NoCredentialsProvider.create()); + return; + default: + throw new IllegalStateException( + "Cannot process Credential type: " + credentialOptions.getCredentialType()); + } + } + + private static Credentials getJwtToken(ServiceAccountCredentials serviceAccount) { + return ServiceAccountJwtAccessCredentials.newBuilder() + .setClientEmail(serviceAccount.getClientEmail()) + .setClientId(serviceAccount.getClientId()) + .setPrivateKey(serviceAccount.getPrivateKey()) + .setPrivateKeyId(serviceAccount.getPrivateKeyId()) + .build(); + } + + public static Credentials getCredentialFromPrivateKeyServiceAccount( + String serviceAccountEmail, String privateKeyFile) throws IOException { + + try { + KeyStore keyStore = KeyStore.getInstance("PKCS12"); + + try (FileInputStream fin = new FileInputStream(privateKeyFile)) { + keyStore.load(fin, "notasecret".toCharArray()); + } + PrivateKey privateKey = + (PrivateKey) keyStore.getKey("privatekey", "notasecret".toCharArray()); + + if (privateKey == null) { + throw new IllegalStateException("private key cannot be null"); + } + return ServiceAccountJwtAccessCredentials.newBuilder() + .setClientEmail(serviceAccountEmail) + .setPrivateKey(privateKey) + .build(); + } catch (GeneralSecurityException exception) { + throw new RuntimeException("exception while retrieving credentials", exception); + } + } + + static class NoTimeoutsInterceptor implements ClientInterceptor { + static final CallOptions.Key SKIP_DEFAULT_ATTEMPT_TIMEOUT = + CallOptions.Key.createWithDefault("SKIP_DEFAULT_ATTEMPT_TIMEOUT", false); + + @Override + public ClientCall interceptCall( + MethodDescriptor methodDescriptor, CallOptions callOptions, Channel channel) { + + if (!callOptions.getOption(SKIP_DEFAULT_ATTEMPT_TIMEOUT)) { + callOptions = callOptions.withDeadline(Deadline.after(6, TimeUnit.MINUTES)); + } else { + callOptions = callOptions.withDeadline(null); + } + + return channel.newCall(methodDescriptor, callOptions); + } + } + + static class BigtableIOOperationTimeouts { + private final boolean useTimeouts; + private final OperationTimeouts unaryTimeouts; + private final OperationTimeouts bulkReadRowsTimeouts; + private final OperationTimeouts bulkMutateTimeouts; + + BigtableIOOperationTimeouts( + boolean useTimeouts, + OperationTimeouts unaryTimeouts, + OperationTimeouts bulkReadRowsTimeouts, + OperationTimeouts bulkMutateTimeouts) { + this.useTimeouts = useTimeouts; + this.unaryTimeouts = unaryTimeouts; + this.bulkReadRowsTimeouts = bulkReadRowsTimeouts; + this.bulkMutateTimeouts = bulkMutateTimeouts; + } + + boolean getUseTimeouts() { + return useTimeouts; + } + + OperationTimeouts getUnaryTimeouts() { + return unaryTimeouts; + } + + OperationTimeouts getBulkReadRowsTimeouts() { + return bulkReadRowsTimeouts; + } + + OperationTimeouts getBulkMutateTimeouts() { + return bulkMutateTimeouts; + } + } + + static class OperationTimeouts { + static final OperationTimeouts EMPTY = + new OperationTimeouts( + Optional.absent(), Optional.absent(), Optional.absent()); + + // responseTimeouts are only relevant to streaming RPCs, they limit the amount of timeout a + // stream will wait for the next response message. This is synonymous with attemptTimeouts in + // unary RPCs since they receive a single response (so its ignored). + private final Optional responseTimeout; + private final Optional attemptTimeout; + private final Optional operationTimeout; + + OperationTimeouts( + Optional responseTimeout, + Optional attemptTimeout, + Optional operationTimeout) { + this.responseTimeout = responseTimeout; + this.attemptTimeout = attemptTimeout; + this.operationTimeout = operationTimeout; + } + + Optional getResponseTimeout() { + return responseTimeout; + } + + Optional getAttemptTimeout() { + return attemptTimeout; + } + + Optional getOperationTimeout() { + return operationTimeout; + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 1ea0240ff565..4329d99084cd 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -26,8 +26,8 @@ import com.google.bigtable.v2.Mutation; import com.google.bigtable.v2.Row; import com.google.bigtable.v2.RowFilter; -import com.google.bigtable.v2.SampleRowKeysResponse; import com.google.cloud.bigtable.config.BigtableOptions; +import com.google.cloud.bigtable.data.v2.models.KeyOffset; import com.google.protobuf.ByteString; import java.io.IOException; import java.util.ArrayList; @@ -960,8 +960,7 @@ protected BigtableSource withEstimatedSizeBytes(Long estimatedSizeBytes) { * boundaries and estimated sizes. We can use these samples to ensure that splits are on * different tablets, and possibly generate sub-splits within tablets. */ - private List getSampleRowKeys(PipelineOptions pipelineOptions) - throws IOException { + private List getSampleRowKeys(PipelineOptions pipelineOptions) throws IOException { return config.getBigtableService(pipelineOptions).getSampleRowKeys(this); } @@ -1079,7 +1078,7 @@ private static List mergeRanges(List ranges) { /** Helper that splits this source into bundles based on Cloud Bigtable sampled row keys. */ private List splitBasedOnSamples( - long desiredBundleSizeBytes, List sampleRowKeys) { + long desiredBundleSizeBytes, List sampleRowKeys) { // There are no regions, or no samples available. Just scan the entire range. if (sampleRowKeys.isEmpty()) { LOG.info("Not splitting source {} because no sample row keys are available.", this); @@ -1103,9 +1102,7 @@ private List splitBasedOnSamples( * keys. */ private List splitRangeBasedOnSamples( - long desiredBundleSizeBytes, - List sampleRowKeys, - ByteKeyRange range) { + long desiredBundleSizeBytes, List sampleRowKeys, ByteKeyRange range) { // Loop through all sampled responses and generate splits from the ones that overlap the // scan range. The main complication is that we must track the end range of the previous @@ -1113,9 +1110,9 @@ private List splitRangeBasedOnSamples( ByteKey lastEndKey = ByteKey.EMPTY; long lastOffset = 0; ImmutableList.Builder splits = ImmutableList.builder(); - for (SampleRowKeysResponse response : sampleRowKeys) { - ByteKey responseEndKey = makeByteKey(response.getRowKey()); - long responseOffset = response.getOffsetBytes(); + for (KeyOffset keyOffset : sampleRowKeys) { + ByteKey responseEndKey = makeByteKey(keyOffset.getKey()); + long responseOffset = keyOffset.getOffsetBytes(); checkState( responseOffset >= lastOffset, "Expected response byte offset %s to come after the last offset %s", @@ -1184,16 +1181,16 @@ public long getEstimatedSizeBytes(PipelineOptions options) throws IOException { * Computes the estimated size in bytes based on the total size of all samples that overlap the * key ranges this source will scan. */ - private long getEstimatedSizeBytesBasedOnSamples(List samples) { + private long getEstimatedSizeBytesBasedOnSamples(List samples) { long estimatedSizeBytes = 0; long lastOffset = 0; ByteKey currentStartKey = ByteKey.EMPTY; // Compute the total estimated size as the size of each sample that overlaps the scan range. // TODO: In future, Bigtable service may provide finer grained APIs, e.g., to sample given a // filter or to sample on a given key range. - for (SampleRowKeysResponse response : samples) { - ByteKey currentEndKey = makeByteKey(response.getRowKey()); - long currentOffset = response.getOffsetBytes(); + for (KeyOffset keyOffset : samples) { + ByteKey currentEndKey = makeByteKey(keyOffset.getKey()); + long currentOffset = keyOffset.getOffsetBytes(); if (!currentStartKey.isEmpty() && currentStartKey.equals(currentEndKey)) { // Skip an empty region. lastOffset = currentOffset; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java index 1f266979fcc2..a84f55f66c44 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java @@ -20,8 +20,8 @@ import com.google.bigtable.v2.MutateRowResponse; import com.google.bigtable.v2.Mutation; import com.google.bigtable.v2.Row; -import com.google.bigtable.v2.SampleRowKeysResponse; import com.google.cloud.bigtable.config.BigtableOptions; +import com.google.cloud.bigtable.data.v2.models.KeyOffset; import com.google.protobuf.ByteString; import java.io.IOException; import java.io.Serializable; @@ -101,5 +101,5 @@ interface Reader { * Returns a set of row keys sampled from the underlying table. These contain information about * the distribution of keys within the table. */ - List getSampleRowKeys(BigtableSource source) throws IOException; + List getSampleRowKeys(BigtableSource source) throws IOException; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java index 22db0026e685..d843026e67b1 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java @@ -17,44 +17,56 @@ */ package org.apache.beam.sdk.io.gcp.bigtable; +import static org.apache.beam.sdk.io.gcp.bigtable.BigtableHBaseVeneeringSettings.BigtableIOOperationTimeouts; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.MoreExecutors.directExecutor; -import com.google.bigtable.admin.v2.GetTableRequest; +import com.google.api.gax.batching.Batcher; +import com.google.api.gax.grpc.GrpcCallContext; +import com.google.api.gax.rpc.ResponseObserver; +import com.google.api.gax.rpc.StreamController; +import com.google.bigtable.v2.Cell; +import com.google.bigtable.v2.Column; +import com.google.bigtable.v2.Family; import com.google.bigtable.v2.MutateRowResponse; -import com.google.bigtable.v2.MutateRowsRequest; import com.google.bigtable.v2.Mutation; import com.google.bigtable.v2.ReadRowsRequest; import com.google.bigtable.v2.Row; import com.google.bigtable.v2.RowFilter; import com.google.bigtable.v2.RowRange; import com.google.bigtable.v2.RowSet; -import com.google.bigtable.v2.SampleRowKeysRequest; -import com.google.bigtable.v2.SampleRowKeysResponse; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; import com.google.cloud.bigtable.config.BigtableOptions; -import com.google.cloud.bigtable.grpc.BigtableSession; -import com.google.cloud.bigtable.grpc.BigtableTableName; -import com.google.cloud.bigtable.grpc.async.BulkMutation; -import com.google.cloud.bigtable.grpc.scanner.FlatRow; -import com.google.cloud.bigtable.grpc.scanner.FlatRowConverter; -import com.google.cloud.bigtable.grpc.scanner.ResultScanner; -import com.google.cloud.bigtable.grpc.scanner.ScanHandler; -import com.google.cloud.bigtable.util.ByteStringComparator; +import com.google.cloud.bigtable.data.v2.BigtableDataClient; +import com.google.cloud.bigtable.data.v2.BigtableDataSettings; +import com.google.cloud.bigtable.data.v2.internal.ByteStringComparator; +import com.google.cloud.bigtable.data.v2.internal.NameUtil; +import com.google.cloud.bigtable.data.v2.models.Filters; +import com.google.cloud.bigtable.data.v2.models.KeyOffset; +import com.google.cloud.bigtable.data.v2.models.Query; +import com.google.cloud.bigtable.data.v2.models.RowAdapter; +import com.google.cloud.bigtable.data.v2.models.RowMutationEntry; import com.google.protobuf.ByteString; +import io.grpc.CallOptions; +import io.grpc.Deadline; import io.grpc.Status.Code; import io.grpc.StatusRuntimeException; -import io.grpc.stub.StreamObserver; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.Comparator; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; +import java.util.Objects; import java.util.Queue; +import java.util.TreeMap; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.TimeUnit; import javax.annotation.Nonnull; import org.apache.beam.runners.core.metrics.GcpResourceIdentifiers; import org.apache.beam.runners.core.metrics.MonitoringInfoConstants; @@ -66,7 +78,6 @@ 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.collect.ComparisonChain; 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.io.Closer; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.FutureCallback; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.Futures; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.SettableFuture; @@ -90,10 +101,16 @@ class BigtableServiceImpl implements BigtableService { private static final long MIN_BYTE_BUFFER_SIZE = 100 * 1024 * 1024; // 100MB public BigtableServiceImpl(BigtableOptions options) { - this.options = options; + try { + this.options = options; + this.veneeringSettings = BigtableHBaseVeneeringSettings.create(options); + } catch (IOException e) { + throw new RuntimeException(e); + } } private final BigtableOptions options; + private final BigtableHBaseVeneeringSettings veneeringSettings; @Override public BigtableOptions getBigtableOptions() { @@ -102,19 +119,18 @@ public BigtableOptions getBigtableOptions() { @Override public BigtableWriterImpl openForWriting(String tableId) throws IOException { - BigtableSession session = new BigtableSession(options); - BigtableTableName tableName = options.getInstanceName().toTableName(tableId); - return new BigtableWriterImpl(session, tableName); + LOG.info("Opening for writing with settings " + veneeringSettings.getDataSettings().toString()); + return new BigtableWriterImpl( + BigtableDataClient.create(veneeringSettings.getDataSettings()), + veneeringSettings.getDataSettings(), + tableId); } @Override public boolean tableExists(String tableId) throws IOException { - try (BigtableSession session = new BigtableSession(options)) { - GetTableRequest getTable = - GetTableRequest.newBuilder() - .setName(options.getInstanceName().toTableNameStr(tableId)) - .build(); - session.getTableAdminClient().getTable(getTable); + try (BigtableTableAdminClient adminClient = + BigtableTableAdminClient.create(veneeringSettings.getTableAdminSettings())) { + adminClient.getTable(tableId); return true; } catch (StatusRuntimeException e) { if (e.getStatus().getCode() == Code.NOT_FOUND) { @@ -130,40 +146,48 @@ public boolean tableExists(String tableId) throws IOException { @VisibleForTesting static class BigtableReaderImpl implements Reader { - private BigtableSession session; + private BigtableDataSettings settings; + private BigtableDataClient client; private final BigtableSource source; - private ResultScanner results; + private Iterator results; + private BigtableIOOperationTimeouts operationTimeouts; + private Row currentRow; @VisibleForTesting - BigtableReaderImpl(BigtableSession session, BigtableSource source) { - this.session = session; + BigtableReaderImpl( + BigtableDataClient client, + BigtableDataSettings settings, + BigtableSource source, + BigtableIOOperationTimeouts operationTimeouts) { + this.settings = settings; + this.client = client; this.source = source; + this.operationTimeouts = operationTimeouts; } @Override public boolean start() throws IOException { - RowSet.Builder rowSetBuilder = RowSet.newBuilder(); + ServiceCallMetric serviceCallMetric = + createCallMetric( + settings.getProjectId(), settings.getInstanceId(), source.getTableId().get()); + + Query query = Query.create(source.getTableId().get()); for (ByteKeyRange sourceRange : source.getRanges()) { - rowSetBuilder = - rowSetBuilder.addRowRanges( - RowRange.newBuilder() - .setStartKeyClosed(ByteString.copyFrom(sourceRange.getStartKey().getValue())) - .setEndKeyOpen(ByteString.copyFrom(sourceRange.getEndKey().getValue()))); + query.range( + ByteString.copyFrom(sourceRange.getStartKey().getValue()), + ByteString.copyFrom(sourceRange.getEndKey().getValue())); } - RowSet rowSet = rowSetBuilder.build(); - String tableNameSr = - session.getOptions().getInstanceName().toTableNameStr(source.getTableId().get()); - - ServiceCallMetric serviceCallMetric = createCallMetric(session, source.getTableId().get()); - ReadRowsRequest.Builder requestB = - ReadRowsRequest.newBuilder().setRows(rowSet).setTableName(tableNameSr); if (source.getRowFilter() != null) { - requestB.setFilter(source.getRowFilter()); + query.filter(Filters.FILTERS.fromProto(source.getRowFilter())); } try { - results = session.getDataClient().readRows(requestB.build()); + results = + client + .readRowsCallable(new BeamRowAdapter()) + .call(query, BigtableServiceImpl.createScanCallContext(operationTimeouts)) + .iterator(); serviceCallMetric.call("ok"); } catch (StatusRuntimeException e) { serviceCallMetric.call(e.getStatus().getCode().toString()); @@ -174,8 +198,11 @@ public boolean start() throws IOException { @Override public boolean advance() throws IOException { - currentRow = results.next(); - return currentRow != null; + if (results.hasNext()) { + currentRow = results.next(); + return true; + } + return false; } @Override @@ -183,23 +210,13 @@ public void close() throws IOException { // Goal: by the end of this function, both results and session are null and closed, // independent of what errors they throw or prior state. - if (session == null) { + if (client == null) { // Only possible when previously closed, so we know that results is also null. return; } - // Session does not implement Closeable -- it's AutoCloseable. So we can't register it with - // the Closer, but we can use the Closer to simplify the error handling. - try (Closer closer = Closer.create()) { - if (results != null) { - closer.register(results); - results = null; - } - - session.close(); - } finally { - session = null; - } + client.close(); + client = null; } @Override @@ -213,8 +230,7 @@ public Row getCurrentRow() throws NoSuchElementException { @VisibleForTesting static class BigtableSegmentReaderImpl implements Reader { - private BigtableSession session; - + private BigtableDataClient client; private @Nullable ReadRowsRequest nextRequest; private @Nullable Row currentRow; private @Nullable Future future; @@ -222,6 +238,7 @@ static class BigtableSegmentReaderImpl implements Reader { private final int refillSegmentWaterMark; private final long maxSegmentByteSize; private ServiceCallMetric serviceCallMetric; + private BigtableIOOperationTimeouts operationTimeouts; private static class UpstreamResults { private final List rows; @@ -233,7 +250,12 @@ private UpstreamResults(List rows, @Nullable ReadRowsRequest nextRequest) { } } - static BigtableSegmentReaderImpl create(BigtableSession session, BigtableSource source) { + static BigtableSegmentReaderImpl create( + BigtableDataClient client, + BigtableDataSettings settings, + BigtableSource source, + BigtableIOOperationTimeouts operationTimeouts) { + RowSet.Builder rowSetBuilder = RowSet.newBuilder(); if (source.getRanges().isEmpty()) { rowSetBuilder = RowSet.newBuilder().addRowRanges(RowRange.getDefaultInstance()); @@ -257,42 +279,50 @@ static BigtableSegmentReaderImpl create(BigtableSession session, BigtableSource (Runtime.getRuntime().totalMemory() * DEFAULT_BYTE_LIMIT_PERCENTAGE)); return new BigtableSegmentReaderImpl( - session, - session.getOptions().getInstanceName().toTableNameStr(source.getTableId().get()), + client, + settings, + source.getTableId().get(), rowSet, source.getMaxBufferElementCount(), maxSegmentByteSize, filter, - createCallMetric(session, source.getTableId().get())); + createCallMetric( + settings.getProjectId(), settings.getInstanceId(), source.getTableId().get()), + operationTimeouts); } @VisibleForTesting BigtableSegmentReaderImpl( - BigtableSession session, - String tableName, + BigtableDataClient client, + BigtableDataSettings settings, + String tableId, RowSet rowSet, int maxRowsInBuffer, long maxSegmentByteSize, RowFilter filter, - ServiceCallMetric serviceCallMetric) { + ServiceCallMetric serviceCallMetric, + BigtableIOOperationTimeouts operationTimeouts) { if (rowSet.equals(rowSet.getDefaultInstanceForType())) { rowSet = RowSet.newBuilder().addRowRanges(RowRange.getDefaultInstance()).build(); } ReadRowsRequest request = ReadRowsRequest.newBuilder() - .setTableName(tableName) + .setTableName( + NameUtil.formatTableName( + settings.getProjectId(), settings.getInstanceId(), tableId)) .setRows(rowSet) .setFilter(filter) .setRowsLimit(maxRowsInBuffer) .build(); - this.session = session; + this.client = client; this.nextRequest = request; this.maxSegmentByteSize = maxSegmentByteSize; this.serviceCallMetric = serviceCallMetric; this.buffer = new ArrayDeque<>(); // Asynchronously refill buffer when there is 10% of the elements are left this.refillSegmentWaterMark = (int) (request.getRowsLimit() * WATERMARK_PERCENTAGE); + this.operationTimeouts = operationTimeouts; } @Override @@ -314,57 +344,61 @@ public boolean advance() throws IOException { } private Future fetchNextSegment() { - SettableFuture f = SettableFuture.create(); + SettableFuture future = SettableFuture.create(); // When the nextRequest is null, the last fill completed and the buffer contains the last rows if (nextRequest == null) { - f.set(new UpstreamResults(ImmutableList.of(), null)); - return f; + future.set(new UpstreamResults(ImmutableList.of(), null)); + return future; } - // TODO(diegomez): Remove atomic ScanHandler for simpler StreamObserver/Future implementation - AtomicReference atomicScanHandler = new AtomicReference<>(); - ScanHandler handler = - session - .getDataClient() - .readFlatRows( - nextRequest, - new StreamObserver() { - List rows = new ArrayList<>(); - long currentByteSize = 0; - boolean byteLimitReached = false; - - @Override - public void onNext(FlatRow flatRow) { - Row row = FlatRowConverter.convert(flatRow); - currentByteSize += row.getSerializedSize(); - rows.add(row); - - if (currentByteSize > maxSegmentByteSize) { - byteLimitReached = true; - atomicScanHandler.get().cancel(); - return; - } - } - - @Override - public void onError(Throwable e) { - f.setException(e); - } - - @Override - public void onCompleted() { - ReadRowsRequest nextNextRequest = null; - - // When requested rows < limit, the current request will be the last - if (byteLimitReached || rows.size() == nextRequest.getRowsLimit()) { - nextNextRequest = - truncateRequest(nextRequest, rows.get(rows.size() - 1).getKey()); - } - f.set(new UpstreamResults(rows, nextNextRequest)); - } - }); - atomicScanHandler.set(handler); - return f; + client + .readRowsCallable(new BeamRowAdapter()) + .call( + Query.fromProto(nextRequest), + new ResponseObserver() { + private StreamController controller; + + List rows = new ArrayList<>(); + + long currentByteSize = 0; + boolean byteLimitReached = false; + + @Override + public void onStart(StreamController controller) { + this.controller = controller; + } + + @Override + public void onResponse(com.google.bigtable.v2.Row response) { + // calculate size of the response + currentByteSize += response.getSerializedSize(); + rows.add(response); + if (currentByteSize > maxSegmentByteSize) { + byteLimitReached = true; + controller.cancel(); + return; + } + } + + @Override + public void onError(Throwable t) { + future.setException(t); + } + + @Override + public void onComplete() { + ReadRowsRequest nextNextRequest = null; + + // When requested rows < limit, the current request will be the last + if (byteLimitReached || rows.size() == nextRequest.getRowsLimit()) { + nextNextRequest = + truncateRequest(nextRequest, rows.get(rows.size() - 1).getKey()); + } + future.set(new UpstreamResults(rows, nextNextRequest)); + } + }, + createScanCallContext(operationTimeouts)); + return future; } private void waitReadRowsFuture() throws IOException { @@ -413,7 +447,7 @@ private ReadRowsRequest truncateRequest(ReadRowsRequest request, ByteString last @Override public void close() throws IOException { - session.close(); + client.close(); } @Override @@ -427,14 +461,17 @@ public Row getCurrentRow() throws NoSuchElementException { @VisibleForTesting static class BigtableWriterImpl implements Writer { - private BigtableSession session; - private BulkMutation bulkMutation; - private BigtableTableName tableName; - - BigtableWriterImpl(BigtableSession session, BigtableTableName tableName) { - this.session = session; - bulkMutation = session.createBulkMutation(tableName); - this.tableName = tableName; + private BigtableDataClient client; + private BigtableDataSettings settings; + private Batcher bulkMutation; + private String tableId; + private GrpcCallContext context = GrpcCallContext.createDefault(); + + BigtableWriterImpl(BigtableDataClient client, BigtableDataSettings settings, String tableId) { + this.client = client; + this.settings = settings; + this.tableId = tableId; + this.bulkMutation = client.newBulkMutationBatcher(tableId, context); } @Override @@ -456,6 +493,7 @@ public void close() throws IOException { if (bulkMutation != null) { try { bulkMutation.flush(); + bulkMutation.close(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); // We fail since flush() operation was interrupted. @@ -464,9 +502,9 @@ public void close() throws IOException { bulkMutation = null; } } finally { - if (session != null) { - session.close(); - session = null; + if (client != null) { + client.close(); + client = null; } } } @@ -474,12 +512,13 @@ public void close() throws IOException { @Override public CompletionStage writeRecord(KV> record) throws IOException { - MutateRowsRequest.Entry request = - MutateRowsRequest.Entry.newBuilder() - .setRowKey(record.getKey()) - .addAllMutations(record.getValue()) - .build(); + com.google.cloud.bigtable.data.v2.models.Mutation mutation = + com.google.cloud.bigtable.data.v2.models.Mutation.fromProtoUnsafe(record.getValue()); + + RowMutationEntry entry = RowMutationEntry.createFromMutationUnsafe(record.getKey(), mutation); + + // Populate metrics HashMap baseLabels = new HashMap<>(); baseLabels.put(MonitoringInfoConstants.Labels.PTRANSFORM, ""); baseLabels.put(MonitoringInfoConstants.Labels.SERVICE, "BigTable"); @@ -487,33 +526,37 @@ public CompletionStage writeRecord(KV result = new CompletableFuture<>(); + Futures.addCallback( - new VendoredListenableFutureAdapter<>(bulkMutation.add(request)), + new VendoredListenableFutureAdapter<>(bulkMutation.add(entry)), new FutureCallback() { @Override public void onSuccess(MutateRowResponse mutateRowResponse) { + // TODO throttling logic to update dataflow counter + // long throttledTime = ((ApiCallContext) + // context).getOption(Batcher.THROTTLED_TIME_KEY); + result.complete(mutateRowResponse); serviceCallMetric.call("ok"); } @Override public void onFailure(Throwable throwable) { + // TODO throttling logic to update dataflow counter + // long throttledTime = ((ApiCallContext) + // context).getOption(Batcher.THROTTLED_TIME_KEY); + if (throwable instanceof StatusRuntimeException) { serviceCallMetric.call( ((StatusRuntimeException) throwable).getStatus().getCode().value()); @@ -535,43 +578,84 @@ public String toString() { @Override public Reader createReader(BigtableSource source) throws IOException { - BigtableSession session = new BigtableSession(options); + LOG.info( + "Creating a Reader for Bigtable with settings: " + veneeringSettings.getDataSettings()); + BigtableDataClient client = BigtableDataClient.create(veneeringSettings.getDataSettings()); if (source.getMaxBufferElementCount() != null) { - return BigtableSegmentReaderImpl.create(session, source); + return BigtableSegmentReaderImpl.create( + client, + veneeringSettings.getDataSettings(), + source, + veneeringSettings.getOperationTimeouts()); } else { - return new BigtableReaderImpl(session, source); + return new BigtableReaderImpl( + client, + veneeringSettings.getDataSettings(), + source, + veneeringSettings.getOperationTimeouts()); } } + // Support 2 bigtable-hbase features not directly available in veneer: + // - disabling timeouts - when timeouts are disabled, bigtable-hbase ignores user configured + // timeouts and forces 6 minute deadlines per attempt for all RPCs except scans. This is + // implemented by an interceptor. However the interceptor must be informed that this is a scan + // - per attempt deadlines - vener doesn't implement deadlines for attempts. To workaround this, + // the timeouts are set per call in the ApiCallContext. However this creates a separate issue of + // over running the operation deadline, so gRPC deadline is also set. + private static GrpcCallContext createScanCallContext( + BigtableIOOperationTimeouts operationTimeouts) { + GrpcCallContext ctx = GrpcCallContext.createDefault(); + + if (!operationTimeouts.getUseTimeouts()) { + ctx = + ctx.withCallOptions( + CallOptions.DEFAULT.withOption( + BigtableHBaseVeneeringSettings.NoTimeoutsInterceptor.SKIP_DEFAULT_ATTEMPT_TIMEOUT, + true)); + } else { + if (operationTimeouts.getBulkReadRowsTimeouts().getOperationTimeout().isPresent()) { + ctx.withCallOptions( + CallOptions.DEFAULT.withDeadline( + Deadline.after( + operationTimeouts + .getBulkMutateTimeouts() + .getOperationTimeout() + .get() + .toMillis(), + TimeUnit.MILLISECONDS))); + } + if (operationTimeouts.getBulkReadRowsTimeouts().getAttemptTimeout().isPresent()) { + ctx = + ctx.withTimeout(operationTimeouts.getBulkReadRowsTimeouts().getAttemptTimeout().get()); + } + } + return ctx; + } + @Override - public List getSampleRowKeys(BigtableSource source) throws IOException { - try (BigtableSession session = new BigtableSession(options)) { - SampleRowKeysRequest request = - SampleRowKeysRequest.newBuilder() - .setTableName(options.getInstanceName().toTableNameStr(source.getTableId().get())) - .build(); - return session.getDataClient().sampleRowKeys(request); + public List getSampleRowKeys(BigtableSource source) throws IOException { + try (BigtableDataClient client = + BigtableDataClient.create(veneeringSettings.getDataSettings())) { + return client.sampleRowKeys(source.getTableId().get()); } } @VisibleForTesting - public static ServiceCallMetric createCallMetric(BigtableSession session, String tableId) { + public static ServiceCallMetric createCallMetric( + String projectId, String instanceId, String tableId) { HashMap baseLabels = new HashMap<>(); baseLabels.put(MonitoringInfoConstants.Labels.PTRANSFORM, ""); baseLabels.put(MonitoringInfoConstants.Labels.SERVICE, "BigTable"); baseLabels.put(MonitoringInfoConstants.Labels.METHOD, "google.bigtable.v2.ReadRows"); baseLabels.put( MonitoringInfoConstants.Labels.RESOURCE, - GcpResourceIdentifiers.bigtableResource( - session.getOptions().getProjectId(), session.getOptions().getInstanceId(), tableId)); - baseLabels.put( - MonitoringInfoConstants.Labels.BIGTABLE_PROJECT_ID, session.getOptions().getProjectId()); - baseLabels.put( - MonitoringInfoConstants.Labels.INSTANCE_ID, session.getOptions().getInstanceId()); + GcpResourceIdentifiers.bigtableResource(projectId, instanceId, tableId)); + baseLabels.put(MonitoringInfoConstants.Labels.BIGTABLE_PROJECT_ID, projectId); + baseLabels.put(MonitoringInfoConstants.Labels.INSTANCE_ID, instanceId); baseLabels.put( MonitoringInfoConstants.Labels.TABLE_ID, - GcpResourceIdentifiers.bigtableTableID( - session.getOptions().getProjectId(), session.getOptions().getInstanceId(), tableId)); + GcpResourceIdentifiers.bigtableTableID(projectId, instanceId, tableId)); return new ServiceCallMetric(MonitoringInfoConstants.Urns.API_REQUEST_COUNT, baseLabels); } @@ -656,4 +740,118 @@ public int compareTo(@Nonnull EndPoint o) { .result(); } } + + static class BeamRowAdapter implements RowAdapter { + @Override + public RowBuilder createRowBuilder() { + return new DefaultRowBuilder(); + } + + @Override + public boolean isScanMarkerRow(com.google.bigtable.v2.Row row) { + return Objects.equals(row, com.google.bigtable.v2.Row.getDefaultInstance()); + } + + @Override + public ByteString getKey(com.google.bigtable.v2.Row row) { + return row.getKey(); + } + + private static class DefaultRowBuilder + implements RowAdapter.RowBuilder { + private com.google.bigtable.v2.Row.Builder protoBuilder = + com.google.bigtable.v2.Row.newBuilder(); + + private TreeMap>> + cellsByFamilyColumn = new TreeMap<>(); + private TreeMap> cellsByColumn = + new TreeMap<>(Comparator.comparing(o -> o.toString(StandardCharsets.UTF_8))); + private ImmutableList.Builder currentColumnCells; + + private ByteString qualifier; + private ByteString previousQualifier; + private String family; + private String previousFamily; + + private ByteString value; + private List labels; + private long timestamp; + + @Override + public void startRow(ByteString key) { + protoBuilder.setKey(key); + } + + @Override + public void startCell( + String family, ByteString qualifier, long timestamp, List labels, long size) { + this.family = family; + this.qualifier = qualifier; + this.timestamp = timestamp; + this.labels = labels; + this.value = ByteString.EMPTY; + } + + @Override + public void cellValue(ByteString value) { + this.value = this.value.concat(value); + } + + @Override + public void finishCell() { + if (!qualifier.equals(previousQualifier)) { + previousQualifier = qualifier; + currentColumnCells = ImmutableList.builder(); + cellsByColumn.put(qualifier, currentColumnCells); + } + if (!family.equals(previousFamily)) { + previousFamily = family; + this.cellsByFamilyColumn.put(family, cellsByColumn); + } + + Cell cell = + Cell.newBuilder() + .setValue(value) + .addAllLabels(labels) + .setTimestampMicros(timestamp) + .build(); + currentColumnCells.add(cell); + } + + @Override + public com.google.bigtable.v2.Row finishRow() { + for (String family : cellsByFamilyColumn.keySet()) { + Family.Builder f = Family.newBuilder().setName(family); + for (ByteString column : cellsByFamilyColumn.get(family).keySet()) { + Column c = + Column.newBuilder() + .setQualifier(column) + .addAllCells(cellsByFamilyColumn.get(family).get(column).build()) + .build(); + f.addColumns(c); + } + protoBuilder.addFamilies(f); + } + return protoBuilder.build(); + } + + @Override + public void reset() { + this.qualifier = null; + this.previousQualifier = null; + this.family = null; + this.previousFamily = null; + + protoBuilder = com.google.bigtable.v2.Row.newBuilder(); + + this.cellsByColumn.clear(); + this.cellsByFamilyColumn.clear(); + } + + @Override + public com.google.bigtable.v2.Row createScanMarkerRow(ByteString key) { + return com.google.bigtable.v2.Row.newBuilder().getDefaultInstanceForType(); + } + } + } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/VendoredListenableFutureAdapter.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/VendoredListenableFutureAdapter.java index d7e0ae4c652e..e67543858420 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/VendoredListenableFutureAdapter.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/VendoredListenableFutureAdapter.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.gcp.bigtable; +import com.google.api.core.ApiFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; @@ -26,10 +27,9 @@ /** Adapts {@link ListenableFuture} from bigtable-client-core to vendored guava. */ class VendoredListenableFutureAdapter implements ListenableFuture { - private final com.google.common.util.concurrent.ListenableFuture underlying; + private final ApiFuture underlying; - VendoredListenableFutureAdapter( - com.google.common.util.concurrent.ListenableFuture underlying) { + VendoredListenableFutureAdapter(ApiFuture underlying) { this.underlying = underlying; } @@ -55,12 +55,12 @@ public boolean isDone() { @Override public V get() throws InterruptedException, ExecutionException { - return underlying.get(); + return (V) underlying.get(); } @Override public V get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { - return underlying.get(timeout, unit); + return (V) underlying.get(timeout, unit); } } 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 a0ade3fc2290..fc1eebd521fa 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 @@ -86,9 +86,6 @@ public void testGcpApiSurface() throws Exception { Matchers.>equalTo(com.google.common.base.Optional.class), Matchers.>equalTo(com.google.common.base.Supplier.class), Matchers.>equalTo(com.google.api.gax.rpc.StatusCode.Code.class), - Matchers.>equalTo(com.google.cloud.bigtable.grpc.BigtableClusterName.class), - Matchers.>equalTo(com.google.cloud.bigtable.grpc.BigtableInstanceName.class), - Matchers.>equalTo(com.google.cloud.bigtable.grpc.BigtableTableName.class), Matchers.>equalTo(com.google.cloud.BaseServiceException.class), Matchers.>equalTo(com.google.cloud.BaseServiceException.Error.class), Matchers.>equalTo(com.google.cloud.BaseServiceException.ExceptionData.class), @@ -121,7 +118,6 @@ public void testGcpApiSurface() throws Exception { classesInPackage("javax"), classesInPackage("org.apache.avro"), classesInPackage("org.apache.beam"), - classesInPackage("org.apache.commons.logging"), classesInPackage("org.codehaus.jackson"), classesInPackage("org.joda.time"), Matchers.>equalTo(org.threeten.bp.Duration.class), diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java index d7e9ded7eec4..6a9fb5392f97 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java @@ -49,12 +49,12 @@ import com.google.bigtable.v2.Mutation.SetCell; import com.google.bigtable.v2.Row; import com.google.bigtable.v2.RowFilter; -import com.google.bigtable.v2.SampleRowKeysResponse; import com.google.cloud.bigtable.config.BigtableOptions; import com.google.cloud.bigtable.config.BulkOptions; import com.google.cloud.bigtable.config.CredentialOptions; import com.google.cloud.bigtable.config.CredentialOptions.CredentialType; import com.google.cloud.bigtable.config.RetryOptions; +import com.google.cloud.bigtable.data.v2.models.KeyOffset; import com.google.protobuf.ByteString; import java.io.IOException; import java.io.Serializable; @@ -1604,7 +1604,7 @@ private static List makeTableData(String tableId, int numRows) { /** A {@link BigtableService} implementation that stores tables and their contents in memory. */ private static class FakeBigtableService implements BigtableService { private final Map> tables = new HashMap<>(); - private final Map> sampleRowKeys = new HashMap<>(); + private final Map> sampleRowKeys = new HashMap<>(); @Override public BigtableOptions getBigtableOptions() { @@ -1645,8 +1645,8 @@ public FakeBigtableWriter openForWriting(String tableId) { } @Override - public List getSampleRowKeys(BigtableSource source) { - List samples = sampleRowKeys.get(source.getTableId().get()); + public List getSampleRowKeys(BigtableSource source) { + List samples = sampleRowKeys.get(source.getTableId().get()); checkNotNull(samples, "No samples found for table %s", source.getTableId().get()); return samples; } @@ -1657,26 +1657,25 @@ void setupSampleRowKeys(String tableId, int numSamples, long bytesPerRow) { checkArgument(numSamples > 0, "Number of samples must be positive: %s", numSamples); checkArgument(bytesPerRow > 0, "Bytes/Row must be positive: %s", bytesPerRow); - ImmutableList.Builder ret = ImmutableList.builder(); + ImmutableList.Builder ret = ImmutableList.builder(); SortedMap rows = getTable(tableId); + System.out.println("num of rows = " + rows.size()); int currentSample = 1; int rowsSoFar = 0; for (Map.Entry entry : rows.entrySet()) { if (((double) rowsSoFar) / rows.size() >= ((double) currentSample) / numSamples) { // add the sample with the total number of bytes in the table before this key. - ret.add( - SampleRowKeysResponse.newBuilder() - .setRowKey(entry.getKey()) - .setOffsetBytes(rowsSoFar * bytesPerRow) - .build()); + ret.add(KeyOffset.create(entry.getKey(), rowsSoFar * bytesPerRow)); // Move on to next sample currentSample++; } ++rowsSoFar; } - // Add the last sample indicating the end of the table, with all rows before it. - ret.add(SampleRowKeysResponse.newBuilder().setOffsetBytes(rows.size() * bytesPerRow).build()); + System.out.println("ret size=" + ret.build().size()); + // Add the last sample indicating the end of the table, with all rows before it + ret.add(KeyOffset.create(ByteString.EMPTY, rows.size() * bytesPerRow)); + sampleRowKeys.put(tableId, ret.build()); } } @@ -1698,7 +1697,7 @@ public FailureBigtableWriter openForWriting(String tableId) { } @Override - public List getSampleRowKeys(BigtableSource source) { + public List getSampleRowKeys(BigtableSource source) { if (failureOptions.getFailAtSplit()) { throw new RuntimeException("Fake Exception in getSampleRowKeys()"); } @@ -1781,7 +1780,7 @@ private boolean rangesContainsKey(List ranges, ByteKey key) { } @Override - public Row getCurrentRow() { + public com.google.bigtable.v2.Row getCurrentRow() { if (currentRow == null) { throw new NoSuchElementException(); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java index 97c51856cc99..5d15eab214fc 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java @@ -17,45 +17,52 @@ */ package org.apache.beam.sdk.io.gcp.bigtable; +import static org.apache.beam.sdk.io.gcp.bigtable.BigtableHBaseVeneeringSettings.BigtableIOOperationTimeouts; +import static org.apache.beam.sdk.io.gcp.bigtable.BigtableHBaseVeneeringSettings.OperationTimeouts.EMPTY; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import com.google.bigtable.v2.MutateRowResponse; -import com.google.bigtable.v2.MutateRowsRequest; -import com.google.bigtable.v2.MutateRowsRequest.Entry; +import com.google.api.core.ApiFuture; +import com.google.api.core.SettableApiFuture; +import com.google.api.gax.batching.Batcher; +import com.google.api.gax.rpc.ApiCallContext; +import com.google.api.gax.rpc.ResponseObserver; +import com.google.api.gax.rpc.ServerStream; +import com.google.api.gax.rpc.ServerStreamingCallable; +import com.google.api.gax.rpc.StreamController; +import com.google.bigtable.v2.Cell; +import com.google.bigtable.v2.Column; +import com.google.bigtable.v2.Family; import com.google.bigtable.v2.Mutation; -import com.google.bigtable.v2.Mutation.SetCell; -import com.google.bigtable.v2.ReadRowsRequest; import com.google.bigtable.v2.Row; import com.google.bigtable.v2.RowFilter; import com.google.bigtable.v2.RowRange; import com.google.bigtable.v2.RowSet; -import com.google.cloud.bigtable.config.BigtableOptions; -import com.google.cloud.bigtable.grpc.BigtableDataClient; -import com.google.cloud.bigtable.grpc.BigtableInstanceName; -import com.google.cloud.bigtable.grpc.BigtableSession; -import com.google.cloud.bigtable.grpc.BigtableTableName; -import com.google.cloud.bigtable.grpc.async.BulkMutation; -import com.google.cloud.bigtable.grpc.scanner.FlatRow; -import com.google.cloud.bigtable.grpc.scanner.FlatRowConverter; -import com.google.cloud.bigtable.grpc.scanner.ResultScanner; -import com.google.cloud.bigtable.grpc.scanner.ScanHandler; -import com.google.common.util.concurrent.SettableFuture; +import com.google.cloud.bigtable.data.v2.BigtableDataClient; +import com.google.cloud.bigtable.data.v2.BigtableDataSettings; +import com.google.cloud.bigtable.data.v2.internal.RequestContext; +import com.google.cloud.bigtable.data.v2.models.Query; +import com.google.cloud.bigtable.data.v2.models.RowAdapter; +import com.google.cloud.bigtable.data.v2.models.RowMutationEntry; +import com.google.cloud.bigtable.data.v2.stub.EnhancedBigtableStub; import com.google.protobuf.ByteString; import io.grpc.Status; import io.grpc.StatusRuntimeException; -import io.grpc.stub.StreamObserver; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.HashMap; +import java.util.Iterator; import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.beam.runners.core.metrics.GcpResourceIdentifiers; @@ -82,7 +89,6 @@ import org.mockito.MockitoAnnotations; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import org.mockito.stubbing.OngoingStubbing; /** Unit tests of BigtableServiceImpl. */ @RunWith(JUnit4.class) @@ -97,31 +103,37 @@ public class BigtableServiceImplTest { private static final long DEFAULT_BYTE_SEGMENT_SIZE = 1024 * 1024 * 1000; private static final String DEFAULT_PREFIX = "b"; - private static final BigtableTableName TABLE_NAME = - new BigtableInstanceName(PROJECT_ID, INSTANCE_ID).toTableName(TABLE_ID); + private static RequestContext requestContext = + RequestContext.create(PROJECT_ID, INSTANCE_ID, "default"); - @Mock private BigtableSession mockSession; - - @Mock private BulkMutation mockBulkMutation; + @Mock private Batcher mockBatcher; @Mock private BigtableDataClient mockBigtableDataClient; - @Mock private BigtableSource mockBigtableSource; + @Mock private EnhancedBigtableStub mockStub; - @Mock private ScanHandler scanHandler; + private BigtableDataSettings bigtableDataSettings; + + @Mock private BigtableSource mockBigtableSource; @Mock private ServiceCallMetric mockCallMetric; - @Captor private ArgumentCaptor requestCaptor; + @Captor private ArgumentCaptor requestCaptor; + + private static AtomicBoolean cancelled = new AtomicBoolean(false); + + private BigtableIOOperationTimeouts operationTimeouts = + new BigtableIOOperationTimeouts(false, EMPTY, EMPTY, EMPTY); @Before public void setup() { MockitoAnnotations.initMocks(this); - BigtableOptions options = - new BigtableOptions.Builder().setProjectId(PROJECT_ID).setInstanceId(INSTANCE_ID).build(); - when(mockSession.getOptions()).thenReturn(options); - when(mockSession.createBulkMutation(eq(TABLE_NAME))).thenReturn(mockBulkMutation); - when(mockSession.getDataClient()).thenReturn(mockBigtableDataClient); + this.bigtableDataSettings = + BigtableDataSettings.newBuilder() + .setProjectId(PROJECT_ID) + .setInstanceId(INSTANCE_ID) + .setAppProfileId("default") + .build(); // Setup the ProcessWideContainer for testing metrics are set. MetricsContainerImpl container = new MetricsContainerImpl(null); MetricsEnvironment.setProcessWideContainer(container); @@ -135,25 +147,42 @@ public void setup() { * @throws InterruptedException */ @Test + @SuppressWarnings("unchecked") public void testRead() throws IOException { ByteKey start = ByteKey.copyFrom("a".getBytes(StandardCharsets.UTF_8)); ByteKey end = ByteKey.copyFrom("b".getBytes(StandardCharsets.UTF_8)); when(mockBigtableSource.getRanges()).thenReturn(Arrays.asList(ByteKeyRange.of(start, end))); when(mockBigtableSource.getTableId()).thenReturn(StaticValueProvider.of(TABLE_ID)); - @SuppressWarnings("unchecked") - ResultScanner mockResultScanner = Mockito.mock(ResultScanner.class); + Row expectedRow = Row.newBuilder().setKey(ByteString.copyFromUtf8("a")).build(); - when(mockResultScanner.next()).thenReturn(expectedRow).thenReturn(null); - when(mockBigtableDataClient.readRows(any(ReadRowsRequest.class))).thenReturn(mockResultScanner); + + // Set up iterator to be returned by ServerStream.iterator() + Iterator mockIterator = Mockito.mock(Iterator.class); + when(mockIterator.next()).thenReturn(expectedRow).thenReturn(null); + when(mockIterator.hasNext()).thenReturn(true).thenReturn(false); + // Set up ServerStream to be returned by callable.call(Query) + ServerStream mockRows = Mockito.mock(ServerStream.class); + when(mockRows.iterator()).thenReturn(mockIterator); + // Set up Callable to be returned by stub.createReadRowsCallable() + ServerStreamingCallable mockCallable = Mockito.mock(ServerStreamingCallable.class); + when(mockCallable.call( + any(com.google.cloud.bigtable.data.v2.models.Query.class), any(ApiCallContext.class))) + .thenReturn(mockRows); + when(mockStub.createReadRowsCallable(any(RowAdapter.class))).thenReturn(mockCallable); + ServerStreamingCallable callable = + mockStub.createReadRowsCallable(new BigtableServiceImpl.BeamRowAdapter()); + // Set up client to return callable + when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); + BigtableService.Reader underTest = - new BigtableServiceImpl.BigtableReaderImpl(mockSession, mockBigtableSource); + new BigtableServiceImpl.BigtableReaderImpl( + mockBigtableDataClient, bigtableDataSettings, mockBigtableSource, operationTimeouts); underTest.start(); Assert.assertEquals(expectedRow, underTest.getCurrentRow()); Assert.assertFalse(underTest.advance()); underTest.close(); - verify(mockResultScanner, times(1)).close(); verifyMetricWasSet("google.bigtable.v2.ReadRows", "ok", 1); } @@ -170,23 +199,45 @@ public void testReadSingleRangeBelowSegmentLimit() throws Exception { generateRowRange( generateByteString(DEFAULT_PREFIX, 0), generateByteString(DEFAULT_PREFIX, 1))); - FlatRow expectedRow = FlatRow.newBuilder().withRowKey(ByteString.copyFromUtf8("a")).build(); + Row expectedRow = Row.newBuilder().setKey(ByteString.copyFromUtf8("a")).build(); - when(mockBigtableDataClient.readFlatRows(any(ReadRowsRequest.class), any())) - .thenAnswer(mockReadRowsAnswer(Arrays.asList(expectedRow))); + // Set up Callable to be returned by stub.createReadRowsCallable() + ServerStreamingCallable mockCallable = Mockito.mock(ServerStreamingCallable.class); + // Set up ResponseObserver to return expectedRow + doAnswer( + new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + ((ResponseObserver) invocation.getArgument(1)).onResponse(expectedRow); + ((ResponseObserver) invocation.getArgument(1)).onComplete(); + return null; + } + }) + .when(mockCallable) + .call( + any(com.google.cloud.bigtable.data.v2.models.Query.class), + any(ResponseObserver.class), + any(ApiCallContext.class)); + when(mockStub.createReadRowsCallable(any(RowAdapter.class))).thenReturn(mockCallable); + ServerStreamingCallable callable = + mockStub.createReadRowsCallable(new BigtableServiceImpl.BeamRowAdapter()); + // Set up client to return callable + when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); BigtableService.Reader underTest = new BigtableServiceImpl.BigtableSegmentReaderImpl( - mockSession, + mockBigtableDataClient, + bigtableDataSettings, TABLE_ID, ranges.build(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, RowFilter.getDefaultInstance(), - mockCallMetric); + mockCallMetric, + operationTimeouts); underTest.start(); - Assert.assertEquals(FlatRowConverter.convert(expectedRow), underTest.getCurrentRow()); + Assert.assertEquals(expectedRow, underTest.getCurrentRow()); Assert.assertFalse(underTest.advance()); underTest.close(); @@ -207,24 +258,37 @@ public void testReadSingleRangeAboveSegmentLimit() throws IOException { generateByteString(DEFAULT_PREFIX, 0), generateByteString(DEFAULT_PREFIX, SEGMENT_SIZE * 2))); - OngoingStubbing stub = - when(mockBigtableDataClient.readFlatRows(any(ReadRowsRequest.class), any())); - List> expectedResults = + // Set up Callable to be returned by stub.createReadRowsCallable() + ServerStreamingCallable mockCallable = Mockito.mock(ServerStreamingCallable.class); + + List> expectedResults = ImmutableList.of( generateSegmentResult(DEFAULT_PREFIX, 0, SEGMENT_SIZE), generateSegmentResult(DEFAULT_PREFIX, SEGMENT_SIZE, SEGMENT_SIZE), ImmutableList.of()); - expectRowResults(stub, expectedResults); + + // Return multiple answers when mockCallable is called + doAnswer(new MultipleAnswer(expectedResults)) + .when(mockCallable) + .call(any(Query.class), any(ResponseObserver.class), any(ApiCallContext.class)); + + when(mockStub.createReadRowsCallable(any(RowAdapter.class))).thenReturn(mockCallable); + ServerStreamingCallable callable = + mockStub.createReadRowsCallable(new BigtableServiceImpl.BeamRowAdapter()); + // Set up client to return callable + when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); BigtableService.Reader underTest = new BigtableServiceImpl.BigtableSegmentReaderImpl( - mockSession, + mockBigtableDataClient, + bigtableDataSettings, TABLE_ID, ranges.build(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, RowFilter.getDefaultInstance(), - mockCallMetric); + mockCallMetric, + operationTimeouts); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); @@ -233,10 +297,7 @@ public void testReadSingleRangeAboveSegmentLimit() throws IOException { } while (underTest.advance()); Assert.assertEquals( - expectedResults.stream() - .flatMap(Collection::stream) - .map(i -> FlatRowConverter.convert(i)) - .collect(Collectors.toList()), + expectedResults.stream().flatMap(Collection::stream).collect(Collectors.toList()), actualResults); underTest.close(); Mockito.verify(mockCallMetric, Mockito.times(3)).call("ok"); @@ -263,24 +324,38 @@ public void testReadMultipleRanges() throws IOException { generateByteString(DEFAULT_PREFIX, SEGMENT_SIZE), generateByteString(DEFAULT_PREFIX, SEGMENT_SIZE * 2))); - OngoingStubbing stub = - when(mockBigtableDataClient.readFlatRows(any(ReadRowsRequest.class), any())); - List> expectedResults = + // Set up Callable to be returned by stub.createReadRowsCallable() + ServerStreamingCallable mockCallable = Mockito.mock(ServerStreamingCallable.class); + + List> expectedResults = ImmutableList.of( generateSegmentResult(DEFAULT_PREFIX, 0, SEGMENT_SIZE), generateSegmentResult(DEFAULT_PREFIX, SEGMENT_SIZE, SEGMENT_SIZE), ImmutableList.of()); - expectRowResults(stub, expectedResults); + + // Return multiple answers when mockCallable is called + doAnswer(new MultipleAnswer(expectedResults)) + .when(mockCallable) + .call(any(Query.class), any(ResponseObserver.class), any(ApiCallContext.class)); + + when(mockStub.createReadRowsCallable(any(RowAdapter.class))).thenReturn(mockCallable); + ServerStreamingCallable callable = + mockStub.createReadRowsCallable(new BigtableServiceImpl.BeamRowAdapter()); + + // Set up client to return callable + when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); BigtableService.Reader underTest = new BigtableServiceImpl.BigtableSegmentReaderImpl( - mockSession, + mockBigtableDataClient, + bigtableDataSettings, TABLE_ID, ranges.build(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, RowFilter.getDefaultInstance(), - mockCallMetric); + mockCallMetric, + operationTimeouts); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); @@ -289,10 +364,7 @@ public void testReadMultipleRanges() throws IOException { } while (underTest.advance()); Assert.assertEquals( - expectedResults.stream() - .flatMap(Collection::stream) - .map(i -> FlatRowConverter.convert(i)) - .collect(Collectors.toList()), + expectedResults.stream().flatMap(Collection::stream).collect(Collectors.toList()), actualResults); underTest.close(); @@ -323,24 +395,35 @@ public void testReadMultipleRangesOverlappingKeys() throws IOException { generateByteString(DEFAULT_PREFIX, (int) (SEGMENT_SIZE * .7)), generateByteString(DEFAULT_PREFIX, SEGMENT_SIZE * 2))); - OngoingStubbing stub = - when(mockBigtableDataClient.readFlatRows(any(ReadRowsRequest.class), any())); - List> expectedResults = + List> expectedResults = ImmutableList.of( generateSegmentResult(DEFAULT_PREFIX, 0, SEGMENT_SIZE), generateSegmentResult(DEFAULT_PREFIX, SEGMENT_SIZE, SEGMENT_SIZE), ImmutableList.of()); - expectRowResults(stub, expectedResults); + + // Set up Callable to be returned by stub.createReadRowsCallable() + ServerStreamingCallable mockCallable = Mockito.mock(ServerStreamingCallable.class); + // Return multiple answers when mockCallable is called + doAnswer(new MultipleAnswer(expectedResults)) + .when(mockCallable) + .call(any(Query.class), any(ResponseObserver.class), any(ApiCallContext.class)); + when(mockStub.createReadRowsCallable(any(RowAdapter.class))).thenReturn(mockCallable); + ServerStreamingCallable callable = + mockStub.createReadRowsCallable(new BigtableServiceImpl.BeamRowAdapter()); + // Set up client to return callable + when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); BigtableService.Reader underTest = new BigtableServiceImpl.BigtableSegmentReaderImpl( - mockSession, + mockBigtableDataClient, + bigtableDataSettings, TABLE_ID, ranges.build(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, RowFilter.getDefaultInstance(), - mockCallMetric); + mockCallMetric, + operationTimeouts); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); @@ -349,10 +432,7 @@ public void testReadMultipleRangesOverlappingKeys() throws IOException { } while (underTest.advance()); Assert.assertEquals( - expectedResults.stream() - .flatMap(Collection::stream) - .map(i -> FlatRowConverter.convert(i)) - .collect(Collectors.toList()), + expectedResults.stream().flatMap(Collection::stream).collect(Collectors.toList()), actualResults); underTest.close(); @@ -367,25 +447,36 @@ public void testReadMultipleRangesOverlappingKeys() throws IOException { */ @Test public void testReadFullTableScan() throws IOException { - OngoingStubbing stub = - when(mockBigtableDataClient.readFlatRows(any(ReadRowsRequest.class), any())); - List> expectedResults = + List> expectedResults = ImmutableList.of( generateSegmentResult(DEFAULT_PREFIX, 0, SEGMENT_SIZE), generateSegmentResult(DEFAULT_PREFIX, SEGMENT_SIZE, SEGMENT_SIZE), generateSegmentResult(DEFAULT_PREFIX, SEGMENT_SIZE * 2, SEGMENT_SIZE), ImmutableList.of()); - expectRowResults(stub, expectedResults); + + // Set up Callable to be returned by stub.createReadRowsCallable() + ServerStreamingCallable mockCallable = Mockito.mock(ServerStreamingCallable.class); + // Return multiple answers when mockCallable is called + doAnswer(new MultipleAnswer(expectedResults)) + .when(mockCallable) + .call(any(Query.class), any(ResponseObserver.class), any(ApiCallContext.class)); + when(mockStub.createReadRowsCallable(any(RowAdapter.class))).thenReturn(mockCallable); + ServerStreamingCallable callable = + mockStub.createReadRowsCallable(new BigtableServiceImpl.BeamRowAdapter()); + // Set up client to return callable + when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); BigtableService.Reader underTest = new BigtableServiceImpl.BigtableSegmentReaderImpl( - mockSession, + mockBigtableDataClient, + bigtableDataSettings, TABLE_ID, RowSet.getDefaultInstance(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, RowFilter.getDefaultInstance(), - mockCallMetric); + mockCallMetric, + operationTimeouts); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); @@ -394,10 +485,7 @@ public void testReadFullTableScan() throws IOException { } while (underTest.advance()); Assert.assertEquals( - expectedResults.stream() - .flatMap(Collection::stream) - .map(i -> FlatRowConverter.convert(i)) - .collect(Collectors.toList()), + expectedResults.stream().flatMap(Collection::stream).collect(Collectors.toList()), actualResults); underTest.close(); @@ -428,43 +516,54 @@ public void testReadFillBuffer() throws IOException { generateByteString(DEFAULT_PREFIX, SEGMENT_SIZE), generateByteString(DEFAULT_PREFIX, SEGMENT_SIZE * 2))); - OngoingStubbing stub = - when(mockBigtableDataClient.readFlatRows(any(ReadRowsRequest.class), any())); - List> expectedResults = + List> expectedResults = ImmutableList.of( generateSegmentResult(DEFAULT_PREFIX, 0, SEGMENT_SIZE), generateSegmentResult(DEFAULT_PREFIX, SEGMENT_SIZE, SEGMENT_SIZE), ImmutableList.of()); - expectRowResults(stub, expectedResults); + + // Set up Callable to be returned by stub.createReadRowsCallable() + ServerStreamingCallable mockCallable = Mockito.mock(ServerStreamingCallable.class); + // Return multiple answers when mockCallable is called + doAnswer(new MultipleAnswer(expectedResults)) + .when(mockCallable) + .call(any(Query.class), any(ResponseObserver.class), any(ApiCallContext.class)); + when(mockStub.createReadRowsCallable(any(RowAdapter.class))).thenReturn(mockCallable); + ServerStreamingCallable callable = + mockStub.createReadRowsCallable(new BigtableServiceImpl.BeamRowAdapter()); + // Set up client to return callable + when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); BigtableService.Reader underTest = new BigtableServiceImpl.BigtableSegmentReaderImpl( - mockSession, + mockBigtableDataClient, + bigtableDataSettings, TABLE_ID, ranges.build(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, RowFilter.getDefaultInstance(), - mockCallMetric); + mockCallMetric, + operationTimeouts); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); - verify(mockBigtableDataClient, times(1)) - .readFlatRows(requestCaptor.capture(), any(StreamObserver.class)); - Assert.assertEquals(3, requestCaptor.getValue().getRows().getRowRangesCount()); + + verify(callable, times(1)) + .call(requestCaptor.capture(), any(ResponseObserver.class), any(ApiCallContext.class)); + Assert.assertEquals( + 3, requestCaptor.getValue().toProto(requestContext).getRows().getRowRangesCount()); do { actualResults.add(underTest.getCurrentRow()); } while (underTest.advance()); - verify(mockBigtableDataClient, times(3)) - .readFlatRows(requestCaptor.capture(), any(StreamObserver.class)); - Assert.assertEquals(1, requestCaptor.getValue().getRows().getRowRangesCount()); + verify(callable, times(3)) + .call(requestCaptor.capture(), any(ResponseObserver.class), any(ApiCallContext.class)); + Assert.assertEquals( + 1, requestCaptor.getValue().toProto(requestContext).getRows().getRowRangesCount()); Assert.assertEquals( - expectedResults.stream() - .flatMap(Collection::stream) - .map(i -> FlatRowConverter.convert(i)) - .collect(Collectors.toList()), + expectedResults.stream().flatMap(Collection::stream).collect(Collectors.toList()), actualResults); underTest.close(); @@ -473,8 +572,8 @@ public void testReadFillBuffer() throws IOException { /** * This test checks that the buffer will stop filling up once the byte limit is reached. It will - * cancel the ScanHandler after reached the limit. This test completes one fill and contains one - * Row after the first buffer has been completed. The test cheaks the current available memory in + * cancel the controller after reached the limit. This test completes one fill and contains one + * Row after the first buffer has been completed. The test checks the current available memory in * the JVM and uses a percent of it to mock the original behavior. Range: [b00000, b00010) * * @throws IOException @@ -489,25 +588,49 @@ public void testReadByteLimitBuffer() throws IOException { generateByteString(DEFAULT_PREFIX, 0), generateByteString(DEFAULT_PREFIX, SEGMENT_SIZE)); - OngoingStubbing stub = - when(mockBigtableDataClient.readFlatRows(any(ReadRowsRequest.class), any())); - List> expectedResults = + List> expectedResults = ImmutableList.of( generateLargeSegmentResult(DEFAULT_PREFIX, 0, numOfRowsInsideBuffer), generateSegmentResult( DEFAULT_PREFIX, numOfRowsInsideBuffer, SEGMENT_SIZE - numOfRowsInsideBuffer), ImmutableList.of()); - expectRowResults(stub, expectedResults); + + // Set up Callable to be returned by stub.createReadRowsCallable() + ServerStreamingCallable mockCallable = Mockito.mock(ServerStreamingCallable.class); + + // Create a mock stream controller + StreamController mockController = Mockito.mock(StreamController.class); + doAnswer( + new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + cancelled.set(true); + return null; + } + }) + .when(mockController) + .cancel(); + // Return multiple answers when mockCallable is called + doAnswer(new MultipleAnswer(expectedResults, mockController)) + .when(mockCallable) + .call(any(Query.class), any(ResponseObserver.class), any(ApiCallContext.class)); + when(mockStub.createReadRowsCallable(any(RowAdapter.class))).thenReturn(mockCallable); + ServerStreamingCallable callable = + mockStub.createReadRowsCallable(new BigtableServiceImpl.BeamRowAdapter()); + // Set up client to return callable + when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); BigtableService.Reader underTest = new BigtableServiceImpl.BigtableSegmentReaderImpl( - mockSession, + mockBigtableDataClient, + bigtableDataSettings, TABLE_ID, RowSet.newBuilder().addRowRanges(mockRowRange).build(), SEGMENT_SIZE, segmentByteLimit, RowFilter.getDefaultInstance(), - mockCallMetric); + mockCallMetric, + operationTimeouts); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); @@ -516,10 +639,7 @@ public void testReadByteLimitBuffer() throws IOException { } while (underTest.advance()); Assert.assertEquals( - expectedResults.stream() - .flatMap(Collection::stream) - .map(i -> FlatRowConverter.convert(i)) - .collect(Collectors.toList()), + expectedResults.stream().flatMap(Collection::stream).collect(Collectors.toList()), actualResults); underTest.close(); @@ -539,31 +659,41 @@ public void testReadSegmentExceptionHandling() throws IOException { generateRowRange( generateByteString(DEFAULT_PREFIX, 0), generateByteString(DEFAULT_PREFIX, 1))); - when(mockBigtableDataClient.readFlatRows(any(ReadRowsRequest.class), any())) - .thenAnswer( - new Answer() { + // Set up Callable to be returned by stub.createReadRowsCallable() + ServerStreamingCallable mockCallable = Mockito.mock(ServerStreamingCallable.class); + doAnswer( + new Answer() { @Override - public ScanHandler answer(InvocationOnMock invocationOnMock) throws Throwable { - StreamObserver flatRowObserver = invocationOnMock.getArgument(1); + public Void answer(InvocationOnMock invocationOnMock) throws Throwable { new Thread() { @Override public void run() { - flatRowObserver.onError(Status.INVALID_ARGUMENT.asRuntimeException()); + ((ResponseObserver) invocationOnMock.getArgument(1)) + .onError(Status.INVALID_ARGUMENT.asRuntimeException()); } }.start(); - return scanHandler; + return null; } - }); + }) + .when(mockCallable) + .call(any(Query.class), any(ResponseObserver.class), any(ApiCallContext.class)); + when(mockStub.createReadRowsCallable(any(RowAdapter.class))).thenReturn(mockCallable); + ServerStreamingCallable callable = + mockStub.createReadRowsCallable(new BigtableServiceImpl.BeamRowAdapter()); + // Set up client to return callable + when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); BigtableService.Reader underTest = new BigtableServiceImpl.BigtableSegmentReaderImpl( - mockSession, + mockBigtableDataClient, + bigtableDataSettings, TABLE_ID, ranges.build(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, RowFilter.getDefaultInstance(), - mockCallMetric); + mockCallMetric, + operationTimeouts); IOException returnedError = null; try { @@ -571,6 +701,7 @@ public void run() { } catch (IOException e) { returnedError = e; } + Assert.assertTrue(returnedError.getCause() instanceof StatusRuntimeException); Mockito.verify(mockCallMetric, Mockito.times(1)) @@ -578,7 +709,7 @@ public void run() { } /** - * This test ensures that protobuf creation and interactions with {@link BulkMutation} work as + * This test ensures that protobuf creation and interactions with {@link Batcher} work as * expected. * * @throws IOException @@ -586,27 +717,36 @@ public void run() { */ @Test public void testWrite() throws IOException, InterruptedException { - when(mockBigtableSource.getTableId()).thenReturn(StaticValueProvider.of(TABLE_ID)); + doReturn(mockBatcher).when(mockBigtableDataClient).newBulkMutationBatcher(any(), any()); + ArgumentCaptor captor = ArgumentCaptor.forClass(RowMutationEntry.class); + ApiFuture fakeResponse = SettableApiFuture.create(); + when(mockBatcher.add(any(RowMutationEntry.class))).thenReturn(fakeResponse); + BigtableService.Writer underTest = - new BigtableServiceImpl.BigtableWriterImpl(mockSession, TABLE_NAME); + new BigtableServiceImpl.BigtableWriterImpl( + mockBigtableDataClient, bigtableDataSettings, TABLE_ID); + ByteString key = ByteString.copyFromUtf8("key"); Mutation mutation = Mutation.newBuilder() - .setSetCell(SetCell.newBuilder().setFamilyName("Family").build()) + .setSetCell( + Mutation.SetCell.newBuilder() + .setFamilyName("Family") + .setColumnQualifier(ByteString.copyFromUtf8("q")) + .setValue(ByteString.copyFromUtf8("value")) + .build()) .build(); - ByteString key = ByteString.copyFromUtf8("key"); - - SettableFuture fakeResponse = SettableFuture.create(); - when(mockBulkMutation.add(any(MutateRowsRequest.Entry.class))).thenReturn(fakeResponse); underTest.writeRecord(KV.of(key, ImmutableList.of(mutation))); - Entry expected = - MutateRowsRequest.Entry.newBuilder().setRowKey(key).addMutations(mutation).build(); - verify(mockBulkMutation, times(1)).add(expected); - underTest.close(); + verify(mockBatcher).add(captor.capture()); - verify(mockBulkMutation, times(1)).flush(); + assertEquals(key, captor.getValue().toProto().getRowKey()); + assertTrue(captor.getValue().toProto().getMutations(0).hasSetCell()); + assertEquals( + "Family", captor.getValue().toProto().getMutations(0).getSetCell().getFamilyName()); + underTest.close(); + verify(mockBatcher, times(1)).flush(); } private void verifyMetricWasSet(String method, String status, long count) { @@ -632,71 +772,87 @@ private void verifyMetricWasSet(String method, String status, long count) { assertEquals(count, (long) container.getCounter(name).getCumulative()); } - private Answer mockReadRowsAnswer(List rows) { - return new Answer() { - @Override - public ScanHandler answer(InvocationOnMock invocationOnMock) throws Throwable { - StreamObserver flatRowObserver = invocationOnMock.getArgument(1); - new Thread() { - @Override - public void run() { - for (int i = 0; i < rows.size(); i++) { - flatRowObserver.onNext(rows.get(i)); - } - flatRowObserver.onCompleted(); - } - }.start(); - - return scanHandler; - } - }; - } - private static RowRange generateRowRange(ByteString start, ByteString end) { return RowRange.newBuilder().setStartKeyClosed(start).setEndKeyOpen(end).build(); } - private static List generateSegmentResult(String prefix, int startIndex, int count) { + private static List generateSegmentResult(String prefix, int startIndex, int count) { return generateSegmentResult(prefix, startIndex, count, false); } - private static List generateLargeSegmentResult( - String prefix, int startIndex, int count) { + private static List generateLargeSegmentResult(String prefix, int startIndex, int count) { return generateSegmentResult(prefix, startIndex, count, true); } - private static List generateSegmentResult( + private static List generateSegmentResult( String prefix, int startIndex, int count, boolean largeRow) { byte[] largeMemory = new byte[(int) DEFAULT_ROW_SIZE]; return IntStream.range(startIndex, startIndex + count) .mapToObj( i -> { - FlatRow.Builder builder = FlatRow.newBuilder(); + Row.Builder builder = Row.newBuilder(); if (!largeRow) { - builder.withRowKey(generateByteString(prefix, i)); + builder.setKey(generateByteString(prefix, i)); } else { builder - .withRowKey(generateByteString(prefix, i)) - .addCell( - "Family", - ByteString.copyFromUtf8("LargeMemoryRow"), - System.currentTimeMillis(), - ByteString.copyFrom(largeMemory)); + .setKey(generateByteString(prefix, i)) + .addFamilies( + Family.newBuilder() + .setName("Family") + .addColumns( + Column.newBuilder() + .setQualifier(ByteString.copyFromUtf8("LargeMemoryRow")) + .addCells( + Cell.newBuilder() + .setValue(ByteString.copyFrom(largeMemory)) + .setTimestampMicros(System.currentTimeMillis()) + .build()) + .build()) + .build()); } return builder.build(); }) .collect(Collectors.toList()); } - private OngoingStubbing expectRowResults( - OngoingStubbing stub, List> results) { - for (List result : results) { - stub = stub.thenAnswer(mockReadRowsAnswer(result)); - } - return stub; - } - private static ByteString generateByteString(String prefix, int index) { return ByteString.copyFromUtf8(prefix + String.format("%05d", index)); } + + private static class MultipleAnswer implements Answer { + + private final List rows = new ArrayList<>(); + private StreamController streamController; + + MultipleAnswer(List> expectedRows) { + this(expectedRows, null); + } + + MultipleAnswer(List> expectedRows, StreamController streamController) { + rows.addAll(expectedRows.stream().flatMap(row -> row.stream()).collect(Collectors.toList())); + this.streamController = streamController; + } + + @Override + public T answer(InvocationOnMock invocation) throws Throwable { + if (streamController != null) { + ((ResponseObserver) invocation.getArgument(1)).onStart(streamController); + } + long rowLimit = ((Query) invocation.getArgument(0)).toProto(requestContext).getRowsLimit(); + for (long i = 0; i < rowLimit; i++) { + if (rows.isEmpty()) { + break; + } + Row row = rows.remove(0); + ((ResponseObserver) invocation.getArgument(1)).onResponse(row); + if (cancelled.get()) { + ((ResponseObserver) invocation.getArgument(1)).onComplete(); + cancelled.set(false); + return null; + } + } + ((ResponseObserver) invocation.getArgument(1)).onComplete(); + return null; + } + } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java index 0e29cf11777b..eeeba49a008e 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java @@ -19,27 +19,22 @@ import static org.hamcrest.MatcherAssert.assertThat; -import com.google.bigtable.admin.v2.ColumnFamily; -import com.google.bigtable.admin.v2.CreateTableRequest; -import com.google.bigtable.admin.v2.DeleteTableRequest; -import com.google.bigtable.admin.v2.GetTableRequest; -import com.google.bigtable.admin.v2.Table; +import com.google.api.gax.rpc.ServerStream; import com.google.bigtable.v2.Mutation; -import com.google.bigtable.v2.ReadRowsRequest; -import com.google.bigtable.v2.Row; -import com.google.bigtable.v2.RowRange; -import com.google.bigtable.v2.RowSet; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; +import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; +import com.google.cloud.bigtable.admin.v2.models.Table; import com.google.cloud.bigtable.config.BigtableOptions; -import com.google.cloud.bigtable.config.CredentialOptions; -import com.google.cloud.bigtable.grpc.BigtableSession; -import com.google.cloud.bigtable.grpc.BigtableTableAdminClient; -import com.google.cloud.bigtable.grpc.scanner.ResultScanner; +import com.google.cloud.bigtable.data.v2.BigtableDataClient; +import com.google.cloud.bigtable.data.v2.models.Query; +import com.google.cloud.bigtable.data.v2.models.Row; import com.google.protobuf.ByteString; -import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; import java.util.Date; +import java.util.Iterator; import java.util.List; +import java.util.stream.Collectors; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.GenerateSequence; @@ -66,8 +61,9 @@ public class BigtableWriteIT implements Serializable { private static final String COLUMN_FAMILY_NAME = "cf"; private static BigtableTestOptions options; + private static BigtableHBaseVeneeringSettings veneerSettings; private BigtableOptions bigtableOptions; - private static BigtableSession session; + private static BigtableDataClient client; private static BigtableTableAdminClient tableAdminClient; private final String tableId = String.format("BigtableWriteIT-%tF-%> testData = generateTableData(numRows); - createEmptyTable(instanceName, tableId); + createEmptyTable(tableId); Pipeline p = Pipeline.create(options); p.apply(GenerateSequence.from(0).to(numRows)) @@ -129,20 +119,22 @@ public void processElement(ProcessContext c) { p.run(); // Test number of column families and column family name equality - Table table = getTable(tableName); - assertThat(table.getColumnFamiliesMap().keySet(), Matchers.hasSize(1)); - assertThat(table.getColumnFamiliesMap(), Matchers.hasKey(COLUMN_FAMILY_NAME)); + Table table = getTable(tableId); + assertThat(table.getColumnFamilies(), Matchers.hasSize(1)); + assertThat( + table.getColumnFamilies().stream().map((c) -> c.getId()).collect(Collectors.toList()), + Matchers.contains(COLUMN_FAMILY_NAME)); // Test table data equality - List> tableData = getTableData(tableName); + List> tableData = getTableData(tableId); assertThat(tableData, Matchers.containsInAnyOrder(testData.toArray())); } @After public void tearDown() throws Exception { - final String tableName = bigtableOptions.getInstanceName().toTableNameStr(tableId); - deleteTable(tableName); - session.close(); + deleteTable(tableId); + tableAdminClient.close(); + client.close(); } //////////////////////////////////////////////////////////////////////////////////////////// @@ -159,54 +151,35 @@ private List> generateTableData(int numRows) { } /** Helper function to create an empty table. */ - private void createEmptyTable(String instanceName, String tableId) { - Table.Builder tableBuilder = Table.newBuilder(); - tableBuilder.putColumnFamilies(COLUMN_FAMILY_NAME, ColumnFamily.newBuilder().build()); - - CreateTableRequest.Builder createTableRequestBuilder = - CreateTableRequest.newBuilder() - .setParent(instanceName) - .setTableId(tableId) - .setTable(tableBuilder.build()); - tableAdminClient.createTable(createTableRequestBuilder.build()); + private void createEmptyTable(String tableId) { + tableAdminClient.createTable(CreateTableRequest.of(tableId).addFamily(COLUMN_FAMILY_NAME)); } /** Helper function to get a table. */ - private Table getTable(String tableName) { - GetTableRequest.Builder getTableRequestBuilder = - GetTableRequest.newBuilder().setName(tableName); - return tableAdminClient.getTable(getTableRequestBuilder.build()); + private Table getTable(String tableId) { + return tableAdminClient.getTable(tableId); } /** Helper function to get a table's data. */ - private List> getTableData(String tableName) throws IOException { - // Add empty range to avoid TARGET_NOT_SET error - RowRange range = - RowRange.newBuilder() - .setStartKeyClosed(ByteString.EMPTY) - .setEndKeyOpen(ByteString.EMPTY) - .build(); - RowSet rowSet = RowSet.newBuilder().addRowRanges(range).build(); - ReadRowsRequest.Builder readRowsRequestBuilder = - ReadRowsRequest.newBuilder().setTableName(tableName).setRows(rowSet); - ResultScanner scanner = session.getDataClient().readRows(readRowsRequestBuilder.build()); + private List> getTableData(String tableId) { + ServerStream rows = client.readRows(Query.create(tableId)); + + Iterator iterator = rows.iterator(); Row currentRow; List> tableData = new ArrayList<>(); - while ((currentRow = scanner.next()) != null) { + while (iterator.hasNext()) { + currentRow = iterator.next(); ByteString key = currentRow.getKey(); - ByteString value = currentRow.getFamilies(0).getColumns(0).getCells(0).getValue(); + ByteString value = currentRow.getCells(COLUMN_FAMILY_NAME).get(0).getValue(); tableData.add(KV.of(key, value)); } - scanner.close(); return tableData; } /** Helper function to delete a table. */ - private void deleteTable(String tableName) { - DeleteTableRequest.Builder deleteTableRequestBuilder = - DeleteTableRequest.newBuilder().setName(tableName); - tableAdminClient.deleteTable(deleteTableRequestBuilder.build()); + private void deleteTable(String tableId) { + tableAdminClient.deleteTable(tableId); } } diff --git a/settings.gradle.kts b/settings.gradle.kts index 8527d17d3cba..76fc8a0ae651 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -262,3 +262,4 @@ include("beam-test-jenkins") project(":beam-test-jenkins").projectDir = file(".test-infra/jenkins") include("beam-validate-runner") project(":beam-validate-runner").projectDir = file(".test-infra/validate-runner") +include("com.google.api.gax.batching") From 5b27c9071122aa8facc69476d525d773d08f8c89 Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Mon, 12 Dec 2022 16:41:13 -0500 Subject: [PATCH 02/23] address comments --- .../io/gcp/bigtable/BigtableServiceImpl.java | 80 ++++++++----------- .../sdk/io/gcp/bigtable/BigtableIOTest.java | 4 +- .../gcp/bigtable/BigtableServiceImplTest.java | 10 +-- 3 files changed, 41 insertions(+), 53 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java index d843026e67b1..ffef972b73d5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java @@ -35,6 +35,7 @@ import com.google.bigtable.v2.RowRange; import com.google.bigtable.v2.RowSet; import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminSettings; import com.google.cloud.bigtable.config.BigtableOptions; import com.google.cloud.bigtable.data.v2.BigtableDataClient; import com.google.cloud.bigtable.data.v2.BigtableDataSettings; @@ -103,14 +104,23 @@ class BigtableServiceImpl implements BigtableService { public BigtableServiceImpl(BigtableOptions options) { try { this.options = options; - this.veneeringSettings = BigtableHBaseVeneeringSettings.create(options); + BigtableHBaseVeneeringSettings veneeringSettings = + BigtableHBaseVeneeringSettings.create(options); + this.dataSettings = veneeringSettings.getDataSettings(); + this.tableAdminSettings = veneeringSettings.getTableAdminSettings(); + this.operationTimeouts = veneeringSettings.getOperationTimeouts(); } catch (IOException e) { throw new RuntimeException(e); } } private final BigtableOptions options; - private final BigtableHBaseVeneeringSettings veneeringSettings; + + private final BigtableDataSettings dataSettings; + + private final BigtableTableAdminSettings tableAdminSettings; + + private final BigtableHBaseVeneeringSettings.BigtableIOOperationTimeouts operationTimeouts; @Override public BigtableOptions getBigtableOptions() { @@ -119,17 +129,14 @@ public BigtableOptions getBigtableOptions() { @Override public BigtableWriterImpl openForWriting(String tableId) throws IOException { - LOG.info("Opening for writing with settings " + veneeringSettings.getDataSettings().toString()); - return new BigtableWriterImpl( - BigtableDataClient.create(veneeringSettings.getDataSettings()), - veneeringSettings.getDataSettings(), - tableId); + LOG.info("Opening for writing with settings " + dataSettings.toString()); + return new BigtableWriterImpl(BigtableDataClient.create(dataSettings), dataSettings, tableId); } @Override public boolean tableExists(String tableId) throws IOException { try (BigtableTableAdminClient adminClient = - BigtableTableAdminClient.create(veneeringSettings.getTableAdminSettings())) { + BigtableTableAdminClient.create(tableAdminSettings)) { adminClient.getTable(tableId); return true; } catch (StatusRuntimeException e) { @@ -207,7 +214,7 @@ public boolean advance() throws IOException { @Override public void close() throws IOException { - // Goal: by the end of this function, both results and session are null and closed, + // Goal: by the end of this function, both results and client are null and closed, // independent of what errors they throw or prior state. if (client == null) { @@ -355,10 +362,10 @@ private Future fetchNextSegment() { .readRowsCallable(new BeamRowAdapter()) .call( Query.fromProto(nextRequest), - new ResponseObserver() { + new ResponseObserver() { private StreamController controller; - List rows = new ArrayList<>(); + List rows = new ArrayList<>(); long currentByteSize = 0; boolean byteLimitReached = false; @@ -369,7 +376,7 @@ public void onStart(StreamController controller) { } @Override - public void onResponse(com.google.bigtable.v2.Row response) { + public void onResponse(Row response) { // calculate size of the response currentByteSize += response.getSerializedSize(); rows.add(response); @@ -543,20 +550,12 @@ public CompletionStage writeRecord(KV() { @Override public void onSuccess(MutateRowResponse mutateRowResponse) { - // TODO throttling logic to update dataflow counter - // long throttledTime = ((ApiCallContext) - // context).getOption(Batcher.THROTTLED_TIME_KEY); - result.complete(mutateRowResponse); serviceCallMetric.call("ok"); } @Override public void onFailure(Throwable throwable) { - // TODO throttling logic to update dataflow counter - // long throttledTime = ((ApiCallContext) - // context).getOption(Batcher.THROTTLED_TIME_KEY); - if (throwable instanceof StatusRuntimeException) { serviceCallMetric.call( ((StatusRuntimeException) throwable).getStatus().getCode().value()); @@ -578,21 +577,12 @@ public String toString() { @Override public Reader createReader(BigtableSource source) throws IOException { - LOG.info( - "Creating a Reader for Bigtable with settings: " + veneeringSettings.getDataSettings()); - BigtableDataClient client = BigtableDataClient.create(veneeringSettings.getDataSettings()); + LOG.info("Creating a Reader for Bigtable with settings: " + dataSettings); + BigtableDataClient client = BigtableDataClient.create(dataSettings); if (source.getMaxBufferElementCount() != null) { - return BigtableSegmentReaderImpl.create( - client, - veneeringSettings.getDataSettings(), - source, - veneeringSettings.getOperationTimeouts()); + return BigtableSegmentReaderImpl.create(client, dataSettings, source, operationTimeouts); } else { - return new BigtableReaderImpl( - client, - veneeringSettings.getDataSettings(), - source, - veneeringSettings.getOperationTimeouts()); + return new BigtableReaderImpl(client, dataSettings, source, operationTimeouts); } } @@ -600,7 +590,7 @@ public Reader createReader(BigtableSource source) throws IOException { // - disabling timeouts - when timeouts are disabled, bigtable-hbase ignores user configured // timeouts and forces 6 minute deadlines per attempt for all RPCs except scans. This is // implemented by an interceptor. However the interceptor must be informed that this is a scan - // - per attempt deadlines - vener doesn't implement deadlines for attempts. To workaround this, + // - per attempt deadlines - veneer doesn't implement deadlines for attempts. To workaround this, // the timeouts are set per call in the ApiCallContext. However this creates a separate issue of // over running the operation deadline, so gRPC deadline is also set. private static GrpcCallContext createScanCallContext( @@ -615,15 +605,16 @@ private static GrpcCallContext createScanCallContext( true)); } else { if (operationTimeouts.getBulkReadRowsTimeouts().getOperationTimeout().isPresent()) { - ctx.withCallOptions( - CallOptions.DEFAULT.withDeadline( - Deadline.after( - operationTimeouts - .getBulkMutateTimeouts() - .getOperationTimeout() - .get() - .toMillis(), - TimeUnit.MILLISECONDS))); + ctx = + ctx.withCallOptions( + CallOptions.DEFAULT.withDeadline( + Deadline.after( + operationTimeouts + .getBulkMutateTimeouts() + .getOperationTimeout() + .get() + .toMillis(), + TimeUnit.MILLISECONDS))); } if (operationTimeouts.getBulkReadRowsTimeouts().getAttemptTimeout().isPresent()) { ctx = @@ -635,8 +626,7 @@ private static GrpcCallContext createScanCallContext( @Override public List getSampleRowKeys(BigtableSource source) throws IOException { - try (BigtableDataClient client = - BigtableDataClient.create(veneeringSettings.getDataSettings())) { + try (BigtableDataClient client = BigtableDataClient.create(dataSettings)) { return client.sampleRowKeys(source.getTableId().get()); } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java index 6a9fb5392f97..027268a1bdb0 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java @@ -1659,7 +1659,6 @@ void setupSampleRowKeys(String tableId, int numSamples, long bytesPerRow) { ImmutableList.Builder ret = ImmutableList.builder(); SortedMap rows = getTable(tableId); - System.out.println("num of rows = " + rows.size()); int currentSample = 1; int rowsSoFar = 0; for (Map.Entry entry : rows.entrySet()) { @@ -1672,7 +1671,6 @@ void setupSampleRowKeys(String tableId, int numSamples, long bytesPerRow) { ++rowsSoFar; } - System.out.println("ret size=" + ret.build().size()); // Add the last sample indicating the end of the table, with all rows before it ret.add(KeyOffset.create(ByteString.EMPTY, rows.size() * bytesPerRow)); @@ -1780,7 +1778,7 @@ private boolean rangesContainsKey(List ranges, ByteKey key) { } @Override - public com.google.bigtable.v2.Row getCurrentRow() { + public Row getCurrentRow() { if (currentRow == null) { throw new NoSuchElementException(); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java index 5d15eab214fc..a7e5d24a90a2 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java @@ -118,7 +118,7 @@ public class BigtableServiceImplTest { @Mock private ServiceCallMetric mockCallMetric; - @Captor private ArgumentCaptor requestCaptor; + @Captor private ArgumentCaptor queryCaptor; private static AtomicBoolean cancelled = new AtomicBoolean(false); @@ -550,17 +550,17 @@ public void testReadFillBuffer() throws IOException { Assert.assertTrue(underTest.start()); verify(callable, times(1)) - .call(requestCaptor.capture(), any(ResponseObserver.class), any(ApiCallContext.class)); + .call(queryCaptor.capture(), any(ResponseObserver.class), any(ApiCallContext.class)); Assert.assertEquals( - 3, requestCaptor.getValue().toProto(requestContext).getRows().getRowRangesCount()); + 3, queryCaptor.getValue().toProto(requestContext).getRows().getRowRangesCount()); do { actualResults.add(underTest.getCurrentRow()); } while (underTest.advance()); verify(callable, times(3)) - .call(requestCaptor.capture(), any(ResponseObserver.class), any(ApiCallContext.class)); + .call(queryCaptor.capture(), any(ResponseObserver.class), any(ApiCallContext.class)); Assert.assertEquals( - 1, requestCaptor.getValue().toProto(requestContext).getRows().getRowRangesCount()); + 1, queryCaptor.getValue().toProto(requestContext).getRows().getRowRangesCount()); Assert.assertEquals( expectedResults.stream().flatMap(Collection::stream).collect(Collectors.toList()), From bb579f683f48b92c82f79f9fe2b24fee67024e99 Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Thu, 2 Feb 2023 21:20:41 -0500 Subject: [PATCH 03/23] refactor setting configurations for veneer --- .../gcp/bigtable/BigtableClientWrapper.java | 26 +- .../sdk/io/gcp/bigtable/BigtableConfig.java | 166 ++-- .../BigtableConfigToVeneerSettings.java | 183 +++++ .../BigtableHBaseVeneeringSettings.java | 758 ------------------ .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 349 +++++++- .../io/gcp/bigtable/BigtableReadOptions.java | 50 +- .../sdk/io/gcp/bigtable/BigtableService.java | 15 +- .../io/gcp/bigtable/BigtableServiceImpl.java | 338 +++++--- .../io/gcp/bigtable/BigtableWriteOptions.java | 127 +++ .../io/gcp/bigtable/BigtableConfigTest.java | 83 +- .../sdk/io/gcp/bigtable/BigtableIOTest.java | 159 ++-- .../gcp/bigtable/BigtableServiceImplTest.java | 123 ++- .../sdk/io/gcp/bigtable/BigtableWriteIT.java | 18 +- 13 files changed, 1275 insertions(+), 1120 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigToVeneerSettings.java delete mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableHBaseVeneeringSettings.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteOptions.java diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableClientWrapper.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableClientWrapper.java index 85a8498f2ecc..ecdac695dbe2 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableClientWrapper.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableClientWrapper.java @@ -23,13 +23,12 @@ import com.google.auth.Credentials; import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; -import com.google.cloud.bigtable.config.BigtableOptions; -import com.google.cloud.bigtable.config.CredentialOptions; import com.google.cloud.bigtable.data.v2.BigtableDataClient; import com.google.cloud.bigtable.data.v2.models.RowMutation; import java.io.IOException; import java.io.Serializable; import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.options.ValueProvider; import org.checkerframework.checker.nullness.qual.Nullable; @Internal @@ -37,7 +36,7 @@ public class BigtableClientWrapper implements Serializable { private final BigtableTableAdminClient tableAdminClient; private final BigtableDataClient dataClient; - private final BigtableOptions bigtableOptions; + private final BigtableConfig bigtableConfig; public BigtableClientWrapper( String project, @@ -45,21 +44,20 @@ public BigtableClientWrapper( @Nullable Integer emulatorPort, @Nullable Credentials gcpCredentials) throws IOException { - BigtableOptions.Builder optionsBuilder = - BigtableOptions.builder() - .setProjectId(project) - .setInstanceId(instanceId) - .setUserAgent("apache-beam-test"); + BigtableConfig.Builder configBuilder = + BigtableConfig.builder() + .setProjectId(ValueProvider.StaticValueProvider.of(project)) + .setInstanceId(ValueProvider.StaticValueProvider.of(instanceId)) + .setUserAgent("apache-beam-test") + .setValidate(true); if (emulatorPort != null) { - optionsBuilder.enableEmulator("localhost", emulatorPort); + configBuilder.setEmulatorHost("localhost:" + emulatorPort); } if (gcpCredentials != null) { - optionsBuilder.setCredentialOptions(CredentialOptions.credential(gcpCredentials)); + configBuilder.setCredentials(gcpCredentials); } - bigtableOptions = optionsBuilder.build(); - - BigtableHBaseVeneeringSettings settings = - BigtableHBaseVeneeringSettings.create(bigtableOptions); + bigtableConfig = configBuilder.build(); + BigtableConfigToVeneerSettings settings = BigtableConfigToVeneerSettings.create(bigtableConfig); tableAdminClient = BigtableTableAdminClient.create(settings.getTableAdminSettings()); dataClient = BigtableDataClient.create(settings.getDataSettings()); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java index 7aebad4679bc..63b5ec4c060d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java @@ -19,10 +19,19 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; +import com.google.api.gax.core.NoCredentialsProvider; +import com.google.auth.Credentials; +import com.google.auth.oauth2.GoogleCredentials; +import com.google.auth.oauth2.ServiceAccountJwtAccessCredentials; import com.google.auto.value.AutoValue; import com.google.cloud.bigtable.config.BigtableOptions; import com.google.cloud.bigtable.config.CredentialOptions; +import java.io.FileInputStream; +import java.io.IOException; import java.io.Serializable; +import java.security.GeneralSecurityException; +import java.security.KeyStore; +import java.security.PrivateKey; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.ValueProvider; @@ -39,15 +48,20 @@ }) abstract class BigtableConfig implements Serializable { + enum CredentialType { + DEFAULT, + P12, + SUPPLIED, + JSON, + NONE + } + /** Returns the project id being written to. */ abstract @Nullable ValueProvider getProjectId(); /** Returns the instance id being written to. */ abstract @Nullable ValueProvider getInstanceId(); - /** Returns the table being read from. */ - abstract @Nullable ValueProvider getTableId(); - /** Returns the app profile id of this workload. */ abstract @Nullable ValueProvider getAppProfileId(); @@ -72,6 +86,12 @@ abstract class BigtableConfig implements Serializable { /** Bigtable emulator. Used only for testing. */ abstract @Nullable String getEmulatorHost(); + /** User agent for this job. */ + abstract @Nullable String getUserAgent(); + + /** Credentials for running the job. */ + abstract @Nullable Credentials getCredentials(); + abstract Builder toBuilder(); static BigtableConfig.Builder builder() { @@ -85,8 +105,6 @@ abstract static class Builder { abstract Builder setInstanceId(ValueProvider instanceId); - abstract Builder setTableId(ValueProvider tableId); - abstract Builder setAppProfileId(ValueProvider appProfileId); /** @deprecated will be replaced by bigtable options configurator. */ @@ -102,6 +120,10 @@ abstract Builder setBigtableOptionsConfigurator( abstract Builder setEmulatorHost(String emulatorHost); + abstract Builder setUserAgent(String userAgent); + + abstract Builder setCredentials(Credentials credentials); + abstract BigtableConfig build(); } @@ -115,11 +137,6 @@ BigtableConfig withInstanceId(ValueProvider instanceId) { return toBuilder().setInstanceId(instanceId).build(); } - BigtableConfig withTableId(ValueProvider tableId) { - checkArgument(tableId != null, "tableId can not be null"); - return toBuilder().setTableId(tableId).build(); - } - BigtableConfig withAppProfileId(@Nullable ValueProvider appProfileId) { checkArgument(appProfileId != null, "App profile id can not be null"); return toBuilder().setAppProfileId(appProfileId).build(); @@ -154,11 +171,11 @@ BigtableConfig withEmulator(String emulatorHost) { return toBuilder().setEmulatorHost(emulatorHost).build(); } - void validate() { - checkArgument( - getTableId() != null && (!getTableId().isAccessible() || !getTableId().get().isEmpty()), - "Could not obtain Bigtable table id"); + BigtableConfig withCredentails(Credentials credentials) { + return toBuilder().setCredentials(credentials).build(); + } + void validate() { checkArgument( (getProjectId() != null && (!getProjectId().isAccessible() || !getProjectId().get().isEmpty())) @@ -182,7 +199,6 @@ void populateDisplayData(DisplayData.Builder builder) { DisplayData.item("projectId", getProjectId()).withLabel("Bigtable Project Id")) .addIfNotNull( DisplayData.item("instanceId", getInstanceId()).withLabel("Bigtable Instance Id")) - .addIfNotNull(DisplayData.item("tableId", getTableId()).withLabel("Bigtable Table Id")) .add(DisplayData.item("withValidation", getValidate()).withLabel("Check is table exists")); if (getBigtableOptions() != null) { @@ -206,52 +222,113 @@ BigtableService getBigtableService(PipelineOptions pipelineOptions) { return getBigtableService(); } - BigtableOptions.Builder bigtableOptions = effectiveUserProvidedBigtableOptions(); + BigtableConfig.Builder config = toBuilder(); - bigtableOptions.setUserAgent(pipelineOptions.getUserAgent()); + if (pipelineOptions instanceof GcpOptions) { + config.setCredentials(((GcpOptions) pipelineOptions).getGcpCredential()); + } - if (bigtableOptions.build().getCredentialOptions().getCredentialType() - == CredentialOptions.CredentialType.DefaultCredentials) { - bigtableOptions.setCredentialOptions( - CredentialOptions.credential(pipelineOptions.as(GcpOptions.class).getGcpCredential())); + try { + translateBigtableOptions(config); + } catch (IOException e) { + throw new RuntimeException(e); } - return new BigtableServiceImpl(bigtableOptions.build()); + config.setUserAgent(pipelineOptions.getUserAgent()); + + return new BigtableServiceImpl(config.build()); } boolean isDataAccessible() { - return getTableId().isAccessible() - && (getProjectId() == null || getProjectId().isAccessible()) + return (getProjectId() == null || getProjectId().isAccessible()) && (getInstanceId() == null || getInstanceId().isAccessible()); } - private BigtableOptions.Builder effectiveUserProvidedBigtableOptions() { - BigtableOptions.Builder effectiveOptions = - getBigtableOptions() != null - ? getBigtableOptions().toBuilder() - : new BigtableOptions.Builder(); + private void translateBigtableOptions(BigtableConfig.Builder builder) throws IOException { + BigtableOptions.Builder effectiveOptionsBuilder = null; + + if (getBigtableOptions() != null) { + effectiveOptionsBuilder = getBigtableOptions().toBuilder(); + } if (getBigtableOptionsConfigurator() != null) { - effectiveOptions = getBigtableOptionsConfigurator().apply(effectiveOptions); + effectiveOptionsBuilder = getBigtableOptionsConfigurator().apply(BigtableOptions.builder()); + } + + if (effectiveOptionsBuilder == null) { + return; } - // Default option that should be forced in most cases - effectiveOptions.setUseCachedDataPool(true); + BigtableOptions effectiveOptions = effectiveOptionsBuilder.build(); - if (getInstanceId() != null) { - effectiveOptions.setInstanceId(getInstanceId().get()); + // Todo decided if we should implement cached channel pool + + if (effectiveOptions.getInstanceId() != null && getInstanceId() == null) { + builder.setInstanceId(ValueProvider.StaticValueProvider.of(effectiveOptions.getInstanceId())); } - if (getProjectId() != null) { - effectiveOptions.setProjectId(getProjectId().get()); + if (effectiveOptions.getProjectId() != null && getProjectId() == null) { + builder.setProjectId(ValueProvider.StaticValueProvider.of(effectiveOptions.getProjectId())); } - if (getEmulatorHost() != null) { - effectiveOptions.enableEmulator(getEmulatorHost()); - effectiveOptions.setUseCachedDataPool(false); + if (!effectiveOptions.getDataHost().equals("bigtable.googleapis.com") + && getEmulatorHost() == null) { + builder.setEmulatorHost( + String.format("%s:%s", effectiveOptions.getDataHost(), effectiveOptions.getPort())); } - return effectiveOptions; + if (effectiveOptions.getCredentialOptions() != null) { + CredentialOptions credOptions = effectiveOptions.getCredentialOptions(); + switch (credOptions.getCredentialType()) { + case DefaultCredentials: + GoogleCredentials credentials = GoogleCredentials.getApplicationDefault(); + builder.setCredentials(credentials); + break; + case P12: + String keyFile = ((CredentialOptions.P12CredentialOptions) credOptions).getKeyFile(); + String serviceAccount = + ((CredentialOptions.P12CredentialOptions) credOptions).getServiceAccount(); + try { + KeyStore keyStore = KeyStore.getInstance("PKCS12"); + + try (FileInputStream fin = new FileInputStream(keyFile)) { + keyStore.load(fin, "notasecret".toCharArray()); + } + PrivateKey privateKey = + (PrivateKey) keyStore.getKey("privatekey", "notasecret".toCharArray()); + + if (privateKey == null) { + throw new IllegalStateException("private key cannot be null"); + } + builder.setCredentials( + ServiceAccountJwtAccessCredentials.newBuilder() + .setClientEmail(serviceAccount) + .setPrivateKey(privateKey) + .build()); + } catch (GeneralSecurityException exception) { + throw new RuntimeException("exception while retrieving credentials", exception); + } + break; + case SuppliedCredentials: + builder.setCredentials( + ((CredentialOptions.UserSuppliedCredentialOptions) credOptions).getCredential()); + break; + case SuppliedJson: + CredentialOptions.JsonCredentialsOptions jsonCredentialsOptions = + (CredentialOptions.JsonCredentialsOptions) credOptions; + synchronized (jsonCredentialsOptions) { + if (jsonCredentialsOptions.getCachedCredentials() == null) { + jsonCredentialsOptions.setCachedCredentails( + GoogleCredentials.fromStream(jsonCredentialsOptions.getInputStream())); + } + builder.setCredentials(jsonCredentialsOptions.getCachedCredentials()); + } + break; + case None: + builder.setCredentials(NoCredentialsProvider.create().getCredentials()); + break; + } + } } @Override @@ -259,14 +336,9 @@ public final String toString() { return MoreObjects.toStringHelper(BigtableConfig.class) .add("projectId", getProjectId()) .add("instanceId", getInstanceId()) - .add("tableId", getTableId()) .add("appProfileId", getAppProfileId()) - .add( - "bigtableOptionsConfigurator", - getBigtableOptionsConfigurator() == null - ? null - : getBigtableOptionsConfigurator().getClass().getName()) - .add("options", getBigtableOptions()) + .add("userAgent", getUserAgent()) + .add("emulator", getEmulatorHost()) .toString(); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigToVeneerSettings.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigToVeneerSettings.java new file mode 100644 index 000000000000..ed82c08c21ee --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigToVeneerSettings.java @@ -0,0 +1,183 @@ +/* + * 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.bigtable; + +import com.google.api.core.ApiFunction; +import com.google.api.gax.core.FixedCredentialsProvider; +import com.google.api.gax.grpc.ChannelPoolSettings; +import com.google.api.gax.grpc.InstantiatingGrpcChannelProvider; +import com.google.api.gax.rpc.FixedHeaderProvider; +import com.google.api.gax.rpc.StubSettings; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminSettings; +import com.google.cloud.bigtable.config.BigtableOptions; +import com.google.cloud.bigtable.data.v2.BigtableDataSettings; +import io.grpc.ManagedChannelBuilder; +import io.grpc.internal.GrpcUtil; +import java.io.IOException; +import java.util.Objects; +import javax.annotation.Nonnull; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; +import org.checkerframework.checker.initialization.qual.UnderInitialization; + +/** Helper class to translate {@link BigtableOptions} to Bigtable Veneer settings. */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +class BigtableConfigToVeneerSettings { + private static final String DEFAULT_DATA_ENDPOINT = "bigtable.googleapis.com:443"; + private static final String DEFAULT_ADMIN_ENDPOINT = "bigtableadmin.googleapis.com:443"; + + private final BigtableDataSettings dataSettings; + private final BigtableTableAdminSettings tableAdminSettings; + + static BigtableConfigToVeneerSettings create(@Nonnull BigtableConfig config) throws IOException { + return new BigtableConfigToVeneerSettings(config); + } + + private BigtableConfigToVeneerSettings(@Nonnull BigtableConfig config) throws IOException { + if (config.getProjectId() == null || config.getInstanceId() == null) { + throw new IOException("can't find project or instance id"); + } + + // Build configs for veneer + this.dataSettings = buildBigtableDataSettings(config); + this.tableAdminSettings = buildBigtableTableAdminSettings(config); + } + + // ************** Getters ************** + /** Utility to convert {@link BigtableOptions} to {@link BigtableDataSettings}. */ + BigtableDataSettings getDataSettings() { + return dataSettings; + } + + /** Utility to convert {@link BigtableOptions} to {@link BigtableTableAdminSettings}. */ + BigtableTableAdminSettings getTableAdminSettings() { + return tableAdminSettings; + } + + // ************** Private Helpers ************** + private BigtableDataSettings buildBigtableDataSettings( + @UnderInitialization BigtableConfigToVeneerSettings this, BigtableConfig config) + throws IOException { + BigtableDataSettings.Builder dataBuilder; + + // Configure the Data connection + dataBuilder = BigtableDataSettings.newBuilder(); + if (config.getEmulatorHost() != null) { + configureConnection( + dataBuilder.stubSettings(), config, Objects.requireNonNull(config.getEmulatorHost())); + } else { + configureConnection(dataBuilder.stubSettings(), config, DEFAULT_DATA_ENDPOINT); + } + configureCredentialProvider(dataBuilder.stubSettings(), config); + configureHeaderProvider(dataBuilder.stubSettings(), config); + + // Configure the target + dataBuilder + .setProjectId(Objects.requireNonNull(config.getProjectId().get())) + .setInstanceId(Objects.requireNonNull(config.getInstanceId().get())); + if (config.getAppProfileId() != null) { + dataBuilder.setAppProfileId(Objects.requireNonNull(config.getAppProfileId().get())); + } + + return dataBuilder.build(); + } + + private BigtableTableAdminSettings buildBigtableTableAdminSettings( + @UnderInitialization BigtableConfigToVeneerSettings this, BigtableConfig config) + throws IOException { + BigtableTableAdminSettings.Builder adminBuilder; + + // Configure connection + adminBuilder = BigtableTableAdminSettings.newBuilder(); + if (config.getEmulatorHost() != null) { + configureConnection( + adminBuilder.stubSettings(), config, Objects.requireNonNull(config.getEmulatorHost())); + } else { + configureConnection(adminBuilder.stubSettings(), config, DEFAULT_ADMIN_ENDPOINT); + } + + configureCredentialProvider(adminBuilder.stubSettings(), config); + + adminBuilder + .setProjectId(Objects.requireNonNull(config.getProjectId().get())) + .setInstanceId(Objects.requireNonNull(config.getInstanceId().get())); + + return adminBuilder.build(); + } + + @SuppressWarnings("rawtypes") + private void configureConnection( + @UnderInitialization BigtableConfigToVeneerSettings this, + StubSettings.Builder stubSettings, + BigtableConfig config, + String endpoint) { + stubSettings.setEndpoint(endpoint); + + final InstantiatingGrpcChannelProvider.Builder channelProvider = + ((InstantiatingGrpcChannelProvider) stubSettings.getTransportChannelProvider()).toBuilder(); + + if (config.getEmulatorHost() != null) { + @SuppressWarnings("rawtypes") + final ApiFunction prevConfigurator = + channelProvider.getChannelConfigurator(); + //noinspection rawtypes + channelProvider.setChannelConfigurator( + new ApiFunction() { + @Override + public ManagedChannelBuilder apply(ManagedChannelBuilder channelBuilder) { + if (prevConfigurator != null) { + channelBuilder = prevConfigurator.apply(channelBuilder); + } + return channelBuilder.usePlaintext(); + } + }); + } + + channelProvider.setChannelPoolSettings( + ChannelPoolSettings.builder() + .setMaxRpcsPerChannel(50) + .setMinRpcsPerChannel(1) + .setPreemptiveRefreshEnabled(true) + .setInitialChannelCount(1) + .setMinChannelCount(1) + .setMaxChannelCount(4) + .build()); + stubSettings.setTransportChannelProvider(channelProvider.build()); + } + + private void configureHeaderProvider( + @UnderInitialization BigtableConfigToVeneerSettings this, + StubSettings.Builder stubSettings, + BigtableConfig config) { + + ImmutableMap.Builder headersBuilder = ImmutableMap.builder(); + headersBuilder.putAll(stubSettings.getHeaderProvider().getHeaders()); + headersBuilder.put( + GrpcUtil.USER_AGENT_KEY.name(), Objects.requireNonNull(config.getUserAgent())); + + stubSettings.setHeaderProvider(FixedHeaderProvider.create(headersBuilder.build())); + } + + private void configureCredentialProvider( + @UnderInitialization BigtableConfigToVeneerSettings this, + StubSettings.Builder stubSettings, + BigtableConfig config) { + stubSettings.setCredentialsProvider(FixedCredentialsProvider.create(config.getCredentials())); + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableHBaseVeneeringSettings.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableHBaseVeneeringSettings.java deleted file mode 100644 index 8f0063d1f5fb..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableHBaseVeneeringSettings.java +++ /dev/null @@ -1,758 +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.sdk.io.gcp.bigtable; - -import com.google.api.core.ApiFunction; -import com.google.api.gax.batching.BatchingSettings; -import com.google.api.gax.batching.FlowControlSettings; -import com.google.api.gax.core.FixedCredentialsProvider; -import com.google.api.gax.core.NoCredentialsProvider; -import com.google.api.gax.grpc.InstantiatingGrpcChannelProvider; -import com.google.api.gax.rpc.FixedHeaderProvider; -import com.google.api.gax.rpc.ServerStreamingCallSettings; -import com.google.api.gax.rpc.StatusCode; -import com.google.api.gax.rpc.StubSettings; -import com.google.api.gax.rpc.UnaryCallSettings; -import com.google.auth.Credentials; -import com.google.auth.oauth2.GoogleCredentials; -import com.google.auth.oauth2.ServiceAccountCredentials; -import com.google.auth.oauth2.ServiceAccountJwtAccessCredentials; -import com.google.cloud.bigtable.Version; -import com.google.cloud.bigtable.admin.v2.BigtableInstanceAdminSettings; -import com.google.cloud.bigtable.admin.v2.BigtableTableAdminSettings; -import com.google.cloud.bigtable.config.BigtableOptions; -import com.google.cloud.bigtable.config.CredentialOptions; -import com.google.cloud.bigtable.data.v2.BigtableDataSettings; -import com.google.cloud.bigtable.data.v2.models.Query; -import com.google.cloud.bigtable.data.v2.models.Row; -import com.google.cloud.bigtable.data.v2.stub.BigtableBatchingCallSettings; -import com.google.cloud.bigtable.data.v2.stub.BigtableBulkReadRowsCallSettings; -import io.grpc.CallOptions; -import io.grpc.Channel; -import io.grpc.ClientCall; -import io.grpc.ClientInterceptor; -import io.grpc.Deadline; -import io.grpc.ManagedChannelBuilder; -import io.grpc.MethodDescriptor; -import io.grpc.Status; -import io.grpc.internal.GrpcUtil; -import java.io.FileInputStream; -import java.io.IOException; -import java.security.GeneralSecurityException; -import java.security.KeyStore; -import java.security.PrivateKey; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.concurrent.TimeUnit; -import javax.annotation.Nonnull; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Optional; -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.Lists; -import org.checkerframework.checker.initialization.qual.UnderInitialization; -import org.threeten.bp.Duration; - -/** Helper class to translate {@link BigtableOptions} to Bigtable Veneer settings. */ -class BigtableHBaseVeneeringSettings { - private static final String DEFAULT_BIGTABLE_BATCH_DATA_ENDPOINT = - "batch-bigtable.googleapis.com:443"; - private static final Duration DEFAULT_UNARY_ATTEMPT_TIMEOUTS = Duration.ofSeconds(20); - private static final Duration DEFAULT_BULK_MUTATE_ATTEMPT_TIMEOUTS = Duration.ofMinutes(6); - - private final BigtableDataSettings dataSettings; - private final BigtableTableAdminSettings tableAdminSettings; - private final BigtableInstanceAdminSettings instanceAdminSettings; - - private final BigtableIOOperationTimeouts clientTimeouts; - - static BigtableHBaseVeneeringSettings create(@Nonnull BigtableOptions options) - throws IOException { - return new BigtableHBaseVeneeringSettings(options); - } - - private BigtableHBaseVeneeringSettings(@Nonnull BigtableOptions options) throws IOException { - // Build configs for veneer - this.clientTimeouts = buildCallSettings(options); - - this.dataSettings = buildBigtableDataSettings(clientTimeouts, options); - this.tableAdminSettings = buildBigtableTableAdminSettings(options); - this.instanceAdminSettings = buildBigtableInstanceAdminSettings(options); - } - - // ************** Getters ************** - /** Utility to convert {@link BigtableOptions} to {@link BigtableDataSettings}. */ - BigtableDataSettings getDataSettings() { - return dataSettings; - } - - /** Utility to convert {@link BigtableOptions} to {@link BigtableTableAdminSettings}. */ - BigtableTableAdminSettings getTableAdminSettings() { - return tableAdminSettings; - } - - BigtableIOOperationTimeouts getOperationTimeouts() { - return clientTimeouts; - } - - /** Utility to convert {@link BigtableOptions} to {@link BigtableInstanceAdminSettings}. */ - BigtableInstanceAdminSettings getInstanceAdminSettings() { - return instanceAdminSettings; - } - - // ************** Private Helpers ************** - private BigtableDataSettings buildBigtableDataSettings( - @UnderInitialization BigtableHBaseVeneeringSettings this, - BigtableIOOperationTimeouts clientTimeouts, - BigtableOptions options) - throws IOException { - BigtableDataSettings.Builder dataBuilder; - - // Configure the Data connection - dataBuilder = BigtableDataSettings.newBuilder(); - if (options.useBatch()) { - configureConnection( - dataBuilder.stubSettings(), DEFAULT_BIGTABLE_BATCH_DATA_ENDPOINT, options); - } else { - configureConnection( - dataBuilder.stubSettings(), options.getDataHost() + ":" + options.getPort(), options); - } - configureCredentialProvider(dataBuilder.stubSettings(), options); - configureHeaderProvider(dataBuilder.stubSettings(), options); - - // Configure the target - dataBuilder.setProjectId(options.getProjectId()).setInstanceId(options.getInstanceId()); - if (options.getAppProfileId() != null) { - dataBuilder.setAppProfileId(options.getAppProfileId()); - } - - // Configure RPCs - this happens in multiple parts: - // - retry settings are configured here - // - timeouts are split into multiple places: - // - timeouts for retries are configured here - // - if USE_TIMEOUTS is explicitly disabled, then an interceptor is added to force all - // deadlines to 6 minutes - configureConnectionCallTimeouts(dataBuilder.stubSettings(), clientTimeouts); - - // Complex RPC method settings - configureBulkMutationSettings( - dataBuilder.stubSettings().bulkMutateRowsSettings(), - clientTimeouts.getBulkMutateTimeouts(), - options); - configureBulkReadRowsSettings( - dataBuilder.stubSettings().bulkReadRowsSettings(), - clientTimeouts.getBulkReadRowsTimeouts(), - options); - configureReadRowsSettings( - dataBuilder.stubSettings().readRowsSettings(), - clientTimeouts.getBulkReadRowsTimeouts(), - options); - - // RPC methods - simple - configureNonRetryableCallSettings( - dataBuilder.stubSettings().checkAndMutateRowSettings(), clientTimeouts.getUnaryTimeouts()); - configureNonRetryableCallSettings( - dataBuilder.stubSettings().readModifyWriteRowSettings(), clientTimeouts.getUnaryTimeouts()); - - configureRetryableCallSettings( - dataBuilder.stubSettings().mutateRowSettings(), clientTimeouts.getUnaryTimeouts(), options); - configureRetryableCallSettings( - dataBuilder.stubSettings().readRowSettings(), clientTimeouts.getUnaryTimeouts(), options); - configureRetryableCallSettings( - dataBuilder.stubSettings().sampleRowKeysSettings(), - clientTimeouts.getUnaryTimeouts(), - options); - - return dataBuilder.build(); - } - - private BigtableTableAdminSettings buildBigtableTableAdminSettings( - @UnderInitialization BigtableHBaseVeneeringSettings this, BigtableOptions options) - throws IOException { - BigtableTableAdminSettings.Builder adminBuilder; - - // Configure connection - adminBuilder = BigtableTableAdminSettings.newBuilder(); - configureConnection( - adminBuilder.stubSettings(), options.getAdminHost() + ":" + options.getPort(), options); - configureCredentialProvider(adminBuilder.stubSettings(), options); - - configureHeaderProvider(adminBuilder.stubSettings(), options); - - adminBuilder.setProjectId(options.getProjectId()).setInstanceId(options.getInstanceId()); - - // timeout/retry settings don't apply to admin operations - // v1 used to use RetryOptions for: - // - createTable - // - getTable - // - listTables - // - deleteTable - // - modifyColumnFamilies - // - dropRowRange - // However data latencies are very different from data latencies and end users shouldn't need to - // change the defaults - // if it turns out that the timeout & retry behavior needs to be configurable, we will expose - // separate settings - - return adminBuilder.build(); - } - - private BigtableInstanceAdminSettings buildBigtableInstanceAdminSettings( - @UnderInitialization BigtableHBaseVeneeringSettings this, BigtableOptions options) - throws IOException { - BigtableInstanceAdminSettings.Builder adminBuilder; - - // Configure connection - adminBuilder = BigtableInstanceAdminSettings.newBuilder(); - configureConnection( - adminBuilder.stubSettings(), options.getAdminHost() + ":" + options.getPort(), options); - configureCredentialProvider(adminBuilder.stubSettings(), options); - - configureHeaderProvider(adminBuilder.stubSettings(), options); - - adminBuilder.setProjectId(options.getProjectId()); - - return adminBuilder.build(); - } - - @SuppressWarnings("rawtypes") - private void configureConnection( - @UnderInitialization BigtableHBaseVeneeringSettings this, - StubSettings.Builder stubSettings, - String endpoint, - BigtableOptions options) { - final InstantiatingGrpcChannelProvider.Builder channelProvider = - ((InstantiatingGrpcChannelProvider) stubSettings.getTransportChannelProvider()).toBuilder(); - - stubSettings.setEndpoint(endpoint); - - if (options.usePlaintextNegotiation()) { - // Make sure to avoid clobbering the old Configurator - @SuppressWarnings("rawtypes") - final ApiFunction prevConfigurator = - channelProvider.getChannelConfigurator(); - //noinspection rawtypes - channelProvider.setChannelConfigurator( - new ApiFunction() { - @Override - public ManagedChannelBuilder apply(ManagedChannelBuilder channelBuilder) { - if (prevConfigurator != null) { - channelBuilder = prevConfigurator.apply(channelBuilder); - } - return channelBuilder.usePlaintext(); - } - }); - } - - channelProvider.setPoolSize(options.getChannelCount()); - - stubSettings.setTransportChannelProvider(channelProvider.build()); - } - - private void configureHeaderProvider( - @UnderInitialization BigtableHBaseVeneeringSettings this, - StubSettings.Builder stubSettings, - BigtableOptions options) { - - ImmutableMap.Builder headersBuilder = ImmutableMap.builder(); - List userAgentParts = Lists.newArrayList(); - userAgentParts.add("bigtable-" + Version.VERSION); - userAgentParts.add("jdk-" + System.getProperty("java.specification.version")); - - String customUserAgent = options.getUserAgent(); - if (customUserAgent != null) { - userAgentParts.add(customUserAgent); - } - - String userAgent = Joiner.on(",").join(userAgentParts); - headersBuilder.put(GrpcUtil.USER_AGENT_KEY.name(), userAgent); - - String tracingCookie = options.getTracingCookie(); - if (tracingCookie != null) { - headersBuilder.put("cookie", tracingCookie); - } - - stubSettings.setHeaderProvider(FixedHeaderProvider.create(headersBuilder.build())); - } - - @SuppressWarnings("rawtypes") - private void configureConnectionCallTimeouts( - @UnderInitialization BigtableHBaseVeneeringSettings this, - StubSettings.Builder stubSettings, - BigtableIOOperationTimeouts clientTimeouts) { - // Only patch settings when timeouts are disabled - if (clientTimeouts.getUseTimeouts()) { - return; - } - InstantiatingGrpcChannelProvider.Builder channelProvider = - ((InstantiatingGrpcChannelProvider) stubSettings.getTransportChannelProvider()).toBuilder(); - - final ApiFunction prevConfigurator = - channelProvider.getChannelConfigurator(); - - channelProvider.setChannelConfigurator( - new ApiFunction() { - @Override - public ManagedChannelBuilder apply(ManagedChannelBuilder managedChannelBuilder) { - if (prevConfigurator != null) { - managedChannelBuilder = prevConfigurator.apply(managedChannelBuilder); - } - return managedChannelBuilder.intercept(new NoTimeoutsInterceptor()); - } - }); - stubSettings.setTransportChannelProvider(channelProvider.build()); - } - - private void configureBulkMutationSettings( - @UnderInitialization BigtableHBaseVeneeringSettings this, - BigtableBatchingCallSettings.Builder builder, - OperationTimeouts operationTimeouts, - BigtableOptions options) { - BatchingSettings.Builder batchingSettingsBuilder = builder.getBatchingSettings().toBuilder(); - - // Start configure retries & timeouts - configureRetryableCallSettings(builder, operationTimeouts, options); - // End configure retries & timeouts - - // Start configure flush triggers - long autoFlushMs = options.getBulkOptions().getAutoflushMs(); - if (autoFlushMs <= 0) { - // setDelayThreshold(null) will cause build error. Ignore autoFlushMs - // if it's disabled and use the default value of 1 second instead until - // we fix the implementation in gax BatcherImpl to annotate with - // @Nonnull or check for 0 duration. - batchingSettingsBuilder.setDelayThreshold(Duration.ofSeconds(1)); - } else { - batchingSettingsBuilder.setDelayThreshold(Duration.ofMillis(autoFlushMs)); - } - - batchingSettingsBuilder.setElementCountThreshold( - Long.valueOf(options.getBulkOptions().getBulkMaxRowKeyCount())); - - batchingSettingsBuilder.setRequestByteThreshold( - Long.valueOf(options.getBulkOptions().getBulkMaxRequestSize())); - // End configure flush triggers - - // Start configure flow control - FlowControlSettings.Builder flowControl = - builder.getBatchingSettings().getFlowControlSettings().toBuilder(); - - // Approximate max inflight rpcs in terms of outstanding elements - int maxInflightRpcCount = options.getBulkOptions().getMaxInflightRpcs(); - - Long bulkMaxRowKeyCount = builder.getBatchingSettings().getElementCountThreshold(); - if (bulkMaxRowKeyCount == null) { - // Using Object.requireNotNull will cause build error "incompatible argument for - // parameter arg0 of requireNonNull" - throw new IllegalStateException("elementCountThreshold can't be null"); - } - Long maxInflightElements = maxInflightRpcCount * bulkMaxRowKeyCount; - flowControl.setMaxOutstandingElementCount(maxInflightElements); - - flowControl.setMaxOutstandingRequestBytes(options.getBulkOptions().getMaxMemory()); - - batchingSettingsBuilder.setFlowControlSettings(flowControl.build()); - // End configure flow control - - builder.setBatchingSettings(batchingSettingsBuilder.build()); - - if (options.getBulkOptions().isEnableBulkMutationThrottling()) { - long latencyMs = options.getBulkOptions().getBulkMutationRpcTargetMs(); - builder.enableLatencyBasedThrottling(latencyMs); - } - } - - private void configureBulkReadRowsSettings( - @UnderInitialization BigtableHBaseVeneeringSettings this, - BigtableBulkReadRowsCallSettings.Builder builder, - OperationTimeouts operationTimeouts, - BigtableOptions options) { - BatchingSettings.Builder bulkReadBatchingBuilder = builder.getBatchingSettings().toBuilder(); - - // Start configure retries & timeouts - configureRetryableCallSettings(builder, operationTimeouts, options); - // End configure retries & timeouts - - // Start config batch settings - long maxRowKeyCount = options.getBulkOptions().getBulkMaxRowKeyCount(); - bulkReadBatchingBuilder.setElementCountThreshold(maxRowKeyCount); - builder.setBatchingSettings(bulkReadBatchingBuilder.build()); - // End config batch settings - - // NOTE: autoflush, flow control settings are not currently exposed - } - - private void configureReadRowsSettings( - @UnderInitialization BigtableHBaseVeneeringSettings this, - ServerStreamingCallSettings.Builder readRowsSettings, - OperationTimeouts operationTimeouts, - BigtableOptions options) { - - // Configure retries - // NOTE: that similar but not the same as unary retry settings: per attempt timeouts don't - // exist, instead we use READ_PARTIAL_ROW_TIMEOUT_MS as the intra-row timeout - if (!options.getRetryOptions().enableRetries()) { - // user explicitly disabled retries, treat it as a non-idempotent method - readRowsSettings.setRetryableCodes(Collections.emptySet()); - } else { - // apply user retry settings - readRowsSettings.setRetryableCodes( - extractRetryCodesFromConfig(readRowsSettings.getRetryableCodes(), options)); - - // Configure backoff - long initialElapsedBackoffMs = options.getRetryOptions().getInitialBackoffMillis(); - readRowsSettings - .retrySettings() - .setInitialRetryDelay(Duration.ofMillis(initialElapsedBackoffMs)); - - if (initialElapsedBackoffMs - > readRowsSettings.retrySettings().getMaxRetryDelay().toMillis()) { - readRowsSettings - .retrySettings() - .setMaxRetryDelay(Duration.ofMillis(initialElapsedBackoffMs)); - } - - readRowsSettings - .retrySettings() - .setMaxAttempts(options.getRetryOptions().getMaxScanTimeoutRetries()); - } - - // Per response timeouts (note: gax maps rpcTimeouts to response timeouts for streaming rpcs) - if (operationTimeouts.getResponseTimeout().isPresent()) { - readRowsSettings - .retrySettings() - .setInitialRpcTimeout(operationTimeouts.getResponseTimeout().get()) - .setMaxRpcTimeout(operationTimeouts.getResponseTimeout().get()); - } - - // attempt timeout is configured in BigtableServiceImpl - - // overall timeout - if (operationTimeouts.getOperationTimeout().isPresent()) { - readRowsSettings - .retrySettings() - .setTotalTimeout(operationTimeouts.getOperationTimeout().get()); - } - } - - private void configureRetryableCallSettings( - @UnderInitialization BigtableHBaseVeneeringSettings this, - UnaryCallSettings.Builder unaryCallSettings, - OperationTimeouts operationTimeouts, - BigtableOptions options) { - - if (!options.getRetryOptions().enableRetries()) { - // user explicitly disabled retries, treat it as a non-idempotent method - configureNonRetryableCallSettings(unaryCallSettings, operationTimeouts); - return; - } - - // apply user user retry settings - unaryCallSettings.setRetryableCodes( - extractRetryCodesFromConfig(unaryCallSettings.getRetryableCodes(), options)); - - // Configure backoff - long initialBackoffMs = options.getRetryOptions().getInitialBackoffMillis(); - unaryCallSettings.retrySettings().setInitialRetryDelay(Duration.ofMillis(initialBackoffMs)); - - if (initialBackoffMs > unaryCallSettings.retrySettings().getMaxRetryDelay().toMillis()) { - unaryCallSettings.retrySettings().setMaxRetryDelay(Duration.ofMillis(initialBackoffMs)); - } - - // Configure overall timeout - if (operationTimeouts.getOperationTimeout().isPresent()) { - unaryCallSettings - .retrySettings() - .setTotalTimeout(operationTimeouts.getOperationTimeout().get()); - } - - // Configure attempt timeout - if the user hasn't explicitly configured it, then fallback to - // overall timeout to match previous behavior - Optional effectiveAttemptTimeout = - operationTimeouts.getAttemptTimeout().or(operationTimeouts.getOperationTimeout()); - if (effectiveAttemptTimeout.isPresent()) { - unaryCallSettings.retrySettings().setInitialRpcTimeout(effectiveAttemptTimeout.get()); - unaryCallSettings.retrySettings().setMaxRpcTimeout(effectiveAttemptTimeout.get()); - } - } - - private void configureNonRetryableCallSettings( - @UnderInitialization BigtableHBaseVeneeringSettings this, - UnaryCallSettings.Builder unaryCallSettings, - OperationTimeouts operationTimeouts) { - unaryCallSettings.setRetryableCodes(Collections.emptySet()); - - // NOTE: attempt timeouts are not configured for non-retriable rpcs - if (operationTimeouts.getOperationTimeout().isPresent()) { - unaryCallSettings - .retrySettings() - .setLogicalTimeout(operationTimeouts.getOperationTimeout().get()); - } - } - - private BigtableIOOperationTimeouts buildCallSettings( - @UnderInitialization BigtableHBaseVeneeringSettings this, BigtableOptions options) { - boolean useTimeouts = options.getCallOptionsConfig().isUseTimeout(); - - Optional bulkMutateOverallTimeout = - Optional.of(Duration.ofMillis(options.getCallOptionsConfig().getMutateRpcTimeoutMs())); - OperationTimeouts bulkMutateTimeouts = - new OperationTimeouts( - Optional.absent(), - extracBulkMutateAttemptTimeout(options), - bulkMutateOverallTimeout); - - Optional bulkReadPartialRowTimeout = - Optional.of(Duration.ofMillis(options.getRetryOptions().getReadPartialRowTimeoutMillis())); - Optional bulkReadRowsOverallTimeout = - Optional.of(Duration.ofMillis(options.getCallOptionsConfig().getReadStreamRpcTimeoutMs())); - OperationTimeouts bulkReadTimeouts = - new OperationTimeouts( - bulkReadPartialRowTimeout, - extractBulkReadRowsAttemptTimeout(options), - bulkReadRowsOverallTimeout); - - OperationTimeouts unaryTimeouts = - new OperationTimeouts( - Optional.absent(), - extractUnaryAttemptTimeout(options), - Optional.of(Duration.ofMillis(options.getCallOptionsConfig().getShortRpcTimeoutMs()))); - - return new BigtableIOOperationTimeouts( - useTimeouts, unaryTimeouts, bulkReadTimeouts, bulkMutateTimeouts); - } - - private Optional extractUnaryAttemptTimeout( - @UnderInitialization BigtableHBaseVeneeringSettings this, BigtableOptions options) { - - if (!options.getCallOptionsConfig().getShortRpcAttemptTimeoutMs().isPresent()) { - return Optional.of(DEFAULT_UNARY_ATTEMPT_TIMEOUTS); - } - long attemptTimeout = options.getCallOptionsConfig().getShortRpcAttemptTimeoutMs().get(); - return Optional.of(Duration.ofMillis(attemptTimeout)); - } - - private Optional extracBulkMutateAttemptTimeout( - @UnderInitialization BigtableHBaseVeneeringSettings this, BigtableOptions options) { - - if (!options.getCallOptionsConfig().getMutateRpcAttemptTimeoutMs().isPresent()) { - return Optional.of(DEFAULT_BULK_MUTATE_ATTEMPT_TIMEOUTS); - } - long attemptTimeout = options.getCallOptionsConfig().getMutateRpcAttemptTimeoutMs().get(); - return Optional.of(Duration.ofMillis(attemptTimeout)); - } - - private Optional extractBulkReadRowsAttemptTimeout( - @UnderInitialization BigtableHBaseVeneeringSettings this, BigtableOptions options) { - - if (!options.getCallOptionsConfig().getReadStreamRpcAttemptTimeoutMs().isPresent()) { - return Optional.absent(); - } - long attemptTimeout = options.getCallOptionsConfig().getReadStreamRpcAttemptTimeoutMs().get(); - return Optional.of(Duration.ofMillis(attemptTimeout)); - } - - private Set extractRetryCodesFromConfig( - @UnderInitialization BigtableHBaseVeneeringSettings this, - Set defaultCodes, - BigtableOptions options) { - - Set codes = new HashSet<>(defaultCodes); - - for (Status.Code code : options.getRetryOptions().getRetryableStatusCodes()) { - codes.add(StatusCode.Code.valueOf(code.name())); - } - - if (options.getRetryOptions().retryOnDeadlineExceeded()) { - codes.add(StatusCode.Code.DEADLINE_EXCEEDED); - } - - return codes; - } - - private void configureCredentialProvider( - @UnderInitialization BigtableHBaseVeneeringSettings this, - StubSettings.Builder stubSettings, - BigtableOptions options) - throws IOException { - CredentialOptions credentialOptions = options.getCredentialOptions(); - switch (credentialOptions.getCredentialType()) { - case DefaultCredentials: - GoogleCredentials credentials = GoogleCredentials.getApplicationDefault(); - - if (credentials instanceof ServiceAccountCredentials) { - stubSettings.setCredentialsProvider( - FixedCredentialsProvider.create( - getJwtToken((ServiceAccountCredentials) credentials))); - } - return; - case P12: - CredentialOptions.P12CredentialOptions p12Options = - (CredentialOptions.P12CredentialOptions) credentialOptions; - stubSettings.setCredentialsProvider( - FixedCredentialsProvider.create( - getCredentialFromPrivateKeyServiceAccount( - p12Options.getServiceAccount(), p12Options.getKeyFile()))); - return; - case SuppliedCredentials: - stubSettings.setCredentialsProvider( - FixedCredentialsProvider.create( - ((CredentialOptions.UserSuppliedCredentialOptions) credentialOptions) - .getCredential())); - return; - case SuppliedJson: - CredentialOptions.JsonCredentialsOptions jsonCredentialsOptions = - (CredentialOptions.JsonCredentialsOptions) credentialOptions; - synchronized (jsonCredentialsOptions) { - if (jsonCredentialsOptions.getCachedCredentials() == null) { - jsonCredentialsOptions.setCachedCredentails( - GoogleCredentials.fromStream(jsonCredentialsOptions.getInputStream())); - } - stubSettings.setCredentialsProvider( - FixedCredentialsProvider.create(jsonCredentialsOptions.getCachedCredentials())); - } - return; - case None: - stubSettings.setCredentialsProvider(NoCredentialsProvider.create()); - return; - default: - throw new IllegalStateException( - "Cannot process Credential type: " + credentialOptions.getCredentialType()); - } - } - - private static Credentials getJwtToken(ServiceAccountCredentials serviceAccount) { - return ServiceAccountJwtAccessCredentials.newBuilder() - .setClientEmail(serviceAccount.getClientEmail()) - .setClientId(serviceAccount.getClientId()) - .setPrivateKey(serviceAccount.getPrivateKey()) - .setPrivateKeyId(serviceAccount.getPrivateKeyId()) - .build(); - } - - public static Credentials getCredentialFromPrivateKeyServiceAccount( - String serviceAccountEmail, String privateKeyFile) throws IOException { - - try { - KeyStore keyStore = KeyStore.getInstance("PKCS12"); - - try (FileInputStream fin = new FileInputStream(privateKeyFile)) { - keyStore.load(fin, "notasecret".toCharArray()); - } - PrivateKey privateKey = - (PrivateKey) keyStore.getKey("privatekey", "notasecret".toCharArray()); - - if (privateKey == null) { - throw new IllegalStateException("private key cannot be null"); - } - return ServiceAccountJwtAccessCredentials.newBuilder() - .setClientEmail(serviceAccountEmail) - .setPrivateKey(privateKey) - .build(); - } catch (GeneralSecurityException exception) { - throw new RuntimeException("exception while retrieving credentials", exception); - } - } - - static class NoTimeoutsInterceptor implements ClientInterceptor { - static final CallOptions.Key SKIP_DEFAULT_ATTEMPT_TIMEOUT = - CallOptions.Key.createWithDefault("SKIP_DEFAULT_ATTEMPT_TIMEOUT", false); - - @Override - public ClientCall interceptCall( - MethodDescriptor methodDescriptor, CallOptions callOptions, Channel channel) { - - if (!callOptions.getOption(SKIP_DEFAULT_ATTEMPT_TIMEOUT)) { - callOptions = callOptions.withDeadline(Deadline.after(6, TimeUnit.MINUTES)); - } else { - callOptions = callOptions.withDeadline(null); - } - - return channel.newCall(methodDescriptor, callOptions); - } - } - - static class BigtableIOOperationTimeouts { - private final boolean useTimeouts; - private final OperationTimeouts unaryTimeouts; - private final OperationTimeouts bulkReadRowsTimeouts; - private final OperationTimeouts bulkMutateTimeouts; - - BigtableIOOperationTimeouts( - boolean useTimeouts, - OperationTimeouts unaryTimeouts, - OperationTimeouts bulkReadRowsTimeouts, - OperationTimeouts bulkMutateTimeouts) { - this.useTimeouts = useTimeouts; - this.unaryTimeouts = unaryTimeouts; - this.bulkReadRowsTimeouts = bulkReadRowsTimeouts; - this.bulkMutateTimeouts = bulkMutateTimeouts; - } - - boolean getUseTimeouts() { - return useTimeouts; - } - - OperationTimeouts getUnaryTimeouts() { - return unaryTimeouts; - } - - OperationTimeouts getBulkReadRowsTimeouts() { - return bulkReadRowsTimeouts; - } - - OperationTimeouts getBulkMutateTimeouts() { - return bulkMutateTimeouts; - } - } - - static class OperationTimeouts { - static final OperationTimeouts EMPTY = - new OperationTimeouts( - Optional.absent(), Optional.absent(), Optional.absent()); - - // responseTimeouts are only relevant to streaming RPCs, they limit the amount of timeout a - // stream will wait for the next response message. This is synonymous with attemptTimeouts in - // unary RPCs since they receive a single response (so its ignored). - private final Optional responseTimeout; - private final Optional attemptTimeout; - private final Optional operationTimeout; - - OperationTimeouts( - Optional responseTimeout, - Optional attemptTimeout, - Optional operationTimeout) { - this.responseTimeout = responseTimeout; - this.attemptTimeout = attemptTimeout; - this.operationTimeout = operationTimeout; - } - - Optional getResponseTimeout() { - return responseTimeout; - } - - Optional getAttemptTimeout() { - return attemptTimeout; - } - - Optional getOperationTimeout() { - return operationTimeout; - } - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 4329d99084cd..be289f4c8798 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -22,6 +22,7 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; +import com.google.auth.Credentials; import com.google.auto.value.AutoValue; import com.google.bigtable.v2.Mutation; import com.google.bigtable.v2.Row; @@ -118,6 +119,16 @@ * .withInstanceId(instanceId) * .withTableId("table") * .withRowFilter(filter)); + * + * // Configure timeouts for reads + * p.apply("read", + * BigtableIO.read() + * .withProjectId(projectId) + * .withInstanceId(instanceId) + * .withTableId("table") + * .withKeyRange(keyRange) + * .withAttemptTimeout(100) // 100 milliseconds + * .withOperationTimeout(60 * 1000)); // 1 minute * } * *

Writing to Cloud Bigtable

@@ -139,6 +150,15 @@ * .withProjectId("project") * .withInstanceId("instance") * .withTableId("table")); + * } + * + * // Configure batch size for writes + * data.apply("write", + * BigtableIO.write() + * .withProjectId("project") + * .withInstanceId("instance") + * .withTableId("table") + * .withBatchElements(100)); // every batch will have 100 elements * } * *

Optionally, BigtableIO.write() may be configured to emit {@link BigtableWriteResult} elements @@ -223,14 +243,14 @@ public abstract static class Read extends PTransform> { /** Returns the table being read from. */ public @Nullable String getTableId() { - ValueProvider tableId = getBigtableConfig().getTableId(); + ValueProvider tableId = getBigtableReadOptions().getTableId(); return tableId != null && tableId.isAccessible() ? tableId.get() : null; } /** * Returns the Google Cloud Bigtable instance being read from, and other parameters. * - * @deprecated will be replaced by bigtable options configurator. + * @deprecated please use {@link #getBigtableReadOptions()}. */ @Deprecated public @Nullable BigtableOptions getBigtableOptions() { @@ -240,13 +260,13 @@ public abstract static class Read extends PTransform> { abstract Builder toBuilder(); static Read create() { - BigtableConfig config = - BigtableConfig.builder().setTableId(StaticValueProvider.of("")).setValidate(true).build(); + BigtableConfig config = BigtableConfig.builder().setValidate(true).build(); return new AutoValue_BigtableIO_Read.Builder() .setBigtableConfig(config) .setBigtableReadOptions( BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of("")) .setKeyRanges( StaticValueProvider.of(Collections.singletonList(ByteKeyRange.ALL_KEYS))) .build()) @@ -315,8 +335,10 @@ public Read withInstanceId(String instanceId) { *

Does not modify this object. */ public Read withTableId(ValueProvider tableId) { - BigtableConfig config = getBigtableConfig(); - return toBuilder().setBigtableConfig(config.withTableId(tableId)).build(); + BigtableReadOptions bigtableReadOptions = getBigtableReadOptions(); + return toBuilder() + .setBigtableReadOptions(bigtableReadOptions.toBuilder().setTableId(tableId).build()) + .build(); } /** @@ -328,6 +350,16 @@ public Read withTableId(String tableId) { return withTableId(StaticValueProvider.of(tableId)); } + /** + * Returns a new {@link BigtableIO.Read} with provided credentials. + * + *

Does not modify this object. + */ + public Read withCredentials(Credentials credentials) { + BigtableConfig config = getBigtableConfig(); + return toBuilder().setBigtableConfig(config.withCredentails(credentials)).build(); + } + /** * WARNING: Should be used only to specify additional parameters for connection to the Cloud * Bigtable, instanceId and projectId should be provided over {@link #withInstanceId} and {@link @@ -338,7 +370,7 @@ public Read withTableId(String tableId) { * *

Does not modify this object. * - * @deprecated will be replaced by bigtable options configurator. + * @deprecated please set the configurations directly. */ @Deprecated public Read withBigtableOptions(BigtableOptions options) { @@ -359,12 +391,11 @@ public Read withBigtableOptions(BigtableOptions options) { * *

Does not modify this object. * - * @deprecated will be replaced by bigtable options configurator. + * @deprecated please set the configurations directly. */ @Deprecated public Read withBigtableOptions(BigtableOptions.Builder optionsBuilder) { BigtableConfig config = getBigtableConfig(); - // TODO: is there a better way to clone a Builder? Want it to be immune from user changes. return toBuilder() .setBigtableConfig(config.withBigtableOptions(optionsBuilder.build().toBuilder().build())) .build(); @@ -378,7 +409,10 @@ public Read withBigtableOptions(BigtableOptions.Builder optionsBuilder) { * {@link #withProjectId} and {@link #withInstanceId}. * *

Does not modify this object. + * + * @deprecated please set the configurations directly. */ + @Deprecated public Read withBigtableOptionsConfigurator( SerializableFunction configurator) { BigtableConfig config = getBigtableConfig(); @@ -497,8 +531,86 @@ public Read withEmulator(String emulatorHost) { return toBuilder().setBigtableConfig(config.withEmulator(emulatorHost)).build(); } + /** + * Configures the attempt timeout in milliseconds of the reads. + * + *

Does not modify this object. + */ + public Read withAttemptTimeout(long timeoutMs) { + checkArgument(timeoutMs > 0, "attempt timeout must be positive"); + BigtableReadOptions readOptions = getBigtableReadOptions(); + return toBuilder() + .setBigtableReadOptions(readOptions.toBuilder().setAttemptTimeout(timeoutMs).build()) + .build(); + } + + /** + * Configures the operation timeout in milliseconds of the reads. + * + *

Does not modify this object. + */ + public Read withOperationTimeout(long timeoutMs) { + checkArgument(timeoutMs > 0, "operation timeout must be positive"); + BigtableReadOptions readOptions = getBigtableReadOptions(); + return toBuilder() + .setBigtableReadOptions(readOptions.toBuilder().setOperationTimeout(timeoutMs).build()) + .build(); + } + + /** + * Configures the initial retry delay in milliseconds. + * + *

Does not modify this object. + */ + public Read withRetryInitialDelay(long initialDelayMs) { + checkArgument(initialDelayMs > 0, "initial delay must be positive"); + BigtableReadOptions readOptions = getBigtableReadOptions(); + return toBuilder() + .setBigtableReadOptions( + readOptions.toBuilder().setRetryInitialDelay(initialDelayMs).build()) + .build(); + } + + /** + * Configures the delay multiplier. + * + *

Does not modify this object. + */ + public Read withRetryDelayMultiplier(double multiplier) { + checkArgument(multiplier > 0, "delay multiplier must be positive"); + BigtableReadOptions readOptions = getBigtableReadOptions(); + return toBuilder() + .setBigtableReadOptions( + readOptions.toBuilder().setRetryDelayMultiplier(multiplier).build()) + .build(); + } + + /** Helper method to translate BigtableOptions to BigtableReadOptions. */ + private void translateBigtableOptions(BigtableOptions options) { + BigtableReadOptions.Builder readOptions = getBigtableReadOptions().toBuilder(); + if (options.getCallOptionsConfig().getReadStreamRpcAttemptTimeoutMs().isPresent()) { + readOptions.setAttemptTimeout( + options.getCallOptionsConfig().getReadStreamRpcAttemptTimeoutMs().get()); + } + readOptions.setOperationTimeout(options.getCallOptionsConfig().getReadStreamRpcTimeoutMs()); + readOptions.setRetryInitialDelay(options.getRetryOptions().getInitialBackoffMillis()); + readOptions.setRetryDelayMultiplier(options.getRetryOptions().getBackoffMultiplier()); + + toBuilder().setBigtableReadOptions(readOptions.build()).build(); + } + @Override public PCollection expand(PBegin input) { + BigtableConfig config = this.getBigtableConfig(); + // If BigtableOptions present, convert BigtableOptions into BigtableReadOptions + if (config.getBigtableOptions() != null) { + translateBigtableOptions(config.getBigtableOptions()); + } else if (config.getBigtableOptionsConfigurator() != null) { + BigtableOptions.Builder options = BigtableOptions.builder(); + options = config.getBigtableOptionsConfigurator().apply(options); + translateBigtableOptions(options.build()); + } + getBigtableConfig().validate(); getBigtableReadOptions().validate(); @@ -509,7 +621,7 @@ public PCollection expand(PBegin input) { @Override public void validate(PipelineOptions options) { - validateTableExists(getBigtableConfig(), options); + validateTableExists(getBigtableConfig(), getBigtableReadOptions(), options); } @Override @@ -553,10 +665,12 @@ public abstract static class Write abstract BigtableConfig getBigtableConfig(); + abstract BigtableWriteOptions getBigtableWriteOptions(); + /** * Returns the Google Cloud Bigtable instance being written to, and other parameters. * - * @deprecated will be replaced by bigtable options configurator. + * @deprecated please configure the write options directly. */ @Deprecated public @Nullable BigtableOptions getBigtableOptions() { @@ -566,14 +680,15 @@ public abstract static class Write abstract Builder toBuilder(); static Write create() { - BigtableConfig config = - BigtableConfig.builder() - .setTableId(StaticValueProvider.of("")) - .setValidate(true) - .setBigtableOptionsConfigurator(enableBulkApiConfigurator(null)) - .build(); + BigtableConfig config = BigtableConfig.builder().setValidate(true).build(); + + BigtableWriteOptions writeOptions = + BigtableWriteOptions.builder().setTableId(StaticValueProvider.of("")).build(); - return new AutoValue_BigtableIO_Write.Builder().setBigtableConfig(config).build(); + return new AutoValue_BigtableIO_Write.Builder() + .setBigtableConfig(config) + .setBigtableWriteOptions(writeOptions) + .build(); } @AutoValue.Builder @@ -581,6 +696,8 @@ abstract static class Builder { abstract Builder setBigtableConfig(BigtableConfig bigtableConfig); + abstract Builder setBigtableWriteOptions(BigtableWriteOptions writeOptions); + abstract Write build(); } @@ -636,8 +753,10 @@ public Write withInstanceId(String instanceId) { *

Does not modify this object. */ public Write withTableId(ValueProvider tableId) { - BigtableConfig config = getBigtableConfig(); - return toBuilder().setBigtableConfig(config.withTableId(tableId)).build(); + BigtableWriteOptions writeOptions = getBigtableWriteOptions(); + return toBuilder() + .setBigtableWriteOptions(writeOptions.toBuilder().setTableId(tableId).build()) + .build(); } /** @@ -649,6 +768,15 @@ public Write withTableId(String tableId) { return withTableId(StaticValueProvider.of(tableId)); } + /** + * Returns a new {@link BigtableIO.Write} with the provided credentials. + * + *

Does not modify this object. + */ + public Write withCredentials(Credentials credentials) { + BigtableConfig config = getBigtableConfig(); + return toBuilder().setBigtableConfig(config.withCredentails(credentials)).build(); + } /** * WARNING: Should be used only to specify additional parameters for connection to the Cloud * Bigtable, instanceId and projectId should be provided over {@link #withInstanceId} and {@link @@ -659,7 +787,7 @@ public Write withTableId(String tableId) { * *

Does not modify this object. * - * @deprecated will be replaced by bigtable options configurator. + * @deprecated please configure the write options directly. */ @Deprecated public Write withBigtableOptions(BigtableOptions options) { @@ -680,14 +808,13 @@ public Write withBigtableOptions(BigtableOptions options) { * *

Does not modify this object. * - * @deprecated will be replaced by bigtable options configurator. + * @deprecated please configure the write options directly. */ @Deprecated public Write withBigtableOptions(BigtableOptions.Builder optionsBuilder) { BigtableConfig config = getBigtableConfig(); - // TODO: is there a better way to clone a Builder? Want it to be immune from user changes. return toBuilder() - .setBigtableConfig(config.withBigtableOptions(optionsBuilder.build().toBuilder().build())) + .setBigtableConfig(config.withBigtableOptions(optionsBuilder.build())) .build(); } @@ -699,7 +826,10 @@ public Write withBigtableOptions(BigtableOptions.Builder optionsBuilder) { * {@link #withProjectId} and {@link #withInstanceId}. * *

Does not modify this object. + * + * @deprecated please configure the write options directly. */ + @Deprecated public Write withBigtableOptionsConfigurator( SerializableFunction configurator) { BigtableConfig config = getBigtableConfig(); @@ -739,16 +869,136 @@ public Write withEmulator(String emulatorHost) { return toBuilder().setBigtableConfig(config.withEmulator(emulatorHost)).build(); } + /** + * Returns a new {@link BigtableIO.Write} with the attempt timeout in milliseconds for writes. + * + *

Does not modify this object. + */ + public Write withAttemptTimeout(long timeoutMs) { + checkArgument(timeoutMs > 0, "attempt timeout must be positive"); + BigtableWriteOptions options = getBigtableWriteOptions(); + return toBuilder() + .setBigtableWriteOptions(options.toBuilder().setAttemptTimeout(timeoutMs).build()) + .build(); + } + + /** + * Returns a new {@link BigtableIO.Write} with the operation timeout in milliseconds for writes. + * + *

Does not modify this object. + */ + public Write withOperationTimeout(long timeoutMs) { + checkArgument(timeoutMs > 0, "operation timeout must be positive"); + BigtableWriteOptions options = getBigtableWriteOptions(); + return toBuilder() + .setBigtableWriteOptions(options.toBuilder().setOperationTimeout(timeoutMs).build()) + .build(); + } + + /** + * Returns a new {@link BigtableIO.Write} with the retry delay in milliseconds. + * + *

Does not modify this object. + */ + public Write withRetryInitialDelay(long delayMs) { + checkArgument(delayMs > 0, "delay must be positive"); + BigtableWriteOptions options = getBigtableWriteOptions(); + return toBuilder() + .setBigtableWriteOptions(options.toBuilder().setRetryInitialDelay(delayMs).build()) + .build(); + } + + /** + * Returns a new {@link BigtableIO.Write} with retry multiplier. + * + *

Does not modify this object. + */ + public Write withRetryDelayMultiplier(double multiplier) { + checkArgument(multiplier > 0, "multiplier must be positive"); + BigtableWriteOptions options = getBigtableWriteOptions(); + return toBuilder() + .setBigtableWriteOptions(options.toBuilder().setRetryDelayMultiplier(multiplier).build()) + .build(); + } + + /** + * Returns a new {@link BigtableIO.Write} with the number of elements in a batch. + * + *

Does not modify this object. + */ + public Write withBatchElements(long size) { + checkArgument(size > 0, "batch element size must be positive"); + BigtableWriteOptions options = getBigtableWriteOptions(); + return toBuilder() + .setBigtableWriteOptions(options.toBuilder().setBatchElements(size).build()) + .build(); + } + + /** + * Returns a new {@link BigtableIO.Write} with the number of bytes in a batch. + * + *

Does not modify this object. + */ + public Write withBatchBytes(long size) { + checkArgument(size > 0, "batch byte size must be positive"); + BigtableWriteOptions options = getBigtableWriteOptions(); + return toBuilder() + .setBigtableWriteOptions(options.toBuilder().setBatchBytes(size).build()) + .build(); + } + + /** + * Returns a new {@link BigtableIO.Write} with the max number of concurrent requests. + * + *

Does not modify this object. + */ + public Write withMaxRequests(long requests) { + checkArgument(requests > 0, "max requests must be positive"); + BigtableWriteOptions options = getBigtableWriteOptions(); + return toBuilder() + .setBigtableWriteOptions(options.toBuilder().setMaxRequests(requests).build()) + .build(); + } + + /** Helper method to translate Bigtable Options to BigtableWriteOptions. */ + private void translateBigtableOptions(BigtableOptions options) { + BigtableWriteOptions.Builder writeOptions = getBigtableWriteOptions().toBuilder(); + // configure timeouts + if (options.getCallOptionsConfig().getMutateRpcAttemptTimeoutMs().isPresent()) { + writeOptions.setAttemptTimeout( + options.getCallOptionsConfig().getMutateRpcAttemptTimeoutMs().get()); + } + writeOptions.setOperationTimeout(options.getCallOptionsConfig().getMutateRpcTimeoutMs()); + // configure retry backoffs + writeOptions.setRetryInitialDelay(options.getRetryOptions().getInitialBackoffMillis()); + writeOptions.setRetryDelayMultiplier(options.getRetryOptions().getBackoffMultiplier()); + // configure batch size + writeOptions.setBatchElements(options.getBulkOptions().getBulkMaxRowKeyCount()); + writeOptions.setBatchBytes(options.getBulkOptions().getBulkMaxRequestSize()); + writeOptions.setMaxRequests(options.getBulkOptions().getMaxInflightRpcs()); + + toBuilder().setBigtableWriteOptions(writeOptions.build()).build(); + } + /** * Returns a {@link BigtableIO.WriteWithResults} that will emit a {@link BigtableWriteResult} * for each batch of rows written. */ public WriteWithResults withWriteResults() { - return new WriteWithResults(getBigtableConfig()); + return new WriteWithResults(getBigtableConfig(), getBigtableWriteOptions()); } @Override public PDone expand(PCollection>> input) { + BigtableConfig config = this.getBigtableConfig(); + if (config.getBigtableOptions() != null) { + translateBigtableOptions(config.getBigtableOptions()); + } else if (config.getBigtableOptionsConfigurator() != null) { + BigtableOptions.Builder options = BigtableOptions.builder(); + options = config.getBigtableOptionsConfigurator().apply(options); + translateBigtableOptions(options.build()); + } + input.apply(withWriteResults()); return PDone.in(input.getPipeline()); } @@ -781,34 +1031,39 @@ public static class WriteWithResults PCollection>>, PCollection> { private final BigtableConfig bigtableConfig; + private final BigtableWriteOptions bigtableWriteOptions; - WriteWithResults(BigtableConfig bigtableConfig) { + WriteWithResults(BigtableConfig bigtableConfig, BigtableWriteOptions bigtableWriteOptions) { this.bigtableConfig = bigtableConfig; + this.bigtableWriteOptions = bigtableWriteOptions; } @Override public PCollection expand( PCollection>> input) { bigtableConfig.validate(); + bigtableWriteOptions.validate(); - return input.apply(ParDo.of(new BigtableWriterFn(bigtableConfig))); + return input.apply(ParDo.of(new BigtableWriterFn(bigtableConfig, bigtableWriteOptions))); } @Override public void validate(PipelineOptions options) { - validateTableExists(bigtableConfig, options); + validateTableExists(bigtableConfig, bigtableWriteOptions, options); } @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); bigtableConfig.populateDisplayData(builder); + bigtableWriteOptions.populateDisplayData(builder); } @Override public String toString() { return MoreObjects.toStringHelper(WriteWithResults.class) .add("config", bigtableConfig) + .add("writeOptions", bigtableWriteOptions) .toString(); } } @@ -816,8 +1071,9 @@ public String toString() { private static class BigtableWriterFn extends DoFn>, BigtableWriteResult> { - BigtableWriterFn(BigtableConfig bigtableConfig) { + BigtableWriterFn(BigtableConfig bigtableConfig, BigtableWriteOptions writeOptions) { this.config = bigtableConfig; + this.writeOptions = writeOptions; this.failures = new ConcurrentLinkedQueue<>(); } @@ -827,7 +1083,7 @@ public void startBundle(StartBundleContext c) throws IOException { bigtableWriter = config .getBigtableService(c.getPipelineOptions()) - .openForWriting(config.getTableId().get()); + .openForWriting(writeOptions.getTableId().get(), writeOptions); } recordsWritten = 0; this.seenWindows = Maps.newHashMapWithExpectedSize(1); @@ -877,6 +1133,7 @@ public void populateDisplayData(DisplayData.Builder builder) { /////////////////////////////////////////////////////////////////////////////// private final BigtableConfig config; + private final BigtableWriteOptions writeOptions; private BigtableService.Writer bigtableWriter; private long recordsWritten; private final ConcurrentLinkedQueue failures; @@ -1223,7 +1480,7 @@ public void validate() { return; } - ValueProvider tableId = config.getTableId(); + ValueProvider tableId = readOptions.getTableId(); checkArgument( tableId != null && tableId.isAccessible() && !tableId.get().isEmpty(), "tableId was not supplied"); @@ -1233,7 +1490,7 @@ public void validate() { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder.add(DisplayData.item("tableId", config.getTableId()).withLabel("Table ID")); + builder.add(DisplayData.item("tableId", readOptions.getTableId()).withLabel("Table ID")); if (getRowFilter() != null) { builder.add( @@ -1281,6 +1538,10 @@ private List splitKeyRangeIntoBundleSizedSubranges( return splits.build(); } + public BigtableReadOptions getReadOptions() { + return readOptions; + } + public List getRanges() { return readOptions.getKeyRanges().get(); } @@ -1295,7 +1556,7 @@ public List getRanges() { } public ValueProvider getTableId() { - return config.getTableId(); + return readOptions.getTableId(); } } @@ -1415,9 +1676,25 @@ public BigtableWriteException(KV> record, Throwab } } - static void validateTableExists(BigtableConfig config, PipelineOptions options) { - if (config.getValidate() && config.isDataAccessible()) { - String tableId = checkNotNull(config.getTableId().get()); + static void validateTableExists( + BigtableConfig config, BigtableWriteOptions writeOptions, PipelineOptions options) { + if (config.getValidate() && config.isDataAccessible() && writeOptions.isDataAccessible()) { + String tableId = checkNotNull(writeOptions.getTableId().get()); + try { + checkArgument( + config.getBigtableService(options).tableExists(tableId), + "Table %s does not exist", + tableId); + } catch (IOException e) { + LOG.warn("Error checking whether table {} exists; proceeding.", tableId, e); + } + } + } + + static void validateTableExists( + BigtableConfig config, BigtableReadOptions readOptions, PipelineOptions options) { + if (config.getValidate() && config.isDataAccessible() && readOptions.isDataAccessible()) { + String tableId = checkNotNull(readOptions.getTableId().get()); try { checkArgument( config.getBigtableService(options).tableExists(tableId), diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadOptions.java index 1aa0e6571f8f..73d2f20f1465 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadOptions.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadOptions.java @@ -29,13 +29,16 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.checkerframework.checker.nullness.qual.Nullable; -/** Configuration for which values to read from Bigtable. */ +/** Configuration for read from Bigtable. */ @AutoValue @SuppressWarnings({ "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) abstract class BigtableReadOptions implements Serializable { + /** Returns the table id. */ + abstract ValueProvider getTableId(); + /** Returns the row filter to use. */ abstract @Nullable ValueProvider getRowFilter(); @@ -45,6 +48,18 @@ abstract class BigtableReadOptions implements Serializable { /** Returns the size limit for reading segements. */ abstract @Nullable Integer getMaxBufferElementCount(); + /** Returns the attempt timeout of the reads. */ + abstract @Nullable Long getAttemptTimeout(); + + /** Returns the operation timeout of the reads. */ + abstract @Nullable Long getOperationTimeout(); + + /** Returns the retry delay. */ + abstract @Nullable Long getRetryInitialDelay(); + + /** Returns the retry delay multiplier. */ + abstract @Nullable Double getRetryDelayMultiplier(); + abstract Builder toBuilder(); static BigtableReadOptions.Builder builder() { @@ -54,12 +69,22 @@ static BigtableReadOptions.Builder builder() { @AutoValue.Builder abstract static class Builder { + abstract Builder setTableId(ValueProvider tableId); + abstract Builder setRowFilter(ValueProvider rowFilter); abstract Builder setMaxBufferElementCount(@Nullable Integer maxBufferElementCount); abstract Builder setKeyRanges(ValueProvider> keyRanges); + abstract Builder setAttemptTimeout(long timeout); + + abstract Builder setOperationTimeout(long timeout); + + abstract Builder setRetryInitialDelay(long delay); + + abstract Builder setRetryDelayMultiplier(double multiplier); + abstract BigtableReadOptions build(); } @@ -79,13 +104,34 @@ BigtableReadOptions withKeyRange(ByteKeyRange keyRange) { return withKeyRanges(Collections.singletonList(keyRange)); } + boolean isDataAccessible() { + return getTableId() != null && getTableId().isAccessible(); + } + void populateDisplayData(DisplayData.Builder builder) { builder + .addIfNotNull(DisplayData.item("tableId", getTableId()).withLabel("Bigtable Table Id")) .addIfNotNull(DisplayData.item("rowFilter", getRowFilter()).withLabel("Row Filter")) - .addIfNotNull(DisplayData.item("keyRanges", getKeyRanges()).withLabel("Key Ranges")); + .addIfNotNull(DisplayData.item("keyRanges", getKeyRanges()).withLabel("Key Ranges")) + .addIfNotNull( + DisplayData.item("attemptTimeout", getAttemptTimeout()) + .withLabel("Read Attempt Timeout")) + .addIfNotNull( + DisplayData.item("operationTimeout", getOperationTimeout()) + .withLabel("Read Operation Timeout")) + .addIfNotNull( + DisplayData.item("retryInitialDelay", getRetryInitialDelay()) + .withLabel("Read retry initial delay")) + .addIfNotNull( + DisplayData.item("retryDelayMultiplier", getRetryDelayMultiplier()) + .withLabel("Read retry delay multiplier")); } void validate() { + checkArgument( + getTableId() != null && (!getTableId().isAccessible() || !getTableId().get().isEmpty()), + "Could not obtain Bigtable table id"); + if (getRowFilter() != null && getRowFilter().isAccessible()) { checkArgument(getRowFilter().get() != null, "rowFilter can not be null"); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java index a84f55f66c44..709abbef3383 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java @@ -17,10 +17,10 @@ */ package org.apache.beam.sdk.io.gcp.bigtable; +import com.google.auth.Credentials; import com.google.bigtable.v2.MutateRowResponse; import com.google.bigtable.v2.Mutation; import com.google.bigtable.v2.Row; -import com.google.cloud.bigtable.config.BigtableOptions; import com.google.cloud.bigtable.data.v2.models.KeyOffset; import com.google.protobuf.ByteString; import java.io.IOException; @@ -85,9 +85,6 @@ interface Reader { Row getCurrentRow() throws NoSuchElementException; } - /** Returns the BigtableOptions used to configure this BigtableService. */ - BigtableOptions getBigtableOptions(); - /** Returns {@code true} if the table with the give name exists. */ boolean tableExists(String tableId) throws IOException; @@ -95,11 +92,19 @@ interface Reader { Reader createReader(BigtableSource source) throws IOException; /** Returns a {@link Writer} that will write to the specified table. */ - Writer openForWriting(String tableId) throws IOException; + Writer openForWriting(String tableId, BigtableWriteOptions writeOptions) throws IOException; /** * Returns a set of row keys sampled from the underlying table. These contain information about * the distribution of keys within the table. */ List getSampleRowKeys(BigtableSource source) throws IOException; + + String getProjectId(); + + String getInstanceId(); + + Credentials getCredentials(); + + String getEmulatorHost(); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java index ffef972b73d5..0435e30fc738 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java @@ -17,13 +17,15 @@ */ package org.apache.beam.sdk.io.gcp.bigtable; -import static org.apache.beam.sdk.io.gcp.bigtable.BigtableHBaseVeneeringSettings.BigtableIOOperationTimeouts; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.MoreExecutors.directExecutor; import com.google.api.gax.batching.Batcher; +import com.google.api.gax.batching.BatchingSettings; import com.google.api.gax.grpc.GrpcCallContext; +import com.google.api.gax.retrying.RetrySettings; import com.google.api.gax.rpc.ResponseObserver; import com.google.api.gax.rpc.StreamController; +import com.google.auth.Credentials; import com.google.bigtable.v2.Cell; import com.google.bigtable.v2.Column; import com.google.bigtable.v2.Family; @@ -36,7 +38,6 @@ import com.google.bigtable.v2.RowSet; import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; import com.google.cloud.bigtable.admin.v2.BigtableTableAdminSettings; -import com.google.cloud.bigtable.config.BigtableOptions; import com.google.cloud.bigtable.data.v2.BigtableDataClient; import com.google.cloud.bigtable.data.v2.BigtableDataSettings; import com.google.cloud.bigtable.data.v2.internal.ByteStringComparator; @@ -46,6 +47,7 @@ import com.google.cloud.bigtable.data.v2.models.Query; import com.google.cloud.bigtable.data.v2.models.RowAdapter; import com.google.cloud.bigtable.data.v2.models.RowMutationEntry; +import com.google.cloud.bigtable.data.v2.stub.BigtableBatchingCallSettings; import com.google.protobuf.ByteString; import io.grpc.CallOptions; import io.grpc.Deadline; @@ -85,6 +87,7 @@ import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.threeten.bp.Duration; /** * An implementation of {@link BigtableService} that actually communicates with the Cloud Bigtable @@ -101,36 +104,34 @@ class BigtableServiceImpl implements BigtableService { private static final double WATERMARK_PERCENTAGE = .1; private static final long MIN_BYTE_BUFFER_SIZE = 100 * 1024 * 1024; // 100MB - public BigtableServiceImpl(BigtableOptions options) { + public BigtableServiceImpl(BigtableConfig config) { try { - this.options = options; - BigtableHBaseVeneeringSettings veneeringSettings = - BigtableHBaseVeneeringSettings.create(options); + this.config = config; + BigtableConfigToVeneerSettings veneeringSettings = + BigtableConfigToVeneerSettings.create(this.config); this.dataSettings = veneeringSettings.getDataSettings(); this.tableAdminSettings = veneeringSettings.getTableAdminSettings(); - this.operationTimeouts = veneeringSettings.getOperationTimeouts(); } catch (IOException e) { throw new RuntimeException(e); } } - private final BigtableOptions options; + private final BigtableConfig config; - private final BigtableDataSettings dataSettings; + private BigtableDataSettings dataSettings; - private final BigtableTableAdminSettings tableAdminSettings; - - private final BigtableHBaseVeneeringSettings.BigtableIOOperationTimeouts operationTimeouts; + private BigtableTableAdminSettings tableAdminSettings; @Override - public BigtableOptions getBigtableOptions() { - return options; - } - - @Override - public BigtableWriterImpl openForWriting(String tableId) throws IOException { - LOG.info("Opening for writing with settings " + dataSettings.toString()); - return new BigtableWriterImpl(BigtableDataClient.create(dataSettings), dataSettings, tableId); + public BigtableWriterImpl openForWriting(String tableId, BigtableWriteOptions options) + throws IOException { + dataSettings = configureWriteSettings(dataSettings.toBuilder(), options); + LOG.info("Opening for writing with settings " + dataSettings); + return new BigtableWriterImpl( + BigtableDataClient.create(dataSettings), + dataSettings.getProjectId(), + dataSettings.getInstanceId(), + tableId); } @Override @@ -145,7 +146,7 @@ public boolean tableExists(String tableId) throws IOException { } String message = String.format( - "Error checking whether table %s (BigtableOptions %s) exists", tableId, options); + "Error checking whether table %s (BigtableOptions %s) exists", tableId, config); LOG.error(message, e); throw new IOException(message, e); } @@ -153,47 +154,61 @@ public boolean tableExists(String tableId) throws IOException { @VisibleForTesting static class BigtableReaderImpl implements Reader { - private BigtableDataSettings settings; private BigtableDataClient client; - private final BigtableSource source; + + private final String projectId; + private final String instanceId; + private final String tableId; + + private final List ranges; + private final RowFilter rowFilter; private Iterator results; - private BigtableIOOperationTimeouts operationTimeouts; + + private final Duration operationTimeout; + private final Duration attemptTimeout; private Row currentRow; @VisibleForTesting BigtableReaderImpl( BigtableDataClient client, - BigtableDataSettings settings, - BigtableSource source, - BigtableIOOperationTimeouts operationTimeouts) { - this.settings = settings; + String projectId, + String instanceId, + String tableId, + List ranges, + @Nullable RowFilter rowFilter, + Duration operationTimeout, + Duration attemptTimeout) { this.client = client; - this.source = source; - this.operationTimeouts = operationTimeouts; + this.projectId = projectId; + this.instanceId = instanceId; + this.tableId = tableId; + this.ranges = ranges; + this.rowFilter = rowFilter; + + this.operationTimeout = operationTimeout; + this.attemptTimeout = attemptTimeout; } @Override public boolean start() throws IOException { - ServiceCallMetric serviceCallMetric = - createCallMetric( - settings.getProjectId(), settings.getInstanceId(), source.getTableId().get()); + ServiceCallMetric serviceCallMetric = createCallMetric(projectId, instanceId, tableId); - Query query = Query.create(source.getTableId().get()); - for (ByteKeyRange sourceRange : source.getRanges()) { + Query query = Query.create(tableId); + for (ByteKeyRange sourceRange : ranges) { query.range( ByteString.copyFrom(sourceRange.getStartKey().getValue()), ByteString.copyFrom(sourceRange.getEndKey().getValue())); } - if (source.getRowFilter() != null) { - query.filter(Filters.FILTERS.fromProto(source.getRowFilter())); + if (rowFilter != null) { + query.filter(Filters.FILTERS.fromProto(rowFilter)); } try { results = client .readRowsCallable(new BeamRowAdapter()) - .call(query, BigtableServiceImpl.createScanCallContext(operationTimeouts)) + .call(query, createScanCallContext(attemptTimeout, operationTimeout)) .iterator(); serviceCallMetric.call("ok"); } catch (StatusRuntimeException e) { @@ -238,6 +253,7 @@ public Row getCurrentRow() throws NoSuchElementException { @VisibleForTesting static class BigtableSegmentReaderImpl implements Reader { private BigtableDataClient client; + private @Nullable ReadRowsRequest nextRequest; private @Nullable Row currentRow; private @Nullable Future future; @@ -245,7 +261,8 @@ static class BigtableSegmentReaderImpl implements Reader { private final int refillSegmentWaterMark; private final long maxSegmentByteSize; private ServiceCallMetric serviceCallMetric; - private BigtableIOOperationTimeouts operationTimeouts; + private final Duration attemptTimeout; + private final Duration operationTimeout; private static class UpstreamResults { private final List rows; @@ -259,16 +276,21 @@ private UpstreamResults(List rows, @Nullable ReadRowsRequest nextRequest) { static BigtableSegmentReaderImpl create( BigtableDataClient client, - BigtableDataSettings settings, - BigtableSource source, - BigtableIOOperationTimeouts operationTimeouts) { + String projectId, + String instanceId, + String tableId, + List ranges, + @Nullable RowFilter rowFilter, + int maxBufferedElementCount, + Duration attemptTimeout, + Duration operationTimeout) { RowSet.Builder rowSetBuilder = RowSet.newBuilder(); - if (source.getRanges().isEmpty()) { + if (ranges.isEmpty()) { rowSetBuilder = RowSet.newBuilder().addRowRanges(RowRange.getDefaultInstance()); } else { // BigtableSource only contains ranges with a closed start key and open end key - for (ByteKeyRange beamRange : source.getRanges()) { + for (ByteKeyRange beamRange : ranges) { RowRange.Builder rangeBuilder = rowSetBuilder.addRowRangesBuilder(); rangeBuilder .setStartKeyClosed(ByteString.copyFrom(beamRange.getStartKey().getValue())) @@ -276,8 +298,7 @@ static BigtableSegmentReaderImpl create( } } RowSet rowSet = rowSetBuilder.build(); - RowFilter filter = - MoreObjects.firstNonNull(source.getRowFilter(), RowFilter.getDefaultInstance()); + RowFilter filter = MoreObjects.firstNonNull(rowFilter, RowFilter.getDefaultInstance()); long maxSegmentByteSize = (long) @@ -287,36 +308,37 @@ static BigtableSegmentReaderImpl create( return new BigtableSegmentReaderImpl( client, - settings, - source.getTableId().get(), + projectId, + instanceId, + tableId, rowSet, - source.getMaxBufferElementCount(), - maxSegmentByteSize, filter, - createCallMetric( - settings.getProjectId(), settings.getInstanceId(), source.getTableId().get()), - operationTimeouts); + maxBufferedElementCount, + maxSegmentByteSize, + attemptTimeout, + operationTimeout, + createCallMetric(projectId, instanceId, tableId)); } @VisibleForTesting BigtableSegmentReaderImpl( BigtableDataClient client, - BigtableDataSettings settings, + String projectId, + String instanceId, String tableId, RowSet rowSet, + @Nullable RowFilter filter, int maxRowsInBuffer, long maxSegmentByteSize, - RowFilter filter, - ServiceCallMetric serviceCallMetric, - BigtableIOOperationTimeouts operationTimeouts) { + Duration attemptTimeout, + Duration operationTimeout, + ServiceCallMetric serviceCallMetric) { if (rowSet.equals(rowSet.getDefaultInstanceForType())) { rowSet = RowSet.newBuilder().addRowRanges(RowRange.getDefaultInstance()).build(); } ReadRowsRequest request = ReadRowsRequest.newBuilder() - .setTableName( - NameUtil.formatTableName( - settings.getProjectId(), settings.getInstanceId(), tableId)) + .setTableName(NameUtil.formatTableName(projectId, instanceId, tableId)) .setRows(rowSet) .setFilter(filter) .setRowsLimit(maxRowsInBuffer) @@ -329,7 +351,8 @@ static BigtableSegmentReaderImpl create( this.buffer = new ArrayDeque<>(); // Asynchronously refill buffer when there is 10% of the elements are left this.refillSegmentWaterMark = (int) (request.getRowsLimit() * WATERMARK_PERCENTAGE); - this.operationTimeouts = operationTimeouts; + this.attemptTimeout = attemptTimeout; + this.operationTimeout = operationTimeout; } @Override @@ -404,7 +427,7 @@ public void onComplete() { future.set(new UpstreamResults(rows, nextNextRequest)); } }, - createScanCallContext(operationTimeouts)); + createScanCallContext(attemptTimeout, operationTimeout)); return future; } @@ -469,16 +492,18 @@ public Row getCurrentRow() throws NoSuchElementException { @VisibleForTesting static class BigtableWriterImpl implements Writer { private BigtableDataClient client; - private BigtableDataSettings settings; private Batcher bulkMutation; + private String projectId; + private String instanceId; private String tableId; - private GrpcCallContext context = GrpcCallContext.createDefault(); - BigtableWriterImpl(BigtableDataClient client, BigtableDataSettings settings, String tableId) { + BigtableWriterImpl( + BigtableDataClient client, String projectId, String instanceId, String tableId) { this.client = client; - this.settings = settings; + this.projectId = projectId; + this.instanceId = instanceId; this.tableId = tableId; - this.bulkMutation = client.newBulkMutationBatcher(tableId, context); + this.bulkMutation = client.newBulkMutationBatcher(tableId); } @Override @@ -532,14 +557,12 @@ public CompletionStage writeRecord(KV getSampleRowKeys(BigtableSource source) throws IOExceptio } } + @Override + public String getProjectId() { + return config.getProjectId().get(); + } + + @Override + public String getInstanceId() { + return config.getInstanceId().get(); + } + + @Override + public Credentials getCredentials() { + return config.getCredentials(); + } + + @Override + public String getEmulatorHost() { + return config.getEmulatorHost(); + } + @VisibleForTesting public static ServiceCallMetric createCallMetric( String projectId, String instanceId, String tableId) { @@ -690,6 +768,64 @@ public int compareTo(@Nonnull StartPoint o) { } } + private BigtableDataSettings configureWriteSettings( + BigtableDataSettings.Builder settings, BigtableWriteOptions writeOptions) { + BigtableBatchingCallSettings.Builder callSettings = + settings.stubSettings().bulkMutateRowsSettings(); + RetrySettings.Builder retrySettings = callSettings.getRetrySettings().toBuilder(); + BatchingSettings.Builder batchingSettings = callSettings.getBatchingSettings().toBuilder(); + if (writeOptions.getAttemptTimeout() != null) { + retrySettings.setInitialRpcTimeout(Duration.ofMillis(writeOptions.getAttemptTimeout())); + + if (writeOptions.getOperationTimeout() == null) { + retrySettings.setTotalTimeout( + Duration.ofMillis( + Math.max( + retrySettings.getTotalTimeout().toMillis(), writeOptions.getAttemptTimeout()))); + } + } + + if (writeOptions.getOperationTimeout() != null) { + retrySettings.setTotalTimeout(Duration.ofMillis(writeOptions.getOperationTimeout())); + } + + if (writeOptions.getRetryInitialDelay() != null) { + retrySettings.setInitialRetryDelay(Duration.ofMillis(writeOptions.getRetryInitialDelay())); + } + + if (writeOptions.getRetryDelayMultiplier() != null) { + retrySettings.setRetryDelayMultiplier(writeOptions.getRetryDelayMultiplier()); + } + + if (writeOptions.getBatchElements() != null) { + batchingSettings.setElementCountThreshold(writeOptions.getBatchElements()); + } + + if (writeOptions.getBatchBytes() != null) { + batchingSettings.setRequestByteThreshold(writeOptions.getBatchBytes()); + } + + if (writeOptions.getMaxRequests() != null) { + BatchingSettings tmpSettings = batchingSettings.build(); + batchingSettings.setFlowControlSettings( + callSettings + .getBatchingSettings() + .getFlowControlSettings() + .toBuilder() + .setMaxOutstandingRequestBytes( + tmpSettings.getElementCountThreshold() * writeOptions.getMaxRequests()) + .setMaxOutstandingRequestBytes( + tmpSettings.getRequestByteThreshold() * writeOptions.getMaxRequests()) + .build()); + } + + callSettings + .setBatchingSettings(batchingSettings.build()) + .setRetrySettings(retrySettings.build()); + + return settings.build(); + } + /** Helper class to ease comparison of RowRange endpoints. */ private static final class EndPoint implements Comparable { private final ByteString value; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteOptions.java new file mode 100644 index 000000000000..eece03ef1682 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteOptions.java @@ -0,0 +1,127 @@ +/* + * 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.bigtable; + +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.value.AutoValue; +import java.io.Serializable; +import org.apache.beam.sdk.options.ValueProvider; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** Configuration for write to Bigtable. */ +@AutoValue +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +abstract class BigtableWriteOptions implements Serializable { + + /** Returns the table id. */ + abstract @Nullable ValueProvider getTableId(); + + /** Returns the attempt timeout for writes. */ + abstract @Nullable Long getAttemptTimeout(); + + /** Returns the operation timeout for writes. */ + abstract @Nullable Long getOperationTimeout(); + + /** Returns the retry delay. */ + abstract @Nullable Long getRetryInitialDelay(); + + /** Returns the retry delay multiplier. */ + abstract @Nullable Double getRetryDelayMultiplier(); + + /** Returns the number of elements of a batch. */ + abstract @Nullable Long getBatchElements(); + + /** Returns the number of bytes of a batch. */ + abstract @Nullable Long getBatchBytes(); + + /** Returns the max number of concurrent requests allowed. */ + abstract @Nullable Long getMaxRequests(); + + abstract Builder toBuilder(); + + static Builder builder() { + return new AutoValue_BigtableWriteOptions.Builder(); + } + + @AutoValue.Builder + abstract static class Builder { + + abstract Builder setTableId(ValueProvider tableId); + + abstract Builder setAttemptTimeout(long timeout); + + abstract Builder setOperationTimeout(long timeout); + + abstract Builder setRetryInitialDelay(long delay); + + abstract Builder setRetryDelayMultiplier(double multiplier); + + abstract Builder setBatchElements(long size); + + abstract Builder setBatchBytes(long bytes); + + abstract Builder setMaxRequests(long count); + + abstract BigtableWriteOptions build(); + } + + boolean isDataAccessible() { + return getTableId() != null && getTableId().isAccessible(); + } + + void populateDisplayData(DisplayData.Builder builder) { + builder + .addIfNotNull(DisplayData.item("tableId", getTableId()).withLabel("Bigtable Table Id")) + .addIfNotNull( + DisplayData.item("attemptTimeout", getAttemptTimeout()) + .withLabel("Write Attempt Timeout")) + .addIfNotNull( + DisplayData.item("operationTimeout", getOperationTimeout()) + .withLabel("Write Operation Timeout")) + .addIfNotNull( + DisplayData.item("retryInitialDelay", getRetryInitialDelay()) + .withLabel("Write retry initial delay")) + .addIfNotNull( + DisplayData.item("retryDelayMultiplier", getRetryDelayMultiplier()) + .withLabel("Write retry delay multiplier")) + .addIfNotNull( + DisplayData.item("batchELements", getBatchElements()) + .withLabel("Write batch element count")) + .addIfNotNull( + DisplayData.item("batchBytes", getBatchBytes()).withLabel("Write batch byte size")) + .addIfNotNull( + DisplayData.item("maxRequests", getMaxRequests()) + .withLabel("Write max concurrent requests")); + } + + void validate() { + checkArgument( + getTableId() != null && (!getTableId().isAccessible() || !getTableId().get().isEmpty()), + "Could not obtain Bigtable table id"); + + if (getAttemptTimeout() != null && getOperationTimeout() != null) { + checkArgument( + getAttemptTimeout() <= getOperationTimeout(), + "attempt timeout can't be greater than operation timeout"); + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTest.java index 3636aaa0d71a..6887c4e96f95 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTest.java @@ -27,8 +27,11 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import com.google.auth.Credentials; import com.google.cloud.bigtable.config.BigtableOptions; import com.google.cloud.bigtable.config.BulkOptions; +import com.google.cloud.bigtable.config.CredentialOptions; +import org.apache.beam.sdk.extensions.gcp.auth.TestCredential; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.ValueProvider; @@ -65,8 +68,6 @@ public boolean isAccessible() { static final ValueProvider INSTANCE_ID = ValueProvider.StaticValueProvider.of("instance_id"); - static final ValueProvider TABLE_ID = ValueProvider.StaticValueProvider.of("table"); - static final SerializableFunction CONFIGURATOR = (SerializableFunction) input -> input; @@ -97,14 +98,6 @@ public void testWithInstanceId() { config.withInstanceId(null); } - @Test - public void testWithTableId() { - assertEquals(TABLE_ID.get(), config.withTableId(TABLE_ID).getTableId().get()); - - thrown.expect(IllegalArgumentException.class); - config.withTableId(null); - } - @Test public void testWithBigtableOptionsConfigurator() { assertEquals( @@ -130,12 +123,12 @@ public void testWithBigtableService() { @Test public void testValidate() { - config.withProjectId(PROJECT_ID).withInstanceId(INSTANCE_ID).withTableId(TABLE_ID).validate(); + config.withProjectId(PROJECT_ID).withInstanceId(INSTANCE_ID).validate(); } @Test public void testValidateFailsWithoutProjectId() { - config.withInstanceId(INSTANCE_ID).withTableId(TABLE_ID); + config.withInstanceId(INSTANCE_ID); thrown.expect(IllegalArgumentException.class); config.validate(); @@ -143,7 +136,7 @@ public void testValidateFailsWithoutProjectId() { @Test public void testValidateFailsWithoutInstanceId() { - config.withProjectId(PROJECT_ID).withTableId(TABLE_ID); + config.withProjectId(PROJECT_ID); thrown.expect(IllegalArgumentException.class); config.validate(); @@ -161,8 +154,7 @@ public void testValidateFailsWithoutTableId() { public void testPopulateDisplayData() { DisplayData displayData = DisplayData.from( - config.withProjectId(PROJECT_ID).withInstanceId(INSTANCE_ID).withTableId(TABLE_ID) - ::populateDisplayData); + config.withProjectId(PROJECT_ID).withInstanceId(INSTANCE_ID)::populateDisplayData); assertThat( displayData, @@ -177,11 +169,6 @@ public void testPopulateDisplayData() { hasKey("instanceId"), hasLabel("Bigtable Instance Id"), hasValue(INSTANCE_ID.get())))); - - assertThat( - displayData, - hasDisplayItem( - allOf(hasKey("tableId"), hasLabel("Bigtable Table Id"), hasValue(TABLE_ID.get())))); } @Test @@ -195,45 +182,34 @@ public void testGetBigtableServiceWithConfigurator() { (SerializableFunction) input -> input - .setInstanceId(INSTANCE_ID.get() + INSTANCE_ID.get()) - .setProjectId(PROJECT_ID.get() + PROJECT_ID.get()) + .setInstanceId(INSTANCE_ID.get()) + .setProjectId(PROJECT_ID.get()) .setBulkOptions(new BulkOptions.Builder().setUseBulkApi(true).build()); BigtableService service = config - .withProjectId(PROJECT_ID) - .withInstanceId(INSTANCE_ID) .withBigtableOptionsConfigurator(configurator) .getBigtableService(PipelineOptionsFactory.as(GcpOptions.class)); - assertEquals(PROJECT_ID.get(), service.getBigtableOptions().getProjectId()); - assertEquals(INSTANCE_ID.get(), service.getBigtableOptions().getInstanceId()); - assertTrue(service.getBigtableOptions().getBulkOptions().useBulkApi()); + assertEquals(PROJECT_ID.get(), service.getProjectId()); + assertEquals(INSTANCE_ID.get(), service.getInstanceId()); } @Test public void testIsDataAccessible() { + assertTrue(config.withProjectId(PROJECT_ID).withInstanceId(INSTANCE_ID).isDataAccessible()); assertTrue( config - .withTableId(TABLE_ID) - .withProjectId(PROJECT_ID) - .withInstanceId(INSTANCE_ID) - .isDataAccessible()); - assertTrue( - config - .withTableId(TABLE_ID) .withProjectId(PROJECT_ID) .withBigtableOptions(new BigtableOptions.Builder().setInstanceId("instance_id").build()) .isDataAccessible()); assertTrue( config - .withTableId(TABLE_ID) .withInstanceId(INSTANCE_ID) .withBigtableOptions(new BigtableOptions.Builder().setProjectId("project_id").build()) .isDataAccessible()); assertTrue( config - .withTableId(TABLE_ID) .withBigtableOptions( new BigtableOptions.Builder() .setProjectId("project_id") @@ -242,22 +218,29 @@ public void testIsDataAccessible() { .isDataAccessible()); assertFalse( - config - .withTableId(NOT_ACCESSIBLE_VALUE) - .withProjectId(PROJECT_ID) - .withInstanceId(INSTANCE_ID) - .isDataAccessible()); - assertFalse( - config - .withTableId(TABLE_ID) - .withProjectId(NOT_ACCESSIBLE_VALUE) - .withInstanceId(INSTANCE_ID) - .isDataAccessible()); + config.withProjectId(NOT_ACCESSIBLE_VALUE).withInstanceId(INSTANCE_ID).isDataAccessible()); assertFalse( + config.withProjectId(PROJECT_ID).withInstanceId(NOT_ACCESSIBLE_VALUE).isDataAccessible()); + } + + @Test + public void testBigtableOptionsAreTranslated() { + BigtableOptions.Builder optionsToTest = BigtableOptions.builder(); + + Credentials credentials = new TestCredential(); + optionsToTest + .enableEmulator("localhost", 1234) + .setCredentialOptions(CredentialOptions.credential(credentials)); + + BigtableService service = config - .withTableId(TABLE_ID) .withProjectId(PROJECT_ID) - .withInstanceId(NOT_ACCESSIBLE_VALUE) - .isDataAccessible()); + .withInstanceId(INSTANCE_ID) + .withBigtableOptions(optionsToTest.build()) + .withValidate(true) + .getBigtableService(PipelineOptionsFactory.as(GcpOptions.class)); + + assertEquals(credentials, service.getCredentials()); + assertEquals("localhost:1234", service.getEmulatorHost()); } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java index 027268a1bdb0..79dc5be268ec 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java @@ -40,6 +40,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +import com.google.auth.Credentials; import com.google.auto.value.AutoValue; import com.google.bigtable.v2.Cell; import com.google.bigtable.v2.Column; @@ -52,7 +53,6 @@ import com.google.cloud.bigtable.config.BigtableOptions; import com.google.cloud.bigtable.config.BulkOptions; import com.google.cloud.bigtable.config.CredentialOptions; -import com.google.cloud.bigtable.config.CredentialOptions.CredentialType; import com.google.cloud.bigtable.config.RetryOptions; import com.google.cloud.bigtable.data.v2.models.KeyOffset; import com.google.protobuf.ByteString; @@ -308,8 +308,11 @@ public void testReadWithReaderStartFailed() throws IOException { makeTableData(failureService, table, numRows); BigtableSource source = new BigtableSource( - failureConfig.withTableId(StaticValueProvider.of(table)), - BigtableReadOptions.builder().setKeyRanges(ALL_KEY_RANGE).build(), + failureConfig, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(ALL_KEY_RANGE) + .build(), null); BoundedReader reader = source.createReader(TestPipeline.testingPipelineOptions()); @@ -330,8 +333,11 @@ public void testReadWithReaderAdvanceFailed() throws IOException { makeTableData(failureService, table, numRows); BigtableSource source = new BigtableSource( - failureConfig.withTableId(StaticValueProvider.of(table)), - BigtableReadOptions.builder().setKeyRanges(ALL_KEY_RANGE).build(), + failureConfig, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(ALL_KEY_RANGE) + .build(), null); BoundedReader reader = source.createReader(TestPipeline.testingPipelineOptions()); @@ -350,7 +356,7 @@ public void testWriteBuildsCorrectly() { .withTableId("table") .withInstanceId("instance") .withProjectId("project"); - assertEquals("table", write.getBigtableConfig().getTableId().get()); + assertEquals("table", write.getBigtableWriteOptions().getTableId().get()); assertEquals("options_project", write.getBigtableOptions().getProjectId()); assertEquals("options_instance", write.getBigtableOptions().getInstanceId()); assertEquals("instance", write.getBigtableConfig().getInstanceId().get()); @@ -426,32 +432,28 @@ private static KV> makeBadWrite(String key) { /** Tests that credentials are used from PipelineOptions if not supplied by BigtableOptions. */ @Test - public void testUsePipelineOptionsCredentialsIfNotSpecifiedInBigtableOptions() throws Exception { - BigtableOptions options = - BIGTABLE_OPTIONS - .toBuilder() - .setCredentialOptions(CredentialOptions.defaultCredentials()) - .build(); + public void testUsePipelineOptionsCredentialsIfNotSpecifiedInBigtableOptions() { GcpOptions pipelineOptions = PipelineOptionsFactory.as(GcpOptions.class); - pipelineOptions.setGcpCredential(new TestCredential()); + Credentials credentials = new TestCredential(); + pipelineOptions.setGcpCredential(credentials); BigtableService readService = BigtableIO.read() - .withBigtableOptions(options) + .withProjectId("project") + .withInstanceId("instance") .withTableId("TEST-TABLE") .getBigtableConfig() .getBigtableService(pipelineOptions); BigtableService writeService = BigtableIO.write() - .withBigtableOptions(options) + .withProjectId("project") + .withInstanceId("instance") + .withProjectId("TEST-PROJECT") + .withInstanceId("TEST-INSTANCE") .withTableId("TEST-TABLE") .getBigtableConfig() .getBigtableService(pipelineOptions); - assertEquals( - CredentialType.SuppliedCredentials, - readService.getBigtableOptions().getCredentialOptions().getCredentialType()); - assertEquals( - CredentialType.SuppliedCredentials, - writeService.getBigtableOptions().getCredentialOptions().getCredentialType()); + assertEquals(credentials, readService.getCredentials()); + assertEquals(credentials, writeService.getCredentials()); } /** Tests that credentials are not used from PipelineOptions if supplied by BigtableOptions. */ @@ -476,12 +478,8 @@ public void testDontUsePipelineOptionsCredentialsIfSpecifiedInBigtableOptions() .withTableId("TEST-TABLE") .getBigtableConfig() .getBigtableService(pipelineOptions); - assertEquals( - CredentialType.None, - readService.getBigtableOptions().getCredentialOptions().getCredentialType()); - assertEquals( - CredentialType.None, - writeService.getBigtableOptions().getCredentialOptions().getCredentialType()); + assertEquals(null, readService.getCredentials()); + assertEquals(null, writeService.getCredentials()); } /** Tests that when reading from a non-existent table, the read fails. */ @@ -723,8 +721,9 @@ public void testReadingSplitAtFractionExhaustive() throws Exception { BigtableSource source = new BigtableSource( - config.withTableId(StaticValueProvider.of(table)), + config, BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) .setKeyRanges( StaticValueProvider.of(Collections.singletonList(service.getTableRange(table)))) .build(), @@ -744,8 +743,9 @@ public void testReadingSplitAtFraction() throws Exception { BigtableSource source = new BigtableSource( - config.withTableId(StaticValueProvider.of(table)), + config, BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) .setKeyRanges( StaticValueProvider.of(Collections.singletonList(service.getTableRange(table)))) .build(), @@ -780,8 +780,11 @@ public void testReadingWithSplits() throws Exception { // Generate source and split it. BigtableSource source = new BigtableSource( - config.withTableId(StaticValueProvider.of(table)), - BigtableReadOptions.builder().setKeyRanges(ALL_KEY_RANGE).build(), + config, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(ALL_KEY_RANGE) + .build(), null /*size*/); List splits = source.split(numRows * bytesPerRow / numSamples, null /* options */); @@ -810,8 +813,11 @@ public void testReadingWithSplitFailed() throws Exception { // Generate source and split it. BigtableSource source = new BigtableSource( - failureConfig.withTableId(StaticValueProvider.of(table)), - BigtableReadOptions.builder().setKeyRanges(ALL_KEY_RANGE).build(), + failureConfig, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(ALL_KEY_RANGE) + .build(), null /*size*/); thrown.expect(RuntimeException.class); @@ -876,8 +882,11 @@ public void testReduceSplitsWithSomeNonAdjacentRanges() throws Exception { // Generate source and split it. BigtableSource source = new BigtableSource( - config.withTableId(StaticValueProvider.of(table)), - BigtableReadOptions.builder().setKeyRanges(StaticValueProvider.of(keyRanges)).build(), + config, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(StaticValueProvider.of(keyRanges)) + .build(), null /*size*/); List splits = new ArrayList<>(); @@ -926,8 +935,11 @@ public void testReduceSplitsWithAllNonAdjacentRange() throws Exception { // Generate source and split it. BigtableSource source = new BigtableSource( - config.withTableId(StaticValueProvider.of(table)), - BigtableReadOptions.builder().setKeyRanges(StaticValueProvider.of(keyRanges)).build(), + config, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(StaticValueProvider.of(keyRanges)) + .build(), null /*size*/); List splits = new ArrayList<>(); @@ -967,8 +979,11 @@ public void tesReduceSplitsWithAdjacentRanges() throws Exception { // Generate source and split it. BigtableSource source = new BigtableSource( - config.withTableId(StaticValueProvider.of(table)), - BigtableReadOptions.builder().setKeyRanges(ALL_KEY_RANGE).build(), + config, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(ALL_KEY_RANGE) + .build(), null /*size*/); List splits = new ArrayList<>(); List keyRanges = @@ -1038,13 +1053,17 @@ public void testReadingWithSplitsWithSeveralKeyRanges() throws Exception { // Generate source and split it. BigtableSource source = new BigtableSource( - config.withTableId(StaticValueProvider.of(table)), - BigtableReadOptions.builder().setKeyRanges(StaticValueProvider.of(keyRanges)).build(), + config, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(StaticValueProvider.of(keyRanges)) + .build(), null /*size*/); BigtableSource referenceSource = new BigtableSource( - config.withTableId(StaticValueProvider.of(table)), + config, BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) .setKeyRanges( StaticValueProvider.of(Collections.singletonList(service.getTableRange(table)))) .build(), @@ -1073,8 +1092,11 @@ public void testReadingWithSubSplits() throws Exception { // Generate source and split it. BigtableSource source = new BigtableSource( - config.withTableId(StaticValueProvider.of(table)), - BigtableReadOptions.builder().setKeyRanges(ALL_KEY_RANGE).build(), + config, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(ALL_KEY_RANGE) + .build(), null /*size*/); List splits = source.split(numRows * bytesPerRow / numSplits, null); @@ -1115,13 +1137,17 @@ public void testReadingWithSubSplitsWithSeveralKeyRanges() throws Exception { // Generate source and split it. BigtableSource source = new BigtableSource( - config.withTableId(StaticValueProvider.of(table)), - BigtableReadOptions.builder().setKeyRanges(StaticValueProvider.of(keyRanges)).build(), + config, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(StaticValueProvider.of(keyRanges)) + .build(), null /*size*/); BigtableSource referenceSource = new BigtableSource( - config.withTableId(StaticValueProvider.of(table)), + config, BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) .setKeyRanges( StaticValueProvider.of(ImmutableList.of(service.getTableRange(table)))) .build(), @@ -1151,8 +1177,9 @@ public void testReadingWithFilterAndSubSplits() throws Exception { RowFilter.newBuilder().setRowKeyRegexFilter(ByteString.copyFromUtf8(".*17.*")).build(); BigtableSource source = new BigtableSource( - config.withTableId(StaticValueProvider.of(table)), + config, BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) .setRowFilter(StaticValueProvider.of(filter)) .setKeyRanges(ALL_KEY_RANGE) .build(), @@ -1492,8 +1519,11 @@ public void testGetSplitPointsConsumed() throws Exception { BigtableSource source = new BigtableSource( - config.withTableId(StaticValueProvider.of(table)), - BigtableReadOptions.builder().setKeyRanges(ALL_KEY_RANGE).build(), + config, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(ALL_KEY_RANGE) + .build(), null); BoundedReader reader = source.createReader(TestPipeline.testingPipelineOptions()); @@ -1606,11 +1636,6 @@ private static class FakeBigtableService implements BigtableService { private final Map> tables = new HashMap<>(); private final Map> sampleRowKeys = new HashMap<>(); - @Override - public BigtableOptions getBigtableOptions() { - return null; - } - public @Nullable SortedMap getTable(String tableId) { return tables.get(tableId); } @@ -1640,7 +1665,7 @@ public FakeBigtableReader createReader(BigtableSource source) { } @Override - public FakeBigtableWriter openForWriting(String tableId) { + public FakeBigtableWriter openForWriting(String tableId, BigtableWriteOptions writeOptions) { return new FakeBigtableWriter(tableId); } @@ -1651,6 +1676,26 @@ public List getSampleRowKeys(BigtableSource source) { return samples; } + @Override + public String getProjectId() { + return null; + } + + @Override + public String getInstanceId() { + return null; + } + + @Override + public Credentials getCredentials() { + return null; + } + + @Override + public String getEmulatorHost() { + return null; + } + /** Sets up the sample row keys for the specified table. */ void setupSampleRowKeys(String tableId, int numSamples, long bytesPerRow) { verifyTableExists(tableId); @@ -1690,7 +1735,7 @@ public FakeBigtableReader createReader(BigtableSource source) { } @Override - public FailureBigtableWriter openForWriting(String tableId) { + public FailureBigtableWriter openForWriting(String tableId, BigtableWriteOptions writeOptions) { return new FailureBigtableWriter(tableId, this, failureOptions); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java index a7e5d24a90a2..b42ec8b9c3b1 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java @@ -17,8 +17,6 @@ */ package org.apache.beam.sdk.io.gcp.bigtable; -import static org.apache.beam.sdk.io.gcp.bigtable.BigtableHBaseVeneeringSettings.BigtableIOOperationTimeouts; -import static org.apache.beam.sdk.io.gcp.bigtable.BigtableHBaseVeneeringSettings.OperationTimeouts.EMPTY; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; @@ -31,6 +29,7 @@ import com.google.api.core.ApiFuture; import com.google.api.core.SettableApiFuture; import com.google.api.gax.batching.Batcher; +import com.google.api.gax.retrying.RetrySettings; import com.google.api.gax.rpc.ApiCallContext; import com.google.api.gax.rpc.ResponseObserver; import com.google.api.gax.rpc.ServerStream; @@ -122,9 +121,6 @@ public class BigtableServiceImplTest { private static AtomicBoolean cancelled = new AtomicBoolean(false); - private BigtableIOOperationTimeouts operationTimeouts = - new BigtableIOOperationTimeouts(false, EMPTY, EMPTY, EMPTY); - @Before public void setup() { MockitoAnnotations.initMocks(this); @@ -174,9 +170,18 @@ public void testRead() throws IOException { // Set up client to return callable when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); + RetrySettings retrySettings = + bigtableDataSettings.getStubSettings().bulkReadRowsSettings().getRetrySettings(); BigtableService.Reader underTest = new BigtableServiceImpl.BigtableReaderImpl( - mockBigtableDataClient, bigtableDataSettings, mockBigtableSource, operationTimeouts); + mockBigtableDataClient, + bigtableDataSettings.getProjectId(), + bigtableDataSettings.getInstanceId(), + mockBigtableSource.getTableId().get(), + mockBigtableSource.getRanges(), + null, + retrySettings.getInitialRpcTimeout(), + retrySettings.getTotalTimeout()); underTest.start(); Assert.assertEquals(expectedRow, underTest.getCurrentRow()); @@ -223,18 +228,23 @@ public Void answer(InvocationOnMock invocation) throws Throwable { mockStub.createReadRowsCallable(new BigtableServiceImpl.BeamRowAdapter()); // Set up client to return callable when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); + when(mockBigtableSource.getTableId()).thenReturn(StaticValueProvider.of(TABLE_ID)); + RetrySettings retrySettings = + bigtableDataSettings.getStubSettings().bulkReadRowsSettings().getRetrySettings(); BigtableService.Reader underTest = new BigtableServiceImpl.BigtableSegmentReaderImpl( mockBigtableDataClient, - bigtableDataSettings, - TABLE_ID, + bigtableDataSettings.getProjectId(), + bigtableDataSettings.getInstanceId(), + mockBigtableSource.getTableId().get(), ranges.build(), + RowFilter.getDefaultInstance(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, - RowFilter.getDefaultInstance(), - mockCallMetric, - operationTimeouts); + retrySettings.getInitialRpcTimeout(), + retrySettings.getTotalTimeout(), + mockCallMetric); underTest.start(); Assert.assertEquals(expectedRow, underTest.getCurrentRow()); @@ -278,17 +288,21 @@ public void testReadSingleRangeAboveSegmentLimit() throws IOException { // Set up client to return callable when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); + RetrySettings retrySettings = + bigtableDataSettings.getStubSettings().bulkReadRowsSettings().getRetrySettings(); BigtableService.Reader underTest = new BigtableServiceImpl.BigtableSegmentReaderImpl( mockBigtableDataClient, - bigtableDataSettings, + bigtableDataSettings.getProjectId(), + bigtableDataSettings.getInstanceId(), TABLE_ID, ranges.build(), + RowFilter.getDefaultInstance(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, - RowFilter.getDefaultInstance(), - mockCallMetric, - operationTimeouts); + retrySettings.getInitialRpcTimeout(), + retrySettings.getTotalTimeout(), + mockCallMetric); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); @@ -345,17 +359,21 @@ public void testReadMultipleRanges() throws IOException { // Set up client to return callable when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); + RetrySettings retrySettings = + bigtableDataSettings.getStubSettings().bulkReadRowsSettings().getRetrySettings(); BigtableService.Reader underTest = new BigtableServiceImpl.BigtableSegmentReaderImpl( mockBigtableDataClient, - bigtableDataSettings, + bigtableDataSettings.getProjectId(), + bigtableDataSettings.getInstanceId(), TABLE_ID, ranges.build(), + RowFilter.getDefaultInstance(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, - RowFilter.getDefaultInstance(), - mockCallMetric, - operationTimeouts); + retrySettings.getInitialRpcTimeout(), + retrySettings.getTotalTimeout(), + mockCallMetric); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); @@ -413,17 +431,21 @@ public void testReadMultipleRangesOverlappingKeys() throws IOException { // Set up client to return callable when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); + RetrySettings retrySettings = + bigtableDataSettings.getStubSettings().bulkReadRowsSettings().getRetrySettings(); BigtableService.Reader underTest = new BigtableServiceImpl.BigtableSegmentReaderImpl( mockBigtableDataClient, - bigtableDataSettings, + bigtableDataSettings.getProjectId(), + bigtableDataSettings.getInstanceId(), TABLE_ID, ranges.build(), + RowFilter.getDefaultInstance(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, - RowFilter.getDefaultInstance(), - mockCallMetric, - operationTimeouts); + retrySettings.getInitialRpcTimeout(), + retrySettings.getTotalTimeout(), + mockCallMetric); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); @@ -466,17 +488,21 @@ public void testReadFullTableScan() throws IOException { // Set up client to return callable when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); + RetrySettings retrySettings = + bigtableDataSettings.getStubSettings().bulkReadRowsSettings().getRetrySettings(); BigtableService.Reader underTest = new BigtableServiceImpl.BigtableSegmentReaderImpl( mockBigtableDataClient, - bigtableDataSettings, + bigtableDataSettings.getProjectId(), + bigtableDataSettings.getInstanceId(), TABLE_ID, RowSet.getDefaultInstance(), + RowFilter.getDefaultInstance(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, - RowFilter.getDefaultInstance(), - mockCallMetric, - operationTimeouts); + retrySettings.getInitialRpcTimeout(), + retrySettings.getTotalTimeout(), + mockCallMetric); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); @@ -534,17 +560,21 @@ public void testReadFillBuffer() throws IOException { // Set up client to return callable when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); + RetrySettings retrySettings = + bigtableDataSettings.getStubSettings().bulkReadRowsSettings().getRetrySettings(); BigtableService.Reader underTest = new BigtableServiceImpl.BigtableSegmentReaderImpl( mockBigtableDataClient, - bigtableDataSettings, + bigtableDataSettings.getProjectId(), + bigtableDataSettings.getInstanceId(), TABLE_ID, ranges.build(), + RowFilter.getDefaultInstance(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, - RowFilter.getDefaultInstance(), - mockCallMetric, - operationTimeouts); + retrySettings.getInitialRpcTimeout(), + retrySettings.getTotalTimeout(), + mockCallMetric); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); @@ -620,17 +650,21 @@ public Void answer(InvocationOnMock invocation) throws Throwable { // Set up client to return callable when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); + RetrySettings retrySettings = + bigtableDataSettings.getStubSettings().bulkReadRowsSettings().getRetrySettings(); BigtableService.Reader underTest = new BigtableServiceImpl.BigtableSegmentReaderImpl( mockBigtableDataClient, - bigtableDataSettings, + bigtableDataSettings.getProjectId(), + bigtableDataSettings.getInstanceId(), TABLE_ID, RowSet.newBuilder().addRowRanges(mockRowRange).build(), + RowFilter.getDefaultInstance(), SEGMENT_SIZE, segmentByteLimit, - RowFilter.getDefaultInstance(), - mockCallMetric, - operationTimeouts); + retrySettings.getInitialRpcTimeout(), + retrySettings.getTotalTimeout(), + mockCallMetric); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); @@ -683,17 +717,21 @@ public void run() { mockStub.createReadRowsCallable(new BigtableServiceImpl.BeamRowAdapter()); // Set up client to return callable when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); + RetrySettings retrySettings = + bigtableDataSettings.getStubSettings().bulkReadRowsSettings().getRetrySettings(); BigtableService.Reader underTest = new BigtableServiceImpl.BigtableSegmentReaderImpl( mockBigtableDataClient, - bigtableDataSettings, + bigtableDataSettings.getProjectId(), + bigtableDataSettings.getInstanceId(), TABLE_ID, ranges.build(), + RowFilter.getDefaultInstance(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, - RowFilter.getDefaultInstance(), - mockCallMetric, - operationTimeouts); + retrySettings.getInitialRpcTimeout(), + retrySettings.getTotalTimeout(), + mockCallMetric); IOException returnedError = null; try { @@ -717,14 +755,17 @@ public void run() { */ @Test public void testWrite() throws IOException, InterruptedException { - doReturn(mockBatcher).when(mockBigtableDataClient).newBulkMutationBatcher(any(), any()); + doReturn(mockBatcher).when(mockBigtableDataClient).newBulkMutationBatcher(any()); ArgumentCaptor captor = ArgumentCaptor.forClass(RowMutationEntry.class); ApiFuture fakeResponse = SettableApiFuture.create(); when(mockBatcher.add(any(RowMutationEntry.class))).thenReturn(fakeResponse); BigtableService.Writer underTest = new BigtableServiceImpl.BigtableWriterImpl( - mockBigtableDataClient, bigtableDataSettings, TABLE_ID); + mockBigtableDataClient, + bigtableDataSettings.getProjectId(), + bigtableDataSettings.getInstanceId(), + TABLE_ID); ByteString key = ByteString.copyFromUtf8("key"); Mutation mutation = diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java index eeeba49a008e..77bae9f054cd 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java @@ -24,7 +24,6 @@ import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; import com.google.cloud.bigtable.admin.v2.models.Table; -import com.google.cloud.bigtable.config.BigtableOptions; import com.google.cloud.bigtable.data.v2.BigtableDataClient; import com.google.cloud.bigtable.data.v2.models.Query; import com.google.cloud.bigtable.data.v2.models.Row; @@ -39,6 +38,7 @@ import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; @@ -61,8 +61,8 @@ public class BigtableWriteIT implements Serializable { private static final String COLUMN_FAMILY_NAME = "cf"; private static BigtableTestOptions options; - private static BigtableHBaseVeneeringSettings veneerSettings; - private BigtableOptions bigtableOptions; + private static BigtableConfigToVeneerSettings veneerSettings; + private BigtableConfig bigtableConfig; private static BigtableDataClient client; private static BigtableTableAdminClient tableAdminClient; private final String tableId = @@ -75,14 +75,14 @@ public void setup() throws Exception { options = TestPipeline.testingPipelineOptions().as(BigtableTestOptions.class); project = options.as(GcpOptions.class).getProject(); - bigtableOptions = - new BigtableOptions.Builder() - .setProjectId(project) - .setInstanceId(options.getInstanceId()) + bigtableConfig = + BigtableConfig.builder() + .setProjectId(ValueProvider.StaticValueProvider.of(project)) + .setInstanceId(ValueProvider.StaticValueProvider.of(options.getInstanceId())) .setUserAgent("apache-beam-test") .build(); - veneerSettings = BigtableHBaseVeneeringSettings.create(bigtableOptions); + veneerSettings = BigtableConfigToVeneerSettings.create(bigtableConfig); client = BigtableDataClient.create(veneerSettings.getDataSettings()); tableAdminClient = BigtableTableAdminClient.create(veneerSettings.getTableAdminSettings()); @@ -115,7 +115,7 @@ public void processElement(ProcessContext c) { c.output(KV.of(testData.get(index).getKey(), mutations)); } })) - .apply(BigtableIO.write().withBigtableOptions(bigtableOptions).withTableId(tableId)); + .apply(BigtableIO.write().withTableId(tableId)); p.run(); // Test number of column families and column family name equality From f284ca0d9c124735e6e9c4614f3ef6e7f85ab8fe Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Fri, 3 Feb 2023 12:16:57 -0500 Subject: [PATCH 04/23] fix client wrapper --- .../bigtable/BigtableClientWrapper.java | 39 +++++++++---------- .../bigtable/BigtableTableFlatTest.java | 1 - .../provider/bigtable/BigtableTableIT.java | 1 - .../bigtable/BigtableTableTestUtils.java | 1 - .../bigtable/BigtableTableWithRowsTest.java | 1 - .../sdk/io/gcp/bigtable/BigtableConfig.java | 2 +- 6 files changed, 20 insertions(+), 25 deletions(-) rename sdks/java/{io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp => extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider}/bigtable/BigtableClientWrapper.java (73%) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableClientWrapper.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableClientWrapper.java similarity index 73% rename from sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableClientWrapper.java rename to sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableClientWrapper.java index ecdac695dbe2..0b71b75276e6 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableClientWrapper.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableClientWrapper.java @@ -15,20 +15,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.io.gcp.bigtable; +package org.apache.beam.sdk.extensions.sql.meta.provider.bigtable; import static org.apache.beam.sdk.io.gcp.bigtable.RowUtils.byteString; import static org.apache.beam.sdk.io.gcp.bigtable.RowUtils.byteStringUtf8; +import com.google.api.gax.rpc.FixedHeaderProvider; import com.google.auth.Credentials; import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminSettings; import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; import com.google.cloud.bigtable.data.v2.BigtableDataClient; +import com.google.cloud.bigtable.data.v2.BigtableDataSettings; import com.google.cloud.bigtable.data.v2.models.RowMutation; import java.io.IOException; import java.io.Serializable; import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.options.ValueProvider; import org.checkerframework.checker.nullness.qual.Nullable; @Internal @@ -36,30 +38,27 @@ public class BigtableClientWrapper implements Serializable { private final BigtableTableAdminClient tableAdminClient; private final BigtableDataClient dataClient; - private final BigtableConfig bigtableConfig; - public BigtableClientWrapper( String project, String instanceId, @Nullable Integer emulatorPort, @Nullable Credentials gcpCredentials) throws IOException { - BigtableConfig.Builder configBuilder = - BigtableConfig.builder() - .setProjectId(ValueProvider.StaticValueProvider.of(project)) - .setInstanceId(ValueProvider.StaticValueProvider.of(instanceId)) - .setUserAgent("apache-beam-test") - .setValidate(true); - if (emulatorPort != null) { - configBuilder.setEmulatorHost("localhost:" + emulatorPort); - } - if (gcpCredentials != null) { - configBuilder.setCredentials(gcpCredentials); - } - bigtableConfig = configBuilder.build(); - BigtableConfigToVeneerSettings settings = BigtableConfigToVeneerSettings.create(bigtableConfig); - tableAdminClient = BigtableTableAdminClient.create(settings.getTableAdminSettings()); - dataClient = BigtableDataClient.create(settings.getDataSettings()); + BigtableDataSettings.Builder settings = + BigtableDataSettings.newBuilderForEmulator(emulatorPort) + .setProjectId(project) + .setInstanceId(instanceId); + + settings + .stubSettings() + .setHeaderProvider(FixedHeaderProvider.create("user-agent", "apache-beam-test")); + dataClient = BigtableDataClient.create(settings.build()); + BigtableTableAdminSettings tableSettings = + BigtableTableAdminSettings.newBuilderForEmulator(emulatorPort) + .setProjectId(project) + .setInstanceId(instanceId) + .build(); + tableAdminClient = BigtableTableAdminClient.create(tableSettings); } public void writeRow( diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableFlatTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableFlatTest.java index 101990857f3a..08082e85e0e1 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableFlatTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableFlatTest.java @@ -44,7 +44,6 @@ import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils; import org.apache.beam.sdk.extensions.sql.meta.Table; import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore; -import org.apache.beam.sdk.io.gcp.bigtable.BigtableClientWrapper; import org.apache.beam.sdk.io.gcp.bigtable.BigtableIO; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.testing.PAssert; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableIT.java index 3abad4b21bf3..d62292a3f6ab 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableIT.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableIT.java @@ -32,7 +32,6 @@ import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils; -import org.apache.beam.sdk.io.gcp.bigtable.BigtableClientWrapper; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptionsFactory; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableTestUtils.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableTestUtils.java index b812f880dc5e..ef66fb89affa 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableTestUtils.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableTestUtils.java @@ -36,7 +36,6 @@ import com.google.bigtable.v2.Column; import com.google.bigtable.v2.Family; import java.util.List; -import org.apache.beam.sdk.io.gcp.bigtable.BigtableClientWrapper; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableWithRowsTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableWithRowsTest.java index 7327a1cc8579..4b60eb13c894 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableWithRowsTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableWithRowsTest.java @@ -38,7 +38,6 @@ import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils; import org.apache.beam.sdk.extensions.sql.meta.Table; import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore; -import org.apache.beam.sdk.io.gcp.bigtable.BigtableClientWrapper; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.MapElements; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java index 63b5ec4c060d..312a87525e80 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java @@ -137,7 +137,7 @@ BigtableConfig withInstanceId(ValueProvider instanceId) { return toBuilder().setInstanceId(instanceId).build(); } - BigtableConfig withAppProfileId(@Nullable ValueProvider appProfileId) { + BigtableConfig withAppProfileId(ValueProvider appProfileId) { checkArgument(appProfileId != null, "App profile id can not be null"); return toBuilder().setAppProfileId(appProfileId).build(); } From 9721a1accc0f48a4b0ffb61d297e308b9b29a3e3 Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Fri, 10 Feb 2023 17:37:54 -0500 Subject: [PATCH 05/23] use shared clients --- .../BigtableConfigToVeneerSettings.java | 9 ------ .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 2 -- .../sdk/io/gcp/bigtable/BigtableService.java | 3 +- .../io/gcp/bigtable/BigtableServiceImpl.java | 22 +++++++------- .../changestreams/dao/DaoFactory.java | 1 - .../sdk/io/gcp/bigtable/BigtableIOTest.java | 6 ++-- .../gcp/bigtable/BigtableServiceImplTest.java | 30 ++++++++++++------- 7 files changed, 37 insertions(+), 36 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigToVeneerSettings.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigToVeneerSettings.java index ed82c08c21ee..5989733ec3af 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigToVeneerSettings.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigToVeneerSettings.java @@ -149,15 +149,6 @@ public ManagedChannelBuilder apply(ManagedChannelBuilder channelBuilder) { }); } - channelProvider.setChannelPoolSettings( - ChannelPoolSettings.builder() - .setMaxRpcsPerChannel(50) - .setMinRpcsPerChannel(1) - .setPreemptiveRefreshEnabled(true) - .setInitialChannelCount(1) - .setMinChannelCount(1) - .setMaxChannelCount(4) - .build()); stubSettings.setTransportChannelProvider(channelProvider.build()); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 640bffd98c78..5352c96aaef8 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -1117,7 +1117,6 @@ public String toString() { } } - private static class BigtableWriterFn extends DoFn>, BigtableWriteResult> { @@ -1631,7 +1630,6 @@ private static class BigtableReader extends BoundedReader { private int id; - public BigtableReader(BigtableSource source, BigtableService service) { checkArgument(source.getRanges().size() == 1, "source must have exactly one key range"); this.source = source; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java index 438b06f45022..35d1c7dae836 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java @@ -92,7 +92,8 @@ interface Reader { Reader createReader(BigtableSource source, int id) throws IOException; /** Returns a {@link Writer} that will write to the specified table. */ - Writer openForWriting(String tableId, BigtableWriteOptions writeOptions, int id) throws IOException; + Writer openForWriting(String tableId, BigtableWriteOptions writeOptions, int id) + throws IOException; /** * Returns a set of row keys sampled from the underlying table. These contain information about diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java index 9ddf670da182..54064d69a10f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java @@ -106,11 +106,11 @@ class BigtableServiceImpl implements BigtableService { // Percentage of max number of rows allowed in the buffer private static final double WATERMARK_PERCENTAGE = .1; private static final long MIN_BYTE_BUFFER_SIZE = 100 * 1024 * 1024; // 100MB - private static ConcurrentHashMap> writeClients - = new ConcurrentHashMap<>(); + private static ConcurrentHashMap> writeClients = + new ConcurrentHashMap<>(); - private static ConcurrentHashMap> readClients - = new ConcurrentHashMap<>(); + private static ConcurrentHashMap> readClients = + new ConcurrentHashMap<>(); public BigtableServiceImpl(BigtableConfig config) { try { @@ -142,10 +142,7 @@ public BigtableWriterImpl openForWriting(String tableId, BigtableWriteOptions op } clientTuple.x().getAndIncrement(); return new BigtableWriterImpl( - clientTuple.y(), - dataSettings.getProjectId(), - dataSettings.getInstanceId(), - tableId, id); + clientTuple.y(), dataSettings.getProjectId(), dataSettings.getInstanceId(), tableId, id); } @Override @@ -341,7 +338,8 @@ static BigtableSegmentReaderImpl create( maxSegmentByteSize, attemptTimeout, operationTimeout, - createCallMetric(projectId, instanceId, tableId), id); + createCallMetric(projectId, instanceId, tableId), + id); } @VisibleForTesting @@ -651,7 +649,8 @@ public Reader createReader(BigtableSource source, int id) throws IOException { source.getRowFilter(), source.getMaxBufferElementCount(), retrySettings.getInitialRpcTimeout(), - retrySettings.getTotalTimeout(), id); + retrySettings.getTotalTimeout(), + id); } else { return new BigtableReaderImpl( client.y(), @@ -661,7 +660,8 @@ public Reader createReader(BigtableSource source, int id) throws IOException { source.getRanges(), source.getRowFilter(), retrySettings.getInitialRpcTimeout(), - retrySettings.getTotalTimeout(), id); + retrySettings.getTotalTimeout(), + id); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/dao/DaoFactory.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/dao/DaoFactory.java index f69313594280..1b5001358d9b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/dao/DaoFactory.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/changestreams/dao/DaoFactory.java @@ -23,7 +23,6 @@ import java.io.Serializable; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.io.gcp.bigtable.BigtableConfig; -import org.apache.beam.sdk.options.ValueProvider; // Allows transient fields to be intialized later @SuppressWarnings("initialization.fields.uninitialized") diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java index 2616e6b77601..67fe82d7b67d 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java @@ -1665,7 +1665,8 @@ public FakeBigtableReader createReader(BigtableSource source, int id) { } @Override - public FakeBigtableWriter openForWriting(String tableId, BigtableWriteOptions writeOptions, int id) { + public FakeBigtableWriter openForWriting( + String tableId, BigtableWriteOptions writeOptions, int id) { return new FakeBigtableWriter(tableId); } @@ -1735,7 +1736,8 @@ public FakeBigtableReader createReader(BigtableSource source, int id) { } @Override - public FailureBigtableWriter openForWriting(String tableId, BigtableWriteOptions writeOptions, int id) { + public FailureBigtableWriter openForWriting( + String tableId, BigtableWriteOptions writeOptions, int id) { return new FailureBigtableWriter(tableId, this, failureOptions); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java index 7486957fc4fd..28f8d82351c0 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java @@ -181,7 +181,8 @@ public void testRead() throws IOException { mockBigtableSource.getRanges(), null, retrySettings.getInitialRpcTimeout(), - retrySettings.getTotalTimeout(), 0); + retrySettings.getTotalTimeout(), + 0); underTest.start(); Assert.assertEquals(expectedRow, underTest.getCurrentRow()); @@ -243,7 +244,8 @@ public Void answer(InvocationOnMock invocation) throws Throwable { DEFAULT_BYTE_SEGMENT_SIZE, retrySettings.getInitialRpcTimeout(), retrySettings.getTotalTimeout(), - mockCallMetric, 0); + mockCallMetric, + 0); underTest.start(); Assert.assertEquals(expectedRow, underTest.getCurrentRow()); @@ -300,7 +302,8 @@ public void testReadSingleRangeAboveSegmentLimit() throws IOException { DEFAULT_BYTE_SEGMENT_SIZE, retrySettings.getInitialRpcTimeout(), retrySettings.getTotalTimeout(), - mockCallMetric, 0); + mockCallMetric, + 0); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); @@ -370,7 +373,8 @@ public void testReadMultipleRanges() throws IOException { DEFAULT_BYTE_SEGMENT_SIZE, retrySettings.getInitialRpcTimeout(), retrySettings.getTotalTimeout(), - mockCallMetric, 0); + mockCallMetric, + 0); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); @@ -441,7 +445,8 @@ public void testReadMultipleRangesOverlappingKeys() throws IOException { DEFAULT_BYTE_SEGMENT_SIZE, retrySettings.getInitialRpcTimeout(), retrySettings.getTotalTimeout(), - mockCallMetric, 0); + mockCallMetric, + 0); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); @@ -497,7 +502,8 @@ public void testReadFullTableScan() throws IOException { DEFAULT_BYTE_SEGMENT_SIZE, retrySettings.getInitialRpcTimeout(), retrySettings.getTotalTimeout(), - mockCallMetric, 0); + mockCallMetric, + 0); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); @@ -568,7 +574,8 @@ public void testReadFillBuffer() throws IOException { DEFAULT_BYTE_SEGMENT_SIZE, retrySettings.getInitialRpcTimeout(), retrySettings.getTotalTimeout(), - mockCallMetric, 0); + mockCallMetric, + 0); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); @@ -657,7 +664,8 @@ public Void answer(InvocationOnMock invocation) throws Throwable { segmentByteLimit, retrySettings.getInitialRpcTimeout(), retrySettings.getTotalTimeout(), - mockCallMetric, 0); + mockCallMetric, + 0); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); @@ -723,7 +731,8 @@ public void run() { DEFAULT_BYTE_SEGMENT_SIZE, retrySettings.getInitialRpcTimeout(), retrySettings.getTotalTimeout(), - mockCallMetric, 0); + mockCallMetric, + 0); IOException returnedError = null; try { @@ -757,7 +766,8 @@ public void testWrite() throws IOException, InterruptedException { mockBigtableDataClient, bigtableDataSettings.getProjectId(), bigtableDataSettings.getInstanceId(), - TABLE_ID, 0); + TABLE_ID, + 0); ByteString key = ByteString.copyFromUtf8("key"); Mutation mutation = From f341ab9923f870308ad94b244d4e1d7ec900bd30 Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Fri, 10 Feb 2023 17:50:06 -0500 Subject: [PATCH 06/23] revert clientwrapper changes --- .../provider/bigtable/BigtableClientWrapper.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableClientWrapper.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableClientWrapper.java index 0b71b75276e6..fd1b8fe0a5e6 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableClientWrapper.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableClientWrapper.java @@ -20,6 +20,7 @@ import static org.apache.beam.sdk.io.gcp.bigtable.RowUtils.byteString; import static org.apache.beam.sdk.io.gcp.bigtable.RowUtils.byteStringUtf8; +import com.google.api.gax.core.FixedCredentialsProvider; import com.google.api.gax.rpc.FixedHeaderProvider; import com.google.auth.Credentials; import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; @@ -30,11 +31,9 @@ import com.google.cloud.bigtable.data.v2.models.RowMutation; import java.io.IOException; import java.io.Serializable; -import org.apache.beam.sdk.annotations.Internal; import org.checkerframework.checker.nullness.qual.Nullable; -@Internal -public class BigtableClientWrapper implements Serializable { +class BigtableClientWrapper implements Serializable { private final BigtableTableAdminClient tableAdminClient; private final BigtableDataClient dataClient; @@ -47,7 +46,8 @@ public BigtableClientWrapper( BigtableDataSettings.Builder settings = BigtableDataSettings.newBuilderForEmulator(emulatorPort) .setProjectId(project) - .setInstanceId(instanceId); + .setInstanceId(instanceId) + .setCredentialsProvider(FixedCredentialsProvider.create(gcpCredentials)); settings .stubSettings() @@ -61,7 +61,7 @@ public BigtableClientWrapper( tableAdminClient = BigtableTableAdminClient.create(tableSettings); } - public void writeRow( + void writeRow( String key, String table, String familyColumn, @@ -75,16 +75,16 @@ public void writeRow( dataClient.mutateRow(rowMutation); } - public void createTable(String tableName, String familyName) { + void createTable(String tableName, String familyName) { CreateTableRequest createTableRequest = CreateTableRequest.of(tableName).addFamily(familyName); tableAdminClient.createTable(createTableRequest); } - public void deleteTable(String tableId) { + void deleteTable(String tableId) { tableAdminClient.deleteTable(tableId); } - public void closeSession() throws IOException { + void closeSession() throws IOException { dataClient.close(); tableAdminClient.close(); } From 33ae4abbb47879b415ee74b5c5d2170359a51d8a Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Wed, 15 Feb 2023 18:11:46 -0500 Subject: [PATCH 07/23] refactor --- .../sdk/io/gcp/bigtable/BigtableConfig.java | 178 ++------- .../BigtableConfigToVeneerSettings.java | 174 --------- .../bigtable/BigtableConfigTranslator.java | 342 ++++++++++++++++++ .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 260 ++++++------- .../sdk/io/gcp/bigtable/BigtableService.java | 19 +- .../gcp/bigtable/BigtableServiceFactory.java | 183 ++++++++++ .../io/gcp/bigtable/BigtableServiceImpl.java | 294 +++------------ .../beam/sdk/io/gcp/GcpApiSurfaceTest.java | 1 + .../io/gcp/bigtable/BigtableConfigTest.java | 59 --- .../BigtableConfigTranslatorTest.java | 257 +++++++++++++ .../sdk/io/gcp/bigtable/BigtableIOTest.java | 174 ++++----- .../gcp/bigtable/BigtableServiceImplTest.java | 30 +- .../sdk/io/gcp/bigtable/BigtableWriteIT.java | 12 +- 13 files changed, 1066 insertions(+), 917 deletions(-) delete mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigToVeneerSettings.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java index c1aaf9b9d3ea..071956a0fb14 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java @@ -19,22 +19,11 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; -import com.google.api.gax.core.NoCredentialsProvider; -import com.google.auth.Credentials; -import com.google.auth.oauth2.GoogleCredentials; -import com.google.auth.oauth2.ServiceAccountJwtAccessCredentials; +import com.google.api.gax.core.CredentialsProvider; import com.google.auto.value.AutoValue; import com.google.cloud.bigtable.config.BigtableOptions; -import com.google.cloud.bigtable.config.CredentialOptions; -import java.io.FileInputStream; -import java.io.IOException; import java.io.Serializable; -import java.security.GeneralSecurityException; -import java.security.KeyStore; -import java.security.PrivateKey; import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; -import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; @@ -50,14 +39,6 @@ @Internal public abstract class BigtableConfig implements Serializable { - enum CredentialType { - DEFAULT, - P12, - SUPPLIED, - JSON, - NONE - } - /** Returns the project id being written to. */ public abstract @Nullable ValueProvider getProjectId(); @@ -82,9 +63,6 @@ enum CredentialType { /** Weather validate that table exists before writing. */ abstract boolean getValidate(); - /** {@link BigtableService} used only for testing. */ - abstract @Nullable BigtableService getBigtableService(); - /** Bigtable emulator. Used only for testing. */ abstract @Nullable String getEmulatorHost(); @@ -92,7 +70,9 @@ enum CredentialType { abstract @Nullable String getUserAgent(); /** Credentials for running the job. */ - abstract @Nullable Credentials getCredentials(); + abstract @Nullable CredentialsProvider getCredentialsProvider(); + + abstract @Nullable Integer getConfigId(); abstract Builder toBuilder(); @@ -109,22 +89,29 @@ abstract static class Builder { abstract Builder setAppProfileId(ValueProvider appProfileId); - /** @deprecated will be replaced by bigtable options configurator. */ + /** @deprecated please set the options directly in BigtableIO. */ @Deprecated abstract Builder setBigtableOptions(BigtableOptions options); abstract Builder setValidate(boolean validate); + /** @deprecated please set the options directly in BigtableIO. */ + @Deprecated abstract Builder setBigtableOptionsConfigurator( SerializableFunction optionsConfigurator); - abstract Builder setBigtableService(BigtableService bigtableService); - abstract Builder setEmulatorHost(String emulatorHost); abstract Builder setUserAgent(String userAgent); - abstract Builder setCredentials(Credentials credentials); + // TODO pass in CredentialProvider + // - defulat credentials + // - user overriding default credentials + // - no credentials (maybe ok) + abstract Builder setCredentialsProvider(CredentialsProvider credentialsProvider); + + @VisibleForTesting + abstract Builder setConfigId(Integer id); abstract BigtableConfig build(); } @@ -144,7 +131,7 @@ BigtableConfig withAppProfileId(ValueProvider appProfileId) { return toBuilder().setAppProfileId(appProfileId).build(); } - /** @deprecated please set the options directly in BigtableIO */ + /** @deprecated please set the options directly in BigtableIO. */ @Deprecated public BigtableConfig withBigtableOptions(BigtableOptions options) { checkArgument(options != null, "Bigtable options can not be null"); @@ -163,20 +150,19 @@ public BigtableConfig withValidate(boolean isEnabled) { return toBuilder().setValidate(isEnabled).build(); } - @VisibleForTesting - public BigtableConfig withBigtableService(BigtableService bigtableService) { - checkArgument(bigtableService != null, "bigtableService can not be null"); - return toBuilder().setBigtableService(bigtableService).build(); - } - @VisibleForTesting public BigtableConfig withEmulator(String emulatorHost) { checkArgument(emulatorHost != null, "emulatorHost can not be null"); return toBuilder().setEmulatorHost(emulatorHost).build(); } - BigtableConfig withCredentails(Credentials credentials) { - return toBuilder().setCredentials(credentials).build(); + BigtableConfig withCredentialsProvider(CredentialsProvider provider) { + return toBuilder().setCredentialsProvider(provider).build(); + } + + @VisibleForTesting + BigtableConfig withConfigId(int id) { + return toBuilder().setConfigId(id).build(); } void validate() { @@ -214,129 +200,11 @@ void populateDisplayData(DisplayData.Builder builder) { } } - /** - * Helper function that either returns the mock Bigtable service supplied by {@link - * #withBigtableService} or creates and returns an implementation that talks to {@code Cloud - * Bigtable}. - * - *

Also populate the credentials option from {@link GcpOptions#getGcpCredential()} if the - * default credentials are being used on {@link BigtableOptions}. - */ - @VisibleForTesting - BigtableService getBigtableService(PipelineOptions pipelineOptions) { - if (getBigtableService() != null) { - return getBigtableService(); - } - - BigtableConfig.Builder config = toBuilder(); - - if (pipelineOptions instanceof GcpOptions) { - config.setCredentials(((GcpOptions) pipelineOptions).getGcpCredential()); - } - - try { - translateBigtableOptions(config); - } catch (IOException e) { - throw new RuntimeException(e); - } - - config.setUserAgent(pipelineOptions.getUserAgent()); - - return new BigtableServiceImpl(config.build()); - } - boolean isDataAccessible() { return (getProjectId() == null || getProjectId().isAccessible()) && (getInstanceId() == null || getInstanceId().isAccessible()); } - private void translateBigtableOptions(BigtableConfig.Builder builder) throws IOException { - BigtableOptions.Builder effectiveOptionsBuilder = null; - - if (getBigtableOptions() != null) { - effectiveOptionsBuilder = getBigtableOptions().toBuilder(); - } - - if (getBigtableOptionsConfigurator() != null) { - effectiveOptionsBuilder = getBigtableOptionsConfigurator().apply(BigtableOptions.builder()); - } - - if (effectiveOptionsBuilder == null) { - return; - } - - BigtableOptions effectiveOptions = effectiveOptionsBuilder.build(); - - // Todo decided if we should implement cached channel pool - - if (effectiveOptions.getInstanceId() != null && getInstanceId() == null) { - builder.setInstanceId(ValueProvider.StaticValueProvider.of(effectiveOptions.getInstanceId())); - } - - if (effectiveOptions.getProjectId() != null && getProjectId() == null) { - builder.setProjectId(ValueProvider.StaticValueProvider.of(effectiveOptions.getProjectId())); - } - - if (!effectiveOptions.getDataHost().equals("bigtable.googleapis.com") - && getEmulatorHost() == null) { - builder.setEmulatorHost( - String.format("%s:%s", effectiveOptions.getDataHost(), effectiveOptions.getPort())); - } - - if (effectiveOptions.getCredentialOptions() != null) { - CredentialOptions credOptions = effectiveOptions.getCredentialOptions(); - switch (credOptions.getCredentialType()) { - case DefaultCredentials: - GoogleCredentials credentials = GoogleCredentials.getApplicationDefault(); - builder.setCredentials(credentials); - break; - case P12: - String keyFile = ((CredentialOptions.P12CredentialOptions) credOptions).getKeyFile(); - String serviceAccount = - ((CredentialOptions.P12CredentialOptions) credOptions).getServiceAccount(); - try { - KeyStore keyStore = KeyStore.getInstance("PKCS12"); - - try (FileInputStream fin = new FileInputStream(keyFile)) { - keyStore.load(fin, "notasecret".toCharArray()); - } - PrivateKey privateKey = - (PrivateKey) keyStore.getKey("privatekey", "notasecret".toCharArray()); - - if (privateKey == null) { - throw new IllegalStateException("private key cannot be null"); - } - builder.setCredentials( - ServiceAccountJwtAccessCredentials.newBuilder() - .setClientEmail(serviceAccount) - .setPrivateKey(privateKey) - .build()); - } catch (GeneralSecurityException exception) { - throw new RuntimeException("exception while retrieving credentials", exception); - } - break; - case SuppliedCredentials: - builder.setCredentials( - ((CredentialOptions.UserSuppliedCredentialOptions) credOptions).getCredential()); - break; - case SuppliedJson: - CredentialOptions.JsonCredentialsOptions jsonCredentialsOptions = - (CredentialOptions.JsonCredentialsOptions) credOptions; - synchronized (jsonCredentialsOptions) { - if (jsonCredentialsOptions.getCachedCredentials() == null) { - jsonCredentialsOptions.setCachedCredentails( - GoogleCredentials.fromStream(jsonCredentialsOptions.getInputStream())); - } - builder.setCredentials(jsonCredentialsOptions.getCachedCredentials()); - } - break; - case None: - builder.setCredentials(NoCredentialsProvider.create().getCredentials()); - break; - } - } - } - @Override public final String toString() { return MoreObjects.toStringHelper(BigtableConfig.class) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigToVeneerSettings.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigToVeneerSettings.java deleted file mode 100644 index 5989733ec3af..000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigToVeneerSettings.java +++ /dev/null @@ -1,174 +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.sdk.io.gcp.bigtable; - -import com.google.api.core.ApiFunction; -import com.google.api.gax.core.FixedCredentialsProvider; -import com.google.api.gax.grpc.ChannelPoolSettings; -import com.google.api.gax.grpc.InstantiatingGrpcChannelProvider; -import com.google.api.gax.rpc.FixedHeaderProvider; -import com.google.api.gax.rpc.StubSettings; -import com.google.cloud.bigtable.admin.v2.BigtableTableAdminSettings; -import com.google.cloud.bigtable.config.BigtableOptions; -import com.google.cloud.bigtable.data.v2.BigtableDataSettings; -import io.grpc.ManagedChannelBuilder; -import io.grpc.internal.GrpcUtil; -import java.io.IOException; -import java.util.Objects; -import javax.annotation.Nonnull; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; -import org.checkerframework.checker.initialization.qual.UnderInitialization; - -/** Helper class to translate {@link BigtableOptions} to Bigtable Veneer settings. */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -class BigtableConfigToVeneerSettings { - private static final String DEFAULT_DATA_ENDPOINT = "bigtable.googleapis.com:443"; - private static final String DEFAULT_ADMIN_ENDPOINT = "bigtableadmin.googleapis.com:443"; - - private final BigtableDataSettings dataSettings; - private final BigtableTableAdminSettings tableAdminSettings; - - static BigtableConfigToVeneerSettings create(@Nonnull BigtableConfig config) throws IOException { - return new BigtableConfigToVeneerSettings(config); - } - - private BigtableConfigToVeneerSettings(@Nonnull BigtableConfig config) throws IOException { - if (config.getProjectId() == null || config.getInstanceId() == null) { - throw new IOException("can't find project or instance id"); - } - - // Build configs for veneer - this.dataSettings = buildBigtableDataSettings(config); - this.tableAdminSettings = buildBigtableTableAdminSettings(config); - } - - // ************** Getters ************** - /** Utility to convert {@link BigtableOptions} to {@link BigtableDataSettings}. */ - BigtableDataSettings getDataSettings() { - return dataSettings; - } - - /** Utility to convert {@link BigtableOptions} to {@link BigtableTableAdminSettings}. */ - BigtableTableAdminSettings getTableAdminSettings() { - return tableAdminSettings; - } - - // ************** Private Helpers ************** - private BigtableDataSettings buildBigtableDataSettings( - @UnderInitialization BigtableConfigToVeneerSettings this, BigtableConfig config) - throws IOException { - BigtableDataSettings.Builder dataBuilder; - - // Configure the Data connection - dataBuilder = BigtableDataSettings.newBuilder(); - if (config.getEmulatorHost() != null) { - configureConnection( - dataBuilder.stubSettings(), config, Objects.requireNonNull(config.getEmulatorHost())); - } else { - configureConnection(dataBuilder.stubSettings(), config, DEFAULT_DATA_ENDPOINT); - } - configureCredentialProvider(dataBuilder.stubSettings(), config); - configureHeaderProvider(dataBuilder.stubSettings(), config); - - // Configure the target - dataBuilder - .setProjectId(Objects.requireNonNull(config.getProjectId().get())) - .setInstanceId(Objects.requireNonNull(config.getInstanceId().get())); - if (config.getAppProfileId() != null) { - dataBuilder.setAppProfileId(Objects.requireNonNull(config.getAppProfileId().get())); - } - - return dataBuilder.build(); - } - - private BigtableTableAdminSettings buildBigtableTableAdminSettings( - @UnderInitialization BigtableConfigToVeneerSettings this, BigtableConfig config) - throws IOException { - BigtableTableAdminSettings.Builder adminBuilder; - - // Configure connection - adminBuilder = BigtableTableAdminSettings.newBuilder(); - if (config.getEmulatorHost() != null) { - configureConnection( - adminBuilder.stubSettings(), config, Objects.requireNonNull(config.getEmulatorHost())); - } else { - configureConnection(adminBuilder.stubSettings(), config, DEFAULT_ADMIN_ENDPOINT); - } - - configureCredentialProvider(adminBuilder.stubSettings(), config); - - adminBuilder - .setProjectId(Objects.requireNonNull(config.getProjectId().get())) - .setInstanceId(Objects.requireNonNull(config.getInstanceId().get())); - - return adminBuilder.build(); - } - - @SuppressWarnings("rawtypes") - private void configureConnection( - @UnderInitialization BigtableConfigToVeneerSettings this, - StubSettings.Builder stubSettings, - BigtableConfig config, - String endpoint) { - stubSettings.setEndpoint(endpoint); - - final InstantiatingGrpcChannelProvider.Builder channelProvider = - ((InstantiatingGrpcChannelProvider) stubSettings.getTransportChannelProvider()).toBuilder(); - - if (config.getEmulatorHost() != null) { - @SuppressWarnings("rawtypes") - final ApiFunction prevConfigurator = - channelProvider.getChannelConfigurator(); - //noinspection rawtypes - channelProvider.setChannelConfigurator( - new ApiFunction() { - @Override - public ManagedChannelBuilder apply(ManagedChannelBuilder channelBuilder) { - if (prevConfigurator != null) { - channelBuilder = prevConfigurator.apply(channelBuilder); - } - return channelBuilder.usePlaintext(); - } - }); - } - - stubSettings.setTransportChannelProvider(channelProvider.build()); - } - - private void configureHeaderProvider( - @UnderInitialization BigtableConfigToVeneerSettings this, - StubSettings.Builder stubSettings, - BigtableConfig config) { - - ImmutableMap.Builder headersBuilder = ImmutableMap.builder(); - headersBuilder.putAll(stubSettings.getHeaderProvider().getHeaders()); - headersBuilder.put( - GrpcUtil.USER_AGENT_KEY.name(), Objects.requireNonNull(config.getUserAgent())); - - stubSettings.setHeaderProvider(FixedHeaderProvider.create(headersBuilder.build())); - } - - private void configureCredentialProvider( - @UnderInitialization BigtableConfigToVeneerSettings this, - StubSettings.Builder stubSettings, - BigtableConfig config) { - stubSettings.setCredentialsProvider(FixedCredentialsProvider.create(config.getCredentials())); - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java new file mode 100644 index 000000000000..ba5b99179853 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java @@ -0,0 +1,342 @@ +/* + * 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.bigtable; + +import com.google.api.gax.batching.BatchingSettings; +import com.google.api.gax.core.FixedCredentialsProvider; +import com.google.api.gax.core.NoCredentialsProvider; +import com.google.api.gax.retrying.RetrySettings; +import com.google.api.gax.rpc.FixedHeaderProvider; +import com.google.api.gax.rpc.StubSettings; +import com.google.auth.Credentials; +import com.google.auth.oauth2.GoogleCredentials; +import com.google.auth.oauth2.ServiceAccountJwtAccessCredentials; +import com.google.cloud.bigtable.config.BigtableOptions; +import com.google.cloud.bigtable.config.CredentialOptions; +import com.google.cloud.bigtable.data.v2.BigtableDataSettings; +import com.google.cloud.bigtable.data.v2.stub.BigtableBatchingCallSettings; +import io.grpc.internal.GrpcUtil; +import java.io.FileInputStream; +import java.io.IOException; +import java.security.GeneralSecurityException; +import java.security.KeyStore; +import java.security.PrivateKey; +import java.util.Objects; +import javax.annotation.Nonnull; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.ValueProvider; +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.collect.ImmutableMap; +import org.threeten.bp.Duration; + +/** Helper class to translate {@link BigtableOptions} to Bigtable Veneer settings. */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +class BigtableConfigTranslator { + + /** Translate BigtableConfig and BigtableReadOptions to Veneer settings. */ + static BigtableDataSettings translateReadToVeneerSettings( + @Nonnull BigtableConfig config, + @Nonnull BigtableReadOptions options, + @Nonnull PipelineOptions pipelineOptions) { + BigtableDataSettings.Builder settings = buildBigtableDataSettings(config, pipelineOptions); + return configureReadSettings(settings, options); + } + + /** Translate BigtableConfig and BigtableWriteOptions to Veneer settings. */ + static BigtableDataSettings translateWriteToVeneerSettings( + @Nonnull BigtableConfig config, + @Nonnull BigtableWriteOptions options, + @Nonnull PipelineOptions pipelineOptions) { + + BigtableDataSettings.Builder settings = buildBigtableDataSettings(config, pipelineOptions); + return configureWriteSettings(settings, options); + } + + private static BigtableDataSettings.Builder buildBigtableDataSettings( + BigtableConfig config, PipelineOptions pipelineOptions) { + BigtableDataSettings.Builder dataBuilder; + if (!Strings.isNullOrEmpty(config.getEmulatorHost())) { + String hostAndPort = config.getEmulatorHost(); + try { + int lastIndexOfCol = hostAndPort.lastIndexOf(":"); + int port = Integer.parseInt(hostAndPort.substring(lastIndexOfCol + 1)); + dataBuilder = + BigtableDataSettings.newBuilderForEmulator( + hostAndPort.substring(0, lastIndexOfCol), port); + } catch (NumberFormatException | IndexOutOfBoundsException ex) { + throw new RuntimeException("Invalid host/port in BigtableConfig " + hostAndPort); + } + } else { + dataBuilder = BigtableDataSettings.newBuilder(); + } + + // Configure target + dataBuilder + .setProjectId(Objects.requireNonNull(config.getProjectId().get())) + .setInstanceId(Objects.requireNonNull(config.getInstanceId().get())); + if (config.getAppProfileId() != null) { + dataBuilder.setAppProfileId(Objects.requireNonNull(config.getAppProfileId().get())); + } + + // Configure credentials, check both PipelineOptions and BigtableConfig + if (pipelineOptions instanceof GcpOptions) { + dataBuilder + .stubSettings() + .setCredentialsProvider( + FixedCredentialsProvider.create(((GcpOptions) pipelineOptions).getGcpCredential())); + } + + if (config.getCredentialsProvider() != null) { + dataBuilder.stubSettings().setCredentialsProvider(config.getCredentialsProvider()); + } + + configureHeaderProvider(dataBuilder.stubSettings(), pipelineOptions); + + return dataBuilder; + } + + private static void configureHeaderProvider( + StubSettings.Builder stubSettings, PipelineOptions pipelineOptions) { + + ImmutableMap.Builder headersBuilder = ImmutableMap.builder(); + headersBuilder.putAll(stubSettings.getHeaderProvider().getHeaders()); + headersBuilder.put( + GrpcUtil.USER_AGENT_KEY.name(), Objects.requireNonNull(pipelineOptions.getUserAgent())); + + stubSettings.setHeaderProvider(FixedHeaderProvider.create(headersBuilder.build())); + } + + private static BigtableDataSettings configureWriteSettings( + BigtableDataSettings.Builder settings, BigtableWriteOptions writeOptions) { + BigtableBatchingCallSettings.Builder callSettings = + settings.stubSettings().bulkMutateRowsSettings(); + RetrySettings.Builder retrySettings = callSettings.getRetrySettings().toBuilder(); + BatchingSettings.Builder batchingSettings = callSettings.getBatchingSettings().toBuilder(); + if (writeOptions.getAttemptTimeout() != null) { + retrySettings.setInitialRpcTimeout(Duration.ofMillis(writeOptions.getAttemptTimeout())); + + if (writeOptions.getOperationTimeout() == null) { + retrySettings.setTotalTimeout( + Duration.ofMillis( + Math.max( + retrySettings.getTotalTimeout().toMillis(), writeOptions.getAttemptTimeout()))); + } + } + + if (writeOptions.getOperationTimeout() != null) { + retrySettings.setTotalTimeout(Duration.ofMillis(writeOptions.getOperationTimeout())); + } + + if (writeOptions.getRetryInitialDelay() != null) { + retrySettings.setInitialRetryDelay(Duration.ofMillis(writeOptions.getRetryInitialDelay())); + } + + if (writeOptions.getRetryDelayMultiplier() != null) { + retrySettings.setRetryDelayMultiplier(writeOptions.getRetryDelayMultiplier()); + } + + if (writeOptions.getBatchElements() != null) { + batchingSettings.setElementCountThreshold(writeOptions.getBatchElements()); + } + + if (writeOptions.getBatchBytes() != null) { + batchingSettings.setRequestByteThreshold(writeOptions.getBatchBytes()); + } + + if (writeOptions.getMaxRequests() != null) { + BatchingSettings tmpSettings = batchingSettings.build(); + batchingSettings = + batchingSettings.setFlowControlSettings( + callSettings + .getBatchingSettings() + .getFlowControlSettings() + .toBuilder() + .setMaxOutstandingElementCount( + tmpSettings.getElementCountThreshold() * writeOptions.getMaxRequests()) + .setMaxOutstandingRequestBytes( + tmpSettings.getRequestByteThreshold() * writeOptions.getMaxRequests()) + .build()); + } + + settings + .stubSettings() + .bulkMutateRowsSettings() + .setRetrySettings(retrySettings.build()) + .setBatchingSettings(batchingSettings.build()); + + return settings.build(); + } + + private static BigtableDataSettings configureReadSettings( + BigtableDataSettings.Builder settings, BigtableReadOptions readOptions) { + + RetrySettings.Builder retrySettings = + settings.stubSettings().bulkReadRowsSettings().getRetrySettings().toBuilder(); + + if (readOptions.getAttemptTimeout() != null) { + retrySettings.setInitialRpcTimeout(Duration.ofMillis(readOptions.getAttemptTimeout())); + + if (readOptions.getOperationTimeout() == null) { + retrySettings.setTotalTimeout( + Duration.ofMillis( + Math.max( + retrySettings.getTotalTimeout().toMillis(), readOptions.getAttemptTimeout()))); + } + } + + if (readOptions.getOperationTimeout() != null) { + retrySettings.setTotalTimeout(Duration.ofMillis(readOptions.getOperationTimeout())); + } + + if (readOptions.getRetryDelayMultiplier() != null) { + retrySettings.setRetryDelayMultiplier(readOptions.getRetryDelayMultiplier()); + } + + if (readOptions.getRetryInitialDelay() != null) { + retrySettings.setInitialRetryDelay(Duration.ofMillis(readOptions.getRetryInitialDelay())); + } + + settings.stubSettings().bulkReadRowsSettings().setRetrySettings(retrySettings.build()); + + return settings.build(); + } + + /** Translate BigtableOptions to BigtableConfig. */ + static BigtableConfig translateToBigtableConfig(BigtableConfig config, BigtableOptions options) { + BigtableConfig.Builder builder = config.toBuilder(); + + if (options.getProjectId() != null && config.getProjectId() == null) { + builder.setProjectId(ValueProvider.StaticValueProvider.of(options.getProjectId())); + } + + if (options.getInstanceId() != null && config.getInstanceId() == null) { + builder.setInstanceId(ValueProvider.StaticValueProvider.of(options.getInstanceId())); + } + + if (options.getAppProfileId() != null && config.getAppProfileId() == null) { + builder.setAppProfileId(ValueProvider.StaticValueProvider.of(options.getAppProfileId())); + } + + if (!options.getDataHost().equals("bigtable.googleapis.com") + && config.getEmulatorHost() == null) { + builder.setEmulatorHost(String.format("%s:%s", options.getDataHost(), options.getPort())); + } + // - instance of credentials, P12, SUPPLIED, suppliedjson - credentails + // - default credentials , needs to be called on the worker GoogleCredentialProvider + + if (options.getCredentialOptions() != null) { + try { + CredentialOptions credOptions = options.getCredentialOptions(); + switch (credOptions.getCredentialType()) { + case DefaultCredentials: + // Veneer uses GoogleDefaultCredentials, so we don't need to reset it + break; + case P12: + String keyFile = ((CredentialOptions.P12CredentialOptions) credOptions).getKeyFile(); + String serviceAccount = + ((CredentialOptions.P12CredentialOptions) credOptions).getServiceAccount(); + try { + KeyStore keyStore = KeyStore.getInstance("PKCS12"); + + try (FileInputStream fin = new FileInputStream(keyFile)) { + keyStore.load(fin, "notasecret".toCharArray()); + } + PrivateKey privateKey = + (PrivateKey) keyStore.getKey("privatekey", "notasecret".toCharArray()); + + if (privateKey == null) { + throw new IllegalStateException("private key cannot be null"); + } + builder.setCredentialsProvider( + FixedCredentialsProvider.create( + ServiceAccountJwtAccessCredentials.newBuilder() + .setClientEmail(serviceAccount) + .setPrivateKey(privateKey) + .build())); + } catch (GeneralSecurityException exception) { + throw new RuntimeException("exception while retrieving credentials", exception); + } + break; + case SuppliedCredentials: + Credentials credentials = + ((CredentialOptions.UserSuppliedCredentialOptions) credOptions).getCredential(); + builder.setCredentialsProvider(FixedCredentialsProvider.create(credentials)); + break; + case SuppliedJson: + CredentialOptions.JsonCredentialsOptions jsonCredentialsOptions = + (CredentialOptions.JsonCredentialsOptions) credOptions; + synchronized (jsonCredentialsOptions) { + if (jsonCredentialsOptions.getCachedCredentials() == null) { + jsonCredentialsOptions.setCachedCredentails( + GoogleCredentials.fromStream(jsonCredentialsOptions.getInputStream())); + } + builder.setCredentialsProvider( + FixedCredentialsProvider.create(jsonCredentialsOptions.getCachedCredentials())); + } + break; + case None: + builder.setCredentialsProvider(NoCredentialsProvider.create()); + break; + } + } catch (IOException e) { + throw new RuntimeException("Failed to translate BigtableOptions to BigtableConfig", e); + } + } + + return builder.build(); + } + + /** Translate BigtableOptions to BigtableReadOptions. */ + static BigtableReadOptions translateToBigtableReadOptions( + BigtableReadOptions readOptions, BigtableOptions options) { + BigtableReadOptions.Builder builder = readOptions.toBuilder(); + if (options.getCallOptionsConfig().getReadStreamRpcAttemptTimeoutMs().isPresent()) { + builder.setAttemptTimeout( + options.getCallOptionsConfig().getReadStreamRpcAttemptTimeoutMs().get()); + } + builder.setOperationTimeout(options.getCallOptionsConfig().getReadStreamRpcTimeoutMs()); + builder.setRetryInitialDelay(options.getRetryOptions().getInitialBackoffMillis()); + builder.setRetryDelayMultiplier(options.getRetryOptions().getBackoffMultiplier()); + return builder.build(); + } + + /** Translate BigtableOptions to BigtableWriteOptions. */ + static BigtableWriteOptions translateToBigtableWriteOptions( + BigtableWriteOptions writeOptions, BigtableOptions options) { + + BigtableWriteOptions.Builder builder = writeOptions.toBuilder(); + // configure timeouts + if (options.getCallOptionsConfig().getMutateRpcAttemptTimeoutMs().isPresent()) { + builder.setAttemptTimeout( + options.getCallOptionsConfig().getMutateRpcAttemptTimeoutMs().get()); + } + builder.setOperationTimeout(options.getCallOptionsConfig().getMutateRpcTimeoutMs()); + // configure retry backoffs + builder.setRetryInitialDelay(options.getRetryOptions().getInitialBackoffMillis()); + builder.setRetryDelayMultiplier(options.getRetryOptions().getBackoffMultiplier()); + // configure batch size + builder.setBatchElements(options.getBulkOptions().getBulkMaxRowKeyCount()); + builder.setBatchBytes(options.getBulkOptions().getBulkMaxRequestSize()); + builder.setMaxRequests(options.getBulkOptions().getMaxInflightRpcs()); + + return builder.build(); + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 5352c96aaef8..9dedf21323d7 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -17,12 +17,14 @@ */ package org.apache.beam.sdk.io.gcp.bigtable; +import static org.apache.beam.sdk.io.gcp.bigtable.BigtableServiceFactory.BigtableServiceEntry; +import static org.apache.beam.sdk.io.gcp.bigtable.BigtableServiceFactory.FACTORY_INSTANCE; import static org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; -import com.google.auth.Credentials; +import com.google.api.gax.core.CredentialsProvider; import com.google.auto.value.AutoValue; import com.google.bigtable.v2.Mutation; import com.google.bigtable.v2.Row; @@ -41,7 +43,6 @@ import java.util.Map; import java.util.NoSuchElementException; import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.atomic.AtomicInteger; import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; @@ -290,6 +291,8 @@ public abstract static class Read extends PTransform> { abstract BigtableReadOptions getBigtableReadOptions(); + private BigtableServiceFactory.ConfigId configId; + /** Returns the table being read from. */ public @Nullable String getTableId() { ValueProvider tableId = getBigtableReadOptions().getTableId(); @@ -404,9 +407,11 @@ public Read withTableId(String tableId) { * *

Does not modify this object. */ - public Read withCredentials(Credentials credentials) { + public Read withCredentialsProvider(CredentialsProvider credentialsProvider) { BigtableConfig config = getBigtableConfig(); - return toBuilder().setBigtableConfig(config.withCredentails(credentials)).build(); + return toBuilder() + .setBigtableConfig(config.withCredentialsProvider(credentialsProvider)) + .build(); } /** @@ -555,20 +560,6 @@ public Read withoutValidation() { return toBuilder().setBigtableConfig(config.withValidate(false)).build(); } - /** - * Returns a new {@link BigtableIO.Read} that will read using the given Cloud Bigtable service - * implementation. - * - *

This is used for testing. - * - *

Does not modify this object. - */ - @VisibleForTesting - Read withBigtableService(BigtableService bigtableService) { - BigtableConfig config = getBigtableConfig(); - return toBuilder().setBigtableConfig(config.withBigtableService(bigtableService)).build(); - } - /** * Returns a new {@link BigtableIO.Read} that will use an official Bigtable emulator. * @@ -634,37 +625,27 @@ public Read withRetryDelayMultiplier(double multiplier) { .build(); } - /** Helper method to translate BigtableOptions to BigtableReadOptions. */ - private void translateBigtableOptions(BigtableOptions options) { - BigtableReadOptions.Builder readOptions = getBigtableReadOptions().toBuilder(); - if (options.getCallOptionsConfig().getReadStreamRpcAttemptTimeoutMs().isPresent()) { - readOptions.setAttemptTimeout( - options.getCallOptionsConfig().getReadStreamRpcAttemptTimeoutMs().get()); - } - readOptions.setOperationTimeout(options.getCallOptionsConfig().getReadStreamRpcTimeoutMs()); - readOptions.setRetryInitialDelay(options.getRetryOptions().getInitialBackoffMillis()); - readOptions.setRetryDelayMultiplier(options.getRetryOptions().getBackoffMultiplier()); - - toBuilder().setBigtableReadOptions(readOptions.build()).build(); + /** Set the configId for testing. */ + @VisibleForTesting + Read withConfigId(int id) { + return toBuilder().setBigtableConfig(getBigtableConfig().withConfigId(id)).build(); } @Override public PCollection expand(PBegin input) { BigtableConfig config = this.getBigtableConfig(); - // If BigtableOptions present, convert BigtableOptions into BigtableReadOptions - if (config.getBigtableOptions() != null) { - translateBigtableOptions(config.getBigtableOptions()); - } else if (config.getBigtableOptionsConfigurator() != null) { - BigtableOptions.Builder options = BigtableOptions.builder(); - options = config.getBigtableOptionsConfigurator().apply(options); - translateBigtableOptions(options.build()); - } - getBigtableConfig().validate(); getBigtableReadOptions().validate(); + // Generate a new configId when the configuration is fixed + if (config.getConfigId() != null) { + this.configId = BigtableServiceFactory.ConfigId.create(config.getConfigId()); + } else { + this.configId = FACTORY_INSTANCE.newId(); + } + BigtableSource source = - new BigtableSource(getBigtableConfig(), getBigtableReadOptions(), null); + new BigtableSource(configId, getBigtableConfig(), getBigtableReadOptions(), null); return input.getPipeline().apply(org.apache.beam.sdk.io.Read.from(source)); } @@ -686,6 +667,21 @@ public final String toString() { MoreObjects.toStringHelper(Read.class).add("config", getBigtableConfig()); return helper.add("readOptions", getBigtableReadOptions()).toString(); } + + private void validateTableExists( + BigtableConfig config, BigtableReadOptions readOptions, PipelineOptions options) { + if (config.getValidate() && config.isDataAccessible() && readOptions.isDataAccessible()) { + String tableId = checkNotNull(readOptions.getTableId().get()); + try { + BigtableServiceEntry entry = + FACTORY_INSTANCE.getServiceForReading(configId, config, readOptions, options); + checkArgument( + entry.getService().tableExists(tableId), "Table %s does not exist", tableId); + } catch (IOException e) { + LOG.warn("Error checking whether table {} exists; proceeding.", tableId, e); + } + } + } } /** @@ -822,9 +818,11 @@ public Write withTableId(String tableId) { * *

Does not modify this object. */ - public Write withCredentials(Credentials credentials) { + public Write withCredentialsProvider(CredentialsProvider credentialsProvider) { BigtableConfig config = getBigtableConfig(); - return toBuilder().setBigtableConfig(config.withCredentails(credentials)).build(); + return toBuilder() + .setBigtableConfig(config.withCredentialsProvider(credentialsProvider)) + .build(); } /** * WARNING: Should be used only to specify additional parameters for connection to the Cloud @@ -894,19 +892,6 @@ public Write withoutValidation() { return toBuilder().setBigtableConfig(config.withValidate(false)).build(); } - /** - * Returns a new {@link BigtableIO.Write} that will write using the given Cloud Bigtable service - * implementation. - * - *

This is used for testing. - * - *

Does not modify this object. - */ - Write withBigtableService(BigtableService bigtableService) { - BigtableConfig config = getBigtableConfig(); - return toBuilder().setBigtableConfig(config.withBigtableService(bigtableService)).build(); - } - /** * Returns a new {@link BigtableIO.Write} that will use an official Bigtable emulator. * @@ -1009,26 +994,6 @@ public Write withMaxRequests(long requests) { .build(); } - /** Helper method to translate Bigtable Options to BigtableWriteOptions. */ - private void translateBigtableOptions(BigtableOptions options) { - BigtableWriteOptions.Builder writeOptions = getBigtableWriteOptions().toBuilder(); - // configure timeouts - if (options.getCallOptionsConfig().getMutateRpcAttemptTimeoutMs().isPresent()) { - writeOptions.setAttemptTimeout( - options.getCallOptionsConfig().getMutateRpcAttemptTimeoutMs().get()); - } - writeOptions.setOperationTimeout(options.getCallOptionsConfig().getMutateRpcTimeoutMs()); - // configure retry backoffs - writeOptions.setRetryInitialDelay(options.getRetryOptions().getInitialBackoffMillis()); - writeOptions.setRetryDelayMultiplier(options.getRetryOptions().getBackoffMultiplier()); - // configure batch size - writeOptions.setBatchElements(options.getBulkOptions().getBulkMaxRowKeyCount()); - writeOptions.setBatchBytes(options.getBulkOptions().getBulkMaxRequestSize()); - writeOptions.setMaxRequests(options.getBulkOptions().getMaxInflightRpcs()); - - toBuilder().setBigtableWriteOptions(writeOptions.build()).build(); - } - /** * Returns a {@link BigtableIO.WriteWithResults} that will emit a {@link BigtableWriteResult} * for each batch of rows written. @@ -1037,17 +1002,14 @@ public WriteWithResults withWriteResults() { return new WriteWithResults(getBigtableConfig(), getBigtableWriteOptions()); } + /** Set the configId for testing. */ + @VisibleForTesting + Write withConfigId(int id) { + return toBuilder().setBigtableConfig(getBigtableConfig().withConfigId(id)).build(); + } + @Override public PDone expand(PCollection>> input) { - BigtableConfig config = this.getBigtableConfig(); - if (config.getBigtableOptions() != null) { - translateBigtableOptions(config.getBigtableOptions()); - } else if (config.getBigtableOptionsConfigurator() != null) { - BigtableOptions.Builder options = BigtableOptions.builder(); - options = config.getBigtableOptionsConfigurator().apply(options); - translateBigtableOptions(options.build()); - } - input.apply(withWriteResults()); return PDone.in(input.getPipeline()); } @@ -1082,9 +1044,16 @@ public static class WriteWithResults private final BigtableConfig bigtableConfig; private final BigtableWriteOptions bigtableWriteOptions; + private final BigtableServiceFactory.ConfigId configId; + WriteWithResults(BigtableConfig bigtableConfig, BigtableWriteOptions bigtableWriteOptions) { this.bigtableConfig = bigtableConfig; this.bigtableWriteOptions = bigtableWriteOptions; + if (bigtableConfig.getConfigId() != null) { + this.configId = BigtableServiceFactory.ConfigId.create(bigtableConfig.getConfigId()); + } else { + this.configId = FACTORY_INSTANCE.newId(); + } } @Override @@ -1093,7 +1062,8 @@ public PCollection expand( bigtableConfig.validate(); bigtableWriteOptions.validate(); - return input.apply(ParDo.of(new BigtableWriterFn(bigtableConfig, bigtableWriteOptions))); + return input.apply( + ParDo.of(new BigtableWriterFn(configId, bigtableConfig, bigtableWriteOptions))); } @Override @@ -1115,6 +1085,21 @@ public String toString() { .add("writeOptions", bigtableWriteOptions) .toString(); } + + private void validateTableExists( + BigtableConfig config, BigtableWriteOptions writeOptions, PipelineOptions options) { + if (config.getValidate() && config.isDataAccessible() && writeOptions.isDataAccessible()) { + String tableId = checkNotNull(writeOptions.getTableId().get()); + try { + BigtableServiceEntry entry = + FACTORY_INSTANCE.getServiceForWriting(configId, config, writeOptions, options); + checkArgument( + entry.getService().tableExists(tableId), "Table %s does not exist", tableId); + } catch (IOException e) { + LOG.warn("Error checking whether table {} exists; proceeding.", tableId, e); + } + } + } } private static class BigtableWriterFn @@ -1122,28 +1107,30 @@ private static class BigtableWriterFn // uuid of each DoFn family // current id - private int id; + private BigtableServiceFactory.ConfigId id; + private BigtableServiceEntry serviceEntry; - private static AtomicInteger uuid = new AtomicInteger(0); - - BigtableWriterFn(BigtableConfig bigtableConfig, BigtableWriteOptions writeOptions) { + BigtableWriterFn( + BigtableServiceFactory.ConfigId configId, + BigtableConfig bigtableConfig, + BigtableWriteOptions writeOptions) { this.config = bigtableConfig; this.writeOptions = writeOptions; this.failures = new ConcurrentLinkedQueue<>(); - this.id = BigtableWriterFn.uuid.getAndIncrement(); + this.id = configId; } @StartBundle public void startBundle(StartBundleContext c) throws IOException { + recordsWritten = 0; + this.seenWindows = Maps.newHashMapWithExpectedSize(1); + // Check if the family id if theres a client, increment reference count if (bigtableWriter == null) { - bigtableWriter = - config - .getBigtableService(c.getPipelineOptions()) - .openForWriting(writeOptions.getTableId().get(), writeOptions, id); + serviceEntry = + FACTORY_INSTANCE.getServiceForWriting(id, config, writeOptions, c.getPipelineOptions()); + bigtableWriter = serviceEntry.getService().openForWriting(writeOptions.getTableId().get()); } - recordsWritten = 0; - this.seenWindows = Maps.newHashMapWithExpectedSize(1); } @ProcessElement @@ -1180,6 +1167,7 @@ public void tearDown() throws Exception { if (bigtableWriter != null) { bigtableWriter.close(); bigtableWriter = null; + FACTORY_INSTANCE.releaseWriteService(serviceEntry); } } @@ -1238,10 +1226,14 @@ private static ByteKey makeByteKey(ByteString key) { static class BigtableSource extends BoundedSource { public BigtableSource( - BigtableConfig config, BigtableReadOptions readOptions, @Nullable Long estimatedSizeBytes) { + BigtableServiceFactory.ConfigId configId, + BigtableConfig config, + BigtableReadOptions readOptions, + @Nullable Long estimatedSizeBytes) { this.config = config; this.readOptions = readOptions; this.estimatedSizeBytes = estimatedSizeBytes; + this.configId = configId; } @Override @@ -1258,15 +1250,18 @@ public String toString() { private final BigtableReadOptions readOptions; private @Nullable Long estimatedSizeBytes; + private final BigtableServiceFactory.ConfigId configId; + /** Creates a new {@link BigtableSource} with just one {@link ByteKeyRange}. */ protected BigtableSource withSingleRange(ByteKeyRange range) { checkArgument(range != null, "range can not be null"); - return new BigtableSource(config, readOptions.withKeyRange(range), estimatedSizeBytes); + return new BigtableSource( + configId, config, readOptions.withKeyRange(range), estimatedSizeBytes); } protected BigtableSource withEstimatedSizeBytes(Long estimatedSizeBytes) { checkArgument(estimatedSizeBytes != null, "estimatedSizeBytes can not be null"); - return new BigtableSource(config, readOptions, estimatedSizeBytes); + return new BigtableSource(configId, config, readOptions, estimatedSizeBytes); } /** @@ -1275,7 +1270,11 @@ protected BigtableSource withEstimatedSizeBytes(Long estimatedSizeBytes) { * different tablets, and possibly generate sub-splits within tablets. */ private List getSampleRowKeys(PipelineOptions pipelineOptions) throws IOException { - return config.getBigtableService(pipelineOptions).getSampleRowKeys(this); + BigtableServiceFactory.BigtableServiceEntry serviceEntry = + FACTORY_INSTANCE.getServiceForReading(configId, config, readOptions, pipelineOptions); + List keyOffsets = serviceEntry.getService().getSampleRowKeys(this); + FACTORY_INSTANCE.releaseReadService(serviceEntry); + return keyOffsets; } private static final long MAX_SPLIT_COUNT = 15_360L; @@ -1319,7 +1318,8 @@ protected List reduceSplits( if (counter == numberToCombine || !checkRangeAdjacency(previousSourceRanges, source.getRanges())) { reducedSplits.add( - new BigtableSource(config, readOptions.withKeyRanges(previousSourceRanges), size)); + new BigtableSource( + configId, config, readOptions.withKeyRanges(previousSourceRanges), size)); counter = 0; size = 0; previousSourceRanges = new ArrayList<>(); @@ -1331,7 +1331,8 @@ protected List reduceSplits( } if (size > 0) { reducedSplits.add( - new BigtableSource(config, readOptions.withKeyRanges(previousSourceRanges), size)); + new BigtableSource( + configId, config, readOptions.withKeyRanges(previousSourceRanges), size)); } return reducedSplits; } @@ -1527,7 +1528,8 @@ private long getEstimatedSizeBytesBasedOnSamples(List samples) { @Override public BoundedReader createReader(PipelineOptions options) throws IOException { - return new BigtableReader(this, config.getBigtableService(options)); + return new BigtableReader( + this, FACTORY_INSTANCE.getServiceForReading(configId, config, readOptions, options)); } @Override @@ -1621,27 +1623,27 @@ private static class BigtableReader extends BoundedReader { // Thread-safety: source is protected via synchronization and is only accessed or modified // inside a synchronized block (or constructor, which is the same). private BigtableSource source; - private BigtableService service; + private final BigtableServiceEntry serviceEntry; private BigtableService.Reader reader; private final ByteKeyRangeTracker rangeTracker; private long recordsReturned; - private static AtomicInteger uuid = new AtomicInteger(0); - - private int id; - - public BigtableReader(BigtableSource source, BigtableService service) { + public BigtableReader(BigtableSource source, BigtableServiceEntry service) { checkArgument(source.getRanges().size() == 1, "source must have exactly one key range"); this.source = source; - this.service = service; + this.serviceEntry = service; rangeTracker = ByteKeyRangeTracker.of(source.getRanges().get(0)); - - this.id = BigtableReader.uuid.getAndIncrement(); } @Override public boolean start() throws IOException { - reader = service.createReader(getCurrentSource(), id); + reader = + serviceEntry + .getService() + .createReader( + getCurrentSource(), + serviceEntry.getAttemptTimeout(), + serviceEntry.getOperationTimeout()); boolean hasRecord = (reader.start() && rangeTracker.tryReturnRecordAt( @@ -1680,7 +1682,7 @@ public Row getCurrent() throws NoSuchElementException { public void close() throws IOException { LOG.info("Closing reader after reading {} records.", recordsReturned); if (reader != null) { - reader.close(); + FACTORY_INSTANCE.releaseReadService(serviceEntry); reader = null; } } @@ -1739,36 +1741,6 @@ public BigtableWriteException(KV> record, Throwab } } - static void validateTableExists( - BigtableConfig config, BigtableWriteOptions writeOptions, PipelineOptions options) { - if (config.getValidate() && config.isDataAccessible() && writeOptions.isDataAccessible()) { - String tableId = checkNotNull(writeOptions.getTableId().get()); - try { - checkArgument( - config.getBigtableService(options).tableExists(tableId), - "Table %s does not exist", - tableId); - } catch (IOException e) { - LOG.warn("Error checking whether table {} exists; proceeding.", tableId, e); - } - } - } - - static void validateTableExists( - BigtableConfig config, BigtableReadOptions readOptions, PipelineOptions options) { - if (config.getValidate() && config.isDataAccessible() && readOptions.isDataAccessible()) { - String tableId = checkNotNull(readOptions.getTableId().get()); - try { - checkArgument( - config.getBigtableService(options).tableExists(tableId), - "Table %s does not exist", - tableId); - } catch (IOException e) { - LOG.warn("Error checking whether table {} exists; proceeding.", tableId, e); - } - } - } - @AutoValue public abstract static class ReadChangeStream extends PTransform>> { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java index 35d1c7dae836..54e55ab82bc2 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java @@ -17,7 +17,6 @@ */ package org.apache.beam.sdk.io.gcp.bigtable; -import com.google.auth.Credentials; import com.google.bigtable.v2.MutateRowResponse; import com.google.bigtable.v2.Mutation; import com.google.bigtable.v2.Row; @@ -30,6 +29,7 @@ import java.util.concurrent.CompletionStage; import org.apache.beam.sdk.io.gcp.bigtable.BigtableIO.BigtableSource; import org.apache.beam.sdk.values.KV; +import org.threeten.bp.Duration; /** An interface for real or fake implementations of Cloud Bigtable. */ interface BigtableService extends Serializable { @@ -71,13 +71,6 @@ interface Reader { /** Attempts to read the next element, and returns true if an element has been read. */ boolean advance() throws IOException; - /** - * Closes the reader. - * - * @throws IOException if there is an error. - */ - void close() throws IOException; - /** * Returns the last row read by a successful start() or advance(), or throws if there is no * current row because the last such call was unsuccessful. @@ -89,11 +82,11 @@ interface Reader { boolean tableExists(String tableId) throws IOException; /** Returns a {@link Reader} that will read from the specified source. */ - Reader createReader(BigtableSource source, int id) throws IOException; + Reader createReader(BigtableSource source, Duration attemptTimeout, Duration operationTimeout) + throws IOException; /** Returns a {@link Writer} that will write to the specified table. */ - Writer openForWriting(String tableId, BigtableWriteOptions writeOptions, int id) - throws IOException; + Writer openForWriting(String tableId) throws IOException; /** * Returns a set of row keys sampled from the underlying table. These contain information about @@ -104,8 +97,4 @@ Writer openForWriting(String tableId, BigtableWriteOptions writeOptions, int id) String getProjectId(); String getInstanceId(); - - Credentials getCredentials(); - - String getEmulatorHost(); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java new file mode 100644 index 000000000000..fc9eae58766c --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java @@ -0,0 +1,183 @@ +/* + * 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.bigtable; + +import com.google.api.gax.retrying.RetrySettings; +import com.google.auto.value.AutoValue; +import com.google.cloud.bigtable.config.BigtableOptions; +import com.google.cloud.bigtable.data.v2.BigtableDataSettings; +import java.io.IOException; +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; +import org.threeten.bp.Duration; + +/** + * Factory class that caches {@link BigtableService} to share between workers with the same {@link + * BigtableConfig} and read / write options. + */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +class BigtableServiceFactory { + + static final BigtableServiceFactory FACTORY_INSTANCE = new BigtableServiceFactory(); + + private int nextId = 0; + + private final Map readEntries = new HashMap<>(); + private final Map writeEntries = new HashMap<>(); + + @AutoValue + abstract static class ConfigId implements Serializable { + + abstract int id(); + + static ConfigId create(int id) { + return new AutoValue_BigtableServiceFactory_ConfigId(id); + } + } + + @AutoValue + abstract static class BigtableServiceEntry { + + abstract ConfigId getConfigId(); + + abstract BigtableService getService(); + + abstract AtomicInteger getRefCount(); + + // Workaround for ReadRows requests which requires to pass the timeouts in + // ApiContext. Can be removed later once it's fixed in Veneer. + abstract Duration getAttemptTimeout(); + + abstract Duration getOperationTimeout(); + + static BigtableServiceEntry create( + ConfigId configId, + BigtableService service, + AtomicInteger refCount, + Duration attemptTimeout, + Duration operationTimeout) { + return new AutoValue_BigtableServiceFactory_BigtableServiceEntry( + configId, service, refCount, attemptTimeout, operationTimeout); + } + } + + synchronized BigtableServiceEntry getServiceForReading( + ConfigId configId, + BigtableConfig config, + BigtableReadOptions opts, + PipelineOptions pipelineOptions) + throws IOException { + BigtableServiceEntry entry = readEntries.get(configId); + if (entry != null) { + entry.getRefCount().incrementAndGet(); + return entry; + } + + BigtableOptions effectiveOptions = config.getBigtableOptions(); + if (effectiveOptions == null && config.getBigtableOptionsConfigurator() != null) { + effectiveOptions = + config.getBigtableOptionsConfigurator().apply(BigtableOptions.builder()).build(); + } + if (effectiveOptions != null) { + // If BigtableOptions is set, convert it to BigtableConfig and BigtableWriteOptions + config = BigtableConfigTranslator.translateToBigtableConfig(config, effectiveOptions); + opts = BigtableConfigTranslator.translateToBigtableReadOptions(opts, effectiveOptions); + } + BigtableDataSettings settings = + BigtableConfigTranslator.translateReadToVeneerSettings(config, opts, pipelineOptions); + BigtableService service = new BigtableServiceImpl(settings); + RetrySettings retrySettings = settings.readRowSettings().getRetrySettings(); + entry = + BigtableServiceEntry.create( + configId, + service, + new AtomicInteger(1), + retrySettings.getInitialRpcTimeout(), + retrySettings.getTotalTimeout()); + readEntries.put(configId, entry); + return entry; + } + + synchronized BigtableServiceEntry getServiceForWriting( + ConfigId configId, + BigtableConfig config, + BigtableWriteOptions opts, + PipelineOptions pipelineOptions) + throws IOException { + BigtableServiceEntry entry = writeEntries.get(configId); + if (entry != null) { + entry.getRefCount().incrementAndGet(); + return entry; + } + + BigtableDataSettings settings = + BigtableConfigTranslator.translateWriteToVeneerSettings(config, opts, pipelineOptions); + BigtableService service = new BigtableServiceImpl(settings); + RetrySettings retrySettings = + settings.getStubSettings().bulkMutateRowsSettings().getRetrySettings(); + entry = + BigtableServiceEntry.create( + configId, + service, + new AtomicInteger(1), + retrySettings.getInitialRpcTimeout(), + retrySettings.getTotalTimeout()); + writeEntries.put(configId, entry); + return entry; + } + + synchronized void releaseReadService(BigtableServiceEntry entry) { + if (entry.getRefCount().decrementAndGet() == 0) { + // entry.getService().close(); + readEntries.remove(entry.getConfigId()); + } + } + + synchronized void releaseWriteService(BigtableServiceEntry entry) { + if (entry.getRefCount().decrementAndGet() == 0) { + // entry.getService().close(); + writeEntries.remove(entry.getConfigId()); + } + } + + synchronized ConfigId newId() { + return ConfigId.create(nextId++); + } + + @VisibleForTesting + synchronized void addFakeWriteService(ConfigId id, BigtableService service) { + writeEntries.put( + id, + BigtableServiceEntry.create( + id, service, new AtomicInteger(1), Duration.ofMillis(100), Duration.ofMillis(1000))); + } + + @VisibleForTesting + synchronized void addFakeReadService(ConfigId id, BigtableService service) { + readEntries.put( + id, + BigtableServiceEntry.create( + id, service, new AtomicInteger(1), Duration.ofMillis(100), Duration.ofMillis(1000))); + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java index 54064d69a10f..40c5853e37fc 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java @@ -20,12 +20,9 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.MoreExecutors.directExecutor; import com.google.api.gax.batching.Batcher; -import com.google.api.gax.batching.BatchingSettings; import com.google.api.gax.grpc.GrpcCallContext; -import com.google.api.gax.retrying.RetrySettings; import com.google.api.gax.rpc.ResponseObserver; import com.google.api.gax.rpc.StreamController; -import com.google.auth.Credentials; import com.google.bigtable.v2.Cell; import com.google.bigtable.v2.Column; import com.google.bigtable.v2.Family; @@ -36,9 +33,6 @@ import com.google.bigtable.v2.RowFilter; import com.google.bigtable.v2.RowRange; import com.google.bigtable.v2.RowSet; -import com.google.cloud.Tuple; -import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; -import com.google.cloud.bigtable.admin.v2.BigtableTableAdminSettings; import com.google.cloud.bigtable.data.v2.BigtableDataClient; import com.google.cloud.bigtable.data.v2.BigtableDataSettings; import com.google.cloud.bigtable.data.v2.internal.ByteStringComparator; @@ -48,7 +42,6 @@ import com.google.cloud.bigtable.data.v2.models.Query; import com.google.cloud.bigtable.data.v2.models.RowAdapter; import com.google.cloud.bigtable.data.v2.models.RowMutationEntry; -import com.google.cloud.bigtable.data.v2.stub.BigtableBatchingCallSettings; import com.google.protobuf.ByteString; import io.grpc.CallOptions; import io.grpc.Deadline; @@ -68,11 +61,9 @@ import java.util.TreeMap; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; import javax.annotation.Nonnull; import org.apache.beam.runners.core.metrics.GcpResourceIdentifiers; import org.apache.beam.runners.core.metrics.MonitoringInfoConstants; @@ -106,58 +97,32 @@ class BigtableServiceImpl implements BigtableService { // Percentage of max number of rows allowed in the buffer private static final double WATERMARK_PERCENTAGE = .1; private static final long MIN_BYTE_BUFFER_SIZE = 100 * 1024 * 1024; // 100MB - private static ConcurrentHashMap> writeClients = - new ConcurrentHashMap<>(); - private static ConcurrentHashMap> readClients = - new ConcurrentHashMap<>(); - - public BigtableServiceImpl(BigtableConfig config) { - try { - this.config = config; - BigtableConfigToVeneerSettings veneeringSettings = - BigtableConfigToVeneerSettings.create(this.config); - this.dataSettings = veneeringSettings.getDataSettings(); - this.tableAdminSettings = veneeringSettings.getTableAdminSettings(); - } catch (IOException e) { - throw new RuntimeException(e); - } + public BigtableServiceImpl(BigtableDataSettings settings) throws IOException { + this.client = BigtableDataClient.create(settings); + this.projectId = settings.getProjectId(); + this.instanceId = settings.getInstanceId(); } - private final BigtableConfig config; - - private BigtableDataSettings dataSettings; - - private BigtableTableAdminSettings tableAdminSettings; + private final BigtableDataClient client; + private final String projectId; + private final String instanceId; @Override - public BigtableWriterImpl openForWriting(String tableId, BigtableWriteOptions options, int id) - throws IOException { - dataSettings = configureWriteSettings(dataSettings.toBuilder(), options); - LOG.info("Opening for writing with settings " + dataSettings); - Tuple clientTuple = writeClients.get(id); - if (clientTuple == null) { - clientTuple = Tuple.of(new AtomicInteger(0), BigtableDataClient.create(dataSettings)); - writeClients.put(id, clientTuple); - } - clientTuple.x().getAndIncrement(); - return new BigtableWriterImpl( - clientTuple.y(), dataSettings.getProjectId(), dataSettings.getInstanceId(), tableId, id); + public BigtableWriterImpl openForWriting(String tableId) { + return new BigtableWriterImpl(client, projectId, instanceId, tableId); } @Override public boolean tableExists(String tableId) throws IOException { - try (BigtableTableAdminClient adminClient = - BigtableTableAdminClient.create(tableAdminSettings)) { - adminClient.getTable(tableId); + try { + client.readRow(tableId, "non-exist-row"); return true; } catch (StatusRuntimeException e) { if (e.getStatus().getCode() == Code.NOT_FOUND) { return false; } - String message = - String.format( - "Error checking whether table %s (BigtableOptions %s) exists", tableId, config); + String message = String.format("Error checking whether table %s exists", tableId); LOG.error(message, e); throw new IOException(message, e); } @@ -165,7 +130,7 @@ public boolean tableExists(String tableId) throws IOException { @VisibleForTesting static class BigtableReaderImpl implements Reader { - private BigtableDataClient client; + private final BigtableDataClient client; private final String projectId; private final String instanceId; @@ -175,13 +140,11 @@ static class BigtableReaderImpl implements Reader { private final RowFilter rowFilter; private Iterator results; - private final Duration operationTimeout; private final Duration attemptTimeout; + private final Duration operationTimeout; private Row currentRow; - private int id; - @VisibleForTesting BigtableReaderImpl( BigtableDataClient client, @@ -190,9 +153,8 @@ static class BigtableReaderImpl implements Reader { String tableId, List ranges, @Nullable RowFilter rowFilter, - Duration operationTimeout, Duration attemptTimeout, - int id) { + Duration operationTimeout) { this.client = client; this.projectId = projectId; this.instanceId = instanceId; @@ -200,10 +162,8 @@ static class BigtableReaderImpl implements Reader { this.ranges = ranges; this.rowFilter = rowFilter; - this.operationTimeout = operationTimeout; this.attemptTimeout = attemptTimeout; - - this.id = id; + this.operationTimeout = operationTimeout; } @Override @@ -243,22 +203,6 @@ public boolean advance() throws IOException { return false; } - @Override - public void close() throws IOException { - // Goal: by the end of this function, both results and client are null and closed, - // independent of what errors they throw or prior state. - - if (client == null) { - // Only possible when previously closed, so we know that results is also null. - return; - } - - if (readClients.get(id) != null && readClients.get(id).x().decrementAndGet() == 0) { - readClients.get(id).y().close(); - readClients.remove(id); - } - } - @Override public Row getCurrentRow() throws NoSuchElementException { if (currentRow == null) { @@ -270,7 +214,7 @@ public Row getCurrentRow() throws NoSuchElementException { @VisibleForTesting static class BigtableSegmentReaderImpl implements Reader { - private BigtableDataClient client; + private final BigtableDataClient client; private @Nullable ReadRowsRequest nextRequest; private @Nullable Row currentRow; @@ -282,8 +226,6 @@ static class BigtableSegmentReaderImpl implements Reader { private final Duration attemptTimeout; private final Duration operationTimeout; - private final int id; - private static class UpstreamResults { private final List rows; private final @Nullable ReadRowsRequest nextRequest; @@ -303,8 +245,7 @@ static BigtableSegmentReaderImpl create( @Nullable RowFilter rowFilter, int maxBufferedElementCount, Duration attemptTimeout, - Duration operationTimeout, - int id) { + Duration operationTimeout) { RowSet.Builder rowSetBuilder = RowSet.newBuilder(); if (ranges.isEmpty()) { @@ -338,8 +279,7 @@ static BigtableSegmentReaderImpl create( maxSegmentByteSize, attemptTimeout, operationTimeout, - createCallMetric(projectId, instanceId, tableId), - id); + createCallMetric(projectId, instanceId, tableId)); } @VisibleForTesting @@ -354,8 +294,7 @@ static BigtableSegmentReaderImpl create( long maxSegmentByteSize, Duration attemptTimeout, Duration operationTimeout, - ServiceCallMetric serviceCallMetric, - int id) { + ServiceCallMetric serviceCallMetric) { if (rowSet.equals(rowSet.getDefaultInstanceForType())) { rowSet = RowSet.newBuilder().addRowRanges(RowRange.getDefaultInstance()).build(); } @@ -376,7 +315,6 @@ static BigtableSegmentReaderImpl create( this.refillSegmentWaterMark = (int) (request.getRowsLimit() * WATERMARK_PERCENTAGE); this.attemptTimeout = attemptTimeout; this.operationTimeout = operationTimeout; - this.id = id; } @Override @@ -499,14 +437,6 @@ private ReadRowsRequest truncateRequest(ReadRowsRequest request, ByteString last return requestBuilder.setRows(segment).build(); } - @Override - public void close() throws IOException { - if (readClients.get(id) != null && readClients.get(id).x().decrementAndGet() == 0) { - readClients.get(id).y().close(); - readClients.remove(id); - } - } - @Override public Row getCurrentRow() throws NoSuchElementException { if (currentRow == null) { @@ -523,15 +453,12 @@ static class BigtableWriterImpl implements Writer { private String instanceId; private String tableId; - private int id; - BigtableWriterImpl( - BigtableDataClient client, String projectId, String instanceId, String tableId, int id) { + BigtableDataClient client, String projectId, String instanceId, String tableId) { this.projectId = projectId; this.instanceId = instanceId; this.tableId = tableId; this.bulkMutation = client.newBulkMutationBatcher(tableId); - this.id = id; } @Override @@ -549,23 +476,16 @@ public void flush() throws IOException { @Override public void close() throws IOException { - try { - if (bulkMutation != null) { - try { - bulkMutation.flush(); - bulkMutation.close(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - // We fail since flush() operation was interrupted. - throw new IOException(e); - } - bulkMutation = null; - } - } finally { - if (writeClients.get(id) != null && writeClients.get(id).x().decrementAndGet() == 0) { - writeClients.get(id).y().close(); - writeClients.remove(id); + if (bulkMutation != null) { + try { + bulkMutation.flush(); + bulkMutation.close(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + // We fail since flush() operation was interrupted. + throw new IOException(e); } + bulkMutation = null; } } @@ -622,81 +542,31 @@ public void onFailure(Throwable throwable) { } @Override - public String toString() { - return MoreObjects.toStringHelper(BigtableServiceImpl.class).add("options", config).toString(); - } - - @Override - public Reader createReader(BigtableSource source, int id) throws IOException { - dataSettings = configureReadSettings(dataSettings.toBuilder(), source.getReadOptions()); - - RetrySettings retrySettings = - dataSettings.getStubSettings().bulkReadRowsSettings().getRetrySettings(); - LOG.info("Creating a Reader for Bigtable with settings: " + dataSettings); - Tuple client = readClients.get(id); - if (client == null) { - client = Tuple.of(new AtomicInteger(0), BigtableDataClient.create(dataSettings)); - readClients.put(id, client); - } - client.x().getAndIncrement(); + public Reader createReader( + BigtableSource source, Duration attemptTimeout, Duration operationTimeout) + throws IOException { if (source.getMaxBufferElementCount() != null) { return BigtableSegmentReaderImpl.create( - client.y(), - dataSettings.getProjectId(), - dataSettings.getInstanceId(), + client, + projectId, + instanceId, source.getTableId().get(), source.getRanges(), source.getRowFilter(), source.getMaxBufferElementCount(), - retrySettings.getInitialRpcTimeout(), - retrySettings.getTotalTimeout(), - id); + attemptTimeout, + operationTimeout); } else { return new BigtableReaderImpl( - client.y(), - dataSettings.getProjectId(), - dataSettings.getInstanceId(), + client, + projectId, + instanceId, source.getTableId().get(), source.getRanges(), source.getRowFilter(), - retrySettings.getInitialRpcTimeout(), - retrySettings.getTotalTimeout(), - id); - } - } - - private BigtableDataSettings configureReadSettings( - BigtableDataSettings.Builder settings, BigtableReadOptions readOptions) { - - RetrySettings.Builder retrySettings = - settings.stubSettings().bulkReadRowsSettings().getRetrySettings().toBuilder(); - - if (readOptions.getAttemptTimeout() != null) { - retrySettings.setInitialRpcTimeout(Duration.ofMillis(readOptions.getAttemptTimeout())); - - if (readOptions.getOperationTimeout() == null) { - retrySettings.setTotalTimeout( - Duration.ofMillis( - Math.max( - retrySettings.getTotalTimeout().toMillis(), readOptions.getAttemptTimeout()))); - } - } - - if (readOptions.getOperationTimeout() != null) { - retrySettings.setTotalTimeout(Duration.ofMillis(readOptions.getOperationTimeout())); - } - - if (readOptions.getRetryDelayMultiplier() != null) { - retrySettings.setRetryDelayMultiplier(readOptions.getRetryDelayMultiplier()); - } - - if (readOptions.getRetryInitialDelay() != null) { - retrySettings.setInitialRetryDelay(Duration.ofMillis(readOptions.getRetryInitialDelay())); + attemptTimeout, + operationTimeout); } - - settings.stubSettings().bulkReadRowsSettings().setRetrySettings(retrySettings.build()); - - return settings.build(); } // Support 2 bigtable-hbase features not directly available in veneer: @@ -718,30 +588,18 @@ private static GrpcCallContext createScanCallContext( } @Override - public List getSampleRowKeys(BigtableSource source) throws IOException { - try (BigtableDataClient client = BigtableDataClient.create(dataSettings)) { - return client.sampleRowKeys(source.getTableId().get()); - } + public List getSampleRowKeys(BigtableSource source) { + return client.sampleRowKeys(source.getTableId().get()); } @Override public String getProjectId() { - return config.getProjectId().get(); + return projectId; } @Override public String getInstanceId() { - return config.getInstanceId().get(); - } - - @Override - public Credentials getCredentials() { - return config.getCredentials(); - } - - @Override - public String getEmulatorHost() { - return config.getEmulatorHost(); + return instanceId; } @VisibleForTesting @@ -803,64 +661,6 @@ public int compareTo(@Nonnull StartPoint o) { } } - private BigtableDataSettings configureWriteSettings( - BigtableDataSettings.Builder settings, BigtableWriteOptions writeOptions) { - BigtableBatchingCallSettings.Builder callSettings = - settings.stubSettings().bulkMutateRowsSettings(); - RetrySettings.Builder retrySettings = callSettings.getRetrySettings().toBuilder(); - BatchingSettings.Builder batchingSettings = callSettings.getBatchingSettings().toBuilder(); - if (writeOptions.getAttemptTimeout() != null) { - retrySettings.setInitialRpcTimeout(Duration.ofMillis(writeOptions.getAttemptTimeout())); - - if (writeOptions.getOperationTimeout() == null) { - retrySettings.setTotalTimeout( - Duration.ofMillis( - Math.max( - retrySettings.getTotalTimeout().toMillis(), writeOptions.getAttemptTimeout()))); - } - } - - if (writeOptions.getOperationTimeout() != null) { - retrySettings.setTotalTimeout(Duration.ofMillis(writeOptions.getOperationTimeout())); - } - - if (writeOptions.getRetryInitialDelay() != null) { - retrySettings.setInitialRetryDelay(Duration.ofMillis(writeOptions.getRetryInitialDelay())); - } - - if (writeOptions.getRetryDelayMultiplier() != null) { - retrySettings.setRetryDelayMultiplier(writeOptions.getRetryDelayMultiplier()); - } - - if (writeOptions.getBatchElements() != null) { - batchingSettings.setElementCountThreshold(writeOptions.getBatchElements()); - } - - if (writeOptions.getBatchBytes() != null) { - batchingSettings.setRequestByteThreshold(writeOptions.getBatchBytes()); - } - - if (writeOptions.getMaxRequests() != null) { - BatchingSettings tmpSettings = batchingSettings.build(); - batchingSettings.setFlowControlSettings( - callSettings - .getBatchingSettings() - .getFlowControlSettings() - .toBuilder() - .setMaxOutstandingRequestBytes( - tmpSettings.getElementCountThreshold() * writeOptions.getMaxRequests()) - .setMaxOutstandingRequestBytes( - tmpSettings.getRequestByteThreshold() * writeOptions.getMaxRequests()) - .build()); - } - - callSettings - .setBatchingSettings(batchingSettings.build()) - .setRetrySettings(retrySettings.build()); - - return settings.build(); - } - /** Helper class to ease comparison of RowRange endpoints. */ private static final class EndPoint implements Comparable { private final ByteString value; 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 fc1eebd521fa..d34f7a7d9298 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 @@ -82,6 +82,7 @@ public void testGcpApiSurface() throws Exception { Matchers.equalTo(com.google.api.gax.rpc.ApiException.class), Matchers.equalTo(com.google.api.gax.paging.Page.class), Matchers.>equalTo(com.google.api.gax.rpc.StatusCode.class), + Matchers.>equalTo(com.google.api.gax.core.CredentialsProvider.class), Matchers.>equalTo(com.google.common.base.Function.class), Matchers.>equalTo(com.google.common.base.Optional.class), Matchers.>equalTo(com.google.common.base.Supplier.class), diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTest.java index 6887c4e96f95..50c79a032771 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTest.java @@ -27,13 +27,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import com.google.auth.Credentials; import com.google.cloud.bigtable.config.BigtableOptions; -import com.google.cloud.bigtable.config.BulkOptions; -import com.google.cloud.bigtable.config.CredentialOptions; -import org.apache.beam.sdk.extensions.gcp.auth.TestCredential; -import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; @@ -113,14 +107,6 @@ public void testWithValidate() { assertTrue(config.withValidate(true).getValidate()); } - @Test - public void testWithBigtableService() { - assertEquals(SERVICE, config.withBigtableService(SERVICE).getBigtableService()); - - thrown.expect(IllegalArgumentException.class); - config.withBigtableService(null); - } - @Test public void testValidate() { config.withProjectId(PROJECT_ID).withInstanceId(INSTANCE_ID).validate(); @@ -171,30 +157,6 @@ public void testPopulateDisplayData() { hasValue(INSTANCE_ID.get())))); } - @Test - public void testGetBigtableServiceWithDefaultService() { - assertEquals(SERVICE, config.withBigtableService(SERVICE).getBigtableService()); - } - - @Test - public void testGetBigtableServiceWithConfigurator() { - SerializableFunction configurator = - (SerializableFunction) - input -> - input - .setInstanceId(INSTANCE_ID.get()) - .setProjectId(PROJECT_ID.get()) - .setBulkOptions(new BulkOptions.Builder().setUseBulkApi(true).build()); - - BigtableService service = - config - .withBigtableOptionsConfigurator(configurator) - .getBigtableService(PipelineOptionsFactory.as(GcpOptions.class)); - - assertEquals(PROJECT_ID.get(), service.getProjectId()); - assertEquals(INSTANCE_ID.get(), service.getInstanceId()); - } - @Test public void testIsDataAccessible() { assertTrue(config.withProjectId(PROJECT_ID).withInstanceId(INSTANCE_ID).isDataAccessible()); @@ -222,25 +184,4 @@ public void testIsDataAccessible() { assertFalse( config.withProjectId(PROJECT_ID).withInstanceId(NOT_ACCESSIBLE_VALUE).isDataAccessible()); } - - @Test - public void testBigtableOptionsAreTranslated() { - BigtableOptions.Builder optionsToTest = BigtableOptions.builder(); - - Credentials credentials = new TestCredential(); - optionsToTest - .enableEmulator("localhost", 1234) - .setCredentialOptions(CredentialOptions.credential(credentials)); - - BigtableService service = - config - .withProjectId(PROJECT_ID) - .withInstanceId(INSTANCE_ID) - .withBigtableOptions(optionsToTest.build()) - .withValidate(true) - .getBigtableService(PipelineOptionsFactory.as(GcpOptions.class)); - - assertEquals(credentials, service.getCredentials()); - assertEquals("localhost:1234", service.getEmulatorHost()); - } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java new file mode 100644 index 000000000000..1a12d476f9d1 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java @@ -0,0 +1,257 @@ +/* + * 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.bigtable; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +import com.google.cloud.bigtable.config.BigtableOptions; +import com.google.cloud.bigtable.config.BulkOptions; +import com.google.cloud.bigtable.config.CallOptionsConfig; +import com.google.cloud.bigtable.config.CredentialOptions; +import com.google.cloud.bigtable.config.RetryOptions; +import com.google.cloud.bigtable.data.v2.BigtableDataSettings; +import com.google.cloud.bigtable.data.v2.stub.EnhancedBigtableStubSettings; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.options.ValueProvider; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.threeten.bp.Duration; + +/** Unit tests for {@link BigtableConfigTranslator}. */ +@RunWith(JUnit4.class) +public class BigtableConfigTranslatorTest { + + @Test + public void testBigtableOptionsToBigtableConfig() throws Exception { + BigtableOptions options = + BigtableOptions.builder() + .setProjectId("project") + .setInstanceId("instance") + .setAppProfileId("app-profile") + .setDataHost("localhost") + .setPort(1234) + .setCredentialOptions(CredentialOptions.nullCredential()) + .build(); + + BigtableConfig config = BigtableConfig.builder().setValidate(true).build(); + config = BigtableConfigTranslator.translateToBigtableConfig(config, options); + + assertNotNull(config.getProjectId()); + assertNotNull(config.getInstanceId()); + assertNotNull(config.getAppProfileId()); + assertNotNull(config.getEmulatorHost()); + assertNotNull(config.getCredentialsProvider()); + + assertEquals(options.getProjectId(), config.getProjectId().get()); + assertEquals(options.getInstanceId(), config.getInstanceId().get()); + assertEquals(options.getAppProfileId(), config.getAppProfileId().get()); + assertEquals("localhost:1234", config.getEmulatorHost()); + assertNull(config.getCredentialsProvider().getCredentials()); + } + + @Test + public void testBigtableOptionsToBigtableReadOptions() { + BigtableOptions options = + BigtableOptions.builder() + .setCallOptionsConfig( + CallOptionsConfig.builder() + .setReadRowsRpcAttemptTimeoutMs(100) + .setReadRowsRpcTimeoutMs(1000) + .build()) + .setRetryOptions( + RetryOptions.builder().setInitialBackoffMillis(5).setBackoffMultiplier(1.5).build()) + .build(); + + BigtableReadOptions readOptions = + BigtableReadOptions.builder() + .setTableId(ValueProvider.StaticValueProvider.of("table")) + .build(); + + readOptions = BigtableConfigTranslator.translateToBigtableReadOptions(readOptions, options); + + assertNotNull(readOptions.getAttemptTimeout()); + assertNotNull(readOptions.getOperationTimeout()); + assertNotNull(readOptions.getRetryInitialDelay()); + assertNotNull(readOptions.getRetryDelayMultiplier()); + + assertEquals(100, (long) readOptions.getAttemptTimeout()); + assertEquals(1000, (long) readOptions.getOperationTimeout()); + assertEquals(5, (long) readOptions.getRetryInitialDelay()); + assertEquals(1.5, (double) readOptions.getRetryDelayMultiplier(), 0); + } + + @Test + public void testBigtableOptionsToBigtableWriteOptions() { + BigtableOptions options = + BigtableOptions.builder() + .setCallOptionsConfig( + CallOptionsConfig.builder() + .setMutateRpcAttemptTimeoutMs(200) + .setMutateRpcTimeoutMs(2000) + .build()) + .setRetryOptions( + RetryOptions.builder() + .setInitialBackoffMillis(15) + .setBackoffMultiplier(2.5) + .build()) + .setBulkOptions( + BulkOptions.builder() + .setBulkMaxRequestSize(20) + .setBulkMaxRowKeyCount(100) + .setMaxInflightRpcs(5) + .build()) + .build(); + + BigtableWriteOptions writeOptions = + BigtableWriteOptions.builder() + .setTableId(ValueProvider.StaticValueProvider.of("table")) + .build(); + + writeOptions = BigtableConfigTranslator.translateToBigtableWriteOptions(writeOptions, options); + + assertNotNull(writeOptions.getAttemptTimeout()); + assertNotNull(writeOptions.getOperationTimeout()); + assertNotNull(writeOptions.getRetryInitialDelay()); + assertNotNull(writeOptions.getRetryDelayMultiplier()); + assertNotNull(writeOptions.getBatchBytes()); + assertNotNull(writeOptions.getBatchElements()); + assertNotNull(writeOptions.getMaxRequests()); + + assertEquals(200, (long) writeOptions.getAttemptTimeout()); + assertEquals(2000, (long) writeOptions.getOperationTimeout()); + assertEquals(15, (long) writeOptions.getRetryInitialDelay()); + assertEquals(2.5, writeOptions.getRetryDelayMultiplier(), 0); + assertEquals(20, (long) writeOptions.getBatchBytes()); + assertEquals(100, (long) writeOptions.getBatchElements()); + assertEquals(5, (long) writeOptions.getMaxRequests()); + } + + @Test + public void testVeneerReadSettings() { + BigtableConfig config = + BigtableConfig.builder() + .setProjectId(ValueProvider.StaticValueProvider.of("project")) + .setInstanceId(ValueProvider.StaticValueProvider.of("instance")) + .setAppProfileId(ValueProvider.StaticValueProvider.of("app")) + .setValidate(true) + .build(); + BigtableReadOptions readOptions = + BigtableReadOptions.builder() + .setTableId(ValueProvider.StaticValueProvider.of("table")) + .setAttemptTimeout(101) + .setOperationTimeout(1001) + .setRetryInitialDelay(5) + .setRetryDelayMultiplier(1.5) + .build(); + PipelineOptions pipelineOptions = PipelineOptionsFactory.create(); + + BigtableDataSettings settings = + BigtableConfigTranslator.translateReadToVeneerSettings( + config, readOptions, pipelineOptions); + + EnhancedBigtableStubSettings stubSettings = settings.getStubSettings(); + + assertEquals(config.getProjectId().get(), stubSettings.getProjectId()); + assertEquals(config.getInstanceId().get(), stubSettings.getInstanceId()); + assertEquals(config.getAppProfileId().get(), stubSettings.getAppProfileId()); + assertEquals( + Duration.ofMillis(101), + stubSettings.bulkReadRowsSettings().getRetrySettings().getInitialRpcTimeout()); + assertEquals( + Duration.ofMillis(1001), + stubSettings.bulkReadRowsSettings().getRetrySettings().getTotalTimeout()); + assertEquals( + Duration.ofMillis(5), + stubSettings.bulkReadRowsSettings().getRetrySettings().getInitialRetryDelay()); + assertEquals( + 1.5, stubSettings.bulkReadRowsSettings().getRetrySettings().getRetryDelayMultiplier(), 0); + } + + @Test + public void testVeneerWriteSettings() { + BigtableConfig config = + BigtableConfig.builder() + .setProjectId(ValueProvider.StaticValueProvider.of("project")) + .setInstanceId(ValueProvider.StaticValueProvider.of("instance")) + .setAppProfileId(ValueProvider.StaticValueProvider.of("app")) + .setValidate(true) + .build(); + BigtableWriteOptions writeOptions = + BigtableWriteOptions.builder() + .setTableId(ValueProvider.StaticValueProvider.of("table")) + .setAttemptTimeout(101) + .setOperationTimeout(1001) + .setRetryInitialDelay(5) + .setRetryDelayMultiplier(1.5) + .setMaxRequests(11) + .setBatchElements(105) + .setBatchBytes(102) + .build(); + PipelineOptions pipelineOptions = PipelineOptionsFactory.create(); + + BigtableDataSettings settings = + BigtableConfigTranslator.translateWriteToVeneerSettings( + config, writeOptions, pipelineOptions); + + EnhancedBigtableStubSettings stubSettings = settings.getStubSettings(); + + assertEquals(config.getProjectId().get(), stubSettings.getProjectId()); + assertEquals(config.getInstanceId().get(), stubSettings.getInstanceId()); + assertEquals(config.getAppProfileId().get(), stubSettings.getAppProfileId()); + assertEquals( + Duration.ofMillis(101), + stubSettings.bulkMutateRowsSettings().getRetrySettings().getInitialRpcTimeout()); + assertEquals( + Duration.ofMillis(1001), + stubSettings.bulkMutateRowsSettings().getRetrySettings().getTotalTimeout()); + assertEquals( + Duration.ofMillis(5), + stubSettings.bulkMutateRowsSettings().getRetrySettings().getInitialRetryDelay()); + assertEquals( + 1.5, stubSettings.bulkMutateRowsSettings().getRetrySettings().getRetryDelayMultiplier(), 0); + assertEquals( + 105, + (long) + stubSettings.bulkMutateRowsSettings().getBatchingSettings().getElementCountThreshold()); + assertEquals( + 102, + (long) + stubSettings.bulkMutateRowsSettings().getBatchingSettings().getRequestByteThreshold()); + assertEquals( + 105 * 11, + (long) + stubSettings + .bulkMutateRowsSettings() + .getBatchingSettings() + .getFlowControlSettings() + .getMaxOutstandingElementCount()); + + assertEquals( + 102 * 11, + (long) + stubSettings + .bulkMutateRowsSettings() + .getBatchingSettings() + .getFlowControlSettings() + .getMaxOutstandingRequestBytes()); + } +} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java index 67fe82d7b67d..b174860c1e98 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java @@ -40,7 +40,6 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; -import com.google.auth.Credentials; import com.google.auto.value.AutoValue; import com.google.bigtable.v2.Cell; import com.google.bigtable.v2.Column; @@ -52,7 +51,6 @@ import com.google.bigtable.v2.RowFilter; import com.google.cloud.bigtable.config.BigtableOptions; import com.google.cloud.bigtable.config.BulkOptions; -import com.google.cloud.bigtable.config.CredentialOptions; import com.google.cloud.bigtable.config.RetryOptions; import com.google.cloud.bigtable.data.v2.models.KeyOffset; import com.google.protobuf.ByteString; @@ -79,7 +77,6 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarLongCoder; -import org.apache.beam.sdk.extensions.gcp.auth.TestCredential; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.extensions.protobuf.ByteStringCoder; import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; @@ -173,10 +170,19 @@ public boolean isAccessible() { .setProjectId("options_project") .setInstanceId("options_instance") .build(); + + private static final BigtableServiceFactory.ConfigId configId = + BigtableServiceFactory.FACTORY_INSTANCE.newId(); private static BigtableIO.Read defaultRead = - BigtableIO.read().withInstanceId("instance").withProjectId("project"); + BigtableIO.read() + .withInstanceId("instance") + .withProjectId("project") + .withConfigId(configId.id()); private static BigtableIO.Write defaultWrite = - BigtableIO.write().withInstanceId("instance").withProjectId("project"); + BigtableIO.write() + .withInstanceId("instance") + .withProjectId("project") + .withConfigId(configId.id()); private Coder>> bigtableCoder; private static final TypeDescriptor>> BIGTABLE_WRITE_TYPE = new TypeDescriptor>>() {}; @@ -190,11 +196,13 @@ public boolean isAccessible() { @Before public void setup() throws Exception { service = new FakeBigtableService(); - defaultRead = defaultRead.withBigtableService(service); - defaultWrite = defaultWrite.withBigtableService(service); + + BigtableServiceFactory.FACTORY_INSTANCE.addFakeReadService(configId, service); + BigtableServiceFactory.FACTORY_INSTANCE.addFakeWriteService(configId, service); + bigtableCoder = p.getCoderRegistry().getCoder(BIGTABLE_WRITE_TYPE); - config = BigtableConfig.builder().setValidate(true).setBigtableService(service).build(); + config = BigtableConfig.builder().setValidate(true).build(); } private static ByteKey makeByteKey(ByteString key) { @@ -301,13 +309,18 @@ public void testReadWithRuntimeParametersValidationDisabled() { public void testReadWithReaderStartFailed() throws IOException { FailureBigtableService failureService = new FailureBigtableService(FailureOptions.builder().setFailAtStart(true).build()); - BigtableConfig failureConfig = - BigtableConfig.builder().setValidate(true).setBigtableService(failureService).build(); + + BigtableServiceFactory.ConfigId newId = BigtableServiceFactory.FACTORY_INSTANCE.newId(); + + BigtableServiceFactory.FACTORY_INSTANCE.addFakeReadService(newId, failureService); + + BigtableConfig failureConfig = BigtableConfig.builder().setValidate(true).build(); final String table = "TEST-TABLE"; final int numRows = 100; makeTableData(failureService, table, numRows); BigtableSource source = new BigtableSource( + newId, failureConfig, BigtableReadOptions.builder() .setTableId(StaticValueProvider.of(table)) @@ -326,13 +339,18 @@ public void testReadWithReaderStartFailed() throws IOException { public void testReadWithReaderAdvanceFailed() throws IOException { FailureBigtableService failureService = new FailureBigtableService(FailureOptions.builder().setFailAtAdvance(true).build()); - BigtableConfig failureConfig = - BigtableConfig.builder().setValidate(true).setBigtableService(failureService).build(); + + BigtableServiceFactory.ConfigId newId = BigtableServiceFactory.FACTORY_INSTANCE.newId(); + + BigtableServiceFactory.FACTORY_INSTANCE.addFakeReadService(newId, failureService); + + BigtableConfig failureConfig = BigtableConfig.builder().setValidate(true).build(); final String table = "TEST-TABLE"; final int numRows = 100; makeTableData(failureService, table, numRows); BigtableSource source = new BigtableSource( + newId, failureConfig, BigtableReadOptions.builder() .setTableId(StaticValueProvider.of(table)) @@ -430,74 +448,23 @@ private static KV> makeBadWrite(String key) { return KV.of(ByteString.copyFromUtf8(key), mutations); } - /** Tests that credentials are used from PipelineOptions if not supplied by BigtableOptions. */ - @Test - public void testUsePipelineOptionsCredentialsIfNotSpecifiedInBigtableOptions() { - GcpOptions pipelineOptions = PipelineOptionsFactory.as(GcpOptions.class); - Credentials credentials = new TestCredential(); - pipelineOptions.setGcpCredential(credentials); - BigtableService readService = - BigtableIO.read() - .withProjectId("project") - .withInstanceId("instance") - .withTableId("TEST-TABLE") - .getBigtableConfig() - .getBigtableService(pipelineOptions); - BigtableService writeService = - BigtableIO.write() - .withProjectId("project") - .withInstanceId("instance") - .withProjectId("TEST-PROJECT") - .withInstanceId("TEST-INSTANCE") - .withTableId("TEST-TABLE") - .getBigtableConfig() - .getBigtableService(pipelineOptions); - assertEquals(credentials, readService.getCredentials()); - assertEquals(credentials, writeService.getCredentials()); - } - - /** Tests that credentials are not used from PipelineOptions if supplied by BigtableOptions. */ - @Test - public void testDontUsePipelineOptionsCredentialsIfSpecifiedInBigtableOptions() throws Exception { - BigtableOptions options = - BIGTABLE_OPTIONS - .toBuilder() - .setCredentialOptions(CredentialOptions.nullCredential()) - .build(); - GcpOptions pipelineOptions = PipelineOptionsFactory.as(GcpOptions.class); - pipelineOptions.setGcpCredential(new TestCredential()); - BigtableService readService = - BigtableIO.read() - .withBigtableOptions(options) - .withTableId("TEST-TABLE") - .getBigtableConfig() - .getBigtableService(pipelineOptions); - BigtableService writeService = - BigtableIO.write() - .withBigtableOptions(options) - .withTableId("TEST-TABLE") - .getBigtableConfig() - .getBigtableService(pipelineOptions); - assertEquals(null, readService.getCredentials()); - assertEquals(null, writeService.getCredentials()); - } - /** Tests that when reading from a non-existent table, the read fails. */ @Test public void testReadingFailsTableDoesNotExist() throws Exception { final String table = "TEST-TABLE"; + BigtableServiceFactory.ConfigId newId = BigtableServiceFactory.FACTORY_INSTANCE.newId(); + + BigtableServiceFactory.FACTORY_INSTANCE.addFakeReadService(newId, service); + BigtableIO.Read read = - BigtableIO.read() - .withBigtableOptions(BIGTABLE_OPTIONS) - .withTableId(table) - .withBigtableService(service); + BigtableIO.read().withBigtableOptions(BIGTABLE_OPTIONS).withTableId(table); // Exception will be thrown by read.validate() when read is applied. thrown.expect(IllegalArgumentException.class); thrown.expectMessage(String.format("Table %s does not exist", table)); - p.apply(read); + p.apply(read.withConfigId(newId.id())); p.run(); } @@ -661,7 +628,7 @@ public void testReadingWithKeyRanges() throws Exception { /** Tests reading all rows using a filter. */ @Test - public void testReadingWithFilter() throws Exception { + public void testReadingWithFilter() { final String table = "TEST-FILTER-TABLE"; final int numRows = 1001; List testRows = makeTableData(table, numRows); @@ -721,6 +688,7 @@ public void testReadingSplitAtFractionExhaustive() throws Exception { BigtableSource source = new BigtableSource( + configId, config, BigtableReadOptions.builder() .setTableId(StaticValueProvider.of(table)) @@ -743,6 +711,7 @@ public void testReadingSplitAtFraction() throws Exception { BigtableSource source = new BigtableSource( + configId, config, BigtableReadOptions.builder() .setTableId(StaticValueProvider.of(table)) @@ -780,6 +749,7 @@ public void testReadingWithSplits() throws Exception { // Generate source and split it. BigtableSource source = new BigtableSource( + configId, config, BigtableReadOptions.builder() .setTableId(StaticValueProvider.of(table)) @@ -798,8 +768,12 @@ public void testReadingWithSplits() throws Exception { public void testReadingWithSplitFailed() throws Exception { FailureBigtableService failureService = new FailureBigtableService(FailureOptions.builder().setFailAtSplit(true).build()); - BigtableConfig failureConfig = - BigtableConfig.builder().setValidate(true).setBigtableService(failureService).build(); + + BigtableServiceFactory.ConfigId newId = BigtableServiceFactory.FACTORY_INSTANCE.newId(); + + BigtableServiceFactory.FACTORY_INSTANCE.addFakeReadService(newId, failureService); + + BigtableConfig failureConfig = BigtableConfig.builder().setValidate(true).build(); final String table = "TEST-MANY-ROWS-SPLITS-TABLE"; final int numRows = 1500; @@ -813,6 +787,7 @@ public void testReadingWithSplitFailed() throws Exception { // Generate source and split it. BigtableSource source = new BigtableSource( + newId, failureConfig, BigtableReadOptions.builder() .setTableId(StaticValueProvider.of(table)) @@ -882,6 +857,7 @@ public void testReduceSplitsWithSomeNonAdjacentRanges() throws Exception { // Generate source and split it. BigtableSource source = new BigtableSource( + configId, config, BigtableReadOptions.builder() .setTableId(StaticValueProvider.of(table)) @@ -935,6 +911,7 @@ public void testReduceSplitsWithAllNonAdjacentRange() throws Exception { // Generate source and split it. BigtableSource source = new BigtableSource( + configId, config, BigtableReadOptions.builder() .setTableId(StaticValueProvider.of(table)) @@ -979,6 +956,7 @@ public void tesReduceSplitsWithAdjacentRanges() throws Exception { // Generate source and split it. BigtableSource source = new BigtableSource( + configId, config, BigtableReadOptions.builder() .setTableId(StaticValueProvider.of(table)) @@ -1053,6 +1031,7 @@ public void testReadingWithSplitsWithSeveralKeyRanges() throws Exception { // Generate source and split it. BigtableSource source = new BigtableSource( + configId, config, BigtableReadOptions.builder() .setTableId(StaticValueProvider.of(table)) @@ -1061,6 +1040,7 @@ public void testReadingWithSplitsWithSeveralKeyRanges() throws Exception { null /*size*/); BigtableSource referenceSource = new BigtableSource( + configId, config, BigtableReadOptions.builder() .setTableId(StaticValueProvider.of(table)) @@ -1092,6 +1072,7 @@ public void testReadingWithSubSplits() throws Exception { // Generate source and split it. BigtableSource source = new BigtableSource( + configId, config, BigtableReadOptions.builder() .setTableId(StaticValueProvider.of(table)) @@ -1137,6 +1118,7 @@ public void testReadingWithSubSplitsWithSeveralKeyRanges() throws Exception { // Generate source and split it. BigtableSource source = new BigtableSource( + configId, config, BigtableReadOptions.builder() .setTableId(StaticValueProvider.of(table)) @@ -1145,6 +1127,7 @@ public void testReadingWithSubSplitsWithSeveralKeyRanges() throws Exception { null /*size*/); BigtableSource referenceSource = new BigtableSource( + configId, config, BigtableReadOptions.builder() .setTableId(StaticValueProvider.of(table)) @@ -1177,6 +1160,7 @@ public void testReadingWithFilterAndSubSplits() throws Exception { RowFilter.newBuilder().setRowKeyRegexFilter(ByteString.copyFromUtf8(".*17.*")).build(); BigtableSource source = new BigtableSource( + configId, config, BigtableReadOptions.builder() .setTableId(StaticValueProvider.of(table)) @@ -1260,7 +1244,6 @@ public void testReadWithoutValidate() { BigtableIO.read() .withBigtableOptions(BIGTABLE_OPTIONS) .withTableId(table) - .withBigtableService(service) .withoutValidation(); // validate() will throw if withoutValidation() isn't working @@ -1274,7 +1257,6 @@ public void testWriteWithoutValidate() { BigtableIO.write() .withBigtableOptions(BIGTABLE_OPTIONS) .withTableId(table) - .withBigtableService(service) .withoutValidation(); // validate() will throw if withoutValidation() isn't working @@ -1305,7 +1287,7 @@ public void testWriting() throws Exception { /** Tests that at least one result is emitted per element written in the global window. */ @Test - public void testWritingEmitsResultsWhenDoneInGlobalWindow() throws Exception { + public void testWritingEmitsResultsWhenDoneInGlobalWindow() { final String table = "table"; final String key = "key"; final String value = "value"; @@ -1327,7 +1309,7 @@ public void testWritingEmitsResultsWhenDoneInGlobalWindow() throws Exception { * Wait.on transform as the trigger. */ @Test - public void testWritingAndWaitingOnResults() throws Exception { + public void testWritingAndWaitingOnResults() { final String table = "table"; final String key = "key"; final String value = "value"; @@ -1454,11 +1436,16 @@ public void testTableCheckIgnoredWhenCanNotAccessConfig() throws Exception { public void testWritingFailsAtWriteRecord() throws IOException { FailureBigtableService failureService = new FailureBigtableService(FailureOptions.builder().setFailAtWriteRecord(true).build()); + + BigtableServiceFactory.ConfigId newId = BigtableServiceFactory.FACTORY_INSTANCE.newId(); + + BigtableServiceFactory.FACTORY_INSTANCE.addFakeWriteService(newId, failureService); + BigtableIO.Write failureWrite = BigtableIO.write() .withInstanceId("instance") .withProjectId("project") - .withBigtableService(failureService); + .withConfigId(newId.id()); final String table = "table"; final String key = "key"; @@ -1519,6 +1506,7 @@ public void testGetSplitPointsConsumed() throws Exception { BigtableSource source = new BigtableSource( + configId, config, BigtableReadOptions.builder() .setTableId(StaticValueProvider.of(table)) @@ -1660,13 +1648,15 @@ public void verifyTableExists(String tableId) { } @Override - public FakeBigtableReader createReader(BigtableSource source, int id) { + public FakeBigtableReader createReader( + BigtableSource source, + org.threeten.bp.Duration attemptTimeout, + org.threeten.bp.Duration operationTimeout) { return new FakeBigtableReader(source); } @Override - public FakeBigtableWriter openForWriting( - String tableId, BigtableWriteOptions writeOptions, int id) { + public FakeBigtableWriter openForWriting(String tableId) { return new FakeBigtableWriter(tableId); } @@ -1687,16 +1677,6 @@ public String getInstanceId() { return null; } - @Override - public Credentials getCredentials() { - return null; - } - - @Override - public String getEmulatorHost() { - return null; - } - /** Sets up the sample row keys for the specified table. */ void setupSampleRowKeys(String tableId, int numSamples, long bytesPerRow) { verifyTableExists(tableId); @@ -1731,13 +1711,15 @@ public FailureBigtableService(FailureOptions options) { } @Override - public FakeBigtableReader createReader(BigtableSource source, int id) { + public FakeBigtableReader createReader( + BigtableSource source, + org.threeten.bp.Duration attemptTimeout, + org.threeten.bp.Duration operationTimeout) { return new FailureBigtableReader(source, this, failureOptions); } @Override - public FailureBigtableWriter openForWriting( - String tableId, BigtableWriteOptions writeOptions, int id) { + public FailureBigtableWriter openForWriting(String tableId) { return new FailureBigtableWriter(tableId, this, failureOptions); } @@ -1831,12 +1813,6 @@ public Row getCurrentRow() { } return currentRow; } - - @Override - public void close() { - rows = null; - currentRow = null; - } } /** A {@link FakeBigtableReader} implementation that throw exceptions at given stage. */ diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java index 28f8d82351c0..8be6146379ca 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java @@ -181,8 +181,7 @@ public void testRead() throws IOException { mockBigtableSource.getRanges(), null, retrySettings.getInitialRpcTimeout(), - retrySettings.getTotalTimeout(), - 0); + retrySettings.getTotalTimeout()); underTest.start(); Assert.assertEquals(expectedRow, underTest.getCurrentRow()); @@ -244,8 +243,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { DEFAULT_BYTE_SEGMENT_SIZE, retrySettings.getInitialRpcTimeout(), retrySettings.getTotalTimeout(), - mockCallMetric, - 0); + mockCallMetric); underTest.start(); Assert.assertEquals(expectedRow, underTest.getCurrentRow()); @@ -302,8 +300,7 @@ public void testReadSingleRangeAboveSegmentLimit() throws IOException { DEFAULT_BYTE_SEGMENT_SIZE, retrySettings.getInitialRpcTimeout(), retrySettings.getTotalTimeout(), - mockCallMetric, - 0); + mockCallMetric); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); @@ -373,8 +370,7 @@ public void testReadMultipleRanges() throws IOException { DEFAULT_BYTE_SEGMENT_SIZE, retrySettings.getInitialRpcTimeout(), retrySettings.getTotalTimeout(), - mockCallMetric, - 0); + mockCallMetric); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); @@ -445,8 +441,7 @@ public void testReadMultipleRangesOverlappingKeys() throws IOException { DEFAULT_BYTE_SEGMENT_SIZE, retrySettings.getInitialRpcTimeout(), retrySettings.getTotalTimeout(), - mockCallMetric, - 0); + mockCallMetric); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); @@ -502,8 +497,7 @@ public void testReadFullTableScan() throws IOException { DEFAULT_BYTE_SEGMENT_SIZE, retrySettings.getInitialRpcTimeout(), retrySettings.getTotalTimeout(), - mockCallMetric, - 0); + mockCallMetric); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); @@ -574,8 +568,7 @@ public void testReadFillBuffer() throws IOException { DEFAULT_BYTE_SEGMENT_SIZE, retrySettings.getInitialRpcTimeout(), retrySettings.getTotalTimeout(), - mockCallMetric, - 0); + mockCallMetric); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); @@ -664,8 +657,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { segmentByteLimit, retrySettings.getInitialRpcTimeout(), retrySettings.getTotalTimeout(), - mockCallMetric, - 0); + mockCallMetric); List actualResults = new ArrayList<>(); Assert.assertTrue(underTest.start()); @@ -731,8 +723,7 @@ public void run() { DEFAULT_BYTE_SEGMENT_SIZE, retrySettings.getInitialRpcTimeout(), retrySettings.getTotalTimeout(), - mockCallMetric, - 0); + mockCallMetric); IOException returnedError = null; try { @@ -766,8 +757,7 @@ public void testWrite() throws IOException, InterruptedException { mockBigtableDataClient, bigtableDataSettings.getProjectId(), bigtableDataSettings.getInstanceId(), - TABLE_ID, - 0); + TABLE_ID); ByteString key = ByteString.copyFromUtf8("key"); Mutation mutation = diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java index 77bae9f054cd..b2285b0951ad 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java @@ -25,6 +25,7 @@ import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; import com.google.cloud.bigtable.admin.v2.models.Table; import com.google.cloud.bigtable.data.v2.BigtableDataClient; +import com.google.cloud.bigtable.data.v2.BigtableDataSettings; import com.google.cloud.bigtable.data.v2.models.Query; import com.google.cloud.bigtable.data.v2.models.Row; import com.google.protobuf.ByteString; @@ -61,7 +62,7 @@ public class BigtableWriteIT implements Serializable { private static final String COLUMN_FAMILY_NAME = "cf"; private static BigtableTestOptions options; - private static BigtableConfigToVeneerSettings veneerSettings; + private static BigtableDataSettings veneerSettings; private BigtableConfig bigtableConfig; private static BigtableDataClient client; private static BigtableTableAdminClient tableAdminClient; @@ -82,10 +83,13 @@ public void setup() throws Exception { .setUserAgent("apache-beam-test") .build(); - veneerSettings = BigtableConfigToVeneerSettings.create(bigtableConfig); + veneerSettings = + BigtableConfigTranslator.translateWriteToVeneerSettings( + bigtableConfig, + BigtableWriteOptions.builder().build(), + PipelineOptionsFactory.create()); - client = BigtableDataClient.create(veneerSettings.getDataSettings()); - tableAdminClient = BigtableTableAdminClient.create(veneerSettings.getTableAdminSettings()); + client = BigtableDataClient.create(veneerSettings); } @Test From a1dc6b5eb7b7242ff413895bfb36e5825123a075 Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Wed, 15 Feb 2023 23:18:54 -0500 Subject: [PATCH 08/23] clean up comments --- .../apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java | 8 +++----- .../sdk/io/gcp/bigtable/BigtableConfigTranslator.java | 4 ++-- .../org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java | 6 ++---- .../beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java | 4 ---- .../sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java | 8 ++++---- 5 files changed, 11 insertions(+), 19 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java index 071956a0fb14..f0e1fa072d06 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java @@ -63,7 +63,7 @@ public abstract class BigtableConfig implements Serializable { /** Weather validate that table exists before writing. */ abstract boolean getValidate(); - /** Bigtable emulator. Used only for testing. */ + /** Bigtable emulator. */ abstract @Nullable String getEmulatorHost(); /** User agent for this job. */ @@ -72,6 +72,8 @@ public abstract class BigtableConfig implements Serializable { /** Credentials for running the job. */ abstract @Nullable CredentialsProvider getCredentialsProvider(); + /** Get the config id for the current config. Used only for testing. */ + @VisibleForTesting abstract @Nullable Integer getConfigId(); abstract Builder toBuilder(); @@ -104,10 +106,6 @@ abstract Builder setBigtableOptionsConfigurator( abstract Builder setUserAgent(String userAgent); - // TODO pass in CredentialProvider - // - defulat credentials - // - user overriding default credentials - // - no credentials (maybe ok) abstract Builder setCredentialsProvider(CredentialsProvider credentialsProvider); @VisibleForTesting diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java index ba5b99179853..f5baad5e746f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java @@ -189,7 +189,7 @@ private static BigtableDataSettings configureReadSettings( BigtableDataSettings.Builder settings, BigtableReadOptions readOptions) { RetrySettings.Builder retrySettings = - settings.stubSettings().bulkReadRowsSettings().getRetrySettings().toBuilder(); + settings.stubSettings().readRowsSettings().getRetrySettings().toBuilder(); if (readOptions.getAttemptTimeout() != null) { retrySettings.setInitialRpcTimeout(Duration.ofMillis(readOptions.getAttemptTimeout())); @@ -214,7 +214,7 @@ private static BigtableDataSettings configureReadSettings( retrySettings.setInitialRetryDelay(Duration.ofMillis(readOptions.getRetryInitialDelay())); } - settings.stubSettings().bulkReadRowsSettings().setRetrySettings(retrySettings.build()); + settings.stubSettings().readRowsSettings().setRetrySettings(retrySettings.build()); return settings.build(); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 9dedf21323d7..22858c9966a3 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -637,7 +637,6 @@ public PCollection expand(PBegin input) { getBigtableConfig().validate(); getBigtableReadOptions().validate(); - // Generate a new configId when the configuration is fixed if (config.getConfigId() != null) { this.configId = BigtableServiceFactory.ConfigId.create(config.getConfigId()); } else { @@ -677,6 +676,7 @@ private void validateTableExists( FACTORY_INSTANCE.getServiceForReading(configId, config, readOptions, options); checkArgument( entry.getService().tableExists(tableId), "Table %s does not exist", tableId); + FACTORY_INSTANCE.releaseReadService(entry); } catch (IOException e) { LOG.warn("Error checking whether table {} exists; proceeding.", tableId, e); } @@ -1095,6 +1095,7 @@ private void validateTableExists( FACTORY_INSTANCE.getServiceForWriting(configId, config, writeOptions, options); checkArgument( entry.getService().tableExists(tableId), "Table %s does not exist", tableId); + FACTORY_INSTANCE.releaseWriteService(entry); } catch (IOException e) { LOG.warn("Error checking whether table {} exists; proceeding.", tableId, e); } @@ -1105,8 +1106,6 @@ private void validateTableExists( private static class BigtableWriterFn extends DoFn>, BigtableWriteResult> { - // uuid of each DoFn family - // current id private BigtableServiceFactory.ConfigId id; private BigtableServiceEntry serviceEntry; @@ -1125,7 +1124,6 @@ public void startBundle(StartBundleContext c) throws IOException { recordsWritten = 0; this.seenWindows = Maps.newHashMapWithExpectedSize(1); - // Check if the family id if theres a client, increment reference count if (bigtableWriter == null) { serviceEntry = FACTORY_INSTANCE.getServiceForWriting(id, config, writeOptions, c.getPipelineOptions()); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java index 40c5853e37fc..0d748503377b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java @@ -569,10 +569,6 @@ public Reader createReader( } } - // Support 2 bigtable-hbase features not directly available in veneer: - // - disabling timeouts - when timeouts are disabled, bigtable-hbase ignores user configured - // timeouts and forces 6 minute deadlines per attempt for all RPCs except scans. This is - // implemented by an interceptor. However the interceptor must be informed that this is a scan // - per attempt deadlines - veneer doesn't implement deadlines for attempts. To workaround this, // the timeouts are set per call in the ApiCallContext. However this creates a separate issue of // over running the operation deadline, so gRPC deadline is also set. diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java index 1a12d476f9d1..c73131acc565 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java @@ -175,15 +175,15 @@ public void testVeneerReadSettings() { assertEquals(config.getAppProfileId().get(), stubSettings.getAppProfileId()); assertEquals( Duration.ofMillis(101), - stubSettings.bulkReadRowsSettings().getRetrySettings().getInitialRpcTimeout()); + stubSettings.readRowsSettings().getRetrySettings().getInitialRpcTimeout()); assertEquals( Duration.ofMillis(1001), - stubSettings.bulkReadRowsSettings().getRetrySettings().getTotalTimeout()); + stubSettings.readRowsSettings().getRetrySettings().getTotalTimeout()); assertEquals( Duration.ofMillis(5), - stubSettings.bulkReadRowsSettings().getRetrySettings().getInitialRetryDelay()); + stubSettings.readRowsSettings().getRetrySettings().getInitialRetryDelay()); assertEquals( - 1.5, stubSettings.bulkReadRowsSettings().getRetrySettings().getRetryDelayMultiplier(), 0); + 1.5, stubSettings.readRowsSettings().getRetrySettings().getRetryDelayMultiplier(), 0); } @Test From 5b50bd102d5d8e684085bbcdee0bfaff837652a6 Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Fri, 17 Feb 2023 11:37:02 -0500 Subject: [PATCH 09/23] address comments part 1 --- .../sdk/io/gcp/bigtable/BigtableConfig.java | 12 -- .../bigtable/BigtableConfigTranslator.java | 25 +-- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 114 ++++++------- .../sdk/io/gcp/bigtable/BigtableService.java | 11 +- .../gcp/bigtable/BigtableServiceFactory.java | 127 ++++++++------- .../io/gcp/bigtable/BigtableServiceImpl.java | 51 ++++-- .../sdk/io/gcp/bigtable/BigtableIOTest.java | 152 ++++++++++++------ .../gcp/bigtable/BigtableServiceImplTest.java | 18 +-- 8 files changed, 311 insertions(+), 199 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java index f0e1fa072d06..1a33db8fb61c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java @@ -72,10 +72,6 @@ public abstract class BigtableConfig implements Serializable { /** Credentials for running the job. */ abstract @Nullable CredentialsProvider getCredentialsProvider(); - /** Get the config id for the current config. Used only for testing. */ - @VisibleForTesting - abstract @Nullable Integer getConfigId(); - abstract Builder toBuilder(); static BigtableConfig.Builder builder() { @@ -108,9 +104,6 @@ abstract Builder setBigtableOptionsConfigurator( abstract Builder setCredentialsProvider(CredentialsProvider credentialsProvider); - @VisibleForTesting - abstract Builder setConfigId(Integer id); - abstract BigtableConfig build(); } @@ -158,11 +151,6 @@ BigtableConfig withCredentialsProvider(CredentialsProvider provider) { return toBuilder().setCredentialsProvider(provider).build(); } - @VisibleForTesting - BigtableConfig withConfigId(int id) { - return toBuilder().setConfigId(id).build(); - } - void validate() { checkArgument( (getProjectId() != null diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java index f5baad5e746f..4e7d9431e883 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java @@ -70,6 +70,13 @@ static BigtableDataSettings translateWriteToVeneerSettings( return configureWriteSettings(settings, options); } + /** Translate BigtableConfig and BigtableWriteOptions to Veneer settings. */ + static BigtableDataSettings translateToVeneerSettings( + @Nonnull BigtableConfig config, @Nonnull PipelineOptions pipelineOptions) { + + return buildBigtableDataSettings(config, pipelineOptions).build(); + } + private static BigtableDataSettings.Builder buildBigtableDataSettings( BigtableConfig config, PipelineOptions pipelineOptions) { BigtableDataSettings.Builder dataBuilder; @@ -92,7 +99,8 @@ private static BigtableDataSettings.Builder buildBigtableDataSettings( dataBuilder .setProjectId(Objects.requireNonNull(config.getProjectId().get())) .setInstanceId(Objects.requireNonNull(config.getInstanceId().get())); - if (config.getAppProfileId() != null) { + if (config.getAppProfileId() != null + && !Strings.isNullOrEmpty(config.getAppProfileId().get())) { dataBuilder.setAppProfileId(Objects.requireNonNull(config.getAppProfileId().get())); } @@ -235,12 +243,10 @@ static BigtableConfig translateToBigtableConfig(BigtableConfig config, BigtableO builder.setAppProfileId(ValueProvider.StaticValueProvider.of(options.getAppProfileId())); } - if (!options.getDataHost().equals("bigtable.googleapis.com") + if (options.getCredentialOptions().getCredentialType() == CredentialOptions.CredentialType.None && config.getEmulatorHost() == null) { builder.setEmulatorHost(String.format("%s:%s", options.getDataHost(), options.getPort())); } - // - instance of credentials, P12, SUPPLIED, suppliedjson - credentails - // - default credentials , needs to be called on the worker GoogleCredentialProvider if (options.getCredentialOptions() != null) { try { @@ -283,14 +289,9 @@ static BigtableConfig translateToBigtableConfig(BigtableConfig config, BigtableO case SuppliedJson: CredentialOptions.JsonCredentialsOptions jsonCredentialsOptions = (CredentialOptions.JsonCredentialsOptions) credOptions; - synchronized (jsonCredentialsOptions) { - if (jsonCredentialsOptions.getCachedCredentials() == null) { - jsonCredentialsOptions.setCachedCredentails( - GoogleCredentials.fromStream(jsonCredentialsOptions.getInputStream())); - } - builder.setCredentialsProvider( - FixedCredentialsProvider.create(jsonCredentialsOptions.getCachedCredentials())); - } + builder.setCredentialsProvider( + FixedCredentialsProvider.create( + GoogleCredentials.fromStream(jsonCredentialsOptions.getInputStream()))); break; case None: builder.setCredentialsProvider(NoCredentialsProvider.create()); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 22858c9966a3..a4e18ce5b555 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -291,7 +291,8 @@ public abstract static class Read extends PTransform> { abstract BigtableReadOptions getBigtableReadOptions(); - private BigtableServiceFactory.ConfigId configId; + @VisibleForTesting + abstract BigtableServiceFactory getServiceFactory(); /** Returns the table being read from. */ public @Nullable String getTableId() { @@ -322,6 +323,7 @@ static Read create() { .setKeyRanges( StaticValueProvider.of(Collections.singletonList(ByteKeyRange.ALL_KEYS))) .build()) + .setServiceFactory(FACTORY_INSTANCE) .build(); } @@ -332,6 +334,8 @@ abstract static class Builder { abstract Builder setBigtableReadOptions(BigtableReadOptions bigtableReadOptions); + abstract Builder setServiceFactory(BigtableServiceFactory factory); + abstract Read build(); } @@ -625,26 +629,22 @@ public Read withRetryDelayMultiplier(double multiplier) { .build(); } - /** Set the configId for testing. */ - @VisibleForTesting - Read withConfigId(int id) { - return toBuilder().setBigtableConfig(getBigtableConfig().withConfigId(id)).build(); + Read withServiceFactory(BigtableServiceFactory factory) { + return toBuilder().setServiceFactory(factory).build(); } @Override public PCollection expand(PBegin input) { - BigtableConfig config = this.getBigtableConfig(); getBigtableConfig().validate(); getBigtableReadOptions().validate(); - if (config.getConfigId() != null) { - this.configId = BigtableServiceFactory.ConfigId.create(config.getConfigId()); - } else { - this.configId = FACTORY_INSTANCE.newId(); - } - BigtableSource source = - new BigtableSource(configId, getBigtableConfig(), getBigtableReadOptions(), null); + new BigtableSource( + getServiceFactory(), + getServiceFactory().newId(), + getBigtableConfig(), + getBigtableReadOptions(), + null); return input.getPipeline().apply(org.apache.beam.sdk.io.Read.from(source)); } @@ -672,11 +672,10 @@ private void validateTableExists( if (config.getValidate() && config.isDataAccessible() && readOptions.isDataAccessible()) { String tableId = checkNotNull(readOptions.getTableId().get()); try { - BigtableServiceEntry entry = - FACTORY_INSTANCE.getServiceForReading(configId, config, readOptions, options); checkArgument( - entry.getService().tableExists(tableId), "Table %s does not exist", tableId); - FACTORY_INSTANCE.releaseReadService(entry); + getServiceFactory().checkTableExists(config, options, tableId), + "Table %s does not exist", + tableId); } catch (IOException e) { LOG.warn("Error checking whether table {} exists; proceeding.", tableId, e); } @@ -712,6 +711,9 @@ public abstract static class Write abstract BigtableWriteOptions getBigtableWriteOptions(); + @VisibleForTesting + abstract BigtableServiceFactory getServiceFactory(); + /** * Returns the Google Cloud Bigtable instance being written to, and other parameters. * @@ -733,6 +735,7 @@ static Write create() { return new AutoValue_BigtableIO_Write.Builder() .setBigtableConfig(config) .setBigtableWriteOptions(writeOptions) + .setServiceFactory(FACTORY_INSTANCE) .build(); } @@ -743,6 +746,8 @@ abstract static class Builder { abstract Builder setBigtableWriteOptions(BigtableWriteOptions writeOptions); + abstract Builder setServiceFactory(BigtableServiceFactory factory); + abstract Write build(); } @@ -999,13 +1004,13 @@ public Write withMaxRequests(long requests) { * for each batch of rows written. */ public WriteWithResults withWriteResults() { - return new WriteWithResults(getBigtableConfig(), getBigtableWriteOptions()); + return new WriteWithResults( + getBigtableConfig(), getBigtableWriteOptions(), getServiceFactory()); } - /** Set the configId for testing. */ @VisibleForTesting - Write withConfigId(int id) { - return toBuilder().setBigtableConfig(getBigtableConfig().withConfigId(id)).build(); + Write withServiceFactory(BigtableServiceFactory factory) { + return toBuilder().setServiceFactory(factory).build(); } @Override @@ -1044,16 +1049,15 @@ public static class WriteWithResults private final BigtableConfig bigtableConfig; private final BigtableWriteOptions bigtableWriteOptions; - private final BigtableServiceFactory.ConfigId configId; + private final BigtableServiceFactory factory; - WriteWithResults(BigtableConfig bigtableConfig, BigtableWriteOptions bigtableWriteOptions) { + WriteWithResults( + BigtableConfig bigtableConfig, + BigtableWriteOptions bigtableWriteOptions, + BigtableServiceFactory factory) { this.bigtableConfig = bigtableConfig; this.bigtableWriteOptions = bigtableWriteOptions; - if (bigtableConfig.getConfigId() != null) { - this.configId = BigtableServiceFactory.ConfigId.create(bigtableConfig.getConfigId()); - } else { - this.configId = FACTORY_INSTANCE.newId(); - } + this.factory = factory; } @Override @@ -1063,7 +1067,7 @@ public PCollection expand( bigtableWriteOptions.validate(); return input.apply( - ParDo.of(new BigtableWriterFn(configId, bigtableConfig, bigtableWriteOptions))); + ParDo.of(new BigtableWriterFn(factory, bigtableConfig, bigtableWriteOptions))); } @Override @@ -1091,11 +1095,10 @@ private void validateTableExists( if (config.getValidate() && config.isDataAccessible() && writeOptions.isDataAccessible()) { String tableId = checkNotNull(writeOptions.getTableId().get()); try { - BigtableServiceEntry entry = - FACTORY_INSTANCE.getServiceForWriting(configId, config, writeOptions, options); checkArgument( - entry.getService().tableExists(tableId), "Table %s does not exist", tableId); - FACTORY_INSTANCE.releaseWriteService(entry); + factory.checkTableExists(config, options, writeOptions.getTableId().get()), + "Table %s does not exist", + tableId); } catch (IOException e) { LOG.warn("Error checking whether table {} exists; proceeding.", tableId, e); } @@ -1106,17 +1109,19 @@ private void validateTableExists( private static class BigtableWriterFn extends DoFn>, BigtableWriteResult> { + private BigtableServiceFactory factory; private BigtableServiceFactory.ConfigId id; private BigtableServiceEntry serviceEntry; BigtableWriterFn( - BigtableServiceFactory.ConfigId configId, + BigtableServiceFactory factory, BigtableConfig bigtableConfig, BigtableWriteOptions writeOptions) { + this.factory = factory; this.config = bigtableConfig; this.writeOptions = writeOptions; this.failures = new ConcurrentLinkedQueue<>(); - this.id = configId; + this.id = factory.newId(); } @StartBundle @@ -1126,7 +1131,7 @@ public void startBundle(StartBundleContext c) throws IOException { if (bigtableWriter == null) { serviceEntry = - FACTORY_INSTANCE.getServiceForWriting(id, config, writeOptions, c.getPipelineOptions()); + factory.getServiceForWriting(id, config, writeOptions, c.getPipelineOptions()); bigtableWriter = serviceEntry.getService().openForWriting(writeOptions.getTableId().get()); } } @@ -1165,7 +1170,7 @@ public void tearDown() throws Exception { if (bigtableWriter != null) { bigtableWriter.close(); bigtableWriter = null; - FACTORY_INSTANCE.releaseWriteService(serviceEntry); + factory.releaseWriteService(serviceEntry); } } @@ -1224,10 +1229,12 @@ private static ByteKey makeByteKey(ByteString key) { static class BigtableSource extends BoundedSource { public BigtableSource( + BigtableServiceFactory factory, BigtableServiceFactory.ConfigId configId, BigtableConfig config, BigtableReadOptions readOptions, @Nullable Long estimatedSizeBytes) { + this.factory = factory; this.config = config; this.readOptions = readOptions; this.estimatedSizeBytes = estimatedSizeBytes; @@ -1250,16 +1257,18 @@ public String toString() { private final BigtableServiceFactory.ConfigId configId; + private final BigtableServiceFactory factory; + /** Creates a new {@link BigtableSource} with just one {@link ByteKeyRange}. */ protected BigtableSource withSingleRange(ByteKeyRange range) { checkArgument(range != null, "range can not be null"); return new BigtableSource( - configId, config, readOptions.withKeyRange(range), estimatedSizeBytes); + factory, configId, config, readOptions.withKeyRange(range), estimatedSizeBytes); } protected BigtableSource withEstimatedSizeBytes(Long estimatedSizeBytes) { checkArgument(estimatedSizeBytes != null, "estimatedSizeBytes can not be null"); - return new BigtableSource(configId, config, readOptions, estimatedSizeBytes); + return new BigtableSource(factory, configId, config, readOptions, estimatedSizeBytes); } /** @@ -1268,11 +1277,10 @@ protected BigtableSource withEstimatedSizeBytes(Long estimatedSizeBytes) { * different tablets, and possibly generate sub-splits within tablets. */ private List getSampleRowKeys(PipelineOptions pipelineOptions) throws IOException { - BigtableServiceFactory.BigtableServiceEntry serviceEntry = - FACTORY_INSTANCE.getServiceForReading(configId, config, readOptions, pipelineOptions); - List keyOffsets = serviceEntry.getService().getSampleRowKeys(this); - FACTORY_INSTANCE.releaseReadService(serviceEntry); - return keyOffsets; + try (BigtableServiceFactory.BigtableServiceEntry serviceEntry = + factory.getServiceForReading(configId, config, readOptions, pipelineOptions)) { + return serviceEntry.getService().getSampleRowKeys(this); + } } private static final long MAX_SPLIT_COUNT = 15_360L; @@ -1317,7 +1325,11 @@ protected List reduceSplits( || !checkRangeAdjacency(previousSourceRanges, source.getRanges())) { reducedSplits.add( new BigtableSource( - configId, config, readOptions.withKeyRanges(previousSourceRanges), size)); + factory, + configId, + config, + readOptions.withKeyRanges(previousSourceRanges), + size)); counter = 0; size = 0; previousSourceRanges = new ArrayList<>(); @@ -1330,7 +1342,7 @@ protected List reduceSplits( if (size > 0) { reducedSplits.add( new BigtableSource( - configId, config, readOptions.withKeyRanges(previousSourceRanges), size)); + factory, configId, config, readOptions.withKeyRanges(previousSourceRanges), size)); } return reducedSplits; } @@ -1527,7 +1539,7 @@ private long getEstimatedSizeBytesBasedOnSamples(List samples) { @Override public BoundedReader createReader(PipelineOptions options) throws IOException { return new BigtableReader( - this, FACTORY_INSTANCE.getServiceForReading(configId, config, readOptions, options)); + this, factory.getServiceForReading(configId, config, readOptions, options)); } @Override @@ -1635,13 +1647,7 @@ public BigtableReader(BigtableSource source, BigtableServiceEntry service) { @Override public boolean start() throws IOException { - reader = - serviceEntry - .getService() - .createReader( - getCurrentSource(), - serviceEntry.getAttemptTimeout(), - serviceEntry.getOperationTimeout()); + reader = serviceEntry.getService().createReader(getCurrentSource()); boolean hasRecord = (reader.start() && rangeTracker.tryReturnRecordAt( diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java index 54e55ab82bc2..a5c53966caa6 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java @@ -76,14 +76,19 @@ interface Reader { * current row because the last such call was unsuccessful. */ Row getCurrentRow() throws NoSuchElementException; + + // Workaround for ReadRows requests which requires to pass the timeouts in + // ApiContext. Can be removed later once it's fixed in Veneer. + Duration getAttemptTimeout(); + + Duration getOperationTimeout(); } /** Returns {@code true} if the table with the give name exists. */ boolean tableExists(String tableId) throws IOException; /** Returns a {@link Reader} that will read from the specified source. */ - Reader createReader(BigtableSource source, Duration attemptTimeout, Duration operationTimeout) - throws IOException; + Reader createReader(BigtableSource source) throws IOException; /** Returns a {@link Writer} that will write to the specified table. */ Writer openForWriting(String tableId) throws IOException; @@ -97,4 +102,6 @@ Reader createReader(BigtableSource source, Duration attemptTimeout, Duration ope String getProjectId(); String getInstanceId(); + + void close(); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java index fc9eae58766c..eb88011ed4af 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java @@ -17,18 +17,20 @@ */ package org.apache.beam.sdk.io.gcp.bigtable; -import com.google.api.gax.retrying.RetrySettings; import com.google.auto.value.AutoValue; import com.google.cloud.bigtable.config.BigtableOptions; +import com.google.cloud.bigtable.data.v2.BigtableDataClient; import com.google.cloud.bigtable.data.v2.BigtableDataSettings; +import io.grpc.Status; +import io.grpc.StatusRuntimeException; import java.io.IOException; import java.io.Serializable; import java.util.HashMap; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; -import org.threeten.bp.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Factory class that caches {@link BigtableService} to share between workers with the same {@link @@ -37,14 +39,16 @@ @SuppressWarnings({ "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) -class BigtableServiceFactory { +class BigtableServiceFactory implements Serializable { + + private static final Logger LOG = LoggerFactory.getLogger(BigtableServiceFactory.class); static final BigtableServiceFactory FACTORY_INSTANCE = new BigtableServiceFactory(); - private int nextId = 0; + private transient int nextId = 0; - private final Map readEntries = new HashMap<>(); - private final Map writeEntries = new HashMap<>(); + private final transient Map readEntries = new HashMap<>(); + private final transient Map writeEntries = new HashMap<>(); @AutoValue abstract static class ConfigId implements Serializable { @@ -57,7 +61,9 @@ static ConfigId create(int id) { } @AutoValue - abstract static class BigtableServiceEntry { + abstract static class BigtableServiceEntry implements Serializable, AutoCloseable { + + abstract BigtableServiceFactory getServiceFactory(); abstract ConfigId getConfigId(); @@ -65,20 +71,25 @@ abstract static class BigtableServiceEntry { abstract AtomicInteger getRefCount(); - // Workaround for ReadRows requests which requires to pass the timeouts in - // ApiContext. Can be removed later once it's fixed in Veneer. - abstract Duration getAttemptTimeout(); - - abstract Duration getOperationTimeout(); + abstract String getServiceType(); static BigtableServiceEntry create( + BigtableServiceFactory factory, ConfigId configId, BigtableService service, AtomicInteger refCount, - Duration attemptTimeout, - Duration operationTimeout) { + String serviceType) { return new AutoValue_BigtableServiceFactory_BigtableServiceEntry( - configId, service, refCount, attemptTimeout, operationTimeout); + factory, configId, service, refCount, serviceType); + } + + @Override + public void close() { + if (getServiceType().equals("read")) { + getServiceFactory().releaseReadService(this); + } else if (getServiceType().equals("write")) { + getServiceFactory().releaseWriteService(this); + } } } @@ -94,11 +105,7 @@ synchronized BigtableServiceEntry getServiceForReading( return entry; } - BigtableOptions effectiveOptions = config.getBigtableOptions(); - if (effectiveOptions == null && config.getBigtableOptionsConfigurator() != null) { - effectiveOptions = - config.getBigtableOptionsConfigurator().apply(BigtableOptions.builder()).build(); - } + BigtableOptions effectiveOptions = getEffectiveOptions(config); if (effectiveOptions != null) { // If BigtableOptions is set, convert it to BigtableConfig and BigtableWriteOptions config = BigtableConfigTranslator.translateToBigtableConfig(config, effectiveOptions); @@ -107,14 +114,7 @@ synchronized BigtableServiceEntry getServiceForReading( BigtableDataSettings settings = BigtableConfigTranslator.translateReadToVeneerSettings(config, opts, pipelineOptions); BigtableService service = new BigtableServiceImpl(settings); - RetrySettings retrySettings = settings.readRowSettings().getRetrySettings(); - entry = - BigtableServiceEntry.create( - configId, - service, - new AtomicInteger(1), - retrySettings.getInitialRpcTimeout(), - retrySettings.getTotalTimeout()); + entry = BigtableServiceEntry.create(this, configId, service, new AtomicInteger(1), "read"); readEntries.put(configId, entry); return entry; } @@ -131,53 +131,72 @@ synchronized BigtableServiceEntry getServiceForWriting( return entry; } + BigtableOptions effectiveOptions = getEffectiveOptions(config); + if (effectiveOptions != null) { + // If BigtableOptions is set, convert it to BigtableConfig and BigtableWriteOptions + config = BigtableConfigTranslator.translateToBigtableConfig(config, effectiveOptions); + opts = BigtableConfigTranslator.translateToBigtableWriteOptions(opts, effectiveOptions); + } + BigtableDataSettings settings = BigtableConfigTranslator.translateWriteToVeneerSettings(config, opts, pipelineOptions); BigtableService service = new BigtableServiceImpl(settings); - RetrySettings retrySettings = - settings.getStubSettings().bulkMutateRowsSettings().getRetrySettings(); - entry = - BigtableServiceEntry.create( - configId, - service, - new AtomicInteger(1), - retrySettings.getInitialRpcTimeout(), - retrySettings.getTotalTimeout()); + entry = BigtableServiceEntry.create(this, configId, service, new AtomicInteger(1), "write"); writeEntries.put(configId, entry); return entry; } synchronized void releaseReadService(BigtableServiceEntry entry) { if (entry.getRefCount().decrementAndGet() == 0) { - // entry.getService().close(); + entry.getService().close(); readEntries.remove(entry.getConfigId()); } } synchronized void releaseWriteService(BigtableServiceEntry entry) { if (entry.getRefCount().decrementAndGet() == 0) { - // entry.getService().close(); + entry.getService().close(); writeEntries.remove(entry.getConfigId()); } } - synchronized ConfigId newId() { - return ConfigId.create(nextId++); + boolean checkTableExists(BigtableConfig config, PipelineOptions pipelineOptions, String tableId) + throws IOException { + BigtableOptions effectiveOptions = getEffectiveOptions(config); + if (effectiveOptions != null) { + config = BigtableConfigTranslator.translateToBigtableConfig(config, effectiveOptions); + } + + if (config.isDataAccessible()) { + BigtableDataSettings settings = + BigtableConfigTranslator.translateToVeneerSettings(config, pipelineOptions); + + try (BigtableDataClient client = BigtableDataClient.create(settings)) { + try { + client.readRow(tableId, "non-exist-row"); + } catch (StatusRuntimeException e) { + if (e.getStatus().getCode() == Status.Code.NOT_FOUND) { + return false; + } + String message = String.format("Error checking whether table %s exists", tableId); + LOG.error(message, e); + throw new IOException(message, e); + } + } + } + return true; } - @VisibleForTesting - synchronized void addFakeWriteService(ConfigId id, BigtableService service) { - writeEntries.put( - id, - BigtableServiceEntry.create( - id, service, new AtomicInteger(1), Duration.ofMillis(100), Duration.ofMillis(1000))); + synchronized ConfigId newId() { + return ConfigId.create(nextId++); } - @VisibleForTesting - synchronized void addFakeReadService(ConfigId id, BigtableService service) { - readEntries.put( - id, - BigtableServiceEntry.create( - id, service, new AtomicInteger(1), Duration.ofMillis(100), Duration.ofMillis(1000))); + private BigtableOptions getEffectiveOptions(BigtableConfig config) { + BigtableOptions effectiveOptions = config.getBigtableOptions(); + if (effectiveOptions == null && config.getBigtableOptionsConfigurator() != null) { + effectiveOptions = + config.getBigtableOptionsConfigurator().apply(BigtableOptions.builder()).build(); + } + return effectiveOptions; } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java index 0d748503377b..ed23809538d5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java @@ -21,6 +21,7 @@ import com.google.api.gax.batching.Batcher; import com.google.api.gax.grpc.GrpcCallContext; +import com.google.api.gax.retrying.RetrySettings; import com.google.api.gax.rpc.ResponseObserver; import com.google.api.gax.rpc.StreamController; import com.google.bigtable.v2.Cell; @@ -102,12 +103,19 @@ public BigtableServiceImpl(BigtableDataSettings settings) throws IOException { this.client = BigtableDataClient.create(settings); this.projectId = settings.getProjectId(); this.instanceId = settings.getInstanceId(); + RetrySettings retry = settings.getStubSettings().readRowsSettings().getRetrySettings(); + this.readAttemptTimeout = retry.getInitialRpcTimeout(); + this.readOperationTimeout = retry.getTotalTimeout(); } private final BigtableDataClient client; private final String projectId; private final String instanceId; + private final Duration readAttemptTimeout; + + private final Duration readOperationTimeout; + @Override public BigtableWriterImpl openForWriting(String tableId) { return new BigtableWriterImpl(client, projectId, instanceId, tableId); @@ -183,7 +191,7 @@ public boolean start() throws IOException { try { results = client - .readRowsCallable(new BeamRowAdapter()) + .readRowsCallable(new BigtableRowProtoAdapter()) .call(query, createScanCallContext(attemptTimeout, operationTimeout)) .iterator(); serviceCallMetric.call("ok"); @@ -210,6 +218,16 @@ public Row getCurrentRow() throws NoSuchElementException { } return currentRow; } + + @Override + public Duration getAttemptTimeout() { + return attemptTimeout; + } + + @Override + public Duration getOperationTimeout() { + return operationTimeout; + } } @VisibleForTesting @@ -344,7 +362,7 @@ private Future fetchNextSegment() { } client - .readRowsCallable(new BeamRowAdapter()) + .readRowsCallable(new BigtableRowProtoAdapter()) .call( Query.fromProto(nextRequest), new ResponseObserver() { @@ -444,6 +462,16 @@ public Row getCurrentRow() throws NoSuchElementException { } return currentRow; } + + @Override + public Duration getAttemptTimeout() { + return attemptTimeout; + } + + @Override + public Duration getOperationTimeout() { + return operationTimeout; + } } @VisibleForTesting @@ -542,9 +570,7 @@ public void onFailure(Throwable throwable) { } @Override - public Reader createReader( - BigtableSource source, Duration attemptTimeout, Duration operationTimeout) - throws IOException { + public Reader createReader(BigtableSource source) throws IOException { if (source.getMaxBufferElementCount() != null) { return BigtableSegmentReaderImpl.create( client, @@ -554,8 +580,8 @@ public Reader createReader( source.getRanges(), source.getRowFilter(), source.getMaxBufferElementCount(), - attemptTimeout, - operationTimeout); + readAttemptTimeout, + readOperationTimeout); } else { return new BigtableReaderImpl( client, @@ -564,8 +590,8 @@ public Reader createReader( source.getTableId().get(), source.getRanges(), source.getRowFilter(), - attemptTimeout, - operationTimeout); + readAttemptTimeout, + readOperationTimeout); } } @@ -616,6 +642,11 @@ public static ServiceCallMetric createCallMetric( return new ServiceCallMetric(MonitoringInfoConstants.Urns.API_REQUEST_COUNT, baseLabels); } + @Override + public void close() { + client.close(); + } + /** Helper class to ease comparison of RowRange start points. */ private static final class StartPoint implements Comparable { private final ByteString value; @@ -698,7 +729,7 @@ public int compareTo(@Nonnull EndPoint o) { } } - static class BeamRowAdapter implements RowAdapter { + static class BigtableRowProtoAdapter implements RowAdapter { @Override public RowBuilder createRowBuilder() { return new DefaultRowBuilder(); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java index b174860c1e98..8c8c4bcd05ee 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java @@ -70,6 +70,7 @@ import java.util.TreeMap; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import org.apache.beam.sdk.Pipeline.PipelineExecutionException; import org.apache.beam.sdk.coders.Coder; @@ -85,6 +86,7 @@ import org.apache.beam.sdk.io.range.ByteKey; import org.apache.beam.sdk.io.range.ByteKeyRange; import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.testing.ExpectedLogs; @@ -174,15 +176,9 @@ public boolean isAccessible() { private static final BigtableServiceFactory.ConfigId configId = BigtableServiceFactory.FACTORY_INSTANCE.newId(); private static BigtableIO.Read defaultRead = - BigtableIO.read() - .withInstanceId("instance") - .withProjectId("project") - .withConfigId(configId.id()); + BigtableIO.read().withInstanceId("instance").withProjectId("project"); private static BigtableIO.Write defaultWrite = - BigtableIO.write() - .withInstanceId("instance") - .withProjectId("project") - .withConfigId(configId.id()); + BigtableIO.write().withInstanceId("instance").withProjectId("project"); private Coder>> bigtableCoder; private static final TypeDescriptor>> BIGTABLE_WRITE_TYPE = new TypeDescriptor>>() {}; @@ -193,12 +189,17 @@ public boolean isAccessible() { private static final ValueProvider> ALL_KEY_RANGE = StaticValueProvider.of(Collections.singletonList(ByteKeyRange.ALL_KEYS)); + private FakeServiceFactory factory; + @Before public void setup() throws Exception { service = new FakeBigtableService(); - BigtableServiceFactory.FACTORY_INSTANCE.addFakeReadService(configId, service); - BigtableServiceFactory.FACTORY_INSTANCE.addFakeWriteService(configId, service); + factory = new FakeServiceFactory(service); + + defaultRead = defaultRead.withServiceFactory(factory); + + defaultWrite = defaultWrite.withServiceFactory(factory); bigtableCoder = p.getCoderRegistry().getCoder(BIGTABLE_WRITE_TYPE); @@ -310,17 +311,16 @@ public void testReadWithReaderStartFailed() throws IOException { FailureBigtableService failureService = new FailureBigtableService(FailureOptions.builder().setFailAtStart(true).build()); - BigtableServiceFactory.ConfigId newId = BigtableServiceFactory.FACTORY_INSTANCE.newId(); - - BigtableServiceFactory.FACTORY_INSTANCE.addFakeReadService(newId, failureService); - BigtableConfig failureConfig = BigtableConfig.builder().setValidate(true).build(); final String table = "TEST-TABLE"; final int numRows = 100; makeTableData(failureService, table, numRows); + FakeServiceFactory failureFactory = new FakeServiceFactory(failureService); + BigtableSource source = new BigtableSource( - newId, + failureFactory, + BigtableServiceFactory.ConfigId.create(1), failureConfig, BigtableReadOptions.builder() .setTableId(StaticValueProvider.of(table)) @@ -340,9 +340,7 @@ public void testReadWithReaderAdvanceFailed() throws IOException { FailureBigtableService failureService = new FailureBigtableService(FailureOptions.builder().setFailAtAdvance(true).build()); - BigtableServiceFactory.ConfigId newId = BigtableServiceFactory.FACTORY_INSTANCE.newId(); - - BigtableServiceFactory.FACTORY_INSTANCE.addFakeReadService(newId, failureService); + FakeServiceFactory failureFactory = new FakeServiceFactory(failureService); BigtableConfig failureConfig = BigtableConfig.builder().setValidate(true).build(); final String table = "TEST-TABLE"; @@ -350,7 +348,8 @@ public void testReadWithReaderAdvanceFailed() throws IOException { makeTableData(failureService, table, numRows); BigtableSource source = new BigtableSource( - newId, + failureFactory, + failureFactory.newId(), failureConfig, BigtableReadOptions.builder() .setTableId(StaticValueProvider.of(table)) @@ -453,18 +452,17 @@ private static KV> makeBadWrite(String key) { public void testReadingFailsTableDoesNotExist() throws Exception { final String table = "TEST-TABLE"; - BigtableServiceFactory.ConfigId newId = BigtableServiceFactory.FACTORY_INSTANCE.newId(); - - BigtableServiceFactory.FACTORY_INSTANCE.addFakeReadService(newId, service); - BigtableIO.Read read = - BigtableIO.read().withBigtableOptions(BIGTABLE_OPTIONS).withTableId(table); + BigtableIO.read() + .withBigtableOptions(BIGTABLE_OPTIONS) + .withTableId(table) + .withServiceFactory(factory); // Exception will be thrown by read.validate() when read is applied. thrown.expect(IllegalArgumentException.class); thrown.expectMessage(String.format("Table %s does not exist", table)); - p.apply(read.withConfigId(newId.id())); + p.apply(read); p.run(); } @@ -688,6 +686,7 @@ public void testReadingSplitAtFractionExhaustive() throws Exception { BigtableSource source = new BigtableSource( + factory, configId, config, BigtableReadOptions.builder() @@ -711,6 +710,7 @@ public void testReadingSplitAtFraction() throws Exception { BigtableSource source = new BigtableSource( + factory, configId, config, BigtableReadOptions.builder() @@ -749,6 +749,7 @@ public void testReadingWithSplits() throws Exception { // Generate source and split it. BigtableSource source = new BigtableSource( + factory, configId, config, BigtableReadOptions.builder() @@ -769,10 +770,6 @@ public void testReadingWithSplitFailed() throws Exception { FailureBigtableService failureService = new FailureBigtableService(FailureOptions.builder().setFailAtSplit(true).build()); - BigtableServiceFactory.ConfigId newId = BigtableServiceFactory.FACTORY_INSTANCE.newId(); - - BigtableServiceFactory.FACTORY_INSTANCE.addFakeReadService(newId, failureService); - BigtableConfig failureConfig = BigtableConfig.builder().setValidate(true).build(); final String table = "TEST-MANY-ROWS-SPLITS-TABLE"; @@ -784,10 +781,13 @@ public void testReadingWithSplitFailed() throws Exception { makeTableData(failureService, table, numRows); failureService.setupSampleRowKeys(table, numSamples, bytesPerRow); + FakeServiceFactory failureFactory = new FakeServiceFactory(failureService); + // Generate source and split it. BigtableSource source = new BigtableSource( - newId, + failureFactory, + failureFactory.newId(), failureConfig, BigtableReadOptions.builder() .setTableId(StaticValueProvider.of(table)) @@ -857,6 +857,7 @@ public void testReduceSplitsWithSomeNonAdjacentRanges() throws Exception { // Generate source and split it. BigtableSource source = new BigtableSource( + factory, configId, config, BigtableReadOptions.builder() @@ -911,6 +912,7 @@ public void testReduceSplitsWithAllNonAdjacentRange() throws Exception { // Generate source and split it. BigtableSource source = new BigtableSource( + factory, configId, config, BigtableReadOptions.builder() @@ -956,6 +958,7 @@ public void tesReduceSplitsWithAdjacentRanges() throws Exception { // Generate source and split it. BigtableSource source = new BigtableSource( + factory, configId, config, BigtableReadOptions.builder() @@ -1031,6 +1034,7 @@ public void testReadingWithSplitsWithSeveralKeyRanges() throws Exception { // Generate source and split it. BigtableSource source = new BigtableSource( + factory, configId, config, BigtableReadOptions.builder() @@ -1040,6 +1044,7 @@ public void testReadingWithSplitsWithSeveralKeyRanges() throws Exception { null /*size*/); BigtableSource referenceSource = new BigtableSource( + factory, configId, config, BigtableReadOptions.builder() @@ -1072,6 +1077,7 @@ public void testReadingWithSubSplits() throws Exception { // Generate source and split it. BigtableSource source = new BigtableSource( + factory, configId, config, BigtableReadOptions.builder() @@ -1118,6 +1124,7 @@ public void testReadingWithSubSplitsWithSeveralKeyRanges() throws Exception { // Generate source and split it. BigtableSource source = new BigtableSource( + factory, configId, config, BigtableReadOptions.builder() @@ -1127,6 +1134,7 @@ public void testReadingWithSubSplitsWithSeveralKeyRanges() throws Exception { null /*size*/); BigtableSource referenceSource = new BigtableSource( + factory, configId, config, BigtableReadOptions.builder() @@ -1160,6 +1168,7 @@ public void testReadingWithFilterAndSubSplits() throws Exception { RowFilter.newBuilder().setRowKeyRegexFilter(ByteString.copyFromUtf8(".*17.*")).build(); BigtableSource source = new BigtableSource( + factory, configId, config, BigtableReadOptions.builder() @@ -1437,15 +1446,10 @@ public void testWritingFailsAtWriteRecord() throws IOException { FailureBigtableService failureService = new FailureBigtableService(FailureOptions.builder().setFailAtWriteRecord(true).build()); - BigtableServiceFactory.ConfigId newId = BigtableServiceFactory.FACTORY_INSTANCE.newId(); - - BigtableServiceFactory.FACTORY_INSTANCE.addFakeWriteService(newId, failureService); + FakeServiceFactory failureFactory = new FakeServiceFactory(failureService); BigtableIO.Write failureWrite = - BigtableIO.write() - .withInstanceId("instance") - .withProjectId("project") - .withConfigId(newId.id()); + BigtableIO.write().withInstanceId("instance").withProjectId("project"); final String table = "table"; final String key = "key"; @@ -1454,7 +1458,7 @@ public void testWritingFailsAtWriteRecord() throws IOException { failureService.createTable(table); p.apply("single row", Create.of(makeWrite(key, value)).withCoder(bigtableCoder)) - .apply("write", failureWrite.withTableId(table)); + .apply("write", failureWrite.withTableId(table).withServiceFactory(failureFactory)); // Exception will be thrown by writer.writeRecord() when BigtableWriterFn is applied. thrown.expect(IOException.class); @@ -1506,6 +1510,7 @@ public void testGetSplitPointsConsumed() throws Exception { BigtableSource source = new BigtableSource( + factory, configId, config, BigtableReadOptions.builder() @@ -1648,10 +1653,7 @@ public void verifyTableExists(String tableId) { } @Override - public FakeBigtableReader createReader( - BigtableSource source, - org.threeten.bp.Duration attemptTimeout, - org.threeten.bp.Duration operationTimeout) { + public FakeBigtableReader createReader(BigtableSource source) { return new FakeBigtableReader(source); } @@ -1677,6 +1679,9 @@ public String getInstanceId() { return null; } + @Override + public void close() {} + /** Sets up the sample row keys for the specified table. */ void setupSampleRowKeys(String tableId, int numSamples, long bytesPerRow) { verifyTableExists(tableId); @@ -1711,10 +1716,7 @@ public FailureBigtableService(FailureOptions options) { } @Override - public FakeBigtableReader createReader( - BigtableSource source, - org.threeten.bp.Duration attemptTimeout, - org.threeten.bp.Duration operationTimeout) { + public FakeBigtableReader createReader(BigtableSource source) { return new FailureBigtableReader(source, this, failureOptions); } @@ -1731,6 +1733,9 @@ public List getSampleRowKeys(BigtableSource source) { return super.getSampleRowKeys(source); } + @Override + public void close() {} + private final FailureOptions failureOptions; } @@ -1813,6 +1818,16 @@ public Row getCurrentRow() { } return currentRow; } + + @Override + public org.threeten.bp.Duration getAttemptTimeout() { + return org.threeten.bp.Duration.ofMillis(1000); + } + + @Override + public org.threeten.bp.Duration getOperationTimeout() { + return org.threeten.bp.Duration.ofMillis(1000); + } } /** A {@link FakeBigtableReader} implementation that throw exceptions at given stage. */ @@ -1954,4 +1969,49 @@ abstract static class Builder { abstract FailureOptions build(); } } + + static class FakeServiceFactory extends BigtableServiceFactory { + private FakeBigtableService service; + + FakeServiceFactory(FakeBigtableService service) { + this.service = service; + } + + @Override + synchronized BigtableServiceEntry getServiceForReading( + ConfigId configId, + BigtableConfig config, + BigtableReadOptions opts, + PipelineOptions pipelineOptions) + throws IOException { + return BigtableServiceEntry.create(this, configId, service, new AtomicInteger(1), "read"); + } + + @Override + synchronized BigtableServiceEntry getServiceForWriting( + ConfigId configId, + BigtableConfig config, + BigtableWriteOptions opts, + PipelineOptions pipelineOptions) + throws IOException { + return BigtableServiceEntry.create(this, configId, service, new AtomicInteger(1), "write"); + } + + @Override + synchronized void releaseReadService(BigtableServiceEntry entry) {} + + @Override + synchronized void releaseWriteService(BigtableServiceEntry entry) {} + + @Override + boolean checkTableExists(BigtableConfig config, PipelineOptions pipelineOptions, String tableId) + throws IOException { + return service.tableExists(tableId); + } + + @Override + synchronized ConfigId newId() { + return ConfigId.create(1); + } + } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java index 8be6146379ca..b7446eb3009d 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java @@ -166,7 +166,7 @@ public void testRead() throws IOException { .thenReturn(mockRows); when(mockStub.createReadRowsCallable(any(RowAdapter.class))).thenReturn(mockCallable); ServerStreamingCallable callable = - mockStub.createReadRowsCallable(new BigtableServiceImpl.BeamRowAdapter()); + mockStub.createReadRowsCallable(new BigtableServiceImpl.BigtableRowProtoAdapter()); // Set up client to return callable when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); @@ -224,7 +224,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { any(ApiCallContext.class)); when(mockStub.createReadRowsCallable(any(RowAdapter.class))).thenReturn(mockCallable); ServerStreamingCallable callable = - mockStub.createReadRowsCallable(new BigtableServiceImpl.BeamRowAdapter()); + mockStub.createReadRowsCallable(new BigtableServiceImpl.BigtableRowProtoAdapter()); // Set up client to return callable when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); when(mockBigtableSource.getTableId()).thenReturn(StaticValueProvider.of(TABLE_ID)); @@ -282,7 +282,7 @@ public void testReadSingleRangeAboveSegmentLimit() throws IOException { when(mockStub.createReadRowsCallable(any(RowAdapter.class))).thenReturn(mockCallable); ServerStreamingCallable callable = - mockStub.createReadRowsCallable(new BigtableServiceImpl.BeamRowAdapter()); + mockStub.createReadRowsCallable(new BigtableServiceImpl.BigtableRowProtoAdapter()); // Set up client to return callable when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); @@ -351,7 +351,7 @@ public void testReadMultipleRanges() throws IOException { when(mockStub.createReadRowsCallable(any(RowAdapter.class))).thenReturn(mockCallable); ServerStreamingCallable callable = - mockStub.createReadRowsCallable(new BigtableServiceImpl.BeamRowAdapter()); + mockStub.createReadRowsCallable(new BigtableServiceImpl.BigtableRowProtoAdapter()); // Set up client to return callable when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); @@ -423,7 +423,7 @@ public void testReadMultipleRangesOverlappingKeys() throws IOException { .call(any(Query.class), any(ResponseObserver.class), any(ApiCallContext.class)); when(mockStub.createReadRowsCallable(any(RowAdapter.class))).thenReturn(mockCallable); ServerStreamingCallable callable = - mockStub.createReadRowsCallable(new BigtableServiceImpl.BeamRowAdapter()); + mockStub.createReadRowsCallable(new BigtableServiceImpl.BigtableRowProtoAdapter()); // Set up client to return callable when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); @@ -479,7 +479,7 @@ public void testReadFullTableScan() throws IOException { .call(any(Query.class), any(ResponseObserver.class), any(ApiCallContext.class)); when(mockStub.createReadRowsCallable(any(RowAdapter.class))).thenReturn(mockCallable); ServerStreamingCallable callable = - mockStub.createReadRowsCallable(new BigtableServiceImpl.BeamRowAdapter()); + mockStub.createReadRowsCallable(new BigtableServiceImpl.BigtableRowProtoAdapter()); // Set up client to return callable when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); @@ -550,7 +550,7 @@ public void testReadFillBuffer() throws IOException { .call(any(Query.class), any(ResponseObserver.class), any(ApiCallContext.class)); when(mockStub.createReadRowsCallable(any(RowAdapter.class))).thenReturn(mockCallable); ServerStreamingCallable callable = - mockStub.createReadRowsCallable(new BigtableServiceImpl.BeamRowAdapter()); + mockStub.createReadRowsCallable(new BigtableServiceImpl.BigtableRowProtoAdapter()); // Set up client to return callable when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); @@ -639,7 +639,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { .call(any(Query.class), any(ResponseObserver.class), any(ApiCallContext.class)); when(mockStub.createReadRowsCallable(any(RowAdapter.class))).thenReturn(mockCallable); ServerStreamingCallable callable = - mockStub.createReadRowsCallable(new BigtableServiceImpl.BeamRowAdapter()); + mockStub.createReadRowsCallable(new BigtableServiceImpl.BigtableRowProtoAdapter()); // Set up client to return callable when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); @@ -706,7 +706,7 @@ public void run() { .call(any(Query.class), any(ResponseObserver.class), any(ApiCallContext.class)); when(mockStub.createReadRowsCallable(any(RowAdapter.class))).thenReturn(mockCallable); ServerStreamingCallable callable = - mockStub.createReadRowsCallable(new BigtableServiceImpl.BeamRowAdapter()); + mockStub.createReadRowsCallable(new BigtableServiceImpl.BigtableRowProtoAdapter()); // Set up client to return callable when(mockBigtableDataClient.readRowsCallable(any(RowAdapter.class))).thenReturn(callable); RetrySettings retrySettings = From 658a4fd026f425aac46c155f2c37886f051130f2 Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Fri, 17 Feb 2023 12:21:24 -0500 Subject: [PATCH 10/23] address comment p2, rewrite RowAdaptor --- .../io/gcp/bigtable/BigtableServiceImpl.java | 93 +++++++------------ 1 file changed, 35 insertions(+), 58 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java index ed23809538d5..928a4ca681af 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java @@ -49,17 +49,14 @@ import io.grpc.Status.Code; import io.grpc.StatusRuntimeException; import java.io.IOException; -import java.nio.charset.StandardCharsets; import java.util.ArrayDeque; import java.util.ArrayList; -import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; import java.util.Objects; import java.util.Queue; -import java.util.TreeMap; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import java.util.concurrent.ExecutionException; @@ -750,90 +747,70 @@ private static class DefaultRowBuilder private com.google.bigtable.v2.Row.Builder protoBuilder = com.google.bigtable.v2.Row.newBuilder(); - private TreeMap>> - cellsByFamilyColumn = new TreeMap<>(); - private TreeMap> cellsByColumn = - new TreeMap<>(Comparator.comparing(o -> o.toString(StandardCharsets.UTF_8))); - private ImmutableList.Builder currentColumnCells; + private ByteString currentValue; + private Family.Builder lastFamily; + private String lastFamilyName; + private Column.Builder lastColumn; + private ByteString lastColumnName; - private ByteString qualifier; - private ByteString previousQualifier; - private String family; - private String previousFamily; - - private ByteString value; - private List labels; - private long timestamp; + private Cell.Builder lastCell; @Override public void startRow(ByteString key) { protoBuilder.setKey(key); + + lastFamilyName = null; + lastFamily = null; + lastColumnName = null; + lastColumn = null; } @Override public void startCell( String family, ByteString qualifier, long timestamp, List labels, long size) { - this.family = family; - this.qualifier = qualifier; - this.timestamp = timestamp; - this.labels = labels; - this.value = ByteString.EMPTY; + boolean familyChanged = false; + + if (!family.equals(lastFamilyName)) { + familyChanged = true; + lastFamily = protoBuilder.addFamiliesBuilder().setName(family); + lastFamilyName = family; + } + if (!qualifier.equals(lastColumnName) || familyChanged) { + lastColumn = lastFamily.addColumnsBuilder().setQualifier(qualifier); + lastColumnName = qualifier; + } + lastCell = lastColumn.addCellsBuilder().setTimestampMicros(timestamp).addAllLabels(labels); + currentValue = null; } @Override public void cellValue(ByteString value) { - this.value = this.value.concat(value); + if (currentValue == null) { + currentValue = value; + } else { + currentValue = currentValue.concat(value); + } } @Override public void finishCell() { - if (!qualifier.equals(previousQualifier)) { - previousQualifier = qualifier; - currentColumnCells = ImmutableList.builder(); - cellsByColumn.put(qualifier, currentColumnCells); - } - if (!family.equals(previousFamily)) { - previousFamily = family; - this.cellsByFamilyColumn.put(family, cellsByColumn); - } - - Cell cell = - Cell.newBuilder() - .setValue(value) - .addAllLabels(labels) - .setTimestampMicros(timestamp) - .build(); - currentColumnCells.add(cell); + lastCell.setValue(currentValue); } @Override public com.google.bigtable.v2.Row finishRow() { - for (String family : cellsByFamilyColumn.keySet()) { - Family.Builder f = Family.newBuilder().setName(family); - for (ByteString column : cellsByFamilyColumn.get(family).keySet()) { - Column c = - Column.newBuilder() - .setQualifier(column) - .addAllCells(cellsByFamilyColumn.get(family).get(column).build()) - .build(); - f.addColumns(c); - } - protoBuilder.addFamilies(f); - } return protoBuilder.build(); } @Override public void reset() { - this.qualifier = null; - this.previousQualifier = null; - this.family = null; - this.previousFamily = null; + lastFamilyName = null; + lastFamily = null; + lastColumnName = null; + lastColumn = null; + currentValue = null; protoBuilder = com.google.bigtable.v2.Row.newBuilder(); - - this.cellsByColumn.clear(); - this.cellsByFamilyColumn.clear(); } @Override From 7fc9f0058cb85ad9f46d900f80d3736dda07b0c9 Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Fri, 17 Feb 2023 14:04:29 -0500 Subject: [PATCH 11/23] address comments p3, move to GcpCredentialFactory and clean up service factory --- .../sdk/io/gcp/bigtable/BigtableConfig.java | 10 ++-- .../bigtable/BigtableConfigTranslator.java | 36 +++++------ .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 26 ++++---- .../gcp/bigtable/BigtableServiceFactory.java | 60 +++++++++---------- .../beam/sdk/io/gcp/GcpApiSurfaceTest.java | 1 - .../BigtableConfigTranslatorTest.java | 14 +++-- .../sdk/io/gcp/bigtable/BigtableIOTest.java | 15 ++--- 7 files changed, 81 insertions(+), 81 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java index 1a33db8fb61c..fa5ebbb84820 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java @@ -19,11 +19,11 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; -import com.google.api.gax.core.CredentialsProvider; import com.google.auto.value.AutoValue; import com.google.cloud.bigtable.config.BigtableOptions; import java.io.Serializable; import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.extensions.gcp.auth.CredentialFactory; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; @@ -70,7 +70,7 @@ public abstract class BigtableConfig implements Serializable { abstract @Nullable String getUserAgent(); /** Credentials for running the job. */ - abstract @Nullable CredentialsProvider getCredentialsProvider(); + abstract @Nullable CredentialFactory getCredentialFactory(); abstract Builder toBuilder(); @@ -102,7 +102,7 @@ abstract Builder setBigtableOptionsConfigurator( abstract Builder setUserAgent(String userAgent); - abstract Builder setCredentialsProvider(CredentialsProvider credentialsProvider); + abstract Builder setCredentialFactory(CredentialFactory credentialFactory); abstract BigtableConfig build(); } @@ -147,8 +147,8 @@ public BigtableConfig withEmulator(String emulatorHost) { return toBuilder().setEmulatorHost(emulatorHost).build(); } - BigtableConfig withCredentialsProvider(CredentialsProvider provider) { - return toBuilder().setCredentialsProvider(provider).build(); + BigtableConfig withCredentialFactory(CredentialFactory factory) { + return toBuilder().setCredentialFactory(factory).build(); } void validate() { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java index 4e7d9431e883..91cb2b43f514 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java @@ -19,7 +19,6 @@ import com.google.api.gax.batching.BatchingSettings; import com.google.api.gax.core.FixedCredentialsProvider; -import com.google.api.gax.core.NoCredentialsProvider; import com.google.api.gax.retrying.RetrySettings; import com.google.api.gax.rpc.FixedHeaderProvider; import com.google.api.gax.rpc.StubSettings; @@ -38,6 +37,8 @@ import java.security.PrivateKey; import java.util.Objects; import javax.annotation.Nonnull; +import org.apache.beam.sdk.extensions.gcp.auth.GcpCredentialFactory; +import org.apache.beam.sdk.extensions.gcp.auth.NoopCredentialFactory; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.ValueProvider; @@ -104,16 +105,12 @@ private static BigtableDataSettings.Builder buildBigtableDataSettings( dataBuilder.setAppProfileId(Objects.requireNonNull(config.getAppProfileId().get())); } - // Configure credentials, check both PipelineOptions and BigtableConfig - if (pipelineOptions instanceof GcpOptions) { + if (config.getCredentialFactory() != null) { dataBuilder .stubSettings() .setCredentialsProvider( - FixedCredentialsProvider.create(((GcpOptions) pipelineOptions).getGcpCredential())); - } - - if (config.getCredentialsProvider() != null) { - dataBuilder.stubSettings().setCredentialsProvider(config.getCredentialsProvider()); + FixedCredentialsProvider.create( + ((GcpOptions) config.getCredentialFactory()).getGcpCredential())); } configureHeaderProvider(dataBuilder.stubSettings(), pipelineOptions); @@ -228,7 +225,8 @@ private static BigtableDataSettings configureReadSettings( } /** Translate BigtableOptions to BigtableConfig. */ - static BigtableConfig translateToBigtableConfig(BigtableConfig config, BigtableOptions options) { + static BigtableConfig translateToBigtableConfig( + BigtableConfig config, BigtableOptions options, PipelineOptions pipelineOptions) { BigtableConfig.Builder builder = config.toBuilder(); if (options.getProjectId() != null && config.getProjectId() == null) { @@ -271,12 +269,13 @@ static BigtableConfig translateToBigtableConfig(BigtableConfig config, BigtableO if (privateKey == null) { throw new IllegalStateException("private key cannot be null"); } - builder.setCredentialsProvider( - FixedCredentialsProvider.create( + ((GcpOptions) pipelineOptions) + .setGcpCredential( ServiceAccountJwtAccessCredentials.newBuilder() .setClientEmail(serviceAccount) .setPrivateKey(privateKey) - .build())); + .build()); + builder.setCredentialFactory(GcpCredentialFactory.fromOptions(pipelineOptions)); } catch (GeneralSecurityException exception) { throw new RuntimeException("exception while retrieving credentials", exception); } @@ -284,17 +283,20 @@ static BigtableConfig translateToBigtableConfig(BigtableConfig config, BigtableO case SuppliedCredentials: Credentials credentials = ((CredentialOptions.UserSuppliedCredentialOptions) credOptions).getCredential(); - builder.setCredentialsProvider(FixedCredentialsProvider.create(credentials)); + ((GcpOptions) pipelineOptions).setGcpCredential(credentials); + builder.setCredentialFactory(GcpCredentialFactory.fromOptions(pipelineOptions)); break; case SuppliedJson: CredentialOptions.JsonCredentialsOptions jsonCredentialsOptions = (CredentialOptions.JsonCredentialsOptions) credOptions; - builder.setCredentialsProvider( - FixedCredentialsProvider.create( - GoogleCredentials.fromStream(jsonCredentialsOptions.getInputStream()))); + ((GcpOptions) pipelineOptions) + .setGcpCredential( + GoogleCredentials.fromStream(jsonCredentialsOptions.getInputStream())); + builder.setCredentialFactory(GcpCredentialFactory.fromOptions(pipelineOptions)); break; case None: - builder.setCredentialsProvider(NoCredentialsProvider.create()); + // pipelineOptions is ignored + builder.setCredentialFactory(NoopCredentialFactory.fromOptions(pipelineOptions)); break; } } catch (IOException e) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index a4e18ce5b555..8be7f1c4f9c7 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -24,7 +24,6 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; -import com.google.api.gax.core.CredentialsProvider; import com.google.auto.value.AutoValue; import com.google.bigtable.v2.Mutation; import com.google.bigtable.v2.Row; @@ -47,6 +46,7 @@ import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.extensions.gcp.auth.CredentialFactory; import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.BoundedSource.BoundedReader; @@ -411,11 +411,9 @@ public Read withTableId(String tableId) { * *

Does not modify this object. */ - public Read withCredentialsProvider(CredentialsProvider credentialsProvider) { + public Read withCredentialsProvider(CredentialFactory credentialFactory) { BigtableConfig config = getBigtableConfig(); - return toBuilder() - .setBigtableConfig(config.withCredentialsProvider(credentialsProvider)) - .build(); + return toBuilder().setBigtableConfig(config.withCredentialFactory(credentialFactory)).build(); } /** @@ -823,11 +821,9 @@ public Write withTableId(String tableId) { * *

Does not modify this object. */ - public Write withCredentialsProvider(CredentialsProvider credentialsProvider) { + public Write withCredentialFactory(CredentialFactory credentialFactory) { BigtableConfig config = getBigtableConfig(); - return toBuilder() - .setBigtableConfig(config.withCredentialsProvider(credentialsProvider)) - .build(); + return toBuilder().setBigtableConfig(config.withCredentialFactory(credentialFactory)).build(); } /** * WARNING: Should be used only to specify additional parameters for connection to the Cloud @@ -1170,7 +1166,7 @@ public void tearDown() throws Exception { if (bigtableWriter != null) { bigtableWriter.close(); bigtableWriter = null; - factory.releaseWriteService(serviceEntry); + factory.releaseService(serviceEntry); } } @@ -1539,7 +1535,7 @@ private long getEstimatedSizeBytesBasedOnSamples(List samples) { @Override public BoundedReader createReader(PipelineOptions options) throws IOException { return new BigtableReader( - this, factory.getServiceForReading(configId, config, readOptions, options)); + factory, this, factory.getServiceForReading(configId, config, readOptions, options)); } @Override @@ -1633,13 +1629,17 @@ private static class BigtableReader extends BoundedReader { // Thread-safety: source is protected via synchronization and is only accessed or modified // inside a synchronized block (or constructor, which is the same). private BigtableSource source; + + private final BigtableServiceFactory factory; private final BigtableServiceEntry serviceEntry; private BigtableService.Reader reader; private final ByteKeyRangeTracker rangeTracker; private long recordsReturned; - public BigtableReader(BigtableSource source, BigtableServiceEntry service) { + public BigtableReader( + BigtableServiceFactory factory, BigtableSource source, BigtableServiceEntry service) { checkArgument(source.getRanges().size() == 1, "source must have exactly one key range"); + this.factory = factory; this.source = source; this.serviceEntry = service; rangeTracker = ByteKeyRangeTracker.of(source.getRanges().get(0)); @@ -1686,7 +1686,7 @@ public Row getCurrent() throws NoSuchElementException { public void close() throws IOException { LOG.info("Closing reader after reading {} records.", recordsReturned); if (reader != null) { - FACTORY_INSTANCE.releaseReadService(serviceEntry); + factory.releaseService(serviceEntry); reader = null; } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java index eb88011ed4af..3c22ef00daba 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java @@ -24,6 +24,7 @@ import io.grpc.Status; import io.grpc.StatusRuntimeException; import java.io.IOException; +import java.io.ObjectInputStream; import java.io.Serializable; import java.util.HashMap; import java.util.Map; @@ -45,10 +46,9 @@ class BigtableServiceFactory implements Serializable { static final BigtableServiceFactory FACTORY_INSTANCE = new BigtableServiceFactory(); - private transient int nextId = 0; + private transient int nextId; - private final transient Map readEntries = new HashMap<>(); - private final transient Map writeEntries = new HashMap<>(); + private transient Map entries = new HashMap<>(); @AutoValue abstract static class ConfigId implements Serializable { @@ -71,25 +71,18 @@ abstract static class BigtableServiceEntry implements Serializable, AutoCloseabl abstract AtomicInteger getRefCount(); - abstract String getServiceType(); - static BigtableServiceEntry create( BigtableServiceFactory factory, ConfigId configId, BigtableService service, - AtomicInteger refCount, - String serviceType) { + AtomicInteger refCount) { return new AutoValue_BigtableServiceFactory_BigtableServiceEntry( - factory, configId, service, refCount, serviceType); + factory, configId, service, refCount); } @Override public void close() { - if (getServiceType().equals("read")) { - getServiceFactory().releaseReadService(this); - } else if (getServiceType().equals("write")) { - getServiceFactory().releaseWriteService(this); - } + getServiceFactory().releaseService(this); } } @@ -99,7 +92,7 @@ synchronized BigtableServiceEntry getServiceForReading( BigtableReadOptions opts, PipelineOptions pipelineOptions) throws IOException { - BigtableServiceEntry entry = readEntries.get(configId); + BigtableServiceEntry entry = entries.get(configId); if (entry != null) { entry.getRefCount().incrementAndGet(); return entry; @@ -108,14 +101,16 @@ synchronized BigtableServiceEntry getServiceForReading( BigtableOptions effectiveOptions = getEffectiveOptions(config); if (effectiveOptions != null) { // If BigtableOptions is set, convert it to BigtableConfig and BigtableWriteOptions - config = BigtableConfigTranslator.translateToBigtableConfig(config, effectiveOptions); + config = + BigtableConfigTranslator.translateToBigtableConfig( + config, effectiveOptions, pipelineOptions); opts = BigtableConfigTranslator.translateToBigtableReadOptions(opts, effectiveOptions); } BigtableDataSettings settings = BigtableConfigTranslator.translateReadToVeneerSettings(config, opts, pipelineOptions); BigtableService service = new BigtableServiceImpl(settings); - entry = BigtableServiceEntry.create(this, configId, service, new AtomicInteger(1), "read"); - readEntries.put(configId, entry); + entry = BigtableServiceEntry.create(this, configId, service, new AtomicInteger(1)); + entries.put(configId, entry); return entry; } @@ -125,7 +120,7 @@ synchronized BigtableServiceEntry getServiceForWriting( BigtableWriteOptions opts, PipelineOptions pipelineOptions) throws IOException { - BigtableServiceEntry entry = writeEntries.get(configId); + BigtableServiceEntry entry = entries.get(configId); if (entry != null) { entry.getRefCount().incrementAndGet(); return entry; @@ -134,29 +129,24 @@ synchronized BigtableServiceEntry getServiceForWriting( BigtableOptions effectiveOptions = getEffectiveOptions(config); if (effectiveOptions != null) { // If BigtableOptions is set, convert it to BigtableConfig and BigtableWriteOptions - config = BigtableConfigTranslator.translateToBigtableConfig(config, effectiveOptions); + config = + BigtableConfigTranslator.translateToBigtableConfig( + config, effectiveOptions, pipelineOptions); opts = BigtableConfigTranslator.translateToBigtableWriteOptions(opts, effectiveOptions); } BigtableDataSettings settings = BigtableConfigTranslator.translateWriteToVeneerSettings(config, opts, pipelineOptions); BigtableService service = new BigtableServiceImpl(settings); - entry = BigtableServiceEntry.create(this, configId, service, new AtomicInteger(1), "write"); - writeEntries.put(configId, entry); + entry = BigtableServiceEntry.create(this, configId, service, new AtomicInteger(1)); + entries.put(configId, entry); return entry; } - synchronized void releaseReadService(BigtableServiceEntry entry) { + synchronized void releaseService(BigtableServiceEntry entry) { if (entry.getRefCount().decrementAndGet() == 0) { entry.getService().close(); - readEntries.remove(entry.getConfigId()); - } - } - - synchronized void releaseWriteService(BigtableServiceEntry entry) { - if (entry.getRefCount().decrementAndGet() == 0) { - entry.getService().close(); - writeEntries.remove(entry.getConfigId()); + entries.remove(entry.getConfigId()); } } @@ -164,7 +154,9 @@ boolean checkTableExists(BigtableConfig config, PipelineOptions pipelineOptions, throws IOException { BigtableOptions effectiveOptions = getEffectiveOptions(config); if (effectiveOptions != null) { - config = BigtableConfigTranslator.translateToBigtableConfig(config, effectiveOptions); + config = + BigtableConfigTranslator.translateToBigtableConfig( + config, effectiveOptions, pipelineOptions); } if (config.isDataAccessible()) { @@ -199,4 +191,10 @@ private BigtableOptions getEffectiveOptions(BigtableConfig config) { } return effectiveOptions; } + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + entries = new HashMap<>(); + nextId = 0; + } } 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 d34f7a7d9298..fc1eebd521fa 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 @@ -82,7 +82,6 @@ public void testGcpApiSurface() throws Exception { Matchers.equalTo(com.google.api.gax.rpc.ApiException.class), Matchers.equalTo(com.google.api.gax.paging.Page.class), Matchers.>equalTo(com.google.api.gax.rpc.StatusCode.class), - Matchers.>equalTo(com.google.api.gax.core.CredentialsProvider.class), Matchers.>equalTo(com.google.common.base.Function.class), Matchers.>equalTo(com.google.common.base.Optional.class), Matchers.>equalTo(com.google.common.base.Supplier.class), diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java index c73131acc565..bfecb3383e3c 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java @@ -19,7 +19,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; import com.google.cloud.bigtable.config.BigtableOptions; import com.google.cloud.bigtable.config.BulkOptions; @@ -28,6 +27,8 @@ import com.google.cloud.bigtable.config.RetryOptions; import com.google.cloud.bigtable.data.v2.BigtableDataSettings; import com.google.cloud.bigtable.data.v2.stub.EnhancedBigtableStubSettings; +import org.apache.beam.sdk.extensions.gcp.auth.NoopCredentialFactory; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.ValueProvider; @@ -53,19 +54,24 @@ public void testBigtableOptionsToBigtableConfig() throws Exception { .build(); BigtableConfig config = BigtableConfig.builder().setValidate(true).build(); - config = BigtableConfigTranslator.translateToBigtableConfig(config, options); + config = + BigtableConfigTranslator.translateToBigtableConfig( + config, options, PipelineOptionsFactory.as(GcpOptions.class)); assertNotNull(config.getProjectId()); assertNotNull(config.getInstanceId()); assertNotNull(config.getAppProfileId()); assertNotNull(config.getEmulatorHost()); - assertNotNull(config.getCredentialsProvider()); + assertNotNull(config.getCredentialFactory()); + NoopCredentialFactory noopCredentialFactory = + NoopCredentialFactory.fromOptions(PipelineOptionsFactory.as(GcpOptions.class)); assertEquals(options.getProjectId(), config.getProjectId().get()); assertEquals(options.getInstanceId(), config.getInstanceId().get()); assertEquals(options.getAppProfileId(), config.getAppProfileId().get()); assertEquals("localhost:1234", config.getEmulatorHost()); - assertNull(config.getCredentialsProvider().getCredentials()); + assertEquals( + noopCredentialFactory.getCredential(), config.getCredentialFactory().getCredential()); } @Test diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java index 8c8c4bcd05ee..44cf0cd1422b 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java @@ -1982,9 +1982,8 @@ synchronized BigtableServiceEntry getServiceForReading( ConfigId configId, BigtableConfig config, BigtableReadOptions opts, - PipelineOptions pipelineOptions) - throws IOException { - return BigtableServiceEntry.create(this, configId, service, new AtomicInteger(1), "read"); + PipelineOptions pipelineOptions) { + return BigtableServiceEntry.create(this, configId, service, new AtomicInteger(1)); } @Override @@ -1992,16 +1991,12 @@ synchronized BigtableServiceEntry getServiceForWriting( ConfigId configId, BigtableConfig config, BigtableWriteOptions opts, - PipelineOptions pipelineOptions) - throws IOException { - return BigtableServiceEntry.create(this, configId, service, new AtomicInteger(1), "write"); + PipelineOptions pipelineOptions) { + return BigtableServiceEntry.create(this, configId, service, new AtomicInteger(1)); } @Override - synchronized void releaseReadService(BigtableServiceEntry entry) {} - - @Override - synchronized void releaseWriteService(BigtableServiceEntry entry) {} + synchronized void releaseService(BigtableServiceEntry entry) {} @Override boolean checkTableExists(BigtableConfig config, PipelineOptions pipelineOptions, String tableId) From 42b3e1722b2d68b8c9c72a009a858ff932a8b779 Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Fri, 17 Feb 2023 14:23:01 -0500 Subject: [PATCH 12/23] a new round of comments --- .../apache/beam/sdk/io/gcp/bigtable/BigtableIO.java | 10 ++++++++-- .../sdk/io/gcp/bigtable/BigtableServiceFactory.java | 8 ++++---- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 8be7f1c4f9c7..20bf375676e6 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -1166,7 +1166,10 @@ public void tearDown() throws Exception { if (bigtableWriter != null) { bigtableWriter.close(); bigtableWriter = null; + } + if (serviceEntry != null) { factory.releaseService(serviceEntry); + serviceEntry = null; } } @@ -1631,7 +1634,7 @@ private static class BigtableReader extends BoundedReader { private BigtableSource source; private final BigtableServiceFactory factory; - private final BigtableServiceEntry serviceEntry; + private BigtableServiceEntry serviceEntry; private BigtableService.Reader reader; private final ByteKeyRangeTracker rangeTracker; private long recordsReturned; @@ -1686,9 +1689,12 @@ public Row getCurrent() throws NoSuchElementException { public void close() throws IOException { LOG.info("Closing reader after reading {} records.", recordsReturned); if (reader != null) { - factory.releaseService(serviceEntry); reader = null; } + if (serviceEntry != null) { + factory.releaseService(serviceEntry); + serviceEntry = null; + } } @Override diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java index 3c22ef00daba..03ab54bde5b5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java @@ -17,12 +17,12 @@ */ package org.apache.beam.sdk.io.gcp.bigtable; +import com.google.api.gax.grpc.GrpcStatusCode; +import com.google.api.gax.rpc.ApiException; import com.google.auto.value.AutoValue; import com.google.cloud.bigtable.config.BigtableOptions; import com.google.cloud.bigtable.data.v2.BigtableDataClient; import com.google.cloud.bigtable.data.v2.BigtableDataSettings; -import io.grpc.Status; -import io.grpc.StatusRuntimeException; import java.io.IOException; import java.io.ObjectInputStream; import java.io.Serializable; @@ -166,8 +166,8 @@ boolean checkTableExists(BigtableConfig config, PipelineOptions pipelineOptions, try (BigtableDataClient client = BigtableDataClient.create(settings)) { try { client.readRow(tableId, "non-exist-row"); - } catch (StatusRuntimeException e) { - if (e.getStatus().getCode() == Status.Code.NOT_FOUND) { + } catch (ApiException e) { + if (e.getStatusCode().getCode() == GrpcStatusCode.Code.NOT_FOUND) { return false; } String message = String.format("Error checking whether table %s exists", tableId); From f88b3006c79d5f699c78d8b75219a712efc1dce8 Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Fri, 17 Feb 2023 15:11:38 -0500 Subject: [PATCH 13/23] more small fixes --- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 12 ++++---- .../sdk/io/gcp/bigtable/BigtableService.java | 7 ----- .../gcp/bigtable/BigtableServiceFactory.java | 2 +- .../io/gcp/bigtable/BigtableServiceImpl.java | 30 ------------------- .../sdk/io/gcp/bigtable/BigtableIOTest.java | 11 ------- 5 files changed, 7 insertions(+), 55 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 20bf375676e6..bd7e34d9ff04 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -411,7 +411,7 @@ public Read withTableId(String tableId) { * *

Does not modify this object. */ - public Read withCredentialsProvider(CredentialFactory credentialFactory) { + public Read withCredentialFactory(CredentialFactory credentialFactory) { BigtableConfig config = getBigtableConfig(); return toBuilder().setBigtableConfig(config.withCredentialFactory(credentialFactory)).build(); } @@ -995,6 +995,11 @@ public Write withMaxRequests(long requests) { .build(); } + @VisibleForTesting + Write withServiceFactory(BigtableServiceFactory factory) { + return toBuilder().setServiceFactory(factory).build(); + } + /** * Returns a {@link BigtableIO.WriteWithResults} that will emit a {@link BigtableWriteResult} * for each batch of rows written. @@ -1004,11 +1009,6 @@ public WriteWithResults withWriteResults() { getBigtableConfig(), getBigtableWriteOptions(), getServiceFactory()); } - @VisibleForTesting - Write withServiceFactory(BigtableServiceFactory factory) { - return toBuilder().setServiceFactory(factory).build(); - } - @Override public PDone expand(PCollection>> input) { input.apply(withWriteResults()); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java index a5c53966caa6..f65d83e616c4 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java @@ -84,9 +84,6 @@ interface Reader { Duration getOperationTimeout(); } - /** Returns {@code true} if the table with the give name exists. */ - boolean tableExists(String tableId) throws IOException; - /** Returns a {@link Reader} that will read from the specified source. */ Reader createReader(BigtableSource source) throws IOException; @@ -99,9 +96,5 @@ interface Reader { */ List getSampleRowKeys(BigtableSource source) throws IOException; - String getProjectId(); - - String getInstanceId(); - void close(); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java index 03ab54bde5b5..74845582ca34 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java @@ -46,7 +46,7 @@ class BigtableServiceFactory implements Serializable { static final BigtableServiceFactory FACTORY_INSTANCE = new BigtableServiceFactory(); - private transient int nextId; + private transient int nextId = 0; private transient Map entries = new HashMap<>(); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java index 928a4ca681af..1848b7db457e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java @@ -46,7 +46,6 @@ import com.google.protobuf.ByteString; import io.grpc.CallOptions; import io.grpc.Deadline; -import io.grpc.Status.Code; import io.grpc.StatusRuntimeException; import java.io.IOException; import java.util.ArrayDeque; @@ -77,8 +76,6 @@ import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.Futures; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.SettableFuture; import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.threeten.bp.Duration; /** @@ -89,7 +86,6 @@ "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) class BigtableServiceImpl implements BigtableService { - private static final Logger LOG = LoggerFactory.getLogger(BigtableServiceImpl.class); // Default byte limit is a percentage of the JVM's available memory private static final double DEFAULT_BYTE_LIMIT_PERCENTAGE = .1; // Percentage of max number of rows allowed in the buffer @@ -118,21 +114,6 @@ public BigtableWriterImpl openForWriting(String tableId) { return new BigtableWriterImpl(client, projectId, instanceId, tableId); } - @Override - public boolean tableExists(String tableId) throws IOException { - try { - client.readRow(tableId, "non-exist-row"); - return true; - } catch (StatusRuntimeException e) { - if (e.getStatus().getCode() == Code.NOT_FOUND) { - return false; - } - String message = String.format("Error checking whether table %s exists", tableId); - LOG.error(message, e); - throw new IOException(message, e); - } - } - @VisibleForTesting static class BigtableReaderImpl implements Reader { private final BigtableDataClient client; @@ -611,17 +592,6 @@ public List getSampleRowKeys(BigtableSource source) { return client.sampleRowKeys(source.getTableId().get()); } - @Override - public String getProjectId() { - return projectId; - } - - @Override - public String getInstanceId() { - return instanceId; - } - - @VisibleForTesting public static ServiceCallMetric createCallMetric( String projectId, String instanceId, String tableId) { HashMap baseLabels = new HashMap<>(); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java index 44cf0cd1422b..9587fdbb50d9 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java @@ -1643,7 +1643,6 @@ public void createTable(String tableId) { tables.put(tableId, new TreeMap<>(new ByteStringComparator())); } - @Override public boolean tableExists(String tableId) { return tables.containsKey(tableId); } @@ -1669,16 +1668,6 @@ public List getSampleRowKeys(BigtableSource source) { return samples; } - @Override - public String getProjectId() { - return null; - } - - @Override - public String getInstanceId() { - return null; - } - @Override public void close() {} From 5b818ffb9cadab4bbf0c12889724ed91debfe098 Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Fri, 17 Feb 2023 18:07:30 -0500 Subject: [PATCH 14/23] address more comments, make doc better --- .../bigtable/BigtableClientWrapper.java | 2 +- .../sdk/io/gcp/bigtable/BigtableConfig.java | 9 +- .../bigtable/BigtableConfigTranslator.java | 94 +++++++------ .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 127 +++++++----------- .../io/gcp/bigtable/BigtableReadOptions.java | 33 ++--- .../sdk/io/gcp/bigtable/BigtableService.java | 2 +- .../gcp/bigtable/BigtableServiceFactory.java | 3 +- .../io/gcp/bigtable/BigtableServiceImpl.java | 10 +- .../io/gcp/bigtable/BigtableWriteOptions.java | 31 ++--- .../BigtableConfigTranslatorTest.java | 38 ++---- .../sdk/io/gcp/bigtable/BigtableIOTest.java | 8 +- .../gcp/bigtable/BigtableServiceImplTest.java | 37 ++--- 12 files changed, 158 insertions(+), 236 deletions(-) diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableClientWrapper.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableClientWrapper.java index fd1b8fe0a5e6..c2cbc9b68450 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableClientWrapper.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableClientWrapper.java @@ -37,7 +37,7 @@ class BigtableClientWrapper implements Serializable { private final BigtableTableAdminClient tableAdminClient; private final BigtableDataClient dataClient; - public BigtableClientWrapper( + BigtableClientWrapper( String project, String instanceId, @Nullable Integer emulatorPort, diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java index fa5ebbb84820..c2f6d7068c71 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java @@ -24,6 +24,7 @@ import java.io.Serializable; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.extensions.gcp.auth.CredentialFactory; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; @@ -69,7 +70,10 @@ public abstract class BigtableConfig implements Serializable { /** User agent for this job. */ abstract @Nullable String getUserAgent(); - /** Credentials for running the job. */ + /** + * Credentials for running the job. Use the default credentials in {@link GcpOptions} if it's not + * set. + */ abstract @Nullable CredentialFactory getCredentialFactory(); abstract Builder toBuilder(); @@ -188,7 +192,8 @@ void populateDisplayData(DisplayData.Builder builder) { boolean isDataAccessible() { return (getProjectId() == null || getProjectId().isAccessible()) - && (getInstanceId() == null || getInstanceId().isAccessible()); + && (getInstanceId() == null || getInstanceId().isAccessible()) + && (getAppProfileId() == null || getAppProfileId().isAccessible()); } @Override diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java index 91cb2b43f514..b613e2f873df 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java @@ -46,7 +46,14 @@ import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; import org.threeten.bp.Duration; -/** Helper class to translate {@link BigtableOptions} to Bigtable Veneer settings. */ +/** + * Helper class to translate {@link BigtableConfig}, {@link BigtableReadOptions}, {@link + * BigtableWriteOptions} and {@link PipelineOptions} to Bigtable Veneer settings. + * + *

Also translate {@link BigtableOptions} to {@link BigtableConfig} for backward compatibility. + * If the values are set on {@link BigtableConfig} directly, ignore the settings in {@link + * BigtableOptions}. + */ @SuppressWarnings({ "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) @@ -121,10 +128,11 @@ private static BigtableDataSettings.Builder buildBigtableDataSettings( private static void configureHeaderProvider( StubSettings.Builder stubSettings, PipelineOptions pipelineOptions) { - ImmutableMap.Builder headersBuilder = ImmutableMap.builder(); - headersBuilder.putAll(stubSettings.getHeaderProvider().getHeaders()); - headersBuilder.put( - GrpcUtil.USER_AGENT_KEY.name(), Objects.requireNonNull(pipelineOptions.getUserAgent())); + ImmutableMap.Builder headersBuilder = + ImmutableMap.builder() + .put( + GrpcUtil.USER_AGENT_KEY.name(), + Objects.requireNonNull(pipelineOptions.getUserAgent())); stubSettings.setHeaderProvider(FixedHeaderProvider.create(headersBuilder.build())); } @@ -136,26 +144,19 @@ private static BigtableDataSettings configureWriteSettings( RetrySettings.Builder retrySettings = callSettings.getRetrySettings().toBuilder(); BatchingSettings.Builder batchingSettings = callSettings.getBatchingSettings().toBuilder(); if (writeOptions.getAttemptTimeout() != null) { - retrySettings.setInitialRpcTimeout(Duration.ofMillis(writeOptions.getAttemptTimeout())); - - if (writeOptions.getOperationTimeout() == null) { - retrySettings.setTotalTimeout( - Duration.ofMillis( - Math.max( - retrySettings.getTotalTimeout().toMillis(), writeOptions.getAttemptTimeout()))); - } + // Set the user specified attempt timeout and expand the operation timeout if it's shorter + retrySettings.setInitialRpcTimeout( + Duration.ofMillis(writeOptions.getAttemptTimeout().getMillis())); + retrySettings.setTotalTimeout( + Duration.ofMillis( + Math.max( + retrySettings.getTotalTimeout().toMillis(), + writeOptions.getAttemptTimeout().getMillis()))); } if (writeOptions.getOperationTimeout() != null) { - retrySettings.setTotalTimeout(Duration.ofMillis(writeOptions.getOperationTimeout())); - } - - if (writeOptions.getRetryInitialDelay() != null) { - retrySettings.setInitialRetryDelay(Duration.ofMillis(writeOptions.getRetryInitialDelay())); - } - - if (writeOptions.getRetryDelayMultiplier() != null) { - retrySettings.setRetryDelayMultiplier(writeOptions.getRetryDelayMultiplier()); + retrySettings.setTotalTimeout( + Duration.ofMillis(writeOptions.getOperationTimeout().getMillis())); } if (writeOptions.getBatchElements() != null) { @@ -197,26 +198,19 @@ private static BigtableDataSettings configureReadSettings( settings.stubSettings().readRowsSettings().getRetrySettings().toBuilder(); if (readOptions.getAttemptTimeout() != null) { - retrySettings.setInitialRpcTimeout(Duration.ofMillis(readOptions.getAttemptTimeout())); - - if (readOptions.getOperationTimeout() == null) { - retrySettings.setTotalTimeout( - Duration.ofMillis( - Math.max( - retrySettings.getTotalTimeout().toMillis(), readOptions.getAttemptTimeout()))); - } + // Set the user specified attempt timeout and expand the operation timeout if it's shorter + retrySettings.setInitialRpcTimeout( + Duration.ofMillis(readOptions.getAttemptTimeout().getMillis())); + retrySettings.setTotalTimeout( + Duration.ofMillis( + Math.max( + retrySettings.getTotalTimeout().toMillis(), + readOptions.getAttemptTimeout().getMillis()))); } if (readOptions.getOperationTimeout() != null) { - retrySettings.setTotalTimeout(Duration.ofMillis(readOptions.getOperationTimeout())); - } - - if (readOptions.getRetryDelayMultiplier() != null) { - retrySettings.setRetryDelayMultiplier(readOptions.getRetryDelayMultiplier()); - } - - if (readOptions.getRetryInitialDelay() != null) { - retrySettings.setInitialRetryDelay(Duration.ofMillis(readOptions.getRetryInitialDelay())); + retrySettings.setTotalTimeout( + Duration.ofMillis(readOptions.getOperationTimeout().getMillis())); } settings.stubSettings().readRowsSettings().setRetrySettings(retrySettings.build()); @@ -224,7 +218,10 @@ private static BigtableDataSettings configureReadSettings( return settings.build(); } - /** Translate BigtableOptions to BigtableConfig. */ + /** + * Translate BigtableOptions to BigtableConfig for backward compatibility. If the values are set on + * BigtableConfig, ignore the settings in BigtableOptions. + */ static BigtableConfig translateToBigtableConfig( BigtableConfig config, BigtableOptions options, PipelineOptions pipelineOptions) { BigtableConfig.Builder builder = config.toBuilder(); @@ -313,11 +310,11 @@ static BigtableReadOptions translateToBigtableReadOptions( BigtableReadOptions.Builder builder = readOptions.toBuilder(); if (options.getCallOptionsConfig().getReadStreamRpcAttemptTimeoutMs().isPresent()) { builder.setAttemptTimeout( - options.getCallOptionsConfig().getReadStreamRpcAttemptTimeoutMs().get()); + org.joda.time.Duration.millis( + options.getCallOptionsConfig().getReadStreamRpcAttemptTimeoutMs().get())); } - builder.setOperationTimeout(options.getCallOptionsConfig().getReadStreamRpcTimeoutMs()); - builder.setRetryInitialDelay(options.getRetryOptions().getInitialBackoffMillis()); - builder.setRetryDelayMultiplier(options.getRetryOptions().getBackoffMultiplier()); + builder.setOperationTimeout( + org.joda.time.Duration.millis(options.getCallOptionsConfig().getReadStreamRpcTimeoutMs())); return builder.build(); } @@ -329,12 +326,11 @@ static BigtableWriteOptions translateToBigtableWriteOptions( // configure timeouts if (options.getCallOptionsConfig().getMutateRpcAttemptTimeoutMs().isPresent()) { builder.setAttemptTimeout( - options.getCallOptionsConfig().getMutateRpcAttemptTimeoutMs().get()); + org.joda.time.Duration.millis( + options.getCallOptionsConfig().getMutateRpcAttemptTimeoutMs().get())); } - builder.setOperationTimeout(options.getCallOptionsConfig().getMutateRpcTimeoutMs()); - // configure retry backoffs - builder.setRetryInitialDelay(options.getRetryOptions().getInitialBackoffMillis()); - builder.setRetryDelayMultiplier(options.getRetryOptions().getBackoffMultiplier()); + builder.setOperationTimeout( + org.joda.time.Duration.millis(options.getCallOptionsConfig().getMutateRpcTimeoutMs())); // configure batch size builder.setBatchElements(options.getBulkOptions().getBulkMaxRowKeyCount()); builder.setBatchBytes(options.getBulkOptions().getBulkMaxRequestSize()); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index bd7e34d9ff04..2c294cf96a51 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -47,6 +47,7 @@ import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.extensions.gcp.auth.CredentialFactory; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.BoundedSource.BoundedReader; @@ -136,15 +137,19 @@ * .withTableId("table") * .withRowFilter(filter)); * - * // Configure timeouts for reads + * // Configure timeouts for reads. + * // Let each attempt run for 1 second, retry if the attempt failed. + * // Give up after the request is retried for 60 seconds. + * Duration attemptTimeout = Duration.millis(1000); + * Duration operationTimeout = Duration.millis(60 * 1000); * p.apply("read", * BigtableIO.read() * .withProjectId(projectId) * .withInstanceId(instanceId) * .withTableId("table") * .withKeyRange(keyRange) - * .withAttemptTimeout(100) // 100 milliseconds - * .withOperationTimeout(60 * 1000)); // 1 minute + * .withAttemptTimeout(attemptTimeout) + * .withOperationTimeout(attemptTimeout); * } * *

Writing to Cloud Bigtable

@@ -407,7 +412,8 @@ public Read withTableId(String tableId) { } /** - * Returns a new {@link BigtableIO.Read} with provided credentials. + * Returns a new {@link BigtableIO.Read} with provided credentials. If it's not set, it'll use + * the default credentials in {@link GcpOptions}. * *

Does not modify this object. */ @@ -574,56 +580,31 @@ public Read withEmulator(String emulatorHost) { } /** - * Configures the attempt timeout in milliseconds of the reads. + * Returns a new {@link BigtableIO.Read} with the attempt timeout. Attempt timeout controls the + * timeout for each remote call. * *

Does not modify this object. */ - public Read withAttemptTimeout(long timeoutMs) { - checkArgument(timeoutMs > 0, "attempt timeout must be positive"); + public Read withAttemptTimeout(Duration timeout) { + checkArgument(timeout.isLongerThan(Duration.ZERO), "attempt timeout must be positive"); BigtableReadOptions readOptions = getBigtableReadOptions(); return toBuilder() - .setBigtableReadOptions(readOptions.toBuilder().setAttemptTimeout(timeoutMs).build()) + .setBigtableReadOptions(readOptions.toBuilder().setAttemptTimeout(timeout).build()) .build(); } /** - * Configures the operation timeout in milliseconds of the reads. + * Returns a new {@link BigtableIO.Read} with the operation timeout. Operation timeout has + * ultimate control over how long the logic should keep trying the remote call until it gives up + * completely. * *

Does not modify this object. */ - public Read withOperationTimeout(long timeoutMs) { - checkArgument(timeoutMs > 0, "operation timeout must be positive"); + public Read withOperationTimeout(Duration timeout) { + checkArgument(timeout.isLongerThan(Duration.ZERO), "operation timeout must be positive"); BigtableReadOptions readOptions = getBigtableReadOptions(); return toBuilder() - .setBigtableReadOptions(readOptions.toBuilder().setOperationTimeout(timeoutMs).build()) - .build(); - } - - /** - * Configures the initial retry delay in milliseconds. - * - *

Does not modify this object. - */ - public Read withRetryInitialDelay(long initialDelayMs) { - checkArgument(initialDelayMs > 0, "initial delay must be positive"); - BigtableReadOptions readOptions = getBigtableReadOptions(); - return toBuilder() - .setBigtableReadOptions( - readOptions.toBuilder().setRetryInitialDelay(initialDelayMs).build()) - .build(); - } - - /** - * Configures the delay multiplier. - * - *

Does not modify this object. - */ - public Read withRetryDelayMultiplier(double multiplier) { - checkArgument(multiplier > 0, "delay multiplier must be positive"); - BigtableReadOptions readOptions = getBigtableReadOptions(); - return toBuilder() - .setBigtableReadOptions( - readOptions.toBuilder().setRetryDelayMultiplier(multiplier).build()) + .setBigtableReadOptions(readOptions.toBuilder().setOperationTimeout(timeout).build()) .build(); } @@ -817,7 +798,8 @@ public Write withTableId(String tableId) { } /** - * Returns a new {@link BigtableIO.Write} with the provided credentials. + * Returns a new {@link BigtableIO.Write} with the provided credentials. If it's not set, it'll + * use the default credentials in {@link GcpOptions}. * *

Does not modify this object. */ @@ -905,59 +887,37 @@ public Write withEmulator(String emulatorHost) { } /** - * Returns a new {@link BigtableIO.Write} with the attempt timeout in milliseconds for writes. + * Returns a new {@link BigtableIO.Write} with the attempt timeout. Attempt timeout controls the + * timeout for each remote call. * *

Does not modify this object. */ - public Write withAttemptTimeout(long timeoutMs) { - checkArgument(timeoutMs > 0, "attempt timeout must be positive"); + public Write withAttemptTimeout(Duration timeout) { + checkArgument(timeout.isLongerThan(Duration.ZERO), "attempt timeout must be positive"); BigtableWriteOptions options = getBigtableWriteOptions(); return toBuilder() - .setBigtableWriteOptions(options.toBuilder().setAttemptTimeout(timeoutMs).build()) + .setBigtableWriteOptions(options.toBuilder().setAttemptTimeout(timeout).build()) .build(); } /** - * Returns a new {@link BigtableIO.Write} with the operation timeout in milliseconds for writes. + * Returns a new {@link BigtableIO.Write} with the operation timeout. Operation timeout has + * ultimate control over how long the logic should keep trying the remote call until it gives up + * completely. * *

Does not modify this object. */ - public Write withOperationTimeout(long timeoutMs) { - checkArgument(timeoutMs > 0, "operation timeout must be positive"); + public Write withOperationTimeout(Duration timeout) { + checkArgument(timeout.isLongerThan(Duration.ZERO), "operation timeout must be positive"); BigtableWriteOptions options = getBigtableWriteOptions(); return toBuilder() - .setBigtableWriteOptions(options.toBuilder().setOperationTimeout(timeoutMs).build()) + .setBigtableWriteOptions(options.toBuilder().setOperationTimeout(timeout).build()) .build(); } /** - * Returns a new {@link BigtableIO.Write} with the retry delay in milliseconds. - * - *

Does not modify this object. - */ - public Write withRetryInitialDelay(long delayMs) { - checkArgument(delayMs > 0, "delay must be positive"); - BigtableWriteOptions options = getBigtableWriteOptions(); - return toBuilder() - .setBigtableWriteOptions(options.toBuilder().setRetryInitialDelay(delayMs).build()) - .build(); - } - - /** - * Returns a new {@link BigtableIO.Write} with retry multiplier. - * - *

Does not modify this object. - */ - public Write withRetryDelayMultiplier(double multiplier) { - checkArgument(multiplier > 0, "multiplier must be positive"); - BigtableWriteOptions options = getBigtableWriteOptions(); - return toBuilder() - .setBigtableWriteOptions(options.toBuilder().setRetryDelayMultiplier(multiplier).build()) - .build(); - } - - /** - * Returns a new {@link BigtableIO.Write} with the number of elements in a batch. + * Returns a new {@link BigtableIO.Write} with batch element size. After this many elements are + * accumulated, they will be wrapped up in a batch and sent to Bigtable. * *

Does not modify this object. */ @@ -970,7 +930,8 @@ public Write withBatchElements(long size) { } /** - * Returns a new {@link BigtableIO.Write} with the number of bytes in a batch. + * Returns a new {@link BigtableIO.Write} with batch bytes size. After this many bytes are + * accumulated, the elements will be wrapped up in a batch and sent to Bigtable. * *

Does not modify this object. */ @@ -1105,9 +1066,11 @@ private void validateTableExists( private static class BigtableWriterFn extends DoFn>, BigtableWriteResult> { - private BigtableServiceFactory factory; - private BigtableServiceFactory.ConfigId id; - private BigtableServiceEntry serviceEntry; + private final BigtableServiceFactory factory; + private final BigtableServiceFactory.ConfigId id; + + // Assign serviceEntry in startBundle and clear it in tearDown. + @Nullable private BigtableServiceEntry serviceEntry; BigtableWriterFn( BigtableServiceFactory factory, @@ -1634,7 +1597,9 @@ private static class BigtableReader extends BoundedReader { private BigtableSource source; private final BigtableServiceFactory factory; - private BigtableServiceEntry serviceEntry; + + // Assign serviceEntry at construction time and clear it in close(). + @Nullable private BigtableServiceEntry serviceEntry; private BigtableService.Reader reader; private final ByteKeyRangeTracker rangeTracker; private long recordsReturned; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadOptions.java index 73d2f20f1465..3083fb2e0c8b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadOptions.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadOptions.java @@ -28,6 +28,7 @@ import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.display.DisplayData; import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Duration; /** Configuration for read from Bigtable. */ @AutoValue @@ -49,16 +50,10 @@ abstract class BigtableReadOptions implements Serializable { abstract @Nullable Integer getMaxBufferElementCount(); /** Returns the attempt timeout of the reads. */ - abstract @Nullable Long getAttemptTimeout(); + abstract @Nullable Duration getAttemptTimeout(); /** Returns the operation timeout of the reads. */ - abstract @Nullable Long getOperationTimeout(); - - /** Returns the retry delay. */ - abstract @Nullable Long getRetryInitialDelay(); - - /** Returns the retry delay multiplier. */ - abstract @Nullable Double getRetryDelayMultiplier(); + abstract @Nullable Duration getOperationTimeout(); abstract Builder toBuilder(); @@ -77,13 +72,9 @@ abstract static class Builder { abstract Builder setKeyRanges(ValueProvider> keyRanges); - abstract Builder setAttemptTimeout(long timeout); - - abstract Builder setOperationTimeout(long timeout); + abstract Builder setAttemptTimeout(Duration timeout); - abstract Builder setRetryInitialDelay(long delay); - - abstract Builder setRetryDelayMultiplier(double multiplier); + abstract Builder setOperationTimeout(Duration timeout); abstract BigtableReadOptions build(); } @@ -118,13 +109,7 @@ void populateDisplayData(DisplayData.Builder builder) { .withLabel("Read Attempt Timeout")) .addIfNotNull( DisplayData.item("operationTimeout", getOperationTimeout()) - .withLabel("Read Operation Timeout")) - .addIfNotNull( - DisplayData.item("retryInitialDelay", getRetryInitialDelay()) - .withLabel("Read retry initial delay")) - .addIfNotNull( - DisplayData.item("retryDelayMultiplier", getRetryDelayMultiplier()) - .withLabel("Read retry delay multiplier")); + .withLabel("Read Operation Timeout")); } void validate() { @@ -147,5 +132,11 @@ void validate() { checkArgument(range != null, "keyRanges cannot hold null range"); } } + + if (getAttemptTimeout() != null && getOperationTimeout() != null) { + checkArgument( + getAttemptTimeout().isShorterThan(getOperationTimeout()), + "attempt timeout can't be longer than operation timeout"); + } } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java index f65d83e616c4..c0f88331bd6e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java @@ -29,7 +29,7 @@ import java.util.concurrent.CompletionStage; import org.apache.beam.sdk.io.gcp.bigtable.BigtableIO.BigtableSource; import org.apache.beam.sdk.values.KV; -import org.threeten.bp.Duration; +import org.joda.time.Duration; /** An interface for real or fake implementations of Cloud Bigtable. */ interface BigtableService extends Serializable { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java index 74845582ca34..b132da343a12 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java @@ -35,7 +35,8 @@ /** * Factory class that caches {@link BigtableService} to share between workers with the same {@link - * BigtableConfig} and read / write options. + * BigtableConfig} and read / write options. A new {@link ConfigId} is created at graph construction + * time, and each {@link BigtableService} is mapped to one {@link ConfigId}. */ @SuppressWarnings({ "nullness" // TODO(https://github.com/apache/beam/issues/20497) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java index 1848b7db457e..3172ea9dc565 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java @@ -76,7 +76,7 @@ import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.Futures; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.SettableFuture; import org.checkerframework.checker.nullness.qual.Nullable; -import org.threeten.bp.Duration; +import org.joda.time.Duration; /** * An implementation of {@link BigtableService} that actually communicates with the Cloud Bigtable @@ -97,8 +97,8 @@ public BigtableServiceImpl(BigtableDataSettings settings) throws IOException { this.projectId = settings.getProjectId(); this.instanceId = settings.getInstanceId(); RetrySettings retry = settings.getStubSettings().readRowsSettings().getRetrySettings(); - this.readAttemptTimeout = retry.getInitialRpcTimeout(); - this.readOperationTimeout = retry.getTotalTimeout(); + this.readAttemptTimeout = Duration.millis(retry.getInitialRpcTimeout().toMillis()); + this.readOperationTimeout = Duration.millis(retry.getTotalTimeout().toMillis()); } private final BigtableDataClient client; @@ -582,8 +582,8 @@ private static GrpcCallContext createScanCallContext( ctx.withCallOptions( CallOptions.DEFAULT.withDeadline( - Deadline.after(operationTimeout.toMillis(), TimeUnit.MILLISECONDS))); - ctx.withTimeout(attemptTimeout); + Deadline.after(operationTimeout.getMillis(), TimeUnit.MILLISECONDS))); + ctx.withTimeout(org.threeten.bp.Duration.ofMillis(attemptTimeout.getMillis())); return ctx; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteOptions.java index eece03ef1682..83faa8d9d77d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteOptions.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteOptions.java @@ -24,6 +24,7 @@ import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.display.DisplayData; import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Duration; /** Configuration for write to Bigtable. */ @AutoValue @@ -36,16 +37,10 @@ abstract class BigtableWriteOptions implements Serializable { abstract @Nullable ValueProvider getTableId(); /** Returns the attempt timeout for writes. */ - abstract @Nullable Long getAttemptTimeout(); + abstract @Nullable Duration getAttemptTimeout(); /** Returns the operation timeout for writes. */ - abstract @Nullable Long getOperationTimeout(); - - /** Returns the retry delay. */ - abstract @Nullable Long getRetryInitialDelay(); - - /** Returns the retry delay multiplier. */ - abstract @Nullable Double getRetryDelayMultiplier(); + abstract @Nullable Duration getOperationTimeout(); /** Returns the number of elements of a batch. */ abstract @Nullable Long getBatchElements(); @@ -67,13 +62,9 @@ abstract static class Builder { abstract Builder setTableId(ValueProvider tableId); - abstract Builder setAttemptTimeout(long timeout); - - abstract Builder setOperationTimeout(long timeout); + abstract Builder setAttemptTimeout(Duration timeout); - abstract Builder setRetryInitialDelay(long delay); - - abstract Builder setRetryDelayMultiplier(double multiplier); + abstract Builder setOperationTimeout(Duration timeout); abstract Builder setBatchElements(long size); @@ -98,13 +89,7 @@ void populateDisplayData(DisplayData.Builder builder) { DisplayData.item("operationTimeout", getOperationTimeout()) .withLabel("Write Operation Timeout")) .addIfNotNull( - DisplayData.item("retryInitialDelay", getRetryInitialDelay()) - .withLabel("Write retry initial delay")) - .addIfNotNull( - DisplayData.item("retryDelayMultiplier", getRetryDelayMultiplier()) - .withLabel("Write retry delay multiplier")) - .addIfNotNull( - DisplayData.item("batchELements", getBatchElements()) + DisplayData.item("batchElements", getBatchElements()) .withLabel("Write batch element count")) .addIfNotNull( DisplayData.item("batchBytes", getBatchBytes()).withLabel("Write batch byte size")) @@ -120,8 +105,8 @@ void validate() { if (getAttemptTimeout() != null && getOperationTimeout() != null) { checkArgument( - getAttemptTimeout() <= getOperationTimeout(), - "attempt timeout can't be greater than operation timeout"); + getAttemptTimeout().isShorterThan(getOperationTimeout()), + "attempt timeout can't be longer than operation timeout"); } } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java index bfecb3383e3c..d384b808eed7 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java @@ -96,13 +96,9 @@ public void testBigtableOptionsToBigtableReadOptions() { assertNotNull(readOptions.getAttemptTimeout()); assertNotNull(readOptions.getOperationTimeout()); - assertNotNull(readOptions.getRetryInitialDelay()); - assertNotNull(readOptions.getRetryDelayMultiplier()); - assertEquals(100, (long) readOptions.getAttemptTimeout()); - assertEquals(1000, (long) readOptions.getOperationTimeout()); - assertEquals(5, (long) readOptions.getRetryInitialDelay()); - assertEquals(1.5, (double) readOptions.getRetryDelayMultiplier(), 0); + assertEquals(org.joda.time.Duration.millis(100), readOptions.getAttemptTimeout()); + assertEquals(org.joda.time.Duration.millis(1000), readOptions.getOperationTimeout()); } @Test @@ -136,16 +132,12 @@ public void testBigtableOptionsToBigtableWriteOptions() { assertNotNull(writeOptions.getAttemptTimeout()); assertNotNull(writeOptions.getOperationTimeout()); - assertNotNull(writeOptions.getRetryInitialDelay()); - assertNotNull(writeOptions.getRetryDelayMultiplier()); assertNotNull(writeOptions.getBatchBytes()); assertNotNull(writeOptions.getBatchElements()); assertNotNull(writeOptions.getMaxRequests()); - assertEquals(200, (long) writeOptions.getAttemptTimeout()); - assertEquals(2000, (long) writeOptions.getOperationTimeout()); - assertEquals(15, (long) writeOptions.getRetryInitialDelay()); - assertEquals(2.5, writeOptions.getRetryDelayMultiplier(), 0); + assertEquals(org.joda.time.Duration.millis(200), writeOptions.getAttemptTimeout()); + assertEquals(org.joda.time.Duration.millis(2000), writeOptions.getOperationTimeout()); assertEquals(20, (long) writeOptions.getBatchBytes()); assertEquals(100, (long) writeOptions.getBatchElements()); assertEquals(5, (long) writeOptions.getMaxRequests()); @@ -163,10 +155,8 @@ public void testVeneerReadSettings() { BigtableReadOptions readOptions = BigtableReadOptions.builder() .setTableId(ValueProvider.StaticValueProvider.of("table")) - .setAttemptTimeout(101) - .setOperationTimeout(1001) - .setRetryInitialDelay(5) - .setRetryDelayMultiplier(1.5) + .setAttemptTimeout(org.joda.time.Duration.millis(101)) + .setOperationTimeout(org.joda.time.Duration.millis(1001)) .build(); PipelineOptions pipelineOptions = PipelineOptionsFactory.create(); @@ -185,11 +175,6 @@ public void testVeneerReadSettings() { assertEquals( Duration.ofMillis(1001), stubSettings.readRowsSettings().getRetrySettings().getTotalTimeout()); - assertEquals( - Duration.ofMillis(5), - stubSettings.readRowsSettings().getRetrySettings().getInitialRetryDelay()); - assertEquals( - 1.5, stubSettings.readRowsSettings().getRetrySettings().getRetryDelayMultiplier(), 0); } @Test @@ -204,10 +189,8 @@ public void testVeneerWriteSettings() { BigtableWriteOptions writeOptions = BigtableWriteOptions.builder() .setTableId(ValueProvider.StaticValueProvider.of("table")) - .setAttemptTimeout(101) - .setOperationTimeout(1001) - .setRetryInitialDelay(5) - .setRetryDelayMultiplier(1.5) + .setAttemptTimeout(org.joda.time.Duration.millis(101)) + .setOperationTimeout(org.joda.time.Duration.millis(1001)) .setMaxRequests(11) .setBatchElements(105) .setBatchBytes(102) @@ -229,11 +212,6 @@ public void testVeneerWriteSettings() { assertEquals( Duration.ofMillis(1001), stubSettings.bulkMutateRowsSettings().getRetrySettings().getTotalTimeout()); - assertEquals( - Duration.ofMillis(5), - stubSettings.bulkMutateRowsSettings().getRetrySettings().getInitialRetryDelay()); - assertEquals( - 1.5, stubSettings.bulkMutateRowsSettings().getRetrySettings().getRetryDelayMultiplier(), 0); assertEquals( 105, (long) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java index 9587fdbb50d9..7518948b210b 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java @@ -1809,13 +1809,13 @@ public Row getCurrentRow() { } @Override - public org.threeten.bp.Duration getAttemptTimeout() { - return org.threeten.bp.Duration.ofMillis(1000); + public Duration getAttemptTimeout() { + return Duration.millis(100); } @Override - public org.threeten.bp.Duration getOperationTimeout() { - return org.threeten.bp.Duration.ofMillis(1000); + public Duration getOperationTimeout() { + return Duration.millis(1000); } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java index b7446eb3009d..856d0cd22b82 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java @@ -76,6 +76,7 @@ import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.joda.time.Duration; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -180,8 +181,8 @@ public void testRead() throws IOException { mockBigtableSource.getTableId().get(), mockBigtableSource.getRanges(), null, - retrySettings.getInitialRpcTimeout(), - retrySettings.getTotalTimeout()); + Duration.millis(retrySettings.getInitialRpcTimeout().toMillis()), + Duration.millis(retrySettings.getTotalTimeout().toMillis())); underTest.start(); Assert.assertEquals(expectedRow, underTest.getCurrentRow()); @@ -241,8 +242,8 @@ public Void answer(InvocationOnMock invocation) throws Throwable { RowFilter.getDefaultInstance(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, - retrySettings.getInitialRpcTimeout(), - retrySettings.getTotalTimeout(), + Duration.millis(retrySettings.getInitialRpcTimeout().toMillis()), + Duration.millis(retrySettings.getTotalTimeout().toMillis()), mockCallMetric); underTest.start(); @@ -298,8 +299,8 @@ public void testReadSingleRangeAboveSegmentLimit() throws IOException { RowFilter.getDefaultInstance(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, - retrySettings.getInitialRpcTimeout(), - retrySettings.getTotalTimeout(), + Duration.millis(retrySettings.getInitialRpcTimeout().toMillis()), + Duration.millis(retrySettings.getTotalTimeout().toMillis()), mockCallMetric); List actualResults = new ArrayList<>(); @@ -368,8 +369,8 @@ public void testReadMultipleRanges() throws IOException { RowFilter.getDefaultInstance(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, - retrySettings.getInitialRpcTimeout(), - retrySettings.getTotalTimeout(), + Duration.millis(retrySettings.getInitialRpcTimeout().toMillis()), + Duration.millis(retrySettings.getTotalTimeout().toMillis()), mockCallMetric); List actualResults = new ArrayList<>(); @@ -439,8 +440,8 @@ public void testReadMultipleRangesOverlappingKeys() throws IOException { RowFilter.getDefaultInstance(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, - retrySettings.getInitialRpcTimeout(), - retrySettings.getTotalTimeout(), + Duration.millis(retrySettings.getInitialRpcTimeout().toMillis()), + Duration.millis(retrySettings.getTotalTimeout().toMillis()), mockCallMetric); List actualResults = new ArrayList<>(); @@ -495,8 +496,8 @@ public void testReadFullTableScan() throws IOException { RowFilter.getDefaultInstance(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, - retrySettings.getInitialRpcTimeout(), - retrySettings.getTotalTimeout(), + Duration.millis(retrySettings.getInitialRpcTimeout().toMillis()), + Duration.millis(retrySettings.getTotalTimeout().toMillis()), mockCallMetric); List actualResults = new ArrayList<>(); @@ -566,8 +567,8 @@ public void testReadFillBuffer() throws IOException { RowFilter.getDefaultInstance(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, - retrySettings.getInitialRpcTimeout(), - retrySettings.getTotalTimeout(), + Duration.millis(retrySettings.getInitialRpcTimeout().toMillis()), + Duration.millis(retrySettings.getTotalTimeout().toMillis()), mockCallMetric); List actualResults = new ArrayList<>(); @@ -655,8 +656,8 @@ public Void answer(InvocationOnMock invocation) throws Throwable { RowFilter.getDefaultInstance(), SEGMENT_SIZE, segmentByteLimit, - retrySettings.getInitialRpcTimeout(), - retrySettings.getTotalTimeout(), + Duration.millis(retrySettings.getInitialRpcTimeout().toMillis()), + Duration.millis(retrySettings.getTotalTimeout().toMillis()), mockCallMetric); List actualResults = new ArrayList<>(); @@ -721,8 +722,8 @@ public void run() { RowFilter.getDefaultInstance(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, - retrySettings.getInitialRpcTimeout(), - retrySettings.getTotalTimeout(), + Duration.millis(retrySettings.getInitialRpcTimeout().toMillis()), + Duration.millis(retrySettings.getTotalTimeout().toMillis()), mockCallMetric); IOException returnedError = null; From 7da30395ea1d3ff5cf2fc9f8e6a8c3ca6b2eb731 Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Fri, 17 Feb 2023 19:00:31 -0500 Subject: [PATCH 15/23] fix IT test --- .../gcp/bigtable/BigtableConfigTranslator.java | 4 ++-- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 18 ++++++++++++------ .../sdk/io/gcp/bigtable/BigtableWriteIT.java | 15 ++++++++++++++- 3 files changed, 28 insertions(+), 9 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java index b613e2f873df..f096e896683d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java @@ -219,8 +219,8 @@ private static BigtableDataSettings configureReadSettings( } /** - * Translate BigtableOptions to BigtableConfig for backward compatibility. If the values are set on - * BigtableConfig, ignore the settings in BigtableOptions. + * Translate BigtableOptions to BigtableConfig for backward compatibility. If the values are set + * on BigtableConfig, ignore the settings in BigtableOptions. */ static BigtableConfig translateToBigtableConfig( BigtableConfig config, BigtableOptions options, PipelineOptions pipelineOptions) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 2c294cf96a51..6dcf57ad67aa 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -432,7 +432,8 @@ public Read withCredentialFactory(CredentialFactory credentialFactory) { * *

Does not modify this object. * - * @deprecated please set the configurations directly. + * @deprecated please set the configurations directly: + * BigtableIO.read().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId). */ @Deprecated public Read withBigtableOptions(BigtableOptions options) { @@ -453,7 +454,8 @@ public Read withBigtableOptions(BigtableOptions options) { * *

Does not modify this object. * - * @deprecated please set the configurations directly. + * @deprecated please set the configurations directly: + * BigtableIO.read().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId). */ @Deprecated public Read withBigtableOptions(BigtableOptions.Builder optionsBuilder) { @@ -472,7 +474,8 @@ public Read withBigtableOptions(BigtableOptions.Builder optionsBuilder) { * *

Does not modify this object. * - * @deprecated please set the configurations directly. + * @deprecated please set the configurations directly: + * BigtableIO.read().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId). */ @Deprecated public Read withBigtableOptionsConfigurator( @@ -817,7 +820,8 @@ public Write withCredentialFactory(CredentialFactory credentialFactory) { * *

Does not modify this object. * - * @deprecated please configure the write options directly. + * @deprecated please configure the write options directly: + * BigtableIO.write().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId). */ @Deprecated public Write withBigtableOptions(BigtableOptions options) { @@ -838,7 +842,8 @@ public Write withBigtableOptions(BigtableOptions options) { * *

Does not modify this object. * - * @deprecated please configure the write options directly. + * @deprecated please configure the write options directly: + * BigtableIO.write().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId). */ @Deprecated public Write withBigtableOptions(BigtableOptions.Builder optionsBuilder) { @@ -857,7 +862,8 @@ public Write withBigtableOptions(BigtableOptions.Builder optionsBuilder) { * *

Does not modify this object. * - * @deprecated please configure the write options directly. + * @deprecated please configure the write options directly: + * BigtableIO.write().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId). */ @Deprecated public Write withBigtableOptionsConfigurator( diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java index b2285b0951ad..1d0814211804 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java @@ -22,6 +22,7 @@ import com.google.api.gax.rpc.ServerStream; import com.google.bigtable.v2.Mutation; import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminSettings; import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; import com.google.cloud.bigtable.admin.v2.models.Table; import com.google.cloud.bigtable.data.v2.BigtableDataClient; @@ -81,6 +82,7 @@ public void setup() throws Exception { .setProjectId(ValueProvider.StaticValueProvider.of(project)) .setInstanceId(ValueProvider.StaticValueProvider.of(options.getInstanceId())) .setUserAgent("apache-beam-test") + .setValidate(true) .build(); veneerSettings = @@ -89,7 +91,14 @@ public void setup() throws Exception { BigtableWriteOptions.builder().build(), PipelineOptionsFactory.create()); + BigtableTableAdminSettings adminSettings = + BigtableTableAdminSettings.newBuilder() + .setProjectId(project) + .setInstanceId(options.getInstanceId()) + .build(); + client = BigtableDataClient.create(veneerSettings); + tableAdminClient = BigtableTableAdminClient.create(adminSettings); } @Test @@ -119,7 +128,11 @@ public void processElement(ProcessContext c) { c.output(KV.of(testData.get(index).getKey(), mutations)); } })) - .apply(BigtableIO.write().withTableId(tableId)); + .apply( + BigtableIO.write() + .withProjectId(project) + .withInstanceId(options.getInstanceId()) + .withTableId(tableId)); p.run(); // Test number of column families and column family name equality From f2332bcdf7cb40e438ac9a4e11cc7a3cc1e917c7 Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Tue, 21 Feb 2023 10:39:50 -0500 Subject: [PATCH 16/23] update WriteOptions api --- .../bigtable/BigtableConfigTranslator.java | 29 ++++++++++--------- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 23 ++++++++++++--- .../io/gcp/bigtable/BigtableWriteOptions.java | 18 ++++++++---- .../BigtableConfigTranslatorTest.java | 13 +++++---- 4 files changed, 55 insertions(+), 28 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java index f096e896683d..95a156f5bed7 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.gcp.bigtable; import com.google.api.gax.batching.BatchingSettings; +import com.google.api.gax.batching.FlowControlSettings; import com.google.api.gax.core.FixedCredentialsProvider; import com.google.api.gax.retrying.RetrySettings; import com.google.api.gax.rpc.FixedHeaderProvider; @@ -167,20 +168,15 @@ private static BigtableDataSettings configureWriteSettings( batchingSettings.setRequestByteThreshold(writeOptions.getBatchBytes()); } - if (writeOptions.getMaxRequests() != null) { - BatchingSettings tmpSettings = batchingSettings.build(); - batchingSettings = - batchingSettings.setFlowControlSettings( - callSettings - .getBatchingSettings() - .getFlowControlSettings() - .toBuilder() - .setMaxOutstandingElementCount( - tmpSettings.getElementCountThreshold() * writeOptions.getMaxRequests()) - .setMaxOutstandingRequestBytes( - tmpSettings.getRequestByteThreshold() * writeOptions.getMaxRequests()) - .build()); + FlowControlSettings.Builder flowControlSettings = + callSettings.getBatchingSettings().getFlowControlSettings().toBuilder(); + if (writeOptions.getMaxOutstandingElements() != null) { + flowControlSettings.setMaxOutstandingElementCount(writeOptions.getMaxOutstandingElements()); } + if (writeOptions.getMaxOutstandingBytes() != null) { + flowControlSettings.setMaxOutstandingRequestBytes(writeOptions.getMaxOutstandingBytes()); + } + batchingSettings = batchingSettings.setFlowControlSettings(flowControlSettings.build()); settings .stubSettings() @@ -334,7 +330,12 @@ static BigtableWriteOptions translateToBigtableWriteOptions( // configure batch size builder.setBatchElements(options.getBulkOptions().getBulkMaxRowKeyCount()); builder.setBatchBytes(options.getBulkOptions().getBulkMaxRequestSize()); - builder.setMaxRequests(options.getBulkOptions().getMaxInflightRpcs()); + builder.setMaxOutstandingElements( + options.getBulkOptions().getMaxInflightRpcs() + * (long) options.getBulkOptions().getBulkMaxRowKeyCount()); + builder.setMaxOutstandingBytes( + options.getBulkOptions().getMaxInflightRpcs() + * options.getBulkOptions().getBulkMaxRequestSize()); return builder.build(); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 6dcf57ad67aa..b6e3009a042f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -950,15 +950,30 @@ public Write withBatchBytes(long size) { } /** - * Returns a new {@link BigtableIO.Write} with the max number of concurrent requests. + * Returns a new {@link BigtableIO.Write} with the max number of outstanding elements allowed + * before enforcing flow control. * *

Does not modify this object. */ - public Write withMaxRequests(long requests) { - checkArgument(requests > 0, "max requests must be positive"); + public Write withMaxOutstandingElements(long count) { + checkArgument(count > 0, "max outstanding elements must be positive"); BigtableWriteOptions options = getBigtableWriteOptions(); return toBuilder() - .setBigtableWriteOptions(options.toBuilder().setMaxRequests(requests).build()) + .setBigtableWriteOptions(options.toBuilder().setMaxOutstandingElements(count).build()) + .build(); + } + + /** + * Returns a new {@link BigtableIO.Write} with the max number of outstanding bytes allowed + * before enforcing flow control. + * + *

Does not modify this object. + */ + public Write withMaxOutstandingBytes(long bytes) { + checkArgument(bytes > 0, "max outstanding bytes must be positive"); + BigtableWriteOptions options = getBigtableWriteOptions(); + return toBuilder() + .setBigtableWriteOptions(options.toBuilder().setMaxOutstandingBytes(bytes).build()) .build(); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteOptions.java index 83faa8d9d77d..60dc03d9918a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteOptions.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteOptions.java @@ -48,8 +48,11 @@ abstract class BigtableWriteOptions implements Serializable { /** Returns the number of bytes of a batch. */ abstract @Nullable Long getBatchBytes(); - /** Returns the max number of concurrent requests allowed. */ - abstract @Nullable Long getMaxRequests(); + /** Returns the max number of concurrent elements allowed before enforcing flow control. */ + abstract @Nullable Long getMaxOutstandingElements(); + + /** Returns the max number of concurrent bytes allowed before enforcing flow control. */ + abstract @Nullable Long getMaxOutstandingBytes(); abstract Builder toBuilder(); @@ -70,7 +73,9 @@ abstract static class Builder { abstract Builder setBatchBytes(long bytes); - abstract Builder setMaxRequests(long count); + abstract Builder setMaxOutstandingElements(long count); + + abstract Builder setMaxOutstandingBytes(long bytes); abstract BigtableWriteOptions build(); } @@ -94,8 +99,11 @@ void populateDisplayData(DisplayData.Builder builder) { .addIfNotNull( DisplayData.item("batchBytes", getBatchBytes()).withLabel("Write batch byte size")) .addIfNotNull( - DisplayData.item("maxRequests", getMaxRequests()) - .withLabel("Write max concurrent requests")); + DisplayData.item("maxOutstandingElements", getMaxOutstandingElements()) + .withLabel("Write max outstanding elements")) + .addIfNotNull( + DisplayData.item("maxOutstandingBytes", getMaxOutstandingBytes()) + .withLabel("Write max outstanding bytes")); } void validate() { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java index d384b808eed7..6d2ab9d381cc 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java @@ -134,13 +134,15 @@ public void testBigtableOptionsToBigtableWriteOptions() { assertNotNull(writeOptions.getOperationTimeout()); assertNotNull(writeOptions.getBatchBytes()); assertNotNull(writeOptions.getBatchElements()); - assertNotNull(writeOptions.getMaxRequests()); + assertNotNull(writeOptions.getMaxOutstandingElements()); + assertNotNull(writeOptions.getMaxOutstandingBytes()); assertEquals(org.joda.time.Duration.millis(200), writeOptions.getAttemptTimeout()); assertEquals(org.joda.time.Duration.millis(2000), writeOptions.getOperationTimeout()); assertEquals(20, (long) writeOptions.getBatchBytes()); assertEquals(100, (long) writeOptions.getBatchElements()); - assertEquals(5, (long) writeOptions.getMaxRequests()); + assertEquals(5 * 100, (long) writeOptions.getMaxOutstandingElements()); + assertEquals(5 * 20, (long) writeOptions.getMaxOutstandingBytes()); } @Test @@ -191,9 +193,10 @@ public void testVeneerWriteSettings() { .setTableId(ValueProvider.StaticValueProvider.of("table")) .setAttemptTimeout(org.joda.time.Duration.millis(101)) .setOperationTimeout(org.joda.time.Duration.millis(1001)) - .setMaxRequests(11) .setBatchElements(105) .setBatchBytes(102) + .setMaxOutstandingElements(10001) + .setMaxOutstandingBytes(100001) .build(); PipelineOptions pipelineOptions = PipelineOptionsFactory.create(); @@ -221,7 +224,7 @@ public void testVeneerWriteSettings() { (long) stubSettings.bulkMutateRowsSettings().getBatchingSettings().getRequestByteThreshold()); assertEquals( - 105 * 11, + 10001, (long) stubSettings .bulkMutateRowsSettings() @@ -230,7 +233,7 @@ public void testVeneerWriteSettings() { .getMaxOutstandingElementCount()); assertEquals( - 102 * 11, + 100001, (long) stubSettings .bulkMutateRowsSettings() From 5bad27cb18095148db494fb769d290a45570fbfc Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Tue, 21 Feb 2023 11:06:35 -0500 Subject: [PATCH 17/23] Fix Nonnull annotations and add Nullable annotations --- .../bigtable/BigtableConfigTranslator.java | 16 +++++----- .../io/gcp/bigtable/BigtableServiceImpl.java | 31 +++++++++---------- 2 files changed, 23 insertions(+), 24 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java index 95a156f5bed7..d7401cbe67d0 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java @@ -37,7 +37,6 @@ import java.security.KeyStore; import java.security.PrivateKey; import java.util.Objects; -import javax.annotation.Nonnull; import org.apache.beam.sdk.extensions.gcp.auth.GcpCredentialFactory; import org.apache.beam.sdk.extensions.gcp.auth.NoopCredentialFactory; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; @@ -45,6 +44,7 @@ import org.apache.beam.sdk.options.ValueProvider; 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.collect.ImmutableMap; +import org.checkerframework.checker.nullness.qual.NonNull; import org.threeten.bp.Duration; /** @@ -62,18 +62,18 @@ class BigtableConfigTranslator { /** Translate BigtableConfig and BigtableReadOptions to Veneer settings. */ static BigtableDataSettings translateReadToVeneerSettings( - @Nonnull BigtableConfig config, - @Nonnull BigtableReadOptions options, - @Nonnull PipelineOptions pipelineOptions) { + @NonNull BigtableConfig config, + @NonNull BigtableReadOptions options, + @NonNull PipelineOptions pipelineOptions) { BigtableDataSettings.Builder settings = buildBigtableDataSettings(config, pipelineOptions); return configureReadSettings(settings, options); } /** Translate BigtableConfig and BigtableWriteOptions to Veneer settings. */ static BigtableDataSettings translateWriteToVeneerSettings( - @Nonnull BigtableConfig config, - @Nonnull BigtableWriteOptions options, - @Nonnull PipelineOptions pipelineOptions) { + @NonNull BigtableConfig config, + @NonNull BigtableWriteOptions options, + @NonNull PipelineOptions pipelineOptions) { BigtableDataSettings.Builder settings = buildBigtableDataSettings(config, pipelineOptions); return configureWriteSettings(settings, options); @@ -81,7 +81,7 @@ static BigtableDataSettings translateWriteToVeneerSettings( /** Translate BigtableConfig and BigtableWriteOptions to Veneer settings. */ static BigtableDataSettings translateToVeneerSettings( - @Nonnull BigtableConfig config, @Nonnull PipelineOptions pipelineOptions) { + @NonNull BigtableConfig config, @NonNull PipelineOptions pipelineOptions) { return buildBigtableDataSettings(config, pipelineOptions).build(); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java index 3172ea9dc565..5f3a7bbb4c9a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java @@ -61,7 +61,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import javax.annotation.Nonnull; import org.apache.beam.runners.core.metrics.GcpResourceIdentifiers; import org.apache.beam.runners.core.metrics.MonitoringInfoConstants; import org.apache.beam.runners.core.metrics.ServiceCallMetric; @@ -75,6 +74,7 @@ import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.FutureCallback; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.Futures; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.SettableFuture; +import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; @@ -619,8 +619,8 @@ private static final class StartPoint implements Comparable { private final ByteString value; private final boolean isClosed; - @Nonnull - static StartPoint extract(@Nonnull RowRange rowRange) { + @NonNull + static StartPoint extract(@NonNull RowRange rowRange) { switch (rowRange.getStartKeyCase()) { case STARTKEY_NOT_SET: return new StartPoint(ByteString.EMPTY, true); @@ -638,13 +638,13 @@ static StartPoint extract(@Nonnull RowRange rowRange) { } } - private StartPoint(@Nonnull ByteString value, boolean isClosed) { + private StartPoint(@NonNull ByteString value, boolean isClosed) { this.value = value; this.isClosed = isClosed; } @Override - public int compareTo(@Nonnull StartPoint o) { + public int compareTo(@NonNull StartPoint o) { return ComparisonChain.start() // Empty string comes first .compareTrueFirst(value.isEmpty(), o.value.isEmpty()) @@ -660,8 +660,8 @@ private static final class EndPoint implements Comparable { private final ByteString value; private final boolean isClosed; - @Nonnull - static EndPoint extract(@Nonnull RowRange rowRange) { + @NonNull + static EndPoint extract(@NonNull RowRange rowRange) { switch (rowRange.getEndKeyCase()) { case ENDKEY_NOT_SET: return new EndPoint(ByteString.EMPTY, true); @@ -679,13 +679,13 @@ static EndPoint extract(@Nonnull RowRange rowRange) { } } - private EndPoint(@Nonnull ByteString value, boolean isClosed) { + private EndPoint(@NonNull ByteString value, boolean isClosed) { this.value = value; this.isClosed = isClosed; } @Override - public int compareTo(@Nonnull EndPoint o) { + public int compareTo(@NonNull EndPoint o) { return ComparisonChain.start() // Empty string comes last .compareFalseFirst(value.isEmpty(), o.value.isEmpty()) @@ -717,13 +717,12 @@ private static class DefaultRowBuilder private com.google.bigtable.v2.Row.Builder protoBuilder = com.google.bigtable.v2.Row.newBuilder(); - private ByteString currentValue; - private Family.Builder lastFamily; - private String lastFamilyName; - private Column.Builder lastColumn; - private ByteString lastColumnName; - - private Cell.Builder lastCell; + private @Nullable ByteString currentValue; + private Family.@Nullable Builder lastFamily; + private @Nullable String lastFamilyName; + private Column.@Nullable Builder lastColumn; + private @Nullable ByteString lastColumnName; + private Cell.@Nullable Builder lastCell; @Override public void startRow(ByteString key) { From 97e96e85d2aba1ac2e2e2499dc35c02444c3549b Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Wed, 22 Feb 2023 13:26:19 -0500 Subject: [PATCH 18/23] add 2 tests on credentials, update CHANGES.md --- CHANGES.md | 1 + .../bigtable/BigtableConfigTranslator.java | 53 +++++++++----- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 8 +- .../gcp/bigtable/BigtableServiceFactory.java | 12 +-- .../BigtableConfigTranslatorTest.java | 73 ++++++++++++++++--- 5 files changed, 106 insertions(+), 41 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index e5c097439240..97b33263256b 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -66,6 +66,7 @@ * Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). * Added in JmsIO a retry policy for failed publications (Java) ([#24971](https://github.com/apache/beam/issues/24971)). * Support for `LZMA` compression/decompression of text files added to the Python SDK ([#25316](https://github.com/apache/beam/issues/25316)) +* Update BigtableIO to use the idiomatic bigtable client under the hood (Java) ([25592](https://github.com/apache/beam/issues/25592)) ## New Features / Improvements diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java index d7401cbe67d0..71be98965832 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java @@ -37,10 +37,12 @@ import java.security.KeyStore; import java.security.PrivateKey; import java.util.Objects; +import org.apache.beam.sdk.extensions.gcp.auth.CredentialFactory; import org.apache.beam.sdk.extensions.gcp.auth.GcpCredentialFactory; import org.apache.beam.sdk.extensions.gcp.auth.NoopCredentialFactory; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.ValueProvider; 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.collect.ImmutableMap; @@ -64,7 +66,8 @@ class BigtableConfigTranslator { static BigtableDataSettings translateReadToVeneerSettings( @NonNull BigtableConfig config, @NonNull BigtableReadOptions options, - @NonNull PipelineOptions pipelineOptions) { + @NonNull PipelineOptions pipelineOptions) + throws IOException { BigtableDataSettings.Builder settings = buildBigtableDataSettings(config, pipelineOptions); return configureReadSettings(settings, options); } @@ -73,7 +76,8 @@ static BigtableDataSettings translateReadToVeneerSettings( static BigtableDataSettings translateWriteToVeneerSettings( @NonNull BigtableConfig config, @NonNull BigtableWriteOptions options, - @NonNull PipelineOptions pipelineOptions) { + @NonNull PipelineOptions pipelineOptions) + throws IOException { BigtableDataSettings.Builder settings = buildBigtableDataSettings(config, pipelineOptions); return configureWriteSettings(settings, options); @@ -81,13 +85,13 @@ static BigtableDataSettings translateWriteToVeneerSettings( /** Translate BigtableConfig and BigtableWriteOptions to Veneer settings. */ static BigtableDataSettings translateToVeneerSettings( - @NonNull BigtableConfig config, @NonNull PipelineOptions pipelineOptions) { + @NonNull BigtableConfig config, @NonNull PipelineOptions pipelineOptions) throws IOException { return buildBigtableDataSettings(config, pipelineOptions).build(); } private static BigtableDataSettings.Builder buildBigtableDataSettings( - BigtableConfig config, PipelineOptions pipelineOptions) { + BigtableConfig config, PipelineOptions pipelineOptions) throws IOException { BigtableDataSettings.Builder dataBuilder; if (!Strings.isNullOrEmpty(config.getEmulatorHost())) { String hostAndPort = config.getEmulatorHost(); @@ -113,12 +117,23 @@ private static BigtableDataSettings.Builder buildBigtableDataSettings( dataBuilder.setAppProfileId(Objects.requireNonNull(config.getAppProfileId().get())); } - if (config.getCredentialFactory() != null) { + if (((GcpOptions) pipelineOptions).getGcpCredential() != null) { dataBuilder .stubSettings() .setCredentialsProvider( - FixedCredentialsProvider.create( - ((GcpOptions) config.getCredentialFactory()).getGcpCredential())); + FixedCredentialsProvider.create(((GcpOptions) pipelineOptions).getGcpCredential())); + } + + if (config.getCredentialFactory() != null) { + CredentialFactory credentialFactory = config.getCredentialFactory(); + try { + dataBuilder + .stubSettings() + .setCredentialsProvider( + FixedCredentialsProvider.create(credentialFactory.getCredential())); + } catch (GeneralSecurityException e) { + throw new RuntimeException("Exception getting credentials ", e); + } } configureHeaderProvider(dataBuilder.stubSettings(), pipelineOptions); @@ -218,8 +233,7 @@ private static BigtableDataSettings configureReadSettings( * Translate BigtableOptions to BigtableConfig for backward compatibility. If the values are set * on BigtableConfig, ignore the settings in BigtableOptions. */ - static BigtableConfig translateToBigtableConfig( - BigtableConfig config, BigtableOptions options, PipelineOptions pipelineOptions) { + static BigtableConfig translateToBigtableConfig(BigtableConfig config, BigtableOptions options) { BigtableConfig.Builder builder = config.toBuilder(); if (options.getProjectId() != null && config.getProjectId() == null) { @@ -239,12 +253,13 @@ static BigtableConfig translateToBigtableConfig( builder.setEmulatorHost(String.format("%s:%s", options.getDataHost(), options.getPort())); } + GcpOptions pipelineOptions = PipelineOptionsFactory.as(GcpOptions.class); if (options.getCredentialOptions() != null) { try { CredentialOptions credOptions = options.getCredentialOptions(); switch (credOptions.getCredentialType()) { case DefaultCredentials: - // Veneer uses GoogleDefaultCredentials, so we don't need to reset it + // Veneer uses default credentials, so no need to reset here break; case P12: String keyFile = ((CredentialOptions.P12CredentialOptions) credOptions).getKeyFile(); @@ -262,12 +277,11 @@ static BigtableConfig translateToBigtableConfig( if (privateKey == null) { throw new IllegalStateException("private key cannot be null"); } - ((GcpOptions) pipelineOptions) - .setGcpCredential( - ServiceAccountJwtAccessCredentials.newBuilder() - .setClientEmail(serviceAccount) - .setPrivateKey(privateKey) - .build()); + pipelineOptions.setGcpCredential( + ServiceAccountJwtAccessCredentials.newBuilder() + .setClientEmail(serviceAccount) + .setPrivateKey(privateKey) + .build()); builder.setCredentialFactory(GcpCredentialFactory.fromOptions(pipelineOptions)); } catch (GeneralSecurityException exception) { throw new RuntimeException("exception while retrieving credentials", exception); @@ -276,15 +290,14 @@ static BigtableConfig translateToBigtableConfig( case SuppliedCredentials: Credentials credentials = ((CredentialOptions.UserSuppliedCredentialOptions) credOptions).getCredential(); - ((GcpOptions) pipelineOptions).setGcpCredential(credentials); + pipelineOptions.setGcpCredential(credentials); builder.setCredentialFactory(GcpCredentialFactory.fromOptions(pipelineOptions)); break; case SuppliedJson: CredentialOptions.JsonCredentialsOptions jsonCredentialsOptions = (CredentialOptions.JsonCredentialsOptions) credOptions; - ((GcpOptions) pipelineOptions) - .setGcpCredential( - GoogleCredentials.fromStream(jsonCredentialsOptions.getInputStream())); + pipelineOptions.setGcpCredential( + GoogleCredentials.fromStream(jsonCredentialsOptions.getInputStream())); builder.setCredentialFactory(GcpCredentialFactory.fromOptions(pipelineOptions)); break; case None: diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index b6e3009a042f..0777219a7078 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -412,8 +412,9 @@ public Read withTableId(String tableId) { } /** - * Returns a new {@link BigtableIO.Read} with provided credentials. If it's not set, it'll use - * the default credentials in {@link GcpOptions}. + * Returns a new {@link BigtableIO.Read} with the provided credentials. If it's not set, it'll + * use the credentials in {@link GcpOptions}. If credentials in {@link GcpOptions} is also not + * set, it'll use Google default credentials. * *

Does not modify this object. */ @@ -802,7 +803,8 @@ public Write withTableId(String tableId) { /** * Returns a new {@link BigtableIO.Write} with the provided credentials. If it's not set, it'll - * use the default credentials in {@link GcpOptions}. + * use the credentials in {@link GcpOptions}. If credentials in {@link GcpOptions} is also not + * set, it'll use Google default credentials. * *

Does not modify this object. */ diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java index b132da343a12..4cf101f78e45 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java @@ -102,9 +102,7 @@ synchronized BigtableServiceEntry getServiceForReading( BigtableOptions effectiveOptions = getEffectiveOptions(config); if (effectiveOptions != null) { // If BigtableOptions is set, convert it to BigtableConfig and BigtableWriteOptions - config = - BigtableConfigTranslator.translateToBigtableConfig( - config, effectiveOptions, pipelineOptions); + config = BigtableConfigTranslator.translateToBigtableConfig(config, effectiveOptions); opts = BigtableConfigTranslator.translateToBigtableReadOptions(opts, effectiveOptions); } BigtableDataSettings settings = @@ -130,9 +128,7 @@ synchronized BigtableServiceEntry getServiceForWriting( BigtableOptions effectiveOptions = getEffectiveOptions(config); if (effectiveOptions != null) { // If BigtableOptions is set, convert it to BigtableConfig and BigtableWriteOptions - config = - BigtableConfigTranslator.translateToBigtableConfig( - config, effectiveOptions, pipelineOptions); + config = BigtableConfigTranslator.translateToBigtableConfig(config, effectiveOptions); opts = BigtableConfigTranslator.translateToBigtableWriteOptions(opts, effectiveOptions); } @@ -155,9 +151,7 @@ boolean checkTableExists(BigtableConfig config, PipelineOptions pipelineOptions, throws IOException { BigtableOptions effectiveOptions = getEffectiveOptions(config); if (effectiveOptions != null) { - config = - BigtableConfigTranslator.translateToBigtableConfig( - config, effectiveOptions, pipelineOptions); + config = BigtableConfigTranslator.translateToBigtableConfig(config, effectiveOptions); } if (config.isDataAccessible()) { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java index 6d2ab9d381cc..314abd17d567 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java @@ -28,6 +28,7 @@ import com.google.cloud.bigtable.data.v2.BigtableDataSettings; import com.google.cloud.bigtable.data.v2.stub.EnhancedBigtableStubSettings; import org.apache.beam.sdk.extensions.gcp.auth.NoopCredentialFactory; +import org.apache.beam.sdk.extensions.gcp.auth.TestCredential; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -54,9 +55,7 @@ public void testBigtableOptionsToBigtableConfig() throws Exception { .build(); BigtableConfig config = BigtableConfig.builder().setValidate(true).build(); - config = - BigtableConfigTranslator.translateToBigtableConfig( - config, options, PipelineOptionsFactory.as(GcpOptions.class)); + config = BigtableConfigTranslator.translateToBigtableConfig(config, options); assertNotNull(config.getProjectId()); assertNotNull(config.getInstanceId()); @@ -75,7 +74,7 @@ public void testBigtableOptionsToBigtableConfig() throws Exception { } @Test - public void testBigtableOptionsToBigtableReadOptions() { + public void testBigtableOptionsToBigtableReadOptions() throws Exception { BigtableOptions options = BigtableOptions.builder() .setCallOptionsConfig( @@ -102,7 +101,7 @@ public void testBigtableOptionsToBigtableReadOptions() { } @Test - public void testBigtableOptionsToBigtableWriteOptions() { + public void testBigtableOptionsToBigtableWriteOptions() throws Exception { BigtableOptions options = BigtableOptions.builder() .setCallOptionsConfig( @@ -146,7 +145,7 @@ public void testBigtableOptionsToBigtableWriteOptions() { } @Test - public void testVeneerReadSettings() { + public void testVeneerReadSettings() throws Exception { BigtableConfig config = BigtableConfig.builder() .setProjectId(ValueProvider.StaticValueProvider.of("project")) @@ -160,7 +159,8 @@ public void testVeneerReadSettings() { .setAttemptTimeout(org.joda.time.Duration.millis(101)) .setOperationTimeout(org.joda.time.Duration.millis(1001)) .build(); - PipelineOptions pipelineOptions = PipelineOptionsFactory.create(); + + PipelineOptions pipelineOptions = PipelineOptionsFactory.as(GcpOptions.class); BigtableDataSettings settings = BigtableConfigTranslator.translateReadToVeneerSettings( @@ -180,7 +180,7 @@ public void testVeneerReadSettings() { } @Test - public void testVeneerWriteSettings() { + public void testVeneerWriteSettings() throws Exception { BigtableConfig config = BigtableConfig.builder() .setProjectId(ValueProvider.StaticValueProvider.of("project")) @@ -198,7 +198,8 @@ public void testVeneerWriteSettings() { .setMaxOutstandingElements(10001) .setMaxOutstandingBytes(100001) .build(); - PipelineOptions pipelineOptions = PipelineOptionsFactory.create(); + + PipelineOptions pipelineOptions = PipelineOptionsFactory.as(GcpOptions.class); BigtableDataSettings settings = BigtableConfigTranslator.translateWriteToVeneerSettings( @@ -241,4 +242,58 @@ public void testVeneerWriteSettings() { .getFlowControlSettings() .getMaxOutstandingRequestBytes()); } + + @Test + public void testUsingNullCredentialsFromBigtableOptions() throws Exception { + BigtableOptions options = + BigtableOptions.builder() + .setProjectId("project") + .setInstanceId("instance") + .setAppProfileId("app-profile") + .setDataHost("localhost") + .setPort(1234) + .setCredentialOptions(CredentialOptions.nullCredential()) + .build(); + + GcpOptions pipelineOptions = PipelineOptionsFactory.as(GcpOptions.class); + pipelineOptions.setGcpCredential(new TestCredential()); + + BigtableConfig config = BigtableConfig.builder().setValidate(true).build(); + config = BigtableConfigTranslator.translateToBigtableConfig(config, options); + + BigtableDataSettings veneerSettings = + BigtableConfigTranslator.translateToVeneerSettings(config, pipelineOptions); + + assertNotNull(veneerSettings.getStubSettings().getCredentialsProvider()); + + NoopCredentialFactory factory = + NoopCredentialFactory.fromOptions(PipelineOptionsFactory.create()); + assertEquals( + factory.getCredential(), + veneerSettings.getStubSettings().getCredentialsProvider().getCredentials()); + } + + @Test + public void testUsingPipelineOptionsCredential() throws Exception { + GcpOptions pipelineOptions = PipelineOptionsFactory.as(GcpOptions.class); + TestCredential testCredential = new TestCredential(); + pipelineOptions.setGcpCredential(testCredential); + BigtableOptions options = BigtableOptions.builder().build(); + BigtableConfig config = + BigtableConfig.builder() + .setProjectId(ValueProvider.StaticValueProvider.of("project")) + .setInstanceId(ValueProvider.StaticValueProvider.of("instance")) + .setValidate(true) + .build(); + + config = BigtableConfigTranslator.translateToBigtableConfig(config, options); + + BigtableDataSettings veneerSettings = + BigtableConfigTranslator.translateToVeneerSettings(config, pipelineOptions); + + assertNotNull(veneerSettings.getStubSettings().getCredentialsProvider()); + + assertEquals( + testCredential, veneerSettings.getStubSettings().getCredentialsProvider().getCredentials()); + } } From 9eca668f90a5f30cf528e03f2d7be303bee0e389 Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Wed, 22 Feb 2023 15:17:54 -0500 Subject: [PATCH 19/23] use as(GcpOptions.class) instead of casting, fixing tests --- .../bigtable/BigtableConfigTranslator.java | 34 +++++++++++-------- .../sdk/io/gcp/bigtable/BigtableWriteIT.java | 12 +++++-- 2 files changed, 29 insertions(+), 17 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java index 71be98965832..e06ecfac3efe 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java @@ -93,6 +93,7 @@ static BigtableDataSettings translateToVeneerSettings( private static BigtableDataSettings.Builder buildBigtableDataSettings( BigtableConfig config, PipelineOptions pipelineOptions) throws IOException { BigtableDataSettings.Builder dataBuilder; + boolean emulator = false; if (!Strings.isNullOrEmpty(config.getEmulatorHost())) { String hostAndPort = config.getEmulatorHost(); try { @@ -101,6 +102,7 @@ private static BigtableDataSettings.Builder buildBigtableDataSettings( dataBuilder = BigtableDataSettings.newBuilderForEmulator( hostAndPort.substring(0, lastIndexOfCol), port); + emulator = true; } catch (NumberFormatException | IndexOutOfBoundsException ex) { throw new RuntimeException("Invalid host/port in BigtableConfig " + hostAndPort); } @@ -117,22 +119,26 @@ private static BigtableDataSettings.Builder buildBigtableDataSettings( dataBuilder.setAppProfileId(Objects.requireNonNull(config.getAppProfileId().get())); } - if (((GcpOptions) pipelineOptions).getGcpCredential() != null) { - dataBuilder - .stubSettings() - .setCredentialsProvider( - FixedCredentialsProvider.create(((GcpOptions) pipelineOptions).getGcpCredential())); - } - - if (config.getCredentialFactory() != null) { - CredentialFactory credentialFactory = config.getCredentialFactory(); - try { + // Skip resetting the credentials if it's connected to an emulator + if (!emulator) { + if (pipelineOptions.as(GcpOptions.class).getGcpCredential() != null) { dataBuilder .stubSettings() .setCredentialsProvider( - FixedCredentialsProvider.create(credentialFactory.getCredential())); - } catch (GeneralSecurityException e) { - throw new RuntimeException("Exception getting credentials ", e); + FixedCredentialsProvider.create( + (pipelineOptions.as(GcpOptions.class)).getGcpCredential())); + } + + if (config.getCredentialFactory() != null) { + CredentialFactory credentialFactory = config.getCredentialFactory(); + try { + dataBuilder + .stubSettings() + .setCredentialsProvider( + FixedCredentialsProvider.create(credentialFactory.getCredential())); + } catch (GeneralSecurityException e) { + throw new RuntimeException("Exception getting credentials ", e); + } } } @@ -253,7 +259,7 @@ static BigtableConfig translateToBigtableConfig(BigtableConfig config, BigtableO builder.setEmulatorHost(String.format("%s:%s", options.getDataHost(), options.getPort())); } - GcpOptions pipelineOptions = PipelineOptionsFactory.as(GcpOptions.class); + GcpOptions pipelineOptions = PipelineOptionsFactory.create().as(GcpOptions.class); if (options.getCredentialOptions() != null) { try { CredentialOptions credOptions = options.getCredentialOptions(); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java index 1d0814211804..70df5b6c39ef 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java @@ -150,8 +150,12 @@ public void processElement(ProcessContext c) { @After public void tearDown() throws Exception { deleteTable(tableId); - tableAdminClient.close(); - client.close(); + if (tableAdminClient != null) { + tableAdminClient.close(); + } + if (client != null) { + client.close(); + } } //////////////////////////////////////////////////////////////////////////////////////////// @@ -197,6 +201,8 @@ private List> getTableData(String tableId) { /** Helper function to delete a table. */ private void deleteTable(String tableId) { - tableAdminClient.deleteTable(tableId); + if (tableAdminClient != null) { + tableAdminClient.deleteTable(tableId); + } } } From 2aeb1514a32aaee3f220a730e154a77f265155f9 Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Wed, 22 Feb 2023 15:20:29 -0500 Subject: [PATCH 20/23] nit move set emulator = true up --- .../beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java index e06ecfac3efe..4ed560dc49a3 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java @@ -95,6 +95,7 @@ private static BigtableDataSettings.Builder buildBigtableDataSettings( BigtableDataSettings.Builder dataBuilder; boolean emulator = false; if (!Strings.isNullOrEmpty(config.getEmulatorHost())) { + emulator = true; String hostAndPort = config.getEmulatorHost(); try { int lastIndexOfCol = hostAndPort.lastIndexOf(":"); @@ -102,7 +103,6 @@ private static BigtableDataSettings.Builder buildBigtableDataSettings( dataBuilder = BigtableDataSettings.newBuilderForEmulator( hostAndPort.substring(0, lastIndexOfCol), port); - emulator = true; } catch (NumberFormatException | IndexOutOfBoundsException ex) { throw new RuntimeException("Invalid host/port in BigtableConfig " + hostAndPort); } From 0bc19959afdc4c923fab5cab422e546059e754de Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Mon, 27 Feb 2023 18:08:02 -0500 Subject: [PATCH 21/23] revert credential changes, piping channel count --- .../sdk/io/gcp/bigtable/BigtableConfig.java | 9 ++-- .../bigtable/BigtableConfigTranslator.java | 37 +++++++++++----- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 43 +++++------------- .../gcp/bigtable/FixedCredentialFactory.java | 44 +++++++++++++++++++ .../BigtableConfigTranslatorTest.java | 14 +++--- 5 files changed, 95 insertions(+), 52 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/FixedCredentialFactory.java diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java index c2f6d7068c71..f1f276aa6a42 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java @@ -76,6 +76,9 @@ public abstract class BigtableConfig implements Serializable { */ abstract @Nullable CredentialFactory getCredentialFactory(); + /** Get number of channels. */ + abstract @Nullable Integer getChannelCount(); + abstract Builder toBuilder(); static BigtableConfig.Builder builder() { @@ -108,6 +111,8 @@ abstract Builder setBigtableOptionsConfigurator( abstract Builder setCredentialFactory(CredentialFactory credentialFactory); + abstract Builder setChannelCount(int count); + abstract BigtableConfig build(); } @@ -151,10 +156,6 @@ public BigtableConfig withEmulator(String emulatorHost) { return toBuilder().setEmulatorHost(emulatorHost).build(); } - BigtableConfig withCredentialFactory(CredentialFactory factory) { - return toBuilder().setCredentialFactory(factory).build(); - } - void validate() { checkArgument( (getProjectId() != null diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java index 4ed560dc49a3..762149d6e6c6 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java @@ -20,6 +20,8 @@ import com.google.api.gax.batching.BatchingSettings; import com.google.api.gax.batching.FlowControlSettings; import com.google.api.gax.core.FixedCredentialsProvider; +import com.google.api.gax.grpc.ChannelPoolSettings; +import com.google.api.gax.grpc.InstantiatingGrpcChannelProvider; import com.google.api.gax.retrying.RetrySettings; import com.google.api.gax.rpc.FixedHeaderProvider; import com.google.api.gax.rpc.StubSettings; @@ -38,7 +40,6 @@ import java.security.PrivateKey; import java.util.Objects; import org.apache.beam.sdk.extensions.gcp.auth.CredentialFactory; -import org.apache.beam.sdk.extensions.gcp.auth.GcpCredentialFactory; import org.apache.beam.sdk.extensions.gcp.auth.NoopCredentialFactory; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.options.PipelineOptions; @@ -142,6 +143,7 @@ private static BigtableDataSettings.Builder buildBigtableDataSettings( } } + configureChannelPool(dataBuilder.stubSettings(), config); configureHeaderProvider(dataBuilder.stubSettings(), pipelineOptions); return dataBuilder; @@ -159,6 +161,20 @@ private static void configureHeaderProvider( stubSettings.setHeaderProvider(FixedHeaderProvider.create(headersBuilder.build())); } + private static void configureChannelPool( + StubSettings.Builder stubSettings, BigtableConfig config) { + if (config.getChannelCount() != null + && stubSettings.getTransportChannelProvider() instanceof InstantiatingGrpcChannelProvider) { + InstantiatingGrpcChannelProvider grpcChannelProvider = + (InstantiatingGrpcChannelProvider) stubSettings.getTransportChannelProvider(); + stubSettings.setTransportChannelProvider( + grpcChannelProvider + .toBuilder() + .setChannelPoolSettings(ChannelPoolSettings.staticallySized(config.getChannelCount())) + .build()); + } + } + private static BigtableDataSettings configureWriteSettings( BigtableDataSettings.Builder settings, BigtableWriteOptions writeOptions) { BigtableBatchingCallSettings.Builder callSettings = @@ -259,7 +275,8 @@ static BigtableConfig translateToBigtableConfig(BigtableConfig config, BigtableO builder.setEmulatorHost(String.format("%s:%s", options.getDataHost(), options.getPort())); } - GcpOptions pipelineOptions = PipelineOptionsFactory.create().as(GcpOptions.class); + builder.setChannelCount(options.getChannelCount()); + if (options.getCredentialOptions() != null) { try { CredentialOptions credOptions = options.getCredentialOptions(); @@ -283,12 +300,12 @@ static BigtableConfig translateToBigtableConfig(BigtableConfig config, BigtableO if (privateKey == null) { throw new IllegalStateException("private key cannot be null"); } - pipelineOptions.setGcpCredential( + Credentials credentials = ServiceAccountJwtAccessCredentials.newBuilder() .setClientEmail(serviceAccount) .setPrivateKey(privateKey) - .build()); - builder.setCredentialFactory(GcpCredentialFactory.fromOptions(pipelineOptions)); + .build(); + builder.setCredentialFactory(FixedCredentialFactory.create(credentials)); } catch (GeneralSecurityException exception) { throw new RuntimeException("exception while retrieving credentials", exception); } @@ -296,18 +313,18 @@ static BigtableConfig translateToBigtableConfig(BigtableConfig config, BigtableO case SuppliedCredentials: Credentials credentials = ((CredentialOptions.UserSuppliedCredentialOptions) credOptions).getCredential(); - pipelineOptions.setGcpCredential(credentials); - builder.setCredentialFactory(GcpCredentialFactory.fromOptions(pipelineOptions)); + builder.setCredentialFactory(FixedCredentialFactory.create(credentials)); break; case SuppliedJson: CredentialOptions.JsonCredentialsOptions jsonCredentialsOptions = (CredentialOptions.JsonCredentialsOptions) credOptions; - pipelineOptions.setGcpCredential( - GoogleCredentials.fromStream(jsonCredentialsOptions.getInputStream())); - builder.setCredentialFactory(GcpCredentialFactory.fromOptions(pipelineOptions)); + builder.setCredentialFactory( + FixedCredentialFactory.create( + GoogleCredentials.fromStream(jsonCredentialsOptions.getInputStream()))); break; case None: // pipelineOptions is ignored + PipelineOptions pipelineOptions = PipelineOptionsFactory.create(); builder.setCredentialFactory(NoopCredentialFactory.fromOptions(pipelineOptions)); break; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 0777219a7078..321264053a93 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -46,8 +46,6 @@ import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.extensions.gcp.auth.CredentialFactory; -import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.BoundedSource.BoundedReader; @@ -411,18 +409,6 @@ public Read withTableId(String tableId) { return withTableId(StaticValueProvider.of(tableId)); } - /** - * Returns a new {@link BigtableIO.Read} with the provided credentials. If it's not set, it'll - * use the credentials in {@link GcpOptions}. If credentials in {@link GcpOptions} is also not - * set, it'll use Google default credentials. - * - *

Does not modify this object. - */ - public Read withCredentialFactory(CredentialFactory credentialFactory) { - BigtableConfig config = getBigtableConfig(); - return toBuilder().setBigtableConfig(config.withCredentialFactory(credentialFactory)).build(); - } - /** * WARNING: Should be used only to specify additional parameters for connection to the Cloud * Bigtable, instanceId and projectId should be provided over {@link #withInstanceId} and {@link @@ -434,7 +420,8 @@ public Read withCredentialFactory(CredentialFactory credentialFactory) { *

Does not modify this object. * * @deprecated please set the configurations directly: - * BigtableIO.read().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId). + * BigtableIO.read().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId) + * and set credentials in {@link PipelineOptions}. */ @Deprecated public Read withBigtableOptions(BigtableOptions options) { @@ -456,7 +443,8 @@ public Read withBigtableOptions(BigtableOptions options) { *

Does not modify this object. * * @deprecated please set the configurations directly: - * BigtableIO.read().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId). + * BigtableIO.read().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId) + * and set credentials in {@link PipelineOptions}. */ @Deprecated public Read withBigtableOptions(BigtableOptions.Builder optionsBuilder) { @@ -476,7 +464,8 @@ public Read withBigtableOptions(BigtableOptions.Builder optionsBuilder) { *

Does not modify this object. * * @deprecated please set the configurations directly: - * BigtableIO.read().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId). + * BigtableIO.read().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId) + * and set credentials in {@link PipelineOptions}. */ @Deprecated public Read withBigtableOptionsConfigurator( @@ -801,17 +790,6 @@ public Write withTableId(String tableId) { return withTableId(StaticValueProvider.of(tableId)); } - /** - * Returns a new {@link BigtableIO.Write} with the provided credentials. If it's not set, it'll - * use the credentials in {@link GcpOptions}. If credentials in {@link GcpOptions} is also not - * set, it'll use Google default credentials. - * - *

Does not modify this object. - */ - public Write withCredentialFactory(CredentialFactory credentialFactory) { - BigtableConfig config = getBigtableConfig(); - return toBuilder().setBigtableConfig(config.withCredentialFactory(credentialFactory)).build(); - } /** * WARNING: Should be used only to specify additional parameters for connection to the Cloud * Bigtable, instanceId and projectId should be provided over {@link #withInstanceId} and {@link @@ -823,7 +801,8 @@ public Write withCredentialFactory(CredentialFactory credentialFactory) { *

Does not modify this object. * * @deprecated please configure the write options directly: - * BigtableIO.write().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId). + * BigtableIO.write().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId) + * and set credentials in {@link PipelineOptions}. */ @Deprecated public Write withBigtableOptions(BigtableOptions options) { @@ -845,7 +824,8 @@ public Write withBigtableOptions(BigtableOptions options) { *

Does not modify this object. * * @deprecated please configure the write options directly: - * BigtableIO.write().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId). + * BigtableIO.write().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId) + * and set credentials in {@link PipelineOptions}. */ @Deprecated public Write withBigtableOptions(BigtableOptions.Builder optionsBuilder) { @@ -865,7 +845,8 @@ public Write withBigtableOptions(BigtableOptions.Builder optionsBuilder) { *

Does not modify this object. * * @deprecated please configure the write options directly: - * BigtableIO.write().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId). + * BigtableIO.write().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId) + * and set credentials in {@link PipelineOptions}. */ @Deprecated public Write withBigtableOptionsConfigurator( diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/FixedCredentialFactory.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/FixedCredentialFactory.java new file mode 100644 index 000000000000..f45a98753b04 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/FixedCredentialFactory.java @@ -0,0 +1,44 @@ +/* + * 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.bigtable; + +import com.google.auth.Credentials; +import org.apache.beam.sdk.extensions.gcp.auth.CredentialFactory; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * A fixed credential factory to return the credential set by users. This class is for backward + * compatibility if a user is setting credentials through BigtableOptions. + */ +class FixedCredentialFactory implements CredentialFactory { + + private Credentials credentials; + + private FixedCredentialFactory(Credentials credentials) { + this.credentials = credentials; + } + + static FixedCredentialFactory create(Credentials credentials) { + return new FixedCredentialFactory(credentials); + } + + @Override + public @Nullable Credentials getCredential() { + return credentials; + } +} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java index 314abd17d567..d193f1a2f0b2 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; +import com.google.auth.Credentials; import com.google.cloud.bigtable.config.BigtableOptions; import com.google.cloud.bigtable.config.BulkOptions; import com.google.cloud.bigtable.config.CallOptionsConfig; @@ -36,6 +37,7 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +import org.mockito.Mockito; import org.threeten.bp.Duration; /** Unit tests for {@link BigtableConfigTranslator}. */ @@ -244,15 +246,15 @@ public void testVeneerWriteSettings() throws Exception { } @Test - public void testUsingNullCredentialsFromBigtableOptions() throws Exception { + public void testUsingCredentialsFromBigtableOptions() throws Exception { + Credentials fakeCredentials = Mockito.mock(Credentials.class); BigtableOptions options = BigtableOptions.builder() .setProjectId("project") .setInstanceId("instance") .setAppProfileId("app-profile") - .setDataHost("localhost") - .setPort(1234) - .setCredentialOptions(CredentialOptions.nullCredential()) + .setCredentialOptions( + CredentialOptions.UserSuppliedCredentialOptions.credential(fakeCredentials)) .build(); GcpOptions pipelineOptions = PipelineOptionsFactory.as(GcpOptions.class); @@ -266,10 +268,8 @@ public void testUsingNullCredentialsFromBigtableOptions() throws Exception { assertNotNull(veneerSettings.getStubSettings().getCredentialsProvider()); - NoopCredentialFactory factory = - NoopCredentialFactory.fromOptions(PipelineOptionsFactory.create()); assertEquals( - factory.getCredential(), + fakeCredentials, veneerSettings.getStubSettings().getCredentialsProvider().getCredentials()); } From b08fcf1e3324df59b0eb8ac2bc5eaa4419f89e88 Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Mon, 27 Feb 2023 18:10:00 -0500 Subject: [PATCH 22/23] remove bigtable change from CHANGES.md --- CHANGES.md | 1 - 1 file changed, 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 7f42155ff41a..8514990fb03a 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -68,7 +68,6 @@ * Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). * Added in JmsIO a retry policy for failed publications (Java) ([#24971](https://github.com/apache/beam/issues/24971)). * Support for `LZMA` compression/decompression of text files added to the Python SDK ([#25316](https://github.com/apache/beam/issues/25316)) -* Update BigtableIO to use the idiomatic bigtable client under the hood (Java) ([25592](https://github.com/apache/beam/issues/25592)) ## New Features / Improvements From ce231c49d152502e86338143324a10d2cab50ec1 Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Thu, 2 Mar 2023 11:08:54 -0500 Subject: [PATCH 23/23] fix nits --- .../bigtable/BigtableConfigTranslator.java | 15 ++++++------- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 22 +++++++++---------- .../io/gcp/bigtable/BigtableWriteOptions.java | 17 +++++++------- .../BigtableConfigTranslatorTest.java | 12 +++++----- 4 files changed, 33 insertions(+), 33 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java index 762149d6e6c6..9c34091ad99a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java @@ -163,8 +163,7 @@ private static void configureHeaderProvider( private static void configureChannelPool( StubSettings.Builder stubSettings, BigtableConfig config) { - if (config.getChannelCount() != null - && stubSettings.getTransportChannelProvider() instanceof InstantiatingGrpcChannelProvider) { + if (config.getChannelCount() != null) { InstantiatingGrpcChannelProvider grpcChannelProvider = (InstantiatingGrpcChannelProvider) stubSettings.getTransportChannelProvider(); stubSettings.setTransportChannelProvider( @@ -197,12 +196,12 @@ private static BigtableDataSettings configureWriteSettings( Duration.ofMillis(writeOptions.getOperationTimeout().getMillis())); } - if (writeOptions.getBatchElements() != null) { - batchingSettings.setElementCountThreshold(writeOptions.getBatchElements()); + if (writeOptions.getMaxElementsPerBatch() != null) { + batchingSettings.setElementCountThreshold(writeOptions.getMaxElementsPerBatch()); } - if (writeOptions.getBatchBytes() != null) { - batchingSettings.setRequestByteThreshold(writeOptions.getBatchBytes()); + if (writeOptions.getMaxBytesPerBatch() != null) { + batchingSettings.setRequestByteThreshold(writeOptions.getMaxBytesPerBatch()); } FlowControlSettings.Builder flowControlSettings = @@ -364,8 +363,8 @@ static BigtableWriteOptions translateToBigtableWriteOptions( builder.setOperationTimeout( org.joda.time.Duration.millis(options.getCallOptionsConfig().getMutateRpcTimeoutMs())); // configure batch size - builder.setBatchElements(options.getBulkOptions().getBulkMaxRowKeyCount()); - builder.setBatchBytes(options.getBulkOptions().getBulkMaxRequestSize()); + builder.setMaxElementsPerBatch(options.getBulkOptions().getBulkMaxRowKeyCount()); + builder.setMaxBytesPerBatch(options.getBulkOptions().getBulkMaxRequestSize()); builder.setMaxOutstandingElements( options.getBulkOptions().getMaxInflightRpcs() * (long) options.getBulkOptions().getBulkMaxRowKeyCount()); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 321264053a93..750f3d65526b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -905,30 +905,30 @@ public Write withOperationTimeout(Duration timeout) { } /** - * Returns a new {@link BigtableIO.Write} with batch element size. After this many elements are - * accumulated, they will be wrapped up in a batch and sent to Bigtable. + * Returns a new {@link BigtableIO.Write} with the max elements a batch can have. After this + * many elements are accumulated, they will be wrapped up in a batch and sent to Bigtable. * *

Does not modify this object. */ - public Write withBatchElements(long size) { - checkArgument(size > 0, "batch element size must be positive"); + public Write withMaxElementsPerBatch(long size) { + checkArgument(size > 0, "max elements per batch size must be positive"); BigtableWriteOptions options = getBigtableWriteOptions(); return toBuilder() - .setBigtableWriteOptions(options.toBuilder().setBatchElements(size).build()) + .setBigtableWriteOptions(options.toBuilder().setMaxElementsPerBatch(size).build()) .build(); } /** - * Returns a new {@link BigtableIO.Write} with batch bytes size. After this many bytes are - * accumulated, the elements will be wrapped up in a batch and sent to Bigtable. + * Returns a new {@link BigtableIO.Write} with the max bytes a batch can have. After this many + * bytes are accumulated, the elements will be wrapped up in a batch and sent to Bigtable. * *

Does not modify this object. */ - public Write withBatchBytes(long size) { - checkArgument(size > 0, "batch byte size must be positive"); + public Write withMaxBytesPerBatch(long size) { + checkArgument(size > 0, "max bytes per batch size must be positive"); BigtableWriteOptions options = getBigtableWriteOptions(); return toBuilder() - .setBigtableWriteOptions(options.toBuilder().setBatchBytes(size).build()) + .setBigtableWriteOptions(options.toBuilder().setMaxBytesPerBatch(size).build()) .build(); } @@ -1201,10 +1201,10 @@ public BigtableSource( BigtableReadOptions readOptions, @Nullable Long estimatedSizeBytes) { this.factory = factory; + this.configId = configId; this.config = config; this.readOptions = readOptions; this.estimatedSizeBytes = estimatedSizeBytes; - this.configId = configId; } @Override diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteOptions.java index 60dc03d9918a..3e0a6e595fee 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteOptions.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteOptions.java @@ -42,11 +42,11 @@ abstract class BigtableWriteOptions implements Serializable { /** Returns the operation timeout for writes. */ abstract @Nullable Duration getOperationTimeout(); - /** Returns the number of elements of a batch. */ - abstract @Nullable Long getBatchElements(); + /** Returns the max number of elements of a batch. */ + abstract @Nullable Long getMaxElementsPerBatch(); - /** Returns the number of bytes of a batch. */ - abstract @Nullable Long getBatchBytes(); + /** Returns the max number of bytes of a batch. */ + abstract @Nullable Long getMaxBytesPerBatch(); /** Returns the max number of concurrent elements allowed before enforcing flow control. */ abstract @Nullable Long getMaxOutstandingElements(); @@ -69,9 +69,9 @@ abstract static class Builder { abstract Builder setOperationTimeout(Duration timeout); - abstract Builder setBatchElements(long size); + abstract Builder setMaxElementsPerBatch(long size); - abstract Builder setBatchBytes(long bytes); + abstract Builder setMaxBytesPerBatch(long bytes); abstract Builder setMaxOutstandingElements(long count); @@ -94,10 +94,11 @@ void populateDisplayData(DisplayData.Builder builder) { DisplayData.item("operationTimeout", getOperationTimeout()) .withLabel("Write Operation Timeout")) .addIfNotNull( - DisplayData.item("batchElements", getBatchElements()) + DisplayData.item("maxElementsPerBatch", getMaxElementsPerBatch()) .withLabel("Write batch element count")) .addIfNotNull( - DisplayData.item("batchBytes", getBatchBytes()).withLabel("Write batch byte size")) + DisplayData.item("maxBytesPerBatch", getMaxBytesPerBatch()) + .withLabel("Write batch byte size")) .addIfNotNull( DisplayData.item("maxOutstandingElements", getMaxOutstandingElements()) .withLabel("Write max outstanding elements")) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java index d193f1a2f0b2..f3a70fd225d3 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java @@ -133,15 +133,15 @@ public void testBigtableOptionsToBigtableWriteOptions() throws Exception { assertNotNull(writeOptions.getAttemptTimeout()); assertNotNull(writeOptions.getOperationTimeout()); - assertNotNull(writeOptions.getBatchBytes()); - assertNotNull(writeOptions.getBatchElements()); + assertNotNull(writeOptions.getMaxBytesPerBatch()); + assertNotNull(writeOptions.getMaxElementsPerBatch()); assertNotNull(writeOptions.getMaxOutstandingElements()); assertNotNull(writeOptions.getMaxOutstandingBytes()); assertEquals(org.joda.time.Duration.millis(200), writeOptions.getAttemptTimeout()); assertEquals(org.joda.time.Duration.millis(2000), writeOptions.getOperationTimeout()); - assertEquals(20, (long) writeOptions.getBatchBytes()); - assertEquals(100, (long) writeOptions.getBatchElements()); + assertEquals(20, (long) writeOptions.getMaxBytesPerBatch()); + assertEquals(100, (long) writeOptions.getMaxElementsPerBatch()); assertEquals(5 * 100, (long) writeOptions.getMaxOutstandingElements()); assertEquals(5 * 20, (long) writeOptions.getMaxOutstandingBytes()); } @@ -195,8 +195,8 @@ public void testVeneerWriteSettings() throws Exception { .setTableId(ValueProvider.StaticValueProvider.of("table")) .setAttemptTimeout(org.joda.time.Duration.millis(101)) .setOperationTimeout(org.joda.time.Duration.millis(1001)) - .setBatchElements(105) - .setBatchBytes(102) + .setMaxElementsPerBatch(105) + .setMaxBytesPerBatch(102) .setMaxOutstandingElements(10001) .setMaxOutstandingBytes(100001) .build();