diff --git a/bigtable-client-core-parent/bigtable-client-core/src/main/java/com/google/cloud/bigtable/config/BigtableVeneerSettingsFactory.java b/bigtable-client-core-parent/bigtable-client-core/src/main/java/com/google/cloud/bigtable/config/BigtableVeneerSettingsFactory.java new file mode 100644 index 0000000000..4ff493e924 --- /dev/null +++ b/bigtable-client-core-parent/bigtable-client-core/src/main/java/com/google/cloud/bigtable/config/BigtableVeneerSettingsFactory.java @@ -0,0 +1,312 @@ +/* + * Copyright 2019 Google LLC. + * + * Licensed 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 com.google.cloud.bigtable.config; + +import static com.google.api.client.util.Preconditions.checkState; +import static com.google.cloud.bigtable.data.v2.stub.EnhancedBigtableStubSettings.defaultGrpcTransportProviderBuilder; +import static io.grpc.internal.GrpcUtil.USER_AGENT_KEY; +import static org.threeten.bp.Duration.ofMillis; + +import com.google.api.core.ApiFunction; +import com.google.api.gax.batching.BatchingSettings; +import com.google.api.gax.batching.FlowControlSettings; +import com.google.api.gax.core.CredentialsProvider; +import com.google.api.gax.core.FixedCredentialsProvider; +import com.google.api.gax.core.NoCredentialsProvider; +import com.google.api.gax.grpc.GrpcStatusCode; +import com.google.api.gax.retrying.RetrySettings; +import com.google.api.gax.rpc.FixedHeaderProvider; +import com.google.api.gax.rpc.HeaderProvider; +import com.google.api.gax.rpc.StatusCode; +import com.google.api.gax.rpc.TransportChannelProvider; +import com.google.auth.Credentials; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminSettings; +import com.google.cloud.bigtable.admin.v2.stub.BigtableTableAdminStubSettings; +import com.google.cloud.bigtable.data.v2.BigtableDataSettings; +import com.google.cloud.bigtable.data.v2.stub.EnhancedBigtableStubSettings.Builder; +import com.google.common.collect.ImmutableSet; +import io.grpc.ManagedChannelBuilder; +import io.grpc.Status; +import java.io.IOException; +import java.security.GeneralSecurityException; +import java.util.Set; +import javax.annotation.Nonnull; +import org.threeten.bp.Duration; + +/** + * Static methods to convert an instance of {@link BigtableOptions} to a + * {@link BigtableDataSettings} or {@link BigtableTableAdminSettings} instance . + */ +public class BigtableVeneerSettingsFactory { + + /** Constant LOG */ + private static final Logger LOG = new Logger(BigtableVeneerSettingsFactory.class); + + //Identifier to distinguish between CBT or GCJ adapter. + private static final String VENEER_ADAPTER = BigtableVersionInfo.CORE_USER_AGENT+"," + + "veneer-adapter,"; + + private static final int RPC_DEADLINE_MS = 360_000; + private static final int MAX_RETRY_TIMEOUT_MS = 60_000; + + /** + * To create an instance of {@link BigtableDataSettings} from {@link BigtableOptions}. + * + * @param options a {@link BigtableOptions} object. + * @return a {@link BigtableDataSettings} object. + * @throws IOException if any. + */ + public static BigtableDataSettings createBigtableDataSettings( + @Nonnull final BigtableOptions options) throws IOException { + checkState(options.getRetryOptions().enableRetries(), "Disabling retries is not currently supported."); + checkState(!options.useCachedChannel(), "cachedDataPool is not currently supported."); + + final BigtableDataSettings.Builder builder = BigtableDataSettings.newBuilder(); + final Builder dataSettingStub = builder.stubSettings(); + Duration shortRpcTimeoutMs = ofMillis(options.getCallOptionsConfig().getShortRpcTimeoutMs()); + + builder + .setProjectId(options.getProjectId()) + .setInstanceId(options.getInstanceId()) + .setAppProfileId(options.getAppProfileId()); + + dataSettingStub + .setEndpoint(options.getDataHost() + ":" + options.getPort()) + .setHeaderProvider(buildHeaderProvider(options.getUserAgent())) + .setCredentialsProvider(buildCredentialProvider(options.getCredentialOptions())); + + if(options.usePlaintextNegotiation()){ + dataSettingStub + .setTransportChannelProvider(buildChannelProvider(dataSettingStub.getEndpoint(), options)); + } + + // Configuration for rpcTimeout & totalTimeout for non-streaming operations. + dataSettingStub.checkAndMutateRowSettings() + .setSimpleTimeoutNoRetries(shortRpcTimeoutMs); + + dataSettingStub.readModifyWriteRowSettings() + .setSimpleTimeoutNoRetries(shortRpcTimeoutMs); + + buildBulkMutationsSettings(dataSettingStub, options); + + buildReadRowsSettings(dataSettingStub, options); + + buildReadRowSettings(dataSettingStub, options); + + buildMutateRowSettings(dataSettingStub, options); + + buildSampleRowKeysSettings(dataSettingStub, options); + + return builder.build(); + } + + /** + * To create an instance of {@link BigtableTableAdminSettings} from {@link BigtableOptions}. + * + * @param options a {@link BigtableOptions} object. + * @return a {@link BigtableTableAdminSettings} object. + * @throws IOException if any. + */ + public static BigtableTableAdminSettings createTableAdminSettings( + @Nonnull final BigtableOptions options) throws IOException { + final BigtableTableAdminSettings.Builder adminBuilder = BigtableTableAdminSettings.newBuilder(); + BigtableTableAdminStubSettings.Builder adminStub = adminBuilder.stubSettings(); + + adminBuilder + .setProjectId(options.getProjectId()) + .setInstanceId(options.getInstanceId()); + + adminStub + .setHeaderProvider(buildHeaderProvider(options.getUserAgent())) + .setEndpoint(options.getAdminHost() + ":" + options.getPort()) + .setCredentialsProvider(buildCredentialProvider(options.getCredentialOptions())); + + if(options.usePlaintextNegotiation()){ + adminStub + .setTransportChannelProvider(buildChannelProvider(adminStub.getEndpoint(), options)); + } + + return adminBuilder.build(); + } + + /** Creates {@link CredentialsProvider} based on {@link CredentialOptions}. */ + private static CredentialsProvider buildCredentialProvider( + CredentialOptions credentialOptions) throws IOException { + try { + final Credentials credentials = CredentialFactory.getCredentials(credentialOptions); + if (credentials == null) { + LOG.info("Enabling the use of null credentials. This should not be used in production."); + return NoCredentialsProvider.create(); + } + + return FixedCredentialsProvider.create(credentials); + } catch (GeneralSecurityException exception) { + throw new IOException("Could not initialize credentials.", exception); + } + } + + /** Creates {@link HeaderProvider} with VENEER_ADAPTER as prefix for user agent */ + private static HeaderProvider buildHeaderProvider(String userAgent){ + return FixedHeaderProvider.create(USER_AGENT_KEY.name(), VENEER_ADAPTER + userAgent); + } + + /** Builds {@link BatchingSettings} based on {@link BulkOptions} configuration. */ + private static void buildBulkMutationsSettings(Builder builder, + BigtableOptions options) { + BulkOptions bulkOptions = options.getBulkOptions(); + BatchingSettings.Builder batchBuilder = + builder.bulkMutateRowsSettings().getBatchingSettings().toBuilder(); + + long autoFlushMs = bulkOptions.getAutoflushMs(); + long bulkMaxRowKeyCount = bulkOptions.getBulkMaxRowKeyCount(); + long maxInflightRpcs = bulkOptions.getMaxInflightRpcs(); + + if (autoFlushMs > 0) { + batchBuilder.setDelayThreshold(ofMillis(autoFlushMs)); + } + FlowControlSettings.Builder flowControlBuilder = FlowControlSettings.newBuilder(); + if (maxInflightRpcs > 0) { + flowControlBuilder + .setMaxOutstandingRequestBytes(bulkOptions.getMaxMemory()) + .setMaxOutstandingElementCount(maxInflightRpcs * bulkMaxRowKeyCount); + } + + batchBuilder + .setIsEnabled(bulkOptions.useBulkApi()) + .setElementCountThreshold(Long.valueOf(bulkOptions.getBulkMaxRowKeyCount())) + .setRequestByteThreshold(bulkOptions.getBulkMaxRequestSize()) + .setFlowControlSettings(flowControlBuilder.build()); + + RetrySettings retrySettings = + buildIdempotentRetrySettings(builder.bulkMutateRowsSettings().getRetrySettings(), options); + + // TODO(rahulkql): implement bulkMutationThrottling & bulkMutationRpcTargetMs, once available + builder.bulkMutateRowsSettings() + .setBatchingSettings(batchBuilder.build()) + .setRetrySettings(retrySettings) + .setRetryableCodes(buildRetryCodes(options.getRetryOptions())); + } + + /** To build BigtableDataSettings#sampleRowKeysSettings with default Retry settings. */ + private static void buildSampleRowKeysSettings(Builder builder, BigtableOptions options) { + RetrySettings retrySettings = + buildIdempotentRetrySettings(builder.sampleRowKeysSettings().getRetrySettings(), options); + + builder.sampleRowKeysSettings() + .setRetrySettings(retrySettings) + .setRetryableCodes(buildRetryCodes(options.getRetryOptions())); + } + + /** To build BigtableDataSettings#mutateRowSettings with default Retry settings. */ + private static void buildMutateRowSettings(Builder builder, BigtableOptions options) { + RetrySettings retrySettings = + buildIdempotentRetrySettings(builder.mutateRowSettings().getRetrySettings(), options); + + builder.mutateRowSettings() + .setRetrySettings(retrySettings) + .setRetryableCodes(buildRetryCodes(options.getRetryOptions())); + } + + /** To build default Retry settings for Point Read. */ + private static void buildReadRowSettings(Builder builder, BigtableOptions options) { + RetrySettings retrySettings = + buildIdempotentRetrySettings(builder.readRowSettings().getRetrySettings(), options); + + builder.readRowSettings() + .setRetrySettings(retrySettings) + .setRetryableCodes(buildRetryCodes(options.getRetryOptions())); + } + + /** To build BigtableDataSettings#readRowsSettings with default Retry settings. */ + private static void buildReadRowsSettings(Builder builder, BigtableOptions options) { + RetryOptions retryOptions = options.getRetryOptions(); + CallOptionsConfig callOptions = options.getCallOptionsConfig(); + RetrySettings.Builder retryBuilder = builder.readRowsSettings().getRetrySettings().toBuilder(); + + //Timeout for ReadRows + Duration rpcTimeout = ofMillis(retryOptions.getReadPartialRowTimeoutMillis()); + Duration totalTimeout = ofMillis(callOptions.isUseTimeout() + ? callOptions.getLongRpcTimeoutMs() + : retryOptions.getMaxElapsedBackoffMillis()); + + retryBuilder + .setInitialRetryDelay(ofMillis(retryOptions.getInitialBackoffMillis())) + .setRetryDelayMultiplier(retryOptions.getBackoffMultiplier()) + .setMaxRetryDelay(ofMillis(MAX_RETRY_TIMEOUT_MS)) + .setMaxAttempts(retryOptions.getMaxScanTimeoutRetries()) + .setInitialRpcTimeout(rpcTimeout) + .setMaxRpcTimeout(rpcTimeout) + .setTotalTimeout(totalTimeout); + + builder.readRowsSettings() + .setRetrySettings(retryBuilder.build()) + .setRetryableCodes(buildRetryCodes(options.getRetryOptions())); + } + + /** Creates {@link RetrySettings} for non-streaming idempotent method. */ + private static RetrySettings buildIdempotentRetrySettings(RetrySettings retrySettings, + BigtableOptions options) { + RetryOptions retryOptions = options.getRetryOptions(); + CallOptionsConfig callOptions = options.getCallOptionsConfig(); + RetrySettings.Builder retryBuilder = retrySettings.toBuilder(); + + if (retryOptions.allowRetriesWithoutTimestamp()) { + throw new UnsupportedOperationException("Retries without Timestamp does not support yet."); + } + + // if useTimeout is false, then RPC's are defaults to 6 minutes. + Duration rpcTimeout = ofMillis(callOptions.isUseTimeout() + ? callOptions.getShortRpcTimeoutMs() + : RPC_DEADLINE_MS); + + retryBuilder + .setInitialRetryDelay(ofMillis(retryOptions.getInitialBackoffMillis())) + .setRetryDelayMultiplier(retryOptions.getBackoffMultiplier()) + .setMaxRetryDelay(ofMillis(MAX_RETRY_TIMEOUT_MS)) + .setInitialRpcTimeout(rpcTimeout) + .setMaxRpcTimeout(rpcTimeout) + .setMaxAttempts(0) + .setTotalTimeout(ofMillis(retryOptions.getMaxElapsedBackoffMillis())); + + return retryBuilder.build(); + } + + /** Creates {@link Set} of {@link StatusCode.Code} from {@link Status.Code} */ + private static Set buildRetryCodes(RetryOptions retryOptions) { + ImmutableSet.Builder statusCodeBuilder = ImmutableSet.builder(); + for (Status.Code retryCode : retryOptions.getRetryableStatusCodes()) { + statusCodeBuilder.add(GrpcStatusCode.of(retryCode).getCode()); + } + + return statusCodeBuilder.build(); + } + + /** Creates {@link TransportChannelProvider} based on Channel Negotiation type. */ + private static TransportChannelProvider buildChannelProvider(String endpoint, + BigtableOptions options) { + + return defaultGrpcTransportProviderBuilder() + .setEndpoint(endpoint) + .setPoolSize(options.getChannelCount()) + .setChannelConfigurator(new ApiFunction() { + @Override + public ManagedChannelBuilder apply(ManagedChannelBuilder channelBuilder) { + return channelBuilder.usePlaintext(); + } + }) + .build(); + } +} diff --git a/bigtable-client-core-parent/bigtable-client-core/src/test/java/com/google/cloud/bigtable/config/TestBigtableVeneerSettingsFactory.java b/bigtable-client-core-parent/bigtable-client-core/src/test/java/com/google/cloud/bigtable/config/TestBigtableVeneerSettingsFactory.java new file mode 100644 index 0000000000..dca1121c06 --- /dev/null +++ b/bigtable-client-core-parent/bigtable-client-core/src/test/java/com/google/cloud/bigtable/config/TestBigtableVeneerSettingsFactory.java @@ -0,0 +1,346 @@ +/* + * Copyright 2019 Google LLC. + * + * Licensed 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 com.google.cloud.bigtable.config; + +import com.google.api.gax.batching.BatchingSettings; +import com.google.api.gax.core.NoCredentialsProvider; +import com.google.api.gax.retrying.RetrySettings; +import com.google.api.gax.rpc.ServerStream; +import com.google.api.gax.rpc.StatusCode.Code; +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.Mutation; +import com.google.cloud.bigtable.data.v2.models.Query; +import com.google.cloud.bigtable.data.v2.models.Row; +import com.google.cloud.bigtable.data.v2.models.RowCell; +import com.google.cloud.bigtable.data.v2.models.RowMutation; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableSet; +import com.google.protobuf.ByteString; +import java.io.FileInputStream; +import java.io.IOException; +import java.net.ServerSocket; +import java.util.Set; +import java.util.UUID; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.threeten.bp.Duration; + +import static com.google.cloud.bigtable.config.CallOptionsConfig.SHORT_TIMEOUT_MS_DEFAULT; +import static com.google.cloud.bigtable.config.RetryOptions.DEFAULT_BACKOFF_MULTIPLIER; +import static com.google.cloud.bigtable.config.RetryOptions.DEFAULT_INITIAL_BACKOFF_MILLIS; +import static com.google.cloud.bigtable.config.RetryOptions.DEFAULT_MAX_ELAPSED_BACKOFF_MILLIS; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +@RunWith(JUnit4.class) +public class TestBigtableVeneerSettingsFactory { + + private static final Logger LOG = new Logger(TestBigtableVeneerSettingsFactory.class); + + private static final String ACTUAL_PROJECT_ID = System.getProperty("test.client.project.id"); + private static final String ACTUAL_INSTANCE_ID = System.getProperty("test.client.instance.id"); + + private static final String TEST_PROJECT_ID = "fakeProjectID"; + private static final String TEST_INSTANCE_ID = "fakeInstanceID"; + private static final String TEST_USER_AGENT = "sampleUserAgent"; + + /** + * RetryCodes for idempotent Rpcs. + */ + private static final Set DEFAULT_RETRY_CODES = + ImmutableSet.of(Code.DEADLINE_EXCEEDED, Code.UNAVAILABLE, Code.ABORTED, Code.UNAUTHENTICATED); + + private static final boolean endToEndArgMissing = + Strings.isNullOrEmpty(ACTUAL_PROJECT_ID) && Strings.isNullOrEmpty(ACTUAL_INSTANCE_ID); + + + @Rule + public ExpectedException expectException = ExpectedException.none(); + + private BigtableOptions bigtableOptions; + + private BigtableDataSettings dataSettings; + private BigtableTableAdminSettings adminSettings; + private BigtableDataClient dataClient; + private BigtableTableAdminClient adminClient; + + @Before + public void setUp() throws IOException { + ServerSocket serverSocket = new ServerSocket(0); + final int availablePort = serverSocket.getLocalPort(); + serverSocket.close(); + + bigtableOptions = BigtableOptions.builder() + .setProjectId(TEST_PROJECT_ID) + .setInstanceId(TEST_INSTANCE_ID) + .setUserAgent(TEST_USER_AGENT) + .setAdminHost("localhost") + .setDataHost("localhost") + .setCredentialOptions(CredentialOptions.nullCredential()) + .setPort(availablePort) + .build(); + } + + @After + public void tearDown() throws Exception{ + if (dataClient != null) { + dataClient.close(); + } + + if(adminClient != null){ + adminClient.close(); + } + } + + private void initializeClients() throws IOException{ + String josnPath = CredentialOptions.getEnvJsonFile(); + BigtableOptions options = BigtableOptions.builder() + .setProjectId(ACTUAL_PROJECT_ID) + .setInstanceId(ACTUAL_INSTANCE_ID) + .setUserAgent("native-bigtable-test") + .setCredentialOptions(CredentialOptions.jsonCredentials(new FileInputStream(josnPath))) + .build(); + + dataSettings = BigtableVeneerSettingsFactory.createBigtableDataSettings(options); + dataClient = BigtableDataClient.create(dataSettings); + + adminSettings = BigtableVeneerSettingsFactory.createTableAdminSettings(options); + adminClient = BigtableTableAdminClient.create(adminSettings); + } + + /** + * This test runs only if it finds "test.client.project.id" & "test.client.project.id" + * VM arguments. Then it calls to an actual Bigtable Table & performs the checks below: + *
+   *   
    + *
  • Checks if table with TABLE_ID exists.
  • + *
  • Creates a new table with TABLE_ID.
  • + *
  • Mutates a single row with {@link RowMutation}.
  • + *
  • Retrieves output in {@link ServerStream < Row >}.
  • + *
  • Deletes table created with TABLE_ID.
  • + *
+ *
+ */ + @Test + public void testWithActualTables() throws Exception{ + // Checking if both arguments are available or not. + Assume.assumeFalse(endToEndArgMissing); + + if (adminClient == null || dataClient == null) { + initializeClients(); + } + + final String TABLE_ID = "Test-clients-" + UUID.randomUUID().toString(); + final String COLUMN_FAMILY_ID = "CF1"; + final ByteString TEST_QUALIFER = ByteString.copyFromUtf8("qualifier1"); + final ByteString TEST_KEY = ByteString.copyFromUtf8("bigtableDataSettingTest"); + final ByteString TEST_VALUE = ByteString.copyFromUtf8("Test using BigtableDataclient & " + + "BigtableTableAdminClient"); + + //Checking if table already existed in the provided instance. + if (adminClient.exists(TABLE_ID)) { + adminClient.deleteTable(TABLE_ID); + } + try { + CreateTableRequest createTableRequest = + CreateTableRequest.of(TABLE_ID).addFamily(COLUMN_FAMILY_ID); + adminClient.createTable(createTableRequest); + + //Created table with vaneer TableAdminClient. + boolean tableExist = adminClient.exists(TABLE_ID); + LOG.info("Table successfully created : " + tableExist); + assertTrue(tableExist); + + Mutation mutation = Mutation.create(); + mutation.setCell(COLUMN_FAMILY_ID, TEST_QUALIFER, TEST_VALUE); + RowMutation rowMutation = RowMutation.create(TABLE_ID, TEST_KEY, mutation); + + //Write content to Bigtable using vaneer DataClient. + dataClient.mutateRow(rowMutation); + LOG.info("Successfully Mutated"); + + Query query = Query.create(TABLE_ID); + ServerStream rowStream = dataClient.readRows(query); + for (Row outputRow : rowStream) { + + //Checking if the received output's KEY is same as above. + ByteString key = outputRow.getKey(); + LOG.info("found key: " + key.toStringUtf8()); + assertEquals(TEST_KEY, outputRow.getKey()); + + for (RowCell cell : outputRow.getCells()) { + //Checking if the received output is KEY sent above. + ByteString value = cell.getValue(); + LOG.info("Value found: " + value.toStringUtf8()); + assertEquals(TEST_VALUE, value); + } + } + + //Removing the table. + adminClient.deleteTable(TABLE_ID); + } finally { + //Removing Table in case of exceptions. + boolean tableExist = adminClient.exists(TABLE_ID); + if (tableExist) { + adminClient.deleteTable(TABLE_ID); + } + assertFalse(adminClient.exists(TABLE_ID)); + } + } + + @Test + public void testWhenRetriesAreDisabled() throws IOException { + RetryOptions retryOptions = RetryOptions.builder().setEnableRetries(false).build(); + BigtableOptions options = + BigtableOptions.builder() + .setProjectId(TEST_PROJECT_ID).setInstanceId(TEST_INSTANCE_ID) + .setRetryOptions(retryOptions).build(); + + expectException.expect(IllegalStateException.class); + expectException.expectMessage("Disabling retries is not currently supported."); + dataSettings = BigtableVeneerSettingsFactory.createBigtableDataSettings(options); + } + + @Test + public void testWithNullCredentials() throws IOException { + BigtableOptions options = + BigtableOptions.builder() + .setProjectId(TEST_PROJECT_ID) + .setInstanceId(TEST_INSTANCE_ID) + .setCredentialOptions(CredentialOptions.nullCredential()) + .setUserAgent(TEST_USER_AGENT).build(); + dataSettings = BigtableVeneerSettingsFactory.createBigtableDataSettings(options); + assertTrue(dataSettings.getStubSettings().getCredentialsProvider() instanceof NoCredentialsProvider); + } + + @Test + public void testConfigValues() throws IOException { + dataSettings = BigtableVeneerSettingsFactory.createBigtableDataSettings(bigtableOptions); + + //Streaming operation's RetrySettings & RetryCodes of retryable methods. + //sampleRowKeys + verifyRetry(dataSettings.getStubSettings().sampleRowKeysSettings().getRetrySettings()); + assertEquals(DEFAULT_RETRY_CODES, dataSettings.getStubSettings().sampleRowKeysSettings().getRetryableCodes()); + + //mutateRowSettings + verifyRetry(dataSettings.getStubSettings().mutateRowSettings().getRetrySettings()); + assertEquals(DEFAULT_RETRY_CODES, dataSettings.getStubSettings().mutateRowSettings().getRetryableCodes()); + + //bulkMutationsSettings + verifyRetry(dataSettings.getStubSettings().bulkMutateRowsSettings().getRetrySettings()); + assertEquals(DEFAULT_RETRY_CODES, + dataSettings.getStubSettings().bulkMutateRowsSettings().getRetryableCodes()); + + //Non-streaming operation's verifying RetrySettings & RetryCodes of non-retryable methods. + //readModifyWriteRowSettings + verifyDisabledRetry(dataSettings.getStubSettings().readModifyWriteRowSettings().getRetrySettings()); + assertTrue(dataSettings.getStubSettings().readModifyWriteRowSettings().getRetryableCodes().isEmpty()); + + //checkAndMutateRowSettings + verifyDisabledRetry(dataSettings.getStubSettings().checkAndMutateRowSettings().getRetrySettings()); + assertTrue(dataSettings.getStubSettings().checkAndMutateRowSettings().getRetryableCodes().isEmpty()); + } + + private void verifyRetry(RetrySettings retrySettings) { + assertEquals(DEFAULT_INITIAL_BACKOFF_MILLIS, retrySettings.getInitialRetryDelay().toMillis()); + assertEquals(DEFAULT_BACKOFF_MULTIPLIER,retrySettings.getRetryDelayMultiplier(), 0); + assertEquals(DEFAULT_MAX_ELAPSED_BACKOFF_MILLIS, retrySettings.getMaxRetryDelay().toMillis()); + assertEquals(0, retrySettings.getMaxAttempts()); + assertEquals(360_000, retrySettings.getInitialRpcTimeout().toMillis()); + assertEquals(360_000, retrySettings.getMaxRpcTimeout().toMillis()); + assertEquals(60_000, retrySettings.getTotalTimeout().toMillis()); + } + + private void verifyDisabledRetry(RetrySettings retrySettings) { + assertEquals(Duration.ZERO , retrySettings.getInitialRetryDelay()); + assertEquals(1 , retrySettings.getRetryDelayMultiplier(), 0); + assertEquals(Duration.ZERO, retrySettings.getMaxRetryDelay()); + assertEquals(1, retrySettings.getMaxAttempts()); + assertEquals(SHORT_TIMEOUT_MS_DEFAULT, retrySettings.getInitialRpcTimeout().toMillis()); + assertEquals(SHORT_TIMEOUT_MS_DEFAULT, retrySettings.getMaxRpcTimeout().toMillis()); + assertEquals(SHORT_TIMEOUT_MS_DEFAULT, retrySettings.getTotalTimeout().toMillis()); + assertEquals(1, retrySettings.getMaxAttempts()); + } + + @Test + public void testWhenBulkOptionIsDisabled() throws IOException { + BulkOptions bulkOptions = BulkOptions.builder().setUseBulkApi(false).build(); + BigtableOptions options = BigtableOptions.builder() + .setProjectId(TEST_PROJECT_ID) + .setInstanceId(TEST_INSTANCE_ID) + .setCredentialOptions(CredentialOptions.nullCredential()) + .setBulkOptions(bulkOptions) + .build(); + dataSettings = BigtableVeneerSettingsFactory.createBigtableDataSettings(options); + assertFalse(dataSettings.getStubSettings().bulkMutateRowsSettings().getBatchingSettings().getIsEnabled()); + } + + @Test + public void testBulkMutation() throws IOException { + BigtableOptions options = + BigtableOptions.builder() + .setProjectId(TEST_PROJECT_ID) + .setInstanceId(TEST_INSTANCE_ID) + .setCredentialOptions(CredentialOptions.nullCredential()) + .setUserAgent("Test-user-agent") + .build(); + dataSettings = BigtableVeneerSettingsFactory.createBigtableDataSettings(options); + + BulkOptions bulkOptions = options.getBulkOptions(); + BatchingSettings batchingSettings = dataSettings.getStubSettings().bulkMutateRowsSettings().getBatchingSettings(); + long outstandingElementCount = + bulkOptions.getMaxInflightRpcs() * bulkOptions.getBulkMaxRowKeyCount(); + + assertTrue(batchingSettings.getIsEnabled()); + assertEquals(bulkOptions.getBulkMaxRequestSize(), + batchingSettings.getRequestByteThreshold().longValue()); + assertEquals(bulkOptions.getBulkMaxRowKeyCount(), + batchingSettings.getElementCountThreshold().longValue()); + assertEquals(bulkOptions.getMaxMemory(), + batchingSettings.getFlowControlSettings().getMaxOutstandingRequestBytes().longValue()); + assertEquals(outstandingElementCount, + batchingSettings.getFlowControlSettings().getMaxOutstandingElementCount().longValue()); + } + + @Test + public void testTableAdminWithNullCredentials() throws IOException { + ServerSocket serverSocket = new ServerSocket(0); + final int availablePort = serverSocket.getLocalPort(); + serverSocket.close(); + BigtableOptions options = + BigtableOptions.builder() + .setProjectId(TEST_PROJECT_ID).setInstanceId(TEST_INSTANCE_ID) + .setCredentialOptions(CredentialOptions.nullCredential()) + .setUserAgent(TEST_USER_AGENT) + .setAdminHost("localhost") + .setPort(availablePort) + .build(); + adminSettings = BigtableVeneerSettingsFactory.createTableAdminSettings(options); + assertTrue( + adminSettings.getStubSettings().getCredentialsProvider() instanceof NoCredentialsProvider); + } +} diff --git a/bigtable-client-core-parent/bigtable-client-core/src/test/java/com/google/cloud/bigtable/config/TestUserAgent.java b/bigtable-client-core-parent/bigtable-client-core/src/test/java/com/google/cloud/bigtable/config/TestUserAgent.java new file mode 100644 index 0000000000..2620f664a8 --- /dev/null +++ b/bigtable-client-core-parent/bigtable-client-core/src/test/java/com/google/cloud/bigtable/config/TestUserAgent.java @@ -0,0 +1,237 @@ +/* + * Copyright 2019 Google LLC. + * + * Licensed 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 com.google.cloud.bigtable.config; + +import com.google.api.core.ApiFunction; +import com.google.api.gax.core.NoCredentialsProvider; +import com.google.api.gax.grpc.InstantiatingGrpcChannelProvider; +import com.google.api.gax.rpc.FixedHeaderProvider; +import com.google.api.gax.rpc.HeaderProvider; +import com.google.bigtable.v2.BigtableGrpc.BigtableImplBase; +import com.google.bigtable.v2.ReadRowsRequest; +import com.google.bigtable.v2.ReadRowsResponse; +import com.google.cloud.bigtable.data.v2.BigtableDataClient; +import com.google.cloud.bigtable.data.v2.BigtableDataSettings; +import com.google.common.io.Resources; +import io.grpc.ForwardingServerCall; +import io.grpc.ManagedChannelBuilder; +import io.grpc.Metadata; +import io.grpc.Server; +import io.grpc.ServerBuilder; +import io.grpc.ServerCall; +import io.grpc.ServerCallHandler; +import io.grpc.ServerInterceptor; +import io.grpc.ServerInterceptors; +import io.grpc.netty.shaded.io.grpc.netty.GrpcSslContexts; +import io.grpc.netty.shaded.io.grpc.netty.NettyChannelBuilder; +import io.grpc.netty.shaded.io.netty.handler.ssl.SslContext; +import io.grpc.netty.shaded.io.netty.handler.ssl.SslContextBuilder; +import io.grpc.stub.StreamObserver; +import java.io.File; +import java.net.ServerSocket; +import java.net.URL; +import java.util.regex.Pattern; +import javax.net.ssl.SSLException; +import org.junit.After; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import static io.grpc.internal.GrpcUtil.USER_AGENT_KEY; + +/** + * This class tests value present in User-Agent's on netty server. + */ +@RunWith(JUnit4.class) +public class TestUserAgent { + + private static final Logger logger = new Logger(TestUserAgent.class); + + private static final String TEST_PROJECT_ID = "ProjectId"; + private static final String TEST_INSTANCE_ID = "InstanceId"; + private static final String TEST_USER_AGENT = "test-user-agent"; + private static final Pattern EXPECTED_HEADER_PATTERN = + Pattern.compile(".*" + TEST_USER_AGENT + ".*"); + private static final String TABLE_ID = "my-table-id"; + private static final String ROWKEY = "row-key"; + + private BigtableDataSettings dataSettings; + private BigtableDataClient dataClient; + private Server server; + + @After + public void tearDown() throws Exception { + if(dataClient != null){ + dataClient.close(); + } + if (server != null) { + server.shutdown(); + server.awaitTermination(); + } + } + + /** + * To Test UserAgent & PlainText Negotiation type + * when {@link BigtableDataSettings} is created using {@link BigtableOptions}. + */ + @Test + public void testUserAgentUsingPlainTextNegotiation() throws Exception{ + ServerSocket serverSocket = new ServerSocket(0); + final int availablePort = serverSocket.getLocalPort(); + serverSocket.close(); + + //Creates non-ssl server. + createServer(availablePort); + + BigtableOptions bigtableOptions = + BigtableOptions.builder() + .setDataHost("localhost") + .setAdminHost("localhost") + .setProjectId(TEST_PROJECT_ID) + .setInstanceId(TEST_INSTANCE_ID) + .setUserAgent(TEST_USER_AGENT) + .setUsePlaintextNegotiation(true) + .setCredentialOptions(CredentialOptions.nullCredential()) + .setPort(availablePort) + .build(); + + dataSettings = BigtableVeneerSettingsFactory.createBigtableDataSettings(bigtableOptions); + + dataClient = BigtableDataClient.create(dataSettings); + dataClient.readRow(TABLE_ID, ROWKEY); + } + + /** + * Verify userAgent on TLS Negotiation with {@link InstantiatingGrpcChannelProvider} using SSL + * enabled server. + */ + @Test + public void testUserAgentUsingTLSNegotiation() throws Exception { + ServerSocket serverSocket = new ServerSocket(0); + final int availablePort = serverSocket.getLocalPort(); + serverSocket.close(); + + //Creates SSL enabled server. + createSecuredServer(availablePort); + + BigtableDataSettings.Builder builder = + BigtableDataSettings.newBuilder() + .setProjectId(TEST_PROJECT_ID) + .setInstanceId(TEST_INSTANCE_ID) + .setCredentialsProvider(NoCredentialsProvider.create()); + + //Loads secured Certificate + final SslContext sslContext = buildSslContext(); + + final String endpoint = "localhost" + ":" + availablePort; + HeaderProvider headers = FixedHeaderProvider.create(USER_AGENT_KEY.name(), TEST_USER_AGENT); + + builder.setTransportChannelProvider(InstantiatingGrpcChannelProvider.newBuilder() + .setHeaderProvider(headers) + .setEndpoint(endpoint) + .setChannelConfigurator(new ApiFunction() { + @Override + public ManagedChannelBuilder apply(ManagedChannelBuilder input) { + return ((NettyChannelBuilder)input).sslContext(sslContext); + } + }) + .build()); + + dataClient = BigtableDataClient.create(builder.build()); + dataClient.readRow(TABLE_ID, ROWKEY); + } + + /** Creates simple server to intercept plainText Negotiation RPCs. */ + private void createServer(int port) throws Exception{ + server = ServerBuilder.forPort(port).addService( + ServerInterceptors.intercept(new BigtableExtendedImpl(), new HeaderServerInterceptor())) + .build(); + server.start(); + } + + /** Creates secured server to intercept TLS Negotiation RPCs. */ + private void createSecuredServer(int port) throws Exception { + ServerBuilder builder = ServerBuilder.forPort(port).addService( + ServerInterceptors.intercept(new BigtableExtendedImpl(), new HeaderServerInterceptor())); + + try { + URL serverCertChain = Resources.getResource("sslCertificates/server_trust.crt"); + URL privateKey = Resources.getResource("sslCertificates/server_key.pem"); + + builder.useTransportSecurity(new File(serverCertChain.getFile()), + new File(privateKey.getFile())); + } catch (Exception ex) { + throw new AssertionError("No server certificates found"); + } + server = builder.build(); + server.start(); + } + + /** Creates SSLContext from certificate & private key provided in classpath. */ + private static SslContext buildSslContext() throws SSLException { + SslContextBuilder builder = GrpcSslContexts.forClient(); + + try { + URL url = Resources.getResource("sslCertificates/client_trust.crt"); + builder.trustManager(new File(url.getFile())); + } catch (Exception ex) { + throw new AssertionError("No client trust certificate found"); + } + + return builder.build(); + } + + /** + * Overrides {@link BigtableImplBase#readRows(ReadRowsRequest, StreamObserver)} and returns + * dummy response. + * */ + private static class BigtableExtendedImpl extends BigtableImplBase { + @Override + public void readRows(ReadRowsRequest request, + StreamObserver responseObserver) { + responseObserver.onNext(ReadRowsResponse.getDefaultInstance()); + responseObserver.onCompleted(); + } + } + + /** + * Asserts value of UserAgent header with EXPECTED_HEADER_PATTERN passed to the + * {@link InstantiatingGrpcChannelProvider}. + * + * Throws {@link AssertionError} when UserAgent's pattern does not match. + */ + private class HeaderServerInterceptor implements ServerInterceptor { + @Override + public ServerCall.Listener interceptCall(ServerCall call, + final Metadata requestHeaders, ServerCallHandler next) { + + //Logging all available headers. + logger.info("headers received from BigtableDataClient:" + requestHeaders); + + Metadata.Key USER_AGENT_KEY = + Metadata.Key.of("user-agent", Metadata.ASCII_STRING_MARSHALLER); + String headerValue = requestHeaders.get(USER_AGENT_KEY); + + //In case of user-agent not matching, throwing AssertionError. + if (!EXPECTED_HEADER_PATTERN.matcher(headerValue).matches()) { + throw new AssertionError("User-Agent's format did not match"); + } + return next.startCall(new ForwardingServerCall.SimpleForwardingServerCall(call) { + }, requestHeaders); + } + } +} + diff --git a/bigtable-client-core-parent/bigtable-client-core/src/test/resources/sslCertificates/client_trust.crt b/bigtable-client-core-parent/bigtable-client-core/src/test/resources/sslCertificates/client_trust.crt new file mode 100644 index 0000000000..8f20db5051 --- /dev/null +++ b/bigtable-client-core-parent/bigtable-client-core/src/test/resources/sslCertificates/client_trust.crt @@ -0,0 +1,27 @@ +-----BEGIN CERTIFICATE----- +MIIEpDCCAowCCQDVy3LQHkRQtDANBgkqhkiG9w0BAQsFADAUMRIwEAYDVQQDDAls +b2NhbGhvc3QwHhcNMTkwMTE2MDc1NTE3WhcNMjAwMTE2MDc1NTE3WjAUMRIwEAYD +VQQDDAlsb2NhbGhvc3QwggIiMA0GCSqGSIb3DQEBAQUAA4ICDwAwggIKAoICAQDi +6mUxMsBRC6LEi5Cv0qdaEjEcFlF8qx5EGKs1p77+7Nx66VBnlrmmcwwBySwhArpD +JGXXMF3lpXGVHLENYXSwtq101OCTAGgwuZqvIxDcjckQqDu8Q5tLj94+axtq05Hx +50Ot7wtRBX9oS+8AOACgPsO5IQp/T92pN11UKY4K3tIs34HLAsqkoHFHsnzP7waV +0HIh0HdWd9oeWNQCIwALFj33dfZYmzc7592RSHPrEZdspIs3GEpA9WT2E9OrCHem +Hql3CDaNTWcO+SppJqSraRI2pKJSTDDVGWjkkbT+vYcX58gNlpqe98xzDhgt+s9X +X6WqZP5PiHJhVzxSC5oNLHtjsuoOJpy0WLccgjpqNiK747Kf5wm88TJf1KJ885O9 +BoX7ij8EFbdK0FoAfNkMiadFF7niRFUv2AnXd7MMHYRoeDDjfQ08cnLuKfQgRVIv +eY88b2C9Fvonf088mOqCGfdCPt+QB16LfarHmFE15ANmjzkIKgxfqo4ptUuRmsm2 +3i71ZLOq3t7dUvLSFmaY2SioXzB6lrKMvnrgEyCd8ka3LHXGohrwHEHFdhwHCVjw +7ZRBxK4kqihg60uENduPhGWVcwY4QLDNX/77aiMeWI8aXlV4qI/8IkPJTqTh+ZDe +QW93fw4BsrY2R6PfBVcQnxqAdkg+aExu40ESKz6tawIDAQABMA0GCSqGSIb3DQEB +CwUAA4ICAQBE+dvwx5Dry6PUiKBSAbCwmFsqj6Q+P11Q9dUzinfORNRAiMfHiJd+ +ngIqiMR5D/d0wYK6XR1MckjEM9N8i4N25B4qaYpOgI23/iU5cIYRIjVxpfNNmMON +fXcvSMhqS6e14spTnxrg1X5Yj9oE8gE9FArA6EqQpEarELSXDmiRgilJZKQ4RGFk +WOlZcTphwl3gerOD6l37DMWZXWx5DJCfQpRpw/3j2i86qdHN16Le5Ija5CBK+cTU +Vw5RsveumZPjTqsTtyQ8LncoY/fBh/I4QZoelh5s++QqOSvf48Db5WIPNcyGOPKZ +AgXgOabEGX2rEj73zA/80h5YQkVyiywXGxqp3Jy/tPXrXVYJqHNqF3ZCMJAgKtYe +p7JOv/bmhjuSyt+j0gv2Ot/ulWX1gjKHSPrFwd03rT5wXe9ooUWn83VwrmhW5D+l +nn8gi80dqy7GfQWGoQcYmtrd1U3YPQy1kgqO1v807v/QWHyLGWGqXesiO5HqgKp+ +OLXUvnmQYs4HERPCKbrWQ+blgVml0Kiochz50x8sI+aYjErwuinLTXtr0tZmtAO2 +23olBde5MvCBq1XCfJ5hG+2+YHb8LpvkjlbPbpopIftvIFZBSMPCjiZGvpxLS5pA +S2ngXkxoAce7xY/IO+W2FgCvC77Uy6UXiJIZ1PbSdFLN47OzRTMmOg== +-----END CERTIFICATE----- diff --git a/bigtable-client-core-parent/bigtable-client-core/src/test/resources/sslCertificates/server_key.pem b/bigtable-client-core-parent/bigtable-client-core/src/test/resources/sslCertificates/server_key.pem new file mode 100644 index 0000000000..0c75bb8ba3 --- /dev/null +++ b/bigtable-client-core-parent/bigtable-client-core/src/test/resources/sslCertificates/server_key.pem @@ -0,0 +1,52 @@ +-----BEGIN PRIVATE KEY----- +MIIJQgIBADANBgkqhkiG9w0BAQEFAASCCSwwggkoAgEAAoICAQCchlHZDz4QYh5a +k8ryDoaqux6CEXko/ZOcaFFJ19Oz85lKCJbRguW0L9MraL4W4TxuPKinEl0iKj2U +UpAhngGLpLMLwKRAI5gGnbDdbecKwJ7gzjw+owvdY8rq+EfUOQpLG6lwnKGe3ITW +dmcrRZ7afOXDbuGn2RMYNSBRJCbmvqGptTEVK6S5ktXYS2RWJWnKBuatqxzDOgS0 +iRBqOCG3jw8gV6b9I1dMWqCLXCUiL84xrPCy4cn6MRe5Hibz3qDVKsIbkmOUeM44 +hT/lME6IRNkM6PmbLEAEimUV56pwUm4fGECZYxQ3GfWXwFvSDrsB++mP7hx0dlTD +QdA08UaC99VjnJfIKDQGNn1l7TWpuIGPlmLBnj/mwnAk/J4fFS2354x7T5KS3yc6 +SAhf5qStcMyTBp50vjV3CFGwtUrM2ywHIEnsa/NubhSf+rA4+3IrY4AFQEIHrbAM +sWuH47CB6qvrAKeNErTlrMoNSnJ1ifNh8Mtrp7421pX1H3PjTRkSYRDh8TulV2Fd +mLMsxNWZtydaIqUJpETmz7QF8k/KLKmFf3CH6wauQCGZ7WkVuVUYoWQ5EaNZdpZD +3lZblIbVbNyoE04pA7IxS1jTigvNUCIjTBPxbqlaJ8fDQa1j3T3p9vU5CWlEoMlc +M4GunZ/KuQmMkWOTFe9rlwZdVsqXiQIDAQABAoICAB+9GNGjxpj53F/PgiJtEZ+D +C9+h/WhzasAXYwxPKE1gAuBBN5vKKBvOasptlcX77A4KXMRr89ieqhfuWe2HJZcU +UX37lRKwZP5KcXek9ml2v4CkMREebKR6vMXCjIYa/txWApikGh2lZ4Y5QhhDKL/y +/YAQds9bNAT/mMr7QqwdUZig9g/t5pv+i0AtoW3abkYZ+iKTBOUNWbDl7h8rvKKE +7rNHMsQ3q1b/NXisuZbiDLSD9da/8/24gK/3zjw7E2/GGNAQcgLEMETDuD4X+oSl +OUHl598wLNb+3RqA/E2EWXISr96qeYnmyKb5yWShugff9e9KgBiZefyR1s81cW+B +6zBZCK4bG4G/DGHxZUAdY/siBGJ1Vw2kTvt/g4P/5p3s1Hg3FFpAAElBRM4bFMYa +4fEf8EoVgH9KoRQYrWHcdaPmS/F8mwNSanFANEGF/8MXoYgwDFqoezFwEpmEpjpz +UfHcUwlQo+czFlJkvhBX30rVqTBT8gBXaMTO5x9q1OifBmVzCk9rHEVL+bNPBf48 +jeXPhVS+FbVtmxv+etNHSameL48lvlwgWUw6YXZEMs+bxK8P6PwGDc+ygL0V6Y3e +A3AX/ROaU6EpcCgFoW6FRULuAzvllMSosIiPHPsf0+zE1eWGk/6qW+xBb/YDhrNY +gTO0BmAqvnbzAt/9FqylAoIBAQDOgcSmZV4+aaM4aZpiDUSG+SRS5VDYYdKPA8IF +BIYodqKdF2IP3ZRHC4ZPShNCAeNK2QZUa7vxReey0ZQ3iuVVw1FlfT8Bg5AS2PTY +gYJZWnXyqNowXagGzgyx8vCnDSRnWyQHtmxoCyHRvgqdIgEnr2e/4IsYBwACoyji +rCA+TcZUOVowpLA2lOvjftlvk/ONkzCIVaqMouwFRvo25Pd4d6Cte3uyrREwW5YU +tFnImzMeoLVqBQRJxzHxkaLq4onSCelermALam/yRWxJ7so9UIySWdCt48iX6Y6R +wg1L90PGfk4bMyo6r3vr+HOggMFMmiFyQFjj968A/qhtIfPDAoIBAQDCCeRscGwJ +UGPoDlfxvDLTLPtR/GGn69ph6i9Avs49UsdSFBCRx7tp4igG4cu68ixX/5vXBeTc +h0mdbN8trdMG+CmQzpHH0coGSHxKR8zqsX3u+DC265WEcQmCXpyHmXYt/MhQ05Vf +4DpFn/foO/S90Lnjeb1YGBF8fM0hQNmAVIaEkVKr4qz55CZ/CK/dFNrqhmG1/WDq +FXUwPtHe7KXumsnViXOavbZIq9zS3/0d/sTsk1aQwEvohV7cBQNu5LMFxXhOdq70 +oomO1exA6ebMs6YI/tg1SYXSRHlQAy62vl0gGLWjEk5KFG6Jm4q2md3r4Cq8nqX+ +mPuSAd6jrc7DAoIBAQCV7xQ3d/vwNsTCLbJgpJ9vnC6qXULQiF3XZnQVLo61jEVO +525UdJ2244A1y2OmFjbP3jjBTKNdb1Llu1Lw7NDL7w8HL+NyFvKhLJLBK27Xr11Z +vQ0F9soLfskq3mep8lnPDRZg3ytgc0yvu1p7cfgCt1ENuFtlK5fvUkVaJn0Wn2Nz +PVzlWn1yc7e9Khl28KXx0b94DTLR/2x7/GepOZ8QAVNRAgny1cNYJ/cUu0lA5KY+ +A5MUEeZmJtF1QjdWcPp93KJrExI616HF9m/EUjLWR8h4beuchXjcC0BvRcRftnyn +/oZ5KzHrAARqdK6HB1TRaEsGZ//iLLAMedkRu4f/AoIBAFeCyJ9TcJYeyIAe//DJ +FQ8CqXIHZi+qgjpIuPj1VOR5UeWloPduYgrV8YFxEkYE9JTXdfxa7yW6aivyrxDV +5WeuQmjJkRvSdM+yn0OM8FEkrYw6KRZGV498v3Ipo2TQhGknKVyyoBq2cPF3vQHj +EHUPDtcVz5xMdo5zZrxJmA51kJl3RRa6S5MEH4yJaW9G+Zht9OF0HeJUvR5+Jn7o +sclTzBVfQWEXkH9j61q7pFeD0GJ17MOpxlJe3DJUnIDF6it/LMwOYMd6Cpn9xuhb +1TibCRXjgbU+rHjxYtkAJnBKmItiOYELP33Piy7KlJvX9ytrVTruPiT97++goj2l +pKsCggEAYk5INBQad3Ay3roIKSNplnp0/F/AARuVlfd78o0Lt91Sjulq76eHVvuu +8VDXL2nClMmJ7tL8M0yNOarUUS8rYiUq/ntwNvayyJq5IuQffpq+aY82IS2tNRnj +gBiaUtmT3+KjXFTy+AylDDLCVhRi3Sfk1cznTJ2Fjxifmrbs8HiVH5v96mmBoe/P +DFGjtMVF/afI/j+NVqC0dIVCiCxBkkr7CRVjFy2hy87l4IF8Z2kc9rNXZUjGRpkV +0ZqD40jkS/mMZI8NRa5U1adoHopgVDxXoMuG8AI73szPynuzEeGS8ki6iCYUn2Go +vEanFeXQY0SsYfTJlOcVEgVbA8hHBQ== +-----END PRIVATE KEY----- diff --git a/bigtable-client-core-parent/bigtable-client-core/src/test/resources/sslCertificates/server_trust.crt b/bigtable-client-core-parent/bigtable-client-core/src/test/resources/sslCertificates/server_trust.crt new file mode 100644 index 0000000000..b6d72c8312 --- /dev/null +++ b/bigtable-client-core-parent/bigtable-client-core/src/test/resources/sslCertificates/server_trust.crt @@ -0,0 +1,27 @@ +-----BEGIN CERTIFICATE----- +MIIEnDCCAoQCAQEwDQYJKoZIhvcNAQEFBQAwFDESMBAGA1UEAwwJbG9jYWxob3N0 +MB4XDTE5MDExNjA3NTUzMVoXDTIwMDExNjA3NTUzMVowFDESMBAGA1UEAwwJbG9j +YWxob3N0MIICIjANBgkqhkiG9w0BAQEFAAOCAg8AMIICCgKCAgEAnIZR2Q8+EGIe +WpPK8g6GqrseghF5KP2TnGhRSdfTs/OZSgiW0YLltC/TK2i+FuE8bjyopxJdIio9 +lFKQIZ4Bi6SzC8CkQCOYBp2w3W3nCsCe4M48PqML3WPK6vhH1DkKSxupcJyhntyE +1nZnK0We2nzlw27hp9kTGDUgUSQm5r6hqbUxFSukuZLV2EtkViVpygbmrascwzoE +tIkQajght48PIFem/SNXTFqgi1wlIi/OMazwsuHJ+jEXuR4m896g1SrCG5JjlHjO +OIU/5TBOiETZDOj5myxABIplFeeqcFJuHxhAmWMUNxn1l8Bb0g67Afvpj+4cdHZU +w0HQNPFGgvfVY5yXyCg0BjZ9Ze01qbiBj5ZiwZ4/5sJwJPyeHxUtt+eMe0+Skt8n +OkgIX+akrXDMkwaedL41dwhRsLVKzNssByBJ7Gvzbm4Un/qwOPtyK2OABUBCB62w +DLFrh+Owgeqr6wCnjRK05azKDUpydYnzYfDLa6e+NtaV9R9z400ZEmEQ4fE7pVdh +XZizLMTVmbcnWiKlCaRE5s+0BfJPyiyphX9wh+sGrkAhme1pFblVGKFkORGjWXaW +Q95WW5SG1WzcqBNOKQOyMUtY04oLzVAiI0wT8W6pWifHw0GtY9096fb1OQlpRKDJ +XDOBrp2fyrkJjJFjkxXva5cGXVbKl4kCAwEAATANBgkqhkiG9w0BAQUFAAOCAgEA +TSbf7OhFnW05HE4TvKuv1rvh1TBCFzwhjQvKFi8oYUA4g8grcD7n2bJGgRjrniSa +CZBqbmPQc+T4jUuMFEzZ5tz30P/2ma5ZH1F6jpd2E6Igqwmrwa0z+iyQnjR2sMZ6 +lKQ7wGeNBTfgreCiVSmTQE/xck2Z+if7vZXabdgzAtagfA7Q8SpsV5VbYNJSjBLk +PSilineI8cWh4j+FARcICfDPGyqljsSSk1gIHqmwX3Y09MmslVFqSktVGSGV7Svv +GI/LkO0rwPhg8f2W7bkgEa0v46Rk9Ss43Pqpr8umQ7/iFqR/bEUdgXXPBmqw+l42 +kD9WlxCiq8C+rF3ULvkvVA3vj+H5iUqsoO0oc4ooAkDWcSeoPw+MGcet3S1elAKE +xjbbb4i5gF7TJkJai/hC7sGXnjBb6e1j7j685fv/B1st7mz98TMXbKEiTrBeS8q1 +N7AFCrFPztf9/+ItAmL9eqK/XBswNl4MtRZ5td337UhWzNBiBpvXFSQJmXWqs/ol +cFAJNsDRCl6ssHA/n53bW26L7VZtFViKT9huxAWmaWM7UVweRCreUmKw6FR+q4fN +BM8ffiUO92kyk9QZXXNDk0RZNGoKdIT7iaP3rUQAyTHQuF5flQX/afR+1bRx45HR +ERvwHniD31P0ekB+sS23MMjxWa2+b0k9pb2iwb0oWT4= +-----END CERTIFICATE-----