{
+ /** Returns true if previousThrowable is an {@link ApiException} that is retryable. */
@Override
- public TimedAttemptSettings createNextAttempt(
- Throwable prevThrowable, ResponseT prevResponse, TimedAttemptSettings prevSettings) {
- if (prevThrowable != null && prevThrowable instanceof DeadlineExceededException) {
- return TimedAttemptSettings.newBuilder()
- .setGlobalSettings(prevSettings.getGlobalSettings())
- .setRetryDelay(prevSettings.getRetryDelay())
- .setRpcTimeout(prevSettings.getRpcTimeout())
- .setRandomizedRetryDelay(DEADLINE_SLEEP_DURATION)
- .setAttemptCount(prevSettings.getAttemptCount() + 1)
- .setFirstAttemptStartTimeNanos(prevSettings.getFirstAttemptStartTimeNanos())
- .build();
- }
- return null;
+ public boolean shouldRetry(Throwable previousThrowable, ResponseT previousResponse) {
+ return (previousThrowable instanceof ApiException)
+ && ((ApiException) previousThrowable).isRetryable();
}
+ /**
+ * If {@link RetryingContext#getRetryableCodes()} is not null: Returns true if the status code of
+ * previousThrowable is in the list of retryable code of the {@link RetryingContext}.
+ *
+ * Otherwise it returns the result of {@link #shouldRetry(Throwable, Object)}.
+ */
@Override
- public boolean shouldRetry(Throwable prevThrowable, ResponseT prevResponse) {
- return (prevThrowable instanceof ApiException) && ((ApiException) prevThrowable).isRetryable();
+ public boolean shouldRetry(
+ RetryingContext context, Throwable previousThrowable, ResponseT previousResponse) {
+ if (context.getRetryableCodes() != null) {
+ // Ignore the isRetryable() value of the throwable if the RetryingContext has a specific list
+ // of codes that should be retried.
+ return (previousThrowable instanceof ApiException)
+ && context
+ .getRetryableCodes()
+ .contains(((ApiException) previousThrowable).getStatusCode().getCode());
+ }
+ return shouldRetry(previousThrowable, previousResponse);
}
}
diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/gaxx/retrying/AttemptCallable.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/gaxx/retrying/AttemptCallable.java
new file mode 100644
index 0000000000..3599e1e4df
--- /dev/null
+++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/gaxx/retrying/AttemptCallable.java
@@ -0,0 +1,84 @@
+/*
+ * Copyright 2023 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
+ *
+ * https://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.gaxx.retrying;
+
+import com.google.api.core.ApiFuture;
+import com.google.api.core.ApiFutures;
+import com.google.api.core.InternalApi;
+import com.google.api.gax.retrying.NonCancellableFuture;
+import com.google.api.gax.retrying.RetryingFuture;
+import com.google.api.gax.rpc.ApiCallContext;
+import com.google.api.gax.rpc.UnaryCallable;
+import com.google.common.base.Preconditions;
+import java.util.concurrent.Callable;
+import org.threeten.bp.Duration;
+
+// TODO: remove this once ApiResultRetryAlgorithm is added to gax.
+/**
+ * A callable representing an attempt to make an RPC call. This class is used from {@link
+ * RetryingCallable}.
+ *
+ * @param request type
+ * @param response type
+ */
+@InternalApi
+public class AttemptCallable implements Callable {
+ private final UnaryCallable callable;
+ private final RequestT request;
+ private final ApiCallContext originalCallContext;
+
+ private volatile RetryingFuture externalFuture;
+
+ AttemptCallable(
+ UnaryCallable callable, RequestT request, ApiCallContext callContext) {
+ this.callable = Preconditions.checkNotNull(callable);
+ this.request = Preconditions.checkNotNull(request);
+ this.originalCallContext = Preconditions.checkNotNull(callContext);
+ }
+
+ public void setExternalFuture(RetryingFuture externalFuture) {
+ this.externalFuture = Preconditions.checkNotNull(externalFuture);
+ }
+
+ @Override
+ public ResponseT call() {
+ ApiCallContext callContext = originalCallContext;
+
+ try {
+ // Set the RPC timeout if the caller did not provide their own.
+ Duration rpcTimeout = externalFuture.getAttemptSettings().getRpcTimeout();
+ if (!rpcTimeout.isZero() && callContext.getTimeout() == null) {
+ callContext = callContext.withTimeout(rpcTimeout);
+ }
+
+ externalFuture.setAttemptFuture(new NonCancellableFuture());
+ if (externalFuture.isDone()) {
+ return null;
+ }
+
+ callContext
+ .getTracer()
+ .attemptStarted(request, externalFuture.getAttemptSettings().getOverallAttemptCount());
+
+ ApiFuture internalFuture = callable.futureCall(request, callContext);
+ externalFuture.setAttemptFuture(internalFuture);
+ } catch (Throwable e) {
+ externalFuture.setAttemptFuture(ApiFutures.immediateFailedFuture(e));
+ }
+
+ return null;
+ }
+}
diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/gaxx/retrying/Callables.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/gaxx/retrying/Callables.java
new file mode 100644
index 0000000000..a78e7643b0
--- /dev/null
+++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/gaxx/retrying/Callables.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright 2023 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
+ *
+ * https://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.gaxx.retrying;
+
+import com.google.api.core.InternalApi;
+import com.google.api.gax.retrying.ExponentialRetryAlgorithm;
+import com.google.api.gax.retrying.RetryAlgorithm;
+import com.google.api.gax.retrying.ScheduledRetryingExecutor;
+import com.google.api.gax.retrying.StreamingRetryAlgorithm;
+import com.google.api.gax.rpc.ClientContext;
+import com.google.api.gax.rpc.ServerStreamingCallSettings;
+import com.google.api.gax.rpc.ServerStreamingCallable;
+import com.google.api.gax.rpc.UnaryCallSettings;
+import com.google.api.gax.rpc.UnaryCallable;
+
+// TODO: remove this once ApiResultRetryAlgorithm is added to gax.
+/**
+ * Class with utility methods to create callable objects using provided settings.
+ *
+ * The callable objects wrap a given direct callable with features like retry and exception
+ * translation.
+ */
+@InternalApi
+public class Callables {
+
+ private Callables() {}
+
+ public static UnaryCallable retrying(
+ UnaryCallable innerCallable,
+ UnaryCallSettings, ?> callSettings,
+ ClientContext clientContext) {
+
+ UnaryCallSettings, ?> settings = callSettings;
+
+ RetryAlgorithm retryAlgorithm =
+ new RetryAlgorithm<>(
+ new RetryInfoRetryAlgorithm<>(),
+ new ExponentialRetryAlgorithm(settings.getRetrySettings(), clientContext.getClock()));
+ ScheduledRetryingExecutor executor =
+ new ScheduledRetryingExecutor<>(retryAlgorithm, clientContext.getExecutor());
+
+ return new RetryingCallable<>(clientContext.getDefaultCallContext(), innerCallable, executor);
+ }
+
+ public static ServerStreamingCallable retrying(
+ ServerStreamingCallable innerCallable,
+ ServerStreamingCallSettings callSettings,
+ ClientContext clientContext) {
+
+ ServerStreamingCallSettings settings = callSettings;
+
+ StreamingRetryAlgorithm retryAlgorithm =
+ new StreamingRetryAlgorithm<>(
+ new RetryInfoRetryAlgorithm<>(),
+ new ExponentialRetryAlgorithm(settings.getRetrySettings(), clientContext.getClock()));
+
+ ScheduledRetryingExecutor retryingExecutor =
+ new ScheduledRetryingExecutor<>(retryAlgorithm, clientContext.getExecutor());
+
+ return new RetryingServerStreamingCallable<>(
+ innerCallable, retryingExecutor, settings.getResumptionStrategy());
+ }
+}
diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/gaxx/retrying/RetryInfoRetryAlgorithm.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/gaxx/retrying/RetryInfoRetryAlgorithm.java
new file mode 100644
index 0000000000..085b48bbb5
--- /dev/null
+++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/gaxx/retrying/RetryInfoRetryAlgorithm.java
@@ -0,0 +1,102 @@
+/*
+ * Copyright 2023 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
+ *
+ * https://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.gaxx.retrying;
+
+import com.google.api.core.InternalApi;
+import com.google.api.gax.retrying.BasicResultRetryAlgorithm;
+import com.google.api.gax.retrying.RetryingContext;
+import com.google.api.gax.retrying.TimedAttemptSettings;
+import com.google.api.gax.rpc.ApiException;
+import com.google.protobuf.util.Durations;
+import com.google.rpc.RetryInfo;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.threeten.bp.Duration;
+
+// TODO move this algorithm to gax
+/**
+ * This retry algorithm checks the metadata of an exception for additional error details. If the
+ * metadata has a RetryInfo field, use the retry delay to set the wait time between attempts.
+ */
+@InternalApi
+public class RetryInfoRetryAlgorithm extends BasicResultRetryAlgorithm {
+
+ @Override
+ public TimedAttemptSettings createNextAttempt(
+ Throwable prevThrowable, ResponseT prevResponse, TimedAttemptSettings prevSettings) {
+ Duration retryDelay = extractRetryDelay(prevThrowable);
+ if (retryDelay != null) {
+ return prevSettings
+ .toBuilder()
+ .setRandomizedRetryDelay(retryDelay)
+ .setAttemptCount(prevSettings.getAttemptCount() + 1)
+ .setOverallAttemptCount(prevSettings.getAttemptCount() + 1)
+ .build();
+ }
+ return null;
+ }
+
+ /** Returns true if previousThrowable is an {@link ApiException} that is retryable. */
+ @Override
+ public boolean shouldRetry(Throwable previousThrowable, ResponseT previousResponse) {
+ return shouldRetry(null, previousThrowable, previousResponse);
+ }
+
+ /**
+ * If {@link RetryingContext#getRetryableCodes()} is not null: Returns true if the status code of
+ * previousThrowable is in the list of retryable code of the {@link RetryingContext}.
+ *
+ * Otherwise it returns the result of {@link #shouldRetry(Throwable, Object)}.
+ */
+ @Override
+ public boolean shouldRetry(
+ @Nullable RetryingContext context, Throwable previousThrowable, ResponseT previousResponse) {
+ if (extractRetryDelay(previousThrowable) != null) {
+ // First check if server wants us to retry
+ return true;
+ }
+ if (context != null && context.getRetryableCodes() != null) {
+ // Ignore the isRetryable() value of the throwable if the RetryingContext has a specific list
+ // of codes that should be retried.
+ return ((previousThrowable instanceof ApiException)
+ && context
+ .getRetryableCodes()
+ .contains(((ApiException) previousThrowable).getStatusCode().getCode()));
+ }
+ // Server didn't have retry information and there's no retry context, use the local status
+ // code config.
+ return previousThrowable instanceof ApiException
+ && ((ApiException) previousThrowable).isRetryable();
+ }
+
+ @Nullable
+ static Duration extractRetryDelay(@Nullable Throwable throwable) {
+ if (throwable == null) {
+ return null;
+ }
+ if (!(throwable instanceof ApiException)) {
+ return null;
+ }
+ ApiException exception = (ApiException) throwable;
+ if (exception.getErrorDetails() == null) {
+ return null;
+ }
+ if (exception.getErrorDetails().getRetryInfo() == null) {
+ return null;
+ }
+ RetryInfo retryInfo = exception.getErrorDetails().getRetryInfo();
+ return Duration.ofMillis(Durations.toMillis(retryInfo.getRetryDelay()));
+ }
+}
diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/gaxx/retrying/RetryingCallable.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/gaxx/retrying/RetryingCallable.java
new file mode 100644
index 0000000000..d78bf08322
--- /dev/null
+++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/gaxx/retrying/RetryingCallable.java
@@ -0,0 +1,58 @@
+/*
+ * Copyright 2023 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
+ *
+ * https://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.gaxx.retrying;
+
+import com.google.api.core.InternalApi;
+import com.google.api.gax.retrying.RetryingExecutorWithContext;
+import com.google.api.gax.retrying.RetryingFuture;
+import com.google.api.gax.rpc.ApiCallContext;
+import com.google.api.gax.rpc.UnaryCallable;
+import com.google.common.base.Preconditions;
+
+// TODO: remove this once ApiResultRetryAlgorithm is added to gax.
+/**
+ * A UnaryCallable that will keep issuing calls to an inner callable until it succeeds or times out.
+ */
+@InternalApi
+public class RetryingCallable extends UnaryCallable {
+ private final ApiCallContext callContextPrototype;
+ private final UnaryCallable callable;
+ private final RetryingExecutorWithContext executor;
+
+ public RetryingCallable(
+ ApiCallContext callContextPrototype,
+ UnaryCallable callable,
+ RetryingExecutorWithContext executor) {
+ this.callContextPrototype = (ApiCallContext) Preconditions.checkNotNull(callContextPrototype);
+ this.callable = (UnaryCallable) Preconditions.checkNotNull(callable);
+ this.executor = (RetryingExecutorWithContext) Preconditions.checkNotNull(executor);
+ }
+
+ public RetryingFuture futureCall(RequestT request, ApiCallContext inputContext) {
+ ApiCallContext context = this.callContextPrototype.nullToSelf(inputContext);
+ AttemptCallable retryCallable =
+ new AttemptCallable(this.callable, request, context);
+ RetryingFuture retryingFuture =
+ this.executor.createFuture(retryCallable, inputContext);
+ retryCallable.setExternalFuture(retryingFuture);
+ retryCallable.call();
+ return retryingFuture;
+ }
+
+ public String toString() {
+ return String.format("retrying(%s)", this.callable);
+ }
+}
diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/gaxx/retrying/RetryingServerStreamingCallable.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/gaxx/retrying/RetryingServerStreamingCallable.java
new file mode 100644
index 0000000000..504cf4f2b7
--- /dev/null
+++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/gaxx/retrying/RetryingServerStreamingCallable.java
@@ -0,0 +1,99 @@
+/*
+ * Copyright 2023 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
+ *
+ * https://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.gaxx.retrying;
+
+import static com.google.common.util.concurrent.MoreExecutors.directExecutor;
+
+import com.google.api.core.ApiFutureCallback;
+import com.google.api.core.ApiFutures;
+import com.google.api.core.InternalApi;
+import com.google.api.gax.retrying.RetryingFuture;
+import com.google.api.gax.retrying.ScheduledRetryingExecutor;
+import com.google.api.gax.retrying.ServerStreamingAttemptException;
+import com.google.api.gax.retrying.StreamResumptionStrategy;
+import com.google.api.gax.rpc.ApiCallContext;
+import com.google.api.gax.rpc.ResponseObserver;
+import com.google.api.gax.rpc.ServerStreamingCallable;
+
+// TODO: remove this once ApiResultRetryAlgorithm is added to gax.
+/**
+ * A ServerStreamingCallable that implements resumable retries.
+ *
+ * Wraps a request, a {@link ResponseObserver} and an inner {@link ServerStreamingCallable} and
+ * coordinates retries between them. When the inner callable throws an error, this class will
+ * schedule retries using the configured {@link ScheduledRetryingExecutor}.
+ *
+ *
Streams can be resumed using a {@link StreamResumptionStrategy}. The {@link
+ * StreamResumptionStrategy} is notified of incoming responses and is expected to track the progress
+ * of the stream. Upon receiving an error, the {@link StreamResumptionStrategy} is asked to modify
+ * the original request to resume the stream.
+ */
+@InternalApi
+public final class RetryingServerStreamingCallable
+ extends ServerStreamingCallable {
+
+ private final ServerStreamingCallable innerCallable;
+ private final ScheduledRetryingExecutor executor;
+ private final StreamResumptionStrategy resumptionStrategyPrototype;
+
+ public RetryingServerStreamingCallable(
+ ServerStreamingCallable innerCallable,
+ ScheduledRetryingExecutor executor,
+ StreamResumptionStrategy resumptionStrategyPrototype) {
+ this.innerCallable = innerCallable;
+ this.executor = executor;
+ this.resumptionStrategyPrototype = resumptionStrategyPrototype;
+ }
+
+ @Override
+ public void call(
+ RequestT request,
+ final ResponseObserver responseObserver,
+ ApiCallContext context) {
+
+ ServerStreamingAttemptCallable attemptCallable =
+ new ServerStreamingAttemptCallable<>(
+ innerCallable,
+ resumptionStrategyPrototype.createNew(),
+ request,
+ context,
+ responseObserver);
+
+ RetryingFuture retryingFuture = executor.createFuture(attemptCallable, context);
+ attemptCallable.setExternalFuture(retryingFuture);
+ attemptCallable.start();
+
+ // Bridge the future result back to the external responseObserver
+ ApiFutures.addCallback(
+ retryingFuture,
+ new ApiFutureCallback() {
+ @Override
+ public void onFailure(Throwable throwable) {
+ // Make sure to unwrap the underlying ApiException
+ if (throwable instanceof ServerStreamingAttemptException) {
+ throwable = throwable.getCause();
+ }
+ responseObserver.onError(throwable);
+ }
+
+ @Override
+ public void onSuccess(Void ignored) {
+ responseObserver.onComplete();
+ }
+ },
+ directExecutor());
+ }
+}
diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/gaxx/retrying/ServerStreamingAttemptCallable.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/gaxx/retrying/ServerStreamingAttemptCallable.java
new file mode 100644
index 0000000000..793cf2e91c
--- /dev/null
+++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/gaxx/retrying/ServerStreamingAttemptCallable.java
@@ -0,0 +1,366 @@
+/*
+ * Copyright 2023 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
+ *
+ * https://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.gaxx.retrying;
+
+import com.google.api.core.InternalApi;
+import com.google.api.core.SettableApiFuture;
+import com.google.api.gax.retrying.RetryingFuture;
+import com.google.api.gax.retrying.ServerStreamingAttemptException;
+import com.google.api.gax.retrying.StreamResumptionStrategy;
+import com.google.api.gax.rpc.ApiCallContext;
+import com.google.api.gax.rpc.ResponseObserver;
+import com.google.api.gax.rpc.ServerStreamingCallable;
+import com.google.api.gax.rpc.StateCheckingResponseObserver;
+import com.google.api.gax.rpc.StreamController;
+import com.google.common.base.Preconditions;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CancellationException;
+
+// TODO: remove this once ApiResultRetryAlgorithm is added to gax.
+/**
+ * A callable that generates Server Streaming attempts. At any one time, it is responsible for at
+ * most a single outstanding attempt. During an attempt, it proxies all incoming message to the
+ * outer {@link ResponseObserver} and the {@link StreamResumptionStrategy}. Once the attempt
+ * completes, the external {@link RetryingFuture} future is notified. If the {@link RetryingFuture}
+ * decides to retry the attempt, it will invoke {@link #call()}.
+ *
+ * The lifecycle of this class is:
+ *
+ *
+ * - The caller instantiates this class.
+ *
- The caller sets the {@link RetryingFuture} via {@link #setExternalFuture(RetryingFuture)}.
+ * The {@link RetryingFuture} will be responsible for scheduling future attempts.
+ *
- The caller calls {@link #start()}. This notifies the outer {@link ResponseObserver} that
+ * call is about to start.
+ *
- The outer {@link ResponseObserver} configures inbound flow control via the {@link
+ * StreamController} that it received in {@link ResponseObserver#onStart(StreamController)}.
+ *
- The attempt call is sent via the inner/upstream {@link ServerStreamingCallable}.
+ *
- A future representing the end state of the inner attempt is passed to the outer {@link
+ * RetryingFuture}.
+ *
- All messages received from the inner {@link ServerStreamingCallable} are recorded by the
+ * {@link StreamResumptionStrategy}.
+ *
- All messages received from the inner {@link ServerStreamingCallable} are forwarded to the
+ * outer {@link ResponseObserver}.
+ *
- Upon attempt completion (either success or failure) are communicated to the outer {@link
+ * RetryingFuture}.
+ *
- If the {@link RetryingFuture} decides to resume the RPC, it will invoke {@link #call()},
+ * which will consult the {@link StreamResumptionStrategy} for the resuming request and
+ * restart the process at step 5.
+ *
- Once the {@link RetryingFuture} decides to stop the retry loop, it will notify the outer
+ * {@link ResponseObserver}.
+ *
+ *
+ * This class is meant to be used as middleware between an outer {@link ResponseObserver} and an
+ * inner {@link ServerStreamingCallable}. As such it follows the general threading model of {@link
+ * ServerStreamingCallable}s:
+ *
+ *
+ * - {@code onStart} must be called in the same thread that invoked {@code call()}
+ *
- The outer {@link ResponseObserver} can call {@code request()} and {@code cancel()} on this
+ * class' {@link StreamController} from any thread
+ *
- The inner callable will serialize calls to {@code onResponse()}, {@code onError()} and
+ * {@code onComplete}
+ *
+ *
+ * With this model in mind, this class only needs to synchronize access data that is shared
+ * between: the outer {@link ResponseObserver} (via this class' {@link StreamController}) and the
+ * inner {@link ServerStreamingCallable}: pendingRequests, cancellationCause and the current
+ * innerController.
+ *
+ * @param request type
+ * @param response type
+ */
+@InternalApi
+public final class ServerStreamingAttemptCallable implements Callable {
+ private final Object lock = new Object();
+
+ private final ServerStreamingCallable innerCallable;
+ private final StreamResumptionStrategy resumptionStrategy;
+ private final RequestT initialRequest;
+ private ApiCallContext context;
+ private final ResponseObserver outerObserver;
+
+ // Start state
+ private boolean autoFlowControl = true;
+ private boolean isStarted;
+
+ // Outer state
+ private Throwable cancellationCause;
+
+ private int pendingRequests;
+
+ private RetryingFuture outerRetryingFuture;
+
+ // Internal retry state
+ private int numAttempts;
+
+ private StreamController innerController;
+
+ private boolean seenSuccessSinceLastError;
+ private SettableApiFuture innerAttemptFuture;
+
+ public ServerStreamingAttemptCallable(
+ ServerStreamingCallable innerCallable,
+ StreamResumptionStrategy resumptionStrategy,
+ RequestT initialRequest,
+ ApiCallContext context,
+ ResponseObserver outerObserver) {
+ this.innerCallable = innerCallable;
+ this.resumptionStrategy = resumptionStrategy;
+ this.initialRequest = initialRequest;
+ this.context = context;
+ this.outerObserver = outerObserver;
+ }
+
+ /** Sets controlling {@link RetryingFuture}. Must be called be before {@link #start()}. */
+ void setExternalFuture(RetryingFuture retryingFuture) {
+ Preconditions.checkState(!isStarted, "Can't change the RetryingFuture once the call has start");
+ Preconditions.checkNotNull(retryingFuture, "RetryingFuture can't be null");
+
+ this.outerRetryingFuture = retryingFuture;
+ }
+
+ /**
+ * Starts the initial call. The call is attempted on the caller's thread. Further call attempts
+ * will be scheduled by the {@link RetryingFuture}.
+ */
+ public void start() {
+ Preconditions.checkState(!isStarted, "Already started");
+
+ // Initialize the outer observer
+ outerObserver.onStart(
+ new StreamController() {
+ @Override
+ public void disableAutoInboundFlowControl() {
+ Preconditions.checkState(
+ !isStarted, "Can't disable auto flow control once the stream is started");
+ autoFlowControl = false;
+ }
+
+ @Override
+ public void request(int count) {
+ onRequest(count);
+ }
+
+ @Override
+ public void cancel() {
+ onCancel();
+ }
+ });
+
+ if (autoFlowControl) {
+ synchronized (lock) {
+ pendingRequests = Integer.MAX_VALUE;
+ }
+ }
+ isStarted = true;
+
+ // Call the inner callable
+ call();
+ }
+
+ /**
+ * Sends the actual RPC. The request being sent will first be transformed by the {@link
+ * StreamResumptionStrategy}.
+ *
+ * This method expects to be called by one thread at a time. Furthermore, it expects that the
+ * current RPC finished before the next time it's called.
+ */
+ @Override
+ public Void call() {
+ Preconditions.checkState(isStarted, "Must be started first");
+
+ RequestT request =
+ (++numAttempts == 1) ? initialRequest : resumptionStrategy.getResumeRequest(initialRequest);
+
+ // Should never happen. onAttemptError will check if ResumptionStrategy can create a resume
+ // request,
+ // which the RetryingFuture/StreamResumptionStrategy should respect.
+ Preconditions.checkState(request != null, "ResumptionStrategy returned a null request.");
+
+ innerAttemptFuture = SettableApiFuture.create();
+ seenSuccessSinceLastError = false;
+
+ ApiCallContext attemptContext = context;
+
+ if (!outerRetryingFuture.getAttemptSettings().getRpcTimeout().isZero()
+ && attemptContext.getTimeout() == null) {
+ attemptContext =
+ attemptContext.withTimeout(outerRetryingFuture.getAttemptSettings().getRpcTimeout());
+ }
+
+ attemptContext
+ .getTracer()
+ .attemptStarted(request, outerRetryingFuture.getAttemptSettings().getOverallAttemptCount());
+
+ innerCallable.call(
+ request,
+ new StateCheckingResponseObserver() {
+ @Override
+ public void onStartImpl(StreamController controller) {
+ onAttemptStart(controller);
+ }
+
+ @Override
+ public void onResponseImpl(ResponseT response) {
+ onAttemptResponse(response);
+ }
+
+ @Override
+ public void onErrorImpl(Throwable t) {
+ onAttemptError(t);
+ }
+
+ @Override
+ public void onCompleteImpl() {
+ onAttemptComplete();
+ }
+ },
+ attemptContext);
+
+ outerRetryingFuture.setAttemptFuture(innerAttemptFuture);
+
+ return null;
+ }
+
+ /**
+ * Called by the inner {@link ServerStreamingCallable} when the call is about to start. This will
+ * transfer unfinished state from the previous attempt.
+ *
+ * @see ResponseObserver#onStart(StreamController)
+ */
+ private void onAttemptStart(StreamController controller) {
+ if (!autoFlowControl) {
+ controller.disableAutoInboundFlowControl();
+ }
+
+ Throwable localCancellationCause;
+ int numToRequest = 0;
+
+ synchronized (lock) {
+ innerController = controller;
+
+ localCancellationCause = this.cancellationCause;
+
+ if (!autoFlowControl) {
+ numToRequest = pendingRequests;
+ }
+ }
+
+ if (localCancellationCause != null) {
+ controller.cancel();
+ } else if (numToRequest > 0) {
+ controller.request(numToRequest);
+ }
+ }
+
+ /**
+ * Called when the outer {@link ResponseObserver} wants to prematurely cancel the stream.
+ *
+ * @see StreamController#cancel()
+ */
+ private void onCancel() {
+ StreamController localInnerController;
+
+ synchronized (lock) {
+ if (cancellationCause != null) {
+ return;
+ }
+ // NOTE: BasicRetryingFuture will replace j.u.c.CancellationExceptions with it's own,
+ // which will not have the current stacktrace, so a special wrapper has be used here.
+ cancellationCause =
+ new ServerStreamingAttemptException(
+ new CancellationException("User cancelled stream"),
+ resumptionStrategy.canResume(),
+ seenSuccessSinceLastError);
+ localInnerController = innerController;
+ }
+
+ if (localInnerController != null) {
+ localInnerController.cancel();
+ }
+ }
+
+ /**
+ * Called when the outer {@link ResponseObserver} is ready for more data.
+ *
+ * @see StreamController#request(int)
+ */
+ private void onRequest(int count) {
+ Preconditions.checkState(!autoFlowControl, "Automatic flow control is enabled");
+ Preconditions.checkArgument(count > 0, "Count must be > 0");
+
+ final StreamController localInnerController;
+
+ synchronized (lock) {
+ int maxInc = Integer.MAX_VALUE - pendingRequests;
+ count = Math.min(maxInc, count);
+
+ pendingRequests += count;
+ localInnerController = this.innerController;
+ }
+
+ // Note: there is a race condition here where the count might go to the previous attempt's
+ // StreamController after it failed. But it doesn't matter, because the controller will just
+ // ignore it and the current controller will pick it up onStart.
+ if (localInnerController != null) {
+ localInnerController.request(count);
+ }
+ }
+
+ /** Called when the inner callable has responses to deliver. */
+ private void onAttemptResponse(ResponseT message) {
+ if (!autoFlowControl) {
+ synchronized (lock) {
+ pendingRequests--;
+ }
+ }
+ // Update local state to allow for future resume.
+ seenSuccessSinceLastError = true;
+ message = resumptionStrategy.processResponse(message);
+ // Notify the outer observer.
+ outerObserver.onResponse(message);
+ }
+
+ /**
+ * Called when the current RPC fails. The error will be bubbled up to the outer {@link
+ * RetryingFuture} via the {@link #innerAttemptFuture}.
+ */
+ private void onAttemptError(Throwable throwable) {
+ Throwable localCancellationCause;
+ synchronized (lock) {
+ localCancellationCause = cancellationCause;
+ }
+
+ if (localCancellationCause != null) {
+ // Take special care to preserve the cancellation's stack trace.
+ innerAttemptFuture.setException(localCancellationCause);
+ } else {
+ // Wrap the original exception and provide more context for StreamingRetryAlgorithm.
+ innerAttemptFuture.setException(
+ new ServerStreamingAttemptException(
+ throwable, resumptionStrategy.canResume(), seenSuccessSinceLastError));
+ }
+ }
+
+ /**
+ * Called when the current RPC successfully completes. Notifies the outer {@link RetryingFuture}
+ * via {@link #innerAttemptFuture}.
+ */
+ private void onAttemptComplete() {
+ innerAttemptFuture.set(null);
+ }
+}
diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/functional/ReadRowsTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/functional/ReadRowsTest.java
new file mode 100644
index 0000000000..1a74eb5aa8
--- /dev/null
+++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/functional/ReadRowsTest.java
@@ -0,0 +1,110 @@
+/*
+ * Copyright 2024 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
+ *
+ * https://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.data.v2.functional;
+
+import com.google.api.gax.rpc.InternalException;
+import com.google.bigtable.v2.BigtableGrpc;
+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.cloud.bigtable.data.v2.FakeServiceBuilder;
+import com.google.cloud.bigtable.data.v2.models.Query;
+import com.google.cloud.bigtable.data.v2.models.Row;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.BytesValue;
+import com.google.protobuf.StringValue;
+import io.grpc.Server;
+import io.grpc.stub.StreamObserver;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class ReadRowsTest {
+ private FakeService service;
+ private Server server;
+
+ @Before
+ public void setUp() throws Exception {
+ service = new FakeService();
+ server = FakeServiceBuilder.create(service).start();
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ server.shutdown();
+ }
+
+ @Test
+ public void rowMergingErrorsUseInternalStatus() throws Exception {
+ BigtableDataSettings settings =
+ BigtableDataSettings.newBuilderForEmulator(server.getPort())
+ .setProjectId("fake-project")
+ .setInstanceId("fake-instance")
+ .build();
+
+ service.readRowsResponses.add(
+ ReadRowsResponse.newBuilder()
+ .addChunks(
+ ReadRowsResponse.CellChunk.newBuilder()
+ .setRowKey(ByteString.copyFromUtf8("z"))
+ .setFamilyName(StringValue.newBuilder().setValue("f"))
+ .setQualifier(
+ BytesValue.newBuilder().setValue(ByteString.copyFromUtf8("q")).build())
+ .setTimestampMicros(1000)
+ .setValue(ByteString.copyFromUtf8("v"))
+ .setCommitRow(true))
+ .addChunks(
+ ReadRowsResponse.CellChunk.newBuilder()
+ .setRowKey(ByteString.copyFromUtf8("a"))
+ .setFamilyName(StringValue.newBuilder().setValue("f"))
+ .setQualifier(
+ BytesValue.newBuilder().setValue(ByteString.copyFromUtf8("q")).build())
+ .setTimestampMicros(1000)
+ .setValue(ByteString.copyFromUtf8("v"))
+ .setCommitRow(true))
+ .build());
+
+ try (BigtableDataClient client = BigtableDataClient.create(settings)) {
+ Assert.assertThrows(
+ InternalException.class,
+ () -> {
+ for (Row ignored : client.readRows(Query.create("fake-table"))) {}
+ });
+ }
+ }
+
+ static class FakeService extends BigtableGrpc.BigtableImplBase {
+ private List readRowsResponses =
+ Collections.synchronizedList(new ArrayList<>());
+
+ @Override
+ public void readRows(
+ ReadRowsRequest request, StreamObserver responseObserver) {
+ for (ReadRowsResponse r : readRowsResponses) {
+ responseObserver.onNext(r);
+ }
+ responseObserver.onCompleted();
+ }
+ }
+}
diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/it/ReadIT.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/it/ReadIT.java
index 7b58e14f7c..6578dbad24 100644
--- a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/it/ReadIT.java
+++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/it/ReadIT.java
@@ -18,13 +18,17 @@
import static com.google.common.truth.Truth.assertThat;
import static com.google.common.truth.TruthJUnit.assume;
+import com.google.api.core.ApiFunction;
import com.google.api.core.ApiFuture;
import com.google.api.core.ApiFutureCallback;
import com.google.api.core.ApiFutures;
import com.google.api.core.SettableApiFuture;
+import com.google.api.gax.batching.Batcher;
+import com.google.api.gax.grpc.InstantiatingGrpcChannelProvider;
import com.google.api.gax.rpc.ResponseObserver;
import com.google.api.gax.rpc.StreamController;
import com.google.cloud.bigtable.data.v2.BigtableDataClient;
+import com.google.cloud.bigtable.data.v2.BigtableDataSettings;
import com.google.cloud.bigtable.data.v2.models.BulkMutation;
import com.google.cloud.bigtable.data.v2.models.Query;
import com.google.cloud.bigtable.data.v2.models.Range.ByteStringRange;
@@ -38,8 +42,17 @@
import com.google.common.collect.Lists;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.protobuf.ByteString;
+import io.grpc.CallOptions;
+import io.grpc.Channel;
+import io.grpc.ClientCall;
+import io.grpc.ClientInterceptor;
+import io.grpc.ManagedChannelBuilder;
+import io.grpc.MethodDescriptor;
+import java.io.IOException;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.List;
+import java.util.Random;
import java.util.UUID;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
@@ -48,6 +61,7 @@
import java.util.concurrent.atomic.AtomicReference;
import org.junit.Before;
import org.junit.ClassRule;
+import org.junit.Ignore;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;
@@ -308,6 +322,93 @@ public void reversed() {
.inOrder();
}
+ @Test
+ @Ignore("Test taking too long to run, ignore for now")
+ public void reversedWithForcedResumption() throws IOException, InterruptedException {
+ assume()
+ .withMessage("reverse scans are not supported in the emulator")
+ .that(testEnvRule.env())
+ .isNotInstanceOf(EmulatorEnv.class);
+
+ BigtableDataClient client = testEnvRule.env().getDataClient();
+ String tableId = testEnvRule.env().getTableId();
+ String familyId = testEnvRule.env().getFamilyId();
+ String uniqueKey = prefix + "-rev-queries2";
+
+ // Add enough rows that ensures resumption logic is forced
+ Random random;
+ List expectedResults;
+ try (Batcher batcher = client.newBulkMutationBatcher(tableId)) {
+
+ byte[] valueBytes = new byte[1024];
+ random = new Random();
+
+ expectedResults = new ArrayList<>();
+
+ for (int i = 0; i < 2 * 1024; i++) {
+ ByteString key = ByteString.copyFromUtf8(String.format("%s-%05d", uniqueKey, i));
+ ByteString qualifier = ByteString.copyFromUtf8("q");
+ long timestamp = System.currentTimeMillis() * 1000;
+ random.nextBytes(valueBytes);
+ ByteString value = ByteString.copyFrom(valueBytes);
+
+ batcher.add(RowMutationEntry.create(key).setCell(familyId, qualifier, timestamp, value));
+ expectedResults.add(
+ Row.create(
+ key,
+ ImmutableList.of(
+ RowCell.create(familyId, qualifier, timestamp, ImmutableList.of(), value))));
+ }
+ }
+ Collections.reverse(expectedResults);
+
+ BigtableDataSettings.Builder settingsBuilder =
+ testEnvRule.env().getDataClientSettings().toBuilder();
+
+ settingsBuilder.stubSettings().readRowsSettings().retrySettings().setMaxAttempts(100);
+
+ InstantiatingGrpcChannelProvider.Builder transport =
+ ((InstantiatingGrpcChannelProvider)
+ settingsBuilder.stubSettings().getTransportChannelProvider())
+ .toBuilder();
+ ApiFunction oldConfigurator =
+ transport.getChannelConfigurator();
+
+ // Randomly camp the deadline to force a timeout to force a retry
+ transport.setChannelConfigurator(
+ (ManagedChannelBuilder c) -> {
+ if (oldConfigurator != null) {
+ c = oldConfigurator.apply(c);
+ }
+ return c.intercept(
+ new ClientInterceptor() {
+ @Override
+ public ClientCall interceptCall(
+ MethodDescriptor method, CallOptions callOptions, Channel next) {
+ if (method.getBareMethodName().equals("ReadRows")) {
+ callOptions =
+ callOptions.withDeadlineAfter(random.nextInt(200), TimeUnit.MILLISECONDS);
+ }
+
+ return next.newCall(method, callOptions);
+ }
+ });
+ });
+ settingsBuilder.stubSettings().setTransportChannelProvider(transport.build());
+
+ try (BigtableDataClient patchedClient = BigtableDataClient.create(settingsBuilder.build())) {
+ for (int i = 0; i < 10; i++) {
+ List actualResults = new ArrayList<>();
+ for (Row row :
+ patchedClient.readRows(Query.create(tableId).prefix(uniqueKey).reversed(true))) {
+ actualResults.add(row);
+ Thread.sleep(1);
+ }
+ assertThat(actualResults).containsExactlyElementsIn(expectedResults).inOrder();
+ }
+ }
+ }
+
@Test
public void readSingleNonexistentAsyncCallback() throws Exception {
ApiFuture future =
diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/CookiesHolderTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/CookiesHolderTest.java
index 5dac053523..edf0b87fd9 100644
--- a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/CookiesHolderTest.java
+++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/CookiesHolderTest.java
@@ -37,10 +37,13 @@
import com.google.bigtable.v2.ReadRowsResponse;
import com.google.bigtable.v2.SampleRowKeysRequest;
import com.google.bigtable.v2.SampleRowKeysResponse;
+import com.google.bigtable.v2.StreamContinuationToken;
import com.google.cloud.bigtable.data.v2.BigtableDataClient;
+import com.google.cloud.bigtable.data.v2.BigtableDataClientFactory;
import com.google.cloud.bigtable.data.v2.BigtableDataSettings;
import com.google.cloud.bigtable.data.v2.FakeServiceBuilder;
import com.google.cloud.bigtable.data.v2.models.BulkMutation;
+import com.google.cloud.bigtable.data.v2.models.ChangeStreamRecord;
import com.google.cloud.bigtable.data.v2.models.ConditionalRowMutation;
import com.google.cloud.bigtable.data.v2.models.Mutation;
import com.google.cloud.bigtable.data.v2.models.Query;
@@ -58,6 +61,7 @@
import io.grpc.Status;
import io.grpc.StatusRuntimeException;
import io.grpc.stub.StreamObserver;
+import java.io.IOException;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
@@ -77,12 +81,18 @@ public class CookiesHolderTest {
Metadata.Key.of("x-goog-cbt-cookie-routing", Metadata.ASCII_STRING_MARSHALLER);
private static final Metadata.Key ROUTING_COOKIE_2 =
Metadata.Key.of("x-goog-cbt-cookie-random", Metadata.ASCII_STRING_MARSHALLER);
+ private static final Metadata.Key ROUTING_COOKIE_HEADER =
+ Metadata.Key.of("x-goog-cbt-cookie-header", Metadata.ASCII_STRING_MARSHALLER);
private static final Metadata.Key BAD_KEY =
Metadata.Key.of("x-goog-cbt-not-cookie", Metadata.ASCII_STRING_MARSHALLER);
+
+ private static final String testHeaderCookie = "header-cookie";
private static final String testCookie = "test-routing-cookie";
+ private static final String routingCookie1Header = "should-be-overridden";
private Server server;
private final FakeService fakeService = new FakeService();
+ private BigtableDataSettings.Builder settings;
private BigtableDataClient client;
private final List serverMetadata = new ArrayList<>();
@@ -101,7 +111,16 @@ public ServerCall.Listener interceptCall(
if (metadata.containsKey(ROUTING_COOKIE_1)) {
methods.add(serverCall.getMethodDescriptor().getBareMethodName());
}
- return serverCallHandler.startCall(serverCall, metadata);
+ return serverCallHandler.startCall(
+ new ForwardingServerCall.SimpleForwardingServerCall(serverCall) {
+ @Override
+ public void sendHeaders(Metadata responseHeaders) {
+ responseHeaders.put(ROUTING_COOKIE_HEADER, testHeaderCookie);
+ responseHeaders.put(ROUTING_COOKIE_1, routingCookie1Header);
+ super.sendHeaders(responseHeaders);
+ }
+ },
+ metadata);
}
};
@@ -138,6 +157,8 @@ public ServerCall.Listener interceptCall(
.build())
.setRetryableCodes(StatusCode.Code.UNAVAILABLE);
+ this.settings = settings;
+
client = BigtableDataClient.create(settings.build());
}
@@ -161,7 +182,13 @@ public void testReadRows() {
Metadata lastMetadata = serverMetadata.get(fakeService.count.get() - 1);
assertThat(lastMetadata)
- .containsAtLeast(ROUTING_COOKIE_1.name(), "readRows", ROUTING_COOKIE_2.name(), testCookie);
+ .containsAtLeast(
+ ROUTING_COOKIE_1.name(),
+ "readRows",
+ ROUTING_COOKIE_2.name(),
+ testCookie,
+ ROUTING_COOKIE_HEADER.name(),
+ testHeaderCookie);
assertThat(lastMetadata).doesNotContainKeys(BAD_KEY.name());
serverMetadata.clear();
@@ -177,7 +204,13 @@ public void testReadRow() {
Metadata lastMetadata = serverMetadata.get(fakeService.count.get() - 1);
assertThat(lastMetadata)
- .containsAtLeast(ROUTING_COOKIE_1.name(), "readRows", ROUTING_COOKIE_2.name(), testCookie);
+ .containsAtLeast(
+ ROUTING_COOKIE_1.name(),
+ "readRows",
+ ROUTING_COOKIE_2.name(),
+ testCookie,
+ ROUTING_COOKIE_HEADER.name(),
+ testHeaderCookie);
assertThat(lastMetadata).doesNotContainKeys(BAD_KEY.name());
serverMetadata.clear();
@@ -196,7 +229,12 @@ public void testMutateRows() {
assertThat(lastMetadata)
.containsAtLeast(
- ROUTING_COOKIE_1.name(), "mutateRows", ROUTING_COOKIE_2.name(), testCookie);
+ ROUTING_COOKIE_1.name(),
+ "mutateRows",
+ ROUTING_COOKIE_2.name(),
+ testCookie,
+ ROUTING_COOKIE_HEADER.name(),
+ testHeaderCookie);
assertThat(lastMetadata).doesNotContainKeys(BAD_KEY.name());
serverMetadata.clear();
@@ -212,7 +250,13 @@ public void testMutateRow() {
Metadata lastMetadata = serverMetadata.get(fakeService.count.get() - 1);
assertThat(lastMetadata)
- .containsAtLeast(ROUTING_COOKIE_1.name(), "mutateRow", ROUTING_COOKIE_2.name(), testCookie);
+ .containsAtLeast(
+ ROUTING_COOKIE_1.name(),
+ "mutateRow",
+ ROUTING_COOKIE_2.name(),
+ testCookie,
+ ROUTING_COOKIE_HEADER.name(),
+ testHeaderCookie);
assertThat(lastMetadata).doesNotContainKeys(BAD_KEY.name());
serverMetadata.clear();
@@ -230,7 +274,58 @@ public void testSampleRowKeys() {
assertThat(lastMetadata)
.containsAtLeast(
- ROUTING_COOKIE_1.name(), "sampleRowKeys", ROUTING_COOKIE_2.name(), testCookie);
+ ROUTING_COOKIE_1.name(),
+ "sampleRowKeys",
+ ROUTING_COOKIE_2.name(),
+ testCookie,
+ ROUTING_COOKIE_HEADER.name(),
+ testHeaderCookie);
+ assertThat(lastMetadata).doesNotContainKeys(BAD_KEY.name());
+
+ serverMetadata.clear();
+ }
+
+ @Test
+ public void testReadChangeStream() {
+ for (ChangeStreamRecord record :
+ client.readChangeStream(ReadChangeStreamQuery.create("table"))) {}
+
+ assertThat(fakeService.count.get()).isGreaterThan(1);
+ assertThat(serverMetadata).hasSize(fakeService.count.get());
+
+ Metadata lastMetadata = serverMetadata.get(fakeService.count.get() - 1);
+
+ assertThat(lastMetadata)
+ .containsAtLeast(
+ ROUTING_COOKIE_1.name(),
+ "readChangeStream",
+ ROUTING_COOKIE_2.name(),
+ testCookie,
+ ROUTING_COOKIE_HEADER.name(),
+ testHeaderCookie);
+ assertThat(lastMetadata).doesNotContainKeys(BAD_KEY.name());
+
+ serverMetadata.clear();
+ }
+
+ @Test
+ public void testGenerateInitialChangeStreamPartition() {
+ client.generateInitialChangeStreamPartitions("table").iterator().hasNext();
+
+ assertThat(fakeService.count.get()).isGreaterThan(1);
+ assertThat(serverMetadata).hasSize(fakeService.count.get());
+
+ Metadata lastMetadata = serverMetadata.get(fakeService.count.get() - 1);
+
+ // generateInitialChangeStreamPartition uses SimpleStreamResumptionStrategy which means
+ // it can't resume from the middle of the stream. So we are not able to send a header
+ // for error responses.
+ assertThat(lastMetadata)
+ .containsAtLeast(
+ ROUTING_COOKIE_1.name(),
+ "generateInitialChangeStreamPartitions",
+ ROUTING_COOKIE_2.name(),
+ testCookie);
assertThat(lastMetadata).doesNotContainKeys(BAD_KEY.name());
serverMetadata.clear();
@@ -247,7 +342,9 @@ public void testNoCookieSucceedReadRows() {
Metadata lastMetadata = serverMetadata.get(fakeService.count.get() - 1);
- assertThat(lastMetadata).doesNotContainKeys(ROUTING_COOKIE_1.name(), ROUTING_COOKIE_2.name());
+ assertThat(lastMetadata).doesNotContainKeys(ROUTING_COOKIE_2.name());
+ // Should contain initial metadata
+ assertThat(lastMetadata).containsAtLeast(ROUTING_COOKIE_1.name(), routingCookie1Header);
serverMetadata.clear();
}
@@ -263,8 +360,8 @@ public void testNoCookieSucceedReadRow() {
Metadata lastMetadata = serverMetadata.get(fakeService.count.get() - 1);
- assertThat(lastMetadata)
- .doesNotContainKeys(ROUTING_COOKIE_1.name(), ROUTING_COOKIE_2.name(), BAD_KEY.name());
+ assertThat(lastMetadata).doesNotContainKeys(ROUTING_COOKIE_2.name(), BAD_KEY.name());
+ assertThat(lastMetadata).containsAtLeast(ROUTING_COOKIE_1.name(), routingCookie1Header);
serverMetadata.clear();
}
@@ -282,8 +379,8 @@ public void testNoCookieSucceedMutateRows() {
Metadata lastMetadata = serverMetadata.get(fakeService.count.get() - 1);
- assertThat(lastMetadata)
- .doesNotContainKeys(ROUTING_COOKIE_1.name(), ROUTING_COOKIE_2.name(), BAD_KEY.name());
+ assertThat(lastMetadata).doesNotContainKeys(ROUTING_COOKIE_2.name(), BAD_KEY.name());
+ assertThat(lastMetadata).containsAtLeast(ROUTING_COOKIE_1.name(), routingCookie1Header);
serverMetadata.clear();
}
@@ -299,8 +396,8 @@ public void testNoCookieSucceedMutateRow() {
Metadata lastMetadata = serverMetadata.get(fakeService.count.get() - 1);
- assertThat(lastMetadata)
- .doesNotContainKeys(ROUTING_COOKIE_1.name(), ROUTING_COOKIE_2.name(), BAD_KEY.name());
+ assertThat(lastMetadata).doesNotContainKeys(ROUTING_COOKIE_2.name(), BAD_KEY.name());
+ assertThat(lastMetadata).containsAtLeast(ROUTING_COOKIE_1.name(), routingCookie1Header);
serverMetadata.clear();
}
@@ -316,6 +413,43 @@ public void testNoCookieSucceedSampleRowKeys() {
Metadata lastMetadata = serverMetadata.get(fakeService.count.get() - 1);
+ assertThat(lastMetadata).doesNotContainKeys(ROUTING_COOKIE_2.name(), BAD_KEY.name());
+ assertThat(lastMetadata).containsAtLeast(ROUTING_COOKIE_1.name(), routingCookie1Header);
+
+ serverMetadata.clear();
+ }
+
+ @Test
+ public void testNoCookieSucceedReadChangeStream() {
+ fakeService.returnCookie = false;
+
+ for (ChangeStreamRecord record :
+ client.readChangeStream(ReadChangeStreamQuery.create("table"))) {}
+
+ assertThat(fakeService.count.get()).isGreaterThan(1);
+ assertThat(serverMetadata).hasSize(fakeService.count.get());
+
+ Metadata lastMetadata = serverMetadata.get(fakeService.count.get() - 1);
+
+ assertThat(lastMetadata).doesNotContainKeys(ROUTING_COOKIE_2.name(), BAD_KEY.name());
+ assertThat(lastMetadata).containsAtLeast(ROUTING_COOKIE_1.name(), routingCookie1Header);
+
+ serverMetadata.clear();
+
+ serverMetadata.clear();
+ }
+
+ @Test
+ public void testNoCookieSucceedGenerateInitialChangeStreamParition() {
+ fakeService.returnCookie = false;
+
+ client.generateInitialChangeStreamPartitions("table").iterator().hasNext();
+
+ assertThat(fakeService.count.get()).isGreaterThan(1);
+ assertThat(serverMetadata).hasSize(fakeService.count.get());
+
+ Metadata lastMetadata = serverMetadata.get(fakeService.count.get() - 1);
+
assertThat(lastMetadata)
.doesNotContainKeys(ROUTING_COOKIE_1.name(), ROUTING_COOKIE_2.name(), BAD_KEY.name());
@@ -379,7 +513,7 @@ public void sendHeaders(Metadata headers) {
}
@Test
- public void testAllMethodsAreCalled() throws InterruptedException {
+ public void testAllMethodsAreCalled() {
// This test ensures that all methods respect the retry cookie except for the ones that are
// explicitly added to the methods list. It requires that any newly method is exercised in this
// test. This is enforced by introspecting grpc method descriptors.
@@ -409,7 +543,8 @@ public void testAllMethodsAreCalled() throws InterruptedException {
client.generateInitialChangeStreamPartitions("fake-table").iterator().hasNext();
fakeService.count.set(0);
- client.readChangeStream(ReadChangeStreamQuery.create("fake-table")).iterator().hasNext();
+ for (ChangeStreamRecord record :
+ client.readChangeStream(ReadChangeStreamQuery.create("fake-table"))) {}
Set expected =
BigtableGrpc.getServiceDescriptor().getMethods().stream()
@@ -422,6 +557,103 @@ public void testAllMethodsAreCalled() throws InterruptedException {
assertThat(methods).containsExactlyElementsIn(expected);
}
+ @Test
+ public void testCookieSetWithBigtableClientFactory() throws Exception {
+ try (BigtableDataClientFactory factory = BigtableDataClientFactory.create(settings.build())) {
+ BigtableDataClient client1 = factory.createDefault();
+ BigtableDataClient client2 = factory.createForAppProfile("app-profile");
+
+ client1.readRows(Query.create("fake-table")).iterator().hasNext();
+
+ assertThat(fakeService.count.get()).isGreaterThan(1);
+ assertThat(serverMetadata).hasSize(fakeService.count.get());
+
+ Metadata lastMetadata = serverMetadata.get(fakeService.count.get() - 1);
+
+ assertThat(lastMetadata)
+ .containsAtLeast(
+ ROUTING_COOKIE_1.name(),
+ "readRows",
+ ROUTING_COOKIE_2.name(),
+ testCookie,
+ ROUTING_COOKIE_HEADER.name(),
+ testHeaderCookie);
+ assertThat(lastMetadata).doesNotContainKeys(BAD_KEY.name());
+
+ // Reset fake service status
+ fakeService.count.set(0);
+ serverMetadata.clear();
+
+ client2.readRows(Query.create("fake-table")).iterator().hasNext();
+
+ assertThat(fakeService.count.get()).isGreaterThan(1);
+ assertThat(serverMetadata).hasSize(fakeService.count.get());
+
+ lastMetadata = serverMetadata.get(fakeService.count.get() - 1);
+
+ assertThat(lastMetadata)
+ .containsAtLeast(
+ ROUTING_COOKIE_1.name(),
+ "readRows",
+ ROUTING_COOKIE_2.name(),
+ testCookie,
+ ROUTING_COOKIE_HEADER.name(),
+ testHeaderCookie);
+ assertThat(lastMetadata).doesNotContainKeys(BAD_KEY.name());
+
+ serverMetadata.clear();
+ }
+ }
+
+ @Test
+ public void testDisableRoutingCookie() throws IOException {
+ // This test disables routing cookie in the client settings and ensures that none of the routing
+ // cookie
+ // is added.
+ settings.stubSettings().setEnableRoutingCookie(false);
+ try (BigtableDataClient client = BigtableDataClient.create(settings.build())) {
+ client.readRows(Query.create("fake-table")).iterator().hasNext();
+ assertThat(fakeService.count.get()).isEqualTo(2);
+ fakeService.count.set(0);
+
+ client.mutateRow(RowMutation.create("fake-table", "key").setCell("cf", "q", "v"));
+ assertThat(fakeService.count.get()).isEqualTo(2);
+ fakeService.count.set(0);
+
+ client.bulkMutateRows(
+ BulkMutation.create("fake-table")
+ .add(RowMutationEntry.create("key").setCell("cf", "q", "v")));
+ assertThat(fakeService.count.get()).isEqualTo(2);
+ fakeService.count.set(0);
+
+ client.sampleRowKeys("fake-table");
+ assertThat(fakeService.count.get()).isEqualTo(2);
+ fakeService.count.set(0);
+
+ client.checkAndMutateRow(
+ ConditionalRowMutation.create("fake-table", "key")
+ .then(Mutation.create().setCell("cf", "q", "v")));
+ assertThat(fakeService.count.get()).isEqualTo(2);
+ fakeService.count.set(0);
+
+ client.readModifyWriteRow(
+ ReadModifyWriteRow.create("fake-table", "key").append("cf", "q", "v"));
+ assertThat(fakeService.count.get()).isEqualTo(2);
+ fakeService.count.set(0);
+
+ client.generateInitialChangeStreamPartitions("fake-table").iterator().hasNext();
+ assertThat(fakeService.count.get()).isEqualTo(2);
+ fakeService.count.set(0);
+
+ for (ChangeStreamRecord record :
+ client.readChangeStream(ReadChangeStreamQuery.create("fake-table"))) {}
+
+ assertThat(fakeService.count.get()).isEqualTo(2);
+
+ assertThat(methods).isEmpty();
+ }
+ }
+
static class FakeService extends BigtableGrpc.BigtableImplBase {
private boolean returnCookie = true;
@@ -448,6 +680,7 @@ public void mutateRow(
if (count.getAndIncrement() < 1) {
Metadata trailers = new Metadata();
maybePopulateCookie(trailers, "mutateRow");
+ responseObserver.onNext(MutateRowResponse.getDefaultInstance());
StatusRuntimeException exception = new StatusRuntimeException(Status.UNAVAILABLE, trailers);
responseObserver.onError(exception);
return;
@@ -462,6 +695,7 @@ public void mutateRows(
if (count.getAndIncrement() < 1) {
Metadata trailers = new Metadata();
maybePopulateCookie(trailers, "mutateRows");
+ responseObserver.onNext(MutateRowsResponse.getDefaultInstance());
StatusRuntimeException exception = new StatusRuntimeException(Status.UNAVAILABLE, trailers);
responseObserver.onError(exception);
return;
@@ -479,6 +713,7 @@ public void sampleRowKeys(
if (count.getAndIncrement() < 1) {
Metadata trailers = new Metadata();
maybePopulateCookie(trailers, "sampleRowKeys");
+ responseObserver.onNext(SampleRowKeysResponse.getDefaultInstance());
StatusRuntimeException exception = new StatusRuntimeException(Status.UNAVAILABLE, trailers);
responseObserver.onError(exception);
return;
@@ -524,6 +759,14 @@ public void readChangeStream(
if (count.getAndIncrement() < 1) {
Metadata trailers = new Metadata();
maybePopulateCookie(trailers, "readChangeStream");
+ responseObserver.onNext(
+ ReadChangeStreamResponse.newBuilder()
+ .setHeartbeat(
+ ReadChangeStreamResponse.Heartbeat.newBuilder()
+ .setContinuationToken(
+ StreamContinuationToken.newBuilder().setToken("a").build())
+ .build())
+ .build());
StatusRuntimeException exception = new StatusRuntimeException(Status.UNAVAILABLE, trailers);
responseObserver.onError(exception);
return;
diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStubSettingsTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStubSettingsTest.java
index fbd6442e0c..a57d42f6f1 100644
--- a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStubSettingsTest.java
+++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStubSettingsTest.java
@@ -20,7 +20,6 @@
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.grpc.InstantiatingGrpcChannelProvider;
import com.google.api.gax.retrying.RetrySettings;
import com.google.api.gax.rpc.ServerStreamingCallSettings;
@@ -77,6 +76,8 @@ public void settingsAreNotLostTest() {
CredentialsProvider credentialsProvider = Mockito.mock(CredentialsProvider.class);
WatchdogProvider watchdogProvider = Mockito.mock(WatchdogProvider.class);
Duration watchdogInterval = Duration.ofSeconds(12);
+ boolean enableRoutingCookie = false;
+ boolean enableRetryInfo = false;
EnhancedBigtableStubSettings.Builder builder =
EnhancedBigtableStubSettings.newBuilder()
@@ -87,7 +88,9 @@ public void settingsAreNotLostTest() {
.setEndpoint(endpoint)
.setCredentialsProvider(credentialsProvider)
.setStreamWatchdogProvider(watchdogProvider)
- .setStreamWatchdogCheckInterval(watchdogInterval);
+ .setStreamWatchdogCheckInterval(watchdogInterval)
+ .setEnableRoutingCookie(enableRoutingCookie)
+ .setEnableRetryInfo(enableRetryInfo);
verifyBuilder(
builder,
@@ -98,7 +101,9 @@ public void settingsAreNotLostTest() {
endpoint,
credentialsProvider,
watchdogProvider,
- watchdogInterval);
+ watchdogInterval,
+ enableRoutingCookie,
+ enableRetryInfo);
verifySettings(
builder.build(),
projectId,
@@ -108,7 +113,9 @@ public void settingsAreNotLostTest() {
endpoint,
credentialsProvider,
watchdogProvider,
- watchdogInterval);
+ watchdogInterval,
+ enableRoutingCookie,
+ enableRetryInfo);
verifyBuilder(
builder.build().toBuilder(),
projectId,
@@ -118,7 +125,9 @@ public void settingsAreNotLostTest() {
endpoint,
credentialsProvider,
watchdogProvider,
- watchdogInterval);
+ watchdogInterval,
+ enableRoutingCookie,
+ enableRetryInfo);
}
private void verifyBuilder(
@@ -130,7 +139,9 @@ private void verifyBuilder(
String endpoint,
CredentialsProvider credentialsProvider,
WatchdogProvider watchdogProvider,
- Duration watchdogInterval) {
+ Duration watchdogInterval,
+ boolean enableRoutingCookie,
+ boolean enableRetryInfo) {
assertThat(builder.getProjectId()).isEqualTo(projectId);
assertThat(builder.getInstanceId()).isEqualTo(instanceId);
assertThat(builder.getAppProfileId()).isEqualTo(appProfileId);
@@ -139,6 +150,8 @@ private void verifyBuilder(
assertThat(builder.getCredentialsProvider()).isEqualTo(credentialsProvider);
assertThat(builder.getStreamWatchdogProvider()).isSameInstanceAs(watchdogProvider);
assertThat(builder.getStreamWatchdogCheckInterval()).isEqualTo(watchdogInterval);
+ assertThat(builder.getEnableRoutingCookie()).isEqualTo(enableRoutingCookie);
+ assertThat(builder.getEnableRetryInfo()).isEqualTo(enableRetryInfo);
}
private void verifySettings(
@@ -150,7 +163,9 @@ private void verifySettings(
String endpoint,
CredentialsProvider credentialsProvider,
WatchdogProvider watchdogProvider,
- Duration watchdogInterval) {
+ Duration watchdogInterval,
+ boolean enableRoutingCookie,
+ boolean enableRetryInfo) {
assertThat(settings.getProjectId()).isEqualTo(projectId);
assertThat(settings.getInstanceId()).isEqualTo(instanceId);
assertThat(settings.getAppProfileId()).isEqualTo(appProfileId);
@@ -159,6 +174,8 @@ private void verifySettings(
assertThat(settings.getCredentialsProvider()).isEqualTo(credentialsProvider);
assertThat(settings.getStreamWatchdogProvider()).isSameInstanceAs(watchdogProvider);
assertThat(settings.getStreamWatchdogCheckInterval()).isEqualTo(watchdogInterval);
+ assertThat(settings.getEnableRoutingCookie()).isEqualTo(enableRoutingCookie);
+ assertThat(settings.getEnableRetryInfo()).isEqualTo(enableRetryInfo);
}
@Test
@@ -781,6 +798,72 @@ public void isRefreshingChannelFalseValueTest() {
assertThat(builder.build().toBuilder().isRefreshingChannel()).isFalse();
}
+ @Test
+ public void routingCookieIsEnabled() throws IOException {
+ String dummyProjectId = "my-project";
+ String dummyInstanceId = "my-instance";
+ CredentialsProvider credentialsProvider = Mockito.mock(CredentialsProvider.class);
+ Mockito.when(credentialsProvider.getCredentials()).thenReturn(new FakeCredentials());
+ EnhancedBigtableStubSettings.Builder builder =
+ EnhancedBigtableStubSettings.newBuilder()
+ .setProjectId(dummyProjectId)
+ .setInstanceId(dummyInstanceId)
+ .setCredentialsProvider(credentialsProvider);
+
+ assertThat(builder.getEnableRoutingCookie()).isTrue();
+ assertThat(builder.build().getEnableRoutingCookie()).isTrue();
+ assertThat(builder.build().toBuilder().getEnableRoutingCookie()).isTrue();
+ }
+
+ public void enableRetryInfoDefaultValueTest() throws IOException {
+ String dummyProjectId = "my-project";
+ String dummyInstanceId = "my-instance";
+ CredentialsProvider credentialsProvider = Mockito.mock(CredentialsProvider.class);
+ Mockito.when(credentialsProvider.getCredentials()).thenReturn(new FakeCredentials());
+ EnhancedBigtableStubSettings.Builder builder =
+ EnhancedBigtableStubSettings.newBuilder()
+ .setProjectId(dummyProjectId)
+ .setInstanceId(dummyInstanceId)
+ .setCredentialsProvider(credentialsProvider);
+ assertThat(builder.getEnableRetryInfo()).isTrue();
+ assertThat(builder.build().getEnableRetryInfo()).isTrue();
+ assertThat(builder.build().toBuilder().getEnableRetryInfo()).isTrue();
+ }
+
+ @Test
+ public void routingCookieFalseValueSet() throws IOException {
+ String dummyProjectId = "my-project";
+ String dummyInstanceId = "my-instance";
+ CredentialsProvider credentialsProvider = Mockito.mock(CredentialsProvider.class);
+ Mockito.when(credentialsProvider.getCredentials()).thenReturn(new FakeCredentials());
+ EnhancedBigtableStubSettings.Builder builder =
+ EnhancedBigtableStubSettings.newBuilder()
+ .setProjectId(dummyProjectId)
+ .setInstanceId(dummyInstanceId)
+ .setEnableRoutingCookie(false)
+ .setCredentialsProvider(credentialsProvider);
+ assertThat(builder.getEnableRoutingCookie()).isFalse();
+ assertThat(builder.build().getEnableRoutingCookie()).isFalse();
+ assertThat(builder.build().toBuilder().getEnableRoutingCookie()).isFalse();
+ }
+
+ @Test
+ public void enableRetryInfoFalseValueTest() throws IOException {
+ String dummyProjectId = "my-project";
+ String dummyInstanceId = "my-instance";
+ CredentialsProvider credentialsProvider = Mockito.mock(CredentialsProvider.class);
+ Mockito.when(credentialsProvider.getCredentials()).thenReturn(new FakeCredentials());
+ EnhancedBigtableStubSettings.Builder builder =
+ EnhancedBigtableStubSettings.newBuilder()
+ .setProjectId(dummyProjectId)
+ .setInstanceId(dummyInstanceId)
+ .setEnableRetryInfo(false)
+ .setCredentialsProvider(credentialsProvider);
+ assertThat(builder.getEnableRetryInfo()).isFalse();
+ assertThat(builder.build().getEnableRetryInfo()).isFalse();
+ assertThat(builder.build().toBuilder().getEnableRetryInfo()).isFalse();
+ }
+
static final String[] SETTINGS_LIST = {
"projectId",
"instanceId",
@@ -788,6 +871,8 @@ public void isRefreshingChannelFalseValueTest() {
"isRefreshingChannel",
"primedTableIds",
"jwtAudienceMapping",
+ "enableRoutingCookie",
+ "enableRetryInfo",
"readRowsSettings",
"readRowSettings",
"sampleRowKeysSettings",
@@ -872,16 +957,6 @@ public void refreshingChannelSetFixedCredentialProvider() throws Exception {
.setRefreshingChannel(true)
.setCredentialsProvider(credentialsProvider);
assertThat(builder.isRefreshingChannel()).isTrue();
- // Verify that isRefreshing setting is not lost and stubSettings will always return the same
- // credential
- EnhancedBigtableStubSettings stubSettings = builder.build();
- assertThat(stubSettings.isRefreshingChannel()).isTrue();
- assertThat(stubSettings.getCredentialsProvider()).isInstanceOf(FixedCredentialsProvider.class);
- assertThat(stubSettings.getCredentialsProvider().getCredentials())
- .isEqualTo(expectedCredentials);
- assertThat(stubSettings.toBuilder().isRefreshingChannel()).isTrue();
- assertThat(stubSettings.toBuilder().getCredentialsProvider().getCredentials())
- .isEqualTo(expectedCredentials);
}
private static class FakeCredentials extends Credentials {
diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStubTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStubTest.java
index e36eb1a8a9..eacf145bcb 100644
--- a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStubTest.java
+++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStubTest.java
@@ -549,6 +549,7 @@ public void testBulkMutationFlowControlFeatureFlagIsNotSet() throws Exception {
FeatureFlags featureFlags = FeatureFlags.parseFrom(decodedFlags);
assertThat(featureFlags.getMutateRowsRateLimit()).isFalse();
assertThat(featureFlags.getMutateRowsRateLimit2()).isFalse();
+ stub.close();
}
@Test
diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/RetryInfoTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/RetryInfoTest.java
new file mode 100644
index 0000000000..ba61ee5350
--- /dev/null
+++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/RetryInfoTest.java
@@ -0,0 +1,767 @@
+/*
+ * Copyright 2023 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
+ *
+ * https://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.data.v2.stub;
+
+import static com.google.common.truth.Truth.assertThat;
+import static org.junit.Assert.assertThrows;
+
+import com.google.api.gax.core.NoCredentialsProvider;
+import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.grpc.GrpcTransportChannel;
+import com.google.api.gax.rpc.ApiException;
+import com.google.api.gax.rpc.ErrorDetails;
+import com.google.api.gax.rpc.FixedTransportChannelProvider;
+import com.google.api.gax.rpc.InternalException;
+import com.google.api.gax.rpc.UnavailableException;
+import com.google.bigtable.v2.BigtableGrpc;
+import com.google.bigtable.v2.CheckAndMutateRowRequest;
+import com.google.bigtable.v2.CheckAndMutateRowResponse;
+import com.google.bigtable.v2.GenerateInitialChangeStreamPartitionsRequest;
+import com.google.bigtable.v2.GenerateInitialChangeStreamPartitionsResponse;
+import com.google.bigtable.v2.MutateRowRequest;
+import com.google.bigtable.v2.MutateRowResponse;
+import com.google.bigtable.v2.MutateRowsRequest;
+import com.google.bigtable.v2.MutateRowsResponse;
+import com.google.bigtable.v2.ReadChangeStreamRequest;
+import com.google.bigtable.v2.ReadChangeStreamResponse;
+import com.google.bigtable.v2.ReadModifyWriteRowRequest;
+import com.google.bigtable.v2.ReadModifyWriteRowResponse;
+import com.google.bigtable.v2.ReadRowsRequest;
+import com.google.bigtable.v2.ReadRowsResponse;
+import com.google.bigtable.v2.SampleRowKeysRequest;
+import com.google.bigtable.v2.SampleRowKeysResponse;
+import com.google.cloud.bigtable.data.v2.BigtableDataClient;
+import com.google.cloud.bigtable.data.v2.BigtableDataSettings;
+import com.google.cloud.bigtable.data.v2.models.BulkMutation;
+import com.google.cloud.bigtable.data.v2.models.ConditionalRowMutation;
+import com.google.cloud.bigtable.data.v2.models.Filters;
+import com.google.cloud.bigtable.data.v2.models.MutateRowsException;
+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.ReadChangeStreamQuery;
+import com.google.cloud.bigtable.data.v2.models.ReadModifyWriteRow;
+import com.google.cloud.bigtable.data.v2.models.RowMutation;
+import com.google.cloud.bigtable.data.v2.models.RowMutationEntry;
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Queues;
+import com.google.protobuf.Any;
+import com.google.rpc.RetryInfo;
+import io.grpc.Metadata;
+import io.grpc.Status;
+import io.grpc.StatusRuntimeException;
+import io.grpc.stub.StreamObserver;
+import io.grpc.testing.GrpcServerRule;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Queue;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class RetryInfoTest {
+
+ @Rule public GrpcServerRule serverRule = new GrpcServerRule();
+
+ private static final Metadata.Key ERROR_DETAILS_KEY =
+ Metadata.Key.of("grpc-status-details-bin", Metadata.BINARY_BYTE_MARSHALLER);
+
+ private FakeBigtableService service;
+ private BigtableDataClient client;
+ private BigtableDataSettings.Builder settings;
+
+ private AtomicInteger attemptCounter = new AtomicInteger();
+ private com.google.protobuf.Duration delay =
+ com.google.protobuf.Duration.newBuilder().setSeconds(2).setNanos(0).build();
+
+ @Before
+ public void setUp() throws IOException {
+ service = new FakeBigtableService();
+ serverRule.getServiceRegistry().addService(service);
+
+ settings =
+ BigtableDataSettings.newBuilder()
+ .setProjectId("fake-project")
+ .setInstanceId("fake-instance")
+ .setCredentialsProvider(NoCredentialsProvider.create());
+
+ settings
+ .stubSettings()
+ .setTransportChannelProvider(
+ FixedTransportChannelProvider.create(
+ GrpcTransportChannel.create(serverRule.getChannel())))
+ // channel priming doesn't work with FixedTransportChannelProvider. Disable it for the test
+ .setRefreshingChannel(false)
+ .build();
+
+ this.client = BigtableDataClient.create(settings.build());
+ }
+
+ @Test
+ public void testReadRow() {
+ verifyRetryInfoIsUsed(() -> client.readRow("table", "row"), true);
+ }
+
+ @Test
+ public void testReadRowNonRetryableErrorWithRetryInfo() {
+ verifyRetryInfoIsUsed(() -> client.readRow("table", "row"), false);
+ }
+
+ @Test
+ public void testReadRowDisableRetryInfo() throws IOException {
+ settings.stubSettings().setEnableRetryInfo(false);
+
+ try (BigtableDataClient newClient = BigtableDataClient.create(settings.build())) {
+ verifyRetryInfoCanBeDisabled(() -> newClient.readRow("table", "row"));
+ }
+ }
+
+ @Test
+ public void testReadRowServerNotReturningRetryInfo() {
+ verifyNoRetryInfo(() -> client.readRow("table", "row"), true);
+ }
+
+ @Test
+ public void testReadRowServerNotReturningRetryInfoClientDisabledHandling() throws IOException {
+ settings.stubSettings().setEnableRetryInfo(false);
+
+ try (BigtableDataClient newClient = BigtableDataClient.create(settings.build())) {
+ verifyNoRetryInfo(() -> newClient.readRow("table", "row"), true);
+ }
+ }
+
+ @Test
+ public void testReadRows() {
+ verifyRetryInfoIsUsed(() -> client.readRows(Query.create("table")).iterator().hasNext(), true);
+ }
+
+ @Test
+ public void testReadRowsNonRetraybleErrorWithRetryInfo() {
+ verifyRetryInfoIsUsed(() -> client.readRows(Query.create("table")).iterator().hasNext(), false);
+ }
+
+ @Test
+ public void testReadRowsDisableRetryInfo() throws IOException {
+ settings.stubSettings().setEnableRetryInfo(false);
+
+ try (BigtableDataClient newClient = BigtableDataClient.create(settings.build())) {
+ verifyRetryInfoCanBeDisabled(
+ () -> newClient.readRows(Query.create("table")).iterator().hasNext());
+ }
+ }
+
+ @Test
+ public void testReadRowsServerNotReturningRetryInfo() {
+ verifyNoRetryInfo(() -> client.readRows(Query.create("table")).iterator().hasNext(), true);
+ }
+
+ @Test
+ public void testReadRowsServerNotReturningRetryInfoClientDisabledHandling() throws IOException {
+ settings.stubSettings().setEnableRetryInfo(false);
+
+ try (BigtableDataClient newClient = BigtableDataClient.create(settings.build())) {
+ verifyNoRetryInfo(() -> newClient.readRows(Query.create("table")).iterator().hasNext(), true);
+ }
+ }
+
+ @Test
+ public void testMutateRows() {
+ verifyRetryInfoIsUsed(
+ () ->
+ client.bulkMutateRows(
+ BulkMutation.create("fake-table")
+ .add(RowMutationEntry.create("row-key-1").setCell("cf", "q", "v"))),
+ true);
+ }
+
+ @Test
+ public void testMutateRowsNonRetryableErrorWithRetryInfo() {
+ verifyRetryInfoIsUsed(
+ () ->
+ client.bulkMutateRows(
+ BulkMutation.create("fake-table")
+ .add(RowMutationEntry.create("row-key-1").setCell("cf", "q", "v"))),
+ false);
+ }
+
+ @Test
+ public void testMutateRowsDisableRetryInfo() throws IOException {
+ settings.stubSettings().setEnableRetryInfo(false);
+
+ try (BigtableDataClient newClient = BigtableDataClient.create(settings.build())) {
+ verifyRetryInfoCanBeDisabled(
+ () ->
+ newClient.bulkMutateRows(
+ BulkMutation.create("fake-table")
+ .add(RowMutationEntry.create("row-key-1").setCell("cf", "q", "v"))));
+ }
+ }
+
+ @Test
+ public void testMutateRowsServerNotReturningRetryInfo() {
+ verifyNoRetryInfo(
+ () ->
+ client.bulkMutateRows(
+ BulkMutation.create("fake-table")
+ .add(RowMutationEntry.create("row-key-1").setCell("cf", "q", "v"))),
+ true);
+ }
+
+ @Test
+ public void testMutateRowsServerNotReturningRetryInfoClientDisabledHandling() throws IOException {
+ settings.stubSettings().setEnableRetryInfo(false);
+
+ try (BigtableDataClient newClient = BigtableDataClient.create(settings.build())) {
+ verifyNoRetryInfo(
+ () ->
+ newClient.bulkMutateRows(
+ BulkMutation.create("fake-table")
+ .add(RowMutationEntry.create("row-key-1").setCell("cf", "q", "v"))),
+ true);
+ }
+ }
+
+ @Test
+ public void testMutateRow() {
+ verifyRetryInfoIsUsed(
+ () -> client.mutateRow(RowMutation.create("table", "key").setCell("cf", "q", "v")), true);
+ }
+
+ @Test
+ public void testMutateRowNonRetryableErrorWithRetryInfo() {
+ verifyRetryInfoIsUsed(
+ () -> client.mutateRow(RowMutation.create("table", "key").setCell("cf", "q", "v")), false);
+ }
+
+ @Test
+ public void testMutateRowDisableRetryInfo() throws IOException {
+ settings.stubSettings().setEnableRetryInfo(false);
+
+ try (BigtableDataClient newClient = BigtableDataClient.create(settings.build())) {
+
+ verifyRetryInfoCanBeDisabled(
+ () -> newClient.mutateRow(RowMutation.create("table", "key").setCell("cf", "q", "v")));
+ }
+ }
+
+ @Test
+ public void testMutateRowServerNotReturningRetryInfo() {
+ verifyNoRetryInfo(
+ () -> client.mutateRow(RowMutation.create("table", "key").setCell("cf", "q", "v")), true);
+ }
+
+ @Test
+ public void testMutateRowServerNotReturningRetryInfoClientDisabledHandling() throws IOException {
+ settings.stubSettings().setEnableRetryInfo(false);
+
+ try (BigtableDataClient newClient = BigtableDataClient.create(settings.build())) {
+ verifyNoRetryInfo(
+ () -> newClient.mutateRow(RowMutation.create("table", "key").setCell("cf", "q", "v")),
+ true);
+ }
+ }
+
+ @Test
+ public void testSampleRowKeys() {
+ verifyRetryInfoIsUsed(() -> client.sampleRowKeys("table"), true);
+ }
+
+ @Test
+ public void testSampleRowKeysNonRetryableErrorWithRetryInfo() {
+ verifyRetryInfoIsUsed(() -> client.sampleRowKeys("table"), false);
+ }
+
+ @Test
+ public void testSampleRowKeysDisableRetryInfo() throws IOException {
+ settings.stubSettings().setEnableRetryInfo(false);
+
+ try (BigtableDataClient newClient = BigtableDataClient.create(settings.build())) {
+ verifyRetryInfoCanBeDisabled(() -> newClient.sampleRowKeys("table"));
+ }
+ }
+
+ @Test
+ public void testSampleRowKeysServerNotReturningRetryInfo() {
+ verifyNoRetryInfo(() -> client.sampleRowKeys("table"), true);
+ }
+
+ @Test
+ public void testSampleRowKeysServerNotReturningRetryInfoClientDisabledHandling()
+ throws IOException {
+ settings.stubSettings().setEnableRetryInfo(false);
+
+ try (BigtableDataClient newClient = BigtableDataClient.create(settings.build())) {
+ verifyNoRetryInfo(() -> newClient.sampleRowKeys("table"), true);
+ }
+ }
+
+ @Test
+ public void testCheckAndMutateRow() {
+ verifyRetryInfoIsUsed(
+ () ->
+ client.checkAndMutateRow(
+ ConditionalRowMutation.create("table", "key")
+ .condition(Filters.FILTERS.value().regex("old-value"))
+ .then(Mutation.create().setCell("cf", "q", "v"))),
+ true);
+ }
+
+ @Test
+ public void testCheckAndMutateDisableRetryInfo() throws IOException {
+ settings.stubSettings().setEnableRetryInfo(false);
+
+ try (BigtableDataClient client = BigtableDataClient.create(settings.build())) {
+ ApiException exception = enqueueNonRetryableExceptionWithDelay(delay);
+ try {
+ client.checkAndMutateRow(
+ ConditionalRowMutation.create("table", "key")
+ .condition(Filters.FILTERS.value().regex("old-value"))
+ .then(Mutation.create().setCell("cf", "q", "v")));
+ } catch (ApiException e) {
+ assertThat(e.getStatusCode()).isEqualTo(exception.getStatusCode());
+ }
+ assertThat(attemptCounter.get()).isEqualTo(1);
+ }
+ }
+
+ @Test
+ public void testCheckAndMutateServerNotReturningRetryInfo() {
+ verifyNoRetryInfo(
+ () ->
+ client.checkAndMutateRow(
+ ConditionalRowMutation.create("table", "key")
+ .condition(Filters.FILTERS.value().regex("old-value"))
+ .then(Mutation.create().setCell("cf", "q", "v"))),
+ false);
+ }
+
+ @Test
+ public void testCheckAndMutateServerNotReturningRetryInfoClientDisabledHandling()
+ throws IOException {
+ settings.stubSettings().setEnableRetryInfo(false);
+
+ try (BigtableDataClient newClient = BigtableDataClient.create(settings.build())) {
+ verifyNoRetryInfo(
+ () ->
+ newClient.checkAndMutateRow(
+ ConditionalRowMutation.create("table", "key")
+ .condition(Filters.FILTERS.value().regex("old-value"))
+ .then(Mutation.create().setCell("cf", "q", "v"))),
+ false);
+ }
+ }
+
+ @Test
+ public void testReadModifyWrite() {
+ verifyRetryInfoIsUsed(
+ () ->
+ client.readModifyWriteRow(
+ ReadModifyWriteRow.create("table", "row").append("cf", "q", "v")),
+ true);
+ }
+
+ @Test
+ public void testReadModifyWriteDisableRetryInfo() throws IOException {
+ settings.stubSettings().setEnableRetryInfo(false);
+
+ try (BigtableDataClient client = BigtableDataClient.create(settings.build())) {
+ ApiException exception = enqueueNonRetryableExceptionWithDelay(delay);
+ try {
+ client.readModifyWriteRow(ReadModifyWriteRow.create("table", "row").append("cf", "q", "v"));
+ } catch (ApiException e) {
+ assertThat(e.getStatusCode()).isEqualTo(exception.getStatusCode());
+ }
+ assertThat(attemptCounter.get()).isEqualTo(1);
+ }
+ }
+
+ @Test
+ public void testReadModifyWriteServerNotReturningRetryInfo() {
+ verifyNoRetryInfo(
+ () ->
+ client.readModifyWriteRow(
+ ReadModifyWriteRow.create("table", "row").append("cf", "q", "v")),
+ false);
+ }
+
+ @Test
+ public void testReadModifyWriteNotReturningRetryInfoClientDisabledHandling() throws IOException {
+ settings.stubSettings().setEnableRetryInfo(false);
+
+ try (BigtableDataClient newClient = BigtableDataClient.create(settings.build())) {
+ verifyNoRetryInfo(
+ () ->
+ newClient.readModifyWriteRow(
+ ReadModifyWriteRow.create("table", "row").append("cf", "q", "v")),
+ false);
+ }
+ }
+
+ @Test
+ public void testReadChangeStream() {
+ verifyRetryInfoIsUsed(
+ () -> client.readChangeStream(ReadChangeStreamQuery.create("table")).iterator().hasNext(),
+ true);
+ }
+
+ @Test
+ public void testReadChangeStreamNonRetryableErrorWithRetryInfo() {
+ verifyRetryInfoIsUsed(
+ () -> client.readChangeStream(ReadChangeStreamQuery.create("table")).iterator().hasNext(),
+ false);
+ }
+
+ @Test
+ public void testReadChangeStreamDisableRetryInfo() throws IOException {
+ settings.stubSettings().setEnableRetryInfo(false);
+
+ try (BigtableDataClient newClient = BigtableDataClient.create(settings.build())) {
+ verifyRetryInfoCanBeDisabled(
+ () ->
+ newClient
+ .readChangeStream(ReadChangeStreamQuery.create("table"))
+ .iterator()
+ .hasNext());
+ }
+ }
+
+ @Test
+ public void testReadChangeStreamServerNotReturningRetryInfo() {
+ verifyNoRetryInfo(
+ () -> client.readChangeStream(ReadChangeStreamQuery.create("table")).iterator().hasNext(),
+ true);
+ }
+
+ @Test
+ public void testReadChangeStreamNotReturningRetryInfoClientDisabledHandling() throws IOException {
+ settings.stubSettings().setEnableRetryInfo(false);
+
+ try (BigtableDataClient newClient = BigtableDataClient.create(settings.build())) {
+ verifyNoRetryInfo(
+ () ->
+ newClient
+ .readChangeStream(ReadChangeStreamQuery.create("table"))
+ .iterator()
+ .hasNext(),
+ true);
+ }
+ }
+
+ @Test
+ public void testGenerateInitialChangeStreamPartition() {
+ verifyRetryInfoIsUsed(
+ () -> client.generateInitialChangeStreamPartitions("table").iterator().hasNext(), true);
+ }
+
+ @Test
+ public void testGenerateInitialChangeStreamPartitionNonRetryableError() {
+ verifyRetryInfoIsUsed(
+ () -> client.generateInitialChangeStreamPartitions("table").iterator().hasNext(), false);
+ }
+
+ @Test
+ public void testGenerateInitialChangeStreamPartitionDisableRetryInfo() throws IOException {
+ settings.stubSettings().setEnableRetryInfo(false);
+
+ try (BigtableDataClient newClient = BigtableDataClient.create(settings.build())) {
+ verifyRetryInfoCanBeDisabled(
+ () -> newClient.generateInitialChangeStreamPartitions("table").iterator().hasNext());
+ }
+ }
+
+ @Test
+ public void testGenerateInitialChangeStreamServerNotReturningRetryInfo() {
+ verifyNoRetryInfo(
+ () -> client.generateInitialChangeStreamPartitions("table").iterator().hasNext(), true);
+ }
+
+ @Test
+ public void testGenerateInitialChangeStreamServerNotReturningRetryInfoClientDisabledHandling()
+ throws IOException {
+ settings.stubSettings().setEnableRetryInfo(false);
+
+ try (BigtableDataClient newClient = BigtableDataClient.create(settings.build())) {
+ verifyNoRetryInfo(
+ () -> newClient.generateInitialChangeStreamPartitions("table").iterator().hasNext(),
+ true);
+ }
+ }
+
+ // Test the case where server returns retry info and client enables handling of retry info
+ private void verifyRetryInfoIsUsed(Runnable runnable, boolean retryableError) {
+ if (retryableError) {
+ enqueueRetryableExceptionWithDelay(delay);
+ } else {
+ enqueueNonRetryableExceptionWithDelay(delay);
+ }
+ Stopwatch stopwatch = Stopwatch.createStarted();
+ runnable.run();
+ stopwatch.stop();
+
+ assertThat(attemptCounter.get()).isEqualTo(2);
+ assertThat(stopwatch.elapsed()).isAtLeast(Duration.ofSeconds(delay.getSeconds()));
+ }
+
+ // Test the case where server returns retry info but client disabled handling of retry info
+ private void verifyRetryInfoCanBeDisabled(Runnable runnable) {
+ enqueueRetryableExceptionWithDelay(delay);
+ Stopwatch stopwatch = Stopwatch.createStarted();
+ runnable.run();
+ stopwatch.stop();
+
+ assertThat(attemptCounter.get()).isEqualTo(2);
+ assertThat(stopwatch.elapsed()).isLessThan(Duration.ofSeconds(delay.getSeconds()));
+
+ attemptCounter.set(0);
+ ApiException expectedApiException = enqueueNonRetryableExceptionWithDelay(delay);
+ ApiException actualException =
+ assertThrows("non retryable operations should fail", ApiException.class, runnable::run);
+ if (actualException instanceof MutateRowsException) {
+ assertThat(
+ ((MutateRowsException) actualException)
+ .getFailedMutations()
+ .get(0)
+ .getError()
+ .getStatusCode())
+ .isEqualTo(expectedApiException.getStatusCode());
+ } else {
+ assertThat(actualException.getStatusCode()).isEqualTo(expectedApiException.getStatusCode());
+ }
+ assertThat(attemptCounter.get()).isEqualTo(1);
+ }
+
+ // Test the case where server does not return retry info
+ private void verifyNoRetryInfo(Runnable runnable, boolean operationRetryable) {
+ enqueueRetryableExceptionNoRetryInfo();
+
+ if (!operationRetryable) {
+ assertThrows("non retryable operation should fail", ApiException.class, runnable::run);
+ assertThat(attemptCounter.get()).isEqualTo(1);
+ } else {
+ Stopwatch stopwatch = Stopwatch.createStarted();
+ runnable.run();
+ stopwatch.stop();
+
+ assertThat(attemptCounter.get()).isEqualTo(2);
+ assertThat(stopwatch.elapsed()).isLessThan(Duration.ofSeconds(delay.getSeconds()));
+ }
+
+ attemptCounter.set(0);
+
+ ApiException expectedApiException = enqueueNonRetryableExceptionNoRetryInfo();
+
+ ApiException actualApiException =
+ assertThrows("non retryable error should fail", ApiException.class, runnable::run);
+ if (actualApiException instanceof MutateRowsException) {
+ assertThat(
+ ((MutateRowsException) actualApiException)
+ .getFailedMutations()
+ .get(0)
+ .getError()
+ .getStatusCode())
+ .isEqualTo(expectedApiException.getStatusCode());
+ } else {
+ assertThat(actualApiException.getStatusCode())
+ .isEqualTo(expectedApiException.getStatusCode());
+ }
+
+ assertThat(attemptCounter.get()).isEqualTo(1);
+ }
+
+ private void enqueueRetryableExceptionWithDelay(com.google.protobuf.Duration delay) {
+ Metadata trailers = new Metadata();
+ RetryInfo retryInfo = RetryInfo.newBuilder().setRetryDelay(delay).build();
+ ErrorDetails errorDetails =
+ ErrorDetails.builder().setRawErrorMessages(ImmutableList.of(Any.pack(retryInfo))).build();
+ byte[] status =
+ com.google.rpc.Status.newBuilder().addDetails(Any.pack(retryInfo)).build().toByteArray();
+ trailers.put(ERROR_DETAILS_KEY, status);
+
+ ApiException exception =
+ new UnavailableException(
+ new StatusRuntimeException(Status.UNAVAILABLE, trailers),
+ GrpcStatusCode.of(Status.Code.UNAVAILABLE),
+ true,
+ errorDetails);
+
+ service.expectations.add(exception);
+ }
+
+ private ApiException enqueueNonRetryableExceptionWithDelay(com.google.protobuf.Duration delay) {
+ Metadata trailers = new Metadata();
+ RetryInfo retryInfo = RetryInfo.newBuilder().setRetryDelay(delay).build();
+ ErrorDetails errorDetails =
+ ErrorDetails.builder().setRawErrorMessages(ImmutableList.of(Any.pack(retryInfo))).build();
+ byte[] status =
+ com.google.rpc.Status.newBuilder().addDetails(Any.pack(retryInfo)).build().toByteArray();
+ trailers.put(ERROR_DETAILS_KEY, status);
+
+ ApiException exception =
+ new InternalException(
+ new StatusRuntimeException(Status.INTERNAL, trailers),
+ GrpcStatusCode.of(Status.Code.INTERNAL),
+ false,
+ errorDetails);
+
+ service.expectations.add(exception);
+
+ return exception;
+ }
+
+ private void enqueueRetryableExceptionNoRetryInfo() {
+ ApiException exception =
+ new UnavailableException(
+ new StatusRuntimeException(Status.UNAVAILABLE),
+ GrpcStatusCode.of(Status.Code.UNAVAILABLE),
+ true);
+ service.expectations.add(exception);
+ }
+
+ private ApiException enqueueNonRetryableExceptionNoRetryInfo() {
+ ApiException exception =
+ new InternalException(
+ new StatusRuntimeException(Status.INTERNAL),
+ GrpcStatusCode.of(Status.Code.INTERNAL),
+ false);
+
+ service.expectations.add(exception);
+
+ return exception;
+ }
+
+ private class FakeBigtableService extends BigtableGrpc.BigtableImplBase {
+ Queue expectations = Queues.newArrayDeque();
+
+ @Override
+ public void readRows(
+ ReadRowsRequest request, StreamObserver responseObserver) {
+ attemptCounter.incrementAndGet();
+ if (expectations.isEmpty()) {
+ responseObserver.onNext(ReadRowsResponse.getDefaultInstance());
+ responseObserver.onCompleted();
+ } else {
+ Exception expectedRpc = expectations.poll();
+ responseObserver.onError(expectedRpc);
+ }
+ }
+
+ @Override
+ public void mutateRow(
+ MutateRowRequest request, StreamObserver responseObserver) {
+ attemptCounter.incrementAndGet();
+ if (expectations.isEmpty()) {
+ responseObserver.onNext(MutateRowResponse.getDefaultInstance());
+ responseObserver.onCompleted();
+ } else {
+ Exception expectedRpc = expectations.poll();
+ responseObserver.onError(expectedRpc);
+ }
+ }
+
+ @Override
+ public void mutateRows(
+ MutateRowsRequest request, StreamObserver responseObserver) {
+ attemptCounter.incrementAndGet();
+ if (expectations.isEmpty()) {
+ MutateRowsResponse.Builder builder = MutateRowsResponse.newBuilder();
+ for (int i = 0; i < request.getEntriesCount(); i++) {
+ builder.addEntriesBuilder().setIndex(i);
+ }
+ responseObserver.onNext(builder.build());
+ responseObserver.onCompleted();
+ } else {
+ Exception expectedRpc = expectations.poll();
+ responseObserver.onError(expectedRpc);
+ }
+ }
+
+ @Override
+ public void sampleRowKeys(
+ SampleRowKeysRequest request, StreamObserver responseObserver) {
+ attemptCounter.incrementAndGet();
+ if (expectations.isEmpty()) {
+ responseObserver.onNext(SampleRowKeysResponse.getDefaultInstance());
+ responseObserver.onCompleted();
+ } else {
+ Exception expectedRpc = expectations.poll();
+ responseObserver.onError(expectedRpc);
+ }
+ }
+
+ @Override
+ public void checkAndMutateRow(
+ CheckAndMutateRowRequest request,
+ StreamObserver responseObserver) {
+ attemptCounter.incrementAndGet();
+ if (expectations.isEmpty()) {
+ responseObserver.onNext(CheckAndMutateRowResponse.getDefaultInstance());
+ responseObserver.onCompleted();
+ } else {
+ Exception expectedRpc = expectations.poll();
+ responseObserver.onError(expectedRpc);
+ }
+ }
+
+ @Override
+ public void readModifyWriteRow(
+ ReadModifyWriteRowRequest request,
+ StreamObserver responseObserver) {
+ attemptCounter.incrementAndGet();
+ if (expectations.isEmpty()) {
+ responseObserver.onNext(ReadModifyWriteRowResponse.getDefaultInstance());
+ responseObserver.onCompleted();
+ } else {
+ Exception expectedRpc = expectations.poll();
+ responseObserver.onError(expectedRpc);
+ }
+ }
+
+ @Override
+ public void generateInitialChangeStreamPartitions(
+ GenerateInitialChangeStreamPartitionsRequest request,
+ StreamObserver responseObserver) {
+ attemptCounter.incrementAndGet();
+ if (expectations.isEmpty()) {
+ responseObserver.onNext(GenerateInitialChangeStreamPartitionsResponse.getDefaultInstance());
+ responseObserver.onCompleted();
+ } else {
+ Exception expectedRpc = expectations.poll();
+ responseObserver.onError(expectedRpc);
+ }
+ }
+
+ @Override
+ public void readChangeStream(
+ ReadChangeStreamRequest request,
+ StreamObserver responseObserver) {
+ attemptCounter.incrementAndGet();
+ if (expectations.isEmpty()) {
+ responseObserver.onNext(
+ ReadChangeStreamResponse.newBuilder()
+ .setCloseStream(ReadChangeStreamResponse.CloseStream.getDefaultInstance())
+ .build());
+ responseObserver.onCompleted();
+ } else {
+ Exception expectedRpc = expectations.poll();
+ responseObserver.onError(expectedRpc);
+ }
+ }
+ }
+}
diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/metrics/BigtableTracerCallableTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/metrics/BigtableTracerCallableTest.java
index d8e3402b84..527e41e046 100644
--- a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/metrics/BigtableTracerCallableTest.java
+++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/metrics/BigtableTracerCallableTest.java
@@ -126,8 +126,13 @@ public void sendHeaders(Metadata headers) {
.setAppProfileId(APP_PROFILE_ID)
.build();
EnhancedBigtableStubSettings stubSettings =
- EnhancedBigtableStub.finalizeSettings(
- settings.getStubSettings(), Tags.getTagger(), localStats.getStatsRecorder());
+ settings
+ .getStubSettings()
+ .toBuilder()
+ .setTracerFactory(
+ EnhancedBigtableStub.createBigtableTracerFactory(
+ settings.getStubSettings(), Tags.getTagger(), localStats.getStatsRecorder()))
+ .build();
attempts = stubSettings.readRowsSettings().getRetrySettings().getMaxAttempts();
stub = new EnhancedBigtableStub(stubSettings, ClientContext.create(stubSettings));
@@ -142,8 +147,15 @@ public void sendHeaders(Metadata headers) {
.setAppProfileId(APP_PROFILE_ID)
.build();
EnhancedBigtableStubSettings noHeaderStubSettings =
- EnhancedBigtableStub.finalizeSettings(
- noHeaderSettings.getStubSettings(), Tags.getTagger(), localStats.getStatsRecorder());
+ noHeaderSettings
+ .getStubSettings()
+ .toBuilder()
+ .setTracerFactory(
+ EnhancedBigtableStub.createBigtableTracerFactory(
+ noHeaderSettings.getStubSettings(),
+ Tags.getTagger(),
+ localStats.getStatsRecorder()))
+ .build();
noHeaderStub =
new EnhancedBigtableStub(noHeaderStubSettings, ClientContext.create(noHeaderStubSettings));
}
diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/metrics/MetricsTracerTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/metrics/MetricsTracerTest.java
index da989b65dc..2894568f27 100644
--- a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/metrics/MetricsTracerTest.java
+++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/metrics/MetricsTracerTest.java
@@ -121,8 +121,13 @@ public void setUp() throws Exception {
.setAppProfileId(APP_PROFILE_ID)
.build();
EnhancedBigtableStubSettings stubSettings =
- EnhancedBigtableStub.finalizeSettings(
- settings.getStubSettings(), Tags.getTagger(), localStats.getStatsRecorder());
+ settings
+ .getStubSettings()
+ .toBuilder()
+ .setTracerFactory(
+ EnhancedBigtableStub.createBigtableTracerFactory(
+ settings.getStubSettings(), Tags.getTagger(), localStats.getStatsRecorder()))
+ .build();
stub = new EnhancedBigtableStub(stubSettings, ClientContext.create(stubSettings));
}
diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsAttemptCallableTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsAttemptCallableTest.java
index 358ff01cde..e5d12ccaeb 100644
--- a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsAttemptCallableTest.java
+++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsAttemptCallableTest.java
@@ -16,16 +16,19 @@
package com.google.cloud.bigtable.data.v2.stub.mutaterows;
import static com.google.common.truth.Truth.assertThat;
+import static org.mockito.Mockito.any;
import com.google.api.core.AbstractApiFuture;
import com.google.api.core.ApiFuture;
import com.google.api.core.ApiFutures;
import com.google.api.gax.grpc.GrpcCallContext;
import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.retrying.RetryAlgorithm;
import com.google.api.gax.retrying.RetrySettings;
import com.google.api.gax.retrying.RetryingFuture;
import com.google.api.gax.retrying.TimedAttemptSettings;
import com.google.api.gax.rpc.ApiCallContext;
+import com.google.api.gax.rpc.ApiException;
import com.google.api.gax.rpc.StatusCode.Code;
import com.google.api.gax.rpc.UnaryCallable;
import com.google.api.gax.rpc.UnavailableException;
@@ -47,6 +50,7 @@
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;
+import org.mockito.Mockito;
import org.threeten.bp.Duration;
@RunWith(JUnit4.class)
@@ -64,6 +68,8 @@ public class MutateRowsAttemptCallableTest {
private Set retryCodes;
private ApiCallContext callContext;
private MockRetryingFuture parentFuture;
+ private final RetryAlgorithm mockRetryAlgorithm =
+ Mockito.mock(RetryAlgorithm.class);
@Before
public void setUp() {
@@ -71,6 +77,12 @@ public void setUp() {
retryCodes = ImmutableSet.of(Code.DEADLINE_EXCEEDED, Code.UNAVAILABLE);
callContext = GrpcCallContext.createDefault();
parentFuture = new MockRetryingFuture();
+ Mockito.when(mockRetryAlgorithm.shouldRetry(any(), any(), any(), any()))
+ .thenAnswer(
+ input -> {
+ Throwable throwable = input.getArgument(1);
+ return ((ApiException) throwable).isRetryable();
+ });
}
@Test
@@ -84,7 +96,8 @@ public void singleEntrySuccessTest() throws Exception {
.build());
MutateRowsAttemptCallable attemptCallable =
- new MutateRowsAttemptCallable(innerCallable, request, callContext, retryCodes);
+ new MutateRowsAttemptCallable(
+ innerCallable, request, callContext, retryCodes, mockRetryAlgorithm);
attemptCallable.setExternalFuture(parentFuture);
attemptCallable.call();
@@ -107,7 +120,8 @@ public void missingEntry() {
.build());
MutateRowsAttemptCallable attemptCallable =
- new MutateRowsAttemptCallable(innerCallable, request, callContext, retryCodes);
+ new MutateRowsAttemptCallable(
+ innerCallable, request, callContext, retryCodes, mockRetryAlgorithm);
attemptCallable.setExternalFuture(parentFuture);
attemptCallable.call();
@@ -140,7 +154,8 @@ public void testNoRpcTimeout() {
.build());
MutateRowsAttemptCallable attemptCallable =
- new MutateRowsAttemptCallable(innerCallable, request, callContext, retryCodes);
+ new MutateRowsAttemptCallable(
+ innerCallable, request, callContext, retryCodes, mockRetryAlgorithm);
attemptCallable.setExternalFuture(parentFuture);
attemptCallable.call();
@@ -172,7 +187,8 @@ public void mixedTest() {
.build());
MutateRowsAttemptCallable attemptCallable =
- new MutateRowsAttemptCallable(innerCallable, request, callContext, retryCodes);
+ new MutateRowsAttemptCallable(
+ innerCallable, request, callContext, retryCodes, mockRetryAlgorithm);
attemptCallable.setExternalFuture(parentFuture);
// Make the only call
@@ -230,7 +246,8 @@ public void nextAttemptTest() {
.build());
MutateRowsAttemptCallable attemptCallable =
- new MutateRowsAttemptCallable(innerCallable, request, callContext, retryCodes);
+ new MutateRowsAttemptCallable(
+ innerCallable, request, callContext, retryCodes, mockRetryAlgorithm);
attemptCallable.setExternalFuture(parentFuture);
// Make the first call
@@ -295,7 +312,8 @@ public ApiFuture> futureCall(
// Make the call
MutateRowsAttemptCallable attemptCallable =
- new MutateRowsAttemptCallable(innerCallable, request, callContext, retryCodes);
+ new MutateRowsAttemptCallable(
+ innerCallable, request, callContext, retryCodes, mockRetryAlgorithm);
attemptCallable.setExternalFuture(parentFuture);
attemptCallable.call();
@@ -347,7 +365,8 @@ public ApiFuture> futureCall(
// Make the call
MutateRowsAttemptCallable attemptCallable =
- new MutateRowsAttemptCallable(innerCallable, request, callContext, retryCodes);
+ new MutateRowsAttemptCallable(
+ innerCallable, request, callContext, retryCodes, mockRetryAlgorithm);
attemptCallable.setExternalFuture(parentFuture);
attemptCallable.call();
diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsBatchingDescriptorTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsBatchingDescriptorTest.java
index 81d5c67396..237444ba84 100644
--- a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsBatchingDescriptorTest.java
+++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsBatchingDescriptorTest.java
@@ -138,7 +138,7 @@ public void splitExceptionWithFailedMutationsTest() {
// Threw an exception at 1st and 3rd entry
MutateRowsException serverError =
- new MutateRowsException(
+ MutateRowsException.create(
null,
ImmutableList.of(
MutateRowsException.FailedMutation.create(
diff --git a/grpc-google-cloud-bigtable-admin-v2/pom.xml b/grpc-google-cloud-bigtable-admin-v2/pom.xml
index 2c9af872ff..0ecbd8a400 100644
--- a/grpc-google-cloud-bigtable-admin-v2/pom.xml
+++ b/grpc-google-cloud-bigtable-admin-v2/pom.xml
@@ -4,13 +4,13 @@
4.0.0
com.google.api.grpc
grpc-google-cloud-bigtable-admin-v2
- 2.30.0
+ 2.31.0
grpc-google-cloud-bigtable-admin-v2
GRPC library for grpc-google-cloud-bigtable-admin-v2
com.google.cloud
google-cloud-bigtable-parent
- 2.30.0
+ 2.31.0
@@ -18,14 +18,14 @@
com.google.cloud
google-cloud-bigtable-deps-bom
- 2.30.0
+ 2.31.0
pom
import
com.google.cloud
google-cloud-bigtable-bom
- 2.30.0
+ 2.31.0
pom
import
diff --git a/grpc-google-cloud-bigtable-v2/pom.xml b/grpc-google-cloud-bigtable-v2/pom.xml
index 5f0ad3cb46..3b233c9223 100644
--- a/grpc-google-cloud-bigtable-v2/pom.xml
+++ b/grpc-google-cloud-bigtable-v2/pom.xml
@@ -4,13 +4,13 @@
4.0.0
com.google.api.grpc
grpc-google-cloud-bigtable-v2
- 2.30.0
+ 2.31.0
grpc-google-cloud-bigtable-v2
GRPC library for grpc-google-cloud-bigtable-v2
com.google.cloud
google-cloud-bigtable-parent
- 2.30.0
+ 2.31.0
@@ -18,14 +18,14 @@
com.google.cloud
google-cloud-bigtable-deps-bom
- 2.30.0
+ 2.31.0
pom
import
com.google.cloud
google-cloud-bigtable-bom
- 2.30.0
+ 2.31.0
pom
import
diff --git a/pom.xml b/pom.xml
index d1ceadc642..f38a4edf88 100644
--- a/pom.xml
+++ b/pom.xml
@@ -4,7 +4,7 @@
google-cloud-bigtable-parent
pom
- 2.30.0
+ 2.31.0
Google Cloud Bigtable Parent
https://github.com/googleapis/java-bigtable
@@ -14,7 +14,7 @@
com.google.cloud
google-cloud-shared-config
- 1.6.1
+ 1.7.1
@@ -153,27 +153,27 @@
com.google.api.grpc
proto-google-cloud-bigtable-v2
- 2.30.0
+ 2.31.0
com.google.api.grpc
proto-google-cloud-bigtable-admin-v2
- 2.30.0
+ 2.31.0
com.google.api.grpc
grpc-google-cloud-bigtable-v2
- 2.30.0
+ 2.31.0
com.google.api.grpc
grpc-google-cloud-bigtable-admin-v2
- 2.30.0
+ 2.31.0
com.google.cloud
google-cloud-bigtable
- 2.30.0
+ 2.31.0
@@ -184,12 +184,12 @@
com.google.truth
truth
- 1.1.5
+ 1.2.0
com.google.truth.extensions
truth-proto-extension
- 1.1.5
+ 1.2.0
test
diff --git a/proto-google-cloud-bigtable-admin-v2/pom.xml b/proto-google-cloud-bigtable-admin-v2/pom.xml
index 60044ab20a..01ad910df2 100644
--- a/proto-google-cloud-bigtable-admin-v2/pom.xml
+++ b/proto-google-cloud-bigtable-admin-v2/pom.xml
@@ -4,13 +4,13 @@
4.0.0
com.google.api.grpc
proto-google-cloud-bigtable-admin-v2
- 2.30.0
+ 2.31.0
proto-google-cloud-bigtable-admin-v2
PROTO library for proto-google-cloud-bigtable-admin-v2
com.google.cloud
google-cloud-bigtable-parent
- 2.30.0
+ 2.31.0
@@ -18,14 +18,14 @@
com.google.cloud
google-cloud-bigtable-deps-bom
- 2.30.0
+ 2.31.0
pom
import
com.google.cloud
google-cloud-bigtable-bom
- 2.30.0
+ 2.31.0
pom
import
diff --git a/proto-google-cloud-bigtable-admin-v2/src/main/java/com/google/bigtable/admin/v2/BigtableTableAdminProto.java b/proto-google-cloud-bigtable-admin-v2/src/main/java/com/google/bigtable/admin/v2/BigtableTableAdminProto.java
index 3c575dd878..31db57ad7a 100644
--- a/proto-google-cloud-bigtable-admin-v2/src/main/java/com/google/bigtable/admin/v2/BigtableTableAdminProto.java
+++ b/proto-google-cloud-bigtable-admin-v2/src/main/java/com/google/bigtable/admin/v2/BigtableTableAdminProto.java
@@ -250,224 +250,225 @@ public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() {
+ "\203\001\n\025UndeleteTableMetadata\022\014\n\004name\030\001 \001(\t\022"
+ ".\n\nstart_time\030\002 \001(\0132\032.google.protobuf.Ti"
+ "mestamp\022,\n\010end_time\030\003 \001(\0132\032.google.proto"
- + "buf.Timestamp\"\341\002\n\033ModifyColumnFamiliesRe"
+ + "buf.Timestamp\"\200\003\n\033ModifyColumnFamiliesRe"
+ "quest\0229\n\004name\030\001 \001(\tB+\342A\001\002\372A$\n\"bigtablead"
+ "min.googleapis.com/Table\022_\n\rmodification"
+ "s\030\002 \003(\0132B.google.bigtable.admin.v2.Modif"
+ "yColumnFamiliesRequest.ModificationB\004\342A\001"
- + "\002\032\245\001\n\014Modification\022\n\n\002id\030\001 \001(\t\0228\n\006create"
- + "\030\002 \001(\0132&.google.bigtable.admin.v2.Column"
- + "FamilyH\000\0228\n\006update\030\003 \001(\0132&.google.bigtab"
- + "le.admin.v2.ColumnFamilyH\000\022\016\n\004drop\030\004 \001(\010"
- + "H\000B\005\n\003mod\"\\\n\037GenerateConsistencyTokenReq"
- + "uest\0229\n\004name\030\001 \001(\tB+\342A\001\002\372A$\n\"bigtableadm"
- + "in.googleapis.com/Table\"=\n GenerateConsi"
- + "stencyTokenResponse\022\031\n\021consistency_token"
- + "\030\001 \001(\t\"u\n\027CheckConsistencyRequest\0229\n\004nam"
- + "e\030\001 \001(\tB+\342A\001\002\372A$\n\"bigtableadmin.googleap"
- + "is.com/Table\022\037\n\021consistency_token\030\002 \001(\tB"
- + "\004\342A\001\002\".\n\030CheckConsistencyResponse\022\022\n\ncon"
- + "sistent\030\001 \001(\010\"\351\001\n\024SnapshotTableRequest\0229"
- + "\n\004name\030\001 \001(\tB+\342A\001\002\372A$\n\"bigtableadmin.goo"
- + "gleapis.com/Table\022>\n\007cluster\030\002 \001(\tB-\342A\001\002"
- + "\372A&\n$bigtableadmin.googleapis.com/Cluste"
- + "r\022\031\n\013snapshot_id\030\003 \001(\tB\004\342A\001\002\022&\n\003ttl\030\004 \001("
- + "\0132\031.google.protobuf.Duration\022\023\n\013descript"
- + "ion\030\005 \001(\t\"R\n\022GetSnapshotRequest\022<\n\004name\030"
- + "\001 \001(\tB.\342A\001\002\372A\'\n%bigtableadmin.googleapis"
- + ".com/Snapshot\"|\n\024ListSnapshotsRequest\022=\n"
- + "\006parent\030\001 \001(\tB-\342A\001\002\372A&\n$bigtableadmin.go"
- + "ogleapis.com/Cluster\022\021\n\tpage_size\030\002 \001(\005\022"
- + "\022\n\npage_token\030\003 \001(\t\"g\n\025ListSnapshotsResp"
- + "onse\0225\n\tsnapshots\030\001 \003(\0132\".google.bigtabl"
- + "e.admin.v2.Snapshot\022\027\n\017next_page_token\030\002"
- + " \001(\t\"U\n\025DeleteSnapshotRequest\022<\n\004name\030\001 "
- + "\001(\tB.\342A\001\002\372A\'\n%bigtableadmin.googleapis.c"
- + "om/Snapshot\"\304\001\n\025SnapshotTableMetadata\022H\n"
- + "\020original_request\030\001 \001(\0132..google.bigtabl"
- + "e.admin.v2.SnapshotTableRequest\0220\n\014reque"
- + "st_time\030\002 \001(\0132\032.google.protobuf.Timestam"
- + "p\022/\n\013finish_time\030\003 \001(\0132\032.google.protobuf"
- + ".Timestamp\"\330\001\n\037CreateTableFromSnapshotMe"
- + "tadata\022R\n\020original_request\030\001 \001(\01328.googl"
- + "e.bigtable.admin.v2.CreateTableFromSnaps"
- + "hotRequest\0220\n\014request_time\030\002 \001(\0132\032.googl"
- + "e.protobuf.Timestamp\022/\n\013finish_time\030\003 \001("
- + "\0132\032.google.protobuf.Timestamp\"\245\001\n\023Create"
- + "BackupRequest\022=\n\006parent\030\001 \001(\tB-\342A\001\002\372A&\n$"
- + "bigtableadmin.googleapis.com/Cluster\022\027\n\t"
- + "backup_id\030\002 \001(\tB\004\342A\001\002\0226\n\006backup\030\003 \001(\0132 ."
- + "google.bigtable.admin.v2.BackupB\004\342A\001\002\"\230\001"
- + "\n\024CreateBackupMetadata\022\014\n\004name\030\001 \001(\t\022\024\n\014"
- + "source_table\030\002 \001(\t\022.\n\nstart_time\030\003 \001(\0132\032"
- + ".google.protobuf.Timestamp\022,\n\010end_time\030\004"
- + " \001(\0132\032.google.protobuf.Timestamp\"\204\001\n\023Upd"
- + "ateBackupRequest\0226\n\006backup\030\001 \001(\0132 .googl"
- + "e.bigtable.admin.v2.BackupB\004\342A\001\002\0225\n\013upda"
- + "te_mask\030\002 \001(\0132\032.google.protobuf.FieldMas"
- + "kB\004\342A\001\002\"N\n\020GetBackupRequest\022:\n\004name\030\001 \001("
- + "\tB,\342A\001\002\372A%\n#bigtableadmin.googleapis.com"
- + "/Backup\"Q\n\023DeleteBackupRequest\022:\n\004name\030\001"
- + " \001(\tB,\342A\001\002\372A%\n#bigtableadmin.googleapis."
- + "com/Backup\"\234\001\n\022ListBackupsRequest\022=\n\006par"
- + "ent\030\001 \001(\tB-\342A\001\002\372A&\n$bigtableadmin.google"
- + "apis.com/Cluster\022\016\n\006filter\030\002 \001(\t\022\020\n\010orde"
- + "r_by\030\003 \001(\t\022\021\n\tpage_size\030\004 \001(\005\022\022\n\npage_to"
- + "ken\030\005 \001(\t\"a\n\023ListBackupsResponse\0221\n\007back"
- + "ups\030\001 \003(\0132 .google.bigtable.admin.v2.Bac"
- + "kup\022\027\n\017next_page_token\030\002 \001(\t\"\347\001\n\021CopyBac"
- + "kupRequest\022=\n\006parent\030\001 \001(\tB-\342A\001\002\372A&\n$big"
- + "tableadmin.googleapis.com/Cluster\022\027\n\tbac"
- + "kup_id\030\002 \001(\tB\004\342A\001\002\022C\n\rsource_backup\030\003 \001("
- + "\tB,\342A\001\002\372A%\n#bigtableadmin.googleapis.com"
- + "/Backup\0225\n\013expire_time\030\004 \001(\0132\032.google.pr"
- + "otobuf.TimestampB\004\342A\001\002\"\315\001\n\022CopyBackupMet"
- + "adata\0226\n\004name\030\001 \001(\tB(\372A%\n#bigtableadmin."
- + "googleapis.com/Backup\022@\n\022source_backup_i"
- + "nfo\030\002 \001(\0132$.google.bigtable.admin.v2.Bac"
- + "kupInfo\022=\n\010progress\030\003 \001(\0132+.google.bigta"
- + "ble.admin.v2.OperationProgress2\242*\n\022Bigta"
- + "bleTableAdmin\022\253\001\n\013CreateTable\022,.google.b"
- + "igtable.admin.v2.CreateTableRequest\032\037.go"
- + "ogle.bigtable.admin.v2.Table\"M\332A\025parent,"
- + "table_id,table\202\323\344\223\002/\"*/v2/{parent=projec"
- + "ts/*/instances/*}/tables:\001*\022\212\002\n\027CreateTa"
- + "bleFromSnapshot\0228.google.bigtable.admin."
- + "v2.CreateTableFromSnapshotRequest\032\035.goog"
- + "le.longrunning.Operation\"\225\001\312A(\n\005Table\022\037C"
- + "reateTableFromSnapshotMetadata\332A\037parent,"
- + "table_id,source_snapshot\202\323\344\223\002B\"=/v2/{par"
- + "ent=projects/*/instances/*}/tables:creat"
- + "eFromSnapshot:\001*\022\244\001\n\nListTables\022+.google"
- + ".bigtable.admin.v2.ListTablesRequest\032,.g"
- + "oogle.bigtable.admin.v2.ListTablesRespon"
- + "se\";\332A\006parent\202\323\344\223\002,\022*/v2/{parent=project"
- + "s/*/instances/*}/tables\022\221\001\n\010GetTable\022).g"
- + "oogle.bigtable.admin.v2.GetTableRequest\032"
- + "\037.google.bigtable.admin.v2.Table\"9\332A\004nam"
- + "e\202\323\344\223\002,\022*/v2/{name=projects/*/instances/"
- + "*/tables/*}\022\316\001\n\013UpdateTable\022,.google.big"
- + "table.admin.v2.UpdateTableRequest\032\035.goog"
- + "le.longrunning.Operation\"r\312A\034\n\005Table\022\023Up"
- + "dateTableMetadata\332A\021table,update_mask\202\323\344"
- + "\223\002920/v2/{table.name=projects/*/instance"
- + "s/*/tables/*}:\005table\022\216\001\n\013DeleteTable\022,.g"
- + "oogle.bigtable.admin.v2.DeleteTableReque"
- + "st\032\026.google.protobuf.Empty\"9\332A\004name\202\323\344\223\002"
- + ",**/v2/{name=projects/*/instances/*/tabl"
- + "es/*}\022\306\001\n\rUndeleteTable\022..google.bigtabl"
- + "e.admin.v2.UndeleteTableRequest\032\035.google"
- + ".longrunning.Operation\"f\312A\036\n\005Table\022\025Unde"
- + "leteTableMetadata\332A\004name\202\323\344\223\0028\"3/v2/{nam"
- + "e=projects/*/instances/*/tables/*}:undel"
- + "ete:\001*\022\317\001\n\024ModifyColumnFamilies\0225.google"
- + ".bigtable.admin.v2.ModifyColumnFamiliesR"
- + "equest\032\037.google.bigtable.admin.v2.Table\""
- + "_\332A\022name,modifications\202\323\344\223\002D\"?/v2/{name="
- + "projects/*/instances/*/tables/*}:modifyC"
- + "olumnFamilies:\001*\022\231\001\n\014DropRowRange\022-.goog"
- + "le.bigtable.admin.v2.DropRowRangeRequest"
- + "\032\026.google.protobuf.Empty\"B\202\323\344\223\002<\"7/v2/{n"
- + "ame=projects/*/instances/*/tables/*}:dro"
- + "pRowRange:\001*\022\350\001\n\030GenerateConsistencyToke"
- + "n\0229.google.bigtable.admin.v2.GenerateCon"
- + "sistencyTokenRequest\032:.google.bigtable.a"
- + "dmin.v2.GenerateConsistencyTokenResponse"
- + "\"U\332A\004name\202\323\344\223\002H\"C/v2/{name=projects/*/in"
- + "stances/*/tables/*}:generateConsistencyT"
- + "oken:\001*\022\332\001\n\020CheckConsistency\0221.google.bi"
- + "gtable.admin.v2.CheckConsistencyRequest\032"
- + "2.google.bigtable.admin.v2.CheckConsiste"
- + "ncyResponse\"_\332A\026name,consistency_token\202\323"
- + "\344\223\002@\";/v2/{name=projects/*/instances/*/t"
- + "ables/*}:checkConsistency:\001*\022\352\001\n\rSnapsho"
- + "tTable\022..google.bigtable.admin.v2.Snapsh"
- + "otTableRequest\032\035.google.longrunning.Oper"
- + "ation\"\211\001\312A!\n\010Snapshot\022\025SnapshotTableMeta"
- + "data\332A$name,cluster,snapshot_id,descript"
- + "ion\202\323\344\223\0028\"3/v2/{name=projects/*/instance"
- + "s/*/tables/*}:snapshot:\001*\022\250\001\n\013GetSnapsho"
- + "t\022,.google.bigtable.admin.v2.GetSnapshot"
- + "Request\032\".google.bigtable.admin.v2.Snaps"
- + "hot\"G\332A\004name\202\323\344\223\002:\0228/v2/{name=projects/*"
- + "/instances/*/clusters/*/snapshots/*}\022\273\001\n"
- + "\rListSnapshots\022..google.bigtable.admin.v"
- + "2.ListSnapshotsRequest\032/.google.bigtable"
- + ".admin.v2.ListSnapshotsResponse\"I\332A\006pare"
- + "nt\202\323\344\223\002:\0228/v2/{parent=projects/*/instanc"
- + "es/*/clusters/*}/snapshots\022\242\001\n\016DeleteSna"
- + "pshot\022/.google.bigtable.admin.v2.DeleteS"
- + "napshotRequest\032\026.google.protobuf.Empty\"G"
- + "\332A\004name\202\323\344\223\002:*8/v2/{name=projects/*/inst"
- + "ances/*/clusters/*/snapshots/*}\022\340\001\n\014Crea"
- + "teBackup\022-.google.bigtable.admin.v2.Crea"
- + "teBackupRequest\032\035.google.longrunning.Ope"
- + "ration\"\201\001\312A\036\n\006Backup\022\024CreateBackupMetada"
- + "ta\332A\027parent,backup_id,backup\202\323\344\223\002@\"6/v2/"
- + "{parent=projects/*/instances/*/clusters/"
- + "*}/backups:\006backup\022\240\001\n\tGetBackup\022*.googl"
- + "e.bigtable.admin.v2.GetBackupRequest\032 .g"
- + "oogle.bigtable.admin.v2.Backup\"E\332A\004name\202"
- + "\323\344\223\0028\0226/v2/{name=projects/*/instances/*/"
- + "clusters/*/backups/*}\022\303\001\n\014UpdateBackup\022-"
- + ".google.bigtable.admin.v2.UpdateBackupRe"
- + "quest\032 .google.bigtable.admin.v2.Backup\""
- + "b\332A\022backup,update_mask\202\323\344\223\002G2=/v2/{backu"
- + "p.name=projects/*/instances/*/clusters/*"
- + "/backups/*}:\006backup\022\234\001\n\014DeleteBackup\022-.g"
- + "oogle.bigtable.admin.v2.DeleteBackupRequ"
- + "est\032\026.google.protobuf.Empty\"E\332A\004name\202\323\344\223"
- + "\0028*6/v2/{name=projects/*/instances/*/clu"
- + "sters/*/backups/*}\022\263\001\n\013ListBackups\022,.goo"
- + "gle.bigtable.admin.v2.ListBackupsRequest"
- + "\032-.google.bigtable.admin.v2.ListBackupsR"
- + "esponse\"G\332A\006parent\202\323\344\223\0028\0226/v2/{parent=pr"
- + "ojects/*/instances/*/clusters/*}/backups"
- + "\022\273\001\n\014RestoreTable\022-.google.bigtable.admi"
- + "n.v2.RestoreTableRequest\032\035.google.longru"
- + "nning.Operation\"]\312A\035\n\005Table\022\024RestoreTabl"
- + "eMetadata\202\323\344\223\0027\"2/v2/{parent=projects/*/"
- + "instances/*}/tables:restore:\001*\022\355\001\n\nCopyB"
- + "ackup\022+.google.bigtable.admin.v2.CopyBac"
- + "kupRequest\032\035.google.longrunning.Operatio"
- + "n\"\222\001\312A\034\n\006Backup\022\022CopyBackupMetadata\332A*pa"
- + "rent,backup_id,source_backup,expire_time"
- + "\202\323\344\223\002@\";/v2/{parent=projects/*/instances"
- + "/*/clusters/*}/backups:copy:\001*\022\354\001\n\014GetIa"
- + "mPolicy\022\".google.iam.v1.GetIamPolicyRequ"
- + "est\032\025.google.iam.v1.Policy\"\240\001\332A\010resource"
- + "\202\323\344\223\002\216\001\";/v2/{resource=projects/*/instan"
- + "ces/*/tables/*}:getIamPolicy:\001*ZL\"G/v2/{"
- + "resource=projects/*/instances/*/clusters"
- + "/*/backups/*}:getIamPolicy:\001*\022\363\001\n\014SetIam"
- + "Policy\022\".google.iam.v1.SetIamPolicyReque"
- + "st\032\025.google.iam.v1.Policy\"\247\001\332A\017resource,"
- + "policy\202\323\344\223\002\216\001\";/v2/{resource=projects/*/"
- + "instances/*/tables/*}:setIamPolicy:\001*ZL\""
- + "G/v2/{resource=projects/*/instances/*/cl"
- + "usters/*/backups/*}:setIamPolicy:\001*\022\244\002\n\022"
- + "TestIamPermissions\022(.google.iam.v1.TestI"
- + "amPermissionsRequest\032).google.iam.v1.Tes"
- + "tIamPermissionsResponse\"\270\001\332A\024resource,pe"
- + "rmissions\202\323\344\223\002\232\001\"A/v2/{resource=projects"
- + "/*/instances/*/tables/*}:testIamPermissi"
- + "ons:\001*ZR\"M/v2/{resource=projects/*/insta"
- + "nces/*/clusters/*/backups/*}:testIamPerm"
- + "issions:\001*\032\336\002\312A\034bigtableadmin.googleapis"
- + ".com\322A\273\002https://www.googleapis.com/auth/"
- + "bigtable.admin,https://www.googleapis.co"
- + "m/auth/bigtable.admin.table,https://www."
- + "googleapis.com/auth/cloud-bigtable.admin"
- + ",https://www.googleapis.com/auth/cloud-b"
- + "igtable.admin.table,https://www.googleap"
- + "is.com/auth/cloud-platform,https://www.g"
- + "oogleapis.com/auth/cloud-platform.read-o"
- + "nlyB\337\001\n\034com.google.bigtable.admin.v2B\027Bi"
- + "gtableTableAdminProtoP\001Z=google.golang.o"
- + "rg/genproto/googleapis/bigtable/admin/v2"
- + ";admin\252\002\036Google.Cloud.Bigtable.Admin.V2\312"
- + "\002\036Google\\Cloud\\Bigtable\\Admin\\V2\352\002\"Googl"
- + "e::Cloud::Bigtable::Admin::V2b\006proto3"
+ + "\002\022\035\n\017ignore_warnings\030\003 \001(\010B\004\342A\001\001\032\245\001\n\014Mod"
+ + "ification\022\n\n\002id\030\001 \001(\t\0228\n\006create\030\002 \001(\0132&."
+ + "google.bigtable.admin.v2.ColumnFamilyH\000\022"
+ + "8\n\006update\030\003 \001(\0132&.google.bigtable.admin."
+ + "v2.ColumnFamilyH\000\022\016\n\004drop\030\004 \001(\010H\000B\005\n\003mod"
+ + "\"\\\n\037GenerateConsistencyTokenRequest\0229\n\004n"
+ + "ame\030\001 \001(\tB+\342A\001\002\372A$\n\"bigtableadmin.google"
+ + "apis.com/Table\"=\n GenerateConsistencyTok"
+ + "enResponse\022\031\n\021consistency_token\030\001 \001(\t\"u\n"
+ + "\027CheckConsistencyRequest\0229\n\004name\030\001 \001(\tB+"
+ + "\342A\001\002\372A$\n\"bigtableadmin.googleapis.com/Ta"
+ + "ble\022\037\n\021consistency_token\030\002 \001(\tB\004\342A\001\002\".\n\030"
+ + "CheckConsistencyResponse\022\022\n\nconsistent\030\001"
+ + " \001(\010\"\351\001\n\024SnapshotTableRequest\0229\n\004name\030\001 "
+ + "\001(\tB+\342A\001\002\372A$\n\"bigtableadmin.googleapis.c"
+ + "om/Table\022>\n\007cluster\030\002 \001(\tB-\342A\001\002\372A&\n$bigt"
+ + "ableadmin.googleapis.com/Cluster\022\031\n\013snap"
+ + "shot_id\030\003 \001(\tB\004\342A\001\002\022&\n\003ttl\030\004 \001(\0132\031.googl"
+ + "e.protobuf.Duration\022\023\n\013description\030\005 \001(\t"
+ + "\"R\n\022GetSnapshotRequest\022<\n\004name\030\001 \001(\tB.\342A"
+ + "\001\002\372A\'\n%bigtableadmin.googleapis.com/Snap"
+ + "shot\"|\n\024ListSnapshotsRequest\022=\n\006parent\030\001"
+ + " \001(\tB-\342A\001\002\372A&\n$bigtableadmin.googleapis."
+ + "com/Cluster\022\021\n\tpage_size\030\002 \001(\005\022\022\n\npage_t"
+ + "oken\030\003 \001(\t\"g\n\025ListSnapshotsResponse\0225\n\ts"
+ + "napshots\030\001 \003(\0132\".google.bigtable.admin.v"
+ + "2.Snapshot\022\027\n\017next_page_token\030\002 \001(\t\"U\n\025D"
+ + "eleteSnapshotRequest\022<\n\004name\030\001 \001(\tB.\342A\001\002"
+ + "\372A\'\n%bigtableadmin.googleapis.com/Snapsh"
+ + "ot\"\304\001\n\025SnapshotTableMetadata\022H\n\020original"
+ + "_request\030\001 \001(\0132..google.bigtable.admin.v"
+ + "2.SnapshotTableRequest\0220\n\014request_time\030\002"
+ + " \001(\0132\032.google.protobuf.Timestamp\022/\n\013fini"
+ + "sh_time\030\003 \001(\0132\032.google.protobuf.Timestam"
+ + "p\"\330\001\n\037CreateTableFromSnapshotMetadata\022R\n"
+ + "\020original_request\030\001 \001(\01328.google.bigtabl"
+ + "e.admin.v2.CreateTableFromSnapshotReques"
+ + "t\0220\n\014request_time\030\002 \001(\0132\032.google.protobu"
+ + "f.Timestamp\022/\n\013finish_time\030\003 \001(\0132\032.googl"
+ + "e.protobuf.Timestamp\"\245\001\n\023CreateBackupReq"
+ + "uest\022=\n\006parent\030\001 \001(\tB-\342A\001\002\372A&\n$bigtablea"
+ + "dmin.googleapis.com/Cluster\022\027\n\tbackup_id"
+ + "\030\002 \001(\tB\004\342A\001\002\0226\n\006backup\030\003 \001(\0132 .google.bi"
+ + "gtable.admin.v2.BackupB\004\342A\001\002\"\230\001\n\024CreateB"
+ + "ackupMetadata\022\014\n\004name\030\001 \001(\t\022\024\n\014source_ta"
+ + "ble\030\002 \001(\t\022.\n\nstart_time\030\003 \001(\0132\032.google.p"
+ + "rotobuf.Timestamp\022,\n\010end_time\030\004 \001(\0132\032.go"
+ + "ogle.protobuf.Timestamp\"\204\001\n\023UpdateBackup"
+ + "Request\0226\n\006backup\030\001 \001(\0132 .google.bigtabl"
+ + "e.admin.v2.BackupB\004\342A\001\002\0225\n\013update_mask\030\002"
+ + " \001(\0132\032.google.protobuf.FieldMaskB\004\342A\001\002\"N"
+ + "\n\020GetBackupRequest\022:\n\004name\030\001 \001(\tB,\342A\001\002\372A"
+ + "%\n#bigtableadmin.googleapis.com/Backup\"Q"
+ + "\n\023DeleteBackupRequest\022:\n\004name\030\001 \001(\tB,\342A\001"
+ + "\002\372A%\n#bigtableadmin.googleapis.com/Backu"
+ + "p\"\234\001\n\022ListBackupsRequest\022=\n\006parent\030\001 \001(\t"
+ + "B-\342A\001\002\372A&\n$bigtableadmin.googleapis.com/"
+ + "Cluster\022\016\n\006filter\030\002 \001(\t\022\020\n\010order_by\030\003 \001("
+ + "\t\022\021\n\tpage_size\030\004 \001(\005\022\022\n\npage_token\030\005 \001(\t"
+ + "\"a\n\023ListBackupsResponse\0221\n\007backups\030\001 \003(\013"
+ + "2 .google.bigtable.admin.v2.Backup\022\027\n\017ne"
+ + "xt_page_token\030\002 \001(\t\"\347\001\n\021CopyBackupReques"
+ + "t\022=\n\006parent\030\001 \001(\tB-\342A\001\002\372A&\n$bigtableadmi"
+ + "n.googleapis.com/Cluster\022\027\n\tbackup_id\030\002 "
+ + "\001(\tB\004\342A\001\002\022C\n\rsource_backup\030\003 \001(\tB,\342A\001\002\372A"
+ + "%\n#bigtableadmin.googleapis.com/Backup\0225"
+ + "\n\013expire_time\030\004 \001(\0132\032.google.protobuf.Ti"
+ + "mestampB\004\342A\001\002\"\315\001\n\022CopyBackupMetadata\0226\n\004"
+ + "name\030\001 \001(\tB(\372A%\n#bigtableadmin.googleapi"
+ + "s.com/Backup\022@\n\022source_backup_info\030\002 \001(\013"
+ + "2$.google.bigtable.admin.v2.BackupInfo\022="
+ + "\n\010progress\030\003 \001(\0132+.google.bigtable.admin"
+ + ".v2.OperationProgress2\242*\n\022BigtableTableA"
+ + "dmin\022\253\001\n\013CreateTable\022,.google.bigtable.a"
+ + "dmin.v2.CreateTableRequest\032\037.google.bigt"
+ + "able.admin.v2.Table\"M\332A\025parent,table_id,"
+ + "table\202\323\344\223\002/\"*/v2/{parent=projects/*/inst"
+ + "ances/*}/tables:\001*\022\212\002\n\027CreateTableFromSn"
+ + "apshot\0228.google.bigtable.admin.v2.Create"
+ + "TableFromSnapshotRequest\032\035.google.longru"
+ + "nning.Operation\"\225\001\312A(\n\005Table\022\037CreateTabl"
+ + "eFromSnapshotMetadata\332A\037parent,table_id,"
+ + "source_snapshot\202\323\344\223\002B\"=/v2/{parent=proje"
+ + "cts/*/instances/*}/tables:createFromSnap"
+ + "shot:\001*\022\244\001\n\nListTables\022+.google.bigtable"
+ + ".admin.v2.ListTablesRequest\032,.google.big"
+ + "table.admin.v2.ListTablesResponse\";\332A\006pa"
+ + "rent\202\323\344\223\002,\022*/v2/{parent=projects/*/insta"
+ + "nces/*}/tables\022\221\001\n\010GetTable\022).google.big"
+ + "table.admin.v2.GetTableRequest\032\037.google."
+ + "bigtable.admin.v2.Table\"9\332A\004name\202\323\344\223\002,\022*"
+ + "/v2/{name=projects/*/instances/*/tables/"
+ + "*}\022\316\001\n\013UpdateTable\022,.google.bigtable.adm"
+ + "in.v2.UpdateTableRequest\032\035.google.longru"
+ + "nning.Operation\"r\312A\034\n\005Table\022\023UpdateTable"
+ + "Metadata\332A\021table,update_mask\202\323\344\223\002920/v2/"
+ + "{table.name=projects/*/instances/*/table"
+ + "s/*}:\005table\022\216\001\n\013DeleteTable\022,.google.big"
+ + "table.admin.v2.DeleteTableRequest\032\026.goog"
+ + "le.protobuf.Empty\"9\332A\004name\202\323\344\223\002,**/v2/{n"
+ + "ame=projects/*/instances/*/tables/*}\022\306\001\n"
+ + "\rUndeleteTable\022..google.bigtable.admin.v"
+ + "2.UndeleteTableRequest\032\035.google.longrunn"
+ + "ing.Operation\"f\312A\036\n\005Table\022\025UndeleteTable"
+ + "Metadata\332A\004name\202\323\344\223\0028\"3/v2/{name=project"
+ + "s/*/instances/*/tables/*}:undelete:\001*\022\317\001"
+ + "\n\024ModifyColumnFamilies\0225.google.bigtable"
+ + ".admin.v2.ModifyColumnFamiliesRequest\032\037."
+ + "google.bigtable.admin.v2.Table\"_\332A\022name,"
+ + "modifications\202\323\344\223\002D\"?/v2/{name=projects/"
+ + "*/instances/*/tables/*}:modifyColumnFami"
+ + "lies:\001*\022\231\001\n\014DropRowRange\022-.google.bigtab"
+ + "le.admin.v2.DropRowRangeRequest\032\026.google"
+ + ".protobuf.Empty\"B\202\323\344\223\002<\"7/v2/{name=proje"
+ + "cts/*/instances/*/tables/*}:dropRowRange"
+ + ":\001*\022\350\001\n\030GenerateConsistencyToken\0229.googl"
+ + "e.bigtable.admin.v2.GenerateConsistencyT"
+ + "okenRequest\032:.google.bigtable.admin.v2.G"
+ + "enerateConsistencyTokenResponse\"U\332A\004name"
+ + "\202\323\344\223\002H\"C/v2/{name=projects/*/instances/*"
+ + "/tables/*}:generateConsistencyToken:\001*\022\332"
+ + "\001\n\020CheckConsistency\0221.google.bigtable.ad"
+ + "min.v2.CheckConsistencyRequest\0322.google."
+ + "bigtable.admin.v2.CheckConsistencyRespon"
+ + "se\"_\332A\026name,consistency_token\202\323\344\223\002@\";/v2"
+ + "/{name=projects/*/instances/*/tables/*}:"
+ + "checkConsistency:\001*\022\352\001\n\rSnapshotTable\022.."
+ + "google.bigtable.admin.v2.SnapshotTableRe"
+ + "quest\032\035.google.longrunning.Operation\"\211\001\312"
+ + "A!\n\010Snapshot\022\025SnapshotTableMetadata\332A$na"
+ + "me,cluster,snapshot_id,description\202\323\344\223\0028"
+ + "\"3/v2/{name=projects/*/instances/*/table"
+ + "s/*}:snapshot:\001*\022\250\001\n\013GetSnapshot\022,.googl"
+ + "e.bigtable.admin.v2.GetSnapshotRequest\032\""
+ + ".google.bigtable.admin.v2.Snapshot\"G\332A\004n"
+ + "ame\202\323\344\223\002:\0228/v2/{name=projects/*/instance"
+ + "s/*/clusters/*/snapshots/*}\022\273\001\n\rListSnap"
+ + "shots\022..google.bigtable.admin.v2.ListSna"
+ + "pshotsRequest\032/.google.bigtable.admin.v2"
+ + ".ListSnapshotsResponse\"I\332A\006parent\202\323\344\223\002:\022"
+ + "8/v2/{parent=projects/*/instances/*/clus"
+ + "ters/*}/snapshots\022\242\001\n\016DeleteSnapshot\022/.g"
+ + "oogle.bigtable.admin.v2.DeleteSnapshotRe"
+ + "quest\032\026.google.protobuf.Empty\"G\332A\004name\202\323"
+ + "\344\223\002:*8/v2/{name=projects/*/instances/*/c"
+ + "lusters/*/snapshots/*}\022\340\001\n\014CreateBackup\022"
+ + "-.google.bigtable.admin.v2.CreateBackupR"
+ + "equest\032\035.google.longrunning.Operation\"\201\001"
+ + "\312A\036\n\006Backup\022\024CreateBackupMetadata\332A\027pare"
+ + "nt,backup_id,backup\202\323\344\223\002@\"6/v2/{parent=p"
+ + "rojects/*/instances/*/clusters/*}/backup"
+ + "s:\006backup\022\240\001\n\tGetBackup\022*.google.bigtabl"
+ + "e.admin.v2.GetBackupRequest\032 .google.big"
+ + "table.admin.v2.Backup\"E\332A\004name\202\323\344\223\0028\0226/v"
+ + "2/{name=projects/*/instances/*/clusters/"
+ + "*/backups/*}\022\303\001\n\014UpdateBackup\022-.google.b"
+ + "igtable.admin.v2.UpdateBackupRequest\032 .g"
+ + "oogle.bigtable.admin.v2.Backup\"b\332A\022backu"
+ + "p,update_mask\202\323\344\223\002G2=/v2/{backup.name=pr"
+ + "ojects/*/instances/*/clusters/*/backups/"
+ + "*}:\006backup\022\234\001\n\014DeleteBackup\022-.google.big"
+ + "table.admin.v2.DeleteBackupRequest\032\026.goo"
+ + "gle.protobuf.Empty\"E\332A\004name\202\323\344\223\0028*6/v2/{"
+ + "name=projects/*/instances/*/clusters/*/b"
+ + "ackups/*}\022\263\001\n\013ListBackups\022,.google.bigta"
+ + "ble.admin.v2.ListBackupsRequest\032-.google"
+ + ".bigtable.admin.v2.ListBackupsResponse\"G"
+ + "\332A\006parent\202\323\344\223\0028\0226/v2/{parent=projects/*/"
+ + "instances/*/clusters/*}/backups\022\273\001\n\014Rest"
+ + "oreTable\022-.google.bigtable.admin.v2.Rest"
+ + "oreTableRequest\032\035.google.longrunning.Ope"
+ + "ration\"]\312A\035\n\005Table\022\024RestoreTableMetadata"
+ + "\202\323\344\223\0027\"2/v2/{parent=projects/*/instances"
+ + "/*}/tables:restore:\001*\022\355\001\n\nCopyBackup\022+.g"
+ + "oogle.bigtable.admin.v2.CopyBackupReques"
+ + "t\032\035.google.longrunning.Operation\"\222\001\312A\034\n\006"
+ + "Backup\022\022CopyBackupMetadata\332A*parent,back"
+ + "up_id,source_backup,expire_time\202\323\344\223\002@\";/"
+ + "v2/{parent=projects/*/instances/*/cluste"
+ + "rs/*}/backups:copy:\001*\022\354\001\n\014GetIamPolicy\022\""
+ + ".google.iam.v1.GetIamPolicyRequest\032\025.goo"
+ + "gle.iam.v1.Policy\"\240\001\332A\010resource\202\323\344\223\002\216\001\";"
+ + "/v2/{resource=projects/*/instances/*/tab"
+ + "les/*}:getIamPolicy:\001*ZL\"G/v2/{resource="
+ + "projects/*/instances/*/clusters/*/backup"
+ + "s/*}:getIamPolicy:\001*\022\363\001\n\014SetIamPolicy\022\"."
+ + "google.iam.v1.SetIamPolicyRequest\032\025.goog"
+ + "le.iam.v1.Policy\"\247\001\332A\017resource,policy\202\323\344"
+ + "\223\002\216\001\";/v2/{resource=projects/*/instances"
+ + "/*/tables/*}:setIamPolicy:\001*ZL\"G/v2/{res"
+ + "ource=projects/*/instances/*/clusters/*/"
+ + "backups/*}:setIamPolicy:\001*\022\244\002\n\022TestIamPe"
+ + "rmissions\022(.google.iam.v1.TestIamPermiss"
+ + "ionsRequest\032).google.iam.v1.TestIamPermi"
+ + "ssionsResponse\"\270\001\332A\024resource,permissions"
+ + "\202\323\344\223\002\232\001\"A/v2/{resource=projects/*/instan"
+ + "ces/*/tables/*}:testIamPermissions:\001*ZR\""
+ + "M/v2/{resource=projects/*/instances/*/cl"
+ + "usters/*/backups/*}:testIamPermissions:\001"
+ + "*\032\336\002\312A\034bigtableadmin.googleapis.com\322A\273\002h"
+ + "ttps://www.googleapis.com/auth/bigtable."
+ + "admin,https://www.googleapis.com/auth/bi"
+ + "gtable.admin.table,https://www.googleapi"
+ + "s.com/auth/cloud-bigtable.admin,https://"
+ + "www.googleapis.com/auth/cloud-bigtable.a"
+ + "dmin.table,https://www.googleapis.com/au"
+ + "th/cloud-platform,https://www.googleapis"
+ + ".com/auth/cloud-platform.read-onlyB\337\001\n\034c"
+ + "om.google.bigtable.admin.v2B\027BigtableTab"
+ + "leAdminProtoP\001Z=google.golang.org/genpro"
+ + "to/googleapis/bigtable/admin/v2;admin\252\002\036"
+ + "Google.Cloud.Bigtable.Admin.V2\312\002\036Google\\"
+ + "Cloud\\Bigtable\\Admin\\V2\352\002\"Google::Cloud:"
+ + ":Bigtable::Admin::V2b\006proto3"
};
descriptor =
com.google.protobuf.Descriptors.FileDescriptor.internalBuildGeneratedFileFrom(
@@ -620,7 +621,7 @@ public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() {
new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
internal_static_google_bigtable_admin_v2_ModifyColumnFamiliesRequest_descriptor,
new java.lang.String[] {
- "Name", "Modifications",
+ "Name", "Modifications", "IgnoreWarnings",
});
internal_static_google_bigtable_admin_v2_ModifyColumnFamiliesRequest_Modification_descriptor =
internal_static_google_bigtable_admin_v2_ModifyColumnFamiliesRequest_descriptor
diff --git a/proto-google-cloud-bigtable-admin-v2/src/main/java/com/google/bigtable/admin/v2/ModifyColumnFamiliesRequest.java b/proto-google-cloud-bigtable-admin-v2/src/main/java/com/google/bigtable/admin/v2/ModifyColumnFamiliesRequest.java
index 5eaa481fee..6c0d53debd 100644
--- a/proto-google-cloud-bigtable-admin-v2/src/main/java/com/google/bigtable/admin/v2/ModifyColumnFamiliesRequest.java
+++ b/proto-google-cloud-bigtable-admin-v2/src/main/java/com/google/bigtable/admin/v2/ModifyColumnFamiliesRequest.java
@@ -1819,6 +1819,24 @@ public com.google.bigtable.admin.v2.ModifyColumnFamiliesRequest.Modification get
return modifications_.get(index);
}
+ public static final int IGNORE_WARNINGS_FIELD_NUMBER = 3;
+ private boolean ignoreWarnings_ = false;
+ /**
+ *
+ *
+ *
+ * Optional. If true, ignore safety checks when modifying the column families.
+ *
+ *
+ * bool ignore_warnings = 3 [(.google.api.field_behavior) = OPTIONAL];
+ *
+ * @return The ignoreWarnings.
+ */
+ @java.lang.Override
+ public boolean getIgnoreWarnings() {
+ return ignoreWarnings_;
+ }
+
private byte memoizedIsInitialized = -1;
@java.lang.Override
@@ -1839,6 +1857,9 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io
for (int i = 0; i < modifications_.size(); i++) {
output.writeMessage(2, modifications_.get(i));
}
+ if (ignoreWarnings_ != false) {
+ output.writeBool(3, ignoreWarnings_);
+ }
getUnknownFields().writeTo(output);
}
@@ -1854,6 +1875,9 @@ public int getSerializedSize() {
for (int i = 0; i < modifications_.size(); i++) {
size += com.google.protobuf.CodedOutputStream.computeMessageSize(2, modifications_.get(i));
}
+ if (ignoreWarnings_ != false) {
+ size += com.google.protobuf.CodedOutputStream.computeBoolSize(3, ignoreWarnings_);
+ }
size += getUnknownFields().getSerializedSize();
memoizedSize = size;
return size;
@@ -1872,6 +1896,7 @@ public boolean equals(final java.lang.Object obj) {
if (!getName().equals(other.getName())) return false;
if (!getModificationsList().equals(other.getModificationsList())) return false;
+ if (getIgnoreWarnings() != other.getIgnoreWarnings()) return false;
if (!getUnknownFields().equals(other.getUnknownFields())) return false;
return true;
}
@@ -1889,6 +1914,8 @@ public int hashCode() {
hash = (37 * hash) + MODIFICATIONS_FIELD_NUMBER;
hash = (53 * hash) + getModificationsList().hashCode();
}
+ hash = (37 * hash) + IGNORE_WARNINGS_FIELD_NUMBER;
+ hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(getIgnoreWarnings());
hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash;
return hash;
@@ -2038,6 +2065,7 @@ public Builder clear() {
modificationsBuilder_.clear();
}
bitField0_ = (bitField0_ & ~0x00000002);
+ ignoreWarnings_ = false;
return this;
}
@@ -2091,6 +2119,9 @@ private void buildPartial0(com.google.bigtable.admin.v2.ModifyColumnFamiliesRequ
if (((from_bitField0_ & 0x00000001) != 0)) {
result.name_ = name_;
}
+ if (((from_bitField0_ & 0x00000004) != 0)) {
+ result.ignoreWarnings_ = ignoreWarnings_;
+ }
}
@java.lang.Override
@@ -2171,6 +2202,9 @@ public Builder mergeFrom(com.google.bigtable.admin.v2.ModifyColumnFamiliesReques
}
}
}
+ if (other.getIgnoreWarnings() != false) {
+ setIgnoreWarnings(other.getIgnoreWarnings());
+ }
this.mergeUnknownFields(other.getUnknownFields());
onChanged();
return this;
@@ -2218,6 +2252,12 @@ public Builder mergeFrom(
}
break;
} // case 18
+ case 24:
+ {
+ ignoreWarnings_ = input.readBool();
+ bitField0_ |= 0x00000004;
+ break;
+ } // case 24
default:
{
if (!super.parseUnknownField(input, extensionRegistry, tag)) {
@@ -2833,6 +2873,59 @@ public Builder removeModifications(int index) {
return modificationsBuilder_;
}
+ private boolean ignoreWarnings_;
+ /**
+ *
+ *
+ *
+ * Optional. If true, ignore safety checks when modifying the column families.
+ *
+ *
+ * bool ignore_warnings = 3 [(.google.api.field_behavior) = OPTIONAL];
+ *
+ * @return The ignoreWarnings.
+ */
+ @java.lang.Override
+ public boolean getIgnoreWarnings() {
+ return ignoreWarnings_;
+ }
+ /**
+ *
+ *
+ *
+ * Optional. If true, ignore safety checks when modifying the column families.
+ *
+ *
+ * bool ignore_warnings = 3 [(.google.api.field_behavior) = OPTIONAL];
+ *
+ * @param value The ignoreWarnings to set.
+ * @return This builder for chaining.
+ */
+ public Builder setIgnoreWarnings(boolean value) {
+
+ ignoreWarnings_ = value;
+ bitField0_ |= 0x00000004;
+ onChanged();
+ return this;
+ }
+ /**
+ *
+ *
+ *
+ * Optional. If true, ignore safety checks when modifying the column families.
+ *
+ *
+ * bool ignore_warnings = 3 [(.google.api.field_behavior) = OPTIONAL];
+ *
+ * @return This builder for chaining.
+ */
+ public Builder clearIgnoreWarnings() {
+ bitField0_ = (bitField0_ & ~0x00000004);
+ ignoreWarnings_ = false;
+ onChanged();
+ return this;
+ }
+
@java.lang.Override
public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) {
return super.setUnknownFields(unknownFields);
diff --git a/proto-google-cloud-bigtable-admin-v2/src/main/java/com/google/bigtable/admin/v2/ModifyColumnFamiliesRequestOrBuilder.java b/proto-google-cloud-bigtable-admin-v2/src/main/java/com/google/bigtable/admin/v2/ModifyColumnFamiliesRequestOrBuilder.java
index e386edffea..382407649d 100644
--- a/proto-google-cloud-bigtable-admin-v2/src/main/java/com/google/bigtable/admin/v2/ModifyColumnFamiliesRequestOrBuilder.java
+++ b/proto-google-cloud-bigtable-admin-v2/src/main/java/com/google/bigtable/admin/v2/ModifyColumnFamiliesRequestOrBuilder.java
@@ -135,4 +135,17 @@ public interface ModifyColumnFamiliesRequestOrBuilder
*/
com.google.bigtable.admin.v2.ModifyColumnFamiliesRequest.ModificationOrBuilder
getModificationsOrBuilder(int index);
+
+ /**
+ *
+ *
+ *
+ * Optional. If true, ignore safety checks when modifying the column families.
+ *
+ *
+ * bool ignore_warnings = 3 [(.google.api.field_behavior) = OPTIONAL];
+ *
+ * @return The ignoreWarnings.
+ */
+ boolean getIgnoreWarnings();
}
diff --git a/proto-google-cloud-bigtable-admin-v2/src/main/proto/google/bigtable/admin/v2/bigtable_table_admin.proto b/proto-google-cloud-bigtable-admin-v2/src/main/proto/google/bigtable/admin/v2/bigtable_table_admin.proto
index 8c516abe7a..62cd7d6555 100644
--- a/proto-google-cloud-bigtable-admin-v2/src/main/proto/google/bigtable/admin/v2/bigtable_table_admin.proto
+++ b/proto-google-cloud-bigtable-admin-v2/src/main/proto/google/bigtable/admin/v2/bigtable_table_admin.proto
@@ -734,6 +734,9 @@ message ModifyColumnFamiliesRequest {
// family, for example).
repeated Modification modifications = 2
[(google.api.field_behavior) = REQUIRED];
+
+ // Optional. If true, ignore safety checks when modifying the column families.
+ bool ignore_warnings = 3 [(google.api.field_behavior) = OPTIONAL];
}
// Request message for
diff --git a/proto-google-cloud-bigtable-v2/pom.xml b/proto-google-cloud-bigtable-v2/pom.xml
index b59ed7b835..9949e24f5f 100644
--- a/proto-google-cloud-bigtable-v2/pom.xml
+++ b/proto-google-cloud-bigtable-v2/pom.xml
@@ -4,13 +4,13 @@
4.0.0
com.google.api.grpc
proto-google-cloud-bigtable-v2
- 2.30.0
+ 2.31.0
proto-google-cloud-bigtable-v2
PROTO library for proto-google-cloud-bigtable-v2
com.google.cloud
google-cloud-bigtable-parent
- 2.30.0
+ 2.31.0
@@ -18,14 +18,14 @@
com.google.cloud
google-cloud-bigtable-deps-bom
- 2.30.0
+ 2.31.0
pom
import
com.google.cloud
google-cloud-bigtable-bom
- 2.30.0
+ 2.31.0
pom
import
diff --git a/proto-google-cloud-bigtable-v2/src/main/java/com/google/bigtable/v2/FeatureFlags.java b/proto-google-cloud-bigtable-v2/src/main/java/com/google/bigtable/v2/FeatureFlags.java
index 953adab411..ed3a97f9a8 100644
--- a/proto-google-cloud-bigtable-v2/src/main/java/com/google/bigtable/v2/FeatureFlags.java
+++ b/proto-google-cloud-bigtable-v2/src/main/java/com/google/bigtable/v2/FeatureFlags.java
@@ -145,6 +145,44 @@ public boolean getLastScannedRowResponses() {
return lastScannedRowResponses_;
}
+ public static final int ROUTING_COOKIE_FIELD_NUMBER = 6;
+ private boolean routingCookie_ = false;
+ /**
+ *
+ *
+ *
+ * Notify the server that the client supports using encoded routing cookie
+ * strings to retry requests with.
+ *
+ *
+ * bool routing_cookie = 6;
+ *
+ * @return The routingCookie.
+ */
+ @java.lang.Override
+ public boolean getRoutingCookie() {
+ return routingCookie_;
+ }
+
+ public static final int RETRY_INFO_FIELD_NUMBER = 7;
+ private boolean retryInfo_ = false;
+ /**
+ *
+ *
+ *
+ * Notify the server that the client supports using retry info back off
+ * durations to retry requests with.
+ *
+ *
+ * bool retry_info = 7;
+ *
+ * @return The retryInfo.
+ */
+ @java.lang.Override
+ public boolean getRetryInfo() {
+ return retryInfo_;
+ }
+
private byte memoizedIsInitialized = -1;
@java.lang.Override
@@ -171,6 +209,12 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io
if (mutateRowsRateLimit2_ != false) {
output.writeBool(5, mutateRowsRateLimit2_);
}
+ if (routingCookie_ != false) {
+ output.writeBool(6, routingCookie_);
+ }
+ if (retryInfo_ != false) {
+ output.writeBool(7, retryInfo_);
+ }
getUnknownFields().writeTo(output);
}
@@ -192,6 +236,12 @@ public int getSerializedSize() {
if (mutateRowsRateLimit2_ != false) {
size += com.google.protobuf.CodedOutputStream.computeBoolSize(5, mutateRowsRateLimit2_);
}
+ if (routingCookie_ != false) {
+ size += com.google.protobuf.CodedOutputStream.computeBoolSize(6, routingCookie_);
+ }
+ if (retryInfo_ != false) {
+ size += com.google.protobuf.CodedOutputStream.computeBoolSize(7, retryInfo_);
+ }
size += getUnknownFields().getSerializedSize();
memoizedSize = size;
return size;
@@ -211,6 +261,8 @@ public boolean equals(final java.lang.Object obj) {
if (getMutateRowsRateLimit() != other.getMutateRowsRateLimit()) return false;
if (getMutateRowsRateLimit2() != other.getMutateRowsRateLimit2()) return false;
if (getLastScannedRowResponses() != other.getLastScannedRowResponses()) return false;
+ if (getRoutingCookie() != other.getRoutingCookie()) return false;
+ if (getRetryInfo() != other.getRetryInfo()) return false;
if (!getUnknownFields().equals(other.getUnknownFields())) return false;
return true;
}
@@ -230,6 +282,10 @@ public int hashCode() {
hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(getMutateRowsRateLimit2());
hash = (37 * hash) + LAST_SCANNED_ROW_RESPONSES_FIELD_NUMBER;
hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(getLastScannedRowResponses());
+ hash = (37 * hash) + ROUTING_COOKIE_FIELD_NUMBER;
+ hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(getRoutingCookie());
+ hash = (37 * hash) + RETRY_INFO_FIELD_NUMBER;
+ hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(getRetryInfo());
hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash;
return hash;
@@ -379,6 +435,8 @@ public Builder clear() {
mutateRowsRateLimit_ = false;
mutateRowsRateLimit2_ = false;
lastScannedRowResponses_ = false;
+ routingCookie_ = false;
+ retryInfo_ = false;
return this;
}
@@ -426,6 +484,12 @@ private void buildPartial0(com.google.bigtable.v2.FeatureFlags result) {
if (((from_bitField0_ & 0x00000008) != 0)) {
result.lastScannedRowResponses_ = lastScannedRowResponses_;
}
+ if (((from_bitField0_ & 0x00000010) != 0)) {
+ result.routingCookie_ = routingCookie_;
+ }
+ if (((from_bitField0_ & 0x00000020) != 0)) {
+ result.retryInfo_ = retryInfo_;
+ }
}
@java.lang.Override
@@ -485,6 +549,12 @@ public Builder mergeFrom(com.google.bigtable.v2.FeatureFlags other) {
if (other.getLastScannedRowResponses() != false) {
setLastScannedRowResponses(other.getLastScannedRowResponses());
}
+ if (other.getRoutingCookie() != false) {
+ setRoutingCookie(other.getRoutingCookie());
+ }
+ if (other.getRetryInfo() != false) {
+ setRetryInfo(other.getRetryInfo());
+ }
this.mergeUnknownFields(other.getUnknownFields());
onChanged();
return this;
@@ -535,6 +605,18 @@ public Builder mergeFrom(
bitField0_ |= 0x00000004;
break;
} // case 40
+ case 48:
+ {
+ routingCookie_ = input.readBool();
+ bitField0_ |= 0x00000010;
+ break;
+ } // case 48
+ case 56:
+ {
+ retryInfo_ = input.readBool();
+ bitField0_ |= 0x00000020;
+ break;
+ } // case 56
default:
{
if (!super.parseUnknownField(input, extensionRegistry, tag)) {
@@ -784,6 +866,118 @@ public Builder clearLastScannedRowResponses() {
return this;
}
+ private boolean routingCookie_;
+ /**
+ *
+ *
+ *
+ * Notify the server that the client supports using encoded routing cookie
+ * strings to retry requests with.
+ *
+ *
+ * bool routing_cookie = 6;
+ *
+ * @return The routingCookie.
+ */
+ @java.lang.Override
+ public boolean getRoutingCookie() {
+ return routingCookie_;
+ }
+ /**
+ *
+ *
+ *
+ * Notify the server that the client supports using encoded routing cookie
+ * strings to retry requests with.
+ *
+ *
+ * bool routing_cookie = 6;
+ *
+ * @param value The routingCookie to set.
+ * @return This builder for chaining.
+ */
+ public Builder setRoutingCookie(boolean value) {
+
+ routingCookie_ = value;
+ bitField0_ |= 0x00000010;
+ onChanged();
+ return this;
+ }
+ /**
+ *
+ *
+ *
+ * Notify the server that the client supports using encoded routing cookie
+ * strings to retry requests with.
+ *
+ *
+ * bool routing_cookie = 6;
+ *
+ * @return This builder for chaining.
+ */
+ public Builder clearRoutingCookie() {
+ bitField0_ = (bitField0_ & ~0x00000010);
+ routingCookie_ = false;
+ onChanged();
+ return this;
+ }
+
+ private boolean retryInfo_;
+ /**
+ *
+ *
+ *
+ * Notify the server that the client supports using retry info back off
+ * durations to retry requests with.
+ *
+ *
+ * bool retry_info = 7;
+ *
+ * @return The retryInfo.
+ */
+ @java.lang.Override
+ public boolean getRetryInfo() {
+ return retryInfo_;
+ }
+ /**
+ *
+ *
+ *
+ * Notify the server that the client supports using retry info back off
+ * durations to retry requests with.
+ *
+ *
+ * bool retry_info = 7;
+ *
+ * @param value The retryInfo to set.
+ * @return This builder for chaining.
+ */
+ public Builder setRetryInfo(boolean value) {
+
+ retryInfo_ = value;
+ bitField0_ |= 0x00000020;
+ onChanged();
+ return this;
+ }
+ /**
+ *
+ *
+ *
+ * Notify the server that the client supports using retry info back off
+ * durations to retry requests with.
+ *
+ *
+ * bool retry_info = 7;
+ *
+ * @return This builder for chaining.
+ */
+ public Builder clearRetryInfo() {
+ bitField0_ = (bitField0_ & ~0x00000020);
+ retryInfo_ = false;
+ onChanged();
+ return this;
+ }
+
@java.lang.Override
public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) {
return super.setUnknownFields(unknownFields);
diff --git a/proto-google-cloud-bigtable-v2/src/main/java/com/google/bigtable/v2/FeatureFlagsOrBuilder.java b/proto-google-cloud-bigtable-v2/src/main/java/com/google/bigtable/v2/FeatureFlagsOrBuilder.java
index 0696b9d05a..40dc1a2ad9 100644
--- a/proto-google-cloud-bigtable-v2/src/main/java/com/google/bigtable/v2/FeatureFlagsOrBuilder.java
+++ b/proto-google-cloud-bigtable-v2/src/main/java/com/google/bigtable/v2/FeatureFlagsOrBuilder.java
@@ -80,4 +80,32 @@ public interface FeatureFlagsOrBuilder
* @return The lastScannedRowResponses.
*/
boolean getLastScannedRowResponses();
+
+ /**
+ *
+ *
+ *
+ * Notify the server that the client supports using encoded routing cookie
+ * strings to retry requests with.
+ *
+ *
+ * bool routing_cookie = 6;
+ *
+ * @return The routingCookie.
+ */
+ boolean getRoutingCookie();
+
+ /**
+ *
+ *
+ *
+ * Notify the server that the client supports using retry info back off
+ * durations to retry requests with.
+ *
+ *
+ * bool retry_info = 7;
+ *
+ * @return The retryInfo.
+ */
+ boolean getRetryInfo();
}
diff --git a/proto-google-cloud-bigtable-v2/src/main/java/com/google/bigtable/v2/FeatureFlagsProto.java b/proto-google-cloud-bigtable-v2/src/main/java/com/google/bigtable/v2/FeatureFlagsProto.java
index 91b2c97e8c..2072fb2852 100644
--- a/proto-google-cloud-bigtable-v2/src/main/java/com/google/bigtable/v2/FeatureFlagsProto.java
+++ b/proto-google-cloud-bigtable-v2/src/main/java/com/google/bigtable/v2/FeatureFlagsProto.java
@@ -41,16 +41,17 @@ public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() {
static {
java.lang.String[] descriptorData = {
"\n&google/bigtable/v2/feature_flags.proto"
- + "\022\022google.bigtable.v2\"\212\001\n\014FeatureFlags\022\025\n"
+ + "\022\022google.bigtable.v2\"\266\001\n\014FeatureFlags\022\025\n"
+ "\rreverse_scans\030\001 \001(\010\022\036\n\026mutate_rows_rate"
+ "_limit\030\003 \001(\010\022\037\n\027mutate_rows_rate_limit2\030"
+ "\005 \001(\010\022\"\n\032last_scanned_row_responses\030\004 \001("
- + "\010B\275\001\n\026com.google.bigtable.v2B\021FeatureFla"
- + "gsProtoP\001Z:google.golang.org/genproto/go"
- + "ogleapis/bigtable/v2;bigtable\252\002\030Google.C"
- + "loud.Bigtable.V2\312\002\030Google\\Cloud\\Bigtable"
- + "\\V2\352\002\033Google::Cloud::Bigtable::V2b\006proto"
- + "3"
+ + "\010\022\026\n\016routing_cookie\030\006 \001(\010\022\022\n\nretry_info\030"
+ + "\007 \001(\010B\275\001\n\026com.google.bigtable.v2B\021Featur"
+ + "eFlagsProtoP\001Z:google.golang.org/genprot"
+ + "o/googleapis/bigtable/v2;bigtable\252\002\030Goog"
+ + "le.Cloud.Bigtable.V2\312\002\030Google\\Cloud\\Bigt"
+ + "able\\V2\352\002\033Google::Cloud::Bigtable::V2b\006p"
+ + "roto3"
};
descriptor =
com.google.protobuf.Descriptors.FileDescriptor.internalBuildGeneratedFileFrom(
@@ -65,6 +66,8 @@ public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() {
"MutateRowsRateLimit",
"MutateRowsRateLimit2",
"LastScannedRowResponses",
+ "RoutingCookie",
+ "RetryInfo",
});
}
diff --git a/proto-google-cloud-bigtable-v2/src/main/proto/google/bigtable/v2/feature_flags.proto b/proto-google-cloud-bigtable-v2/src/main/proto/google/bigtable/v2/feature_flags.proto
index 942646c6b5..d3128c5c67 100644
--- a/proto-google-cloud-bigtable-v2/src/main/proto/google/bigtable/v2/feature_flags.proto
+++ b/proto-google-cloud-bigtable-v2/src/main/proto/google/bigtable/v2/feature_flags.proto
@@ -50,4 +50,12 @@ message FeatureFlags {
// Notify the server that the client supports the last_scanned_row field
// in ReadRowsResponse for long-running scans.
bool last_scanned_row_responses = 4;
+
+ // Notify the server that the client supports using encoded routing cookie
+ // strings to retry requests with.
+ bool routing_cookie = 6;
+
+ // Notify the server that the client supports using retry info back off
+ // durations to retry requests with.
+ bool retry_info = 7;
}
diff --git a/samples/install-without-bom/pom.xml b/samples/install-without-bom/pom.xml
index e7462e744d..6209a9e2ff 100644
--- a/samples/install-without-bom/pom.xml
+++ b/samples/install-without-bom/pom.xml
@@ -29,7 +29,7 @@
com.google.cloud
google-cloud-bigtable
- 2.29.1
+ 2.30.0
@@ -42,7 +42,7 @@
com.google.truth
truth
- 1.1.5
+ 1.2.0
test
diff --git a/samples/native-image-sample/pom.xml b/samples/native-image-sample/pom.xml
index 377134cef1..2350b0d0f4 100644
--- a/samples/native-image-sample/pom.xml
+++ b/samples/native-image-sample/pom.xml
@@ -52,7 +52,7 @@
com.google.truth
truth
- 1.1.5
+ 1.2.0
test
diff --git a/samples/snapshot/pom.xml b/samples/snapshot/pom.xml
index 7bd6ab705d..b7fad804aa 100644
--- a/samples/snapshot/pom.xml
+++ b/samples/snapshot/pom.xml
@@ -28,7 +28,7 @@
com.google.cloud
google-cloud-bigtable
- 2.30.0
+ 2.31.0
@@ -41,7 +41,7 @@
com.google.truth
truth
- 1.1.5
+ 1.2.0
test
diff --git a/samples/snippets/pom.xml b/samples/snippets/pom.xml
index b5e05baa96..388cd638b5 100644
--- a/samples/snippets/pom.xml
+++ b/samples/snippets/pom.xml
@@ -53,7 +53,7 @@
com.google.truth
truth
- 1.1.5
+ 1.2.0
test
diff --git a/test-proxy/pom.xml b/test-proxy/pom.xml
index c3c1549277..3990f657ef 100644
--- a/test-proxy/pom.xml
+++ b/test-proxy/pom.xml
@@ -12,11 +12,11 @@
google-cloud-bigtable-parent
com.google.cloud
- 2.30.0
+ 2.31.0
- 2.30.0
+ 2.31.0
diff --git a/test-proxy/src/main/java/com/google/cloud/bigtable/testproxy/CbtTestProxy.java b/test-proxy/src/main/java/com/google/cloud/bigtable/testproxy/CbtTestProxy.java
index 2ebb609388..6e563d4df0 100644
--- a/test-proxy/src/main/java/com/google/cloud/bigtable/testproxy/CbtTestProxy.java
+++ b/test-proxy/src/main/java/com/google/cloud/bigtable/testproxy/CbtTestProxy.java
@@ -208,6 +208,9 @@ public synchronized void createClient(
.setInstanceId(request.getInstanceId())
.setAppProfileId(request.getAppProfileId());
+ settingsBuilder.stubSettings().setEnableRoutingCookie(false);
+ settingsBuilder.stubSettings().setEnableRetryInfo(false);
+
if (request.hasPerOperationTimeout()) {
Duration newTimeout = Duration.ofMillis(Durations.toMillis(request.getPerOperationTimeout()));
settingsBuilder = overrideTimeoutSetting(newTimeout, settingsBuilder);
@@ -231,6 +234,8 @@ public synchronized void createClient(
}
}
settingsBuilder.stubSettings().bulkMutateRowsSettings().setServerInitiatedFlowControl(true);
+ settingsBuilder.stubSettings().setEnableRoutingCookie(true);
+ settingsBuilder.stubSettings().setEnableRetryInfo(true);
}
// Create and store CbtClient for later use
diff --git a/versions.txt b/versions.txt
index d46263f551..d652a91293 100644
--- a/versions.txt
+++ b/versions.txt
@@ -1,10 +1,10 @@
# Format:
# module:released-version:current-version
-google-cloud-bigtable:2.30.0:2.30.0
-grpc-google-cloud-bigtable-admin-v2:2.30.0:2.30.0
-grpc-google-cloud-bigtable-v2:2.30.0:2.30.0
-proto-google-cloud-bigtable-admin-v2:2.30.0:2.30.0
-proto-google-cloud-bigtable-v2:2.30.0:2.30.0
-google-cloud-bigtable-emulator:0.167.0:0.167.0
-google-cloud-bigtable-emulator-core:0.167.0:0.167.0
+google-cloud-bigtable:2.31.0:2.31.0
+grpc-google-cloud-bigtable-admin-v2:2.31.0:2.31.0
+grpc-google-cloud-bigtable-v2:2.31.0:2.31.0
+proto-google-cloud-bigtable-admin-v2:2.31.0:2.31.0
+proto-google-cloud-bigtable-v2:2.31.0:2.31.0
+google-cloud-bigtable-emulator:0.168.0:0.168.0
+google-cloud-bigtable-emulator-core:0.168.0:0.168.0