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 7c04d81ef58a..b1e3fdc009ff 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -598,7 +598,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 de692d2d651e..19ebcc465c8e 100644 --- a/sdks/java/extensions/sql/build.gradle +++ b/sdks/java/extensions/sql/build.gradle @@ -117,7 +117,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 index 6a8b343b18f6..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 @@ -20,17 +20,15 @@ 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.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 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.checkerframework.checker.nullness.qual.Nullable; @@ -38,8 +36,6 @@ class BigtableClientWrapper implements Serializable { private final BigtableTableAdminClient tableAdminClient; private final BigtableDataClient dataClient; - private final BigtableSession session; - private final BigtableOptions bigtableOptions; BigtableClientWrapper( String project, @@ -47,22 +43,22 @@ class BigtableClientWrapper implements Serializable { @Nullable Integer emulatorPort, @Nullable Credentials gcpCredentials) throws IOException { - BigtableOptions.Builder optionsBuilder = - BigtableOptions.builder() + BigtableDataSettings.Builder settings = + BigtableDataSettings.newBuilderForEmulator(emulatorPort) .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(); + .setCredentialsProvider(FixedCredentialsProvider.create(gcpCredentials)); - session = new BigtableSession(bigtableOptions); - tableAdminClient = session.getTableAdminClient(); - dataClient = session.getDataClient(); + 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); } void writeRow( @@ -72,44 +68,24 @@ void writeRow( 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); + RowMutation rowMutation = + RowMutation.create(table, key) + .setCell( + familyColumn, byteStringUtf8(columnQualifier), timestampMicros, byteString(value)); + dataClient.mutateRow(rowMutation); } 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()); + CreateTableRequest createTableRequest = CreateTableRequest.of(tableName).addFamily(familyName); + tableAdminClient.createTable(createTableRequest); } void deleteTable(String tableId) { - final String tableName = bigtableOptions.getInstanceName().toTableNameStr(tableId); - DeleteTableRequest.Builder deleteTableRequestBuilder = - DeleteTableRequest.newBuilder().setName(tableName); - tableAdminClient.deleteTable(deleteTableRequestBuilder.build()); + tableAdminClient.deleteTable(tableId); } void closeSession() throws IOException { - session.close(); + dataClient.close(); + tableAdminClient.close(); } } diff --git a/sdks/java/io/google-cloud-platform/build.gradle b/sdks/java/io/google-cloud-platform/build.gradle index 076c90c56464..8b264a9c8b36 100644 --- a/sdks/java/io/google-cloud-platform/build.gradle +++ b/sdks/java/io/google-cloud-platform/build.gradle @@ -65,10 +65,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 @@ -133,7 +130,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/BigtableConfig.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java index 3e2d9bc5e060..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 @@ -21,11 +21,10 @@ import com.google.auto.value.AutoValue; import com.google.cloud.bigtable.config.BigtableOptions; -import com.google.cloud.bigtable.config.CredentialOptions; 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.PipelineOptions; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; @@ -47,9 +46,6 @@ public abstract class BigtableConfig implements Serializable { /** Returns the instance id being written to. */ public abstract @Nullable ValueProvider getInstanceId(); - /** Returns the table being read from. */ - public abstract @Nullable ValueProvider getTableId(); - /** Returns the app profile being read from. */ public abstract @Nullable ValueProvider getAppProfileId(); @@ -68,12 +64,21 @@ public abstract class BigtableConfig implements Serializable { /** 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. */ + /** Bigtable emulator. */ abstract @Nullable String getEmulatorHost(); + /** User agent for this job. */ + abstract @Nullable String getUserAgent(); + + /** + * Credentials for running the job. Use the default credentials in {@link GcpOptions} if it's not + * set. + */ + abstract @Nullable CredentialFactory getCredentialFactory(); + + /** Get number of channels. */ + abstract @Nullable Integer getChannelCount(); + abstract Builder toBuilder(); static BigtableConfig.Builder builder() { @@ -87,23 +92,27 @@ 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. */ + /** @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 setCredentialFactory(CredentialFactory credentialFactory); + + abstract Builder setChannelCount(int count); + abstract BigtableConfig build(); } @@ -117,23 +126,20 @@ public BigtableConfig withInstanceId(ValueProvider instanceId) { return toBuilder().setInstanceId(instanceId).build(); } - public BigtableConfig withTableId(ValueProvider tableId) { - checkArgument(tableId != null, "tableId can not be null"); - return toBuilder().setTableId(tableId).build(); - } - - public BigtableConfig withAppProfileId(ValueProvider appProfileId) { - checkArgument(appProfileId != null, "tableId can not be null"); + BigtableConfig withAppProfileId(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 please set the options directly in BigtableIO. */ @Deprecated public BigtableConfig withBigtableOptions(BigtableOptions options) { checkArgument(options != null, "Bigtable options can not be null"); return toBuilder().setBigtableOptions(options).build(); } + /** @deprecated please set the options directly in BigtableIO. */ + @Deprecated public BigtableConfig withBigtableOptionsConfigurator( SerializableFunction configurator) { checkArgument(configurator != null, "configurator can not be null"); @@ -144,12 +150,6 @@ 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"); @@ -157,10 +157,6 @@ public BigtableConfig withEmulator(String emulatorHost) { } void validate() { - checkArgument( - getTableId() != null && (!getTableId().isAccessible() || !getTableId().get().isEmpty()), - "Could not obtain Bigtable table id"); - checkArgument( (getProjectId() != null && (!getProjectId().isAccessible() || !getProjectId().get().isEmpty())) @@ -184,11 +180,9 @@ 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")) .addIfNotNull( DisplayData.item("appProfileId", getAppProfileId()) - .withLabel("Bigtable App Profile Id")) - .add(DisplayData.item("withValidation", getValidate()).withLabel("Check is table exists")); + .withLabel("Bigtable App Profile Id")); if (getBigtableOptions() != null) { builder.add( @@ -197,66 +191,10 @@ 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(); - } - - BigtableOptions.Builder bigtableOptions = effectiveUserProvidedBigtableOptions(); - - bigtableOptions.setUserAgent(pipelineOptions.getUserAgent()); - - if (bigtableOptions.build().getCredentialOptions().getCredentialType() - == CredentialOptions.CredentialType.DefaultCredentials) { - bigtableOptions.setCredentialOptions( - CredentialOptions.credential(pipelineOptions.as(GcpOptions.class).getGcpCredential())); - } - - return new BigtableServiceImpl(bigtableOptions.build()); - } - boolean isDataAccessible() { - return getTableId().isAccessible() - && (getProjectId() == null || getProjectId().isAccessible()) - && (getInstanceId() == null || getInstanceId().isAccessible()); - } - - private BigtableOptions.Builder effectiveUserProvidedBigtableOptions() { - BigtableOptions.Builder effectiveOptions = - getBigtableOptions() != null - ? getBigtableOptions().toBuilder() - : new BigtableOptions.Builder(); - - if (getBigtableOptionsConfigurator() != null) { - effectiveOptions = getBigtableOptionsConfigurator().apply(effectiveOptions); - } - - // Default option that should be forced in most cases - effectiveOptions.setUseCachedDataPool(true); - - if (getInstanceId() != null) { - effectiveOptions.setInstanceId(getInstanceId().get()); - } - - if (getProjectId() != null) { - effectiveOptions.setProjectId(getProjectId().get()); - } - - if (getEmulatorHost() != null) { - effectiveOptions.enableEmulator(getEmulatorHost()); - effectiveOptions.setUseCachedDataPool(false); - } - - return effectiveOptions; + return (getProjectId() == null || getProjectId().isAccessible()) + && (getInstanceId() == null || getInstanceId().isAccessible()) + && (getAppProfileId() == null || getAppProfileId().isAccessible()); } @Override @@ -264,14 +202,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/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..9c34091ad99a --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslator.java @@ -0,0 +1,377 @@ +/* + * 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.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; +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 org.apache.beam.sdk.extensions.gcp.auth.CredentialFactory; +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; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.threeten.bp.Duration; + +/** + * 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) +}) +class BigtableConfigTranslator { + + /** Translate BigtableConfig and BigtableReadOptions to Veneer settings. */ + static BigtableDataSettings translateReadToVeneerSettings( + @NonNull BigtableConfig config, + @NonNull BigtableReadOptions options, + @NonNull PipelineOptions pipelineOptions) + throws IOException { + 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) + throws IOException { + + BigtableDataSettings.Builder settings = buildBigtableDataSettings(config, pipelineOptions); + return configureWriteSettings(settings, options); + } + + /** Translate BigtableConfig and BigtableWriteOptions to Veneer settings. */ + static BigtableDataSettings translateToVeneerSettings( + @NonNull BigtableConfig config, @NonNull PipelineOptions pipelineOptions) throws IOException { + + return buildBigtableDataSettings(config, pipelineOptions).build(); + } + + private static BigtableDataSettings.Builder buildBigtableDataSettings( + BigtableConfig config, PipelineOptions pipelineOptions) throws IOException { + BigtableDataSettings.Builder dataBuilder; + boolean emulator = false; + if (!Strings.isNullOrEmpty(config.getEmulatorHost())) { + emulator = true; + 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 + && !Strings.isNullOrEmpty(config.getAppProfileId().get())) { + dataBuilder.setAppProfileId(Objects.requireNonNull(config.getAppProfileId().get())); + } + + // 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( + (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); + } + } + } + + configureChannelPool(dataBuilder.stubSettings(), config); + configureHeaderProvider(dataBuilder.stubSettings(), pipelineOptions); + + return dataBuilder; + } + + private static void configureHeaderProvider( + StubSettings.Builder stubSettings, PipelineOptions pipelineOptions) { + + ImmutableMap.Builder headersBuilder = + ImmutableMap.builder() + .put( + GrpcUtil.USER_AGENT_KEY.name(), + Objects.requireNonNull(pipelineOptions.getUserAgent())); + + stubSettings.setHeaderProvider(FixedHeaderProvider.create(headersBuilder.build())); + } + + private static void configureChannelPool( + StubSettings.Builder stubSettings, BigtableConfig config) { + if (config.getChannelCount() != null) { + 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 = + settings.stubSettings().bulkMutateRowsSettings(); + RetrySettings.Builder retrySettings = callSettings.getRetrySettings().toBuilder(); + BatchingSettings.Builder batchingSettings = callSettings.getBatchingSettings().toBuilder(); + if (writeOptions.getAttemptTimeout() != null) { + // 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().getMillis())); + } + + if (writeOptions.getMaxElementsPerBatch() != null) { + batchingSettings.setElementCountThreshold(writeOptions.getMaxElementsPerBatch()); + } + + if (writeOptions.getMaxBytesPerBatch() != null) { + batchingSettings.setRequestByteThreshold(writeOptions.getMaxBytesPerBatch()); + } + + 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() + .bulkMutateRowsSettings() + .setRetrySettings(retrySettings.build()) + .setBatchingSettings(batchingSettings.build()); + + return settings.build(); + } + + private static BigtableDataSettings configureReadSettings( + BigtableDataSettings.Builder settings, BigtableReadOptions readOptions) { + + RetrySettings.Builder retrySettings = + settings.stubSettings().readRowsSettings().getRetrySettings().toBuilder(); + + if (readOptions.getAttemptTimeout() != null) { + // 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().getMillis())); + } + + settings.stubSettings().readRowsSettings().setRetrySettings(retrySettings.build()); + + return settings.build(); + } + + /** + * 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) { + 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.getCredentialOptions().getCredentialType() == CredentialOptions.CredentialType.None + && config.getEmulatorHost() == null) { + builder.setEmulatorHost(String.format("%s:%s", options.getDataHost(), options.getPort())); + } + + builder.setChannelCount(options.getChannelCount()); + + if (options.getCredentialOptions() != null) { + try { + CredentialOptions credOptions = options.getCredentialOptions(); + switch (credOptions.getCredentialType()) { + case DefaultCredentials: + // Veneer uses default credentials, so no need to reset here + 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"); + } + Credentials credentials = + ServiceAccountJwtAccessCredentials.newBuilder() + .setClientEmail(serviceAccount) + .setPrivateKey(privateKey) + .build(); + builder.setCredentialFactory(FixedCredentialFactory.create(credentials)); + } catch (GeneralSecurityException exception) { + throw new RuntimeException("exception while retrieving credentials", exception); + } + break; + case SuppliedCredentials: + Credentials credentials = + ((CredentialOptions.UserSuppliedCredentialOptions) credOptions).getCredential(); + builder.setCredentialFactory(FixedCredentialFactory.create(credentials)); + break; + case SuppliedJson: + CredentialOptions.JsonCredentialsOptions jsonCredentialsOptions = + (CredentialOptions.JsonCredentialsOptions) credOptions; + builder.setCredentialFactory( + FixedCredentialFactory.create( + GoogleCredentials.fromStream(jsonCredentialsOptions.getInputStream()))); + break; + case None: + // pipelineOptions is ignored + PipelineOptions pipelineOptions = PipelineOptionsFactory.create(); + builder.setCredentialFactory(NoopCredentialFactory.fromOptions(pipelineOptions)); + 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( + org.joda.time.Duration.millis( + options.getCallOptionsConfig().getReadStreamRpcAttemptTimeoutMs().get())); + } + builder.setOperationTimeout( + org.joda.time.Duration.millis(options.getCallOptionsConfig().getReadStreamRpcTimeoutMs())); + 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( + org.joda.time.Duration.millis( + options.getCallOptionsConfig().getMutateRpcAttemptTimeoutMs().get())); + } + builder.setOperationTimeout( + org.joda.time.Duration.millis(options.getCallOptionsConfig().getMutateRpcTimeoutMs())); + // configure batch size + builder.setMaxElementsPerBatch(options.getBulkOptions().getBulkMaxRowKeyCount()); + builder.setMaxBytesPerBatch(options.getBulkOptions().getBulkMaxRequestSize()); + 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 7541af0daf46..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 @@ -17,6 +17,8 @@ */ 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; @@ -26,9 +28,9 @@ 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.Timestamp; 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.time.Instant; @@ -132,6 +134,20 @@ * .withInstanceId(instanceId) * .withTableId("table") * .withRowFilter(filter)); + * + * // 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(attemptTimeout) + * .withOperationTimeout(attemptTimeout); * } * *

Writing to Cloud Bigtable

@@ -153,6 +169,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 @@ -269,16 +294,19 @@ public abstract static class Read extends PTransform> { abstract BigtableReadOptions getBigtableReadOptions(); + @VisibleForTesting + abstract BigtableServiceFactory getServiceFactory(); + /** 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() { @@ -288,16 +316,17 @@ 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()) + .setServiceFactory(FACTORY_INSTANCE) .build(); } @@ -308,6 +337,8 @@ abstract static class Builder { abstract Builder setBigtableReadOptions(BigtableReadOptions bigtableReadOptions); + abstract Builder setServiceFactory(BigtableServiceFactory factory); + abstract Read build(); } @@ -363,8 +394,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(); } /** @@ -386,7 +419,9 @@ public Read withTableId(String tableId) { * *

Does not modify this object. * - * @deprecated will be replaced by bigtable options configurator. + * @deprecated please set the configurations directly: + * BigtableIO.read().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId) + * and set credentials in {@link PipelineOptions}. */ @Deprecated public Read withBigtableOptions(BigtableOptions options) { @@ -407,12 +442,13 @@ public Read withBigtableOptions(BigtableOptions options) { * *

Does not modify this object. * - * @deprecated will be replaced by bigtable options configurator. + * @deprecated please set the configurations directly: + * BigtableIO.read().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId) + * and set credentials in {@link PipelineOptions}. */ @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(); @@ -426,7 +462,12 @@ public Read withBigtableOptions(BigtableOptions.Builder optionsBuilder) { * {@link #withProjectId} and {@link #withInstanceId}. * *

Does not modify this object. + * + * @deprecated please set the configurations directly: + * BigtableIO.read().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId) + * and set credentials in {@link PipelineOptions}. */ + @Deprecated public Read withBigtableOptionsConfigurator( SerializableFunction configurator) { BigtableConfig config = getBigtableConfig(); @@ -521,28 +562,47 @@ public Read withoutValidation() { } /** - * Returns a new {@link BigtableIO.Read} that will read using the given Cloud Bigtable service - * implementation. + * Returns a new {@link BigtableIO.Read} that will use an official Bigtable emulator. * *

This is used for testing. - * - *

Does not modify this object. */ @VisibleForTesting - Read withBigtableService(BigtableService bigtableService) { + public Read withEmulator(String emulatorHost) { BigtableConfig config = getBigtableConfig(); - return toBuilder().setBigtableConfig(config.withBigtableService(bigtableService)).build(); + return toBuilder().setBigtableConfig(config.withEmulator(emulatorHost)).build(); } /** - * Returns a new {@link BigtableIO.Read} that will use an official Bigtable emulator. + * Returns a new {@link BigtableIO.Read} with the attempt timeout. Attempt timeout controls the + * timeout for each remote call. * - *

This is used for testing. + *

Does not modify this object. */ - @VisibleForTesting - public Read withEmulator(String emulatorHost) { - BigtableConfig config = getBigtableConfig(); - return toBuilder().setBigtableConfig(config.withEmulator(emulatorHost)).build(); + public Read withAttemptTimeout(Duration timeout) { + checkArgument(timeout.isLongerThan(Duration.ZERO), "attempt timeout must be positive"); + BigtableReadOptions readOptions = getBigtableReadOptions(); + return toBuilder() + .setBigtableReadOptions(readOptions.toBuilder().setAttemptTimeout(timeout).build()) + .build(); + } + + /** + * 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(Duration timeout) { + checkArgument(timeout.isLongerThan(Duration.ZERO), "operation timeout must be positive"); + BigtableReadOptions readOptions = getBigtableReadOptions(); + return toBuilder() + .setBigtableReadOptions(readOptions.toBuilder().setOperationTimeout(timeout).build()) + .build(); + } + + Read withServiceFactory(BigtableServiceFactory factory) { + return toBuilder().setServiceFactory(factory).build(); } @Override @@ -551,13 +611,18 @@ public PCollection expand(PBegin input) { getBigtableReadOptions().validate(); BigtableSource source = - new BigtableSource(getBigtableConfig(), getBigtableReadOptions(), null); + new BigtableSource( + getServiceFactory(), + getServiceFactory().newId(), + getBigtableConfig(), + getBigtableReadOptions(), + null); return input.getPipeline().apply(org.apache.beam.sdk.io.Read.from(source)); } @Override public void validate(PipelineOptions options) { - validateTableExists(getBigtableConfig(), options); + validateTableExists(getBigtableConfig(), getBigtableReadOptions(), options); } @Override @@ -573,6 +638,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 { + checkArgument( + getServiceFactory().checkTableExists(config, options, tableId), + "Table %s does not exist", + tableId); + } catch (IOException e) { + LOG.warn("Error checking whether table {} exists; proceeding.", tableId, e); + } + } + } } /** @@ -601,10 +681,15 @@ public abstract static class Write abstract BigtableConfig getBigtableConfig(); + abstract BigtableWriteOptions getBigtableWriteOptions(); + + @VisibleForTesting + abstract BigtableServiceFactory getServiceFactory(); + /** * 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() { @@ -614,14 +699,16 @@ 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(); - return new AutoValue_BigtableIO_Write.Builder().setBigtableConfig(config).build(); + BigtableWriteOptions writeOptions = + BigtableWriteOptions.builder().setTableId(StaticValueProvider.of("")).build(); + + return new AutoValue_BigtableIO_Write.Builder() + .setBigtableConfig(config) + .setBigtableWriteOptions(writeOptions) + .setServiceFactory(FACTORY_INSTANCE) + .build(); } @AutoValue.Builder @@ -629,6 +716,10 @@ abstract static class Builder { abstract Builder setBigtableConfig(BigtableConfig bigtableConfig); + abstract Builder setBigtableWriteOptions(BigtableWriteOptions writeOptions); + + abstract Builder setServiceFactory(BigtableServiceFactory factory); + abstract Write build(); } @@ -684,8 +775,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(); } /** @@ -707,7 +800,9 @@ 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: + * BigtableIO.write().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId) + * and set credentials in {@link PipelineOptions}. */ @Deprecated public Write withBigtableOptions(BigtableOptions options) { @@ -728,14 +823,15 @@ 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: + * BigtableIO.write().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId) + * and set credentials in {@link PipelineOptions}. */ @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(); } @@ -747,7 +843,12 @@ public Write withBigtableOptions(BigtableOptions.Builder optionsBuilder) { * {@link #withProjectId} and {@link #withInstanceId}. * *

Does not modify this object. + * + * @deprecated please configure the write options directly: + * BigtableIO.write().withProjectId(projectId).withInstanceId(instanceId).withTableId(tableId) + * and set credentials in {@link PipelineOptions}. */ + @Deprecated public Write withBigtableOptionsConfigurator( SerializableFunction configurator) { BigtableConfig config = getBigtableConfig(); @@ -764,27 +865,104 @@ public Write withoutValidation() { } /** - * Returns a new {@link BigtableIO.Write} that will write using the given Cloud Bigtable service - * implementation. + * Returns a new {@link BigtableIO.Write} that will use an official Bigtable emulator. * *

This is used for testing. + */ + @VisibleForTesting + public Write withEmulator(String emulatorHost) { + BigtableConfig config = getBigtableConfig(); + return toBuilder().setBigtableConfig(config.withEmulator(emulatorHost)).build(); + } + + /** + * Returns a new {@link BigtableIO.Write} with the attempt timeout. Attempt timeout controls the + * timeout for each remote call. * *

Does not modify this object. */ - Write withBigtableService(BigtableService bigtableService) { - BigtableConfig config = getBigtableConfig(); - return toBuilder().setBigtableConfig(config.withBigtableService(bigtableService)).build(); + public Write withAttemptTimeout(Duration timeout) { + checkArgument(timeout.isLongerThan(Duration.ZERO), "attempt timeout must be positive"); + BigtableWriteOptions options = getBigtableWriteOptions(); + return toBuilder() + .setBigtableWriteOptions(options.toBuilder().setAttemptTimeout(timeout).build()) + .build(); } /** - * Returns a new {@link BigtableIO.Write} that will use an official Bigtable emulator. + * 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. * - *

This is used for testing. + *

Does not modify this object. */ + public Write withOperationTimeout(Duration timeout) { + checkArgument(timeout.isLongerThan(Duration.ZERO), "operation timeout must be positive"); + BigtableWriteOptions options = getBigtableWriteOptions(); + return toBuilder() + .setBigtableWriteOptions(options.toBuilder().setOperationTimeout(timeout).build()) + .build(); + } + + /** + * 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 withMaxElementsPerBatch(long size) { + checkArgument(size > 0, "max elements per batch size must be positive"); + BigtableWriteOptions options = getBigtableWriteOptions(); + return toBuilder() + .setBigtableWriteOptions(options.toBuilder().setMaxElementsPerBatch(size).build()) + .build(); + } + + /** + * 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 withMaxBytesPerBatch(long size) { + checkArgument(size > 0, "max bytes per batch size must be positive"); + BigtableWriteOptions options = getBigtableWriteOptions(); + return toBuilder() + .setBigtableWriteOptions(options.toBuilder().setMaxBytesPerBatch(size).build()) + .build(); + } + + /** + * 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 withMaxOutstandingElements(long count) { + checkArgument(count > 0, "max outstanding elements must be positive"); + BigtableWriteOptions options = getBigtableWriteOptions(); + return toBuilder() + .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(); + } + @VisibleForTesting - public Write withEmulator(String emulatorHost) { - BigtableConfig config = getBigtableConfig(); - return toBuilder().setBigtableConfig(config.withEmulator(emulatorHost)).build(); + Write withServiceFactory(BigtableServiceFactory factory) { + return toBuilder().setServiceFactory(factory).build(); } /** @@ -792,7 +970,8 @@ public Write withEmulator(String emulatorHost) { * for each batch of rows written. */ public WriteWithResults withWriteResults() { - return new WriteWithResults(getBigtableConfig()); + return new WriteWithResults( + getBigtableConfig(), getBigtableWriteOptions(), getServiceFactory()); } @Override @@ -829,56 +1008,95 @@ public static class WriteWithResults PCollection>>, PCollection> { private final BigtableConfig bigtableConfig; + private final BigtableWriteOptions bigtableWriteOptions; - WriteWithResults(BigtableConfig bigtableConfig) { + private final BigtableServiceFactory factory; + + WriteWithResults( + BigtableConfig bigtableConfig, + BigtableWriteOptions bigtableWriteOptions, + BigtableServiceFactory factory) { this.bigtableConfig = bigtableConfig; + this.bigtableWriteOptions = bigtableWriteOptions; + this.factory = factory; } @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(factory, 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(); } + + private void validateTableExists( + BigtableConfig config, BigtableWriteOptions writeOptions, PipelineOptions options) { + if (config.getValidate() && config.isDataAccessible() && writeOptions.isDataAccessible()) { + String tableId = checkNotNull(writeOptions.getTableId().get()); + try { + checkArgument( + 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); + } + } + } } private static class BigtableWriterFn extends DoFn>, BigtableWriteResult> { - BigtableWriterFn(BigtableConfig bigtableConfig) { + 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, + BigtableConfig bigtableConfig, + BigtableWriteOptions writeOptions) { + this.factory = factory; this.config = bigtableConfig; + this.writeOptions = writeOptions; this.failures = new ConcurrentLinkedQueue<>(); + this.id = factory.newId(); } @StartBundle public void startBundle(StartBundleContext c) throws IOException { - if (bigtableWriter == null) { - bigtableWriter = - config - .getBigtableService(c.getPipelineOptions()) - .openForWriting(config.getTableId().get()); - } recordsWritten = 0; this.seenWindows = Maps.newHashMapWithExpectedSize(1); + + if (bigtableWriter == null) { + serviceEntry = + factory.getServiceForWriting(id, config, writeOptions, c.getPipelineOptions()); + bigtableWriter = serviceEntry.getService().openForWriting(writeOptions.getTableId().get()); + } } @ProcessElement @@ -916,6 +1134,10 @@ public void tearDown() throws Exception { bigtableWriter.close(); bigtableWriter = null; } + if (serviceEntry != null) { + factory.releaseService(serviceEntry); + serviceEntry = null; + } } @Override @@ -925,6 +1147,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; @@ -972,7 +1195,13 @@ private static ByteKey makeByteKey(ByteString key) { static class BigtableSource extends BoundedSource { public BigtableSource( - BigtableConfig config, BigtableReadOptions readOptions, @Nullable Long estimatedSizeBytes) { + BigtableServiceFactory factory, + BigtableServiceFactory.ConfigId configId, + BigtableConfig config, + BigtableReadOptions readOptions, + @Nullable Long estimatedSizeBytes) { + this.factory = factory; + this.configId = configId; this.config = config; this.readOptions = readOptions; this.estimatedSizeBytes = estimatedSizeBytes; @@ -992,15 +1221,20 @@ public String toString() { private final BigtableReadOptions readOptions; private @Nullable Long estimatedSizeBytes; + 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(config, readOptions.withKeyRange(range), estimatedSizeBytes); + return new BigtableSource( + factory, 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(factory, configId, config, readOptions, estimatedSizeBytes); } /** @@ -1008,9 +1242,11 @@ 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 { - return config.getBigtableService(pipelineOptions).getSampleRowKeys(this); + private List getSampleRowKeys(PipelineOptions pipelineOptions) throws IOException { + try (BigtableServiceFactory.BigtableServiceEntry serviceEntry = + factory.getServiceForReading(configId, config, readOptions, pipelineOptions)) { + return serviceEntry.getService().getSampleRowKeys(this); + } } private static final long MAX_SPLIT_COUNT = 15_360L; @@ -1054,7 +1290,12 @@ protected List reduceSplits( if (counter == numberToCombine || !checkRangeAdjacency(previousSourceRanges, source.getRanges())) { reducedSplits.add( - new BigtableSource(config, readOptions.withKeyRanges(previousSourceRanges), size)); + new BigtableSource( + factory, + configId, + config, + readOptions.withKeyRanges(previousSourceRanges), + size)); counter = 0; size = 0; previousSourceRanges = new ArrayList<>(); @@ -1066,7 +1307,8 @@ protected List reduceSplits( } if (size > 0) { reducedSplits.add( - new BigtableSource(config, readOptions.withKeyRanges(previousSourceRanges), size)); + new BigtableSource( + factory, configId, config, readOptions.withKeyRanges(previousSourceRanges), size)); } return reducedSplits; } @@ -1127,7 +1369,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); @@ -1151,9 +1393,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 @@ -1161,9 +1401,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", @@ -1232,16 +1472,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; @@ -1264,7 +1504,8 @@ private long getEstimatedSizeBytesBasedOnSamples(List sam @Override public BoundedReader createReader(PipelineOptions options) throws IOException { - return new BigtableReader(this, config.getBigtableService(options)); + return new BigtableReader( + factory, this, factory.getServiceForReading(configId, config, readOptions, options)); } @Override @@ -1274,7 +1515,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"); @@ -1284,7 +1525,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( @@ -1332,6 +1573,10 @@ private List splitKeyRangeIntoBundleSizedSubranges( return splits.build(); } + public BigtableReadOptions getReadOptions() { + return readOptions; + } + public List getRanges() { return readOptions.getKeyRanges().get(); } @@ -1346,7 +1591,7 @@ public List getRanges() { } public ValueProvider getTableId() { - return config.getTableId(); + return readOptions.getTableId(); } } @@ -1354,21 +1599,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 BigtableServiceFactory factory; + + // 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; - public BigtableReader(BigtableSource source, BigtableService 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.service = service; + this.serviceEntry = service; rangeTracker = ByteKeyRangeTracker.of(source.getRanges().get(0)); } @Override public boolean start() throws IOException { - reader = service.createReader(getCurrentSource()); + reader = serviceEntry.getService().createReader(getCurrentSource()); boolean hasRecord = (reader.start() && rangeTracker.tryReturnRecordAt( @@ -1407,9 +1658,12 @@ public Row getCurrent() throws NoSuchElementException { public void close() throws IOException { LOG.info("Closing reader after reading {} records.", recordsReturned); if (reader != null) { - reader.close(); reader = null; } + if (serviceEntry != null) { + factory.releaseService(serviceEntry); + serviceEntry = null; + } } @Override @@ -1466,29 +1720,13 @@ public BigtableWriteException(KV> record, Throwab } } - static void validateTableExists(BigtableConfig config, PipelineOptions options) { - if (config.getValidate() && config.isDataAccessible()) { - String tableId = checkNotNull(config.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>> { static ReadChangeStream create() { - BigtableConfig config = - BigtableConfig.builder().setTableId(StaticValueProvider.of("")).setValidate(true).build(); - BigtableConfig metadataTableconfig = - BigtableConfig.builder().setTableId(StaticValueProvider.of("")).setValidate(true).build(); + BigtableConfig config = BigtableConfig.builder().setValidate(true).build(); + BigtableConfig metadataTableconfig = BigtableConfig.builder().setValidate(true).build(); return new AutoValue_BigtableIO_ReadChangeStream.Builder() .setBigtableConfig(config) @@ -1498,6 +1736,8 @@ static ReadChangeStream create() { abstract BigtableConfig getBigtableConfig(); + abstract @Nullable String getTableId(); + abstract @Nullable Timestamp getStartTime(); abstract @Nullable Timestamp getEndTime(); @@ -1508,6 +1748,8 @@ static ReadChangeStream create() { abstract BigtableConfig getMetadataTableBigtableConfig(); + abstract @Nullable String getMetadataTableId(); + abstract ReadChangeStream.Builder toBuilder(); /** * Returns a new {@link BigtableIO.ReadChangeStream} that will stream from the Cloud Bigtable @@ -1543,10 +1785,7 @@ public ReadChangeStream withInstanceId(String instanceId) { *

Does not modify this object. */ public ReadChangeStream withTableId(String tableId) { - BigtableConfig config = getBigtableConfig(); - return toBuilder() - .setBigtableConfig(config.withTableId(StaticValueProvider.of(tableId))) - .build(); + return toBuilder().setTableId(tableId).build(); } /** @@ -1645,10 +1884,7 @@ public ReadChangeStream withMetadataTableInstanceId(String instanceId) { *

Does not modify this object. */ public ReadChangeStream withMetadataTableTableId(String tableId) { - BigtableConfig config = getMetadataTableBigtableConfig(); - return toBuilder() - .setMetadataTableBigtableConfig(config.withTableId(StaticValueProvider.of(tableId))) - .build(); + return toBuilder().setMetadataTableId(tableId).build(); } /** @@ -1674,8 +1910,8 @@ public PCollection> expand(PBegin input) { checkArgument(getBigtableConfig() != null); checkArgument(getBigtableConfig().getProjectId() != null); checkArgument(getBigtableConfig().getInstanceId() != null); - checkArgument(getBigtableConfig().getTableId() != null); checkArgument(getBigtableConfig().getAppProfileId() != null); + checkArgument(getTableId() != null); BigtableConfig metadataTableConfig = getMetadataTableBigtableConfig(); if (metadataTableConfig.getProjectId() == null @@ -1687,11 +1923,9 @@ public PCollection> expand(PBegin input) { metadataTableConfig = metadataTableConfig.withInstanceId(getBigtableConfig().getInstanceId()); } - if (metadataTableConfig.getTableId() == null - || metadataTableConfig.getTableId().get().isEmpty()) { - metadataTableConfig = - metadataTableConfig.withTableId( - StaticValueProvider.of(MetadataTableAdminDao.DEFAULT_METADATA_TABLE_NAME)); + String metadataTableId = getMetadataTableId(); + if (metadataTableId == null || metadataTableId.isEmpty()) { + metadataTableId = MetadataTableAdminDao.DEFAULT_METADATA_TABLE_NAME; } if (metadataTableConfig.getAppProfileId() == null || metadataTableConfig.getAppProfileId().get().isEmpty()) { @@ -1714,7 +1948,12 @@ public PCollection> expand(PBegin input) { ActionFactory actionFactory = new ActionFactory(); DaoFactory daoFactory = - new DaoFactory(getBigtableConfig(), metadataTableConfig, changeStreamName); + new DaoFactory( + getBigtableConfig(), + metadataTableConfig, + getTableId(), + metadataTableId, + changeStreamName); ChangeStreamMetrics metrics = new ChangeStreamMetrics(); InitializeDoFn initializeDoFn = new InitializeDoFn(daoFactory, metadataTableConfig.getAppProfileId().get(), startTime); @@ -1735,9 +1974,13 @@ abstract static class Builder { abstract ReadChangeStream.Builder setBigtableConfig(BigtableConfig bigtableConfig); + abstract ReadChangeStream.Builder setTableId(String tableId); + abstract ReadChangeStream.Builder setMetadataTableBigtableConfig( BigtableConfig bigtableConfig); + abstract ReadChangeStream.Builder setMetadataTableId(String tableId); + abstract ReadChangeStream.Builder setStartTime(Timestamp startTime); abstract ReadChangeStream.Builder setEndTime(Timestamp endTime); 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..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,14 +28,18 @@ 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 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 +49,12 @@ 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 Duration getAttemptTimeout(); + + /** Returns the operation timeout of the reads. */ + abstract @Nullable Duration getOperationTimeout(); + abstract Builder toBuilder(); static BigtableReadOptions.Builder builder() { @@ -54,12 +64,18 @@ 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(Duration timeout); + + abstract Builder setOperationTimeout(Duration timeout); + abstract BigtableReadOptions build(); } @@ -79,13 +95,28 @@ 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")); } 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"); } @@ -101,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 1f266979fcc2..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 @@ -20,8 +20,7 @@ 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; @@ -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.joda.time.Duration; /** An interface for real or fake implementations of Cloud Bigtable. */ interface BigtableService extends Serializable { @@ -71,25 +71,18 @@ 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. */ Row getCurrentRow() throws NoSuchElementException; - } - /** Returns the BigtableOptions used to configure this BigtableService. */ - BigtableOptions getBigtableOptions(); + // Workaround for ReadRows requests which requires to pass the timeouts in + // ApiContext. Can be removed later once it's fixed in Veneer. + Duration getAttemptTimeout(); - /** Returns {@code true} if the table with the give name exists. */ - boolean tableExists(String tableId) throws IOException; + Duration getOperationTimeout(); + } /** Returns a {@link Reader} that will read from the specified source. */ Reader createReader(BigtableSource source) throws IOException; @@ -101,5 +94,7 @@ 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; + + 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 new file mode 100644 index 000000000000..4cf101f78e45 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceFactory.java @@ -0,0 +1,195 @@ +/* + * 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.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 java.io.IOException; +import java.io.ObjectInputStream; +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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Factory class that caches {@link BigtableService} to share between workers with the same {@link + * 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) +}) +class BigtableServiceFactory implements Serializable { + + private static final Logger LOG = LoggerFactory.getLogger(BigtableServiceFactory.class); + + static final BigtableServiceFactory FACTORY_INSTANCE = new BigtableServiceFactory(); + + private transient int nextId = 0; + + private transient Map entries = 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 implements Serializable, AutoCloseable { + + abstract BigtableServiceFactory getServiceFactory(); + + abstract ConfigId getConfigId(); + + abstract BigtableService getService(); + + abstract AtomicInteger getRefCount(); + + static BigtableServiceEntry create( + BigtableServiceFactory factory, + ConfigId configId, + BigtableService service, + AtomicInteger refCount) { + return new AutoValue_BigtableServiceFactory_BigtableServiceEntry( + factory, configId, service, refCount); + } + + @Override + public void close() { + getServiceFactory().releaseService(this); + } + } + + synchronized BigtableServiceEntry getServiceForReading( + ConfigId configId, + BigtableConfig config, + BigtableReadOptions opts, + PipelineOptions pipelineOptions) + throws IOException { + BigtableServiceEntry entry = entries.get(configId); + if (entry != null) { + entry.getRefCount().incrementAndGet(); + 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.translateToBigtableReadOptions(opts, effectiveOptions); + } + BigtableDataSettings settings = + BigtableConfigTranslator.translateReadToVeneerSettings(config, opts, pipelineOptions); + BigtableService service = new BigtableServiceImpl(settings); + entry = BigtableServiceEntry.create(this, configId, service, new AtomicInteger(1)); + entries.put(configId, entry); + return entry; + } + + synchronized BigtableServiceEntry getServiceForWriting( + ConfigId configId, + BigtableConfig config, + BigtableWriteOptions opts, + PipelineOptions pipelineOptions) + throws IOException { + BigtableServiceEntry entry = entries.get(configId); + if (entry != null) { + entry.getRefCount().incrementAndGet(); + 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); + entry = BigtableServiceEntry.create(this, configId, service, new AtomicInteger(1)); + entries.put(configId, entry); + return entry; + } + + synchronized void releaseService(BigtableServiceEntry entry) { + if (entry.getRefCount().decrementAndGet() == 0) { + entry.getService().close(); + entries.remove(entry.getConfigId()); + } + } + + 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 (ApiException e) { + if (e.getStatusCode().getCode() == GrpcStatusCode.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; + } + + synchronized ConfigId newId() { + return ConfigId.create(nextId++); + } + + private BigtableOptions getEffectiveOptions(BigtableConfig config) { + BigtableOptions effectiveOptions = config.getBigtableOptions(); + if (effectiveOptions == null && config.getBigtableOptionsConfigurator() != null) { + effectiveOptions = + config.getBigtableOptionsConfigurator().apply(BigtableOptions.builder()).build(); + } + 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/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..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 @@ -19,43 +19,48 @@ 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.retrying.RetrySettings; +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.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.Status.Code; +import io.grpc.CallOptions; +import io.grpc.Deadline; import io.grpc.StatusRuntimeException; -import io.grpc.stub.StreamObserver; import java.io.IOException; import java.util.ArrayDeque; import java.util.ArrayList; 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.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicReference; -import javax.annotation.Nonnull; +import java.util.concurrent.TimeUnit; import org.apache.beam.runners.core.metrics.GcpResourceIdentifiers; import org.apache.beam.runners.core.metrics.MonitoringInfoConstants; import org.apache.beam.runners.core.metrics.ServiceCallMetric; @@ -66,13 +71,12 @@ 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; +import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.joda.time.Duration; /** * An implementation of {@link BigtableService} that actually communicates with the Cloud Bigtable @@ -82,88 +86,92 @@ "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 private static final double WATERMARK_PERCENTAGE = .1; private static final long MIN_BYTE_BUFFER_SIZE = 100 * 1024 * 1024; // 100MB - public BigtableServiceImpl(BigtableOptions options) { - this.options = options; + 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 = Duration.millis(retry.getInitialRpcTimeout().toMillis()); + this.readOperationTimeout = Duration.millis(retry.getTotalTimeout().toMillis()); } - private final BigtableOptions options; + private final BigtableDataClient client; + private final String projectId; + private final String instanceId; - @Override - public BigtableOptions getBigtableOptions() { - return options; - } + private final Duration readAttemptTimeout; - @Override - public BigtableWriterImpl openForWriting(String tableId) throws IOException { - BigtableSession session = new BigtableSession(options); - BigtableTableName tableName = options.getInstanceName().toTableName(tableId); - return new BigtableWriterImpl(session, tableName); - } + private final Duration readOperationTimeout; @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); - 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, options); - LOG.error(message, e); - throw new IOException(message, e); - } + public BigtableWriterImpl openForWriting(String tableId) { + return new BigtableWriterImpl(client, projectId, instanceId, tableId); } @VisibleForTesting static class BigtableReaderImpl implements Reader { - private BigtableSession session; - private final BigtableSource source; - private ResultScanner results; + private final BigtableDataClient client; + + private final String projectId; + private final String instanceId; + private final String tableId; + + private final List ranges; + private final RowFilter rowFilter; + private Iterator results; + + private final Duration attemptTimeout; + private final Duration operationTimeout; + private Row currentRow; @VisibleForTesting - BigtableReaderImpl(BigtableSession session, BigtableSource source) { - this.session = session; - this.source = source; + BigtableReaderImpl( + BigtableDataClient client, + String projectId, + String instanceId, + String tableId, + List ranges, + @Nullable RowFilter rowFilter, + Duration attemptTimeout, + Duration operationTimeout) { + this.client = client; + this.projectId = projectId; + this.instanceId = instanceId; + this.tableId = tableId; + this.ranges = ranges; + this.rowFilter = rowFilter; + + this.attemptTimeout = attemptTimeout; + this.operationTimeout = operationTimeout; } @Override public boolean start() throws IOException { - RowSet.Builder rowSetBuilder = RowSet.newBuilder(); - for (ByteKeyRange sourceRange : source.getRanges()) { - rowSetBuilder = - rowSetBuilder.addRowRanges( - RowRange.newBuilder() - .setStartKeyClosed(ByteString.copyFrom(sourceRange.getStartKey().getValue())) - .setEndKeyOpen(ByteString.copyFrom(sourceRange.getEndKey().getValue()))); - } - RowSet rowSet = rowSetBuilder.build(); + ServiceCallMetric serviceCallMetric = createCallMetric(projectId, instanceId, tableId); - String tableNameSr = - session.getOptions().getInstanceName().toTableNameStr(source.getTableId().get()); + Query query = Query.create(tableId); + for (ByteKeyRange sourceRange : ranges) { + query.range( + ByteString.copyFrom(sourceRange.getStartKey().getValue()), + ByteString.copyFrom(sourceRange.getEndKey().getValue())); + } - ServiceCallMetric serviceCallMetric = createCallMetric(session, source.getTableId().get()); - ReadRowsRequest.Builder requestB = - ReadRowsRequest.newBuilder().setRows(rowSet).setTableName(tableNameSr); - if (source.getRowFilter() != null) { - requestB.setFilter(source.getRowFilter()); + if (rowFilter != null) { + query.filter(Filters.FILTERS.fromProto(rowFilter)); } try { - results = session.getDataClient().readRows(requestB.build()); + results = + client + .readRowsCallable(new BigtableRowProtoAdapter()) + .call(query, createScanCallContext(attemptTimeout, operationTimeout)) + .iterator(); serviceCallMetric.call("ok"); } catch (StatusRuntimeException e) { serviceCallMetric.call(e.getStatus().getCode().toString()); @@ -174,32 +182,11 @@ public boolean start() throws IOException { @Override public boolean advance() throws IOException { - currentRow = results.next(); - return currentRow != null; - } - - @Override - 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) { - // 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; + if (results.hasNext()) { + currentRow = results.next(); + return true; } + return false; } @Override @@ -209,11 +196,21 @@ public Row getCurrentRow() throws NoSuchElementException { } return currentRow; } + + @Override + public Duration getAttemptTimeout() { + return attemptTimeout; + } + + @Override + public Duration getOperationTimeout() { + return operationTimeout; + } } @VisibleForTesting static class BigtableSegmentReaderImpl implements Reader { - private BigtableSession session; + private final BigtableDataClient client; private @Nullable ReadRowsRequest nextRequest; private @Nullable Row currentRow; @@ -222,6 +219,8 @@ static class BigtableSegmentReaderImpl implements Reader { private final int refillSegmentWaterMark; private final long maxSegmentByteSize; private ServiceCallMetric serviceCallMetric; + private final Duration attemptTimeout; + private final Duration operationTimeout; private static class UpstreamResults { private final List rows; @@ -233,13 +232,23 @@ private UpstreamResults(List rows, @Nullable ReadRowsRequest nextRequest) { } } - static BigtableSegmentReaderImpl create(BigtableSession session, BigtableSource source) { + static BigtableSegmentReaderImpl create( + BigtableDataClient client, + 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())) @@ -247,8 +256,7 @@ static BigtableSegmentReaderImpl create(BigtableSession session, BigtableSource } } RowSet rowSet = rowSetBuilder.build(); - RowFilter filter = - MoreObjects.firstNonNull(source.getRowFilter(), RowFilter.getDefaultInstance()); + RowFilter filter = MoreObjects.firstNonNull(rowFilter, RowFilter.getDefaultInstance()); long maxSegmentByteSize = (long) @@ -257,42 +265,52 @@ 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, + projectId, + instanceId, + tableId, rowSet, - source.getMaxBufferElementCount(), - maxSegmentByteSize, filter, - createCallMetric(session, source.getTableId().get())); + maxBufferedElementCount, + maxSegmentByteSize, + attemptTimeout, + operationTimeout, + createCallMetric(projectId, instanceId, tableId)); } @VisibleForTesting BigtableSegmentReaderImpl( - BigtableSession session, - String tableName, + BigtableDataClient client, + String projectId, + String instanceId, + String tableId, RowSet rowSet, + @Nullable RowFilter filter, int maxRowsInBuffer, long maxSegmentByteSize, - RowFilter filter, + Duration attemptTimeout, + Duration operationTimeout, ServiceCallMetric serviceCallMetric) { if (rowSet.equals(rowSet.getDefaultInstanceForType())) { rowSet = RowSet.newBuilder().addRowRanges(RowRange.getDefaultInstance()).build(); } ReadRowsRequest request = ReadRowsRequest.newBuilder() - .setTableName(tableName) + .setTableName(NameUtil.formatTableName(projectId, instanceId, 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.attemptTimeout = attemptTimeout; + this.operationTimeout = operationTimeout; } @Override @@ -314,57 +332,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 BigtableRowProtoAdapter()) + .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(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(attemptTimeout, operationTimeout)); + return future; } private void waitReadRowsFuture() throws IOException { @@ -411,11 +433,6 @@ private ReadRowsRequest truncateRequest(ReadRowsRequest request, ByteString last return requestBuilder.setRows(segment).build(); } - @Override - public void close() throws IOException { - session.close(); - } - @Override public Row getCurrentRow() throws NoSuchElementException { if (currentRow == null) { @@ -423,18 +440,31 @@ public Row getCurrentRow() throws NoSuchElementException { } return currentRow; } + + @Override + public Duration getAttemptTimeout() { + return attemptTimeout; + } + + @Override + public Duration getOperationTimeout() { + return operationTimeout; + } } @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 Batcher bulkMutation; + private String projectId; + private String instanceId; + private String tableId; + + BigtableWriterImpl( + BigtableDataClient client, String projectId, String instanceId, String tableId) { + this.projectId = projectId; + this.instanceId = instanceId; + this.tableId = tableId; + this.bulkMutation = client.newBulkMutationBatcher(tableId); } @Override @@ -452,59 +482,48 @@ public void flush() throws IOException { @Override public void close() throws IOException { - try { - if (bulkMutation != null) { - try { - bulkMutation.flush(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - // We fail since flush() operation was interrupted. - throw new IOException(e); - } - bulkMutation = null; - } - } finally { - if (session != null) { - session.close(); - session = null; + 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; } } @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"); baseLabels.put(MonitoringInfoConstants.Labels.METHOD, "google.bigtable.v2.MutateRows"); baseLabels.put( MonitoringInfoConstants.Labels.RESOURCE, - GcpResourceIdentifiers.bigtableResource( - session.getOptions().getProjectId(), - session.getOptions().getInstanceId(), - tableName.getTableId())); - 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(), - tableName.getTableId())); + GcpResourceIdentifiers.bigtableTableID(projectId, instanceId, tableId)); ServiceCallMetric serviceCallMetric = new ServiceCallMetric(MonitoringInfoConstants.Urns.API_REQUEST_COUNT, baseLabels); + CompletableFuture result = new CompletableFuture<>(); + Futures.addCallback( - new VendoredListenableFutureAdapter<>(bulkMutation.add(request)), + new VendoredListenableFutureAdapter<>(bulkMutation.add(entry)), new FutureCallback() { @Override public void onSuccess(MutateRowResponse mutateRowResponse) { @@ -528,60 +547,80 @@ public void onFailure(Throwable throwable) { } } - @Override - public String toString() { - return MoreObjects.toStringHelper(BigtableServiceImpl.class).add("options", options).toString(); - } - @Override public Reader createReader(BigtableSource source) throws IOException { - BigtableSession session = new BigtableSession(options); if (source.getMaxBufferElementCount() != null) { - return BigtableSegmentReaderImpl.create(session, source); + return BigtableSegmentReaderImpl.create( + client, + projectId, + instanceId, + source.getTableId().get(), + source.getRanges(), + source.getRowFilter(), + source.getMaxBufferElementCount(), + readAttemptTimeout, + readOperationTimeout); } else { - return new BigtableReaderImpl(session, source); + return new BigtableReaderImpl( + client, + projectId, + instanceId, + source.getTableId().get(), + source.getRanges(), + source.getRowFilter(), + readAttemptTimeout, + readOperationTimeout); } } + // - 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( + Duration attemptTimeout, Duration operationTimeout) { + GrpcCallContext ctx = GrpcCallContext.createDefault(); + + ctx.withCallOptions( + CallOptions.DEFAULT.withDeadline( + Deadline.after(operationTimeout.getMillis(), TimeUnit.MILLISECONDS))); + ctx.withTimeout(org.threeten.bp.Duration.ofMillis(attemptTimeout.getMillis())); + 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) { + 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); } + @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; 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); @@ -599,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()) @@ -621,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); @@ -640,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()) @@ -656,4 +695,97 @@ public int compareTo(@Nonnull EndPoint o) { .result(); } } + + static class BigtableRowProtoAdapter 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 @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) { + 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) { + 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) { + if (currentValue == null) { + currentValue = value; + } else { + currentValue = currentValue.concat(value); + } + } + + @Override + public void finishCell() { + lastCell.setValue(currentValue); + } + + @Override + public com.google.bigtable.v2.Row finishRow() { + return protoBuilder.build(); + } + + @Override + public void reset() { + lastFamilyName = null; + lastFamily = null; + lastColumnName = null; + lastColumn = null; + currentValue = null; + + protoBuilder = com.google.bigtable.v2.Row.newBuilder(); + } + + @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/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..3e0a6e595fee --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteOptions.java @@ -0,0 +1,121 @@ +/* + * 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; +import org.joda.time.Duration; + +/** 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 Duration getAttemptTimeout(); + + /** Returns the operation timeout for writes. */ + abstract @Nullable Duration getOperationTimeout(); + + /** Returns the max number of elements of a batch. */ + abstract @Nullable Long getMaxElementsPerBatch(); + + /** 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(); + + /** Returns the max number of concurrent bytes allowed before enforcing flow control. */ + abstract @Nullable Long getMaxOutstandingBytes(); + + 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(Duration timeout); + + abstract Builder setOperationTimeout(Duration timeout); + + abstract Builder setMaxElementsPerBatch(long size); + + abstract Builder setMaxBytesPerBatch(long bytes); + + abstract Builder setMaxOutstandingElements(long count); + + abstract Builder setMaxOutstandingBytes(long bytes); + + 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("maxElementsPerBatch", getMaxElementsPerBatch()) + .withLabel("Write batch element count")) + .addIfNotNull( + DisplayData.item("maxBytesPerBatch", getMaxBytesPerBatch()) + .withLabel("Write batch byte size")) + .addIfNotNull( + DisplayData.item("maxOutstandingElements", getMaxOutstandingElements()) + .withLabel("Write max outstanding elements")) + .addIfNotNull( + DisplayData.item("maxOutstandingBytes", getMaxOutstandingBytes()) + .withLabel("Write max outstanding bytes")); + } + + void validate() { + checkArgument( + getTableId() != null && (!getTableId().isAccessible() || !getTableId().get().isEmpty()), + "Could not obtain Bigtable table id"); + + 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/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/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/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 b2a29edcf29b..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 @@ -36,15 +36,23 @@ public class DaoFactory implements Serializable { private final BigtableConfig changeStreamConfig; private final BigtableConfig metadataTableConfig; + + private final String tableId; + + private final String metadataTableId; private final String changeStreamName; public DaoFactory( BigtableConfig changeStreamConfig, BigtableConfig metadataTableConfig, + String tableId, + String metadataTableId, String changeStreamName) { this.changeStreamConfig = changeStreamConfig; this.metadataTableConfig = metadataTableConfig; this.changeStreamName = changeStreamName; + this.tableId = tableId; + this.metadataTableId = metadataTableId; } public String getChangeStreamName() { @@ -60,7 +68,7 @@ public String getStreamTableDebugString() { + "App Profile Id: %s", this.changeStreamConfig.getProjectId(), this.changeStreamConfig.getInstanceId(), - this.changeStreamConfig.getTableId(), + this.tableId, this.changeStreamConfig.getAppProfileId()); } @@ -73,7 +81,7 @@ public String getMetadataTableDebugString() { + "App Profile Id: %s", this.metadataTableConfig.getProjectId(), this.metadataTableConfig.getInstanceId(), - this.metadataTableConfig.getTableId(), + this.metadataTableId, this.metadataTableConfig.getAppProfileId()); } @@ -81,7 +89,7 @@ public synchronized ChangeStreamDao getChangeStreamDao() throws IOException { if (changeStreamDao == null) { checkArgumentNotNull(changeStreamConfig.getProjectId()); checkArgumentNotNull(changeStreamConfig.getInstanceId()); - String tableId = checkArgumentNotNull(changeStreamConfig.getTableId()).get(); + String tableId = this.tableId; checkArgumentNotNull(changeStreamConfig.getAppProfileId()); changeStreamDao = new ChangeStreamDao(tableId); } @@ -92,7 +100,7 @@ public synchronized MetadataTableDao getMetadataTableDao() throws IOException { if (metadataTableDao == null) { checkArgumentNotNull(metadataTableConfig.getProjectId()); checkArgumentNotNull(metadataTableConfig.getInstanceId()); - checkArgumentNotNull(metadataTableConfig.getTableId()); + checkArgumentNotNull(this.metadataTableId); checkArgumentNotNull(metadataTableConfig.getAppProfileId()); metadataTableDao = new MetadataTableDao( @@ -106,7 +114,7 @@ public synchronized MetadataTableAdminDao getMetadataTableAdminDao() throws IOEx if (metadataTableAdminDao == null) { checkArgumentNotNull(metadataTableConfig.getProjectId()); checkArgumentNotNull(metadataTableConfig.getInstanceId()); - String tableId = checkArgumentNotNull(metadataTableConfig.getTableId()).get(); + String tableId = checkArgumentNotNull(this.metadataTableId); checkArgumentNotNull(metadataTableConfig.getAppProfileId()); metadataTableAdminDao = new MetadataTableAdminDao(changeStreamName, tableId); } 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/BigtableConfigTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTest.java index 3636aaa0d71a..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 @@ -28,9 +28,6 @@ import static org.junit.Assert.assertTrue; import com.google.cloud.bigtable.config.BigtableOptions; -import com.google.cloud.bigtable.config.BulkOptions; -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; @@ -65,8 +62,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 +92,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( @@ -120,22 +107,14 @@ 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).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 +122,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 +140,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,63 +155,23 @@ 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 - public void testGetBigtableServiceWithDefaultService() { - assertEquals(SERVICE, config.withBigtableService(SERVICE).getBigtableService()); - } - - @Test - public void testGetBigtableServiceWithConfigurator() { - SerializableFunction configurator = - (SerializableFunction) - input -> - input - .setInstanceId(INSTANCE_ID.get() + INSTANCE_ID.get()) - .setProjectId(PROJECT_ID.get() + 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()); } @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 +180,8 @@ 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 - .withTableId(TABLE_ID) - .withProjectId(PROJECT_ID) - .withInstanceId(NOT_ACCESSIBLE_VALUE) - .isDataAccessible()); + config.withProjectId(PROJECT_ID).withInstanceId(NOT_ACCESSIBLE_VALUE).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 new file mode 100644 index 000000000000..f3a70fd225d3 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTranslatorTest.java @@ -0,0 +1,299 @@ +/* + * 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 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; +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.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; +import org.apache.beam.sdk.options.ValueProvider; +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}. */ +@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.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()); + assertEquals( + noopCredentialFactory.getCredential(), config.getCredentialFactory().getCredential()); + } + + @Test + public void testBigtableOptionsToBigtableReadOptions() throws Exception { + 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()); + + assertEquals(org.joda.time.Duration.millis(100), readOptions.getAttemptTimeout()); + assertEquals(org.joda.time.Duration.millis(1000), readOptions.getOperationTimeout()); + } + + @Test + public void testBigtableOptionsToBigtableWriteOptions() throws Exception { + 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.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.getMaxBytesPerBatch()); + assertEquals(100, (long) writeOptions.getMaxElementsPerBatch()); + assertEquals(5 * 100, (long) writeOptions.getMaxOutstandingElements()); + assertEquals(5 * 20, (long) writeOptions.getMaxOutstandingBytes()); + } + + @Test + public void testVeneerReadSettings() throws Exception { + 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(org.joda.time.Duration.millis(101)) + .setOperationTimeout(org.joda.time.Duration.millis(1001)) + .build(); + + PipelineOptions pipelineOptions = PipelineOptionsFactory.as(GcpOptions.class); + + 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.readRowsSettings().getRetrySettings().getInitialRpcTimeout()); + assertEquals( + Duration.ofMillis(1001), + stubSettings.readRowsSettings().getRetrySettings().getTotalTimeout()); + } + + @Test + public void testVeneerWriteSettings() throws Exception { + 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(org.joda.time.Duration.millis(101)) + .setOperationTimeout(org.joda.time.Duration.millis(1001)) + .setMaxElementsPerBatch(105) + .setMaxBytesPerBatch(102) + .setMaxOutstandingElements(10001) + .setMaxOutstandingBytes(100001) + .build(); + + PipelineOptions pipelineOptions = PipelineOptionsFactory.as(GcpOptions.class); + + 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( + 105, + (long) + stubSettings.bulkMutateRowsSettings().getBatchingSettings().getElementCountThreshold()); + assertEquals( + 102, + (long) + stubSettings.bulkMutateRowsSettings().getBatchingSettings().getRequestByteThreshold()); + assertEquals( + 10001, + (long) + stubSettings + .bulkMutateRowsSettings() + .getBatchingSettings() + .getFlowControlSettings() + .getMaxOutstandingElementCount()); + + assertEquals( + 100001, + (long) + stubSettings + .bulkMutateRowsSettings() + .getBatchingSettings() + .getFlowControlSettings() + .getMaxOutstandingRequestBytes()); + } + + @Test + public void testUsingCredentialsFromBigtableOptions() throws Exception { + Credentials fakeCredentials = Mockito.mock(Credentials.class); + BigtableOptions options = + BigtableOptions.builder() + .setProjectId("project") + .setInstanceId("instance") + .setAppProfileId("app-profile") + .setCredentialOptions( + CredentialOptions.UserSuppliedCredentialOptions.credential(fakeCredentials)) + .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()); + + assertEquals( + fakeCredentials, + 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()); + } +} 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..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 @@ -49,12 +49,10 @@ 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; @@ -72,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; @@ -79,7 +78,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; @@ -88,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; @@ -173,6 +172,9 @@ 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"); private static BigtableIO.Write defaultWrite = @@ -187,14 +189,21 @@ 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(); - defaultRead = defaultRead.withBigtableService(service); - defaultWrite = defaultWrite.withBigtableService(service); + + factory = new FakeServiceFactory(service); + + defaultRead = defaultRead.withServiceFactory(factory); + + defaultWrite = defaultWrite.withServiceFactory(factory); + 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,15 +310,22 @@ 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(); + + 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( - failureConfig.withTableId(StaticValueProvider.of(table)), - BigtableReadOptions.builder().setKeyRanges(ALL_KEY_RANGE).build(), + failureFactory, + BigtableServiceFactory.ConfigId.create(1), + failureConfig, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(ALL_KEY_RANGE) + .build(), null); BoundedReader reader = source.createReader(TestPipeline.testingPipelineOptions()); @@ -323,15 +339,22 @@ 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(); + + FakeServiceFactory failureFactory = new FakeServiceFactory(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( - failureConfig.withTableId(StaticValueProvider.of(table)), - BigtableReadOptions.builder().setKeyRanges(ALL_KEY_RANGE).build(), + failureFactory, + failureFactory.newId(), + failureConfig, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(ALL_KEY_RANGE) + .build(), null); BoundedReader reader = source.createReader(TestPipeline.testingPipelineOptions()); @@ -350,7 +373,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()); @@ -424,66 +447,6 @@ 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() throws Exception { - BigtableOptions options = - BIGTABLE_OPTIONS - .toBuilder() - .setCredentialOptions(CredentialOptions.defaultCredentials()) - .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( - CredentialType.SuppliedCredentials, - readService.getBigtableOptions().getCredentialOptions().getCredentialType()); - assertEquals( - CredentialType.SuppliedCredentials, - writeService.getBigtableOptions().getCredentialOptions().getCredentialType()); - } - - /** 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( - CredentialType.None, - readService.getBigtableOptions().getCredentialOptions().getCredentialType()); - assertEquals( - CredentialType.None, - writeService.getBigtableOptions().getCredentialOptions().getCredentialType()); - } - /** Tests that when reading from a non-existent table, the read fails. */ @Test public void testReadingFailsTableDoesNotExist() throws Exception { @@ -493,7 +456,7 @@ public void testReadingFailsTableDoesNotExist() throws Exception { BigtableIO.read() .withBigtableOptions(BIGTABLE_OPTIONS) .withTableId(table) - .withBigtableService(service); + .withServiceFactory(factory); // Exception will be thrown by read.validate() when read is applied. thrown.expect(IllegalArgumentException.class); @@ -663,7 +626,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); @@ -723,8 +686,11 @@ public void testReadingSplitAtFractionExhaustive() throws Exception { BigtableSource source = new BigtableSource( - config.withTableId(StaticValueProvider.of(table)), + factory, + configId, + config, BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) .setKeyRanges( StaticValueProvider.of(Collections.singletonList(service.getTableRange(table)))) .build(), @@ -744,8 +710,11 @@ public void testReadingSplitAtFraction() throws Exception { BigtableSource source = new BigtableSource( - config.withTableId(StaticValueProvider.of(table)), + factory, + configId, + config, BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) .setKeyRanges( StaticValueProvider.of(Collections.singletonList(service.getTableRange(table)))) .build(), @@ -780,8 +749,13 @@ 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(), + factory, + configId, + config, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(ALL_KEY_RANGE) + .build(), null /*size*/); List splits = source.split(numRows * bytesPerRow / numSamples, null /* options */); @@ -795,8 +769,8 @@ 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(); + + BigtableConfig failureConfig = BigtableConfig.builder().setValidate(true).build(); final String table = "TEST-MANY-ROWS-SPLITS-TABLE"; final int numRows = 1500; @@ -807,11 +781,18 @@ 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( - failureConfig.withTableId(StaticValueProvider.of(table)), - BigtableReadOptions.builder().setKeyRanges(ALL_KEY_RANGE).build(), + failureFactory, + failureFactory.newId(), + failureConfig, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(ALL_KEY_RANGE) + .build(), null /*size*/); thrown.expect(RuntimeException.class); @@ -876,8 +857,13 @@ 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(), + factory, + configId, + config, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(StaticValueProvider.of(keyRanges)) + .build(), null /*size*/); List splits = new ArrayList<>(); @@ -926,8 +912,13 @@ 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(), + factory, + configId, + config, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(StaticValueProvider.of(keyRanges)) + .build(), null /*size*/); List splits = new ArrayList<>(); @@ -967,8 +958,13 @@ 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(), + factory, + configId, + config, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(ALL_KEY_RANGE) + .build(), null /*size*/); List splits = new ArrayList<>(); List keyRanges = @@ -1038,13 +1034,21 @@ 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(), + factory, + configId, + config, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(StaticValueProvider.of(keyRanges)) + .build(), null /*size*/); BigtableSource referenceSource = new BigtableSource( - config.withTableId(StaticValueProvider.of(table)), + factory, + configId, + config, BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) .setKeyRanges( StaticValueProvider.of(Collections.singletonList(service.getTableRange(table)))) .build(), @@ -1073,8 +1077,13 @@ 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(), + factory, + configId, + config, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(ALL_KEY_RANGE) + .build(), null /*size*/); List splits = source.split(numRows * bytesPerRow / numSplits, null); @@ -1115,13 +1124,21 @@ 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(), + factory, + configId, + config, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(StaticValueProvider.of(keyRanges)) + .build(), null /*size*/); BigtableSource referenceSource = new BigtableSource( - config.withTableId(StaticValueProvider.of(table)), + factory, + configId, + config, BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) .setKeyRanges( StaticValueProvider.of(ImmutableList.of(service.getTableRange(table)))) .build(), @@ -1151,8 +1168,11 @@ public void testReadingWithFilterAndSubSplits() throws Exception { RowFilter.newBuilder().setRowKeyRegexFilter(ByteString.copyFromUtf8(".*17.*")).build(); BigtableSource source = new BigtableSource( - config.withTableId(StaticValueProvider.of(table)), + factory, + configId, + config, BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) .setRowFilter(StaticValueProvider.of(filter)) .setKeyRanges(ALL_KEY_RANGE) .build(), @@ -1233,7 +1253,6 @@ public void testReadWithoutValidate() { BigtableIO.read() .withBigtableOptions(BIGTABLE_OPTIONS) .withTableId(table) - .withBigtableService(service) .withoutValidation(); // validate() will throw if withoutValidation() isn't working @@ -1247,7 +1266,6 @@ public void testWriteWithoutValidate() { BigtableIO.write() .withBigtableOptions(BIGTABLE_OPTIONS) .withTableId(table) - .withBigtableService(service) .withoutValidation(); // validate() will throw if withoutValidation() isn't working @@ -1278,7 +1296,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"; @@ -1300,7 +1318,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"; @@ -1427,11 +1445,11 @@ public void testTableCheckIgnoredWhenCanNotAccessConfig() throws Exception { public void testWritingFailsAtWriteRecord() throws IOException { FailureBigtableService failureService = new FailureBigtableService(FailureOptions.builder().setFailAtWriteRecord(true).build()); + + FakeServiceFactory failureFactory = new FakeServiceFactory(failureService); + BigtableIO.Write failureWrite = - BigtableIO.write() - .withInstanceId("instance") - .withProjectId("project") - .withBigtableService(failureService); + BigtableIO.write().withInstanceId("instance").withProjectId("project"); final String table = "table"; final String key = "key"; @@ -1440,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); @@ -1492,8 +1510,13 @@ public void testGetSplitPointsConsumed() throws Exception { BigtableSource source = new BigtableSource( - config.withTableId(StaticValueProvider.of(table)), - BigtableReadOptions.builder().setKeyRanges(ALL_KEY_RANGE).build(), + factory, + configId, + config, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(ALL_KEY_RANGE) + .build(), null); BoundedReader reader = source.createReader(TestPipeline.testingPipelineOptions()); @@ -1604,12 +1627,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<>(); - - @Override - public BigtableOptions getBigtableOptions() { - return null; - } + private final Map> sampleRowKeys = new HashMap<>(); public @Nullable SortedMap getTable(String tableId) { return tables.get(tableId); @@ -1625,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); } @@ -1645,38 +1662,38 @@ 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; } + @Override + public void close() {} + /** Sets up the sample row keys for the specified table. */ void setupSampleRowKeys(String tableId, int numSamples, long bytesPerRow) { verifyTableExists(tableId); 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); 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()); + // 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,13 +1715,16 @@ 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()"); } return super.getSampleRowKeys(source); } + @Override + public void close() {} + private final FailureOptions failureOptions; } @@ -1789,9 +1809,13 @@ public Row getCurrentRow() { } @Override - public void close() { - rows = null; - currentRow = null; + public Duration getAttemptTimeout() { + return Duration.millis(100); + } + + @Override + public Duration getOperationTimeout() { + return Duration.millis(1000); } } @@ -1934,4 +1958,44 @@ 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) { + return BigtableServiceEntry.create(this, configId, service, new AtomicInteger(1)); + } + + @Override + synchronized BigtableServiceEntry getServiceForWriting( + ConfigId configId, + BigtableConfig config, + BigtableWriteOptions opts, + PipelineOptions pipelineOptions) { + return BigtableServiceEntry.create(this, configId, service, new AtomicInteger(1)); + } + + @Override + synchronized void releaseService(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 97c51856cc99..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 @@ -18,44 +18,50 @@ package org.apache.beam.sdk.io.gcp.bigtable; 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.retrying.RetrySettings; +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; @@ -70,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; @@ -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,34 @@ 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 queryCaptor; + + private static AtomicBoolean cancelled = new AtomicBoolean(false); @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 +144,50 @@ 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.BigtableRowProtoAdapter()); + // 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(mockSession, mockBigtableSource); + new BigtableServiceImpl.BigtableReaderImpl( + mockBigtableDataClient, + bigtableDataSettings.getProjectId(), + bigtableDataSettings.getInstanceId(), + mockBigtableSource.getTableId().get(), + mockBigtableSource.getRanges(), + null, + Duration.millis(retrySettings.getInitialRpcTimeout().toMillis()), + Duration.millis(retrySettings.getTotalTimeout().toMillis())); 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,25 +204,51 @@ 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.BigtableRowProtoAdapter()); + // 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( - mockSession, - TABLE_ID, + mockBigtableDataClient, + bigtableDataSettings.getProjectId(), + bigtableDataSettings.getInstanceId(), + mockBigtableSource.getTableId().get(), ranges.build(), + RowFilter.getDefaultInstance(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, - RowFilter.getDefaultInstance(), + Duration.millis(retrySettings.getInitialRpcTimeout().toMillis()), + Duration.millis(retrySettings.getTotalTimeout().toMillis()), mockCallMetric); underTest.start(); - Assert.assertEquals(FlatRowConverter.convert(expectedRow), underTest.getCurrentRow()); + Assert.assertEquals(expectedRow, underTest.getCurrentRow()); Assert.assertFalse(underTest.advance()); - underTest.close(); Mockito.verify(mockCallMetric, Mockito.times(2)).call("ok"); } @@ -207,23 +267,40 @@ 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.BigtableRowProtoAdapter()); + // 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( - mockSession, + mockBigtableDataClient, + bigtableDataSettings.getProjectId(), + bigtableDataSettings.getInstanceId(), TABLE_ID, ranges.build(), + RowFilter.getDefaultInstance(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, - RowFilter.getDefaultInstance(), + Duration.millis(retrySettings.getInitialRpcTimeout().toMillis()), + Duration.millis(retrySettings.getTotalTimeout().toMillis()), mockCallMetric); List actualResults = new ArrayList<>(); @@ -233,12 +310,8 @@ 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,23 +336,41 @@ 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.BigtableRowProtoAdapter()); + + // 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( - mockSession, + mockBigtableDataClient, + bigtableDataSettings.getProjectId(), + bigtableDataSettings.getInstanceId(), TABLE_ID, ranges.build(), + RowFilter.getDefaultInstance(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, - RowFilter.getDefaultInstance(), + Duration.millis(retrySettings.getInitialRpcTimeout().toMillis()), + Duration.millis(retrySettings.getTotalTimeout().toMillis()), mockCallMetric); List actualResults = new ArrayList<>(); @@ -289,12 +380,8 @@ 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(); Mockito.verify(mockCallMetric, Mockito.times(3)).call("ok"); } @@ -323,23 +410,38 @@ 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.BigtableRowProtoAdapter()); + // 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( - mockSession, + mockBigtableDataClient, + bigtableDataSettings.getProjectId(), + bigtableDataSettings.getInstanceId(), TABLE_ID, ranges.build(), + RowFilter.getDefaultInstance(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, - RowFilter.getDefaultInstance(), + Duration.millis(retrySettings.getInitialRpcTimeout().toMillis()), + Duration.millis(retrySettings.getTotalTimeout().toMillis()), mockCallMetric); List actualResults = new ArrayList<>(); @@ -349,12 +451,8 @@ 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(); Mockito.verify(mockCallMetric, Mockito.times(3)).call("ok"); } @@ -367,24 +465,39 @@ 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.BigtableRowProtoAdapter()); + // 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( - mockSession, + mockBigtableDataClient, + bigtableDataSettings.getProjectId(), + bigtableDataSettings.getInstanceId(), TABLE_ID, RowSet.getDefaultInstance(), + RowFilter.getDefaultInstance(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, - RowFilter.getDefaultInstance(), + Duration.millis(retrySettings.getInitialRpcTimeout().toMillis()), + Duration.millis(retrySettings.getTotalTimeout().toMillis()), mockCallMetric); List actualResults = new ArrayList<>(); @@ -394,12 +507,8 @@ 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(); Mockito.verify(mockCallMetric, Mockito.times(4)).call("ok"); ; @@ -428,53 +537,67 @@ 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.BigtableRowProtoAdapter()); + // 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( - mockSession, + mockBigtableDataClient, + bigtableDataSettings.getProjectId(), + bigtableDataSettings.getInstanceId(), TABLE_ID, ranges.build(), + RowFilter.getDefaultInstance(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, - RowFilter.getDefaultInstance(), + Duration.millis(retrySettings.getInitialRpcTimeout().toMillis()), + Duration.millis(retrySettings.getTotalTimeout().toMillis()), mockCallMetric); 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(queryCaptor.capture(), any(ResponseObserver.class), any(ApiCallContext.class)); + Assert.assertEquals( + 3, queryCaptor.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(queryCaptor.capture(), any(ResponseObserver.class), any(ApiCallContext.class)); + Assert.assertEquals( + 1, queryCaptor.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(); Mockito.verify(mockCallMetric, Mockito.times(3)).call("ok"); } /** * 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,24 +612,52 @@ 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.BigtableRowProtoAdapter()); + // 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( - mockSession, + mockBigtableDataClient, + bigtableDataSettings.getProjectId(), + bigtableDataSettings.getInstanceId(), TABLE_ID, RowSet.newBuilder().addRowRanges(mockRowRange).build(), + RowFilter.getDefaultInstance(), SEGMENT_SIZE, segmentByteLimit, - RowFilter.getDefaultInstance(), + Duration.millis(retrySettings.getInitialRpcTimeout().toMillis()), + Duration.millis(retrySettings.getTotalTimeout().toMillis()), mockCallMetric); List actualResults = new ArrayList<>(); @@ -516,12 +667,8 @@ 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(); Mockito.verify(mockCallMetric, Mockito.times(3)).call("ok"); } @@ -539,30 +686,44 @@ 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.BigtableRowProtoAdapter()); + // 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( - mockSession, + mockBigtableDataClient, + bigtableDataSettings.getProjectId(), + bigtableDataSettings.getInstanceId(), TABLE_ID, ranges.build(), + RowFilter.getDefaultInstance(), SEGMENT_SIZE, DEFAULT_BYTE_SEGMENT_SIZE, - RowFilter.getDefaultInstance(), + Duration.millis(retrySettings.getInitialRpcTimeout().toMillis()), + Duration.millis(retrySettings.getTotalTimeout().toMillis()), mockCallMetric); IOException returnedError = null; @@ -571,6 +732,7 @@ public void run() { } catch (IOException e) { returnedError = e; } + Assert.assertTrue(returnedError.getCause() instanceof StatusRuntimeException); Mockito.verify(mockCallMetric, Mockito.times(1)) @@ -578,7 +740,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 +748,39 @@ public void run() { */ @Test public void testWrite() throws IOException, InterruptedException { - when(mockBigtableSource.getTableId()).thenReturn(StaticValueProvider.of(TABLE_ID)); + 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(mockSession, TABLE_NAME); + new BigtableServiceImpl.BigtableWriterImpl( + mockBigtableDataClient, + bigtableDataSettings.getProjectId(), + bigtableDataSettings.getInstanceId(), + 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 +806,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..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 @@ -19,31 +19,28 @@ 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.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.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; +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; -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; 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; @@ -66,8 +63,9 @@ public class BigtableWriteIT implements Serializable { private static final String COLUMN_FAMILY_NAME = "cf"; private static BigtableTestOptions options; - private BigtableOptions bigtableOptions; - private static BigtableSession session; + private static BigtableDataSettings veneerSettings; + private BigtableConfig bigtableConfig; + 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)) @@ -125,24 +128,34 @@ public void processElement(ProcessContext c) { c.output(KV.of(testData.get(index).getKey(), mutations)); } })) - .apply(BigtableIO.write().withBigtableOptions(bigtableOptions).withTableId(tableId)); + .apply( + BigtableIO.write() + .withProjectId(project) + .withInstanceId(options.getInstanceId()) + .withTableId(tableId)); 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); + if (tableAdminClient != null) { + tableAdminClient.close(); + } + if (client != null) { + client.close(); + } } //////////////////////////////////////////////////////////////////////////////////////////// @@ -159,54 +172,37 @@ 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) { + if (tableAdminClient != null) { + tableAdminClient.deleteTable(tableId); + } } } diff --git a/settings.gradle.kts b/settings.gradle.kts index 3c19893774ec..37837239c830 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -272,3 +272,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")