extract(MutateRowRequest mutateRowRequest) {
}
/**
- * Internal helper to create the base MutateRows callable chain. The chain is responsible for
- * retrying individual entry in case of error.
+ * Creates a callable chain to handle MutatesRows RPCs. This is meant to be used for manual
+ * batching. The chain will:
*
- * NOTE: the caller is responsible for adding tracing & metrics.
+ *
+ * - Convert a {@link BulkMutation} into a {@link MutateRowsRequest}.
+ *
- Process the response and schedule retries. At the end of each attempt, entries that have
+ * been applied, are filtered from the next attempt. Also, any entries that failed with a
+ * nontransient error, are filtered from the next attempt. This will continue until there
+ * are no more entries or there are no more retry attempts left.
+ *
- Wrap batch failures in a {@link MutateRowsAttemptResult}.
+ *
- Add tracing & metrics.
+ *
+ *
+ * This callable returns an internal type {@link MutateRowsAttemptResult}.
*
- * @see MutateRowsRetryingCallable for more details
+ * This function should not be exposed to external users, as it could cause a data loss.
*/
- private UnaryCallable createMutateRowsBaseCallable() {
+ private UnaryCallable createMutateRowsBaseCallable() {
ServerStreamingCallable base =
GrpcRawCallableFactory.createServerStreamingCallable(
GrpcCallSettings.newBuilder()
@@ -706,55 +721,38 @@ public Map extract(MutateRowsRequest mutateRowsRequest) {
ServerStreamingCallable withBigtableTracer =
new BigtableTracerStreamingCallable<>(convertException);
- BasicResultRetryAlgorithm resultRetryAlgorithm;
+ BasicResultRetryAlgorithm resultRetryAlgorithm;
if (settings.getEnableRetryInfo()) {
resultRetryAlgorithm = new RetryInfoRetryAlgorithm<>();
} else {
resultRetryAlgorithm = new ApiResultRetryAlgorithm<>();
}
+ MutateRowsPartialErrorRetryAlgorithm mutateRowsPartialErrorRetryAlgorithm =
+ new MutateRowsPartialErrorRetryAlgorithm(resultRetryAlgorithm);
- RetryAlgorithm retryAlgorithm =
+ RetryAlgorithm retryAlgorithm =
new RetryAlgorithm<>(
- resultRetryAlgorithm,
+ mutateRowsPartialErrorRetryAlgorithm,
new ExponentialRetryAlgorithm(
settings.bulkMutateRowsSettings().getRetrySettings(), clientContext.getClock()));
- RetryingExecutorWithContext retryingExecutor =
+ RetryingExecutorWithContext retryingExecutor =
new ScheduledRetryingExecutor<>(retryAlgorithm, clientContext.getExecutor());
+ UnaryCallable baseCallable =
+ new MutateRowsRetryingCallable(
+ clientContext.getDefaultCallContext(),
+ withBigtableTracer,
+ retryingExecutor,
+ settings.bulkMutateRowsSettings().getRetryableCodes(),
+ retryAlgorithm);
- return new MutateRowsRetryingCallable(
- clientContext.getDefaultCallContext(),
- withBigtableTracer,
- retryingExecutor,
- settings.bulkMutateRowsSettings().getRetryableCodes(),
- retryAlgorithm);
- }
-
- /**
- * Creates a callable chain to handle MutatesRows RPCs. This is meant to be used for manual
- * batching. The chain will:
- *
- *
- * - Convert a {@link BulkMutation} into a {@link MutateRowsRequest}.
- *
- Process the response and schedule retries. At the end of each attempt, entries that have
- * been applied, are filtered from the next attempt. Also, any entries that failed with a
- * nontransient error, are filtered from the next attempt. This will continue until there
- * are no more entries or there are no more retry attempts left.
- *
- Wrap batch failures in a {@link
- * com.google.cloud.bigtable.data.v2.models.MutateRowsException}.
- *
- Add tracing & metrics.
- *
- */
- private UnaryCallable createBulkMutateRowsCallable() {
- UnaryCallable baseCallable = createMutateRowsBaseCallable();
-
- UnaryCallable withCookie = baseCallable;
+ UnaryCallable withCookie = baseCallable;
if (settings.getEnableRoutingCookie()) {
withCookie = new CookiesUnaryCallable<>(baseCallable);
}
- UnaryCallable flowControlCallable = null;
+ UnaryCallable flowControlCallable = null;
if (settings.bulkMutateRowsSettings().isLatencyBasedThrottlingEnabled()) {
flowControlCallable =
new DynamicFlowControlCallable(
@@ -764,16 +762,16 @@ private UnaryCallable createBulkMutateRowsCallable() {
settings.bulkMutateRowsSettings().getTargetRpcLatencyMs(),
FLOW_CONTROL_ADJUSTING_INTERVAL_MS);
}
- UnaryCallable userFacing =
+ UnaryCallable userFacing =
new BulkMutateRowsUserFacingCallable(
flowControlCallable != null ? flowControlCallable : withCookie, requestContext);
SpanName spanName = getSpanName("MutateRows");
- UnaryCallable tracedBatcherUnaryCallable =
+ UnaryCallable tracedBatcherUnaryCallable =
new TracedBatcherUnaryCallable<>(userFacing);
- UnaryCallable traced =
+ UnaryCallable traced =
new TracedUnaryCallable<>(
tracedBatcherUnaryCallable, clientContext.getTracerFactory(), spanName);
@@ -1171,10 +1169,15 @@ public UnaryCallable mutateRowCallable() {
}
/**
- * Returns the callable chain created in {@link #createBulkMutateRowsCallable()} ()} during stub
+ * Returns the callable chain created in {@link #createMutateRowsBaseCallable()} during stub
* construction.
*/
public UnaryCallable bulkMutateRowsCallable() {
+ return externalBulkMutateRowsCallable;
+ }
+
+ @InternalApi
+ public UnaryCallable internalBulkMutateRowsCallable() {
return bulkMutateRowsCallable;
}
diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/MutateRowsErrorConverterUnaryCallable.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/MutateRowsErrorConverterUnaryCallable.java
new file mode 100644
index 0000000000..2b118df61e
--- /dev/null
+++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/MutateRowsErrorConverterUnaryCallable.java
@@ -0,0 +1,61 @@
+/*
+ * 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.stub;
+
+import com.google.api.core.ApiFuture;
+import com.google.api.core.ApiFutures;
+import com.google.api.core.InternalApi;
+import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.rpc.ApiCallContext;
+import com.google.api.gax.rpc.UnaryCallable;
+import com.google.cloud.bigtable.data.v2.models.BulkMutation;
+import com.google.cloud.bigtable.data.v2.models.MutateRowsException;
+import com.google.cloud.bigtable.data.v2.stub.mutaterows.MutateRowsAttemptResult;
+import com.google.common.util.concurrent.MoreExecutors;
+import io.grpc.Status;
+
+/**
+ * This callable converts partial batch failures into an exception. This is necessary to make sure
+ * that the caller properly handles issues and avoids possible data loss on partial failures
+ */
+@InternalApi
+public class MutateRowsErrorConverterUnaryCallable extends UnaryCallable {
+
+ private final UnaryCallable innerCallable;
+
+ public MutateRowsErrorConverterUnaryCallable(
+ UnaryCallable callable) {
+ this.innerCallable = callable;
+ }
+
+ @Override
+ public ApiFuture futureCall(BulkMutation request, ApiCallContext context) {
+ ApiFuture future = innerCallable.futureCall(request, context);
+ return ApiFutures.transform(
+ future,
+ result -> {
+ if (!result.getFailedMutations().isEmpty()) {
+ throw MutateRowsException.create(
+ null,
+ GrpcStatusCode.of(Status.Code.OK),
+ result.getFailedMutations(),
+ result.getIsRetryable());
+ }
+ return null;
+ },
+ MoreExecutors.directExecutor());
+ }
+}
diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/BulkMutateRowsUserFacingCallable.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/BulkMutateRowsUserFacingCallable.java
index 8048cceaad..94980a80a2 100644
--- a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/BulkMutateRowsUserFacingCallable.java
+++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/BulkMutateRowsUserFacingCallable.java
@@ -30,18 +30,22 @@
* applications.
*/
@InternalApi
-public final class BulkMutateRowsUserFacingCallable extends UnaryCallable {
- private final UnaryCallable innerCallable;
+public final class BulkMutateRowsUserFacingCallable
+ extends UnaryCallable {
+
+ private final UnaryCallable innerCallable;
private final RequestContext requestContext;
public BulkMutateRowsUserFacingCallable(
- UnaryCallable innerCallable, RequestContext requestContext) {
+ UnaryCallable innerCallable,
+ RequestContext requestContext) {
this.innerCallable = innerCallable;
this.requestContext = requestContext;
}
@Override
- public ApiFuture futureCall(BulkMutation request, ApiCallContext context) {
+ public ApiFuture futureCall(
+ BulkMutation request, ApiCallContext context) {
return innerCallable.futureCall(request.toProto(requestContext), context);
}
}
diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsAttemptCallable.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsAttemptCallable.java
index 155ea43211..b07e67ba94 100644
--- a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsAttemptCallable.java
+++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsAttemptCallable.java
@@ -87,7 +87,7 @@
*
* Package-private for internal use.
*/
-class MutateRowsAttemptCallable implements Callable {
+class MutateRowsAttemptCallable implements Callable {
// Synthetic status for Mutations that didn't get a result (because the whole RPC failed). It will
// be exposed in MutateRowsException's FailedMutations.
private static final StatusCode LOCAL_UNKNOWN_STATUS =
@@ -116,17 +116,17 @@ public Object getTransportCode() {
@Nonnull private TimedAttemptSettings attemptSettings;
// Parent controller
- private RetryingFuture externalFuture;
+ private RetryingFuture externalFuture;
// Simple wrappers for handling result futures
- private final ApiFunction, Void> attemptSuccessfulCallback =
- new ApiFunction, Void>() {
- @Override
- public Void apply(List responses) {
- handleAttemptSuccess(responses);
- return null;
- }
- };
+ private final ApiFunction, MutateRowsAttemptResult>
+ attemptSuccessfulCallback =
+ new ApiFunction, MutateRowsAttemptResult>() {
+ @Override
+ public MutateRowsAttemptResult apply(List responses) {
+ return handleAttemptSuccess(responses);
+ }
+ };
private final ApiFunction> attemptFailedCallback =
new ApiFunction>() {
@@ -153,7 +153,7 @@ public List apply(Throwable throwable) {
permanentFailures = Lists.newArrayList();
}
- public void setExternalFuture(RetryingFuture externalFuture) {
+ public void setExternalFuture(RetryingFuture externalFuture) {
this.externalFuture = externalFuture;
}
@@ -166,7 +166,7 @@ public void setExternalFuture(RetryingFuture externalFuture) {
* return of this method should just be ignored.
*/
@Override
- public Void call() {
+ public MutateRowsAttemptResult call() {
try {
// externalFuture is set from MutateRowsRetryingCallable before invoking this method. It
// shouldn't be null unless the code changed
@@ -192,7 +192,7 @@ public Void call() {
}
// Handle concurrent cancellation
- externalFuture.setAttemptFuture(new NonCancellableFuture());
+ externalFuture.setAttemptFuture(new NonCancellableFuture<>());
if (externalFuture.isDone()) {
return null;
}
@@ -208,13 +208,13 @@ public Void call() {
// Inspect the results and either propagate the success, or prepare to retry the failed
// mutations
- ApiFuture transformed =
+ ApiFuture transformed =
ApiFutures.transform(catching, attemptSuccessfulCallback, MoreExecutors.directExecutor());
// Notify the parent of the attempt
externalFuture.setAttemptFuture(transformed);
} catch (Throwable e) {
- externalFuture.setAttemptFuture(ApiFutures.immediateFailedFuture(e));
+ externalFuture.setAttemptFuture(ApiFutures.immediateFailedFuture(e));
}
return null;
@@ -257,7 +257,8 @@ private void handleAttemptError(Throwable rpcError) {
currentRequest = builder.build();
originalIndexes = newOriginalIndexes;
- throw MutateRowsException.create(rpcError, allFailures.build(), builder.getEntriesCount() > 0);
+ throw MutateRowsException.create(
+ rpcError, entryError.getStatusCode(), allFailures.build(), builder.getEntriesCount() > 0);
}
/**
@@ -267,7 +268,7 @@ private void handleAttemptError(Throwable rpcError) {
* {@link MutateRowsException}. If no errors exist, then the attempt future is successfully
* completed. We don't currently handle RetryInfo on entry level failures.
*/
- private void handleAttemptSuccess(List responses) {
+ private MutateRowsAttemptResult handleAttemptSuccess(List responses) {
List allFailures = Lists.newArrayList(permanentFailures);
MutateRowsRequest lastRequest = currentRequest;
@@ -326,8 +327,9 @@ private void handleAttemptSuccess(List responses) {
if (!allFailures.isEmpty()) {
boolean isRetryable = builder.getEntriesCount() > 0;
- throw MutateRowsException.create(null, allFailures, isRetryable);
+ return MutateRowsAttemptResult.create(allFailures, isRetryable);
}
+ return MutateRowsAttemptResult.success();
}
/**
diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsAttemptResult.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsAttemptResult.java
new file mode 100644
index 0000000000..d668c2a50f
--- /dev/null
+++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsAttemptResult.java
@@ -0,0 +1,49 @@
+/*
+ * 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.stub.mutaterows;
+
+import com.google.api.core.InternalApi;
+import com.google.auto.value.AutoValue;
+import com.google.cloud.bigtable.data.v2.models.MutateRowsException.FailedMutation;
+import java.util.ArrayList;
+import java.util.List;
+import javax.annotation.Nonnull;
+
+/**
+ * This class represents the result of a MutateRows attempt. It contains a potentially empty list of
+ * failed mutations, along with an indicator whether these errors are retryable.
+ */
+@InternalApi
+@AutoValue
+public abstract class MutateRowsAttemptResult {
+
+ public abstract List getFailedMutations();
+
+ public abstract boolean getIsRetryable();
+
+ @InternalApi
+ @Nonnull
+ public static MutateRowsAttemptResult create(
+ List failedMutations, boolean isRetryable) {
+ return new AutoValue_MutateRowsAttemptResult(failedMutations, isRetryable);
+ }
+
+ @InternalApi
+ @Nonnull
+ public static MutateRowsAttemptResult success() {
+ return new AutoValue_MutateRowsAttemptResult(new ArrayList<>(), false);
+ }
+}
diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsBatchingDescriptor.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsBatchingDescriptor.java
index 65cc781169..87f5c88d3e 100644
--- a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsBatchingDescriptor.java
+++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsBatchingDescriptor.java
@@ -37,7 +37,7 @@
*/
@InternalApi("For internal use only")
public class MutateRowsBatchingDescriptor
- implements BatchingDescriptor {
+ implements BatchingDescriptor {
@Override
public BatchingRequestBuilder newRequestBuilder(
@@ -46,7 +46,15 @@ public BatchingRequestBuilder newRequestBuilder(
}
@Override
- public void splitResponse(Void response, List> entries) {
+ public void splitResponse(
+ MutateRowsAttemptResult response, List> entries) {
+ // For every failed mutation in the response, we set the exception on the matching requested
+ // mutation. It is important to set the correct error on the correct mutation. When the entry is
+ // later read, it resolves the exception first, and only later it goes to the value set by
+ // set().
+ for (FailedMutation mutation : response.getFailedMutations()) {
+ entries.get(mutation.getIndex()).getResultFuture().setException(mutation.getError());
+ }
for (BatchEntry batchResponse : entries) {
batchResponse.getResultFuture().set(null);
}
diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsPartialErrorRetryAlgorithm.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsPartialErrorRetryAlgorithm.java
new file mode 100644
index 0000000000..9c7035db96
--- /dev/null
+++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsPartialErrorRetryAlgorithm.java
@@ -0,0 +1,79 @@
+/*
+ * 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.stub.mutaterows;
+
+import com.google.api.core.InternalApi;
+import com.google.api.gax.retrying.ResultRetryAlgorithmWithContext;
+import com.google.api.gax.retrying.RetryingContext;
+import com.google.api.gax.retrying.TimedAttemptSettings;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * This algorithm will retry if there was a retryable failed mutation, or if there wasn't but the
+ * underlying algorithm allows a retry.
+ */
+@InternalApi
+public class MutateRowsPartialErrorRetryAlgorithm
+ implements ResultRetryAlgorithmWithContext {
+ private final ResultRetryAlgorithmWithContext retryAlgorithm;
+
+ public MutateRowsPartialErrorRetryAlgorithm(
+ ResultRetryAlgorithmWithContext retryAlgorithm) {
+ this.retryAlgorithm = retryAlgorithm;
+ }
+
+ @Override
+ public boolean shouldRetry(
+ Throwable previousThrowable, MutateRowsAttemptResult previousResponse) {
+ // handle partial retryable failures
+ if (previousResponse != null && !previousResponse.getFailedMutations().isEmpty()) {
+ return previousResponse.getIsRetryable();
+ }
+ // business as usual
+ return retryAlgorithm.shouldRetry(previousThrowable, previousResponse);
+ }
+
+ @Override
+ public boolean shouldRetry(
+ @Nullable RetryingContext context,
+ Throwable previousThrowable,
+ MutateRowsAttemptResult previousResponse) {
+ // handle partial retryable failures
+ if (previousResponse != null && !previousResponse.getFailedMutations().isEmpty()) {
+ return previousResponse.getIsRetryable();
+ }
+ // business as usual
+ return retryAlgorithm.shouldRetry(context, previousThrowable, previousResponse);
+ }
+
+ @Override
+ public TimedAttemptSettings createNextAttempt(
+ Throwable previousThrowable,
+ MutateRowsAttemptResult previousResponse,
+ TimedAttemptSettings previousSettings) {
+ return retryAlgorithm.createNextAttempt(previousThrowable, previousResponse, previousSettings);
+ }
+
+ @Override
+ public TimedAttemptSettings createNextAttempt(
+ RetryingContext context,
+ Throwable previousThrowable,
+ MutateRowsAttemptResult previousResponse,
+ TimedAttemptSettings previousSettings) {
+ return retryAlgorithm.createNextAttempt(
+ context, previousThrowable, previousResponse, previousSettings);
+ }
+}
diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsRetryingCallable.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsRetryingCallable.java
index 8ad1db258d..354a5ea54a 100644
--- a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsRetryingCallable.java
+++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsRetryingCallable.java
@@ -40,17 +40,18 @@
* @see MutateRowsAttemptCallable for more details.
*/
@InternalApi
-public class MutateRowsRetryingCallable extends UnaryCallable {
+public class MutateRowsRetryingCallable
+ extends UnaryCallable {
private final ApiCallContext callContextPrototype;
private final ServerStreamingCallable callable;
- private final RetryingExecutorWithContext executor;
+ private final RetryingExecutorWithContext executor;
private final ImmutableSet retryCodes;
private final RetryAlgorithm retryAlgorithm;
public MutateRowsRetryingCallable(
@Nonnull ApiCallContext callContextPrototype,
@Nonnull ServerStreamingCallable callable,
- @Nonnull RetryingExecutorWithContext executor,
+ @Nonnull RetryingExecutorWithContext executor,
@Nonnull Set retryCodes,
@Nonnull RetryAlgorithm retryAlgorithm) {
this.callContextPrototype = Preconditions.checkNotNull(callContextPrototype);
@@ -61,12 +62,14 @@ public MutateRowsRetryingCallable(
}
@Override
- public RetryingFuture futureCall(MutateRowsRequest request, ApiCallContext inputContext) {
+ public RetryingFuture futureCall(
+ MutateRowsRequest request, ApiCallContext inputContext) {
ApiCallContext context = callContextPrototype.nullToSelf(inputContext);
MutateRowsAttemptCallable retryCallable =
new MutateRowsAttemptCallable(callable.all(), request, context, retryCodes, retryAlgorithm);
- RetryingFuture retryingFuture = executor.createFuture(retryCallable, context);
+ RetryingFuture retryingFuture =
+ executor.createFuture(retryCallable, context);
retryCallable.setExternalFuture(retryingFuture);
retryCallable.call();
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 eacf145bcb..2eb0700488 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
@@ -16,11 +16,13 @@
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.client.json.gson.GsonFactory;
import com.google.api.client.json.webtoken.JsonWebSignature;
import com.google.api.gax.batching.Batcher;
import com.google.api.gax.batching.BatcherImpl;
+import com.google.api.gax.batching.BatchingException;
import com.google.api.gax.batching.BatchingSettings;
import com.google.api.gax.batching.FlowControlSettings;
import com.google.api.gax.batching.FlowController.LimitExceededBehavior;
@@ -51,13 +53,14 @@
import com.google.cloud.bigtable.data.v2.FakeServiceBuilder;
import com.google.cloud.bigtable.data.v2.internal.RequestContext;
import com.google.cloud.bigtable.data.v2.models.*;
-import com.google.cloud.bigtable.data.v2.models.Row;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Queues;
import com.google.common.io.BaseEncoding;
import com.google.protobuf.ByteString;
import com.google.protobuf.BytesValue;
import com.google.protobuf.StringValue;
+import com.google.rpc.Code;
+import com.google.rpc.Status;
import io.grpc.Context;
import io.grpc.Deadline;
import io.grpc.ManagedChannel;
@@ -94,6 +97,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)
@@ -118,7 +122,7 @@ public class EnhancedBigtableStubTest {
public void setUp() throws IOException, IllegalAccessException, InstantiationException {
metadataInterceptor = new MetadataInterceptor();
contextInterceptor = new ContextInterceptor();
- fakeDataService = new FakeDataService();
+ fakeDataService = Mockito.spy(new FakeDataService());
server =
FakeServiceBuilder.create(fakeDataService)
@@ -592,6 +596,69 @@ public void testReadChangeStreamWaitTimeoutIsSet() throws Exception {
}
}
+ @Test
+ public void testBatchMutationsPartialFailure() {
+ Batcher batcher =
+ enhancedBigtableStub.newMutateRowsBatcher("table1", GrpcCallContext.createDefault());
+
+ batcher.add(RowMutationEntry.create("key0").deleteRow());
+ batcher.add(RowMutationEntry.create("key1").deleteRow());
+
+ Mockito.doAnswer(
+ invocationOnMock -> {
+ StreamObserver observer = invocationOnMock.getArgument(1);
+ observer.onNext(
+ MutateRowsResponse.newBuilder()
+ .addEntries(
+ MutateRowsResponse.Entry.newBuilder()
+ .setIndex(0)
+ .setStatus(Status.newBuilder().setCode(Code.OK_VALUE))
+ .build())
+ .addEntries(
+ MutateRowsResponse.Entry.newBuilder()
+ .setIndex(1)
+ .setStatus(
+ Status.newBuilder()
+ .setCode(Code.PERMISSION_DENIED_VALUE)
+ .setMessage("fake partial error"))
+ .build())
+ .build());
+ observer.onCompleted();
+ return null;
+ })
+ .when(fakeDataService)
+ .mutateRows(Mockito.any(MutateRowsRequest.class), Mockito.any(StreamObserver.class));
+ BatchingException batchingException =
+ assertThrows(BatchingException.class, () -> batcher.close());
+ assertThat(batchingException.getMessage())
+ .contains(
+ "Batching finished with 1 partial failures. The 1 partial failures contained 1 entries that failed with: 1 ApiException(1 PERMISSION_DENIED).");
+ assertThat(batchingException.getMessage()).contains("fake partial error");
+ assertThat(batchingException.getMessage()).doesNotContain("INTERNAL");
+ }
+
+ @Test
+ public void testBatchMutationRPCErrorCode() {
+ Batcher batcher =
+ enhancedBigtableStub.newMutateRowsBatcher("table1", GrpcCallContext.createDefault());
+
+ Mockito.doAnswer(
+ invocationOnMock -> {
+ StreamObserver observer = invocationOnMock.getArgument(1);
+ observer.onError(io.grpc.Status.PERMISSION_DENIED.asException());
+ return null;
+ })
+ .when(fakeDataService)
+ .mutateRows(Mockito.any(MutateRowsRequest.class), Mockito.any(StreamObserver.class));
+
+ batcher.add(RowMutationEntry.create("key0").deleteRow());
+ BatchingException batchingException =
+ assertThrows(BatchingException.class, () -> batcher.close());
+ assertThat(batchingException.getMessage())
+ .contains(
+ "Batching finished with 1 batches failed to apply due to: 1 ApiException(1 PERMISSION_DENIED) and 0 partial failures");
+ }
+
private static class MetadataInterceptor implements ServerInterceptor {
final BlockingQueue headers = Queues.newLinkedBlockingDeque();
diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/metrics/BuiltinMetricsTracerTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/metrics/BuiltinMetricsTracerTest.java
index c2be1ea0ff..8f62060c97 100644
--- a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/metrics/BuiltinMetricsTracerTest.java
+++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/metrics/BuiltinMetricsTracerTest.java
@@ -17,6 +17,7 @@
import static com.google.api.gax.tracing.ApiTracerFactory.OperationType;
import static com.google.common.truth.Truth.assertThat;
+import static org.junit.Assert.assertThrows;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.timeout;
import static org.mockito.Mockito.times;
@@ -27,6 +28,7 @@
import com.google.api.core.ApiFunction;
import com.google.api.core.SettableApiFuture;
import com.google.api.gax.batching.Batcher;
+import com.google.api.gax.batching.BatchingException;
import com.google.api.gax.batching.BatchingSettings;
import com.google.api.gax.batching.FlowControlSettings;
import com.google.api.gax.grpc.InstantiatingGrpcChannelProvider;
@@ -74,6 +76,7 @@
import io.grpc.StatusRuntimeException;
import io.grpc.stub.ServerCallStreamObserver;
import io.grpc.stub.StreamObserver;
+import java.nio.charset.Charset;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
@@ -449,6 +452,55 @@ public void testMutateRowAttemptsTagValues() {
assertThat(tableId.getAllValues()).containsExactly(TABLE_ID, TABLE_ID, TABLE_ID);
}
+ @Test
+ public void testMutateRowsPartialError() throws InterruptedException {
+ int numMutations = 6;
+ when(mockFactory.newTracer(any(), any(), any()))
+ .thenReturn(
+ new BuiltinMetricsTracer(
+ OperationType.Unary, SpanName.of("Bigtable", "MutateRows"), statsRecorderWrapper));
+
+ Batcher batcher = stub.newMutateRowsBatcher(TABLE_ID, null);
+ for (int i = 0; i < numMutations; i++) {
+ String key = i % 2 == 0 ? "key" : "fail-key";
+ batcher.add(RowMutationEntry.create(key).setCell("f", "q", "v"));
+ }
+
+ assertThrows(BatchingException.class, () -> batcher.close());
+
+ int expectedNumRequests = numMutations / batchElementCount;
+ verify(statsRecorderWrapper, timeout(100).times(expectedNumRequests))
+ .recordAttempt(status.capture(), tableId.capture(), zone.capture(), cluster.capture());
+
+ assertThat(zone.getAllValues()).containsExactly(ZONE, ZONE, ZONE);
+ assertThat(cluster.getAllValues()).containsExactly(CLUSTER, CLUSTER, CLUSTER);
+ assertThat(status.getAllValues()).containsExactly("OK", "OK", "OK");
+ }
+
+ @Test
+ public void testMutateRowsRpcError() {
+ int numMutations = 6;
+ when(mockFactory.newTracer(any(), any(), any()))
+ .thenReturn(
+ new BuiltinMetricsTracer(
+ OperationType.Unary, SpanName.of("Bigtable", "MutateRows"), statsRecorderWrapper));
+
+ Batcher batcher = stub.newMutateRowsBatcher(BAD_TABLE_ID, null);
+ for (int i = 0; i < numMutations; i++) {
+ batcher.add(RowMutationEntry.create("key").setCell("f", "q", "v"));
+ }
+
+ assertThrows(BatchingException.class, () -> batcher.close());
+
+ int expectedNumRequests = numMutations / batchElementCount;
+ verify(statsRecorderWrapper, timeout(100).times(expectedNumRequests))
+ .recordAttempt(status.capture(), tableId.capture(), zone.capture(), cluster.capture());
+
+ assertThat(zone.getAllValues()).containsExactly("global", "global", "global");
+ assertThat(cluster.getAllValues()).containsExactly("unspecified", "unspecified", "unspecified");
+ assertThat(status.getAllValues()).containsExactly("NOT_FOUND", "NOT_FOUND", "NOT_FOUND");
+ }
+
@Test
public void testReadRowsAttemptsTagValues() {
when(mockFactory.newTracer(any(), any(), any()))
@@ -644,12 +696,30 @@ public void mutateRow(
@Override
public void mutateRows(
MutateRowsRequest request, StreamObserver responseObserver) {
+ if (request.getTableName().contains(BAD_TABLE_ID)) {
+ responseObserver.onError(new StatusRuntimeException(Status.NOT_FOUND));
+ return;
+ }
try {
Thread.sleep(SERVER_LATENCY);
} catch (InterruptedException e) {
}
MutateRowsResponse.Builder builder = MutateRowsResponse.newBuilder();
for (int i = 0; i < request.getEntriesCount(); i++) {
+ if (request
+ .getEntries(i)
+ .getRowKey()
+ .toString(Charset.availableCharsets().get("UTF-8"))
+ .startsWith("fail")) {
+ builder
+ .addEntriesBuilder()
+ .setIndex(i)
+ .setStatus(
+ com.google.rpc.Status.newBuilder()
+ .setCode(com.google.rpc.Code.PERMISSION_DENIED_VALUE)
+ .build());
+ continue;
+ }
builder.addEntriesBuilder().setIndex(i);
}
responseObserver.onNext(builder.build());
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 2894568f27..15bd9171f0 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
@@ -448,7 +448,7 @@ public Object answer(InvocationOnMock invocation) {
try (Batcher batcher =
new BatcherImpl<>(
batchingDescriptor,
- stub.bulkMutateRowsCallable().withDefaultCallContext(defaultContext),
+ stub.internalBulkMutateRowsCallable().withDefaultCallContext(defaultContext),
BulkMutation.create(TABLE_ID),
settings.getStubSettings().bulkMutateRowsSettings().getBatchingSettings(),
Executors.newSingleThreadScheduledExecutor(),
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 e5d12ccaeb..6dd1ff9bd0 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
@@ -44,8 +44,6 @@
import java.util.List;
import java.util.Set;
import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
@@ -101,14 +99,18 @@ public void singleEntrySuccessTest() throws Exception {
attemptCallable.setExternalFuture(parentFuture);
attemptCallable.call();
- // Attempt completed successfully and the useless response has been suppressed
- assertThat(parentFuture.attemptFuture.get()).isNull();
+ // Attempt completed successfully
+ MutateRowsAttemptResult result = parentFuture.attemptFuture.get();
+
+ assertThat(result).isNotNull();
+ assertThat(result.getFailedMutations()).hasSize(0);
+ assertThat(result.getIsRetryable()).isFalse();
// innerCallable received the request
assertThat(innerCallable.lastRequest).isEqualTo(request);
}
@Test
- public void missingEntry() {
+ public void missingEntry() throws Exception {
MutateRowsRequest request =
MutateRowsRequest.newBuilder()
.addEntries(Entry.getDefaultInstance())
@@ -125,14 +127,10 @@ public void missingEntry() {
attemptCallable.setExternalFuture(parentFuture);
attemptCallable.call();
- ExecutionException executionException =
- Assert.assertThrows(ExecutionException.class, () -> parentFuture.attemptFuture.get());
- assertThat(executionException).hasCauseThat().isInstanceOf(MutateRowsException.class);
- MutateRowsException e = (MutateRowsException) executionException.getCause();
+ MutateRowsAttemptResult result = parentFuture.attemptFuture.get();
- assertThat(e).hasMessageThat().contains("Some mutations failed to apply");
- assertThat(e.getFailedMutations()).hasSize(1);
- FailedMutation failedMutation = e.getFailedMutations().get(0);
+ assertThat(result.getFailedMutations()).hasSize(1);
+ FailedMutation failedMutation = result.getFailedMutations().get(0);
assertThat(failedMutation.getIndex()).isEqualTo(1);
assertThat(failedMutation.getError())
.hasMessageThat()
@@ -163,7 +161,7 @@ public void testNoRpcTimeout() {
}
@Test
- public void mixedTest() {
+ public void mixedTest() throws Exception {
// Setup the request & response
MutateRowsRequest request =
MutateRowsRequest.newBuilder()
@@ -194,20 +192,11 @@ public void mixedTest() {
// Make the only call
attemptCallable.call();
- // Overall error expectations
- Throwable actualError = null;
- try {
- parentFuture.attemptFuture.get();
- } catch (Throwable t) {
- actualError = t.getCause();
- }
-
- assertThat(actualError).isInstanceOf(MutateRowsException.class);
- assertThat(((MutateRowsException) actualError).isRetryable()).isTrue();
+ MutateRowsAttemptResult result = parentFuture.attemptFuture.get();
// Entry expectations
@SuppressWarnings("ConstantConditions")
- List failedMutations = ((MutateRowsException) actualError).getFailedMutations();
+ List failedMutations = result.getFailedMutations();
assertThat(failedMutations).hasSize(2);
assertThat(failedMutations.get(0).getIndex()).isEqualTo(1);
@@ -222,7 +211,7 @@ public void mixedTest() {
}
@Test
- public void nextAttemptTest() {
+ public void nextAttemptTest() throws Exception {
// Setup the request & response for the first call
MutateRowsRequest request =
MutateRowsRequest.newBuilder()
@@ -267,19 +256,11 @@ public void nextAttemptTest() {
assertThat(innerCallable.lastRequest.getEntries(0).getRowKey())
.isEqualTo(ByteString.copyFromUtf8("1-unavailable"));
- // Overall error expectations
- Throwable actualError = null;
- try {
- parentFuture.attemptFuture.get();
- } catch (Throwable t) {
- actualError = t.getCause();
- }
- assertThat(actualError).isInstanceOf(MutateRowsException.class);
- assertThat(((MutateRowsException) actualError).isRetryable()).isFalse();
+ MutateRowsAttemptResult result = parentFuture.attemptFuture.get();
// Entry expectations
@SuppressWarnings("ConstantConditions")
- List failedMutations = ((MutateRowsException) actualError).getFailedMutations();
+ List failedMutations = result.getFailedMutations();
assertThat(failedMutations).hasSize(1);
assertThat(failedMutations.get(0).getIndex()).isEqualTo(2);
@@ -411,8 +392,9 @@ public ApiFuture> futureCall(
}
}
- static class MockRetryingFuture extends AbstractApiFuture implements RetryingFuture {
- ApiFuture attemptFuture;
+ static class MockRetryingFuture extends AbstractApiFuture
+ implements RetryingFuture {
+ ApiFuture attemptFuture;
TimedAttemptSettings timedAttemptSettings;
@@ -433,7 +415,7 @@ static class MockRetryingFuture extends AbstractApiFuture implements Retry
}
@Override
- public void setAttemptFuture(ApiFuture attemptFuture) {
+ public void setAttemptFuture(ApiFuture attemptFuture) {
this.attemptFuture = attemptFuture;
}
@@ -443,17 +425,17 @@ public TimedAttemptSettings getAttemptSettings() {
}
@Override
- public Callable getCallable() {
+ public Callable getCallable() {
throw new UnsupportedOperationException("not used");
}
@Override
- public ApiFuture peekAttemptResult() {
+ public ApiFuture peekAttemptResult() {
throw new UnsupportedOperationException("not used");
}
@Override
- public ApiFuture getAttemptResult() {
+ public ApiFuture getAttemptResult() {
throw new UnsupportedOperationException("not used");
}
}
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 237444ba84..c5f11d91d5 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
@@ -16,21 +16,26 @@
package com.google.cloud.bigtable.data.v2.stub.mutaterows;
import static com.google.common.truth.Truth.assertThat;
+import static org.junit.Assert.assertThrows;
import com.google.api.core.SettableApiFuture;
import com.google.api.gax.batching.BatchEntry;
import com.google.api.gax.batching.BatchResource;
import com.google.api.gax.batching.BatchingRequestBuilder;
import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.rpc.ApiExceptionFactory;
import com.google.api.gax.rpc.DeadlineExceededException;
+import com.google.api.gax.rpc.InternalException;
import com.google.api.gax.rpc.UnavailableException;
import com.google.cloud.bigtable.data.v2.internal.RequestContext;
import com.google.cloud.bigtable.data.v2.models.BulkMutation;
import com.google.cloud.bigtable.data.v2.models.MutateRowsException;
+import com.google.cloud.bigtable.data.v2.models.MutateRowsException.FailedMutation;
import com.google.cloud.bigtable.data.v2.models.Mutation;
import com.google.cloud.bigtable.data.v2.models.RowMutationEntry;
import com.google.common.collect.ImmutableList;
import io.grpc.Status;
+import java.util.Arrays;
import java.util.List;
import java.util.concurrent.ExecutionException;
import org.junit.Test;
@@ -92,11 +97,58 @@ public void splitResponseTest() {
assertThat(batchResponse.get(1).getResultFuture().isDone()).isFalse();
MutateRowsBatchingDescriptor underTest = new MutateRowsBatchingDescriptor();
- underTest.splitResponse(null, batchResponse);
+ underTest.splitResponse(MutateRowsAttemptResult.success(), batchResponse);
assertThat(batchResponse.get(0).getResultFuture().isDone()).isTrue();
assertThat(batchResponse.get(1).getResultFuture().isDone()).isTrue();
}
+ @Test
+ public void splitResponsePartialErrorsTest() {
+ BatchEntry batchEntry1 =
+ BatchEntry.create(
+ RowMutationEntry.create("key1").deleteRow(), SettableApiFuture.create());
+ BatchEntry batchEntry2 =
+ BatchEntry.create(
+ RowMutationEntry.create("key2").deleteRow(), SettableApiFuture.create());
+
+ List> batchResponse =
+ ImmutableList.of(batchEntry1, batchEntry2);
+ assertThat(batchResponse.get(0).getResultFuture().isDone()).isFalse();
+ assertThat(batchResponse.get(1).getResultFuture().isDone()).isFalse();
+
+ MutateRowsBatchingDescriptor underTest = new MutateRowsBatchingDescriptor();
+ underTest.splitResponse(
+ MutateRowsAttemptResult.create(
+ Arrays.asList(
+ FailedMutation.create(
+ 0,
+ ApiExceptionFactory.createException(
+ "error message",
+ null,
+ GrpcStatusCode.of(io.grpc.Status.Code.INTERNAL),
+ false))),
+ true),
+ batchResponse);
+ assertThat(batchResponse.get(0).getResultFuture().isDone()).isTrue();
+ assertThat(batchResponse.get(1).getResultFuture().isDone()).isTrue();
+
+ Throwable unexpectedError = null;
+ try {
+ batchResponse.get(1).getResultFuture().get();
+
+ } catch (Throwable t) {
+ unexpectedError = t;
+ }
+ assertThat(unexpectedError).isNull();
+
+ Throwable actualError =
+ assertThrows(ExecutionException.class, () -> batchResponse.get(0).getResultFuture().get())
+ .getCause();
+
+ assertThat(actualError).isInstanceOf(InternalException.class);
+ assertThat(actualError).hasMessageThat().contains("error message");
+ }
+
@Test
public void splitExceptionTest() {
BatchEntry batchEntry1 =
@@ -140,6 +192,7 @@ public void splitExceptionWithFailedMutationsTest() {
MutateRowsException serverError =
MutateRowsException.create(
null,
+ GrpcStatusCode.of(Status.Code.UNAVAILABLE),
ImmutableList.of(
MutateRowsException.FailedMutation.create(
0,
diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsErrorConverterUnaryCallableTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsErrorConverterUnaryCallableTest.java
new file mode 100644
index 0000000000..170aa66188
--- /dev/null
+++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/mutaterows/MutateRowsErrorConverterUnaryCallableTest.java
@@ -0,0 +1,108 @@
+/*
+ * 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.stub.mutaterows;
+
+import static com.google.common.truth.Truth.assertThat;
+
+import com.google.api.core.SettableApiFuture;
+import com.google.api.gax.grpc.GrpcStatusCode;
+import com.google.api.gax.rpc.ApiExceptionFactory;
+import com.google.api.gax.rpc.UnaryCallable;
+import com.google.cloud.bigtable.data.v2.models.BulkMutation;
+import com.google.cloud.bigtable.data.v2.models.MutateRowsException;
+import com.google.cloud.bigtable.data.v2.models.MutateRowsException.FailedMutation;
+import com.google.cloud.bigtable.data.v2.stub.MutateRowsErrorConverterUnaryCallable;
+import java.util.Arrays;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+@RunWith(JUnit4.class)
+public class MutateRowsErrorConverterUnaryCallableTest {
+ @Mock private UnaryCallable innerCallable;
+ @Captor private ArgumentCaptor innerMutation;
+ private SettableApiFuture innerResult;
+
+ @Rule public final MockitoRule mockitoRule = MockitoJUnit.rule();
+
+ @Before
+ public void setUp() {
+ innerResult = SettableApiFuture.create();
+ Mockito.when(innerCallable.futureCall(innerMutation.capture(), Mockito.any()))
+ .thenReturn(innerResult);
+ }
+
+ @Test
+ public void testSuccess() {
+ MutateRowsErrorConverterUnaryCallable callable =
+ new MutateRowsErrorConverterUnaryCallable(innerCallable);
+
+ innerResult.set(MutateRowsAttemptResult.success());
+
+ Throwable unexpectedError = null;
+ try {
+ callable.call(BulkMutation.create("fake-table"));
+ } catch (Throwable t) {
+ unexpectedError = t;
+ }
+ assertThat(unexpectedError).isNull();
+ }
+
+ @Test
+ public void testPartialFailure() {
+ MutateRowsErrorConverterUnaryCallable callable =
+ new MutateRowsErrorConverterUnaryCallable(innerCallable);
+
+ innerResult.set(
+ MutateRowsAttemptResult.create(
+ Arrays.asList(
+ FailedMutation.create(
+ 0,
+ ApiExceptionFactory.createException(
+ null, GrpcStatusCode.of(io.grpc.Status.Code.INTERNAL), false))),
+ true));
+
+ MutateRowsException exception =
+ Assert.assertThrows(
+ MutateRowsException.class, () -> callable.call(BulkMutation.create("fake-table")));
+
+ assertThat(exception).isInstanceOf(MutateRowsException.class);
+ assertThat((exception).isRetryable()).isTrue();
+ }
+
+ @Test
+ public void testRPCFailure() {
+ MutateRowsErrorConverterUnaryCallable callable =
+ new MutateRowsErrorConverterUnaryCallable(innerCallable);
+
+ innerResult.setException(new Exception("RPC error"));
+
+ Exception exception =
+ Assert.assertThrows(
+ Exception.class, () -> callable.call(BulkMutation.create("fake-table")));
+
+ assertThat(exception).isInstanceOf(Exception.class);
+ }
+}
From 13d1df3910e3041b57485a09317f717932b21727 Mon Sep 17 00:00:00 2001
From: Lixia Chen
Date: Wed, 27 Mar 2024 16:59:42 -0400
Subject: [PATCH 08/10] feat: Add admin APIs for AuthorizedView (#2175)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
* feat: Add admin APIs for AuthorizedView
Change-Id: Ie31eae6da61ed0d0462e029f6247924785b239bf
* chore: Mark AuthorizedViewType as InternalApi
Change-Id: If52e3a32c3259f652f1f7d34b013d1ec1fc0a773
* chore: Add tests and change AuthorizedViewType to InternalExtensionOnly
Change-Id: I2e7f04d0f7815d014928a924d4a4f26adb2b655d
* 🦉 Updates from OwlBot post-processor
See https://github.com/googleapis/repo-automation-bots/blob/main/packages/owl-bot/README.md
* Chore: fix a comment
Change-Id: Iac28b6cbef3088e4e2d43d90655155369361c347
* setFamilySubset instead of add and use List rather than ImmutableList
Change-Id: Ibdb2c8a62dc55c44059d5ec2296c57c7d430baa4
---------
Co-authored-by: Lixia Chen
Co-authored-by: Owl Bot
---
.../admin/v2/BigtableTableAdminClient.java | 535 ++++++++++++++++++
.../bigtable/admin/v2/internal/NameUtil.java | 17 +
.../admin/v2/models/AuthorizedView.java | 124 ++++
.../models/CreateAuthorizedViewRequest.java | 117 ++++
.../admin/v2/models/FamilySubsets.java | 119 ++++
.../bigtable/admin/v2/models/SubsetView.java | 123 ++++
.../models/UpdateAuthorizedViewRequest.java | 154 +++++
.../v2/BigtableTableAdminClientTests.java | 261 +++++++++
.../admin/v2/internal/NameUtilTest.java | 23 +
.../admin/v2/it/BigtableAuthorizedViewIT.java | 275 +++++++++
.../admin/v2/models/AuthorizedViewTest.java | 184 ++++++
.../CreateAuthorizedViewRequestTest.java | 108 ++++
.../admin/v2/models/FamilySubsetsTest.java | 82 +++
.../admin/v2/models/SubsetViewTest.java | 162 ++++++
.../UpdateAuthorizedViewRequestTest.java | 158 ++++++
15 files changed, 2442 insertions(+)
create mode 100644 google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/models/AuthorizedView.java
create mode 100644 google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/models/CreateAuthorizedViewRequest.java
create mode 100644 google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/models/FamilySubsets.java
create mode 100644 google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/models/SubsetView.java
create mode 100644 google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/models/UpdateAuthorizedViewRequest.java
create mode 100644 google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/it/BigtableAuthorizedViewIT.java
create mode 100644 google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/models/AuthorizedViewTest.java
create mode 100644 google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/models/CreateAuthorizedViewRequestTest.java
create mode 100644 google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/models/FamilySubsetsTest.java
create mode 100644 google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/models/SubsetViewTest.java
create mode 100644 google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/models/UpdateAuthorizedViewRequestTest.java
diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/BigtableTableAdminClient.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/BigtableTableAdminClient.java
index f83b01b3d4..f640bb6a30 100644
--- a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/BigtableTableAdminClient.java
+++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/BigtableTableAdminClient.java
@@ -22,11 +22,14 @@
import com.google.api.gax.longrunning.OperationFuture;
import com.google.api.gax.rpc.ApiExceptions;
import com.google.api.gax.rpc.NotFoundException;
+import com.google.bigtable.admin.v2.DeleteAuthorizedViewRequest;
import com.google.bigtable.admin.v2.DeleteBackupRequest;
import com.google.bigtable.admin.v2.DeleteTableRequest;
import com.google.bigtable.admin.v2.DropRowRangeRequest;
+import com.google.bigtable.admin.v2.GetAuthorizedViewRequest;
import com.google.bigtable.admin.v2.GetBackupRequest;
import com.google.bigtable.admin.v2.GetTableRequest;
+import com.google.bigtable.admin.v2.ListAuthorizedViewsRequest;
import com.google.bigtable.admin.v2.ListBackupsRequest;
import com.google.bigtable.admin.v2.ListTablesRequest;
import com.google.bigtable.admin.v2.RestoreTableMetadata;
@@ -34,13 +37,17 @@
import com.google.bigtable.admin.v2.Table.View;
import com.google.cloud.Policy;
import com.google.cloud.Policy.DefaultMarshaller;
+import com.google.cloud.bigtable.admin.v2.BaseBigtableTableAdminClient.ListAuthorizedViewsPage;
+import com.google.cloud.bigtable.admin.v2.BaseBigtableTableAdminClient.ListAuthorizedViewsPagedResponse;
import com.google.cloud.bigtable.admin.v2.BaseBigtableTableAdminClient.ListBackupsPage;
import com.google.cloud.bigtable.admin.v2.BaseBigtableTableAdminClient.ListBackupsPagedResponse;
import com.google.cloud.bigtable.admin.v2.BaseBigtableTableAdminClient.ListTablesPage;
import com.google.cloud.bigtable.admin.v2.BaseBigtableTableAdminClient.ListTablesPagedResponse;
import com.google.cloud.bigtable.admin.v2.internal.NameUtil;
+import com.google.cloud.bigtable.admin.v2.models.AuthorizedView;
import com.google.cloud.bigtable.admin.v2.models.Backup;
import com.google.cloud.bigtable.admin.v2.models.CopyBackupRequest;
+import com.google.cloud.bigtable.admin.v2.models.CreateAuthorizedViewRequest;
import com.google.cloud.bigtable.admin.v2.models.CreateBackupRequest;
import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest;
import com.google.cloud.bigtable.admin.v2.models.EncryptionInfo;
@@ -50,6 +57,7 @@
import com.google.cloud.bigtable.admin.v2.models.RestoreTableRequest;
import com.google.cloud.bigtable.admin.v2.models.RestoredTableResult;
import com.google.cloud.bigtable.admin.v2.models.Table;
+import com.google.cloud.bigtable.admin.v2.models.UpdateAuthorizedViewRequest;
import com.google.cloud.bigtable.admin.v2.models.UpdateBackupRequest;
import com.google.cloud.bigtable.admin.v2.models.UpdateTableRequest;
import com.google.cloud.bigtable.admin.v2.stub.EnhancedBigtableTableAdminStub;
@@ -1434,6 +1442,352 @@ public ApiFuture awaitReplicationAsync(final String tableId) {
return stub.awaitReplicationCallable().futureCall(tableName);
}
+ /**
+ * Creates a new authorized view with the specified configuration.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * CreateAuthorizedViewRequest request =
+ * CreateAuthorizedViewRequest.of("my-table", "my-new-authorized-view")
+ * .setDeletionProtection(true)
+ * .setAuthorizedViewType(
+ * SubsetView.create()
+ * .addRowPrefix("row#")
+ * .addFamilySubsets(
+ * "my-family", FamilySubsets.create().addQualifier("column")));
+ *
+ * AuthorizedView response = client.createAuthorizedView(request);
+ * }
+ *
+ * @see CreateAuthorizedViewRequest for available options.
+ */
+ public AuthorizedView createAuthorizedView(CreateAuthorizedViewRequest request) {
+ return ApiExceptions.callAndTranslateApiException(createAuthorizedViewAsync(request));
+ }
+
+ /**
+ * Asynchronously creates a new authorized view with the specified configuration.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * CreateAuthorizedViewRequest request =
+ * CreateAuthorizedViewRequest.of("my-table", "my-new-authorized-view")
+ * .setDeletionProtection(true)
+ * .setAuthorizedViewType(
+ * SubsetView.create()
+ * .addRowPrefix("row#")
+ * .addFamilySubsets(
+ * "my-family", FamilySubsets.create().addQualifier("column")));
+ *
+ * ApiFuture future = client.createAuthorizedViewAsync(request);
+ *
+ * ApiFutures.addCallback(
+ * future,
+ * new ApiFutureCallback() {
+ * public void onSuccess(AuthorizedView authorizedView) {
+ * System.out.println("Successfully created the authorized view: " + authorizedView.getId());
+ * }
+ *
+ * public void onFailure(Throwable t) {
+ * t.printStackTrace();
+ * }
+ * },
+ * MoreExecutors.directExecutor()
+ * );
+ * }
+ *
+ * @see CreateAuthorizedViewRequest for available options.
+ */
+ public ApiFuture createAuthorizedViewAsync(CreateAuthorizedViewRequest request) {
+ return ApiFutures.transform(
+ stub.createAuthorizedViewOperationCallable()
+ .futureCall(request.toProto(projectId, instanceId)),
+ new ApiFunction() {
+ @Override
+ public AuthorizedView apply(
+ com.google.bigtable.admin.v2.AuthorizedView authorizedViewProto) {
+ return AuthorizedView.fromProto(authorizedViewProto);
+ }
+ },
+ MoreExecutors.directExecutor());
+ }
+
+ /**
+ * Updates an existing authorized view with the specified configuration.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * AuthorizedView existingAuthorizedView = client.getAuthorizedView("my-table", "my-authorized-view");
+ *
+ * UpdateAuthorizedViewRequest request =
+ * UpdateAuthorizedViewRequest.of(existingAuthorizedView).setDeletionProtection(true);
+ *
+ * AuthorizedView response = client.updateAuthorizedView(request);
+ * }
+ *
+ * @see UpdateAuthorizedViewRequest for available options.
+ */
+ public AuthorizedView updateAuthorizedView(UpdateAuthorizedViewRequest request) {
+ return ApiExceptions.callAndTranslateApiException(updateAuthorizedViewAsync(request));
+ }
+
+ /**
+ * Asynchronously updates an existing authorized view with the specified configuration.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * AuthorizedView existingAuthorizedView = client.getAuthorizedView("my-table", "my-authorized-view");
+ *
+ * UpdateAuthorizedViewRequest request =
+ * UpdateAuthorizedViewRequest.of(existingAuthorizedView).setDeletionProtection(true);
+ *
+ * ApiFuture future = client.updateAuthorizedViewAsync(request);
+ *
+ * ApiFutures.addCallback(
+ * future,
+ * new ApiFutureCallback() {
+ * public void onSuccess(AuthorizedView authorizedView) {
+ * System.out.println("Successfully updated the authorized view: " + authorizedView.getId());
+ * }
+ *
+ * public void onFailure(Throwable t) {
+ * t.printStackTrace();
+ * }
+ * },
+ * MoreExecutors.directExecutor()
+ * );
+ * }
+ *
+ * @see UpdateAuthorizedViewRequest for available options.
+ */
+ public ApiFuture updateAuthorizedViewAsync(UpdateAuthorizedViewRequest request) {
+ return ApiFutures.transform(
+ stub.updateAuthorizedViewOperationCallable()
+ .futureCall(request.toProto(projectId, instanceId)),
+ new ApiFunction() {
+ @Override
+ public AuthorizedView apply(
+ com.google.bigtable.admin.v2.AuthorizedView authorizedViewProto) {
+ return AuthorizedView.fromProto(authorizedViewProto);
+ }
+ },
+ MoreExecutors.directExecutor());
+ }
+
+ /**
+ * Gets an authorized view with the specified authorized view ID in the specified table.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * AuthorizedView authorizedView = client.getAuthorizedView("my-table", "my-authorized-view");
+ * }
+ */
+ public AuthorizedView getAuthorizedView(String tableId, String authorizedViewId) {
+ return ApiExceptions.callAndTranslateApiException(
+ getAuthorizedViewAsync(tableId, authorizedViewId));
+ }
+
+ /**
+ * Asynchronously gets an authorized view with the specified authorized view ID in the specified
+ * table.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * ApiFuture future = client.getAuthorizedViewAsync("my-table", "my-authorized-view");
+ *
+ * ApiFutures.addCallback(
+ * future,
+ * new ApiFutureCallback() {
+ * public void onSuccess(AuthorizedView authorizedView) {
+ * System.out.println("Successfully get the authorized view: " + authorizedView.getId());
+ * }
+ *
+ * public void onFailure(Throwable t) {
+ * t.printStackTrace();
+ * }
+ * },
+ * MoreExecutors.directExecutor()
+ * );
+ * }
+ */
+ public ApiFuture getAuthorizedViewAsync(String tableId, String authorizedViewId) {
+ GetAuthorizedViewRequest request =
+ GetAuthorizedViewRequest.newBuilder()
+ .setName(
+ NameUtil.formatAuthorizedViewName(projectId, instanceId, tableId, authorizedViewId))
+ .build();
+ return ApiFutures.transform(
+ stub.getAuthorizedViewCallable().futureCall(request),
+ new ApiFunction() {
+ @Override
+ public AuthorizedView apply(
+ com.google.bigtable.admin.v2.AuthorizedView authorizedViewProto) {
+ return AuthorizedView.fromProto(authorizedViewProto);
+ }
+ },
+ MoreExecutors.directExecutor());
+ }
+
+ /**
+ * Lists all authorized view IDs in the specified table.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * List authorizedViews = client.listAuthorizedViews("my-table");
+ * }
+ */
+ public List listAuthorizedViews(String tableId) {
+ return ApiExceptions.callAndTranslateApiException(listAuthorizedViewsAsync(tableId));
+ }
+
+ /**
+ * Asynchronously lists all authorized view IDs in the specified table.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * ApiFuture> future = client.listAuthorizedViewsAsync("my-table");
+ *
+ * ApiFutures.addCallback(
+ * future,
+ * new ApiFutureCallback>() {
+ * public void onSuccess(List authorizedViewIds) {
+ * System.out.println("Successfully get list of authorized views:");
+ * for (AuthorizedView authorizedViewId : authorizedViewIds) {
+ * System.out.println(authorizedViewId);
+ * }
+ * }
+ *
+ * public void onFailure(Throwable t) {
+ * t.printStackTrace();
+ * }
+ * },
+ * MoreExecutors.directExecutor()
+ * );
+ * }
+ */
+ public ApiFuture> listAuthorizedViewsAsync(String tableId) {
+ ListAuthorizedViewsRequest request =
+ ListAuthorizedViewsRequest.newBuilder()
+ .setParent(NameUtil.formatTableName(projectId, instanceId, tableId))
+ .build();
+
+ // TODO(igorbernstein2): try to upstream pagination spooling or figure out a way to expose the
+ // paginated responses while maintaining the wrapper facade.
+
+ // Fetches the first page.
+ ApiFuture firstPageFuture =
+ ApiFutures.transform(
+ stub.listAuthorizedViewsPagedCallable().futureCall(request),
+ new ApiFunction() {
+ @Override
+ public ListAuthorizedViewsPage apply(ListAuthorizedViewsPagedResponse response) {
+ return response.getPage();
+ }
+ },
+ MoreExecutors.directExecutor());
+
+ // Fetches the rest of the pages by chaining the futures.
+ ApiFuture> allProtos =
+ ApiFutures.transformAsync(
+ firstPageFuture,
+ new ApiAsyncFunction<
+ ListAuthorizedViewsPage, List>() {
+ List responseAccumulator =
+ Lists.newArrayList();
+
+ @Override
+ public ApiFuture> apply(
+ ListAuthorizedViewsPage page) {
+ // Add all entries from the page
+ responseAccumulator.addAll(Lists.newArrayList(page.getValues()));
+
+ // If this is the last page, just return the accumulated responses.
+ if (!page.hasNextPage()) {
+ return ApiFutures.immediateFuture(responseAccumulator);
+ }
+
+ // Otherwise fetch the next page.
+ return ApiFutures.transformAsync(
+ page.getNextPageAsync(), this, MoreExecutors.directExecutor());
+ }
+ },
+ MoreExecutors.directExecutor());
+
+ // Wraps all of the accumulated protos.
+ return ApiFutures.transform(
+ allProtos,
+ new ApiFunction, List>() {
+ @Override
+ public List apply(List protos) {
+ List results = Lists.newArrayListWithCapacity(protos.size());
+ for (com.google.bigtable.admin.v2.AuthorizedView proto : protos) {
+ results.add(NameUtil.extractAuthorizedViewIdFromAuthorizedViewName(proto.getName()));
+ }
+ return results;
+ }
+ },
+ MoreExecutors.directExecutor());
+ }
+
+ /**
+ * Deletes an authorized view with the specified authorized view ID in the specified table. Note
+ * that the deletion is prohibited if the authorized view has deletion_protection field set to
+ * true.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * client.deleteAuthorizedView("my-table", "my-authorized-view");
+ * }
+ */
+ public void deleteAuthorizedView(String tableId, String authorizedViewId) {
+ ApiExceptions.callAndTranslateApiException(
+ deleteAuthorizedViewAsync(tableId, authorizedViewId));
+ }
+
+ /**
+ * Asynchronously deletes an authorized view with the specified authorized view ID in the
+ * specified table. Note that the deletion is prohibited if the authorized view has
+ * deletion_protection field set to true.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * ApiFuture future = client.deleteAuthorizedViewAsync("my-table", "my-authorized-view");
+ *
+ * ApiFutures.addCallback(
+ * future,
+ * new ApiFutureCallback() {
+ * public void onSuccess(Void ignored) {
+ * System.out.println("Successfully deleted the authorized view");
+ * }
+ *
+ * public void onFailure(Throwable t) {
+ * t.printStackTrace();
+ * }
+ * },
+ * MoreExecutors.directExecutor()
+ * );
+ * }
+ */
+ public ApiFuture deleteAuthorizedViewAsync(String tableId, String authorizedViewId) {
+ DeleteAuthorizedViewRequest request =
+ DeleteAuthorizedViewRequest.newBuilder()
+ .setName(
+ NameUtil.formatAuthorizedViewName(projectId, instanceId, tableId, authorizedViewId))
+ .build();
+
+ return transformToVoid(this.stub.deleteAuthorizedViewCallable().futureCall(request));
+ }
+
/**
* Helper method to construct the table name in format:
* projects/{project}/instances/{instance}/tables/{tableId}
@@ -1818,6 +2172,187 @@ public ApiFuture> testBackupIamPermissionAsync(
return testResourceIamPermissions(backupName, permissions);
}
+ /**
+ * Gets the IAM access control policy for the specified authorized view.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * Policy policy = client.getAuthorizedViewIamPolicy("my-table-id", "my-authorized-view-id");
+ * for(Map.Entry> entry : policy.getBindings().entrySet()) {
+ * System.out.printf("Role: %s Identities: %s\n", entry.getKey(), entry.getValue());
+ * }
+ * }
+ *
+ * @see Table-level
+ * IAM management
+ */
+ @SuppressWarnings("WeakerAccess")
+ public Policy getAuthorizedViewIamPolicy(String tableId, String authorizedViewId) {
+ return ApiExceptions.callAndTranslateApiException(
+ getAuthorizedViewIamPolicyAsync(tableId, authorizedViewId));
+ }
+
+ /**
+ * Asynchronously gets the IAM access control policy for the specified authorized view.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * ApiFuture policyFuture = client.getAuthorizedViewIamPolicyAsync("my-table-id", "my-authorized-view-id");
+ *
+ * ApiFutures.addCallback(policyFuture,
+ * new ApiFutureCallback() {
+ * public void onSuccess(Policy policy) {
+ * for (Entry> entry : policy.getBindings().entrySet()) {
+ * System.out.printf("Role: %s Identities: %s\n", entry.getKey(), entry.getValue());
+ * }
+ * }
+ *
+ * public void onFailure(Throwable t) {
+ * t.printStackTrace();
+ * }
+ * },
+ * MoreExecutors.directExecutor());
+ * }
+ *
+ * @see Table-level
+ * IAM management
+ */
+ @SuppressWarnings("WeakerAccess")
+ public ApiFuture getAuthorizedViewIamPolicyAsync(
+ String tableId, String authorizedViewId) {
+ String authorizedViewName =
+ NameUtil.formatAuthorizedViewName(projectId, instanceId, tableId, authorizedViewId);
+ return getResourceIamPolicy(authorizedViewName);
+ }
+
+ /**
+ * Replaces the IAM policy associated with the specified authorized view.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * Policy newPolicy = client.setAuthorizedViewIamPolicy("my-table-id", "my-authorized-view-id",
+ * Policy.newBuilder()
+ * .addIdentity(Role.of("bigtable.user"), Identity.user("someone@example.com"))
+ * .addIdentity(Role.of("bigtable.admin"), Identity.group("admins@example.com"))
+ * .build());
+ * }
+ *
+ * @see Table-level
+ * IAM management
+ */
+ @SuppressWarnings("WeakerAccess")
+ public Policy setAuthorizedViewIamPolicy(String tableId, String authorizedViewId, Policy policy) {
+ return ApiExceptions.callAndTranslateApiException(
+ setAuthorizedViewIamPolicyAsync(tableId, authorizedViewId, policy));
+ }
+
+ /**
+ * Asynchronously replaces the IAM policy associated with the specified authorized view.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * ApiFuture newPolicyFuture = client.setAuthorizedViewIamPolicyAsync("my-table-id", "my-authorized-view-id",
+ * Policy.newBuilder()
+ * .addIdentity(Role.of("bigtable.user"), Identity.user("someone@example.com"))
+ * .addIdentity(Role.of("bigtable.admin"), Identity.group("admins@example.com"))
+ * .build());
+ *
+ * ApiFutures.addCallback(newPolicyFuture,
+ * new ApiFutureCallback() {
+ * public void onSuccess(Policy policy) {
+ * for (Entry> entry : policy.getBindings().entrySet()) {
+ * System.out.printf("Role: %s Identities: %s\n", entry.getKey(), entry.getValue());
+ * }
+ * }
+ *
+ * public void onFailure(Throwable t) {
+ * t.printStackTrace();
+ * }
+ * },
+ * MoreExecutors.directExecutor());
+ * }
+ *
+ * @see Table-level
+ * IAM management
+ */
+ @SuppressWarnings("WeakerAccess")
+ public ApiFuture setAuthorizedViewIamPolicyAsync(
+ String tableId, String authorizedViewId, Policy policy) {
+ String authorizedViewName =
+ NameUtil.formatAuthorizedViewName(projectId, instanceId, tableId, authorizedViewId);
+ return setResourceIamPolicy(policy, authorizedViewName);
+ }
+
+ /**
+ * Tests whether the caller has the given permissions for the specified authorized view. Returns a
+ * subset of the specified permissions that the caller has.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * List grantedPermissions = client.testAuthorizedViewIamPermission("my-table-id", "my-authorized-view-id",
+ * "bigtable.authorizedViews.get", "bigtable.authorizedViews.delete");
+ * }
+ *
+ * System.out.println("Has get access: " +
+ * grantedPermissions.contains("bigtable.authorizedViews.get"));
+ *
+ * System.out.println("Has delete access: " +
+ * grantedPermissions.contains("bigtable.authorizedViews.delete"));
+ *
+ * @see Cloud Bigtable
+ * permissions
+ */
+ @SuppressWarnings({"WeakerAccess"})
+ public List testAuthorizedViewIamPermission(
+ String tableId, String authorizedViewId, String... permissions) {
+ return ApiExceptions.callAndTranslateApiException(
+ testAuthorizedViewIamPermissionAsync(tableId, authorizedViewId, permissions));
+ }
+
+ /**
+ * Asynchronously tests whether the caller has the given permissions for the specified authorized
+ * view. Returns a subset of the specified permissions that the caller has.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * ApiFuture> grantedPermissionsFuture = client.testAuthorizedViewIamPermissionAsync("my-table-id", "my-authorized-view-id",
+ * "bigtable.authorizedViews.get", "bigtable.authorizedViews.delete");
+ *
+ * ApiFutures.addCallback(grantedPermissionsFuture,
+ * new ApiFutureCallback>() {
+ * public void onSuccess(List grantedPermissions) {
+ * System.out.println("Has get access: " + grantedPermissions.contains("bigtable.authorizedViews.get"));
+ * System.out.println("Has delete access: " + grantedPermissions.contains("bigtable.authorizedViews.delete"));
+ * }
+ *
+ * public void onFailure(Throwable t) {
+ * t.printStackTrace();
+ * }
+ * },
+ * MoreExecutors.directExecutor());
+ * }
+ *
+ * @see Cloud Bigtable
+ * permissions
+ */
+ @SuppressWarnings({"WeakerAccess"})
+ public ApiFuture> testAuthorizedViewIamPermissionAsync(
+ String tableId, String authorizedViewId, String... permissions) {
+ String authorizedViewName =
+ NameUtil.formatAuthorizedViewName(projectId, instanceId, tableId, authorizedViewId);
+ return testResourceIamPermissions(authorizedViewName, permissions);
+ }
+
private ApiFuture getResourceIamPolicy(String name) {
GetIamPolicyRequest request = GetIamPolicyRequest.newBuilder().setResource(name).build();
diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/internal/NameUtil.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/internal/NameUtil.java
index 8cccf3d578..a2b59d6b5b 100644
--- a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/internal/NameUtil.java
+++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/internal/NameUtil.java
@@ -34,6 +34,9 @@ public class NameUtil {
private static final Pattern BACKUP_PATTERN =
Pattern.compile("projects/([^/]+)/instances/([^/]+)/clusters/([^/]+)/backups/([^/]+)");
+ private static final Pattern AUTHORIZED_VIEW_PATTERN =
+ Pattern.compile("projects/([^/]+)/instances/([^/]+)/tables/([^/]+)/authorizedViews/([^/]+)");
+
public static String formatProjectName(String projectId) {
return "projects/" + projectId;
}
@@ -55,6 +58,11 @@ public static String formatBackupName(
return formatClusterName(projectId, instanceId, clusterId) + "/backups/" + backupId;
}
+ public static String formatAuthorizedViewName(
+ String projectId, String instanceId, String tableId, String viewId) {
+ return formatTableName(projectId, instanceId, tableId) + "/authorizedViews/" + viewId;
+ }
+
public static String extractTableIdFromTableName(String fullTableName) {
Matcher matcher = TABLE_PATTERN.matcher(fullTableName);
if (!matcher.matches()) {
@@ -71,6 +79,15 @@ public static String extractBackupIdFromBackupName(String fullBackupName) {
return matcher.group(4);
}
+ public static String extractAuthorizedViewIdFromAuthorizedViewName(
+ String fullAuthorizedViewName) {
+ Matcher matcher = AUTHORIZED_VIEW_PATTERN.matcher(fullAuthorizedViewName);
+ if (!matcher.matches()) {
+ throw new IllegalArgumentException("Invalid authorized view name: " + fullAuthorizedViewName);
+ }
+ return matcher.group(4);
+ }
+
public static String extractZoneIdFromLocationName(String fullLocationName) {
Matcher matcher = LOCATION_PATTERN.matcher(fullLocationName);
if (!matcher.matches()) {
diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/models/AuthorizedView.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/models/AuthorizedView.java
new file mode 100644
index 0000000000..33e40f6458
--- /dev/null
+++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/models/AuthorizedView.java
@@ -0,0 +1,124 @@
+/*
+ * 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.admin.v2.models;
+
+import com.google.api.core.InternalApi;
+import com.google.api.core.InternalExtensionOnly;
+import com.google.bigtable.admin.v2.AuthorizedViewName;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import javax.annotation.Nonnull;
+
+/**
+ * A class that wraps the {@link com.google.bigtable.admin.v2.AuthorizedView} protocol buffer
+ * object.
+ *
+ * An AuthorizedView represents subsets of a particular table based on rules. The access to each
+ * AuthorizedView can be configured separately from the Table.
+ *
+ *
Users can perform read/write operation on an AuthorizedView by providing an authorizedView id
+ * besides a table id, in which case the semantics remain identical as reading/writing on a Table
+ * except that visibility is restricted to the subset of the Table that the AuthorizedView
+ * represents.
+ */
+public final class AuthorizedView {
+ private final com.google.bigtable.admin.v2.AuthorizedView proto;
+
+ /**
+ * Wraps the protobuf. This method is considered an internal implementation detail and not meant
+ * to be used by applications.
+ */
+ @InternalApi
+ public static AuthorizedView fromProto(
+ @Nonnull com.google.bigtable.admin.v2.AuthorizedView proto) {
+ return new AuthorizedView(proto);
+ }
+
+ private AuthorizedView(@Nonnull com.google.bigtable.admin.v2.AuthorizedView proto) {
+ Preconditions.checkNotNull(proto);
+ Preconditions.checkArgument(!proto.getName().isEmpty(), "AuthorizedView must have a name");
+ Preconditions.checkArgument(
+ proto.hasSubsetView(), "AuthorizedView must have a subset_view field");
+ this.proto = proto;
+ }
+
+ /** Gets the authorized view's id. */
+ public String getId() {
+ // Constructor ensures that name is not null.
+ AuthorizedViewName fullName = AuthorizedViewName.parse(proto.getName());
+
+ //noinspection ConstantConditions
+ return fullName.getAuthorizedView();
+ }
+
+ /** Gets the id of the table that owns this authorized view. */
+ public String getTableId() {
+ // Constructor ensures that name is not null.
+ AuthorizedViewName fullName = AuthorizedViewName.parse(proto.getName());
+
+ //noinspection ConstantConditions
+ return fullName.getTable();
+ }
+
+ /** Returns whether this authorized view is deletion protected. */
+ public boolean isDeletionProtected() {
+ return proto.getDeletionProtection();
+ }
+
+ /** Gets the type of this authorized view, which currently can only be a subset view. */
+ public AuthorizedViewType getAuthorizedViewType() {
+ if (proto.hasSubsetView()) {
+ return SubsetView.fromProto(proto.getSubsetView());
+ } else {
+ // Should never happen because the constructor verifies that one must exist.
+ throw new IllegalStateException("This AuthorizedView doesn't have a valid type specified");
+ }
+ }
+
+ /**
+ * Creates the request protobuf. This method is considered an internal implementation detail and
+ * not meant to be used by applications.
+ */
+ @InternalApi
+ public com.google.bigtable.admin.v2.AuthorizedView toProto() {
+ return proto;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ AuthorizedView that = (AuthorizedView) o;
+ return Objects.equal(proto, that.proto);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hashCode(proto);
+ }
+
+ /**
+ * Represents a subset of a Table. Please check the implementations of this interface for more
+ * details.
+ */
+ @InternalExtensionOnly
+ public interface AuthorizedViewType {}
+}
diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/models/CreateAuthorizedViewRequest.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/models/CreateAuthorizedViewRequest.java
new file mode 100644
index 0000000000..0c251fa666
--- /dev/null
+++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/models/CreateAuthorizedViewRequest.java
@@ -0,0 +1,117 @@
+/*
+ * 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.admin.v2.models;
+
+import com.google.api.core.InternalApi;
+import com.google.cloud.bigtable.admin.v2.internal.NameUtil;
+import com.google.cloud.bigtable.admin.v2.models.AuthorizedView.AuthorizedViewType;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import javax.annotation.Nonnull;
+
+/**
+ * Parameters for creating a new Cloud Bigtable {@link AuthorizedView}, which represents subsets of
+ * a particular table.
+ *
+ *
Sample code:
+ *
+ *
{@code
+ * CreateAuthorizedViewRequest request =
+ * CreateAuthorizedViewRequest.of("my-table", "my-new-authorized-view")
+ * .setAuthorizedViewType(
+ * SubsetView.create()
+ * .addRowPrefix("row#")
+ * .addFamilySubsets(
+ * "my-family", FamilySubsets.create().addQualifier("column")));
+ * }
+ *
+ * @see AuthorizedView for more details.
+ */
+public final class CreateAuthorizedViewRequest {
+ private final String tableId;
+ private final com.google.bigtable.admin.v2.CreateAuthorizedViewRequest.Builder requestBuilder =
+ com.google.bigtable.admin.v2.CreateAuthorizedViewRequest.newBuilder();
+
+ public static CreateAuthorizedViewRequest of(
+ @Nonnull String tableId, @Nonnull String authorizedViewId) {
+ return new CreateAuthorizedViewRequest(tableId, authorizedViewId);
+ }
+
+ private CreateAuthorizedViewRequest(@Nonnull String tableId, @Nonnull String authorizedViewId) {
+ Preconditions.checkNotNull(tableId, "tableId must be set");
+ Preconditions.checkNotNull(authorizedViewId, "authorizedViewId must be set");
+
+ this.tableId = tableId;
+ requestBuilder.setAuthorizedViewId(authorizedViewId);
+ }
+
+ /** Configures if the authorized view is deletion protected. */
+ public CreateAuthorizedViewRequest setDeletionProtection(boolean deletionProtection) {
+ requestBuilder.getAuthorizedViewBuilder().setDeletionProtection(deletionProtection);
+ return this;
+ }
+
+ /**
+ * Sets the implementation for this authorized view.
+ *
+ * @see AuthorizedViewType for details.
+ */
+ public CreateAuthorizedViewRequest setAuthorizedViewType(
+ @Nonnull AuthorizedViewType authorizedViewType) {
+ Preconditions.checkNotNull(authorizedViewType, "authorizedViewType must be set");
+
+ if (authorizedViewType instanceof SubsetView) {
+ requestBuilder
+ .getAuthorizedViewBuilder()
+ .setSubsetView(((SubsetView) authorizedViewType).toProto());
+ } else {
+ throw new IllegalArgumentException("Unknown authorizedViewType: " + authorizedViewType);
+ }
+
+ return this;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ CreateAuthorizedViewRequest that = (CreateAuthorizedViewRequest) o;
+ return Objects.equal(requestBuilder.build(), that.requestBuilder.build())
+ && Objects.equal(tableId, that.tableId);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hashCode(requestBuilder.build(), tableId);
+ }
+
+ /**
+ * Creates the request protobuf. This method is considered an internal implementation detail and
+ * not meant to be used by applications.
+ */
+ @InternalApi
+ public com.google.bigtable.admin.v2.CreateAuthorizedViewRequest toProto(
+ @Nonnull String projectId, @Nonnull String instanceId) {
+ return requestBuilder
+ .setParent(NameUtil.formatTableName(projectId, instanceId, tableId))
+ .build();
+ }
+}
diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/models/FamilySubsets.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/models/FamilySubsets.java
new file mode 100644
index 0000000000..e80452b2af
--- /dev/null
+++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/models/FamilySubsets.java
@@ -0,0 +1,119 @@
+/*
+ * 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.admin.v2.models;
+
+import com.google.api.core.InternalApi;
+import com.google.common.base.Objects;
+import com.google.common.collect.ImmutableList;
+import com.google.protobuf.ByteString;
+import java.util.List;
+import javax.annotation.Nonnull;
+
+/** Represents subsets of a particular column family that are included in this authorized view. */
+public final class FamilySubsets {
+ private final com.google.bigtable.admin.v2.AuthorizedView.FamilySubsets.Builder builder;
+
+ /**
+ * Wraps the protobuf. This method is considered an internal implementation detail and not meant
+ * to be used by applications.
+ */
+ @InternalApi
+ public static FamilySubsets fromProto(
+ @Nonnull com.google.bigtable.admin.v2.AuthorizedView.FamilySubsets proto) {
+ return new FamilySubsets(proto);
+ }
+
+ public static FamilySubsets create() {
+ return new FamilySubsets();
+ }
+
+ private FamilySubsets(@Nonnull com.google.bigtable.admin.v2.AuthorizedView.FamilySubsets proto) {
+ this.builder = proto.toBuilder();
+ }
+
+ private FamilySubsets() {
+ this.builder = com.google.bigtable.admin.v2.AuthorizedView.FamilySubsets.newBuilder();
+ }
+
+ /** Gets the list of column qualifiers included in this authorized view. */
+ public List getQualifiers() {
+ return ImmutableList.copyOf(this.builder.getQualifiersList());
+ }
+
+ /** Gets the list of column qualifier prefixes included in this authorized view. */
+ public List getQualifierPrefixes() {
+ return ImmutableList.copyOf(this.builder.getQualifierPrefixesList());
+ }
+
+ /** Adds an individual column qualifier to be included in this authorized view. */
+ public FamilySubsets addQualifier(ByteString qualifier) {
+ this.builder.addQualifiers(qualifier);
+ return this;
+ }
+
+ /** Adds an individual column qualifier to be included in this authorized view. */
+ public FamilySubsets addQualifier(String qualifier) {
+ this.builder.addQualifiers(ByteString.copyFromUtf8(qualifier));
+ return this;
+ }
+
+ /**
+ * Adds a prefix for column qualifiers to be included in this authorized view. Every qualifier
+ * starting with the prefix will be included in this authorized view. An empty string ("") prefix
+ * means to provide access to all qualifiers.
+ */
+ public FamilySubsets addQualifierPrefix(ByteString qualifierPrefix) {
+ this.builder.addQualifierPrefixes(qualifierPrefix);
+ return this;
+ }
+
+ /**
+ * Adds a prefix for column qualifiers to be included in this authorized view. Every qualifier
+ * starting with the prefix will be included in this authorized view. An empty string ("") prefix
+ * means to provide access to all qualifiers.
+ */
+ public FamilySubsets addQualifierPrefix(String qualifierPrefix) {
+ this.builder.addQualifierPrefixes(ByteString.copyFromUtf8(qualifierPrefix));
+ return this;
+ }
+
+ /**
+ * Creates the request protobuf. This method is considered an internal implementation detail and
+ * not meant to be used by applications.
+ */
+ @InternalApi
+ public com.google.bigtable.admin.v2.AuthorizedView.FamilySubsets toProto() {
+ return builder.build();
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ FamilySubsets that = (FamilySubsets) o;
+ return Objects.equal(builder.build(), that.builder.build());
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hashCode(builder.build());
+ }
+}
diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/models/SubsetView.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/models/SubsetView.java
new file mode 100644
index 0000000000..6ace603e46
--- /dev/null
+++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/models/SubsetView.java
@@ -0,0 +1,123 @@
+/*
+ * 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.admin.v2.models;
+
+import com.google.api.core.InternalApi;
+import com.google.cloud.bigtable.admin.v2.models.AuthorizedView.AuthorizedViewType;
+import com.google.common.base.Objects;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.protobuf.ByteString;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import javax.annotation.Nonnull;
+
+/**
+ * Defines a simple authorized view that is a subset of the underlying Table.
+ *
+ * Users can specify the rows in the form of row key prefixes, and specify the column families by
+ * adding the family id along with its familySubsets rule to the family subsets map. The subset is
+ * defined by the intersection of the specified row key prefixes and column family subsets.
+ */
+public class SubsetView implements AuthorizedViewType {
+ private final com.google.bigtable.admin.v2.AuthorizedView.SubsetView.Builder builder;
+
+ /**
+ * Wraps the protobuf. This method is considered an internal implementation detail and not meant
+ * to be used by applications.
+ */
+ @InternalApi
+ public static SubsetView fromProto(
+ @Nonnull com.google.bigtable.admin.v2.AuthorizedView.SubsetView proto) {
+ return new SubsetView(proto);
+ }
+
+ public static SubsetView create() {
+ return new SubsetView();
+ }
+
+ private SubsetView(@Nonnull com.google.bigtable.admin.v2.AuthorizedView.SubsetView proto) {
+ this.builder = proto.toBuilder();
+ }
+
+ private SubsetView() {
+ this.builder = com.google.bigtable.admin.v2.AuthorizedView.SubsetView.newBuilder();
+ }
+
+ /** Gets the row prefixes to be included in this subset view. */
+ public List getRowPrefixes() {
+ return ImmutableList.copyOf(this.builder.getRowPrefixesList());
+ }
+
+ /** Gets the map from familyId to familySubsets in this subset view. */
+ public Map getFamilySubsets() {
+ ImmutableMap.Builder familySubsets = ImmutableMap.builder();
+ for (Entry entry :
+ builder.getFamilySubsetsMap().entrySet()) {
+ familySubsets.put(entry.getKey(), FamilySubsets.fromProto(entry.getValue()));
+ }
+ return familySubsets.build();
+ }
+
+ /** Adds a new rowPrefix to the subset view. */
+ public SubsetView addRowPrefix(ByteString rowPrefix) {
+ this.builder.addRowPrefixes(rowPrefix);
+ return this;
+ }
+
+ /** Adds a new rowPrefix to the subset view. */
+ public SubsetView addRowPrefix(String rowPrefix) {
+ this.builder.addRowPrefixes(ByteString.copyFromUtf8(rowPrefix));
+ return this;
+ }
+
+ /**
+ * Adds a new familyId with its familySubsets to the subset view. Please note that calling this
+ * method with the same familyId will overwrite the previous rule set on the family.
+ */
+ public SubsetView setFamilySubsets(String familyId, FamilySubsets familySubsets) {
+ this.builder.putFamilySubsets(familyId, familySubsets.toProto());
+ return this;
+ }
+
+ /**
+ * Creates the request protobuf. This method is considered an internal implementation detail and
+ * not meant to be used by applications.
+ */
+ @InternalApi
+ public com.google.bigtable.admin.v2.AuthorizedView.SubsetView toProto() {
+ return builder.build();
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ SubsetView that = (SubsetView) o;
+ return Objects.equal(builder.build(), that.builder.build());
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hashCode(builder.build());
+ }
+}
diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/models/UpdateAuthorizedViewRequest.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/models/UpdateAuthorizedViewRequest.java
new file mode 100644
index 0000000000..fbb54c994e
--- /dev/null
+++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/admin/v2/models/UpdateAuthorizedViewRequest.java
@@ -0,0 +1,154 @@
+/*
+ * 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.admin.v2.models;
+
+import com.google.api.core.InternalApi;
+import com.google.cloud.bigtable.admin.v2.internal.NameUtil;
+import com.google.cloud.bigtable.admin.v2.models.AuthorizedView.AuthorizedViewType;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.protobuf.FieldMask;
+import com.google.protobuf.util.FieldMaskUtil;
+import javax.annotation.Nonnull;
+
+/**
+ * Parameters for updating an existing Cloud Bigtable {@link AuthorizedView}.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * AuthorizedView existingAuthorizedView = client.getAuthorizedView("my-table", "my-authorized-view");
+ * UpdateAuthorizedViewRequest request =
+ * UpdateAuthorizedViewRequest.of(existingAuthorizedView).setDeletionProtection(true);
+ * }
+ *
+ * @see AuthorizedView for more details.
+ */
+public final class UpdateAuthorizedViewRequest {
+ private final com.google.bigtable.admin.v2.UpdateAuthorizedViewRequest.Builder requestBuilder;
+ private final String tableId;
+ private final String authorizedViewId;
+
+ /** Builds a new update request using an existing authorized view. */
+ public static UpdateAuthorizedViewRequest of(@Nonnull AuthorizedView authorizedView) {
+ return new UpdateAuthorizedViewRequest(
+ authorizedView.getTableId(),
+ authorizedView.getId(),
+ com.google.bigtable.admin.v2.UpdateAuthorizedViewRequest.newBuilder()
+ .setAuthorizedView(authorizedView.toProto()));
+ }
+
+ /** Builds a new update authorized view request. */
+ public static UpdateAuthorizedViewRequest of(
+ @Nonnull String tableId, @Nonnull String authorizedViewId) {
+ return new UpdateAuthorizedViewRequest(
+ tableId,
+ authorizedViewId,
+ com.google.bigtable.admin.v2.UpdateAuthorizedViewRequest.newBuilder());
+ }
+
+ private UpdateAuthorizedViewRequest(
+ @Nonnull String tableId,
+ @Nonnull String authorizedViewId,
+ @Nonnull com.google.bigtable.admin.v2.UpdateAuthorizedViewRequest.Builder requestBuilder) {
+ Preconditions.checkNotNull(tableId, "tableId must be set");
+ Preconditions.checkNotNull(authorizedViewId, "authorizedViewId must be set");
+ Preconditions.checkNotNull(requestBuilder, "proto builder must be set");
+
+ this.tableId = tableId;
+ this.authorizedViewId = authorizedViewId;
+ this.requestBuilder = requestBuilder;
+ }
+
+ /** Changes the deletion protection of an existing authorized view. */
+ public UpdateAuthorizedViewRequest setDeletionProtection(boolean deletionProtection) {
+ requestBuilder.getAuthorizedViewBuilder().setDeletionProtection(deletionProtection);
+ updateFieldMask(com.google.bigtable.admin.v2.AuthorizedView.DELETION_PROTECTION_FIELD_NUMBER);
+ return this;
+ }
+
+ /**
+ * Updates the implementation for this authorized view.
+ *
+ * @see AuthorizedViewType for details.
+ */
+ public UpdateAuthorizedViewRequest setAuthorizedViewType(
+ @Nonnull AuthorizedViewType authorizedViewType) {
+ Preconditions.checkNotNull(authorizedViewType, "authorizedViewType must be set");
+
+ if (authorizedViewType instanceof SubsetView) {
+ requestBuilder
+ .getAuthorizedViewBuilder()
+ .setSubsetView(((SubsetView) authorizedViewType).toProto());
+ updateFieldMask(com.google.bigtable.admin.v2.AuthorizedView.SUBSET_VIEW_FIELD_NUMBER);
+ } else {
+ throw new IllegalArgumentException("Unknown authorizedViewType: " + authorizedViewType);
+ }
+
+ return this;
+ }
+
+ /**
+ * Configures if safety warnings should be disabled. If set, then updates that making the
+ * authorized view more restrictive are allowed.
+ */
+ @SuppressWarnings("WeakerAccess")
+ public UpdateAuthorizedViewRequest setIgnoreWarnings(boolean value) {
+ requestBuilder.setIgnoreWarnings(value);
+ return this;
+ }
+
+ private void updateFieldMask(int fieldNumber) {
+ FieldMask newMask =
+ FieldMaskUtil.fromFieldNumbers(
+ com.google.bigtable.admin.v2.AuthorizedView.class, fieldNumber);
+ requestBuilder.setUpdateMask(FieldMaskUtil.union(requestBuilder.getUpdateMask(), newMask));
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ UpdateAuthorizedViewRequest that = (UpdateAuthorizedViewRequest) o;
+ return Objects.equal(requestBuilder.build(), that.requestBuilder.build())
+ && Objects.equal(tableId, that.tableId)
+ && Objects.equal(authorizedViewId, that.authorizedViewId);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hashCode(requestBuilder.build(), tableId, authorizedViewId);
+ }
+
+ /**
+ * Creates the request protobuf. This method is considered an internal implementation detail and
+ * not meant to be used by applications.
+ */
+ @InternalApi
+ public com.google.bigtable.admin.v2.UpdateAuthorizedViewRequest toProto(
+ @Nonnull String projectId, @Nonnull String instanceId) {
+ requestBuilder
+ .getAuthorizedViewBuilder()
+ .setName(
+ NameUtil.formatAuthorizedViewName(projectId, instanceId, tableId, authorizedViewId));
+ return requestBuilder.build();
+ }
+}
diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/BigtableTableAdminClientTests.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/BigtableTableAdminClientTests.java
index 86afadd237..e604495c43 100644
--- a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/BigtableTableAdminClientTests.java
+++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/BigtableTableAdminClientTests.java
@@ -27,11 +27,13 @@
import com.google.api.gax.rpc.OperationCallable;
import com.google.api.gax.rpc.UnaryCallable;
import com.google.api.gax.rpc.testing.FakeOperationSnapshot;
+import com.google.bigtable.admin.v2.AuthorizedViewName;
import com.google.bigtable.admin.v2.Backup.State;
import com.google.bigtable.admin.v2.BackupInfo;
import com.google.bigtable.admin.v2.ChangeStreamConfig;
import com.google.bigtable.admin.v2.ColumnFamily;
import com.google.bigtable.admin.v2.CopyBackupMetadata;
+import com.google.bigtable.admin.v2.CreateAuthorizedViewMetadata;
import com.google.bigtable.admin.v2.CreateBackupMetadata;
import com.google.bigtable.admin.v2.DeleteBackupRequest;
import com.google.bigtable.admin.v2.DeleteTableRequest;
@@ -47,25 +49,32 @@
import com.google.bigtable.admin.v2.Table.ClusterState;
import com.google.bigtable.admin.v2.Table.View;
import com.google.bigtable.admin.v2.TableName;
+import com.google.bigtable.admin.v2.UpdateAuthorizedViewMetadata;
import com.google.bigtable.admin.v2.UpdateTableMetadata;
import com.google.cloud.Identity;
import com.google.cloud.Policy;
import com.google.cloud.Role;
+import com.google.cloud.bigtable.admin.v2.BaseBigtableTableAdminClient.ListAuthorizedViewsPage;
+import com.google.cloud.bigtable.admin.v2.BaseBigtableTableAdminClient.ListAuthorizedViewsPagedResponse;
import com.google.cloud.bigtable.admin.v2.BaseBigtableTableAdminClient.ListBackupsPage;
import com.google.cloud.bigtable.admin.v2.BaseBigtableTableAdminClient.ListBackupsPagedResponse;
import com.google.cloud.bigtable.admin.v2.BaseBigtableTableAdminClient.ListTablesPage;
import com.google.cloud.bigtable.admin.v2.BaseBigtableTableAdminClient.ListTablesPagedResponse;
import com.google.cloud.bigtable.admin.v2.internal.NameUtil;
+import com.google.cloud.bigtable.admin.v2.models.AuthorizedView;
import com.google.cloud.bigtable.admin.v2.models.Backup;
import com.google.cloud.bigtable.admin.v2.models.CopyBackupRequest;
+import com.google.cloud.bigtable.admin.v2.models.CreateAuthorizedViewRequest;
import com.google.cloud.bigtable.admin.v2.models.CreateBackupRequest;
import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest;
import com.google.cloud.bigtable.admin.v2.models.EncryptionInfo;
import com.google.cloud.bigtable.admin.v2.models.ModifyColumnFamiliesRequest;
import com.google.cloud.bigtable.admin.v2.models.RestoreTableRequest;
import com.google.cloud.bigtable.admin.v2.models.RestoredTableResult;
+import com.google.cloud.bigtable.admin.v2.models.SubsetView;
import com.google.cloud.bigtable.admin.v2.models.Table;
import com.google.cloud.bigtable.admin.v2.models.Type;
+import com.google.cloud.bigtable.admin.v2.models.UpdateAuthorizedViewRequest;
import com.google.cloud.bigtable.admin.v2.models.UpdateBackupRequest;
import com.google.cloud.bigtable.admin.v2.stub.EnhancedBigtableTableAdminStub;
import com.google.common.collect.ImmutableList;
@@ -75,6 +84,7 @@
import com.google.protobuf.ByteString;
import com.google.protobuf.Duration;
import com.google.protobuf.Empty;
+import com.google.protobuf.FieldMask;
import com.google.protobuf.Timestamp;
import com.google.protobuf.util.Timestamps;
import io.grpc.Status;
@@ -110,6 +120,7 @@ public class BigtableTableAdminClientTests {
private static final String TABLE_ID = "my-table";
private static final String CLUSTER_ID = "my-cluster";
private static final String BACKUP_ID = "my-backup";
+ private static final String AUTHORIZED_VIEW_ID = "my-authorized-view";
private static final String INSTANCE_NAME = NameUtil.formatInstanceName(PROJECT_ID, INSTANCE_ID);
private static final String TABLE_NAME =
@@ -182,6 +193,35 @@ public class BigtableTableAdminClientTests {
CopyBackupMetadata>
mockCopyBackupOperationCallable;
+ @Mock
+ private OperationCallable<
+ com.google.bigtable.admin.v2.CreateAuthorizedViewRequest,
+ com.google.bigtable.admin.v2.AuthorizedView,
+ CreateAuthorizedViewMetadata>
+ mockCreateAuthorizedViewOperationCallable;
+
+ @Mock
+ private OperationCallable<
+ com.google.bigtable.admin.v2.UpdateAuthorizedViewRequest,
+ com.google.bigtable.admin.v2.AuthorizedView,
+ UpdateAuthorizedViewMetadata>
+ mockUpdateAuthorizedViewOperationCallable;
+
+ @Mock
+ private UnaryCallable<
+ com.google.bigtable.admin.v2.GetAuthorizedViewRequest,
+ com.google.bigtable.admin.v2.AuthorizedView>
+ mockGetAuthorizedViewCallable;
+
+ @Mock
+ private UnaryCallable<
+ com.google.bigtable.admin.v2.ListAuthorizedViewsRequest, ListAuthorizedViewsPagedResponse>
+ mockListAuthorizedViewsCallable;
+
+ @Mock
+ private UnaryCallable
+ mockDeleteAuthorizedViewCallable;
+
@Mock
private UnaryCallable
mockGetIamPolicyCallable;
@@ -877,6 +917,227 @@ public void testCopyBackup() {
assertThat(actualResult.getSizeBytes()).isEqualTo(sizeBytes);
}
+ @Test
+ public void testCreateAuthorizedView() {
+ // Setup
+ Mockito.when(mockStub.createAuthorizedViewOperationCallable())
+ .thenReturn(mockCreateAuthorizedViewOperationCallable);
+
+ com.google.bigtable.admin.v2.CreateAuthorizedViewRequest expectedRequest =
+ com.google.bigtable.admin.v2.CreateAuthorizedViewRequest.newBuilder()
+ .setParent(NameUtil.formatTableName(PROJECT_ID, INSTANCE_ID, TABLE_ID))
+ .setAuthorizedViewId(AUTHORIZED_VIEW_ID)
+ .setAuthorizedView(
+ com.google.bigtable.admin.v2.AuthorizedView.newBuilder()
+ .setSubsetView(
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView.newBuilder()
+ .addRowPrefixes(ByteString.copyFromUtf8("row#"))
+ .build())
+ .setDeletionProtection(true)
+ .build())
+ .build();
+
+ com.google.bigtable.admin.v2.AuthorizedView expectedResponse =
+ com.google.bigtable.admin.v2.AuthorizedView.newBuilder()
+ .setName(
+ NameUtil.formatAuthorizedViewName(
+ PROJECT_ID, INSTANCE_ID, TABLE_ID, AUTHORIZED_VIEW_ID))
+ .setSubsetView(
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView.newBuilder()
+ .addRowPrefixes(ByteString.copyFromUtf8("row#"))
+ .build())
+ .setDeletionProtection(true)
+ .build();
+
+ mockOperationResult(
+ mockCreateAuthorizedViewOperationCallable,
+ expectedRequest,
+ expectedResponse,
+ CreateAuthorizedViewMetadata.newBuilder().setOriginalRequest(expectedRequest).build());
+
+ CreateAuthorizedViewRequest req =
+ CreateAuthorizedViewRequest.of(TABLE_ID, AUTHORIZED_VIEW_ID)
+ .setDeletionProtection(true)
+ .setAuthorizedViewType(SubsetView.create().addRowPrefix("row#"));
+
+ // Execute
+ AuthorizedView actualResult = adminClient.createAuthorizedView(req);
+
+ // Verify
+ assertThat(actualResult).isEqualTo(AuthorizedView.fromProto(expectedResponse));
+ }
+
+ @Test
+ public void testUpdateAuthorizedView() {
+ // Setup
+ Mockito.when(mockStub.updateAuthorizedViewOperationCallable())
+ .thenReturn(mockUpdateAuthorizedViewOperationCallable);
+
+ com.google.bigtable.admin.v2.UpdateAuthorizedViewRequest expectedRequest =
+ com.google.bigtable.admin.v2.UpdateAuthorizedViewRequest.newBuilder()
+ .setAuthorizedView(
+ com.google.bigtable.admin.v2.AuthorizedView.newBuilder()
+ .setName(
+ NameUtil.formatAuthorizedViewName(
+ PROJECT_ID, INSTANCE_ID, TABLE_ID, AUTHORIZED_VIEW_ID))
+ .setSubsetView(
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView.newBuilder()
+ .addRowPrefixes(ByteString.copyFromUtf8("row#"))
+ .build())
+ .setDeletionProtection(true)
+ .build())
+ .setUpdateMask(
+ FieldMask.newBuilder().addPaths("deletion_protection").addPaths("subset_view"))
+ .build();
+
+ com.google.bigtable.admin.v2.AuthorizedView expectedResponse =
+ com.google.bigtable.admin.v2.AuthorizedView.newBuilder()
+ .setName(
+ NameUtil.formatAuthorizedViewName(
+ PROJECT_ID, INSTANCE_ID, TABLE_ID, AUTHORIZED_VIEW_ID))
+ .setSubsetView(
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView.newBuilder()
+ .addRowPrefixes(ByteString.copyFromUtf8("row#"))
+ .build())
+ .setDeletionProtection(true)
+ .build();
+
+ mockOperationResult(
+ mockUpdateAuthorizedViewOperationCallable,
+ expectedRequest,
+ expectedResponse,
+ UpdateAuthorizedViewMetadata.newBuilder().setOriginalRequest(expectedRequest).build());
+
+ UpdateAuthorizedViewRequest req =
+ UpdateAuthorizedViewRequest.of(TABLE_ID, AUTHORIZED_VIEW_ID)
+ .setDeletionProtection(true)
+ .setAuthorizedViewType(SubsetView.create().addRowPrefix("row#"));
+
+ // Execute
+ AuthorizedView actualResult = adminClient.updateAuthorizedView(req);
+
+ // Verify
+ assertThat(actualResult).isEqualTo(AuthorizedView.fromProto(expectedResponse));
+ }
+
+ @Test
+ public void testGetAuthorizedView() {
+ // Setup
+ Mockito.when(mockStub.getAuthorizedViewCallable()).thenReturn(mockGetAuthorizedViewCallable);
+
+ com.google.bigtable.admin.v2.GetAuthorizedViewRequest expectedRequest =
+ com.google.bigtable.admin.v2.GetAuthorizedViewRequest.newBuilder()
+ .setName(
+ NameUtil.formatAuthorizedViewName(
+ PROJECT_ID, INSTANCE_ID, TABLE_ID, AUTHORIZED_VIEW_ID))
+ .build();
+
+ com.google.bigtable.admin.v2.AuthorizedView expectedResponse =
+ com.google.bigtable.admin.v2.AuthorizedView.newBuilder()
+ .setName(
+ NameUtil.formatAuthorizedViewName(
+ PROJECT_ID, INSTANCE_ID, TABLE_ID, AUTHORIZED_VIEW_ID))
+ .setSubsetView(
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView.newBuilder()
+ .addRowPrefixes(ByteString.copyFromUtf8("row#"))
+ .build())
+ .setDeletionProtection(true)
+ .build();
+
+ Mockito.when(mockGetAuthorizedViewCallable.futureCall(expectedRequest))
+ .thenReturn(ApiFutures.immediateFuture(expectedResponse));
+
+ // Execute
+ AuthorizedView actualResult = adminClient.getAuthorizedView(TABLE_ID, AUTHORIZED_VIEW_ID);
+
+ // Verify
+ assertThat(actualResult).isEqualTo(AuthorizedView.fromProto(expectedResponse));
+ }
+
+ @Test
+ public void testListAuthorizedViews() {
+ // Setup
+ Mockito.when(mockStub.listAuthorizedViewsPagedCallable())
+ .thenReturn(mockListAuthorizedViewsCallable);
+
+ com.google.bigtable.admin.v2.ListAuthorizedViewsRequest expectedRequest =
+ com.google.bigtable.admin.v2.ListAuthorizedViewsRequest.newBuilder()
+ .setParent(NameUtil.formatTableName(PROJECT_ID, INSTANCE_ID, TABLE_ID))
+ .build();
+
+ // 3 AuthorizedViews spread across 2 pages
+ List expectedProtos = Lists.newArrayList();
+ for (int i = 0; i < 3; i++) {
+ expectedProtos.add(
+ com.google.bigtable.admin.v2.AuthorizedView.newBuilder()
+ .setName(
+ NameUtil.formatAuthorizedViewName(
+ PROJECT_ID, INSTANCE_ID, TABLE_ID, AUTHORIZED_VIEW_ID + i))
+ .build());
+ }
+
+ // 2 on the first page
+ ListAuthorizedViewsPage page0 = Mockito.mock(ListAuthorizedViewsPage.class);
+ Mockito.when(page0.getValues()).thenReturn(expectedProtos.subList(0, 2));
+ Mockito.when(page0.hasNextPage()).thenReturn(true);
+
+ // 1 on the last page
+ ListAuthorizedViewsPage page1 = Mockito.mock(ListAuthorizedViewsPage.class);
+ Mockito.when(page1.getValues()).thenReturn(expectedProtos.subList(2, 3));
+
+ // Link page0 to page1
+ Mockito.when(page0.getNextPageAsync()).thenReturn(ApiFutures.immediateFuture(page1));
+
+ // Link page to the response
+ ListAuthorizedViewsPagedResponse response0 =
+ Mockito.mock(ListAuthorizedViewsPagedResponse.class);
+ Mockito.when(response0.getPage()).thenReturn(page0);
+
+ Mockito.when(mockListAuthorizedViewsCallable.futureCall(expectedRequest))
+ .thenReturn(ApiFutures.immediateFuture(response0));
+
+ // Execute
+ List actualResults = adminClient.listAuthorizedViews(TABLE_ID);
+
+ // Verify
+ List expectedResults = Lists.newArrayList();
+ for (com.google.bigtable.admin.v2.AuthorizedView expectedProto : expectedProtos) {
+ expectedResults.add(AuthorizedViewName.parse(expectedProto.getName()).getAuthorizedView());
+ }
+
+ assertThat(actualResults).containsExactlyElementsIn(expectedResults);
+ }
+
+ @Test
+ public void testDeleteAuthorizedView() {
+ // Setup
+ Mockito.when(mockStub.deleteAuthorizedViewCallable())
+ .thenReturn(mockDeleteAuthorizedViewCallable);
+
+ com.google.bigtable.admin.v2.DeleteAuthorizedViewRequest expectedRequest =
+ com.google.bigtable.admin.v2.DeleteAuthorizedViewRequest.newBuilder()
+ .setName(
+ NameUtil.formatAuthorizedViewName(
+ PROJECT_ID, INSTANCE_ID, TABLE_ID, AUTHORIZED_VIEW_ID))
+ .build();
+
+ final AtomicBoolean wasCalled = new AtomicBoolean(false);
+
+ Mockito.when(mockDeleteAuthorizedViewCallable.futureCall(expectedRequest))
+ .thenAnswer(
+ (Answer>)
+ invocationOnMock -> {
+ wasCalled.set(true);
+ return ApiFutures.immediateFuture(Empty.getDefaultInstance());
+ });
+
+ // Execute
+ adminClient.deleteAuthorizedView(TABLE_ID, AUTHORIZED_VIEW_ID);
+
+ // Verify
+ assertThat(wasCalled.get()).isTrue();
+ }
+
@Test
public void testGetBackupIamPolicy() {
// Setup
diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/internal/NameUtilTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/internal/NameUtilTest.java
index a452a2bc55..cb2325d1a0 100644
--- a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/internal/NameUtilTest.java
+++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/internal/NameUtilTest.java
@@ -45,4 +45,27 @@ public void formatBackupNameTest() {
assertThat(NameUtil.formatBackupName("my-project", "my-instance", "my-cluster", "my-backup"))
.isEqualTo(testBackupName);
}
+
+ @Test
+ public void formatAuthorizedViewNameTest() {
+ String testAuthorizedViewName =
+ "projects/my-project/instances/my-instance/tables/my-table/authorizedViews/my-authorized-view";
+
+ assertThat(
+ NameUtil.formatAuthorizedViewName(
+ "my-project", "my-instance", "my-table", "my-authorized-view"))
+ .isEqualTo(testAuthorizedViewName);
+ }
+
+ @Test
+ public void extractAuthorizedViewIdFromAuthorizedViewNameTest() {
+ String testAuthorizedViewName =
+ "projects/my-project/instances/my-instance/tables/my-table/authorizedViews/my-authorized-view";
+
+ assertThat(NameUtil.extractAuthorizedViewIdFromAuthorizedViewName(testAuthorizedViewName))
+ .isEqualTo("my-authorized-view");
+
+ exception.expect(IllegalArgumentException.class);
+ NameUtil.extractAuthorizedViewIdFromAuthorizedViewName("bad-format");
+ }
}
diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/it/BigtableAuthorizedViewIT.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/it/BigtableAuthorizedViewIT.java
new file mode 100644
index 0000000000..eb23e0325a
--- /dev/null
+++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/it/BigtableAuthorizedViewIT.java
@@ -0,0 +1,275 @@
+/*
+ * 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.admin.v2.it;
+
+import static com.google.common.truth.Truth.assertThat;
+import static com.google.common.truth.Truth.assertWithMessage;
+import static com.google.common.truth.TruthJUnit.assume;
+import static org.junit.Assert.fail;
+
+import com.google.api.gax.batching.Batcher;
+import com.google.api.gax.rpc.FailedPreconditionException;
+import com.google.api.gax.rpc.NotFoundException;
+import com.google.cloud.Policy;
+import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient;
+import com.google.cloud.bigtable.admin.v2.models.AuthorizedView;
+import com.google.cloud.bigtable.admin.v2.models.CreateAuthorizedViewRequest;
+import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest;
+import com.google.cloud.bigtable.admin.v2.models.FamilySubsets;
+import com.google.cloud.bigtable.admin.v2.models.SubsetView;
+import com.google.cloud.bigtable.admin.v2.models.Table;
+import com.google.cloud.bigtable.admin.v2.models.UpdateAuthorizedViewRequest;
+import com.google.cloud.bigtable.data.v2.BigtableDataClient;
+import com.google.cloud.bigtable.data.v2.models.RowMutationEntry;
+import com.google.cloud.bigtable.test_helpers.env.EmulatorEnv;
+import com.google.cloud.bigtable.test_helpers.env.PrefixGenerator;
+import com.google.cloud.bigtable.test_helpers.env.TestEnvRule;
+import com.google.protobuf.ByteString;
+import io.grpc.StatusRuntimeException;
+import java.util.List;
+import java.util.Random;
+import java.util.logging.Logger;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class BigtableAuthorizedViewIT {
+ @ClassRule public static final TestEnvRule testEnvRule = new TestEnvRule();
+ @Rule public final PrefixGenerator prefixGenerator = new PrefixGenerator();
+ private static final Logger LOGGER = Logger.getLogger(BigtableAuthorizedViewIT.class.getName());
+ private static final int[] BACKOFF_DURATION = {2, 4, 8, 16, 32, 64, 128, 256, 512, 1024};
+
+ private static BigtableTableAdminClient tableAdmin;
+ private static BigtableDataClient dataClient;
+ private static Table testTable;
+
+ @BeforeClass
+ public static void setUpClass() throws InterruptedException {
+ assume()
+ .withMessage("BigtableInstanceAdminClient is not supported on Emulator")
+ .that(testEnvRule.env())
+ .isNotInstanceOf(EmulatorEnv.class);
+
+ tableAdmin = testEnvRule.env().getTableAdminClient();
+ dataClient = testEnvRule.env().getDataClient();
+
+ testTable = createAndPopulateTestTable(tableAdmin, dataClient);
+ }
+
+ @AfterClass
+ public static void tearDownClass() {
+ if (testTable != null) {
+ try {
+ tableAdmin.deleteTable(testTable.getId());
+ } catch (Exception e) {
+ // Ignore.
+ }
+ }
+ }
+
+ @Test
+ public void createAuthorizedViewAndGetAuthorizedViewTest() {
+ String authorizedViewId = prefixGenerator.newPrefix();
+
+ CreateAuthorizedViewRequest request =
+ CreateAuthorizedViewRequest.of(testTable.getId(), authorizedViewId)
+ .setAuthorizedViewType(
+ SubsetView.create()
+ .addRowPrefix("row#")
+ .setFamilySubsets(
+ "cf1",
+ FamilySubsets.create()
+ .addQualifier("qualifier")
+ .addQualifierPrefix("prefix#")))
+ .setDeletionProtection(false);
+ try {
+ AuthorizedView response = tableAdmin.createAuthorizedView(request);
+ assertWithMessage("Got wrong authorized view Id in CreateAuthorizedView")
+ .that(response.getId())
+ .isEqualTo(authorizedViewId);
+ assertWithMessage("Got wrong deletion protection in CreateAuthorizedView")
+ .that(response.isDeletionProtected())
+ .isFalse();
+ assertWithMessage("Got wrong subset view in CreateAuthorizedView")
+ .that(((SubsetView) response.getAuthorizedViewType()).getRowPrefixes())
+ .containsExactly(ByteString.copyFromUtf8("row#"));
+ assertWithMessage("Got wrong family subsets in CreateAuthorizedView")
+ .that(((SubsetView) response.getAuthorizedViewType()).getFamilySubsets())
+ .containsExactly(
+ "cf1",
+ FamilySubsets.create().addQualifier("qualifier").addQualifierPrefix("prefix#"));
+
+ response = tableAdmin.getAuthorizedView(testTable.getId(), authorizedViewId);
+ assertWithMessage("Got wrong authorized view Id in getAuthorizedView")
+ .that(response.getId())
+ .isEqualTo(authorizedViewId);
+ assertWithMessage("Got wrong deletion protection in getAuthorizedView")
+ .that(response.isDeletionProtected())
+ .isFalse();
+ assertWithMessage("Got wrong subset view in getAuthorizedView")
+ .that(((SubsetView) response.getAuthorizedViewType()).getRowPrefixes())
+ .containsExactly(ByteString.copyFromUtf8("row#"));
+ assertWithMessage("Got wrong family subsets in getAuthorizedView")
+ .that(((SubsetView) response.getAuthorizedViewType()).getFamilySubsets())
+ .containsExactly(
+ "cf1",
+ FamilySubsets.create().addQualifier("qualifier").addQualifierPrefix("prefix#"));
+ } finally {
+ tableAdmin.deleteAuthorizedView(testTable.getId(), authorizedViewId);
+ }
+ }
+
+ @Test
+ public void listAuthorizedViewsTest() {
+ String authorizedViewId1 = prefixGenerator.newPrefix();
+ String authorizedViewId2 = prefixGenerator.newPrefix();
+
+ try {
+ tableAdmin.createAuthorizedView(createAuthorizedViewRequest(authorizedViewId1));
+ tableAdmin.createAuthorizedView(createAuthorizedViewRequest(authorizedViewId2));
+
+ List response = tableAdmin.listAuthorizedViews(testTable.getId());
+ // Concurrent tests running may cause flakiness. Use containsAtLeast instead of
+ // containsExactly.
+ assertWithMessage("Got wrong authorized view Ids in listAuthorizedViews")
+ .that(response)
+ .containsAtLeast(
+ tableAdmin.getAuthorizedView(testTable.getId(), authorizedViewId1).getId(),
+ tableAdmin.getAuthorizedView(testTable.getId(), authorizedViewId2).getId());
+ } finally {
+ tableAdmin.deleteAuthorizedView(testTable.getId(), authorizedViewId1);
+ tableAdmin.deleteAuthorizedView(testTable.getId(), authorizedViewId2);
+ }
+ }
+
+ @Test
+ public void updateAuthorizedViewAndDeleteAuthorizedViewTest() throws InterruptedException {
+ String authorizedViewId = prefixGenerator.newPrefix();
+
+ // Create a deletion-protected authorized view.
+ CreateAuthorizedViewRequest request =
+ createAuthorizedViewRequest(authorizedViewId).setDeletionProtection(true);
+
+ AuthorizedView response = tableAdmin.createAuthorizedView(request);
+ assertWithMessage("Got wrong deletion protection in CreateAuthorizedView")
+ .that(response.isDeletionProtected())
+ .isTrue();
+
+ // We should not be able to delete the authorized view.
+ try {
+ tableAdmin.deleteAuthorizedView(testTable.getId(), authorizedViewId);
+ fail("A delete-protected authorized view should not have been able to be deleted");
+ } catch (FailedPreconditionException e) {
+ assertWithMessage("Incorrect exception type")
+ .that(e.getCause())
+ .isInstanceOf(StatusRuntimeException.class);
+ }
+
+ // Update the deletion protection bit of the authorized view.
+ UpdateAuthorizedViewRequest updateRequest =
+ UpdateAuthorizedViewRequest.of(testTable.getId(), authorizedViewId)
+ .setDeletionProtection(false);
+ response = tableAdmin.updateAuthorizedView(updateRequest);
+ assertWithMessage("Got wrong deletion protection in UpdateAuthorizedView")
+ .that(response.isDeletionProtected())
+ .isFalse();
+
+ // Now we should be able to successfully delete the AuthorizedView.
+ tableAdmin.deleteAuthorizedView(testTable.getId(), authorizedViewId);
+ try {
+ for (int i = 0; i < BACKOFF_DURATION.length; i++) {
+ tableAdmin.getAuthorizedView(testTable.getId(), authorizedViewId);
+
+ LOGGER.info(
+ "Wait for "
+ + BACKOFF_DURATION[i]
+ + " seconds for deleting authorized view "
+ + authorizedViewId);
+ Thread.sleep(BACKOFF_DURATION[i] * 1000);
+ }
+ fail("AuthorizedView was not deleted.");
+ } catch (NotFoundException e) {
+ assertWithMessage("Incorrect exception type")
+ .that(e.getCause())
+ .isInstanceOf(StatusRuntimeException.class);
+ }
+ }
+
+ @Test
+ public void authorizedViewIamTest() {
+ String authorizedViewId = prefixGenerator.newPrefix();
+
+ try {
+ tableAdmin.createAuthorizedView(createAuthorizedViewRequest(authorizedViewId));
+ Policy policy = tableAdmin.getAuthorizedViewIamPolicy(testTable.getId(), authorizedViewId);
+ assertThat(policy).isNotNull();
+
+ Exception actualEx = null;
+ try {
+ assertThat(
+ tableAdmin.setAuthorizedViewIamPolicy(testTable.getId(), authorizedViewId, policy))
+ .isNotNull();
+ } catch (Exception iamException) {
+ actualEx = iamException;
+ }
+ assertThat(actualEx).isNull();
+
+ List permissions =
+ tableAdmin.testAuthorizedViewIamPermission(
+ testTable.getId(),
+ authorizedViewId,
+ "bigtable.authorizedViews.get",
+ "bigtable.authorizedViews.update",
+ "bigtable.authorizedViews.delete");
+ assertThat(permissions).hasSize(3);
+ } finally {
+ tableAdmin.deleteAuthorizedView(testTable.getId(), authorizedViewId);
+ }
+ }
+
+ private CreateAuthorizedViewRequest createAuthorizedViewRequest(String authorizedViewId) {
+ return CreateAuthorizedViewRequest.of(testTable.getId(), authorizedViewId)
+ .setAuthorizedViewType(SubsetView.create());
+ }
+
+ private static Table createAndPopulateTestTable(
+ BigtableTableAdminClient tableAdmin, BigtableDataClient dataClient)
+ throws InterruptedException {
+ String tableId =
+ PrefixGenerator.newPrefix("BigtableAuthorizedViewIT#createAndPopulateTestTable");
+ Table testTable = tableAdmin.createTable(CreateTableRequest.of(tableId).addFamily("cf1"));
+
+ // Populate test data.
+ byte[] rowBytes = new byte[1024];
+ Random random = new Random();
+ random.nextBytes(rowBytes);
+
+ try (Batcher batcher = dataClient.newBulkMutationBatcher(tableId)) {
+ for (int i = 0; i < 10; i++) {
+ batcher.add(
+ RowMutationEntry.create("test-row-" + i)
+ .setCell("cf1", ByteString.EMPTY, ByteString.copyFrom(rowBytes)));
+ }
+ }
+ return testTable;
+ }
+}
diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/models/AuthorizedViewTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/models/AuthorizedViewTest.java
new file mode 100644
index 0000000000..6eace3ee97
--- /dev/null
+++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/models/AuthorizedViewTest.java
@@ -0,0 +1,184 @@
+/*
+ * 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.admin.v2.models;
+
+import static com.google.common.truth.Truth.assertThat;
+
+import com.google.bigtable.admin.v2.AuthorizedViewName;
+import com.google.protobuf.ByteString;
+import java.util.Map;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class AuthorizedViewTest {
+ private static final String PROJECT_ID = "my-project";
+ private static final String INSTANCE_ID = "my-instance";
+ private static final String TABLE_ID = "my-table";
+ private static final String AUTHORIZED_VIEW_ID = "my-authorized-view";
+
+ @Test
+ public void testFromProto() {
+ AuthorizedViewName authorizedViewName =
+ AuthorizedViewName.of(PROJECT_ID, INSTANCE_ID, TABLE_ID, AUTHORIZED_VIEW_ID);
+
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView subsetViewProto =
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView.newBuilder()
+ .addRowPrefixes(ByteString.copyFromUtf8("row1#"))
+ .addRowPrefixes(ByteString.copyFromUtf8("row2#"))
+ .putFamilySubsets(
+ "family1",
+ com.google.bigtable.admin.v2.AuthorizedView.FamilySubsets.newBuilder()
+ .addQualifiers(ByteString.copyFromUtf8("column1"))
+ .addQualifiers(ByteString.copyFromUtf8("column2"))
+ .addQualifierPrefixes(ByteString.copyFromUtf8("column3#"))
+ .addQualifierPrefixes(ByteString.copyFromUtf8("column4#"))
+ .build())
+ .putFamilySubsets(
+ "family2",
+ com.google.bigtable.admin.v2.AuthorizedView.FamilySubsets.newBuilder()
+ .addQualifiers(ByteString.copyFromUtf8("column5"))
+ .addQualifierPrefixes(ByteString.copyFromUtf8(""))
+ .build())
+ .build();
+
+ com.google.bigtable.admin.v2.AuthorizedView authorizedViewProto =
+ com.google.bigtable.admin.v2.AuthorizedView.newBuilder()
+ .setName(authorizedViewName.toString())
+ .setDeletionProtection(true)
+ .setSubsetView(subsetViewProto)
+ .build();
+
+ AuthorizedView result = AuthorizedView.fromProto(authorizedViewProto);
+
+ assertThat(result.getId()).isEqualTo(AUTHORIZED_VIEW_ID);
+ assertThat(result.getTableId()).isEqualTo(TABLE_ID);
+ assertThat(result.isDeletionProtected()).isTrue();
+ SubsetView subsetViewResult = (SubsetView) result.getAuthorizedViewType();
+ assertThat(subsetViewResult).isEqualTo(SubsetView.fromProto(subsetViewProto));
+ assertThat(subsetViewResult.getRowPrefixes())
+ .containsExactly(ByteString.copyFromUtf8("row1#"), ByteString.copyFromUtf8("row2#"));
+
+ Map familySubsetsResult = subsetViewResult.getFamilySubsets();
+ assertThat(familySubsetsResult)
+ .containsExactly(
+ "family1",
+ FamilySubsets.fromProto(subsetViewProto.getFamilySubsetsOrThrow("family1")),
+ "family2",
+ FamilySubsets.fromProto(subsetViewProto.getFamilySubsetsOrThrow("family2")));
+ assertThat(familySubsetsResult.get("family1").getQualifiers())
+ .containsExactly(ByteString.copyFromUtf8("column1"), ByteString.copyFromUtf8("column2"));
+ assertThat(familySubsetsResult.get("family1").getQualifierPrefixes())
+ .containsExactly(ByteString.copyFromUtf8("column3#"), ByteString.copyFromUtf8("column4#"));
+ assertThat(familySubsetsResult.get("family2").getQualifiers())
+ .containsExactly(ByteString.copyFromUtf8("column5"));
+ assertThat(familySubsetsResult.get("family2").getQualifierPrefixes())
+ .containsExactly(ByteString.copyFromUtf8(""));
+ }
+
+ @Test
+ public void testRequiresName() {
+ com.google.bigtable.admin.v2.AuthorizedView proto =
+ com.google.bigtable.admin.v2.AuthorizedView.newBuilder()
+ .setDeletionProtection(true)
+ .setSubsetView(
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView.newBuilder().build())
+ .build();
+
+ Exception actualException = null;
+
+ try {
+ AuthorizedView.fromProto(proto);
+ } catch (Exception e) {
+ actualException = e;
+ }
+
+ assertThat(actualException).isInstanceOf(IllegalArgumentException.class);
+ }
+
+ @Test
+ public void testRequiresAuthorizedViewType() {
+ AuthorizedViewName authorizedViewName =
+ AuthorizedViewName.of(PROJECT_ID, INSTANCE_ID, TABLE_ID, AUTHORIZED_VIEW_ID);
+ com.google.bigtable.admin.v2.AuthorizedView proto =
+ com.google.bigtable.admin.v2.AuthorizedView.newBuilder()
+ .setName(authorizedViewName.toString())
+ .setDeletionProtection(true)
+ .build();
+ Exception actualException = null;
+
+ try {
+ AuthorizedView.fromProto(proto);
+ } catch (Exception e) {
+ actualException = e;
+ }
+
+ assertThat(actualException).isInstanceOf(IllegalArgumentException.class);
+ }
+
+ @Test
+ public void testEquality() {
+ AuthorizedViewName authorizedViewName =
+ AuthorizedViewName.of(PROJECT_ID, INSTANCE_ID, TABLE_ID, AUTHORIZED_VIEW_ID);
+ com.google.bigtable.admin.v2.AuthorizedView proto =
+ com.google.bigtable.admin.v2.AuthorizedView.newBuilder()
+ .setName(authorizedViewName.toString())
+ .setDeletionProtection(true)
+ .setSubsetView(
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView.newBuilder().build())
+ .build();
+ AuthorizedView authorizedView = AuthorizedView.fromProto(proto);
+
+ assertThat(authorizedView).isEqualTo(AuthorizedView.fromProto(proto));
+
+ assertThat(authorizedView)
+ .isNotEqualTo(
+ com.google.bigtable.admin.v2.AuthorizedView.newBuilder()
+ .setName(authorizedViewName.toString())
+ .setDeletionProtection(false)
+ .setSubsetView(
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView.newBuilder().build())
+ .build());
+ }
+
+ @Test
+ public void testHashCode() {
+ AuthorizedViewName authorizedViewName =
+ AuthorizedViewName.of(PROJECT_ID, INSTANCE_ID, TABLE_ID, AUTHORIZED_VIEW_ID);
+ com.google.bigtable.admin.v2.AuthorizedView proto =
+ com.google.bigtable.admin.v2.AuthorizedView.newBuilder()
+ .setName(authorizedViewName.toString())
+ .setDeletionProtection(true)
+ .setSubsetView(
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView.newBuilder().build())
+ .build();
+ AuthorizedView authorizedView = AuthorizedView.fromProto(proto);
+
+ assertThat(authorizedView.hashCode()).isEqualTo(AuthorizedView.fromProto(proto).hashCode());
+
+ assertThat(authorizedView.hashCode())
+ .isNotEqualTo(
+ com.google.bigtable.admin.v2.AuthorizedView.newBuilder()
+ .setName(authorizedViewName.toString())
+ .setDeletionProtection(false)
+ .setSubsetView(
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView.newBuilder().build())
+ .build()
+ .hashCode());
+ }
+}
diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/models/CreateAuthorizedViewRequestTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/models/CreateAuthorizedViewRequestTest.java
new file mode 100644
index 0000000000..c661509cee
--- /dev/null
+++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/models/CreateAuthorizedViewRequestTest.java
@@ -0,0 +1,108 @@
+/*
+ * 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.admin.v2.models;
+
+import static com.google.common.truth.Truth.assertThat;
+
+import com.google.cloud.bigtable.admin.v2.internal.NameUtil;
+import com.google.protobuf.ByteString;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class CreateAuthorizedViewRequestTest {
+ private static final String PROJECT_ID = "my-project";
+ private static final String INSTANCE_ID = "my-instance";
+ private static final String TABLE_ID = "my-table";
+ private static final String AUTHORIZED_VIEW_ID = "my-authorized-view";
+
+ @Test
+ public void testToProto() {
+ CreateAuthorizedViewRequest request =
+ CreateAuthorizedViewRequest.of(TABLE_ID, AUTHORIZED_VIEW_ID)
+ .setDeletionProtection(true)
+ .setAuthorizedViewType(
+ SubsetView.create()
+ .addRowPrefix("row#")
+ .addRowPrefix("another-row#")
+ .setFamilySubsets(
+ "family",
+ FamilySubsets.create()
+ .addQualifier("column")
+ .addQualifierPrefix("column#")));
+
+ com.google.bigtable.admin.v2.CreateAuthorizedViewRequest requestProto =
+ com.google.bigtable.admin.v2.CreateAuthorizedViewRequest.newBuilder()
+ .setParent(NameUtil.formatTableName(PROJECT_ID, INSTANCE_ID, TABLE_ID))
+ .setAuthorizedViewId(AUTHORIZED_VIEW_ID)
+ .setAuthorizedView(
+ com.google.bigtable.admin.v2.AuthorizedView.newBuilder()
+ .setDeletionProtection(true)
+ .setSubsetView(
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView.newBuilder()
+ .addRowPrefixes(ByteString.copyFromUtf8("row#"))
+ .addRowPrefixes(ByteString.copyFromUtf8("another-row#"))
+ .putFamilySubsets(
+ "family",
+ com.google.bigtable.admin.v2.AuthorizedView.FamilySubsets
+ .newBuilder()
+ .addQualifiers(ByteString.copyFromUtf8("column"))
+ .addQualifierPrefixes(ByteString.copyFromUtf8("column#"))
+ .build())))
+ .build();
+ assertThat(request.toProto(PROJECT_ID, INSTANCE_ID)).isEqualTo(requestProto);
+ }
+
+ @Test
+ public void testEquality() {
+ CreateAuthorizedViewRequest request =
+ CreateAuthorizedViewRequest.of(TABLE_ID, AUTHORIZED_VIEW_ID)
+ .setAuthorizedViewType(SubsetView.create().addRowPrefix("row#"))
+ .setDeletionProtection(false);
+
+ assertThat(request)
+ .isEqualTo(
+ CreateAuthorizedViewRequest.of(TABLE_ID, AUTHORIZED_VIEW_ID)
+ .setAuthorizedViewType(SubsetView.create().addRowPrefix("row#")));
+
+ assertThat(request)
+ .isNotEqualTo(
+ CreateAuthorizedViewRequest.of(TABLE_ID, AUTHORIZED_VIEW_ID)
+ .setAuthorizedViewType(SubsetView.create().addRowPrefix("another-row#")));
+ }
+
+ @Test
+ public void testHashCode() {
+ CreateAuthorizedViewRequest request =
+ CreateAuthorizedViewRequest.of(TABLE_ID, AUTHORIZED_VIEW_ID)
+ .setAuthorizedViewType(SubsetView.create().addRowPrefix("row#"))
+ .setDeletionProtection(false);
+
+ assertThat(request.hashCode())
+ .isEqualTo(
+ CreateAuthorizedViewRequest.of(TABLE_ID, AUTHORIZED_VIEW_ID)
+ .setAuthorizedViewType(SubsetView.create().addRowPrefix("row#"))
+ .hashCode());
+
+ assertThat(request.hashCode())
+ .isNotEqualTo(
+ CreateAuthorizedViewRequest.of(TABLE_ID, AUTHORIZED_VIEW_ID)
+ .setAuthorizedViewType(SubsetView.create().addRowPrefix("another-row#"))
+ .hashCode());
+ }
+}
diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/models/FamilySubsetsTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/models/FamilySubsetsTest.java
new file mode 100644
index 0000000000..bfcc62fe9c
--- /dev/null
+++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/models/FamilySubsetsTest.java
@@ -0,0 +1,82 @@
+/*
+ * 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.admin.v2.models;
+
+import static com.google.common.truth.Truth.assertThat;
+
+import com.google.bigtable.admin.v2.AuthorizedView;
+import com.google.protobuf.ByteString;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class FamilySubsetsTest {
+
+ @Test
+ public void testFromProto() {
+ AuthorizedView.FamilySubsets familySubsetsProto =
+ com.google.bigtable.admin.v2.AuthorizedView.FamilySubsets.newBuilder()
+ .addQualifiers(ByteString.copyFromUtf8("column1"))
+ .addQualifiers(ByteString.copyFromUtf8("column2"))
+ .addQualifierPrefixes(ByteString.copyFromUtf8("column3#"))
+ .addQualifierPrefixes(ByteString.copyFromUtf8("column4#"))
+ .build();
+
+ FamilySubsets result = FamilySubsets.fromProto(familySubsetsProto);
+
+ assertThat(result.getQualifiers())
+ .containsExactly(ByteString.copyFromUtf8("column1"), ByteString.copyFromUtf8("column2"));
+ assertThat(result.getQualifierPrefixes())
+ .containsExactly(ByteString.copyFromUtf8("column3#"), ByteString.copyFromUtf8("column4#"));
+ }
+
+ @Test
+ public void testEquality() {
+ AuthorizedView.FamilySubsets proto =
+ com.google.bigtable.admin.v2.AuthorizedView.FamilySubsets.newBuilder()
+ .addQualifiers(ByteString.copyFromUtf8("column1"))
+ .build();
+ FamilySubsets familySubsets = FamilySubsets.fromProto(proto);
+
+ assertThat(familySubsets).isEqualTo(FamilySubsets.fromProto(proto));
+ assertThat(familySubsets)
+ .isNotEqualTo(
+ FamilySubsets.fromProto(
+ com.google.bigtable.admin.v2.AuthorizedView.FamilySubsets.newBuilder()
+ .addQualifierPrefixes(ByteString.copyFromUtf8("column1"))
+ .build()));
+ }
+
+ @Test
+ public void testHashCode() {
+ AuthorizedView.FamilySubsets proto =
+ com.google.bigtable.admin.v2.AuthorizedView.FamilySubsets.newBuilder()
+ .addQualifiers(ByteString.copyFromUtf8("column1"))
+ .build();
+ FamilySubsets familySubsets = FamilySubsets.fromProto(proto);
+
+ assertThat(familySubsets.hashCode()).isEqualTo(FamilySubsets.fromProto(proto).hashCode());
+ assertThat(familySubsets.hashCode())
+ .isNotEqualTo(
+ FamilySubsets.fromProto(
+ com.google.bigtable.admin.v2.AuthorizedView.FamilySubsets.newBuilder()
+ .addQualifierPrefixes(ByteString.copyFromUtf8("column1"))
+ .build())
+ .hashCode());
+ }
+}
diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/models/SubsetViewTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/models/SubsetViewTest.java
new file mode 100644
index 0000000000..9f5ed078ee
--- /dev/null
+++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/models/SubsetViewTest.java
@@ -0,0 +1,162 @@
+/*
+ * 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.admin.v2.models;
+
+import static com.google.common.truth.Truth.assertThat;
+
+import com.google.protobuf.ByteString;
+import java.util.Map;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class SubsetViewTest {
+
+ @Test
+ public void testFromProto() {
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView subsetViewProto =
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView.newBuilder()
+ .addRowPrefixes(ByteString.copyFromUtf8("row1#"))
+ .addRowPrefixes(ByteString.copyFromUtf8("row2#"))
+ .putFamilySubsets(
+ "family1",
+ com.google.bigtable.admin.v2.AuthorizedView.FamilySubsets.newBuilder()
+ .addQualifiers(ByteString.copyFromUtf8("column1"))
+ .addQualifiers(ByteString.copyFromUtf8("column2"))
+ .addQualifierPrefixes(ByteString.copyFromUtf8("column3#"))
+ .addQualifierPrefixes(ByteString.copyFromUtf8("column4#"))
+ .build())
+ .putFamilySubsets(
+ "family2",
+ com.google.bigtable.admin.v2.AuthorizedView.FamilySubsets.newBuilder()
+ .addQualifiers(ByteString.copyFromUtf8("column5"))
+ .addQualifierPrefixes(ByteString.copyFromUtf8(""))
+ .build())
+ .build();
+
+ SubsetView result = SubsetView.fromProto(subsetViewProto);
+
+ assertThat(result.getRowPrefixes())
+ .containsExactly(ByteString.copyFromUtf8("row1#"), ByteString.copyFromUtf8("row2#"));
+
+ Map familySubsetsResult = result.getFamilySubsets();
+ assertThat(familySubsetsResult)
+ .containsExactly(
+ "family1",
+ FamilySubsets.fromProto(subsetViewProto.getFamilySubsetsOrThrow("family1")),
+ "family2",
+ FamilySubsets.fromProto(subsetViewProto.getFamilySubsetsOrThrow("family2")));
+ assertThat(familySubsetsResult.get("family1").getQualifiers())
+ .containsExactly(ByteString.copyFromUtf8("column1"), ByteString.copyFromUtf8("column2"));
+ assertThat(familySubsetsResult.get("family1").getQualifierPrefixes())
+ .containsExactly(ByteString.copyFromUtf8("column3#"), ByteString.copyFromUtf8("column4#"));
+ assertThat(familySubsetsResult.get("family2").getQualifiers())
+ .containsExactly(ByteString.copyFromUtf8("column5"));
+ assertThat(familySubsetsResult.get("family2").getQualifierPrefixes())
+ .containsExactly(ByteString.copyFromUtf8(""));
+ }
+
+ @Test
+ public void testToProto() {
+ SubsetView subsetView =
+ SubsetView.create()
+ .addRowPrefix("row1#")
+ .addRowPrefix("row2#")
+ .setFamilySubsets(
+ "family1",
+ FamilySubsets.create().addQualifier("column1").addQualifierPrefix("prefix1#"))
+ .setFamilySubsets(
+ "family1",
+ FamilySubsets.create().addQualifier("column2").addQualifierPrefix("prefix2#"))
+ .setFamilySubsets(
+ "family2", FamilySubsets.create().addQualifier("column").addQualifierPrefix(""));
+
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView subsetViewProto =
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView.newBuilder()
+ .addRowPrefixes(ByteString.copyFromUtf8("row1#"))
+ .addRowPrefixes(ByteString.copyFromUtf8("row2#"))
+ .putFamilySubsets(
+ "family1",
+ com.google.bigtable.admin.v2.AuthorizedView.FamilySubsets.newBuilder()
+ .addQualifiers(ByteString.copyFromUtf8("column2"))
+ .addQualifierPrefixes(ByteString.copyFromUtf8("prefix2#"))
+ .build())
+ .putFamilySubsets(
+ "family2",
+ com.google.bigtable.admin.v2.AuthorizedView.FamilySubsets.newBuilder()
+ .addQualifiers(ByteString.copyFromUtf8("column"))
+ .addQualifierPrefixes(ByteString.copyFromUtf8(""))
+ .build())
+ .build();
+
+ assertThat(subsetView.getRowPrefixes())
+ .containsExactly(ByteString.copyFromUtf8("row1#"), ByteString.copyFromUtf8("row2#"));
+ Map familySubsetsResult = subsetView.getFamilySubsets();
+ assertThat(familySubsetsResult)
+ .containsExactly(
+ "family1",
+ FamilySubsets.fromProto(subsetViewProto.getFamilySubsetsOrThrow("family1")),
+ "family2",
+ FamilySubsets.fromProto(subsetViewProto.getFamilySubsetsOrThrow("family2")));
+ assertThat(familySubsetsResult.get("family1").getQualifiers())
+ .containsExactly(ByteString.copyFromUtf8("column2"));
+ assertThat(familySubsetsResult.get("family1").getQualifierPrefixes())
+ .containsExactly(ByteString.copyFromUtf8("prefix2#"));
+ assertThat(familySubsetsResult.get("family2").getQualifiers())
+ .containsExactly(ByteString.copyFromUtf8("column"));
+ assertThat(familySubsetsResult.get("family2").getQualifierPrefixes())
+ .containsExactly(ByteString.copyFromUtf8(""));
+
+ assertThat(subsetView.toProto()).isEqualTo(subsetViewProto);
+ }
+
+ @Test
+ public void testEquality() {
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView proto =
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView.newBuilder()
+ .addRowPrefixes(ByteString.copyFromUtf8("row1#"))
+ .build();
+ SubsetView subsetView = SubsetView.fromProto(proto);
+
+ assertThat(subsetView).isEqualTo(SubsetView.fromProto(proto));
+ assertThat(subsetView)
+ .isNotEqualTo(
+ SubsetView.fromProto(
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView.newBuilder()
+ .addRowPrefixes(ByteString.copyFromUtf8("row2#"))
+ .build()));
+ }
+
+ @Test
+ public void testHashCode() {
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView proto =
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView.newBuilder()
+ .addRowPrefixes(ByteString.copyFromUtf8("row1#"))
+ .build();
+ SubsetView subsetView = SubsetView.fromProto(proto);
+
+ assertThat(subsetView.hashCode()).isEqualTo(SubsetView.fromProto(proto).hashCode());
+ assertThat(subsetView.hashCode())
+ .isNotEqualTo(
+ SubsetView.fromProto(
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView.newBuilder()
+ .addRowPrefixes(ByteString.copyFromUtf8("row2#"))
+ .build())
+ .hashCode());
+ }
+}
diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/models/UpdateAuthorizedViewRequestTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/models/UpdateAuthorizedViewRequestTest.java
new file mode 100644
index 0000000000..06c9a8353e
--- /dev/null
+++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/admin/v2/models/UpdateAuthorizedViewRequestTest.java
@@ -0,0 +1,158 @@
+/*
+ * 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.admin.v2.models;
+
+import static com.google.common.truth.Truth.assertThat;
+
+import com.google.cloud.bigtable.admin.v2.internal.NameUtil;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.FieldMask;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class UpdateAuthorizedViewRequestTest {
+ private static final String PROJECT_ID = "my-project";
+ private static final String INSTANCE_ID = "my-instance";
+ private static final String TABLE_ID = "my-table";
+ private static final String AUTHORIZED_VIEW_ID = "my-authorized-view";
+
+ @Test
+ public void testToProto() {
+ UpdateAuthorizedViewRequest request =
+ UpdateAuthorizedViewRequest.of(TABLE_ID, AUTHORIZED_VIEW_ID)
+ .setDeletionProtection(true)
+ .setAuthorizedViewType(SubsetView.create().addRowPrefix("row#"))
+ .setIgnoreWarnings(true);
+
+ com.google.bigtable.admin.v2.UpdateAuthorizedViewRequest requestProto =
+ com.google.bigtable.admin.v2.UpdateAuthorizedViewRequest.newBuilder()
+ .setAuthorizedView(
+ com.google.bigtable.admin.v2.AuthorizedView.newBuilder()
+ .setName(
+ NameUtil.formatAuthorizedViewName(
+ PROJECT_ID, INSTANCE_ID, TABLE_ID, AUTHORIZED_VIEW_ID))
+ .setDeletionProtection(true)
+ .setSubsetView(
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView.newBuilder()
+ .addRowPrefixes(ByteString.copyFromUtf8("row#"))))
+ .setUpdateMask(
+ FieldMask.newBuilder().addPaths("deletion_protection").addPaths("subset_view"))
+ .setIgnoreWarnings(true)
+ .build();
+ assertThat(request.toProto(PROJECT_ID, INSTANCE_ID)).isEqualTo(requestProto);
+ }
+
+ @Test
+ public void testUpdateDeletionProtection() {
+ com.google.bigtable.admin.v2.AuthorizedView existingAuthorizedView =
+ com.google.bigtable.admin.v2.AuthorizedView.newBuilder()
+ .setName(
+ NameUtil.formatAuthorizedViewName(
+ PROJECT_ID, INSTANCE_ID, TABLE_ID, AUTHORIZED_VIEW_ID))
+ .setDeletionProtection(true)
+ .setSubsetView(
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView.newBuilder()
+ .addRowPrefixes(ByteString.copyFromUtf8("row#")))
+ .build();
+
+ UpdateAuthorizedViewRequest request =
+ UpdateAuthorizedViewRequest.of(AuthorizedView.fromProto(existingAuthorizedView))
+ .setDeletionProtection(false);
+
+ com.google.bigtable.admin.v2.UpdateAuthorizedViewRequest requestProto =
+ com.google.bigtable.admin.v2.UpdateAuthorizedViewRequest.newBuilder()
+ .setAuthorizedView(existingAuthorizedView.toBuilder().setDeletionProtection(false))
+ .setUpdateMask(FieldMask.newBuilder().addPaths("deletion_protection"))
+ .build();
+ assertThat(request.toProto(PROJECT_ID, INSTANCE_ID)).isEqualTo(requestProto);
+ }
+
+ @Test
+ public void testUpdateSubsetView() {
+ com.google.bigtable.admin.v2.AuthorizedView authorizedViewProto =
+ com.google.bigtable.admin.v2.AuthorizedView.newBuilder()
+ .setName(
+ NameUtil.formatAuthorizedViewName(
+ PROJECT_ID, INSTANCE_ID, TABLE_ID, AUTHORIZED_VIEW_ID))
+ .setSubsetView(
+ com.google.bigtable.admin.v2.AuthorizedView.SubsetView.newBuilder()
+ .addRowPrefixes(ByteString.copyFromUtf8("row#"))
+ .putFamilySubsets(
+ "cf",
+ com.google.bigtable.admin.v2.AuthorizedView.FamilySubsets.newBuilder()
+ .addQualifiers(ByteString.copyFromUtf8("qualifier"))
+ .addQualifierPrefixes(ByteString.copyFromUtf8("prefix#"))
+ .build()))
+ .build();
+
+ UpdateAuthorizedViewRequest request =
+ UpdateAuthorizedViewRequest.of(TABLE_ID, AUTHORIZED_VIEW_ID)
+ .setAuthorizedViewType(
+ SubsetView.create()
+ .addRowPrefix("row#")
+ .setFamilySubsets(
+ "cf",
+ FamilySubsets.create()
+ .addQualifier("qualifier")
+ .addQualifierPrefix("prefix#")));
+
+ com.google.bigtable.admin.v2.UpdateAuthorizedViewRequest requestProto =
+ com.google.bigtable.admin.v2.UpdateAuthorizedViewRequest.newBuilder()
+ .setAuthorizedView(authorizedViewProto)
+ .setUpdateMask(FieldMask.newBuilder().addPaths("subset_view"))
+ .build();
+ assertThat(request.toProto(PROJECT_ID, INSTANCE_ID)).isEqualTo(requestProto);
+ }
+
+ @Test
+ public void testEquality() {
+ UpdateAuthorizedViewRequest request =
+ UpdateAuthorizedViewRequest.of(TABLE_ID, AUTHORIZED_VIEW_ID)
+ .setAuthorizedViewType(SubsetView.create().addRowPrefix("row#"));
+
+ assertThat(request)
+ .isEqualTo(
+ UpdateAuthorizedViewRequest.of(TABLE_ID, AUTHORIZED_VIEW_ID)
+ .setAuthorizedViewType(SubsetView.create().addRowPrefix("row#")));
+
+ assertThat(request)
+ .isNotEqualTo(
+ UpdateAuthorizedViewRequest.of(TABLE_ID, AUTHORIZED_VIEW_ID)
+ .setAuthorizedViewType(SubsetView.create().addRowPrefix("another-row#")));
+ }
+
+ @Test
+ public void testHashCode() {
+ UpdateAuthorizedViewRequest request =
+ UpdateAuthorizedViewRequest.of(TABLE_ID, AUTHORIZED_VIEW_ID)
+ .setAuthorizedViewType(SubsetView.create().addRowPrefix("row#"));
+
+ assertThat(request.hashCode())
+ .isEqualTo(
+ UpdateAuthorizedViewRequest.of(TABLE_ID, AUTHORIZED_VIEW_ID)
+ .setAuthorizedViewType(SubsetView.create().addRowPrefix("row#"))
+ .hashCode());
+
+ assertThat(request.hashCode())
+ .isNotEqualTo(
+ UpdateAuthorizedViewRequest.of(TABLE_ID, AUTHORIZED_VIEW_ID)
+ .setAuthorizedViewType(SubsetView.create().addRowPrefix("another-row#"))
+ .hashCode());
+ }
+}
From 4b255d0160b0ea30a4ab6453ccae278f40d32ee2 Mon Sep 17 00:00:00 2001
From: Lixia Chen
Date: Wed, 27 Mar 2024 18:51:03 -0400
Subject: [PATCH 09/10] feat: Support AuthorizedView in bigtable data client
(#2177)
* feat: Support AuthorizedView in bigtable data client
Change-Id: I1e54cab5b384d76166183ac72105a4cbac59979b
chore: Address review comments
Change-Id: I1d9c8bc54d204dbe6752161291cbd4158ee1d6b5
chore: Add @Deprecated annotation and reformat javadoc
Change-Id: I3c4a0fcbecc124a9517e48a1fc747c7b8dade40e
chore: Remove ReadRow(s)Options
Change-Id: Ib35bdac2cd51f302fd919671324fb2c0d630af0e
chore: Minor doc and test fix
Change-Id: I3c8e1fa624ec0423433eae15059b5e59e9727c70
Chore: Address comments
Change-Id: I8886e907ebb797a67b36240417c2e609b6f5857a
Rename SampleRowKeys to SampleRowKeysRequest
Change-Id: I8dda7ee1df31b184d04938cbc2f9f984d84138b4
Add javadoc for extractTargetId and hide scopedForAuthorizedView
Change-Id: I38718ae9badf24db6edc1b62fc06e4d3222faeb8
fix extractTableId to correctly handle authorized view name
Change-Id: I1e66fc1440a29d3861e3ee6464911324633cc5af
* Fix extractTableId() and add a unit test for it
Change-Id: Icc172b2b6d369ef8ce2b77a8e69c37af6e9aa3d7
* Mark SampleRowKeysRequest#fromProto as @InternalApi
Change-Id: I71762fad534fe31f4bf634ca5cb227f014393e37
---------
Co-authored-by: Lixia Chen
---
.../bigtable/data/v2/BigtableDataClient.java | 831 +++++++++++++++++-
.../bigtable/data/v2/internal/NameUtil.java | 68 ++
.../data/v2/models/AuthorizedViewId.java | 55 ++
.../bigtable/data/v2/models/BulkMutation.java | 47 +-
.../v2/models/ConditionalRowMutation.java | 63 +-
.../cloud/bigtable/data/v2/models/Query.java | 56 +-
.../data/v2/models/ReadModifyWriteRow.java | 63 +-
.../bigtable/data/v2/models/RowMutation.java | 122 ++-
.../data/v2/models/SampleRowKeysRequest.java | 84 ++
.../bigtable/data/v2/models/TableId.java | 47 +
.../bigtable/data/v2/models/TargetId.java | 47 +
.../data/v2/stub/EnhancedBigtableStub.java | 204 ++++-
.../SampleRowKeysCallableWithRequest.java | 74 ++
.../bigtable/data/v2/stub/metrics/Util.java | 17 +-
.../admin/v2/internal/NameUtilTest.java | 53 ++
.../data/v2/BigtableDataClientTests.java | 479 +++++++++-
.../bigtable/data/v2/it/BulkMutateIT.java | 121 +++
.../cloud/bigtable/data/v2/it/BulkReadIT.java | 90 ++
.../bigtable/data/v2/it/CheckAndMutateIT.java | 87 ++
.../bigtable/data/v2/it/MutateRowIT.java | 82 ++
.../cloud/bigtable/data/v2/it/ReadIT.java | 314 +++++++
.../data/v2/it/ReadModifyWriteIT.java | 80 ++
.../data/v2/it/RowMutationEntryBatcherIT.java | 68 ++
.../bigtable/data/v2/it/SampleRowsIT.java | 52 ++
.../data/v2/models/AuthorizedViewIdTest.java | 69 ++
.../data/v2/models/BulkMutationTest.java | 128 ++-
.../v2/models/ConditionalRowMutationTest.java | 175 +++-
.../bigtable/data/v2/models/QueryTest.java | 510 ++++++++++-
.../v2/models/ReadModifyWriteRowTest.java | 102 +++
.../data/v2/models/RowMutationTest.java | 136 ++-
.../v2/models/SampleRowKeysRequestTest.java | 178 ++++
.../bigtable/data/v2/models/TableIdTest.java | 61 ++
.../SampleRowKeysCallableWithRequestTest.java | 129 +++
.../metrics/BigtableTracerCallableTest.java | 24 +-
.../metrics/BuiltinMetricsTracerTest.java | 34 +-
.../metrics/StatsHeadersCallableTest.java | 15 +-
.../AuthorizedViewTestHelper.java | 45 +
37 files changed, 4581 insertions(+), 229 deletions(-)
create mode 100644 google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/AuthorizedViewId.java
create mode 100644 google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/SampleRowKeysRequest.java
create mode 100644 google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/TableId.java
create mode 100644 google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/TargetId.java
create mode 100644 google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/SampleRowKeysCallableWithRequest.java
create mode 100644 google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/models/AuthorizedViewIdTest.java
create mode 100644 google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/models/SampleRowKeysRequestTest.java
create mode 100644 google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/models/TableIdTest.java
create mode 100644 google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/SampleRowKeysCallableWithRequestTest.java
create mode 100644 google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/misc_utilities/AuthorizedViewTestHelper.java
diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/BigtableDataClient.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/BigtableDataClient.java
index f84a5dd098..8f08f82d8a 100644
--- a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/BigtableDataClient.java
+++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/BigtableDataClient.java
@@ -44,6 +44,9 @@
import com.google.cloud.bigtable.data.v2.models.RowAdapter;
import com.google.cloud.bigtable.data.v2.models.RowMutation;
import com.google.cloud.bigtable.data.v2.models.RowMutationEntry;
+import com.google.cloud.bigtable.data.v2.models.SampleRowKeysRequest;
+import com.google.cloud.bigtable.data.v2.models.TableId;
+import com.google.cloud.bigtable.data.v2.models.TargetId;
import com.google.cloud.bigtable.data.v2.stub.EnhancedBigtableStub;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.protobuf.ByteString;
@@ -206,7 +209,9 @@ static BigtableDataClient createWithClientContext(
* }
*
* @throws com.google.api.gax.rpc.ApiException when a serverside error occurs
+ * @deprecated Please use {@link BigtableDataClient#exists(TargetId, String)} instead.
*/
+ @Deprecated
public boolean exists(String tableId, String rowKey) {
return ApiExceptions.callAndTranslateApiException(existsAsync(tableId, rowKey));
}
@@ -233,11 +238,71 @@ public boolean exists(String tableId, String rowKey) {
* }
*
* @throws com.google.api.gax.rpc.ApiException when a serverside error occurs
+ * @deprecated Please use {@link BigtableDataClient#exists(TargetId, ByteString)} instead.
*/
+ @Deprecated
public boolean exists(String tableId, ByteString rowKey) {
return ApiExceptions.callAndTranslateApiException(existsAsync(tableId, rowKey));
}
+ /**
+ * Confirms synchronously if given row key exists or not on the specified {@link TargetId}.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * try (BigtableDataClient bigtableDataClient = BigtableDataClient.create("[PROJECT]", "[INSTANCE]")) {
+ * String tableId = "[TABLE]";
+ * String key = "key";
+ *
+ * boolean isRowPresent = bigtableDataClient.exists(TableId.of(tableId), key);
+ *
+ * // Do something with result, for example, display a message
+ * if(isRowPresent) {
+ * System.out.println(key + " is present");
+ * }
+ * } catch(ApiException e) {
+ * e.printStackTrace();
+ * }
+ * }
+ *
+ * @throws com.google.api.gax.rpc.ApiException when a serverside error occurs
+ * @see com.google.cloud.bigtable.data.v2.models.AuthorizedViewId
+ * @see TableId
+ */
+ public boolean exists(TargetId targetId, String rowKey) {
+ return ApiExceptions.callAndTranslateApiException(existsAsync(targetId, rowKey));
+ }
+
+ /**
+ * Confirms synchronously if given row key exists or not on the specified {@link TargetId}.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * try (BigtableDataClient bigtableDataClient = BigtableDataClient.create("[PROJECT]", "[INSTANCE]")) {
+ * String tableId = "[TABLE]";
+ * ByteString key = ByteString.copyFromUtf8("key");
+ *
+ * boolean isRowPresent = bigtableDataClient.exists(TableId.of(tableId), key);
+ *
+ * // Do something with result, for example, display a message
+ * if(isRowPresent) {
+ * System.out.println(key.toStringUtf8() + " is present");
+ * }
+ * } catch(ApiException e) {
+ * e.printStackTrace();
+ * }
+ * }
+ *
+ * @throws com.google.api.gax.rpc.ApiException when a serverside error occurs
+ * @see com.google.cloud.bigtable.data.v2.models.AuthorizedViewId
+ * @see TableId
+ */
+ public boolean exists(TargetId targetId, ByteString rowKey) {
+ return ApiExceptions.callAndTranslateApiException(existsAsync(targetId, rowKey));
+ }
+
/**
* Confirms asynchronously if given row key exists or not.
*
@@ -262,7 +327,10 @@ public boolean exists(String tableId, ByteString rowKey) {
* }, MoreExecutors.directExecutor());
* }
* }
+ *
+ * @deprecated Please use {@link BigtableDataClient#existsAsync(TargetId, String)} instead.
*/
+ @Deprecated
public ApiFuture existsAsync(String tableId, String rowKey) {
return existsAsync(tableId, ByteString.copyFromUtf8(rowKey));
}
@@ -291,10 +359,77 @@ public ApiFuture existsAsync(String tableId, String rowKey) {
* }, MoreExecutors.directExecutor());
* }
* }
+ *
+ * @deprecated Please use {@link BigtableDataClient#existsAsync(TargetId, ByteString)} instead.
*/
+ @Deprecated
public ApiFuture existsAsync(String tableId, ByteString rowKey) {
+ return existsAsync(TableId.of(tableId), rowKey);
+ }
+
+ /**
+ * Confirms asynchronously if given row key exists or not on the specified {@link TargetId}.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * try (BigtableDataClient bigtableDataClient = BigtableDataClient.create("[PROJECT]", "[INSTANCE]")) {
+ * String tableId = "[TABLE]";
+ * final String key = "key";
+ *
+ * ApiFuture futureResult = bigtableDataClient.existsAsync(TableId.of(tableId), key);
+ *
+ * ApiFutures.addCallback(futureResult, new ApiFutureCallback() {
+ * public void onFailure(Throwable t) {
+ * t.printStackTrace();
+ * }
+ * public void onSuccess(Boolean isRowPresent) {
+ * if(isRowPresent) {
+ * System.out.println(key + " is present");
+ * }
+ * }
+ * }, MoreExecutors.directExecutor());
+ * }
+ * }
+ *
+ * @see com.google.cloud.bigtable.data.v2.models.AuthorizedViewId
+ * @see TableId
+ */
+ public ApiFuture existsAsync(TargetId targetId, String rowKey) {
+ return existsAsync(targetId, ByteString.copyFromUtf8(rowKey));
+ }
+
+ /**
+ * Confirms asynchronously if given row key exists or not on the specified {@link TargetId}.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * try (BigtableDataClient bigtableDataClient = BigtableDataClient.create("[PROJECT]", "[INSTANCE]")) {
+ * String tableId = "[TABLE]";
+ * final ByteString key = ByteString.copyFromUtf8("key");
+ *
+ * ApiFuture futureResult = bigtableDataClient.existsAsync(TableId.of(tableId), key);
+ *
+ * ApiFutures.addCallback(futureResult, new ApiFutureCallback() {
+ * public void onFailure(Throwable t) {
+ * t.printStackTrace();
+ * }
+ * public void onSuccess(Boolean isRowPresent) {
+ * if(isRowPresent) {
+ * System.out.println(key.toStringUtf8() + " is present");
+ * }
+ * }
+ * }, MoreExecutors.directExecutor());
+ * }
+ * }
+ *
+ * @see com.google.cloud.bigtable.data.v2.models.AuthorizedViewId
+ * @see TableId
+ */
+ public ApiFuture existsAsync(TargetId targetId, ByteString rowKey) {
Query query =
- Query.create(tableId)
+ Query.create(targetId)
.rowKey(rowKey)
.filter(
FILTERS
@@ -338,7 +473,9 @@ public Boolean apply(Row row) {
* }
*
* @throws com.google.api.gax.rpc.ApiException when a serverside error occurs
+ * @deprecated Please use {@link BigtableDataClient#readRow(TargetId, ByteString)} instead.
*/
+ @Deprecated
public Row readRow(String tableId, ByteString rowKey) {
return ApiExceptions.callAndTranslateApiException(readRowAsync(tableId, rowKey, null));
}
@@ -368,7 +505,9 @@ public Row readRow(String tableId, ByteString rowKey) {
* }
*
* @throws com.google.api.gax.rpc.ApiException when a serverside error occurs
+ * @deprecated Please use {@link BigtableDataClient#readRow(TargetId, String)} instead.
*/
+ @Deprecated
public Row readRow(String tableId, String rowKey) {
return ApiExceptions.callAndTranslateApiException(
readRowAsync(tableId, ByteString.copyFromUtf8(rowKey), null));
@@ -404,7 +543,9 @@ public Row readRow(String tableId, String rowKey) {
* }
*
* @throws com.google.api.gax.rpc.ApiException when a serverside error occurs
+ * @deprecated Please use {@link BigtableDataClient#readRow(TargetId, String, Filter)} instead.
*/
+ @Deprecated
public Row readRow(String tableId, String rowKey, @Nullable Filter filter) {
return ApiExceptions.callAndTranslateApiException(
readRowAsync(tableId, ByteString.copyFromUtf8(rowKey), filter));
@@ -440,11 +581,154 @@ public Row readRow(String tableId, String rowKey, @Nullable Filter filter) {
* }
*
* @throws com.google.api.gax.rpc.ApiException when a serverside error occurs
+ * @deprecated Please use {@link BigtableDataClient#readRow(TargetId, ByteString, Filter)}
+ * instead.
*/
+ @Deprecated
public Row readRow(String tableId, ByteString rowKey, @Nullable Filter filter) {
return ApiExceptions.callAndTranslateApiException(readRowAsync(tableId, rowKey, filter));
}
+ /**
+ * Convenience method for synchronously reading a single row on the specified {@link TargetId}. If
+ * the row does not exist, the value will be null.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * try (BigtableDataClient bigtableDataClient = BigtableDataClient.create("[PROJECT]", "[INSTANCE]")) {
+ * String tableId = "[TABLE]";
+ *
+ * Row row = bigtableDataClient.readRow(TableId.of(tableId), ByteString.copyFromUtf8("key"));
+ * // Do something with row, for example, display all cells
+ * if(row != null) {
+ * System.out.println(row.getKey().toStringUtf8());
+ * for(RowCell cell : row.getCells()) {
+ * System.out.printf("Family: %s Qualifier: %s Value: %s", cell.getFamily(),
+ * cell.getQualifier().toStringUtf8(), cell.getValue().toStringUtf8());
+ * }
+ * }
+ * } catch(ApiException e) {
+ * e.printStackTrace();
+ * }
+ * }
+ *
+ * @throws com.google.api.gax.rpc.ApiException when a serverside error occurs
+ * @see com.google.cloud.bigtable.data.v2.models.AuthorizedViewId
+ * @see TableId
+ */
+ public Row readRow(TargetId targetId, ByteString rowKey) {
+ return ApiExceptions.callAndTranslateApiException(readRowAsync(targetId, rowKey, null));
+ }
+
+ /**
+ * Convenience method for synchronously reading a single row on the specified {@link TargetId}. If
+ * the row does not exist, the value will be null.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * try (BigtableDataClient bigtableDataClient = BigtableDataClient.create("[PROJECT]", "[INSTANCE]")) {
+ * String tableId = "[TABLE]";
+ *
+ * Row row = bigtableDataClient.readRow(TableId.of(tableId), "key");
+ * // Do something with row, for example, display all cells
+ * if(row != null) {
+ * System.out.println(row.getKey().toStringUtf8());
+ * for(RowCell cell : row.getCells()) {
+ * System.out.printf("Family: %s Qualifier: %s Value: %s", cell.getFamily(),
+ * cell.getQualifier().toStringUtf8(), cell.getValue().toStringUtf8());
+ * }
+ * }
+ * } catch(ApiException e) {
+ * e.printStackTrace();
+ * }
+ * }
+ *
+ * @throws com.google.api.gax.rpc.ApiException when a serverside error occurs
+ * @see com.google.cloud.bigtable.data.v2.models.AuthorizedViewId
+ * @see TableId
+ */
+ public Row readRow(TargetId targetId, String rowKey) {
+ return ApiExceptions.callAndTranslateApiException(
+ readRowAsync(targetId, ByteString.copyFromUtf8(rowKey), null));
+ }
+
+ /**
+ * Convenience method for synchronously reading a single row on the specified {@link TargetId}. If
+ * the row does not exist, the value will be null.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * try (BigtableDataClient bigtableDataClient = BigtableDataClient.create("[PROJECT]", "[INSTANCE]")) {
+ * String tableId = "[TABLE]";
+ *
+ * // Build the filter expression
+ * Filter filter = FILTERS.chain()
+ * .filter(FILTERS.qualifier().regex("prefix.*"))
+ * .filter(FILTERS.limit().cellsPerRow(10));
+ *
+ * Row row = bigtableDataClient.readRow(TableId.of(tableId), "key", filter);
+ * // Do something with row, for example, display all cells
+ * if(row != null) {
+ * System.out.println(row.getKey().toStringUtf8());
+ * for(RowCell cell : row.getCells()) {
+ * System.out.printf("Family: %s Qualifier: %s Value: %s", cell.getFamily(),
+ * cell.getQualifier().toStringUtf8(), cell.getValue().toStringUtf8());
+ * }
+ * }
+ * } catch(ApiException e) {
+ * e.printStackTrace();
+ * }
+ * }
+ *
+ * @throws com.google.api.gax.rpc.ApiException when a serverside error occurs
+ * @see com.google.cloud.bigtable.data.v2.models.AuthorizedViewId
+ * @see TableId
+ */
+ public Row readRow(TargetId targetId, String rowKey, @Nullable Filter filter) {
+ return ApiExceptions.callAndTranslateApiException(
+ readRowAsync(targetId, ByteString.copyFromUtf8(rowKey), filter));
+ }
+
+ /**
+ * Convenience method for synchronously reading a single row on the specified {@link TargetId}. If
+ * the row does not exist, the value will be null.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * try (BigtableDataClient bigtableDataClient = BigtableDataClient.create("[PROJECT]", "[INSTANCE]")) {
+ * String tableId = "[TABLE]";
+ *
+ * // Build the filter expression
+ * Filter filter = FILTERS.chain()
+ * .filter(FILTERS.qualifier().regex("prefix.*"))
+ * .filter(FILTERS.limit().cellsPerRow(10));
+ *
+ * Row row = bigtableDataClient.readRow(TableId.of(tableId), ByteString.copyFromUtf8("key"), filter);
+ * // Do something with row, for example, display all cells
+ * if(row != null) {
+ * System.out.println(row.getKey().toStringUtf8());
+ * for(RowCell cell : row.getCells()) {
+ * System.out.printf("Family: %s Qualifier: %s Value: %s", cell.getFamily(),
+ * cell.getQualifier().toStringUtf8(), cell.getValue().toStringUtf8());
+ * }
+ * }
+ * } catch(ApiException e) {
+ * e.printStackTrace();
+ * }
+ * }
+ *
+ * @throws com.google.api.gax.rpc.ApiException when a serverside error occurs
+ * @see com.google.cloud.bigtable.data.v2.models.AuthorizedViewId
+ * @see TableId
+ */
+ public Row readRow(TargetId targetId, ByteString rowKey, @Nullable Filter filter) {
+ return ApiExceptions.callAndTranslateApiException(readRowAsync(targetId, rowKey, filter));
+ }
+
/**
* Convenience method for asynchronously reading a single row. If the row does not exist, the
* future's value will be null.
@@ -473,7 +757,10 @@ public Row readRow(String tableId, ByteString rowKey, @Nullable Filter filter) {
* }, MoreExecutors.directExecutor());
* }
* }
+ *
+ * @deprecated Please use {@link BigtableDataClient#readRowAsync(TargetId, String)} instead.
*/
+ @Deprecated
public ApiFuture readRowAsync(String tableId, String rowKey) {
return readRowAsync(tableId, ByteString.copyFromUtf8(rowKey), null);
}
@@ -506,7 +793,10 @@ public ApiFuture readRowAsync(String tableId, String rowKey) {
* }, MoreExecutors.directExecutor());
* }
* }
+ *
+ * @deprecated Please use {@link BigtableDataClient#readRowAsync(TargetId, ByteString)} instead.
*/
+ @Deprecated
public ApiFuture readRowAsync(String tableId, ByteString rowKey) {
return readRowAsync(tableId, rowKey, null);
}
@@ -544,7 +834,11 @@ public ApiFuture readRowAsync(String tableId, ByteString rowKey) {
* }, MoreExecutors.directExecutor());
* }
* }
+ *
+ * @deprecated Please use {@link BigtableDataClient#readRowAsync(TargetId, String, Filter)}
+ * instead.
*/
+ @Deprecated
public ApiFuture readRowAsync(String tableId, String rowKey, @Nullable Filter filter) {
return readRowAsync(tableId, ByteString.copyFromUtf8(rowKey), filter);
}
@@ -582,9 +876,168 @@ public ApiFuture readRowAsync(String tableId, String rowKey, @Nullable Filt
* }, MoreExecutors.directExecutor());
* }
* }
+ *
+ * @deprecated Please use {@link BigtableDataClient#readRowAsync(TargetId, ByteString, Filter)}
+ * instead.
*/
+ @Deprecated
public ApiFuture readRowAsync(String tableId, ByteString rowKey, @Nullable Filter filter) {
- Query query = Query.create(tableId).rowKey(rowKey);
+ return readRowAsync(TableId.of(tableId), rowKey, filter);
+ }
+
+ /**
+ * Convenience method for asynchronously reading a single row on the specified {@link TargetId}.
+ * If the row does not exist, the future's value will be null.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * try (BigtableDataClient bigtableDataClient = BigtableDataClient.create("[PROJECT]", "[INSTANCE]")) {
+ * String tableId = "[TABLE]";
+ *
+ * ApiFuture futureResult = bigtableDataClient.readRowAsync(TableId.of(tableId), "key");
+ *
+ * ApiFutures.addCallback(futureResult, new ApiFutureCallback() {
+ * public void onFailure(Throwable t) {
+ * if (t instanceof NotFoundException) {
+ * System.out.println("Tried to read a non-existent table");
+ * } else {
+ * t.printStackTrace();
+ * }
+ * }
+ * public void onSuccess(Row result) {
+ * if (result != null) {
+ * System.out.println("Got row: " + result);
+ * }
+ * }
+ * }, MoreExecutors.directExecutor());
+ * }
+ * }
+ *
+ * @see com.google.cloud.bigtable.data.v2.models.AuthorizedViewId
+ * @see TableId
+ */
+ public ApiFuture readRowAsync(TargetId targetId, String rowKey) {
+ return readRowAsync(targetId, ByteString.copyFromUtf8(rowKey), null);
+ }
+
+ /**
+ * Convenience method for asynchronously reading a single row on the specified {@link TargetId}.
+ * If the row does not exist, the future's value will be null.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * try (BigtableDataClient bigtableDataClient = BigtableDataClient.create("[PROJECT]", "[INSTANCE]")) {
+ * String tableId = "[TABLE]";
+ *
+ * ApiFuture futureResult = bigtableDataClient.readRowAsync(TableId.of(tableId), ByteString.copyFromUtf8("key"));
+ *
+ * ApiFutures.addCallback(futureResult, new ApiFutureCallback() {
+ * public void onFailure(Throwable t) {
+ * if (t instanceof NotFoundException) {
+ * System.out.println("Tried to read a non-existent table");
+ * } else {
+ * t.printStackTrace();
+ * }
+ * }
+ * public void onSuccess(Row result) {
+ * if (result != null) {
+ * System.out.println("Got row: " + result);
+ * }
+ * }
+ * }, MoreExecutors.directExecutor());
+ * }
+ * }
+ *
+ * @see com.google.cloud.bigtable.data.v2.models.AuthorizedViewId
+ * @see TableId
+ */
+ public ApiFuture readRowAsync(TargetId targetId, ByteString rowKey) {
+ return readRowAsync(targetId, rowKey, null);
+ }
+
+ /**
+ * Convenience method for asynchronously reading a single row on the specified {@link TargetId}.
+ * If the row does not exist, the future's value will be null.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * try (BigtableDataClient bigtableDataClient = BigtableDataClient.create("[PROJECT]", "[INSTANCE]")) {
+ * String tableId = "[TABLE]";
+ *
+ * // Build the filter expression
+ * Filters.Filter filter = FILTERS.chain()
+ * .filter(FILTERS.qualifier().regex("prefix.*"))
+ * .filter(FILTERS.limit().cellsPerRow(10));
+ *
+ * ApiFuture futureResult = bigtableDataClient.readRowAsync(TableId.of(tableId), "key", filter);
+ *
+ * ApiFutures.addCallback(futureResult, new ApiFutureCallback() {
+ * public void onFailure(Throwable t) {
+ * if (t instanceof NotFoundException) {
+ * System.out.println("Tried to read a non-existent table");
+ * } else {
+ * t.printStackTrace();
+ * }
+ * }
+ * public void onSuccess(Row result) {
+ * if (result != null) {
+ * System.out.println("Got row: " + result);
+ * }
+ * }
+ * }, MoreExecutors.directExecutor());
+ * }
+ * }
+ *
+ * @see com.google.cloud.bigtable.data.v2.models.AuthorizedViewId
+ * @see TableId
+ */
+ public ApiFuture readRowAsync(TargetId targetId, String rowKey, @Nullable Filter filter) {
+ return readRowAsync(targetId, ByteString.copyFromUtf8(rowKey), filter);
+ }
+
+ /**
+ * Convenience method for asynchronously reading a single row on the specified {@link TargetId}.
+ * If the row does not exist, the value will be null.
+ *
+ * Sample code:
+ *
+ *
{@code
+ * try (BigtableDataClient bigtableDataClient = BigtableDataClient.create("[PROJECT]", "[INSTANCE]")) {
+ * String tableId = "[TABLE]";
+ *
+ * // Build the filter expression
+ * Filters.Filter filter = FILTERS.chain()
+ * .filter(FILTERS.qualifier().regex("prefix.*"))
+ * .filter(FILTERS.limit().cellsPerRow(10));
+ *
+ * ApiFuture futureResult = bigtableDataClient.readRowAsync(TableId.of(tableId), ByteString.copyFromUtf8("key"), filter);
+ *
+ * ApiFutures.addCallback(futureResult, new ApiFutureCallback() {
+ * public void onFailure(Throwable t) {
+ * if (t instanceof NotFoundException) {
+ * System.out.println("Tried to read a non-existent table");
+ * } else {
+ * t.printStackTrace();
+ * }
+ * }
+ * public void onSuccess(Row result) {
+ * if (result != null) {
+ * System.out.println("Got row: " + result);
+ * }
+ * }
+ * }, MoreExecutors.directExecutor());
+ * }
+ * }
+ *
+ * @see com.google.cloud.bigtable.data.v2.models.AuthorizedViewId
+ * @see TableId
+ */
+ public ApiFuture readRowAsync(
+ TargetId targetId, ByteString rowKey, @Nullable Filter filter) {
+ Query query = Query.create(targetId).rowKey(rowKey);
if (filter != null) {
query = query.filter(filter);
}
@@ -868,7 +1321,37 @@ public ServerStreamingCallable readRowsCallable(RowAdapter keyOffsets = bigtableDataClient.sampleRowKeys(tableId);
+ * List