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-----