From 0fa2d131952274731deb7c71a3e1b2d3431de97c Mon Sep 17 00:00:00 2001 From: Teng Zhong Date: Thu, 4 Aug 2022 09:19:45 -0400 Subject: [PATCH 01/10] feat: Add readChangeStream callables The merging logic is tested in: ReadChangeStreamMergingCallableTest ReadChangeStreamMergingAcceptanceTest --- google-cloud-bigtable/pom.xml | 33 + .../bigtable/data/v2/BigtableDataClient.java | 164 ++- .../data/v2/models/ChangeStreamMutation.java | 25 +- .../bigtable/data/v2/models/CloseStream.java | 15 +- .../bigtable/data/v2/models/Heartbeat.java | 13 +- .../bigtable/data/v2/models/SetCell.java | 5 +- .../data/v2/stub/EnhancedBigtableStub.java | 123 ++ .../v2/stub/EnhancedBigtableStubSettings.java | 42 +- .../ChangeStreamRecordMerger.java | 112 ++ .../ChangeStreamRecordMergingCallable.java | 63 + .../ReadChangeStreamUserCallable.java | 46 + .../src/main/proto/changestream_tests.proto | 63 + .../src/main/resources/changestream.json | 1053 +++++++++++++++++ .../data/v2/BigtableDataClientTests.java | 35 + .../v2/models/ChangeStreamRecordTest.java | 4 + .../EnhancedBigtableStubSettingsTest.java | 1 + ...ChangeStreamRecordMergingCallableTest.java | 803 +++++++++++++ ...ReadChangeStreamMergingAcceptanceTest.java | 220 ++++ .../ReadChangeStreamUserCallableTest.java | 48 + 19 files changed, 2846 insertions(+), 22 deletions(-) create mode 100644 google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamRecordMerger.java create mode 100644 google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamRecordMergingCallable.java create mode 100644 google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamUserCallable.java create mode 100644 google-cloud-bigtable/src/main/proto/changestream_tests.proto create mode 100644 google-cloud-bigtable/src/main/resources/changestream.json create mode 100644 google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamRecordMergingCallableTest.java create mode 100644 google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java create mode 100644 google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamUserCallableTest.java diff --git a/google-cloud-bigtable/pom.xml b/google-cloud-bigtable/pom.xml index 8cd922d5f4..1ad646c868 100644 --- a/google-cloud-bigtable/pom.xml +++ b/google-cloud-bigtable/pom.xml @@ -36,6 +36,10 @@ batch-bigtable.googleapis.com:443 + + 1.44.0 + 3.19.3 + ${protobuf.version} @@ -545,7 +549,36 @@ + + + kr.motd.maven + os-maven-plugin + 1.6.0 + + + + org.xolstice.maven.plugins + protobuf-maven-plugin + 0.6.1 + + + + compile + compile-custom + + + + + + com.google.protobuf:protoc:${protoc.version}:exe:${os.detected.classifier} + + grpc-java + + io.grpc:protoc-gen-grpc-java:${grpc.version}:exe:${os.detected.classifier} + + + org.codehaus.mojo build-helper-maven-plugin 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 38bc4dc811..7ccf0e49fc 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 @@ -30,17 +30,8 @@ import com.google.api.gax.rpc.ServerStreamingCallable; import com.google.api.gax.rpc.UnaryCallable; import com.google.bigtable.v2.RowRange; -import com.google.cloud.bigtable.data.v2.models.BulkMutation; -import com.google.cloud.bigtable.data.v2.models.ConditionalRowMutation; -import com.google.cloud.bigtable.data.v2.models.Filters; +import com.google.cloud.bigtable.data.v2.models.*; import com.google.cloud.bigtable.data.v2.models.Filters.Filter; -import com.google.cloud.bigtable.data.v2.models.KeyOffset; -import com.google.cloud.bigtable.data.v2.models.Query; -import com.google.cloud.bigtable.data.v2.models.ReadModifyWriteRow; -import com.google.cloud.bigtable.data.v2.models.Row; -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.stub.EnhancedBigtableStub; import com.google.common.util.concurrent.MoreExecutors; import com.google.protobuf.ByteString; @@ -1627,6 +1618,159 @@ public ServerStreamingCallable listChangeStreamPartitionsCalla return stub.listChangeStreamPartitionsCallable(); } + /** + * Convenience method for synchronously streaming the results of a {@link ReadChangeStreamQuery}. + * The returned ServerStream instance is not threadsafe, it can only be used from single thread. + * + *

Sample code: + * + *

{@code
+   * try (BigtableDataClient bigtableDataClient = BigtableDataClient.create("[PROJECT]", "[INSTANCE]")) {
+   *   String tableId = "[TABLE]";
+   *
+   *   ReadChangeStreamQuery query = ReadChangeStreamQuery.create(tableId)
+   *          .streamPartition("[START KEY]", "[END KEY]")
+   *          .startTime(Timestamp.newBuilder().setSeconds(100).build());
+   *
+   *   try {
+   *     ServerStream stream = bigtableDataClient.readChangeStream(query);
+   *     int count = 0;
+   *
+   *     // Iterator style
+   *     for (ChangeStreamRecord record : stream) {
+   *       if (++count > 10) {
+   *         stream.cancel();
+   *         break;
+   *       }
+   *       // Do something with the change stream record.
+   *     }
+   *   } catch (NotFoundException e) {
+   *     System.out.println("Tried to read a non-existent table");
+   *   } catch (RuntimeException e) {
+   *     e.printStackTrace();
+   *   }
+   * }
+   * }
+ * + * @see ServerStreamingCallable For call styles. + * @see ReadChangeStreamQuery For query options. + */ + @InternalApi("Used in Changestream beam pipeline.") + public ServerStream readChangeStream(ReadChangeStreamQuery query) { + return readChangeStreamCallable().call(query); + } + + /** + * Convenience method for asynchronously streaming the results of a {@link ReadChangeStreamQuery}. + * + *

Sample code: + * + *

{@code
+   * try (BigtableDataClient bigtableDataClient = BigtableDataClient.create("[PROJECT]", "[INSTANCE]")) {
+   *   String tableId = "[TABLE]";
+   *
+   *   ReadChangeStreamQuery query = ReadChangeStreamQuery.create(tableId)
+   *          .streamPartition("[START KEY]", "[END KEY]")
+   *          .startTime(Timestamp.newBuilder().setSeconds(100).build());
+   *
+   *   bigtableDataClient.readChangeStreamAsync(query, new ResponseObserver() {
+   *     StreamController controller;
+   *     int count = 0;
+   *
+   *     public void onStart(StreamController controller) {
+   *       this.controller = controller;
+   *     }
+   *     public void onResponse(ChangeStreamRecord record) {
+   *       if (++count > 10) {
+   *         controller.cancel();
+   *         return;
+   *       }
+   *       // Do something with the change stream record.
+   *     }
+   *     public void onError(Throwable t) {
+   *       if (t instanceof NotFoundException) {
+   *         System.out.println("Tried to read a non-existent table");
+   *       } else {
+   *         t.printStackTrace();
+   *       }
+   *     }
+   *     public void onComplete() {
+   *       // Handle stream completion
+   *     }
+   *   });
+   * }
+   * }
+ */ + @InternalApi("Used in Changestream beam pipeline.") + public void readChangeStreamAsync( + ReadChangeStreamQuery query, ResponseObserver observer) { + readChangeStreamCallable().call(query, observer); + } + + /** + * Streams back the results of the query. The returned callable object allows for customization of + * api invocation. + * + *

Sample code: + * + *

{@code
+   * try (BigtableDataClient bigtableDataClient = BigtableDataClient.create("[PROJECT]", "[INSTANCE]")) {
+   *   String tableId = "[TABLE]";
+   *
+   *   ReadChangeStreamQuery query = ReadChangeStreamQuery.create(tableId)
+   *          .streamPartition("[START KEY]", "[END KEY]")
+   *          .startTime(Timestamp.newBuilder().setSeconds(100).build());
+   *
+   *   // Iterator style
+   *   try {
+   *     for(ChangeStreamRecord record : bigtableDataClient.readChangeStreamCallable().call(query)) {
+   *       // Do something with record
+   *     }
+   *   } catch (NotFoundException e) {
+   *     System.out.println("Tried to read a non-existent table");
+   *   } catch (RuntimeException e) {
+   *     e.printStackTrace();
+   *   }
+   *
+   *   // Sync style
+   *   try {
+   *     List records = bigtableDataClient.readChangeStreamCallable().all().call(query);
+   *   } catch (NotFoundException e) {
+   *     System.out.println("Tried to read a non-existent table");
+   *   } catch (RuntimeException e) {
+   *     e.printStackTrace();
+   *   }
+   *
+   *   // Point look up
+   *   ApiFuture recordFuture =
+   *     bigtableDataClient.readChangeStreamCallable().first().futureCall(query);
+   *
+   *   ApiFutures.addCallback(recordFuture, 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(ChangeStreamRecord result) {
+   *       System.out.println("Got record: " + result);
+   *     }
+   *   }, MoreExecutors.directExecutor());
+   *
+   *   // etc
+   * }
+   * }
+ * + * @see ServerStreamingCallable For call styles. + * @see ReadChangeStreamQuery For query options. + */ + @InternalApi("Used in Changestream beam pipeline.") + public ServerStreamingCallable + readChangeStreamCallable() { + return stub.readChangeStreamCallable(); + } + /** Close the clients and releases all associated resources. */ @Override public void close() { diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/ChangeStreamMutation.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/ChangeStreamMutation.java index 10571ecd1f..dfe33c68dd 100644 --- a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/ChangeStreamMutation.java +++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/ChangeStreamMutation.java @@ -15,8 +15,11 @@ */ package com.google.cloud.bigtable.data.v2.models; +import com.google.api.core.InternalApi; import com.google.bigtable.v2.ReadChangeStreamResponse.DataChange.Type; import com.google.cloud.bigtable.data.v2.models.Range.TimestampRange; +import com.google.cloud.bigtable.data.v2.stub.changestream.ChangeStreamRecordMerger; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; import com.google.common.base.Objects; import com.google.common.base.Preconditions; @@ -33,7 +36,7 @@ /** * A ChangeStreamMutation represents a list of mods(represented by List<{@link Entry}>) targeted at - * a single row, which is concatenated by (TODO:ChangeStreamRecordMerger). It represents a logical + * a single row, which is concatenated by {@link ChangeStreamRecordMerger}. It represents a logical * row mutation and can be converted to the original write request(i.e. {@link RowMutation} or * {@link RowMutationEntry}. * @@ -98,7 +101,9 @@ private ChangeStreamMutation(Builder builder) { * ChangeStreamMutation because `token` and `loWatermark` must be set later when we finish * building the logical mutation. */ - static Builder createUserMutation( + @InternalApi("Used in java veneer client.") + @VisibleForTesting + public static Builder createUserMutation( @Nonnull ByteString rowKey, @Nonnull String sourceClusterId, @Nonnull Timestamp commitTimestamp, @@ -111,7 +116,9 @@ static Builder createUserMutation( * because `token` and `loWatermark` must be set later when we finish building the logical * mutation. */ - static Builder createGcMutation( + @InternalApi("Used in java veneer client.") + @VisibleForTesting + public static Builder createGcMutation( @Nonnull ByteString rowKey, @Nonnull Timestamp commitTimestamp, int tieBreaker) { return new Builder(rowKey, Type.GARBAGE_COLLECTION, null, commitTimestamp, tieBreaker); } @@ -223,7 +230,9 @@ private Builder(ChangeStreamMutation changeStreamMutation) { this.lowWatermark = changeStreamMutation.lowWatermark; } - Builder setCell( + @InternalApi("Used in java veneer client.") + @VisibleForTesting + public Builder setCell( @Nonnull String familyName, @Nonnull ByteString qualifier, long timestamp, @@ -232,7 +241,9 @@ Builder setCell( return this; } - Builder deleteCells( + @InternalApi("Used in java veneer client.") + @VisibleForTesting + public Builder deleteCells( @Nonnull String familyName, @Nonnull ByteString qualifier, @Nonnull TimestampRange timestampRange) { @@ -240,7 +251,9 @@ Builder deleteCells( return this; } - Builder deleteFamily(@Nonnull String familyName) { + @InternalApi("Used in java veneer client.") + @VisibleForTesting + public Builder deleteFamily(@Nonnull String familyName) { this.entries.add(DeleteFamily.create(familyName)); return this; } diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/CloseStream.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/CloseStream.java index 403705f676..533e0b0721 100644 --- a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/CloseStream.java +++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/CloseStream.java @@ -18,6 +18,7 @@ import com.google.api.core.InternalApi; import com.google.bigtable.v2.ReadChangeStreamResponse; import com.google.bigtable.v2.StreamContinuationToken; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; import com.google.common.base.Objects; import com.google.common.collect.ImmutableList; @@ -69,10 +70,22 @@ private void writeObject(ObjectOutputStream output) throws IOException { } /** Wraps the protobuf {@link ReadChangeStreamResponse.CloseStream}. */ - static CloseStream fromProto(@Nonnull ReadChangeStreamResponse.CloseStream closeStream) { + @InternalApi("Used in java veneer client.") + @VisibleForTesting + public static CloseStream fromProto(@Nonnull ReadChangeStreamResponse.CloseStream closeStream) { return new CloseStream(closeStream.getStatus(), closeStream.getContinuationTokensList()); } + @VisibleForTesting + public ReadChangeStreamResponse.CloseStream toProto() { + ReadChangeStreamResponse.CloseStream.Builder builder = + ReadChangeStreamResponse.CloseStream.newBuilder().setStatus(getStatus()); + for (ChangeStreamContinuationToken token : getChangeStreamContinuationTokens()) { + builder.addContinuationTokens(token.toProto()); + } + return builder.build(); + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/Heartbeat.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/Heartbeat.java index f2371c8507..c85a5d8631 100644 --- a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/Heartbeat.java +++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/Heartbeat.java @@ -18,6 +18,7 @@ import com.google.api.core.InternalApi; import com.google.auto.value.AutoValue; import com.google.bigtable.v2.ReadChangeStreamResponse; +import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.Timestamp; import java.io.Serializable; import javax.annotation.Nonnull; @@ -32,12 +33,22 @@ private static Heartbeat create( } /** Wraps the protobuf {@link ReadChangeStreamResponse.Heartbeat}. */ - static Heartbeat fromProto(@Nonnull ReadChangeStreamResponse.Heartbeat heartbeat) { + @InternalApi("Used in java veneer client.") + @VisibleForTesting + public static Heartbeat fromProto(@Nonnull ReadChangeStreamResponse.Heartbeat heartbeat) { return create( ChangeStreamContinuationToken.fromProto(heartbeat.getContinuationToken()), heartbeat.getLowWatermark()); } + @VisibleForTesting + public ReadChangeStreamResponse.Heartbeat toProto() { + return ReadChangeStreamResponse.Heartbeat.newBuilder() + .setContinuationToken(getChangeStreamContinuationToken().toProto()) + .setLowWatermark(getLowWatermark()) + .build(); + } + @InternalApi("Used in Changestream beam pipeline.") public abstract ChangeStreamContinuationToken getChangeStreamContinuationToken(); diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/SetCell.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/SetCell.java index a157b5cd73..0c1add67f8 100644 --- a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/SetCell.java +++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/SetCell.java @@ -16,13 +16,14 @@ package com.google.cloud.bigtable.data.v2.models; import com.google.auto.value.AutoValue; +import com.google.cloud.bigtable.data.v2.stub.changestream.ChangeStreamRecordMerger; import com.google.protobuf.ByteString; import java.io.Serializable; import javax.annotation.Nonnull; /** - * Representation of a SetCell mod in a data change, whose value is concatenated by - * (TODO:ChangeStreamRecordMerger) in case of SetCell value chunking. + * Representation of a SetCell mod in a data change, whose value is concatenated by {@link + * ChangeStreamRecordMerger} in case of SetCell value chunking. */ @AutoValue public abstract class SetCell implements Entry, Serializable { diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStub.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStub.java index 7d2cd85b65..630d55c2bb 100644 --- a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStub.java +++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStub.java @@ -53,6 +53,8 @@ import com.google.bigtable.v2.MutateRowResponse; import com.google.bigtable.v2.MutateRowsRequest; import com.google.bigtable.v2.MutateRowsResponse; +import com.google.bigtable.v2.ReadChangeStreamRequest; +import com.google.bigtable.v2.ReadChangeStreamResponse; import com.google.bigtable.v2.ReadModifyWriteRowRequest; import com.google.bigtable.v2.ReadModifyWriteRowResponse; import com.google.bigtable.v2.ReadRowsRequest; @@ -64,16 +66,22 @@ import com.google.cloud.bigtable.data.v2.internal.JwtCredentialsWithAudience; 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.ChangeStreamRecord; +import com.google.cloud.bigtable.data.v2.models.ChangeStreamRecordAdapter; import com.google.cloud.bigtable.data.v2.models.ConditionalRowMutation; +import com.google.cloud.bigtable.data.v2.models.DefaultChangeStreamRecordAdapter; import com.google.cloud.bigtable.data.v2.models.DefaultRowAdapter; import com.google.cloud.bigtable.data.v2.models.KeyOffset; import com.google.cloud.bigtable.data.v2.models.Query; +import com.google.cloud.bigtable.data.v2.models.ReadChangeStreamQuery; import com.google.cloud.bigtable.data.v2.models.ReadModifyWriteRow; import com.google.cloud.bigtable.data.v2.models.Row; 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.stub.changestream.ChangeStreamRecordMergingCallable; import com.google.cloud.bigtable.data.v2.stub.changestream.ListChangeStreamPartitionsUserCallable; +import com.google.cloud.bigtable.data.v2.stub.changestream.ReadChangeStreamUserCallable; import com.google.cloud.bigtable.data.v2.stub.metrics.BigtableTracerStreamingCallable; import com.google.cloud.bigtable.data.v2.stub.metrics.BigtableTracerUnaryCallable; import com.google.cloud.bigtable.data.v2.stub.metrics.BuiltinMetricsTracerFactory; @@ -147,6 +155,9 @@ public class EnhancedBigtableStub implements AutoCloseable { private final ServerStreamingCallable listChangeStreamPartitionsCallable; + private final ServerStreamingCallable + readChangeStreamCallable; + public static EnhancedBigtableStub create(EnhancedBigtableStubSettings settings) throws IOException { settings = finalizeSettings(settings, Tags.getTagger(), Stats.getStatsRecorder()); @@ -290,6 +301,8 @@ public EnhancedBigtableStub(EnhancedBigtableStubSettings settings, ClientContext checkAndMutateRowCallable = createCheckAndMutateRowCallable(); readModifyWriteRowCallable = createReadModifyWriteRowCallable(); listChangeStreamPartitionsCallable = createListChangeStreamPartitionsCallable(); + readChangeStreamCallable = + createReadChangeStreamCallable(new DefaultChangeStreamRecordAdapter()); } // @@ -874,6 +887,110 @@ public Map extract( return traced.withDefaultCallContext(clientContext.getDefaultCallContext()); } + /** + * Creates a callable chain to handle streaming ReadChangeStream RPCs. The chain will: + * + *
    + *
  • Convert a {@link ReadChangeStreamQuery} into a {@link ReadChangeStreamRequest} and + * dispatch the RPC. + *
  • Upon receiving the response stream, it will produce a stream of ChangeStreamRecordT. In + * case of mutations, it will merge the {@link ReadChangeStreamResponse.DataChange}s into + * logical mutations. The actual change stream record implementation can be configured by + * the {@code changeStreamRecordAdapter} parameter. + *
  • TODO: Retry/resume on failure. + *
  • Add tracing & metrics. + *
+ */ + public + ServerStreamingCallable + createReadChangeStreamCallable( + ChangeStreamRecordAdapter changeStreamRecordAdapter) { + ServerStreamingCallable readChangeStreamCallable = + createReadChangeStreamBaseCallable( + settings.readChangeStreamSettings(), changeStreamRecordAdapter); + + ServerStreamingCallable + readChangeStreamUserCallable = + new ReadChangeStreamUserCallable<>(readChangeStreamCallable, requestContext); + + SpanName span = getSpanName("ReadChangeStream"); + ServerStreamingCallable traced = + new TracedServerStreamingCallable<>( + readChangeStreamUserCallable, clientContext.getTracerFactory(), span); + + return traced.withDefaultCallContext(clientContext.getDefaultCallContext()); + } + + /** + * Creates a callable chain to handle ReadRows RPCs. The chain will: + * + *
    + *
  • Dispatch the RPC with {@link ReadChangeStreamRequest}. + *
  • Upon receiving the response stream, it will produce a stream of ChangeStreamRecordT. In + * case of mutations, it will merge the {@link ReadChangeStreamResponse.DataChange}s into + * logical mutations. The actual change stream record implementation can be configured by + * the {@code changeStreamRecordAdapter} parameter. + *
  • Add header tracer for tracking GFE metrics. + *
  • TODO: Retry/resume on failure. + *
+ * + *

NOTE: the caller is responsible for adding tracing & metrics. + */ + private + ServerStreamingCallable + createReadChangeStreamBaseCallable( + ServerStreamingCallSettings readChangeStreamSettings, + ChangeStreamRecordAdapter changeStreamRecordAdapter) { + ServerStreamingCallable base = + GrpcRawCallableFactory.createServerStreamingCallable( + GrpcCallSettings.newBuilder() + .setMethodDescriptor(BigtableGrpc.getReadChangeStreamMethod()) + .setParamsExtractor( + new RequestParamsExtractor() { + @Override + public Map extract( + ReadChangeStreamRequest readChangeStreamRequest) { + return ImmutableMap.of( + "table_name", readChangeStreamRequest.getTableName(), + "app_profile_id", readChangeStreamRequest.getAppProfileId()); + } + }) + .build(), + readChangeStreamSettings.getRetryableCodes()); + + ServerStreamingCallable withStatsHeaders = + new StatsHeadersServerStreamingCallable<>(base); + + // Sometimes ReadChangeStream connections are disconnected via an RST frame. This error is + // transient and should be treated similar to UNAVAILABLE. However, this exception has an + // INTERNAL error code which by default is not retryable. Convert the exception it can be + // retried in the client. + ServerStreamingCallable convertException = + new ConvertStreamExceptionCallable<>(withStatsHeaders); + + ServerStreamingCallable merging = + new ChangeStreamRecordMergingCallable<>(convertException, changeStreamRecordAdapter); + + // Copy idle timeout settings for watchdog. + ServerStreamingCallSettings innerSettings = + ServerStreamingCallSettings.newBuilder() + // TODO: setResumptionStrategy. + .setRetryableCodes(readChangeStreamSettings.getRetryableCodes()) + .setRetrySettings(readChangeStreamSettings.getRetrySettings()) + .setIdleTimeout(readChangeStreamSettings.getIdleTimeout()) + .build(); + + ServerStreamingCallable watched = + Callables.watched(merging, innerSettings, clientContext); + + ServerStreamingCallable withBigtableTracer = + new BigtableTracerStreamingCallable<>(watched); + + // TODO: Add ReadChangeStreamRetryCompletedCallable. + + return Callables.retrying(withBigtableTracer, innerSettings, clientContext); + } + /** * Wraps a callable chain in a user presentable callable that will inject the default call context * and trace the call. @@ -935,6 +1052,12 @@ public UnaryCallable readModifyWriteRowCallable() { public ServerStreamingCallable listChangeStreamPartitionsCallable() { return listChangeStreamPartitionsCallable; } + + /** Returns a streaming read change stream callable. */ + public ServerStreamingCallable + readChangeStreamCallable() { + return readChangeStreamCallable; + } // private SpanName getSpanName(String methodName) { diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStubSettings.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStubSettings.java index 83f0445bc5..68769c9602 100644 --- a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStubSettings.java +++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStubSettings.java @@ -35,9 +35,11 @@ import com.google.auth.Credentials; import com.google.bigtable.v2.RowRange; import com.google.cloud.bigtable.Version; +import com.google.cloud.bigtable.data.v2.models.ChangeStreamRecord; import com.google.cloud.bigtable.data.v2.models.ConditionalRowMutation; import com.google.cloud.bigtable.data.v2.models.KeyOffset; import com.google.cloud.bigtable.data.v2.models.Query; +import com.google.cloud.bigtable.data.v2.models.ReadChangeStreamQuery; import com.google.cloud.bigtable.data.v2.models.ReadModifyWriteRow; import com.google.cloud.bigtable.data.v2.models.Row; import com.google.cloud.bigtable.data.v2.models.RowMutation; @@ -154,6 +156,26 @@ public class EnhancedBigtableStubSettings extends StubSettings READ_CHANGE_STREAM_RETRY_CODES = + ImmutableSet.builder().addAll(IDEMPOTENT_RETRY_CODES).add(Code.ABORTED).build(); + + private static final RetrySettings READ_CHANGE_STREAM_RETRY_SETTINGS = + RetrySettings.newBuilder() + .setInitialRetryDelay(Duration.ofMillis(10)) + .setRetryDelayMultiplier(2.0) + .setMaxRetryDelay(Duration.ofMinutes(1)) + .setMaxAttempts(10) + .setJittered(true) + .setInitialRpcTimeout(Duration.ofMinutes(5)) + .setRpcTimeoutMultiplier(2.0) + .setMaxRpcTimeout(Duration.ofMinutes(5)) + .setTotalTimeout(Duration.ofHours(12)) + .build(); + /** * Scopes that are equivalent to JWT's audience. * @@ -190,8 +212,9 @@ public class EnhancedBigtableStubSettings extends StubSettings checkAndMutateRowSettings; private final UnaryCallSettings readModifyWriteRowSettings; - private final ServerStreamingCallSettings listChangeStreamPartitionsSettings; + private final ServerStreamingCallSettings + readChangeStreamSettings; private EnhancedBigtableStubSettings(Builder builder) { super(builder); @@ -228,6 +251,7 @@ private EnhancedBigtableStubSettings(Builder builder) { checkAndMutateRowSettings = builder.checkAndMutateRowSettings.build(); readModifyWriteRowSettings = builder.readModifyWriteRowSettings.build(); listChangeStreamPartitionsSettings = builder.listChangeStreamPartitionsSettings.build(); + readChangeStreamSettings = builder.readChangeStreamSettings.build(); } /** Create a new builder. */ @@ -515,6 +539,11 @@ public ServerStreamingCallSettings listChangeStreamPartitionsS return listChangeStreamPartitionsSettings; } + public ServerStreamingCallSettings + readChangeStreamSettings() { + return readChangeStreamSettings; + } + /** Returns a builder containing all the values of this settings class. */ public Builder toBuilder() { return new Builder(this); @@ -539,9 +568,10 @@ public static class Builder extends StubSettings.Builder checkAndMutateRowSettings; private final UnaryCallSettings.Builder readModifyWriteRowSettings; - private final ServerStreamingCallSettings.Builder listChangeStreamPartitionsSettings; + private final ServerStreamingCallSettings.Builder + readChangeStreamSettings; /** * Initializes a new Builder with sane defaults for all settings. @@ -659,6 +689,12 @@ private Builder() { .setRetryableCodes(LIST_CHANGE_STREAM_PARTITIONS_RETRY_CODES) .setRetrySettings(LIST_CHANGE_STREAM_PARTITIONS_RETRY_SETTINGS) .setIdleTimeout(Duration.ofMinutes(5)); + + readChangeStreamSettings = ServerStreamingCallSettings.newBuilder(); + readChangeStreamSettings + .setRetryableCodes(READ_CHANGE_STREAM_RETRY_CODES) + .setRetrySettings(READ_CHANGE_STREAM_RETRY_SETTINGS) + .setIdleTimeout(Duration.ofMinutes(5)); } private Builder(EnhancedBigtableStubSettings settings) { @@ -680,6 +716,7 @@ private Builder(EnhancedBigtableStubSettings settings) { checkAndMutateRowSettings = settings.checkAndMutateRowSettings.toBuilder(); readModifyWriteRowSettings = settings.readModifyWriteRowSettings.toBuilder(); listChangeStreamPartitionsSettings = settings.listChangeStreamPartitionsSettings.toBuilder(); + readChangeStreamSettings = settings.readChangeStreamSettings.toBuilder(); } // @@ -892,6 +929,7 @@ public String toString() { .add("checkAndMutateRowSettings", checkAndMutateRowSettings) .add("readModifyWriteRowSettings", readModifyWriteRowSettings) .add("listChangeStreamPartitionsSettings", listChangeStreamPartitionsSettings) + .add("readChangeStreamSettings", readChangeStreamSettings) .add("parent", super.toString()) .toString(); } diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamRecordMerger.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamRecordMerger.java new file mode 100644 index 0000000000..6c77610676 --- /dev/null +++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamRecordMerger.java @@ -0,0 +1,112 @@ +/* + * Copyright 2022 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.changestream; + +import com.google.api.core.InternalApi; +import com.google.bigtable.v2.ReadChangeStreamResponse; +import com.google.cloud.bigtable.data.v2.models.ChangeStreamRecordAdapter; +import com.google.cloud.bigtable.gaxx.reframing.Reframer; +import com.google.cloud.bigtable.gaxx.reframing.ReframingResponseObserver; +import com.google.common.base.Preconditions; +import java.util.ArrayDeque; +import java.util.Queue; + +/** + * An implementation of a {@link Reframer} that feeds the change stream record merging {@link + * ChangeStreamStateMachine}. + * + *

{@link ReframingResponseObserver} pushes {@link ReadChangeStreamResponse}s into this class and + * pops a change stream record containing one of the following: 1) Heartbeat. 2) CloseStream. 3) + * ChangeStreamMutation(a representation of a fully merged logical mutation). + * + *

Example usage: + * + *

{@code
+ * ChangeStreamRecordMerger changeStreamRecordMerger =
+ *     new ChangeStreamRecordMerger<>(myChangeStreamRecordBuilder);
+ *
+ * while(responseIterator.hasNext()) {
+ *   ReadChangeStreamResponse response = responseIterator.next();
+ *
+ *   if (changeStreamRecordMerger.hasFullFrame()) {
+ *     ChangeStreamRecord changeStreamRecord = changeStreamRecordMerger.pop();
+ *     // Do something with change stream record.
+ *   } else {
+ *     changeStreamRecordMerger.push(response);
+ *   }
+ * }
+ *
+ * if (changeStreamRecordMerger.hasPartialFrame()) {
+ *   throw new RuntimeException("Incomplete stream");
+ * }
+ *
+ * }
+ * + *

This class is considered an internal implementation detail and not meant to be used by + * applications. + * + *

Package-private for internal use. + * + * @see ReframingResponseObserver for more details + */ +@InternalApi +public class ChangeStreamRecordMerger + implements Reframer { + private final ChangeStreamStateMachine changeStreamStateMachine; + private final Queue changeStreamRecord; + + public ChangeStreamRecordMerger( + ChangeStreamRecordAdapter.ChangeStreamRecordBuilder + changeStreamRecordBuilder) { + changeStreamStateMachine = new ChangeStreamStateMachine<>(changeStreamRecordBuilder); + changeStreamRecord = new ArrayDeque<>(); + } + + @Override + public void push(ReadChangeStreamResponse response) { + if (response.hasHeartbeat()) { + changeStreamStateMachine.handleHeartbeat(response.getHeartbeat()); + } else if (response.hasCloseStream()) { + changeStreamStateMachine.handleCloseStream(response.getCloseStream()); + } else { + changeStreamStateMachine.handleDataChange(response.getDataChange()); + } + if (changeStreamStateMachine.hasCompleteChangeStreamRecord()) { + changeStreamRecord.add(changeStreamStateMachine.consumeChangeStreamRecord()); + } + } + + @Override + public boolean hasFullFrame() { + return !changeStreamRecord.isEmpty(); + } + + @Override + public boolean hasPartialFrame() { + // Check if buffer in this class contains data. If an assembled is still not available, then + // that means `buffer` has been fully consumed. The last place to check is the + // ChangeStreamStateMachine buffer, to see if it's holding on to an incomplete change + // stream record. + return hasFullFrame() || changeStreamStateMachine.isChangeStreamRecordInProgress(); + } + + @Override + public ChangeStreamRecordT pop() { + return Preconditions.checkNotNull( + changeStreamRecord.poll(), + "ChangeStreamRecordMerger.pop() called when there are no change stream records."); + } +} diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamRecordMergingCallable.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamRecordMergingCallable.java new file mode 100644 index 0000000000..5c6c07451b --- /dev/null +++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamRecordMergingCallable.java @@ -0,0 +1,63 @@ +/* + * Copyright 2022 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.changestream; + +import com.google.api.core.InternalApi; +import com.google.api.gax.rpc.ApiCallContext; +import com.google.api.gax.rpc.ResponseObserver; +import com.google.api.gax.rpc.ServerStreamingCallable; +import com.google.bigtable.v2.ReadChangeStreamRequest; +import com.google.bigtable.v2.ReadChangeStreamResponse; +import com.google.cloud.bigtable.data.v2.models.ChangeStreamRecordAdapter; +import com.google.cloud.bigtable.gaxx.reframing.ReframingResponseObserver; + +/** + * A ServerStreamingCallable that consumes {@link ReadChangeStreamResponse}s and produces change + * stream records. + * + *

This class delegates all the work to gax's {@link ReframingResponseObserver} and the logic to + * {@link ChangeStreamRecordMerger}. + * + *

This class is considered an internal implementation detail and not meant to be used by + * applications. + */ +@InternalApi +public class ChangeStreamRecordMergingCallable + extends ServerStreamingCallable { + private final ServerStreamingCallable inner; + private final ChangeStreamRecordAdapter changeStreamRecordAdapter; + + public ChangeStreamRecordMergingCallable( + ServerStreamingCallable inner, + ChangeStreamRecordAdapter changeStreamRecordAdapter) { + this.inner = inner; + this.changeStreamRecordAdapter = changeStreamRecordAdapter; + } + + @Override + public void call( + ReadChangeStreamRequest request, + ResponseObserver responseObserver, + ApiCallContext context) { + ChangeStreamRecordAdapter.ChangeStreamRecordBuilder + changeStreamRecordBuilder = changeStreamRecordAdapter.createChangeStreamRecordBuilder(); + ChangeStreamRecordMerger merger = + new ChangeStreamRecordMerger<>(changeStreamRecordBuilder); + ReframingResponseObserver innerObserver = + new ReframingResponseObserver<>(responseObserver, merger); + inner.call(request, innerObserver, context); + } +} diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamUserCallable.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamUserCallable.java new file mode 100644 index 0000000000..1ed1e2359b --- /dev/null +++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamUserCallable.java @@ -0,0 +1,46 @@ +/* + * Copyright 2022 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.changestream; + +import com.google.api.gax.rpc.ApiCallContext; +import com.google.api.gax.rpc.ResponseObserver; +import com.google.api.gax.rpc.ServerStreamingCallable; +import com.google.bigtable.v2.ReadChangeStreamRequest; +import com.google.cloud.bigtable.data.v2.internal.RequestContext; +import com.google.cloud.bigtable.data.v2.models.ReadChangeStreamQuery; + +/** A ServerStreamingCallable that reads a change stream given the {@link ReadChangeStreamQuery}. */ +public class ReadChangeStreamUserCallable + extends ServerStreamingCallable { + private final ServerStreamingCallable inner; + private final RequestContext requestContext; + + public ReadChangeStreamUserCallable( + ServerStreamingCallable inner, + RequestContext requestContext) { + this.inner = inner; + this.requestContext = requestContext; + } + + @Override + public void call( + ReadChangeStreamQuery request, + ResponseObserver responseObserver, + ApiCallContext context) { + ReadChangeStreamRequest innerRequest = request.toProto(requestContext); + inner.call(innerRequest, responseObserver, context); + } +} diff --git a/google-cloud-bigtable/src/main/proto/changestream_tests.proto b/google-cloud-bigtable/src/main/proto/changestream_tests.proto new file mode 100644 index 0000000000..4cf3825e20 --- /dev/null +++ b/google-cloud-bigtable/src/main/proto/changestream_tests.proto @@ -0,0 +1,63 @@ +// Copyright 2022, 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. + +syntax = "proto3"; + +package google.cloud.conformance.bigtable.v2; + +import "google/bigtable/v2/bigtable.proto"; +import "google/protobuf/timestamp.proto"; +import "google/bigtable/v2/data.proto"; + +option csharp_namespace = "Google.Cloud.Bigtable.V2.Tests.Conformance"; +option java_outer_classname = "ChangeStreamTestDefinition"; +option java_package = "com.google.cloud.conformance.bigtable.v2"; +option go_package = "google/cloud/conformance/bigtable/v2"; + +message ChangeStreamTestFile { + repeated ReadChangeStreamTest read_change_stream_tests = 1; +} + +message ReadChangeStreamTest { + + message TestChangeStreamMutation { + bytes row_key = 1; + google.bigtable.v2.ReadChangeStreamResponse.DataChange.Type type = 2; + string source_cluster_id = 3; + google.protobuf.Timestamp commit_timestamp = 4; + int64 tiebreaker = 5; + string token = 6; + google.protobuf.Timestamp low_watermark = 7; + repeated google.bigtable.v2.Mutation mutations = 8; + } + + message TestChangeStreamRecord { + oneof record { + google.bigtable.v2.ReadChangeStreamResponse.Heartbeat heartbeat = 1; + google.bigtable.v2.ReadChangeStreamResponse.CloseStream close_stream = 2; + TestChangeStreamMutation change_stream_mutation = 3; + } + } + + // Expected results of reading the change stream. + // Only the last result can be an error. + message Result { + TestChangeStreamRecord record = 1; + bool error = 2; + } + + string description = 1; + repeated google.bigtable.v2.ReadChangeStreamResponse api_responses = 2; + repeated Result results = 3; +} \ No newline at end of file diff --git a/google-cloud-bigtable/src/main/resources/changestream.json b/google-cloud-bigtable/src/main/resources/changestream.json new file mode 100644 index 0000000000..608dc33074 --- /dev/null +++ b/google-cloud-bigtable/src/main/resources/changestream.json @@ -0,0 +1,1053 @@ +{ + "readChangeStreamTests": [ + { + "description": "1 heartbeat", + "api_responses": [ + { + "heartbeat": { + "continuation_token": { + "partition": { + "row_range": { + "start_key_closed": "", + "end_key_open": "" + } + }, + "token": "heartbeat-token" + }, + "low_watermark": "2022-07-01T00:00:00Z" + } + } + ], + "results": [ + { + "record" : { + "heartbeat": { + "continuation_token": { + "partition": { + "row_range": { + "start_key_closed": "", + "end_key_open": "" + } + }, + "token": "heartbeat-token" + }, + "low_watermark": "2022-07-01T00:00:00Z" + } + }, + "error": false + } + ] + }, + { + "description": "1 CloseStream", + "api_responses": [ + { + "close_stream": { + "status": { + "code": "11", + "message": "Partition boundaries are misaligned." + }, + "continuation_tokens": [ + { + "partition": { + "row_range": { + "start_key_closed": "", + "end_key_open": "0000000000000001" + } + }, + "token": "close-stream-token-1" + }, + { + "partition": { + "row_range": { + "start_key_closed": "0000000000000001", + "end_key_open": "0000000000000002" + } + }, + "token": "close-stream-token-2" + } + ] + } + } + ], + "results": [ + { + "record" : { + "close_stream": { + "status": { + "code": "11", + "message": "Partition boundaries are misaligned." + }, + "continuation_tokens": [ + { + "partition": { + "row_range": { + "start_key_closed": "", + "end_key_open": "0000000000000001" + } + }, + "token": "close-stream-token-1" + }, + { + "partition": { + "row_range": { + "start_key_closed": "0000000000000001", + "end_key_open": "0000000000000002" + } + }, + "token": "close-stream-token-2" + } + ] + } + }, + "error": false + } + ] + }, + { + "description": "1 heartbeat + 1 CloseStream", + "api_responses": [ + { + "heartbeat": { + "continuation_token": { + "partition": { + "row_range": { + "start_key_closed": "", + "end_key_open": "" + } + }, + "token": "heartbeat-token" + }, + "low_watermark": "2022-07-01T00:00:00Z" + } + }, + { + "close_stream": { + "status": { + "code": "11", + "message": "Partition boundaries are misaligned." + }, + "continuation_tokens": [ + { + "partition": { + "row_range": { + "start_key_closed": "", + "end_key_open": "0000000000000001" + } + }, + "token": "close-stream-token-1" + } + ] + } + } + ], + "results": [ + { + "record" : { + "heartbeat": { + "continuation_token": { + "partition": { + "row_range": { + "start_key_closed": "", + "end_key_open": "" + } + }, + "token": "heartbeat-token" + }, + "low_watermark": "2022-07-01T00:00:00Z" + } + }, + "error": false + }, + { + "record" : { + "close_stream": { + "status": { + "code": "11", + "message": "Partition boundaries are misaligned." + }, + "continuation_tokens": [ + { + "partition": { + "row_range": { + "start_key_closed": "", + "end_key_open": "0000000000000001" + } + }, + "token": "close-stream-token-1" + } + ] + } + }, + "error": false + } + ] + }, + { + "description": "1 logical mutation no chunking", + "api_responses": [ + { + "data_change": { + "row_key": "0000000000000000", + "type": "USER", + "source_cluster_id": "source-cluster-id", + "commit_timestamp": "2022-07-01T00:00:00Z", + "tiebreaker": 100, + "token": "data-change-token", + "low_watermark": "2022-07-01T00:00:00Z", + "chunks": [ + { + "mutation": { + "delete_from_family": { + "family_name": "family" + } + } + } + ], + "done": true + } + } + ], + "results": [ + { + "record": { + "change_stream_mutation": { + "row_key": "0000000000000000", + "type": "USER", + "source_cluster_id": "source-cluster-id", + "commit_timestamp": "2022-07-01T00:00:00Z", + "low_watermark": "2022-07-01T00:00:00Z", + "tiebreaker": 100, + "token": "data-change-token", + "mutations": [ + { + "delete_from_family": { + "family_name": "family" + } + } + ] + } + }, + "error": false + } + ] + }, + { + "description": "1 incomplete logical mutation", + "api_responses": [ + { + "data_change": { + "row_key": "0000000000000000", + "type": "USER", + "source_cluster_id": "source-cluster-id", + "commit_timestamp": "2022-07-01T00:00:00Z", + "tiebreaker": 100, + "token": "data-change-token", + "chunks": [ + { + "mutation": { + "delete_from_family": { + "family_name": "family" + } + } + } + ] + } + } + ], + "results": [ + { + "error": true + } + ] + }, + { + "description": "GC mutation no source cluster id", + "api_responses": [ + { + "data_change": { + "row_key": "0000000000000000", + "type": "GARBAGE_COLLECTION", + "commit_timestamp": "2022-07-01T00:00:00Z", + "tiebreaker": 100, + "token": "data-change-token", + "low_watermark": "2022-07-01T00:00:00Z", + "chunks": [ + { + "mutation": { + "delete_from_family": { + "family_name": "family" + } + } + } + ], + "done": true + } + } + ], + "results": [ + { + "record": { + "change_stream_mutation": { + "row_key": "0000000000000000", + "type": "GARBAGE_COLLECTION", + "commit_timestamp": "2022-07-01T00:00:00Z", + "tiebreaker": 100, + "token": "data-change-token", + "low_watermark": "2022-07-01T00:00:00Z", + "mutations": [ + { + "delete_from_family": { + "family_name": "family" + } + } + ] + } + }, + "error": false + } + ] + }, + { + "description": "1 chunked SetCell", + "api_responses": [ + { + "data_change": { + "row_key": "0000000000000000", + "type": "USER", + "source_cluster_id": "source-cluster-id", + "commit_timestamp": "2022-07-01T00:00:00Z", + "tiebreaker": 100, + "chunks": [ + { + "chunk_info": { + "chunked_value_size": 1 + }, + "mutation": { + "set_cell": { + "family_name": "family", + "column_qualifier": "0000000000000000", + "timestamp_micros": 1000, + "value": "dg==" + } + } + } + ] + } + }, + { + "data_change": { + "type": "CONTINUATION", + "token": "data-change-token", + "low_watermark": "2022-07-01T00:00:00Z", + "chunks": [ + { + "chunk_info": { + "chunked_value_offset": 1, + "chunked_value_size": 1, + "last_chunk": true + }, + "mutation": { + "set_cell": { + "value": "" + } + } + } + ], + "done": true + } + } + ], + "results": [ + { + "record": { + "change_stream_mutation": { + "row_key": "0000000000000000", + "type": "USER", + "source_cluster_id": "source-cluster-id", + "commit_timestamp": "2022-07-01T00:00:00Z", + "tiebreaker": 100, + "token": "data-change-token", + "low_watermark": "2022-07-01T00:00:00Z", + "mutations": [ + { + "set_cell": { + "family_name": "family", + "column_qualifier": "0000000000000000", + "timestamp_micros": 1000, + "value": "dg==" + } + } + ] + } + }, + "error": false + } + ] + }, + { + "description": "2 chunked SetCells", + "api_responses": [ + { + "data_change": { + "row_key": "0000000000000000", + "type": "USER", + "source_cluster_id": "source-cluster-id", + "commit_timestamp": "2022-07-01T00:00:00Z", + "tiebreaker": 100, + "chunks": [ + { + "chunk_info": { + "chunked_value_size": 1 + }, + "mutation": { + "set_cell": { + "family_name": "family", + "column_qualifier": "0000000000000000", + "timestamp_micros": 1000, + "value": "dg==" + } + } + } + ] + } + }, + { + "data_change": { + "type": "CONTINUATION", + "chunks": [ + { + "chunk_info": { + "chunked_value_offset": 1, + "chunked_value_size": 1, + "last_chunk": true + }, + "mutation": { + "set_cell": { + "value": "" + } + } + }, + { + "chunk_info": { + "chunked_value_size": 1 + }, + "mutation": { + "set_cell": { + "family_name": "family", + "column_qualifier": "0000000000000000", + "timestamp_micros": 1000, + "value": "dg==" + } + } + } + ] + } + }, + { + "data_change": { + "type": "CONTINUATION", + "token": "data-change-token", + "low_watermark": "2022-07-01T00:00:00Z", + "chunks": [ + { + "chunk_info": { + "chunked_value_offset": 1, + "chunked_value_size": 1, + "last_chunk": true + }, + "mutation": { + "set_cell": { + "value": "" + } + } + } + ], + "done": true + } + } + ], + "results": [ + { + "record": { + "change_stream_mutation": { + "row_key": "0000000000000000", + "type": "USER", + "source_cluster_id": "source-cluster-id", + "commit_timestamp": "2022-07-01T00:00:00Z", + "tiebreaker": 100, + "token": "data-change-token", + "low_watermark": "2022-07-01T00:00:00Z", + "mutations": [ + { + "set_cell": { + "family_name": "family", + "column_qualifier": "0000000000000000", + "timestamp_micros": 1000, + "value": "dg==" + } + }, + { + "set_cell": { + "family_name": "family", + "column_qualifier": "0000000000000000", + "timestamp_micros": 1000, + "value": "dg==" + } + } + ] + } + }, + "error": false + } + ] + }, + { + "description": "1 chunked SetCell + 1 unchunked SetCell", + "api_responses": [ + { + "data_change": { + "row_key": "0000000000000000", + "type": "USER", + "source_cluster_id": "source-cluster-id", + "commit_timestamp": "2022-07-01T00:00:00Z", + "tiebreaker": 100, + "chunks": [ + { + "chunk_info": { + "chunked_value_size": 1 + }, + "mutation": { + "set_cell": { + "family_name": "family", + "column_qualifier": "0000000000000000", + "timestamp_micros": 1000, + "value": "dg==" + } + } + } + ] + } + }, + { + "data_change": { + "type": "CONTINUATION", + "token": "data-change-token", + "low_watermark": "2022-07-01T00:00:00Z", + "chunks": [ + { + "chunk_info": { + "chunked_value_offset": 1, + "chunked_value_size": 1, + "last_chunk": true + }, + "mutation": { + "set_cell": { + "value": "" + } + } + }, + { + "mutation": { + "set_cell": { + "family_name": "family", + "column_qualifier": "0000000000000000", + "timestamp_micros": 1000, + "value": "dg==" + } + } + } + ], + "done": true + } + } + ], + "results": [ + { + "record": { + "change_stream_mutation": { + "row_key": "0000000000000000", + "type": "USER", + "source_cluster_id": "source-cluster-id", + "commit_timestamp": "2022-07-01T00:00:00Z", + "tiebreaker": 100, + "token": "data-change-token", + "low_watermark": "2022-07-01T00:00:00Z", + "mutations": [ + { + "set_cell": { + "family_name": "family", + "column_qualifier": "0000000000000000", + "timestamp_micros": 1000, + "value": "dg==" + } + }, + { + "set_cell": { + "family_name": "family", + "column_qualifier": "0000000000000000", + "timestamp_micros": 1000, + "value": "dg==" + } + } + ] + } + }, + "error": false + } + ] + }, + { + "description": "1 unchunked SetCell + 1 chunked SetCell", + "api_responses": [ + { + "data_change": { + "row_key": "0000000000000000", + "type": "USER", + "source_cluster_id": "source-cluster-id", + "commit_timestamp": "2022-07-01T00:00:00Z", + "tiebreaker": 100, + "chunks": [ + { + "mutation": { + "set_cell": { + "family_name": "family", + "column_qualifier": "0000000000000000", + "timestamp_micros": 1000, + "value": "dg==" + } + } + }, + { + "chunk_info": { + "chunked_value_size": 1 + }, + "mutation": { + "set_cell": { + "family_name": "family", + "column_qualifier": "0000000000000000", + "timestamp_micros": 1000, + "value": "dg==" + } + } + } + ] + } + }, + { + "data_change": { + "type": "CONTINUATION", + "token": "data-change-token", + "low_watermark": "2022-07-01T00:00:00Z", + "chunks": [ + { + "chunk_info": { + "chunked_value_offset": 1, + "chunked_value_size": 1, + "last_chunk": true + }, + "mutation": { + "set_cell": { + "value": "" + } + } + } + ], + "done": true + } + } + ], + "results": [ + { + "record": { + "change_stream_mutation": { + "row_key": "0000000000000000", + "type": "USER", + "source_cluster_id": "source-cluster-id", + "commit_timestamp": "2022-07-01T00:00:00Z", + "tiebreaker": 100, + "token": "data-change-token", + "low_watermark": "2022-07-01T00:00:00Z", + "mutations": [ + { + "set_cell": { + "family_name": "family", + "column_qualifier": "0000000000000000", + "timestamp_micros": 1000, + "value": "dg==" + } + }, + { + "set_cell": { + "family_name": "family", + "column_qualifier": "0000000000000000", + "timestamp_micros": 1000, + "value": "dg==" + } + } + ] + } + }, + "error": false + } + ] + }, + { + "description": "1 mod + 1 chunked SetCell + 1 mod", + "api_responses": [ + { + "data_change": { + "row_key": "0000000000000000", + "type": "USER", + "source_cluster_id": "source-cluster-id", + "commit_timestamp": "2022-07-01T00:00:00Z", + "tiebreaker": 100, + "chunks": [ + { + "mutation": { + "delete_from_family": { + "family_name": "family" + } + } + }, + { + "chunk_info": { + "chunked_value_size": 1 + }, + "mutation": { + "set_cell": { + "family_name": "family", + "column_qualifier": "0000000000000000", + "timestamp_micros": 1000, + "value": "dg==" + } + } + } + ] + } + }, + { + "data_change": { + "type": "CONTINUATION", + "token": "data-change-token", + "low_watermark": "2022-07-01T00:00:00Z", + "chunks": [ + { + "chunk_info": { + "chunked_value_offset": 1, + "chunked_value_size": 1, + "last_chunk": true + }, + "mutation": { + "set_cell": { + "value": "" + } + } + }, + { + "mutation": { + "delete_from_family": { + "family_name": "family" + } + } + } + ], + "done": true + } + } + ], + "results": [ + { + "record": { + "change_stream_mutation": { + "row_key": "0000000000000000", + "type": "USER", + "source_cluster_id": "source-cluster-id", + "commit_timestamp": "2022-07-01T00:00:00Z", + "tiebreaker": 100, + "token": "data-change-token", + "low_watermark": "2022-07-01T00:00:00Z", + "mutations": [ + { + "delete_from_family": { + "family_name": "family" + } + }, + { + "set_cell": { + "family_name": "family", + "column_qualifier": "0000000000000000", + "timestamp_micros": 1000, + "value": "dg==" + } + }, + { + "delete_from_family": { + "family_name": "family" + } + } + ] + } + }, + "error": false + } + ] + }, + { + "description": "non SetCell chunking", + "api_responses": [ + { + "data_change": { + "row_key": "0000000000000000", + "type": "USER", + "source_cluster_id": "source-cluster-id", + "commit_timestamp": "2022-07-01T00:00:00Z", + "tiebreaker": 100, + "chunks": [ + { + "mutation": { + "delete_from_family": { + "family_name": "family" + } + } + } + ] + } + }, + { + "data_change": { + "type": "CONTINUATION", + "token": "data-change-token", + "low_watermark": "2022-07-01T00:00:00Z", + "chunks": [ + { + "mutation": { + "delete_from_family": { + "family_name": "family" + } + } + } + ], + "done": true + } + } + ], + "results": [ + { + "record": { + "change_stream_mutation": { + "row_key": "0000000000000000", + "type": "USER", + "source_cluster_id": "source-cluster-id", + "commit_timestamp": "2022-07-01T00:00:00Z", + "tiebreaker": 100, + "token": "data-change-token", + "low_watermark": "2022-07-01T00:00:00Z", + "mutations": [ + { + "delete_from_family": { + "family_name": "family" + } + }, + { + "delete_from_family": { + "family_name": "family" + } + } + ] + } + }, + "error": false + } + ] + }, + { + "description": "2 logical mutations with non SetCell chunking + CloseStream", + "api_responses": [ + { + "data_change": { + "row_key": "0000000000000000", + "type": "USER", + "source_cluster_id": "source-cluster-id", + "commit_timestamp": "2022-07-01T00:00:00Z", + "tiebreaker": 100, + "chunks": [ + { + "mutation": { + "delete_from_family": { + "family_name": "family" + } + } + } + ] + } + }, + { + "data_change": { + "type": "CONTINUATION", + "token": "data-change-token", + "low_watermark": "2022-07-01T00:00:00Z", + "chunks": [ + { + "mutation": { + "delete_from_family": { + "family_name": "family" + } + } + } + ], + "done": true + } + }, + { + "data_change": { + "row_key": "0000000000000000", + "type": "USER", + "source_cluster_id": "source-cluster-id", + "commit_timestamp": "2022-07-01T00:00:00Z", + "tiebreaker": 100, + "chunks": [ + { + "mutation": { + "delete_from_family": { + "family_name": "family" + } + } + } + ] + } + }, + { + "data_change": { + "type": "CONTINUATION", + "token": "data-change-token", + "low_watermark": "2022-07-01T00:00:00Z", + "chunks": [ + { + "mutation": { + "delete_from_family": { + "family_name": "family" + } + } + } + ], + "done": true + } + }, + { + "close_stream": { + "status": { + "code": "11", + "message": "Partition boundaries are misaligned." + }, + "continuation_tokens": [ + { + "partition": { + "row_range": { + "start_key_closed": "", + "end_key_open": "0000000000000001" + } + }, + "token": "close-stream-token-1" + }, + { + "partition": { + "row_range": { + "start_key_closed": "0000000000000001", + "end_key_open": "0000000000000002" + } + }, + "token": "close-stream-token-2" + } + ] + } + } + ], + "results": [ + { + "record": { + "change_stream_mutation": { + "row_key": "0000000000000000", + "type": "USER", + "source_cluster_id": "source-cluster-id", + "commit_timestamp": "2022-07-01T00:00:00Z", + "tiebreaker": 100, + "token": "data-change-token", + "low_watermark": "2022-07-01T00:00:00Z", + "mutations": [ + { + "delete_from_family": { + "family_name": "family" + } + }, + { + "delete_from_family": { + "family_name": "family" + } + } + ] + } + }, + "error": false + }, + { + "record": { + "change_stream_mutation": { + "row_key": "0000000000000000", + "type": "USER", + "source_cluster_id": "source-cluster-id", + "commit_timestamp": "2022-07-01T00:00:00Z", + "tiebreaker": 100, + "token": "data-change-token", + "low_watermark": "2022-07-01T00:00:00Z", + "mutations": [ + { + "delete_from_family": { + "family_name": "family" + } + }, + { + "delete_from_family": { + "family_name": "family" + } + } + ] + } + }, + "error": false + }, + { + "record" : { + "close_stream": { + "status": { + "code": "11", + "message": "Partition boundaries are misaligned." + }, + "continuation_tokens": [ + { + "partition": { + "row_range": { + "start_key_closed": "", + "end_key_open": "0000000000000001" + } + }, + "token": "close-stream-token-1" + }, + { + "partition": { + "row_range": { + "start_key_closed": "0000000000000001", + "end_key_open": "0000000000000002" + } + }, + "token": "close-stream-token-2" + } + ] + } + }, + "error": false + } + ] + } + ] +} \ No newline at end of file diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/BigtableDataClientTests.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/BigtableDataClientTests.java index fcbcc15e30..648a298155 100644 --- a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/BigtableDataClientTests.java +++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/BigtableDataClientTests.java @@ -26,11 +26,13 @@ import com.google.api.gax.rpc.UnaryCallable; import com.google.bigtable.v2.RowRange; import com.google.cloud.bigtable.data.v2.models.BulkMutation; +import com.google.cloud.bigtable.data.v2.models.ChangeStreamRecord; import com.google.cloud.bigtable.data.v2.models.ConditionalRowMutation; import com.google.cloud.bigtable.data.v2.models.Filters.Filter; import com.google.cloud.bigtable.data.v2.models.KeyOffset; import com.google.cloud.bigtable.data.v2.models.Mutation; import com.google.cloud.bigtable.data.v2.models.Query; +import com.google.cloud.bigtable.data.v2.models.ReadChangeStreamQuery; import com.google.cloud.bigtable.data.v2.models.ReadModifyWriteRow; import com.google.cloud.bigtable.data.v2.models.Row; import com.google.cloud.bigtable.data.v2.models.RowCell; @@ -83,6 +85,10 @@ public class BigtableDataClientTests { @Mock(answer = Answers.RETURNS_DEEP_STUBS) private ServerStreamingCallable mockListChangeStreamPartitionsCallable; + @Mock(answer = Answers.RETURNS_DEEP_STUBS) + private ServerStreamingCallable + mockReadChangeStreamCallable; + private BigtableDataClient bigtableDataClient; @Before @@ -165,6 +171,13 @@ public void proxyListChangeStreamPartitionsCallableTest() { .isSameInstanceAs(mockListChangeStreamPartitionsCallable); } + @Test + public void proxyReadChangeStreamCallableTest() { + Mockito.when(mockStub.readChangeStreamCallable()).thenReturn(mockReadChangeStreamCallable); + assertThat(bigtableDataClient.readChangeStreamCallable()) + .isSameInstanceAs(mockReadChangeStreamCallable); + } + @Test public void proxyReadRowAsyncTest() { Mockito.when(mockStub.readRowCallable()).thenReturn(mockReadRowCallable); @@ -334,6 +347,28 @@ public void proxyListChangeStreamPartitionsAsyncTest() { Mockito.verify(mockListChangeStreamPartitionsCallable).call("fake-table", mockObserver); } + @Test + public void proxyReadChangeStreamSyncTest() { + Mockito.when(mockStub.readChangeStreamCallable()).thenReturn(mockReadChangeStreamCallable); + + ReadChangeStreamQuery query = ReadChangeStreamQuery.create("fake-table"); + bigtableDataClient.readChangeStream(query); + + Mockito.verify(mockReadChangeStreamCallable).call(query); + } + + @Test + public void proxyReadChangeStreamAsyncTest() { + Mockito.when(mockStub.readChangeStreamCallable()).thenReturn(mockReadChangeStreamCallable); + + @SuppressWarnings("unchecked") + ResponseObserver mockObserver = Mockito.mock(ResponseObserver.class); + ReadChangeStreamQuery query = ReadChangeStreamQuery.create("fake-table"); + bigtableDataClient.readChangeStreamAsync(query, mockObserver); + + Mockito.verify(mockReadChangeStreamCallable).call(query, mockObserver); + } + @Test public void proxySampleRowKeysCallableTest() { Mockito.when(mockStub.sampleRowKeysCallable()).thenReturn(mockSampleRowKeysCallable); diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/models/ChangeStreamRecordTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/models/ChangeStreamRecordTest.java index 05df603959..b819b4c4ac 100644 --- a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/models/ChangeStreamRecordTest.java +++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/models/ChangeStreamRecordTest.java @@ -121,6 +121,8 @@ public void heartbeatTest() { Assert.assertEquals(actualHeartbeat.getLowWatermark(), lowWatermark); Assert.assertEquals(actualHeartbeat.getChangeStreamContinuationToken().getRowRange(), rowRange); Assert.assertEquals(actualHeartbeat.getChangeStreamContinuationToken().getToken(), token); + + Assert.assertEquals(actualHeartbeat.toProto(), heartbeatProto); } @Test @@ -163,5 +165,7 @@ public void closeStreamTest() { rowRange2, actualCloseStream.getChangeStreamContinuationTokens().get(1).getRowRange()); Assert.assertEquals( token2, actualCloseStream.getChangeStreamContinuationTokens().get(1).getToken()); + + Assert.assertEquals(actualCloseStream.toProto(), closeStreamProto); } } diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStubSettingsTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStubSettingsTest.java index 32ab93d1f2..fa2efbf7e0 100644 --- a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStubSettingsTest.java +++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStubSettingsTest.java @@ -702,6 +702,7 @@ public void isRefreshingChannelFalseValueTest() { "checkAndMutateRowSettings", "readModifyWriteRowSettings", "listChangeStreamPartitionsSettings", + "readChangeStreamSettings", }; @Test diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamRecordMergingCallableTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamRecordMergingCallableTest.java new file mode 100644 index 0000000000..de5a9e8213 --- /dev/null +++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamRecordMergingCallableTest.java @@ -0,0 +1,803 @@ +/* + * Copyright 2022 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.changestream; + +import com.google.bigtable.v2.Mutation; +import com.google.bigtable.v2.ReadChangeStreamRequest; +import com.google.bigtable.v2.ReadChangeStreamResponse; +import com.google.bigtable.v2.StreamContinuationToken; +import com.google.bigtable.v2.TimestampRange; +import com.google.cloud.bigtable.data.v2.models.ChangeStreamMutation; +import com.google.cloud.bigtable.data.v2.models.ChangeStreamRecord; +import com.google.cloud.bigtable.data.v2.models.CloseStream; +import com.google.cloud.bigtable.data.v2.models.DefaultChangeStreamRecordAdapter; +import com.google.cloud.bigtable.data.v2.models.Heartbeat; +import com.google.cloud.bigtable.data.v2.models.Range; +import com.google.cloud.bigtable.gaxx.testing.FakeStreamingApi; +import com.google.cloud.bigtable.gaxx.testing.FakeStreamingApi.ServerStreamingStashCallable; +import com.google.common.truth.Truth; +import com.google.protobuf.ByteString; +import com.google.protobuf.Timestamp; +import com.google.rpc.Status; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Additional tests in addition to {@link + * com.google.cloud.bigtable.data.v2.stub.changestream.ReadChangeStreamMergingAcceptanceTest}. At + * some point they should be reintegrated into the json file. + */ +@RunWith(JUnit4.class) +public class ChangeStreamRecordMergingCallableTest { + + @Test + public void heartbeatTest() { + ReadChangeStreamResponse.Heartbeat heartbeat = + ReadChangeStreamResponse.Heartbeat.newBuilder() + .setLowWatermark(Timestamp.newBuilder().setSeconds(1000).build()) + .setContinuationToken( + StreamContinuationToken.newBuilder().setToken("random-token").build()) + .build(); + ReadChangeStreamResponse response = + ReadChangeStreamResponse.newBuilder().setHeartbeat(heartbeat).build(); + FakeStreamingApi.ServerStreamingStashCallable + inner = new ServerStreamingStashCallable<>(Collections.singletonList(response)); + + ChangeStreamRecordMergingCallable mergingCallable = + new ChangeStreamRecordMergingCallable<>(inner, new DefaultChangeStreamRecordAdapter()); + List results = + mergingCallable.all().call(ReadChangeStreamRequest.getDefaultInstance()); + + Truth.assertThat(results).containsExactly(Heartbeat.fromProto(heartbeat)); + } + + @Test + public void closeStreamTest() { + ReadChangeStreamResponse.CloseStream closeStream = + ReadChangeStreamResponse.CloseStream.newBuilder() + .addContinuationTokens( + StreamContinuationToken.newBuilder().setToken("random-token").build()) + .setStatus(Status.newBuilder().setCode(0).build()) + .build(); + ReadChangeStreamResponse response = + ReadChangeStreamResponse.newBuilder().setCloseStream(closeStream).build(); + FakeStreamingApi.ServerStreamingStashCallable + inner = new ServerStreamingStashCallable<>(Collections.singletonList(response)); + + ChangeStreamRecordMergingCallable mergingCallable = + new ChangeStreamRecordMergingCallable<>(inner, new DefaultChangeStreamRecordAdapter()); + List results = + mergingCallable.all().call(ReadChangeStreamRequest.getDefaultInstance()); + + Truth.assertThat(results).containsExactly(CloseStream.fromProto(closeStream)); + } + + // [{DeleteFamily, DeleteCells, SetCell}, {CloseStream}] + // -> [ChangeStreamMutation{DeleteFamily, DeleteCells, SetCell}, CloseStream] + @Test + public void multipleModesNoChunkingTest() { + // Construct the ReadChangeStreamResponse's. + Timestamp fakeLowWatermark = Timestamp.newBuilder().setSeconds(100).build(); + Mutation deleteFromFamily = + Mutation.newBuilder() + .setDeleteFromFamily( + Mutation.DeleteFromFamily.newBuilder().setFamilyName("fake-family").build()) + .build(); + Mutation deleteFromColumn = + Mutation.newBuilder() + .setDeleteFromColumn( + Mutation.DeleteFromColumn.newBuilder() + .setFamilyName("fake-family") + .setColumnQualifier(ByteString.copyFromUtf8("fake-qualifier")) + .setTimeRange( + TimestampRange.newBuilder() + .setStartTimestampMicros(1000L) + .setEndTimestampMicros(2000L) + .build()) + .build()) + .build(); + Mutation setCell = + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setFamilyName("fake-family") + .setColumnQualifier(ByteString.copyFromUtf8("fake-qualifier")) + .setTimestampMicros(1000L) + .setValue(ByteString.copyFromUtf8("fake-value")) + .build()) + .build(); + ReadChangeStreamResponse.DataChange dataChange = + ReadChangeStreamResponse.DataChange.newBuilder() + .setType(ReadChangeStreamResponse.DataChange.Type.USER) + .setSourceClusterId("fake-source-cluster-id") + .setRowKey(ByteString.copyFromUtf8("key")) + .setCommitTimestamp(Timestamp.newBuilder().setSeconds(100).build()) + .setTiebreaker(100) + .setLowWatermark(fakeLowWatermark) + .setToken("fake-token") + .addChunks( + ReadChangeStreamResponse.MutationChunk.newBuilder().setMutation(deleteFromFamily)) + .addChunks( + ReadChangeStreamResponse.MutationChunk.newBuilder().setMutation(deleteFromColumn)) + .addChunks(ReadChangeStreamResponse.MutationChunk.newBuilder().setMutation(setCell)) + .setDone(true) + .build(); + ReadChangeStreamResponse dataChangeResponse = + ReadChangeStreamResponse.newBuilder().setDataChange(dataChange).build(); + ReadChangeStreamResponse closeStreamResponse = + ReadChangeStreamResponse.newBuilder() + .setCloseStream( + ReadChangeStreamResponse.CloseStream.newBuilder() + .addContinuationTokens( + StreamContinuationToken.newBuilder().setToken("random-token").build()) + .setStatus(Status.newBuilder().setCode(0).build()) + .build()) + .build(); + FakeStreamingApi.ServerStreamingStashCallable + inner = + new ServerStreamingStashCallable<>( + Arrays.asList(dataChangeResponse, closeStreamResponse)); + ChangeStreamRecordMergingCallable mergingCallable = + new ChangeStreamRecordMergingCallable<>(inner, new DefaultChangeStreamRecordAdapter()); + + // Actual results. + List results = + mergingCallable.all().call(ReadChangeStreamRequest.getDefaultInstance()); + + // Expected results. + Timestamp fakeCommitTimestamp = Timestamp.newBuilder().setSeconds(100).build(); + ChangeStreamMutation changeStreamMutation = + ChangeStreamMutation.createUserMutation( + ByteString.copyFromUtf8("key"), "fake-source-cluster-id", fakeCommitTimestamp, 100) + .deleteFamily("fake-family") + .deleteCells( + "fake-family", + ByteString.copyFromUtf8("fake-qualifier"), + Range.TimestampRange.create(1000L, 2000L)) + .setCell( + "fake-family", + ByteString.copyFromUtf8("fake-qualifier"), + 1000L, + ByteString.copyFromUtf8("fake-value")) + .setToken("fake-token") + .setLowWatermark(fakeLowWatermark) + .build(); + Truth.assertThat(results) + .containsExactly( + changeStreamMutation, CloseStream.fromProto(closeStreamResponse.getCloseStream())); + } + + // [{DeleteFamily}] -> [ChangeStreamMutation{DeleteFamily}]. + // Tests that the resulting ChangeStreamMutation for a Garbage Collection mutation + // doesn't have source cluster id. + @Test + public void GcModNoChunkingTest() { + Mutation deleteFromFamily = + Mutation.newBuilder() + .setDeleteFromFamily( + Mutation.DeleteFromFamily.newBuilder().setFamilyName("fake-family").build()) + .build(); + Timestamp fakeLowWatermark = Timestamp.newBuilder().setSeconds(100).build(); + ReadChangeStreamResponse.DataChange dataChange = + ReadChangeStreamResponse.DataChange.newBuilder() + .setType(ReadChangeStreamResponse.DataChange.Type.GARBAGE_COLLECTION) + .setRowKey(ByteString.copyFromUtf8("key")) + .setCommitTimestamp(Timestamp.newBuilder().setSeconds(100).build()) + .setTiebreaker(100) + .setLowWatermark(fakeLowWatermark) + .setToken("fake-token") + .addChunks( + ReadChangeStreamResponse.MutationChunk.newBuilder().setMutation(deleteFromFamily)) + .setDone(true) + .build(); + ReadChangeStreamResponse dataChangeResponse = + ReadChangeStreamResponse.newBuilder().setDataChange(dataChange).build(); + FakeStreamingApi.ServerStreamingStashCallable + inner = new ServerStreamingStashCallable<>(Collections.singletonList(dataChangeResponse)); + ChangeStreamRecordMergingCallable mergingCallable = + new ChangeStreamRecordMergingCallable<>(inner, new DefaultChangeStreamRecordAdapter()); + + // Actual results. + List results = + mergingCallable.all().call(ReadChangeStreamRequest.getDefaultInstance()); + + // Expected results. Note that it shouldn't have source cluster id. + Timestamp fakeCommitTimestamp = Timestamp.newBuilder().setSeconds(100).build(); + ChangeStreamMutation changeStreamMutation = + ChangeStreamMutation.createGcMutation( + ByteString.copyFromUtf8("key"), fakeCommitTimestamp, 100) + .deleteFamily("fake-family") + .setToken("fake-token") + .setLowWatermark(fakeLowWatermark) + .build(); + Truth.assertThat(results).containsExactly(changeStreamMutation); + } + + // [{SetCell_chunk1}, {SetCell_chunk2}] -> [ChangeStreamMutation{SetCell}] + @Test + public void oneChunkedCellTest() { + String firstCellValue = "chunk1-value"; + String secondCellValue = "chunk2-value"; + String totalCellValue = firstCellValue + secondCellValue; + Timestamp fakeLowWatermark = Timestamp.newBuilder().setSeconds(100).build(); + // Build the first ReadChangeStreamResponse which contains the first chunk of a chunked SetCell. + Mutation chunkedCell1 = + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setFamilyName("fake-family") + .setColumnQualifier(ByteString.copyFromUtf8("fake-qualifier")) + .setTimestampMicros(1000L) + .setValue(ByteString.copyFromUtf8(firstCellValue)) + .build()) + .build(); + ReadChangeStreamResponse.MutationChunk.ChunkInfo chunkInfo1 = + ReadChangeStreamResponse.MutationChunk.ChunkInfo.newBuilder() + .setChunkedValueSize(totalCellValue.length()) + .build(); + ReadChangeStreamResponse.DataChange dataChange1 = + ReadChangeStreamResponse.DataChange.newBuilder() + .setType(ReadChangeStreamResponse.DataChange.Type.USER) + .setSourceClusterId("fake-source-cluster-id") + .setRowKey(ByteString.copyFromUtf8("key")) + .setCommitTimestamp(Timestamp.newBuilder().setSeconds(100).build()) + .setTiebreaker(100) + .addChunks( + ReadChangeStreamResponse.MutationChunk.newBuilder() + .setMutation(chunkedCell1) + .setChunkInfo(chunkInfo1)) + .build(); + ReadChangeStreamResponse dataChangeResponse1 = + ReadChangeStreamResponse.newBuilder().setDataChange(dataChange1).build(); + + // Build the second ReadChangeStreamResponse which contains the second chunk of a chunked + // SetCell. + Mutation chunkedCell2 = + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setValue(ByteString.copyFromUtf8(secondCellValue)) + .build()) + .build(); + ReadChangeStreamResponse.MutationChunk.ChunkInfo chunkInfo2 = + ReadChangeStreamResponse.MutationChunk.ChunkInfo.newBuilder() + .setChunkedValueOffset(firstCellValue.length()) + .setChunkedValueSize(totalCellValue.length()) + .setLastChunk(true) + .build(); + ReadChangeStreamResponse.DataChange dataChange2 = + ReadChangeStreamResponse.DataChange.newBuilder() + .setType(ReadChangeStreamResponse.DataChange.Type.CONTINUATION) + .addChunks( + ReadChangeStreamResponse.MutationChunk.newBuilder() + .setMutation(chunkedCell2) + .setChunkInfo(chunkInfo2)) + .setDone(true) + .setToken("fake-token") + .setLowWatermark(fakeLowWatermark) + .build(); + ReadChangeStreamResponse dataChangeResponse2 = + ReadChangeStreamResponse.newBuilder().setDataChange(dataChange2).build(); + + // Feed the two ReadChangeStream responses to the merging callable. + FakeStreamingApi.ServerStreamingStashCallable + inner = + new ServerStreamingStashCallable<>( + Arrays.asList(dataChangeResponse1, dataChangeResponse2)); + ChangeStreamRecordMergingCallable mergingCallable = + new ChangeStreamRecordMergingCallable<>(inner, new DefaultChangeStreamRecordAdapter()); + // Actual results. + List results = + mergingCallable.all().call(ReadChangeStreamRequest.getDefaultInstance()); + + // Expected results. + Timestamp fakeCommitTimestamp = Timestamp.newBuilder().setSeconds(100).build(); + ChangeStreamMutation changeStreamMutation = + ChangeStreamMutation.createUserMutation( + ByteString.copyFromUtf8("key"), "fake-source-cluster-id", fakeCommitTimestamp, 100) + .setCell( + "fake-family", + ByteString.copyFromUtf8("fake-qualifier"), + 1000L, + ByteString.copyFromUtf8(totalCellValue)) + .setToken("fake-token") + .setLowWatermark(fakeLowWatermark) + .build(); + Truth.assertThat(results).containsExactly(changeStreamMutation); + } + + // [{SetCell1_chunk1}, {SetCell1_chunk2, SetCell2_chunk1}, {SetCell2_chunk2}] + // -> [ChangeStreamMutation{SetCell1, SetCell2}] + @Test + public void twoChunkedCellsTest() { + Timestamp fakeLowWatermark = Timestamp.newBuilder().setSeconds(100).build(); + String setCell1Chunk1val = "cell1-chunk1"; + String setCell1Chunk2val = "cell1-chunk2"; + String setCell2Chunk1val = "cell2-chunk1"; + String setCell2Chunk2val = "cell2-chunk2"; + String totalCellValue1 = setCell1Chunk1val + setCell1Chunk2val; + String totalCellValue2 = setCell2Chunk1val + setCell2Chunk2val; + // Build the 1st ReadChangeStreamResponse which has 1/2 of SetCell1. + Mutation setCell1Chunk1 = + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setFamilyName("fake-family") + .setColumnQualifier(ByteString.copyFromUtf8("fake-qualifier")) + .setTimestampMicros(1000L) + .setValue(ByteString.copyFromUtf8(setCell1Chunk1val)) + .build()) + .build(); + ReadChangeStreamResponse.MutationChunk.ChunkInfo setCell1ChunkInfo1 = + ReadChangeStreamResponse.MutationChunk.ChunkInfo.newBuilder() + .setChunkedValueSize(totalCellValue1.length()) + .build(); + ReadChangeStreamResponse.DataChange dataChange1 = + ReadChangeStreamResponse.DataChange.newBuilder() + .setType(ReadChangeStreamResponse.DataChange.Type.USER) + .setSourceClusterId("fake-source-cluster-id") + .setRowKey(ByteString.copyFromUtf8("key")) + .setCommitTimestamp(Timestamp.newBuilder().setSeconds(100).build()) + .setTiebreaker(100) + .addChunks( + ReadChangeStreamResponse.MutationChunk.newBuilder() + .setMutation(setCell1Chunk1) + .setChunkInfo(setCell1ChunkInfo1)) + .build(); + ReadChangeStreamResponse dataChangeResponse1 = + ReadChangeStreamResponse.newBuilder().setDataChange(dataChange1).build(); + + // Build the 2nd ReadChangeStreamResponse which has 2/2 of SetCell1 and 1/2 of SetCell2. + Mutation setCell1Chunk2 = + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setValue(ByteString.copyFromUtf8(setCell1Chunk2val)) + .build()) + .build(); + ReadChangeStreamResponse.MutationChunk.ChunkInfo setCell1ChunkInfo2 = + ReadChangeStreamResponse.MutationChunk.ChunkInfo.newBuilder() + .setChunkedValueOffset(setCell1Chunk1val.length()) + .setChunkedValueSize(totalCellValue1.length()) + .setLastChunk(true) + .build(); + Mutation setCell2Chunk1 = + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setFamilyName("fake-family") + .setColumnQualifier(ByteString.copyFromUtf8("fake-qualifier")) + .setTimestampMicros(1000L) + .setValue(ByteString.copyFromUtf8(setCell2Chunk1val)) + .build()) + .build(); + ReadChangeStreamResponse.MutationChunk.ChunkInfo setCell2ChunkInfo1 = + ReadChangeStreamResponse.MutationChunk.ChunkInfo.newBuilder() + .setChunkedValueSize(totalCellValue2.length()) + .build(); + ReadChangeStreamResponse.DataChange dataChange2 = + ReadChangeStreamResponse.DataChange.newBuilder() + .setType(ReadChangeStreamResponse.DataChange.Type.CONTINUATION) + .addChunks( + ReadChangeStreamResponse.MutationChunk.newBuilder() + .setMutation(setCell1Chunk2) + .setChunkInfo(setCell1ChunkInfo2)) + .addChunks( + ReadChangeStreamResponse.MutationChunk.newBuilder() + .setMutation(setCell2Chunk1) + .setChunkInfo(setCell2ChunkInfo1)) + .build(); + ReadChangeStreamResponse dataChangeResponse2 = + ReadChangeStreamResponse.newBuilder().setDataChange(dataChange2).build(); + + // Build the 3rd ReadChangeStreamResponse which has 2/2 of SetCell2. + Mutation setCell2Chunk2 = + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setValue(ByteString.copyFromUtf8(setCell2Chunk2val)) + .build()) + .build(); + ReadChangeStreamResponse.MutationChunk.ChunkInfo setCell2ChunkInfo2 = + ReadChangeStreamResponse.MutationChunk.ChunkInfo.newBuilder() + .setChunkedValueOffset(setCell2Chunk1val.length()) + .setChunkedValueSize(totalCellValue2.length()) + .setLastChunk(true) + .build(); + ReadChangeStreamResponse.DataChange dataChange3 = + ReadChangeStreamResponse.DataChange.newBuilder() + .setType(ReadChangeStreamResponse.DataChange.Type.CONTINUATION) + .addChunks( + ReadChangeStreamResponse.MutationChunk.newBuilder() + .setMutation(setCell2Chunk2) + .setChunkInfo(setCell2ChunkInfo2)) + .setDone(true) + .setToken("fake-token") + .setLowWatermark(fakeLowWatermark) + .build(); + ReadChangeStreamResponse dataChangeResponse3 = + ReadChangeStreamResponse.newBuilder().setDataChange(dataChange3).build(); + + // Feed the 3 ReadChangeStream responses to the merging callable. + FakeStreamingApi.ServerStreamingStashCallable + inner = + new ServerStreamingStashCallable<>( + Arrays.asList(dataChangeResponse1, dataChangeResponse2, dataChangeResponse3)); + ChangeStreamRecordMergingCallable mergingCallable = + new ChangeStreamRecordMergingCallable<>(inner, new DefaultChangeStreamRecordAdapter()); + // Actual results. + List results = + mergingCallable.all().call(ReadChangeStreamRequest.getDefaultInstance()); + + // Expected results. + Timestamp fakeCommitTimestamp = Timestamp.newBuilder().setSeconds(100).build(); + ChangeStreamMutation changeStreamMutation = + ChangeStreamMutation.createUserMutation( + ByteString.copyFromUtf8("key"), "fake-source-cluster-id", fakeCommitTimestamp, 100) + .setCell( + "fake-family", + ByteString.copyFromUtf8("fake-qualifier"), + 1000L, + ByteString.copyFromUtf8(totalCellValue1)) + .setCell( + "fake-family", + ByteString.copyFromUtf8("fake-qualifier"), + 1000L, + ByteString.copyFromUtf8(totalCellValue2)) + .setToken("fake-token") + .setLowWatermark(fakeLowWatermark) + .build(); + Truth.assertThat(results).containsExactly(changeStreamMutation); + } + + // [{SetCell1_chunk1}, {SetCell1_chunk2, DeleteFamily, DeleteCells, SetCell2}] + // -> [ChangeStreamMutation{SetCell1, DeleteFamily, DeleteCells, SetCell2}] + @Test + public void oneChunkedCell_manyNonChunkedModsTest() { + Timestamp fakeLowWatermark = Timestamp.newBuilder().setSeconds(100).build(); + String firstCellValue = "chunk1-value"; + String secondCellValue = "chunk2-value"; + String totalCellValue = firstCellValue + secondCellValue; + // Build the 1st ReadChangeStreamResponse which contains 1/2 of a chunked SetCell. + Mutation chunkedCell1 = + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setFamilyName("fake-family") + .setColumnQualifier(ByteString.copyFromUtf8("fake-qualifier")) + .setTimestampMicros(1000L) + .setValue(ByteString.copyFromUtf8(firstCellValue)) + .build()) + .build(); + ReadChangeStreamResponse.MutationChunk.ChunkInfo chunkInfo1 = + ReadChangeStreamResponse.MutationChunk.ChunkInfo.newBuilder() + .setChunkedValueSize(totalCellValue.length()) + .build(); + ReadChangeStreamResponse.DataChange dataChange1 = + ReadChangeStreamResponse.DataChange.newBuilder() + .setType(ReadChangeStreamResponse.DataChange.Type.USER) + .setSourceClusterId("fake-source-cluster-id") + .setRowKey(ByteString.copyFromUtf8("key")) + .setCommitTimestamp(Timestamp.newBuilder().setSeconds(100).build()) + .setTiebreaker(100) + .addChunks( + ReadChangeStreamResponse.MutationChunk.newBuilder() + .setMutation(chunkedCell1) + .setChunkInfo(chunkInfo1)) + .build(); + ReadChangeStreamResponse dataChangeResponse1 = + ReadChangeStreamResponse.newBuilder().setDataChange(dataChange1).build(); + + // Build the 2nd ReadChangeStreamResponse which contains 2/2 of a chunked SetCell, a + // deleteFromFamily, a deleteFromColumn, and a non-chunked SetCell. + Mutation chunkedCell2 = + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setValue(ByteString.copyFromUtf8(secondCellValue)) + .build()) + .build(); + ReadChangeStreamResponse.MutationChunk.ChunkInfo chunkInfo2 = + ReadChangeStreamResponse.MutationChunk.ChunkInfo.newBuilder() + .setChunkedValueOffset(firstCellValue.length()) + .setChunkedValueSize(totalCellValue.length()) + .setLastChunk(true) + .build(); + Mutation deleteFromFamily = + Mutation.newBuilder() + .setDeleteFromFamily( + Mutation.DeleteFromFamily.newBuilder().setFamilyName("fake-family").build()) + .build(); + Mutation deleteFromColumn = + Mutation.newBuilder() + .setDeleteFromColumn( + Mutation.DeleteFromColumn.newBuilder() + .setFamilyName("fake-family") + .setColumnQualifier(ByteString.copyFromUtf8("fake-qualifier")) + .setTimeRange( + TimestampRange.newBuilder() + .setStartTimestampMicros(1000L) + .setEndTimestampMicros(2000L) + .build()) + .build()) + .build(); + Mutation nonChunkedCell = + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setFamilyName("fake-family") + .setColumnQualifier(ByteString.copyFromUtf8("fake-qualifier")) + .setTimestampMicros(1000L) + .setValue(ByteString.copyFromUtf8("fake-value")) + .build()) + .build(); + ReadChangeStreamResponse.DataChange dataChange2 = + ReadChangeStreamResponse.DataChange.newBuilder() + .setType(ReadChangeStreamResponse.DataChange.Type.CONTINUATION) + .addChunks( + ReadChangeStreamResponse.MutationChunk.newBuilder() + .setMutation(chunkedCell2) + .setChunkInfo(chunkInfo2)) + .addChunks( + ReadChangeStreamResponse.MutationChunk.newBuilder().setMutation(deleteFromFamily)) + .addChunks( + ReadChangeStreamResponse.MutationChunk.newBuilder().setMutation(deleteFromColumn)) + .addChunks( + ReadChangeStreamResponse.MutationChunk.newBuilder().setMutation(nonChunkedCell)) + .setDone(true) + .setToken("fake-token") + .setLowWatermark(fakeLowWatermark) + .build(); + ReadChangeStreamResponse dataChangeResponse2 = + ReadChangeStreamResponse.newBuilder().setDataChange(dataChange2).build(); + + // Feed the two ReadChangeStream responses to the merging callable. + FakeStreamingApi.ServerStreamingStashCallable + inner = + new ServerStreamingStashCallable<>( + Arrays.asList(dataChangeResponse1, dataChangeResponse2)); + ChangeStreamRecordMergingCallable mergingCallable = + new ChangeStreamRecordMergingCallable<>(inner, new DefaultChangeStreamRecordAdapter()); + // Actual results. + List results = + mergingCallable.all().call(ReadChangeStreamRequest.getDefaultInstance()); + + // Expected results. + Timestamp fakeCommitTimestamp = Timestamp.newBuilder().setSeconds(100).build(); + ChangeStreamMutation changeStreamMutation = + ChangeStreamMutation.createUserMutation( + ByteString.copyFromUtf8("key"), "fake-source-cluster-id", fakeCommitTimestamp, 100) + .setCell( + "fake-family", + ByteString.copyFromUtf8("fake-qualifier"), + 1000L, + ByteString.copyFromUtf8(totalCellValue)) + .deleteFamily("fake-family") + .deleteCells( + "fake-family", + ByteString.copyFromUtf8("fake-qualifier"), + Range.TimestampRange.create(1000L, 2000L)) + .setCell( + "fake-family", + ByteString.copyFromUtf8("fake-qualifier"), + 1000L, + ByteString.copyFromUtf8("fake-value")) + .setToken("fake-token") + .setLowWatermark(fakeLowWatermark) + .build(); + Truth.assertThat(results).containsExactly(changeStreamMutation); + } + + // [{SetCell1, SetCell2_chunk1}, {SetCell2_chunk2}] + // -> [ChangeStreamMutation{SetCell1, SetCell2}] + @Test + public void oneUnChunkedCell_oneChunkedCellTest() { + Timestamp fakeLowWatermark = Timestamp.newBuilder().setSeconds(100).build(); + Mutation nonChunkedCell = + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setFamilyName("fake-family") + .setColumnQualifier(ByteString.copyFromUtf8("fake-qualifier")) + .setTimestampMicros(1000L) + .setValue(ByteString.copyFromUtf8("fake-value")) + .build()) + .build(); + String firstCellValue = "chunk1-value"; + String secondCellValue = "chunk2-value"; + String totalCellValue = firstCellValue + secondCellValue; + // Build the 1st ReadChangeStreamResponse which contains a nonChunkedCell and 1/2 of the chunked + // SetCell. + Mutation chunkedCell1 = + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setFamilyName("fake-family") + .setColumnQualifier(ByteString.copyFromUtf8("fake-qualifier")) + .setTimestampMicros(1000L) + .setValue(ByteString.copyFromUtf8(firstCellValue)) + .build()) + .build(); + ReadChangeStreamResponse.MutationChunk.ChunkInfo chunkInfo1 = + ReadChangeStreamResponse.MutationChunk.ChunkInfo.newBuilder() + .setChunkedValueSize(totalCellValue.length()) + .build(); + ReadChangeStreamResponse.DataChange dataChange1 = + ReadChangeStreamResponse.DataChange.newBuilder() + .setType(ReadChangeStreamResponse.DataChange.Type.USER) + .setSourceClusterId("fake-source-cluster-id") + .setRowKey(ByteString.copyFromUtf8("key")) + .setCommitTimestamp(Timestamp.newBuilder().setSeconds(100).build()) + .setTiebreaker(100) + .addChunks( + ReadChangeStreamResponse.MutationChunk.newBuilder().setMutation(nonChunkedCell)) + .addChunks( + ReadChangeStreamResponse.MutationChunk.newBuilder() + .setMutation(chunkedCell1) + .setChunkInfo(chunkInfo1)) + .build(); + ReadChangeStreamResponse dataChangeResponse1 = + ReadChangeStreamResponse.newBuilder().setDataChange(dataChange1).build(); + + // Build the 2nd ReadChangeStreamResponse which contains 2/2 of the chunked SetCell. + Mutation chunkedCell2 = + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setValue(ByteString.copyFromUtf8(secondCellValue)) + .build()) + .build(); + ReadChangeStreamResponse.MutationChunk.ChunkInfo chunkInfo2 = + ReadChangeStreamResponse.MutationChunk.ChunkInfo.newBuilder() + .setChunkedValueOffset(firstCellValue.length()) + .setChunkedValueSize(totalCellValue.length()) + .setLastChunk(true) + .build(); + ReadChangeStreamResponse.DataChange dataChange2 = + ReadChangeStreamResponse.DataChange.newBuilder() + .setType(ReadChangeStreamResponse.DataChange.Type.CONTINUATION) + .addChunks( + ReadChangeStreamResponse.MutationChunk.newBuilder() + .setMutation(chunkedCell2) + .setChunkInfo(chunkInfo2)) + .setDone(true) + .setToken("fake-token") + .setLowWatermark(fakeLowWatermark) + .build(); + ReadChangeStreamResponse dataChangeResponse2 = + ReadChangeStreamResponse.newBuilder().setDataChange(dataChange2).build(); + + // Feed the two ReadChangeStream responses to the merging callable. + FakeStreamingApi.ServerStreamingStashCallable + inner = + new ServerStreamingStashCallable<>( + Arrays.asList(dataChangeResponse1, dataChangeResponse2)); + ChangeStreamRecordMergingCallable mergingCallable = + new ChangeStreamRecordMergingCallable<>(inner, new DefaultChangeStreamRecordAdapter()); + // Actual results. + List results = + mergingCallable.all().call(ReadChangeStreamRequest.getDefaultInstance()); + + // Expected results. + Timestamp fakeCommitTimestamp = Timestamp.newBuilder().setSeconds(100).build(); + ChangeStreamMutation changeStreamMutation = + ChangeStreamMutation.createUserMutation( + ByteString.copyFromUtf8("key"), "fake-source-cluster-id", fakeCommitTimestamp, 100) + .setCell( + "fake-family", + ByteString.copyFromUtf8("fake-qualifier"), + 1000L, + ByteString.copyFromUtf8("fake-value")) + .setCell( + "fake-family", + ByteString.copyFromUtf8("fake-qualifier"), + 1000L, + ByteString.copyFromUtf8(totalCellValue)) + .setToken("fake-token") + .setLowWatermark(fakeLowWatermark) + .build(); + Truth.assertThat(results).containsExactly(changeStreamMutation); + } + + // [{DeleteFamily1, DeleteCells}, {DeleteFamily2}] + // -> [ChangeStreamMutation{DeleteFamily1, DeleteCells, DeleteFamily2}] + @Test + public void nonSetCellChunkingTest() { + Timestamp fakeLowWatermark = Timestamp.newBuilder().setSeconds(100).build(); + Mutation deleteFromFamily1 = + Mutation.newBuilder() + .setDeleteFromFamily( + Mutation.DeleteFromFamily.newBuilder().setFamilyName("fake-family").build()) + .build(); + Mutation deleteFromColumn = + Mutation.newBuilder() + .setDeleteFromColumn( + Mutation.DeleteFromColumn.newBuilder() + .setFamilyName("fake-family") + .setColumnQualifier(ByteString.copyFromUtf8("fake-qualifier")) + .setTimeRange( + TimestampRange.newBuilder() + .setStartTimestampMicros(1000L) + .setEndTimestampMicros(2000L) + .build()) + .build()) + .build(); + Mutation deleteFromFamily2 = + Mutation.newBuilder() + .setDeleteFromFamily( + Mutation.DeleteFromFamily.newBuilder().setFamilyName("fake-family2").build()) + .build(); + // Build the 1st ReadChangeStreamResponse which contains deleteFromFamily1 and deleteFromColumn. + ReadChangeStreamResponse.DataChange dataChange1 = + ReadChangeStreamResponse.DataChange.newBuilder() + .setType(ReadChangeStreamResponse.DataChange.Type.USER) + .setSourceClusterId("fake-source-cluster-id") + .setRowKey(ByteString.copyFromUtf8("key")) + .setCommitTimestamp(Timestamp.newBuilder().setSeconds(100).build()) + .setTiebreaker(100) + .addChunks( + ReadChangeStreamResponse.MutationChunk.newBuilder().setMutation(deleteFromFamily1)) + .addChunks( + ReadChangeStreamResponse.MutationChunk.newBuilder().setMutation(deleteFromColumn)) + .build(); + ReadChangeStreamResponse dataChangeResponse1 = + ReadChangeStreamResponse.newBuilder().setDataChange(dataChange1).build(); + + // Build the 2nd ReadChangeStreamResponse which contains deleteFromFamily2. + ReadChangeStreamResponse.DataChange dataChange2 = + ReadChangeStreamResponse.DataChange.newBuilder() + .setType(ReadChangeStreamResponse.DataChange.Type.CONTINUATION) + .addChunks( + ReadChangeStreamResponse.MutationChunk.newBuilder().setMutation(deleteFromFamily2)) + .setDone(true) + .setToken("fake-token") + .setLowWatermark(fakeLowWatermark) + .build(); + ReadChangeStreamResponse dataChangeResponse2 = + ReadChangeStreamResponse.newBuilder().setDataChange(dataChange2).build(); + + // Feed the two ReadChangeStream responses to the merging callable. + FakeStreamingApi.ServerStreamingStashCallable + inner = + new ServerStreamingStashCallable<>( + Arrays.asList(dataChangeResponse1, dataChangeResponse2)); + ChangeStreamRecordMergingCallable mergingCallable = + new ChangeStreamRecordMergingCallable<>(inner, new DefaultChangeStreamRecordAdapter()); + // Actual results. + List results = + mergingCallable.all().call(ReadChangeStreamRequest.getDefaultInstance()); + + // Expected results. + Timestamp fakeCommitTimestamp = Timestamp.newBuilder().setSeconds(100).build(); + ChangeStreamMutation changeStreamMutation = + ChangeStreamMutation.createUserMutation( + ByteString.copyFromUtf8("key"), "fake-source-cluster-id", fakeCommitTimestamp, 100) + .deleteFamily("fake-family") + .deleteCells( + "fake-family", + ByteString.copyFromUtf8("fake-qualifier"), + Range.TimestampRange.create(1000L, 2000L)) + .deleteFamily("fake-family2") + .setToken("fake-token") + .setLowWatermark(fakeLowWatermark) + .build(); + Truth.assertThat(results).containsExactly(changeStreamMutation); + } +} diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java new file mode 100644 index 0000000000..2d7ab394ee --- /dev/null +++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java @@ -0,0 +1,220 @@ +/* + * Copyright 2022 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.changestream; + +import static com.google.common.truth.Truth.assertThat; +import static com.google.common.truth.Truth.assertWithMessage; + +import com.google.api.client.util.Lists; +import com.google.api.gax.rpc.ServerStream; +import com.google.api.gax.rpc.ServerStreamingCallable; +import com.google.bigtable.v2.Mutation; +import com.google.bigtable.v2.ReadChangeStreamRequest; +import com.google.bigtable.v2.ReadChangeStreamResponse; +import com.google.bigtable.v2.TimestampRange; +import com.google.cloud.bigtable.data.v2.models.*; +import com.google.cloud.bigtable.gaxx.testing.FakeStreamingApi; +import com.google.cloud.conformance.bigtable.v2.ChangeStreamTestDefinition.ChangeStreamTestFile; +import com.google.cloud.conformance.bigtable.v2.ChangeStreamTestDefinition.ReadChangeStreamTest; +import com.google.common.base.CaseFormat; +import com.google.protobuf.util.JsonFormat; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +/** Parses and runs the acceptance tests for read rows */ +@RunWith(Parameterized.class) +public class ReadChangeStreamMergingAcceptanceTest { + // Location: `google-cloud-bigtable/src/main/resources/changestream.json` + private static final String TEST_DATA_JSON_RESOURCE = "changestream.json"; + + private final ReadChangeStreamTest testCase; + + /** + * @param testData The serialized test data representing the test case. + * @param junitName Not used by the test, but used by the parameterized test runner as the name of + * the test. + */ + public ReadChangeStreamMergingAcceptanceTest( + ReadChangeStreamTest testData, @SuppressWarnings("unused") String junitName) { + this.testCase = testData; + } + + // Each tuple consists of [testData: ReadChangeStreamTest, junitName: String] + @Parameterized.Parameters(name = "{1}") + public static Collection data() throws IOException { + ClassLoader cl = Thread.currentThread().getContextClassLoader(); + InputStream dataJson = cl.getResourceAsStream(TEST_DATA_JSON_RESOURCE); + assertWithMessage("Unable to load test definition: %s", TEST_DATA_JSON_RESOURCE) + .that(dataJson) + .isNotNull(); + + InputStreamReader reader = new InputStreamReader(dataJson); + ChangeStreamTestFile.Builder testBuilder = ChangeStreamTestFile.newBuilder(); + JsonFormat.parser().merge(reader, testBuilder); + ChangeStreamTestFile testDefinition = testBuilder.build(); + + List tests = testDefinition.getReadChangeStreamTestsList(); + ArrayList data = new ArrayList<>(tests.size()); + for (ReadChangeStreamTest test : tests) { + String junitName = + CaseFormat.LOWER_HYPHEN.to( + CaseFormat.LOWER_CAMEL, test.getDescription().replace(" ", "-")); + data.add(new Object[] {test, junitName}); + } + return data; + } + + @Test + public void test() throws Exception { + List responses = testCase.getApiResponsesList(); + System.out.println("testCase: " + responses); + + // Wrap the responses in a callable. + ServerStreamingCallable source = + new FakeStreamingApi.ServerStreamingStashCallable<>(responses); + ChangeStreamRecordMergingCallable mergingCallable = + new ChangeStreamRecordMergingCallable<>(source, new DefaultChangeStreamRecordAdapter()); + + // Invoke the callable to get the change stream records. + ServerStream stream = + mergingCallable.call(ReadChangeStreamRequest.getDefaultInstance()); + + // Transform the change stream records into ReadChangeStreamTest.Result's. + List actualResults = Lists.newArrayList(); + Exception error = null; + + try { + for (ChangeStreamRecord record : stream) { + if (record instanceof Heartbeat) { + actualResults.add( + ReadChangeStreamTest.Result.newBuilder() + .setRecord( + ReadChangeStreamTest.TestChangeStreamRecord.newBuilder() + .setHeartbeat(((Heartbeat) record).toProto()) + .build()) + .build()); + } else if (record instanceof CloseStream) { + actualResults.add( + ReadChangeStreamTest.Result.newBuilder() + .setRecord( + ReadChangeStreamTest.TestChangeStreamRecord.newBuilder() + .setCloseStream(((CloseStream) record).toProto()) + .build()) + .build()); + } else if (record instanceof ChangeStreamMutation) { + ChangeStreamMutation changeStreamMutation = (ChangeStreamMutation) record; + ReadChangeStreamTest.TestChangeStreamMutation.Builder builder = + ReadChangeStreamTest.TestChangeStreamMutation.newBuilder(); + builder.setRowKey(changeStreamMutation.getRowKey()); + builder.setType(changeStreamMutation.getType()); + if (changeStreamMutation.getSourceClusterId() != null) { + builder.setSourceClusterId(changeStreamMutation.getSourceClusterId()); + } + builder.setCommitTimestamp(changeStreamMutation.getCommitTimestamp()); + builder.setTiebreaker(changeStreamMutation.getTieBreaker()); + builder.setToken(changeStreamMutation.getToken()); + builder.setLowWatermark(changeStreamMutation.getLowWatermark()); + for (Entry entry : changeStreamMutation.getEntries()) { + if (entry instanceof DeleteFamily) { + DeleteFamily deleteFamily = (DeleteFamily) entry; + builder.addMutations( + Mutation.newBuilder() + .setDeleteFromFamily( + Mutation.DeleteFromFamily.newBuilder() + .setFamilyName(deleteFamily.getFamilyName()) + .build())); + } else if (entry instanceof DeleteCells) { + DeleteCells deleteCells = (DeleteCells) entry; + builder.addMutations( + Mutation.newBuilder() + .setDeleteFromColumn( + Mutation.DeleteFromColumn.newBuilder() + .setFamilyName(deleteCells.getFamilyName()) + .setColumnQualifier(deleteCells.getQualifier()) + .setTimeRange( + TimestampRange.newBuilder() + .setStartTimestampMicros( + deleteCells.getTimestampRange().getStart()) + .setEndTimestampMicros( + deleteCells.getTimestampRange().getEnd()) + .build()) + .build())); + } else if (entry instanceof SetCell) { + SetCell setCell = (SetCell) entry; + builder.addMutations( + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setFamilyName(setCell.getFamilyName()) + .setColumnQualifier(setCell.getQualifier()) + .setTimestampMicros(setCell.getTimestamp()) + .setValue(setCell.getValue()))); + } else { + throw new IllegalStateException("Unexpected Entry type"); + } + } + actualResults.add( + ReadChangeStreamTest.Result.newBuilder() + .setRecord( + ReadChangeStreamTest.TestChangeStreamRecord.newBuilder() + .setChangeStreamMutation(builder)) + .build()); + } else { + throw new IllegalStateException("Unexpected ChangeStreamRecord type"); + } + } + } catch (Exception e) { + error = e; + } + + // Verify the results. + if (expectsError(testCase)) { + assertThat(error).isNotNull(); + } else { + if (error != null) { + throw error; + } + } + + assertThat(getNonExceptionResults(testCase)).isEqualTo(actualResults); + } + + private static boolean expectsError(ReadChangeStreamTest testCase) { + List results = testCase.getResultsList(); + return results != null && !results.isEmpty() && results.get(results.size() - 1).getError(); + } + + private static List getNonExceptionResults( + ReadChangeStreamTest testCase) { + List results = testCase.getResultsList(); + List response = new ArrayList<>(); + if (results != null) { + for (ReadChangeStreamTest.Result result : results) { + if (!result.getError()) { + response.add(result); + } + } + } + return response; + } +} diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamUserCallableTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamUserCallableTest.java new file mode 100644 index 0000000000..4043ea78cd --- /dev/null +++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamUserCallableTest.java @@ -0,0 +1,48 @@ +/* + * Copyright 2022 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.changestream; + +import com.google.bigtable.v2.*; +import com.google.cloud.bigtable.data.v2.internal.RequestContext; +import com.google.cloud.bigtable.data.v2.models.ChangeStreamRecord; +import com.google.cloud.bigtable.data.v2.models.ReadChangeStreamQuery; +import com.google.cloud.bigtable.gaxx.testing.FakeStreamingApi.ServerStreamingStashCallable; +import com.google.common.truth.Truth; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class ReadChangeStreamUserCallableTest { + private static final RequestContext REQUEST_CONTEXT = + RequestContext.create("fake-project", "fake-instance", "fake-profile"); + + @Test + public void testRequestIsConverted() { + ServerStreamingStashCallable innerCallable = + new ServerStreamingStashCallable<>(); + ReadChangeStreamUserCallable callable = + new ReadChangeStreamUserCallable<>(innerCallable, REQUEST_CONTEXT); + ReadChangeStreamQuery query = + ReadChangeStreamQuery.create("fake-table") + .streamPartition("begin", "end") + .startTime(com.google.protobuf.Timestamp.newBuilder().setSeconds(1000).build()) + .endTime(com.google.protobuf.Timestamp.newBuilder().setSeconds(2000).build()) + .heartbeatDuration(5L); + callable.call(query); + Truth.assertThat(innerCallable.getActualRequest()).isEqualTo(query.toProto(REQUEST_CONTEXT)); + } +} From c4157d3beb509796b1216a466eb80f37758e1027 Mon Sep 17 00:00:00 2001 From: Teng Zhong Date: Thu, 4 Aug 2022 09:33:00 -0400 Subject: [PATCH 02/10] fix: Fix styles --- .../cloud/bigtable/data/v2/BigtableDataClient.java | 13 ++++++++++++- .../src/main/proto/changestream_tests.proto | 2 +- .../ReadChangeStreamMergingAcceptanceTest.java | 11 +++++++++-- 3 files changed, 22 insertions(+), 4 deletions(-) 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 7ccf0e49fc..ca32c4a6a5 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 @@ -30,8 +30,19 @@ import com.google.api.gax.rpc.ServerStreamingCallable; import com.google.api.gax.rpc.UnaryCallable; import com.google.bigtable.v2.RowRange; -import com.google.cloud.bigtable.data.v2.models.*; +import com.google.cloud.bigtable.data.v2.models.BulkMutation; +import com.google.cloud.bigtable.data.v2.models.ChangeStreamRecord; +import com.google.cloud.bigtable.data.v2.models.ConditionalRowMutation; +import com.google.cloud.bigtable.data.v2.models.Filters; import com.google.cloud.bigtable.data.v2.models.Filters.Filter; +import com.google.cloud.bigtable.data.v2.models.KeyOffset; +import com.google.cloud.bigtable.data.v2.models.Query; +import com.google.cloud.bigtable.data.v2.models.ReadChangeStreamQuery; +import com.google.cloud.bigtable.data.v2.models.ReadModifyWriteRow; +import com.google.cloud.bigtable.data.v2.models.Row; +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.stub.EnhancedBigtableStub; import com.google.common.util.concurrent.MoreExecutors; import com.google.protobuf.ByteString; diff --git a/google-cloud-bigtable/src/main/proto/changestream_tests.proto b/google-cloud-bigtable/src/main/proto/changestream_tests.proto index 4cf3825e20..779cdfba35 100644 --- a/google-cloud-bigtable/src/main/proto/changestream_tests.proto +++ b/google-cloud-bigtable/src/main/proto/changestream_tests.proto @@ -60,4 +60,4 @@ message ReadChangeStreamTest { string description = 1; repeated google.bigtable.v2.ReadChangeStreamResponse api_responses = 2; repeated Result results = 3; -} \ No newline at end of file +} diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java index 2d7ab394ee..78745bc484 100644 --- a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java +++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java @@ -25,7 +25,15 @@ import com.google.bigtable.v2.ReadChangeStreamRequest; import com.google.bigtable.v2.ReadChangeStreamResponse; import com.google.bigtable.v2.TimestampRange; -import com.google.cloud.bigtable.data.v2.models.*; +import com.google.cloud.bigtable.data.v2.models.ChangeStreamMutation; +import com.google.cloud.bigtable.data.v2.models.ChangeStreamRecord; +import com.google.cloud.bigtable.data.v2.models.CloseStream; +import com.google.cloud.bigtable.data.v2.models.DefaultChangeStreamRecordAdapter; +import com.google.cloud.bigtable.data.v2.models.DeleteCells; +import com.google.cloud.bigtable.data.v2.models.DeleteFamily; +import com.google.cloud.bigtable.data.v2.models.Entry; +import com.google.cloud.bigtable.data.v2.models.Heartbeat; +import com.google.cloud.bigtable.data.v2.models.SetCell; import com.google.cloud.bigtable.gaxx.testing.FakeStreamingApi; import com.google.cloud.conformance.bigtable.v2.ChangeStreamTestDefinition.ChangeStreamTestFile; import com.google.cloud.conformance.bigtable.v2.ChangeStreamTestDefinition.ReadChangeStreamTest; @@ -102,7 +110,6 @@ public void test() throws Exception { // Transform the change stream records into ReadChangeStreamTest.Result's. List actualResults = Lists.newArrayList(); Exception error = null; - try { for (ChangeStreamRecord record : stream) { if (record instanceof Heartbeat) { From 30e3750be80a6018491270ff482c27928b68a932 Mon Sep 17 00:00:00 2001 From: Teng Zhong Date: Thu, 4 Aug 2022 12:57:59 -0400 Subject: [PATCH 03/10] fix: Make some methods package private Remove all the mutation related tests in ChangeStreamRecordMergingCallableTest. Just use the ReadChangeStreamMergingAcceptanceTest. --- .../data/v2/models/ChangeStreamMutation.java | 28 +- .../bigtable/data/v2/models/CloseStream.java | 15 +- .../bigtable/data/v2/models/Heartbeat.java | 13 +- .../src/main/resources/changestream.json | 195 ++++- .../v2/models/ChangeStreamRecordTest.java | 4 - ...ChangeStreamRecordMergingCallableTest.java | 779 ++---------------- ...ReadChangeStreamMergingAcceptanceTest.java | 46 +- 7 files changed, 278 insertions(+), 802 deletions(-) diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/ChangeStreamMutation.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/ChangeStreamMutation.java index dfe33c68dd..db0891fda1 100644 --- a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/ChangeStreamMutation.java +++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/ChangeStreamMutation.java @@ -15,11 +15,9 @@ */ package com.google.cloud.bigtable.data.v2.models; -import com.google.api.core.InternalApi; import com.google.bigtable.v2.ReadChangeStreamResponse.DataChange.Type; import com.google.cloud.bigtable.data.v2.models.Range.TimestampRange; import com.google.cloud.bigtable.data.v2.stub.changestream.ChangeStreamRecordMerger; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; import com.google.common.base.Objects; import com.google.common.base.Preconditions; @@ -101,9 +99,7 @@ private ChangeStreamMutation(Builder builder) { * ChangeStreamMutation because `token` and `loWatermark` must be set later when we finish * building the logical mutation. */ - @InternalApi("Used in java veneer client.") - @VisibleForTesting - public static Builder createUserMutation( + static Builder createUserMutation( @Nonnull ByteString rowKey, @Nonnull String sourceClusterId, @Nonnull Timestamp commitTimestamp, @@ -116,9 +112,7 @@ public static Builder createUserMutation( * because `token` and `loWatermark` must be set later when we finish building the logical * mutation. */ - @InternalApi("Used in java veneer client.") - @VisibleForTesting - public static Builder createGcMutation( + static Builder createGcMutation( @Nonnull ByteString rowKey, @Nonnull Timestamp commitTimestamp, int tieBreaker) { return new Builder(rowKey, Type.GARBAGE_COLLECTION, null, commitTimestamp, tieBreaker); } @@ -230,9 +224,7 @@ private Builder(ChangeStreamMutation changeStreamMutation) { this.lowWatermark = changeStreamMutation.lowWatermark; } - @InternalApi("Used in java veneer client.") - @VisibleForTesting - public Builder setCell( + Builder setCell( @Nonnull String familyName, @Nonnull ByteString qualifier, long timestamp, @@ -241,9 +233,7 @@ public Builder setCell( return this; } - @InternalApi("Used in java veneer client.") - @VisibleForTesting - public Builder deleteCells( + Builder deleteCells( @Nonnull String familyName, @Nonnull ByteString qualifier, @Nonnull TimestampRange timestampRange) { @@ -251,24 +241,22 @@ public Builder deleteCells( return this; } - @InternalApi("Used in java veneer client.") - @VisibleForTesting - public Builder deleteFamily(@Nonnull String familyName) { + Builder deleteFamily(@Nonnull String familyName) { this.entries.add(DeleteFamily.create(familyName)); return this; } - public Builder setToken(@Nonnull String token) { + Builder setToken(@Nonnull String token) { this.token = token; return this; } - public Builder setLowWatermark(@Nonnull Timestamp lowWatermark) { + Builder setLowWatermark(@Nonnull Timestamp lowWatermark) { this.lowWatermark = lowWatermark; return this; } - public ChangeStreamMutation build() { + ChangeStreamMutation build() { Preconditions.checkArgument( token != null && lowWatermark != null, "ChangeStreamMutation must have a continuation token and low watermark."); diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/CloseStream.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/CloseStream.java index 533e0b0721..403705f676 100644 --- a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/CloseStream.java +++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/CloseStream.java @@ -18,7 +18,6 @@ import com.google.api.core.InternalApi; import com.google.bigtable.v2.ReadChangeStreamResponse; import com.google.bigtable.v2.StreamContinuationToken; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; import com.google.common.base.Objects; import com.google.common.collect.ImmutableList; @@ -70,22 +69,10 @@ private void writeObject(ObjectOutputStream output) throws IOException { } /** Wraps the protobuf {@link ReadChangeStreamResponse.CloseStream}. */ - @InternalApi("Used in java veneer client.") - @VisibleForTesting - public static CloseStream fromProto(@Nonnull ReadChangeStreamResponse.CloseStream closeStream) { + static CloseStream fromProto(@Nonnull ReadChangeStreamResponse.CloseStream closeStream) { return new CloseStream(closeStream.getStatus(), closeStream.getContinuationTokensList()); } - @VisibleForTesting - public ReadChangeStreamResponse.CloseStream toProto() { - ReadChangeStreamResponse.CloseStream.Builder builder = - ReadChangeStreamResponse.CloseStream.newBuilder().setStatus(getStatus()); - for (ChangeStreamContinuationToken token : getChangeStreamContinuationTokens()) { - builder.addContinuationTokens(token.toProto()); - } - return builder.build(); - } - @Override public boolean equals(Object o) { if (this == o) { diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/Heartbeat.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/Heartbeat.java index c85a5d8631..f2371c8507 100644 --- a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/Heartbeat.java +++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/models/Heartbeat.java @@ -18,7 +18,6 @@ import com.google.api.core.InternalApi; import com.google.auto.value.AutoValue; import com.google.bigtable.v2.ReadChangeStreamResponse; -import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.Timestamp; import java.io.Serializable; import javax.annotation.Nonnull; @@ -33,22 +32,12 @@ private static Heartbeat create( } /** Wraps the protobuf {@link ReadChangeStreamResponse.Heartbeat}. */ - @InternalApi("Used in java veneer client.") - @VisibleForTesting - public static Heartbeat fromProto(@Nonnull ReadChangeStreamResponse.Heartbeat heartbeat) { + static Heartbeat fromProto(@Nonnull ReadChangeStreamResponse.Heartbeat heartbeat) { return create( ChangeStreamContinuationToken.fromProto(heartbeat.getContinuationToken()), heartbeat.getLowWatermark()); } - @VisibleForTesting - public ReadChangeStreamResponse.Heartbeat toProto() { - return ReadChangeStreamResponse.Heartbeat.newBuilder() - .setContinuationToken(getChangeStreamContinuationToken().toProto()) - .setLowWatermark(getLowWatermark()) - .build(); - } - @InternalApi("Used in Changestream beam pipeline.") public abstract ChangeStreamContinuationToken getChangeStreamContinuationToken(); diff --git a/google-cloud-bigtable/src/main/resources/changestream.json b/google-cloud-bigtable/src/main/resources/changestream.json index 608dc33074..b48056d402 100644 --- a/google-cloud-bigtable/src/main/resources/changestream.json +++ b/google-cloud-bigtable/src/main/resources/changestream.json @@ -184,7 +184,7 @@ ] }, { - "description": "1 logical mutation no chunking", + "description": "1 logical mutation no chunking([{DF,DC,SC}]->ChangeStreamMutation{DF,DC,SC})", "api_responses": [ { "data_change": { @@ -202,6 +202,28 @@ "family_name": "family" } } + }, + { + "mutation": { + "delete_from_column" : { + "family_name": "family", + "column_qualifier": "dg==", + "time_range": { + "start_timestamp_micros": 5000, + "end_timestamp_micros": 15000 + } + } + } + }, + { + "mutation": { + "set_cell": { + "family_name": "family", + "column_qualifier": "0000000000000000", + "timestamp_micros": 1000, + "value": "dg==" + } + } } ], "done": true @@ -224,6 +246,24 @@ "delete_from_family": { "family_name": "family" } + }, + { + "delete_from_column" : { + "family_name": "family", + "column_qualifier": "dg==", + "time_range": { + "start_timestamp_micros": 5000, + "end_timestamp_micros": 15000 + } + } + }, + { + "set_cell": { + "family_name": "family", + "column_qualifier": "0000000000000000", + "timestamp_micros": 1000, + "value": "dg==" + } } ] } @@ -309,7 +349,7 @@ ] }, { - "description": "1 chunked SetCell", + "description": "1 chunked SetCell([{SC_chunk1}, {SC_chunk2}]->ChangeStreamMutation{SC})", "api_responses": [ { "data_change": { @@ -386,7 +426,7 @@ ] }, { - "description": "2 chunked SetCells", + "description": "2 chunked SetCells([{SC1_chunk1}, {SC1_chunk2, SC2_chunk1}, {SC2_chunk2}]->ChangeStreamMutation{SC1,SC2})", "api_responses": [ { "data_change": { @@ -503,7 +543,7 @@ ] }, { - "description": "1 chunked SetCell + 1 unchunked SetCell", + "description": "1 chunked SetCell + 1 unchunked SetCell([{SC1_chunk1}, {SC1_chunk2, SC2}]->ChangeStreamMutation{SC1,SC2})", "api_responses": [ { "data_change": { @@ -598,7 +638,7 @@ ] }, { - "description": "1 unchunked SetCell + 1 chunked SetCell", + "description": "1 unchunked SetCell + 1 chunked SetCell([{SC1, SC2_chunk1}, {SC2_chunk2}]->ChangeStreamMutation{SC1,SC2})", "api_responses": [ { "data_change": { @@ -693,7 +733,7 @@ ] }, { - "description": "1 mod + 1 chunked SetCell + 1 mod", + "description": "1 mod + 1 chunked SetCell + 1 mod([{DF1,SC_chunk1}, {SC_chunk2, DF2}]->ChangeStreamMutation{DF1,SC,DF2})", "api_responses": [ { "data_change": { @@ -794,7 +834,124 @@ ] }, { - "description": "non SetCell chunking", + "description": "1 chunked SetCell + many nonchunked mods([{SC_chunk1}, {SC_chunk2,DF,DC}]->ChangeStreamMutation{SC,DF,DC})", + "api_responses": [ + { + "data_change": { + "row_key": "0000000000000000", + "type": "USER", + "source_cluster_id": "source-cluster-id", + "commit_timestamp": "2022-07-01T00:00:00Z", + "tiebreaker": 100, + "chunks": [ + { + "chunk_info": { + "chunked_value_size": 1 + }, + "mutation": { + "set_cell": { + "family_name": "family", + "column_qualifier": "0000000000000000", + "timestamp_micros": 1000, + "value": "dg==" + } + } + } + ] + } + }, + { + "data_change": { + "type": "CONTINUATION", + "token": "data-change-token", + "low_watermark": "2022-07-01T00:00:00Z", + "chunks": [ + { + "chunk_info": { + "chunked_value_offset": 1, + "chunked_value_size": 1, + "last_chunk": true + }, + "mutation": { + "set_cell": { + "value": "" + } + } + }, + { + "mutation": { + "delete_from_column" : { + "family_name": "family", + "column_qualifier": "dg==", + "time_range": { + "start_timestamp_micros": 5000, + "end_timestamp_micros": 15000 + } + } + } + }, + { + "mutation": { + "set_cell": { + "family_name": "family", + "column_qualifier": "0000000000000000", + "timestamp_micros": 1000, + "value": "dg==" + } + } + } + ], + "done": true + } + } + ], + "results": [ + { + "record": { + "change_stream_mutation": { + "row_key": "0000000000000000", + "type": "USER", + "source_cluster_id": "source-cluster-id", + "commit_timestamp": "2022-07-01T00:00:00Z", + "tiebreaker": 100, + "token": "data-change-token", + "low_watermark": "2022-07-01T00:00:00Z", + "mutations": [ + { + "set_cell": { + "family_name": "family", + "column_qualifier": "0000000000000000", + "timestamp_micros": 1000, + "value": "dg==" + } + }, + { + "delete_from_column" : { + "family_name": "family", + "column_qualifier": "dg==", + "time_range": { + "start_timestamp_micros": 5000, + "end_timestamp_micros": 15000 + } + } + }, + { + "set_cell": { + "family_name": "family", + "column_qualifier": "0000000000000000", + "timestamp_micros": 1000, + "value": "dg==" + } + } + ] + } + }, + "error": false + } + ] + }, + { + "description": "non SetCell chunking([{DF1},{DF2,DC}]->ChangeStreamMutation{DF1,DF2,DC})", "api_responses": [ { "data_change": { @@ -826,6 +983,18 @@ "family_name": "family" } } + }, + { + "mutation": { + "delete_from_column" : { + "family_name": "family", + "column_qualifier": "dg==", + "time_range": { + "start_timestamp_micros": 5000, + "end_timestamp_micros": 15000 + } + } + } } ], "done": true @@ -853,6 +1022,16 @@ "delete_from_family": { "family_name": "family" } + }, + { + "delete_from_column" : { + "family_name": "family", + "column_qualifier": "dg==", + "time_range": { + "start_timestamp_micros": 5000, + "end_timestamp_micros": 15000 + } + } } ] } @@ -862,7 +1041,7 @@ ] }, { - "description": "2 logical mutations with non SetCell chunking + CloseStream", + "description": "2 logical mutations with non SetCell chunking + CloseStream([{Change1_DF1}, {Change1_DF2}, {Change2_DF3}, {Change2_DF4}, {CloseStream}]->[ChangeStreamMutation1{DF1,DF2}),ChangeStreamMutation2{DF3,DF4}),CloseStream]", "api_responses": [ { "data_change": { diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/models/ChangeStreamRecordTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/models/ChangeStreamRecordTest.java index b819b4c4ac..05df603959 100644 --- a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/models/ChangeStreamRecordTest.java +++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/models/ChangeStreamRecordTest.java @@ -121,8 +121,6 @@ public void heartbeatTest() { Assert.assertEquals(actualHeartbeat.getLowWatermark(), lowWatermark); Assert.assertEquals(actualHeartbeat.getChangeStreamContinuationToken().getRowRange(), rowRange); Assert.assertEquals(actualHeartbeat.getChangeStreamContinuationToken().getToken(), token); - - Assert.assertEquals(actualHeartbeat.toProto(), heartbeatProto); } @Test @@ -165,7 +163,5 @@ public void closeStreamTest() { rowRange2, actualCloseStream.getChangeStreamContinuationTokens().get(1).getRowRange()); Assert.assertEquals( token2, actualCloseStream.getChangeStreamContinuationTokens().get(1).getToken()); - - Assert.assertEquals(actualCloseStream.toProto(), closeStreamProto); } } diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamRecordMergingCallableTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamRecordMergingCallableTest.java index de5a9e8213..d23eb64765 100644 --- a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamRecordMergingCallableTest.java +++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamRecordMergingCallableTest.java @@ -15,48 +15,46 @@ */ package com.google.cloud.bigtable.data.v2.stub.changestream; -import com.google.bigtable.v2.Mutation; import com.google.bigtable.v2.ReadChangeStreamRequest; import com.google.bigtable.v2.ReadChangeStreamResponse; +import com.google.bigtable.v2.RowRange; import com.google.bigtable.v2.StreamContinuationToken; -import com.google.bigtable.v2.TimestampRange; -import com.google.cloud.bigtable.data.v2.models.ChangeStreamMutation; +import com.google.bigtable.v2.StreamPartition; +import com.google.cloud.bigtable.data.v2.models.ChangeStreamContinuationToken; import com.google.cloud.bigtable.data.v2.models.ChangeStreamRecord; import com.google.cloud.bigtable.data.v2.models.CloseStream; import com.google.cloud.bigtable.data.v2.models.DefaultChangeStreamRecordAdapter; import com.google.cloud.bigtable.data.v2.models.Heartbeat; -import com.google.cloud.bigtable.data.v2.models.Range; import com.google.cloud.bigtable.gaxx.testing.FakeStreamingApi; import com.google.cloud.bigtable.gaxx.testing.FakeStreamingApi.ServerStreamingStashCallable; -import com.google.common.truth.Truth; import com.google.protobuf.ByteString; import com.google.protobuf.Timestamp; import com.google.rpc.Status; -import java.util.Arrays; import java.util.Collections; import java.util.List; +import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; /** - * Additional tests in addition to {@link - * com.google.cloud.bigtable.data.v2.stub.changestream.ReadChangeStreamMergingAcceptanceTest}. At - * some point they should be reintegrated into the json file. + * Additional tests in addition to {@link ReadChangeStreamMergingAcceptanceTest}. + * + *

All the ChangeStreamMutation tests are in {@link ReadChangeStreamMergingAcceptanceTest}. */ @RunWith(JUnit4.class) public class ChangeStreamRecordMergingCallableTest { @Test public void heartbeatTest() { - ReadChangeStreamResponse.Heartbeat heartbeat = + ReadChangeStreamResponse.Heartbeat heartbeatProto = ReadChangeStreamResponse.Heartbeat.newBuilder() .setLowWatermark(Timestamp.newBuilder().setSeconds(1000).build()) .setContinuationToken( StreamContinuationToken.newBuilder().setToken("random-token").build()) .build(); ReadChangeStreamResponse response = - ReadChangeStreamResponse.newBuilder().setHeartbeat(heartbeat).build(); + ReadChangeStreamResponse.newBuilder().setHeartbeat(heartbeatProto).build(); FakeStreamingApi.ServerStreamingStashCallable inner = new ServerStreamingStashCallable<>(Collections.singletonList(response)); @@ -65,19 +63,41 @@ public void heartbeatTest() { List results = mergingCallable.all().call(ReadChangeStreamRequest.getDefaultInstance()); - Truth.assertThat(results).containsExactly(Heartbeat.fromProto(heartbeat)); + // Validate the result. + Assert.assertEquals(results.size(), 1); + ChangeStreamRecord record = results.get(0); + Assert.assertTrue(record instanceof Heartbeat); + Heartbeat heartbeat = (Heartbeat) record; + Assert.assertEquals( + heartbeat.getChangeStreamContinuationToken().getRowRange(), + heartbeatProto.getContinuationToken().getPartition().getRowRange()); + Assert.assertEquals( + heartbeat.getChangeStreamContinuationToken().getToken(), + heartbeatProto.getContinuationToken().getToken()); + Assert.assertEquals(heartbeat.getLowWatermark(), heartbeatProto.getLowWatermark()); } @Test public void closeStreamTest() { - ReadChangeStreamResponse.CloseStream closeStream = + StreamContinuationToken streamContinuationToken = + StreamContinuationToken.newBuilder() + .setPartition( + StreamPartition.newBuilder() + .setRowRange( + RowRange.newBuilder() + .setStartKeyClosed(ByteString.copyFromUtf8("")) + .setEndKeyOpen(ByteString.copyFromUtf8("")) + .build()) + .build()) + .setToken("random-token") + .build(); + ReadChangeStreamResponse.CloseStream closeStreamProto = ReadChangeStreamResponse.CloseStream.newBuilder() - .addContinuationTokens( - StreamContinuationToken.newBuilder().setToken("random-token").build()) + .addContinuationTokens(streamContinuationToken) .setStatus(Status.newBuilder().setCode(0).build()) .build(); ReadChangeStreamResponse response = - ReadChangeStreamResponse.newBuilder().setCloseStream(closeStream).build(); + ReadChangeStreamResponse.newBuilder().setCloseStream(closeStreamProto).build(); FakeStreamingApi.ServerStreamingStashCallable inner = new ServerStreamingStashCallable<>(Collections.singletonList(response)); @@ -86,718 +106,19 @@ public void closeStreamTest() { List results = mergingCallable.all().call(ReadChangeStreamRequest.getDefaultInstance()); - Truth.assertThat(results).containsExactly(CloseStream.fromProto(closeStream)); - } - - // [{DeleteFamily, DeleteCells, SetCell}, {CloseStream}] - // -> [ChangeStreamMutation{DeleteFamily, DeleteCells, SetCell}, CloseStream] - @Test - public void multipleModesNoChunkingTest() { - // Construct the ReadChangeStreamResponse's. - Timestamp fakeLowWatermark = Timestamp.newBuilder().setSeconds(100).build(); - Mutation deleteFromFamily = - Mutation.newBuilder() - .setDeleteFromFamily( - Mutation.DeleteFromFamily.newBuilder().setFamilyName("fake-family").build()) - .build(); - Mutation deleteFromColumn = - Mutation.newBuilder() - .setDeleteFromColumn( - Mutation.DeleteFromColumn.newBuilder() - .setFamilyName("fake-family") - .setColumnQualifier(ByteString.copyFromUtf8("fake-qualifier")) - .setTimeRange( - TimestampRange.newBuilder() - .setStartTimestampMicros(1000L) - .setEndTimestampMicros(2000L) - .build()) - .build()) - .build(); - Mutation setCell = - Mutation.newBuilder() - .setSetCell( - Mutation.SetCell.newBuilder() - .setFamilyName("fake-family") - .setColumnQualifier(ByteString.copyFromUtf8("fake-qualifier")) - .setTimestampMicros(1000L) - .setValue(ByteString.copyFromUtf8("fake-value")) - .build()) - .build(); - ReadChangeStreamResponse.DataChange dataChange = - ReadChangeStreamResponse.DataChange.newBuilder() - .setType(ReadChangeStreamResponse.DataChange.Type.USER) - .setSourceClusterId("fake-source-cluster-id") - .setRowKey(ByteString.copyFromUtf8("key")) - .setCommitTimestamp(Timestamp.newBuilder().setSeconds(100).build()) - .setTiebreaker(100) - .setLowWatermark(fakeLowWatermark) - .setToken("fake-token") - .addChunks( - ReadChangeStreamResponse.MutationChunk.newBuilder().setMutation(deleteFromFamily)) - .addChunks( - ReadChangeStreamResponse.MutationChunk.newBuilder().setMutation(deleteFromColumn)) - .addChunks(ReadChangeStreamResponse.MutationChunk.newBuilder().setMutation(setCell)) - .setDone(true) - .build(); - ReadChangeStreamResponse dataChangeResponse = - ReadChangeStreamResponse.newBuilder().setDataChange(dataChange).build(); - ReadChangeStreamResponse closeStreamResponse = - ReadChangeStreamResponse.newBuilder() - .setCloseStream( - ReadChangeStreamResponse.CloseStream.newBuilder() - .addContinuationTokens( - StreamContinuationToken.newBuilder().setToken("random-token").build()) - .setStatus(Status.newBuilder().setCode(0).build()) - .build()) - .build(); - FakeStreamingApi.ServerStreamingStashCallable - inner = - new ServerStreamingStashCallable<>( - Arrays.asList(dataChangeResponse, closeStreamResponse)); - ChangeStreamRecordMergingCallable mergingCallable = - new ChangeStreamRecordMergingCallable<>(inner, new DefaultChangeStreamRecordAdapter()); - - // Actual results. - List results = - mergingCallable.all().call(ReadChangeStreamRequest.getDefaultInstance()); - - // Expected results. - Timestamp fakeCommitTimestamp = Timestamp.newBuilder().setSeconds(100).build(); - ChangeStreamMutation changeStreamMutation = - ChangeStreamMutation.createUserMutation( - ByteString.copyFromUtf8("key"), "fake-source-cluster-id", fakeCommitTimestamp, 100) - .deleteFamily("fake-family") - .deleteCells( - "fake-family", - ByteString.copyFromUtf8("fake-qualifier"), - Range.TimestampRange.create(1000L, 2000L)) - .setCell( - "fake-family", - ByteString.copyFromUtf8("fake-qualifier"), - 1000L, - ByteString.copyFromUtf8("fake-value")) - .setToken("fake-token") - .setLowWatermark(fakeLowWatermark) - .build(); - Truth.assertThat(results) - .containsExactly( - changeStreamMutation, CloseStream.fromProto(closeStreamResponse.getCloseStream())); - } - - // [{DeleteFamily}] -> [ChangeStreamMutation{DeleteFamily}]. - // Tests that the resulting ChangeStreamMutation for a Garbage Collection mutation - // doesn't have source cluster id. - @Test - public void GcModNoChunkingTest() { - Mutation deleteFromFamily = - Mutation.newBuilder() - .setDeleteFromFamily( - Mutation.DeleteFromFamily.newBuilder().setFamilyName("fake-family").build()) - .build(); - Timestamp fakeLowWatermark = Timestamp.newBuilder().setSeconds(100).build(); - ReadChangeStreamResponse.DataChange dataChange = - ReadChangeStreamResponse.DataChange.newBuilder() - .setType(ReadChangeStreamResponse.DataChange.Type.GARBAGE_COLLECTION) - .setRowKey(ByteString.copyFromUtf8("key")) - .setCommitTimestamp(Timestamp.newBuilder().setSeconds(100).build()) - .setTiebreaker(100) - .setLowWatermark(fakeLowWatermark) - .setToken("fake-token") - .addChunks( - ReadChangeStreamResponse.MutationChunk.newBuilder().setMutation(deleteFromFamily)) - .setDone(true) - .build(); - ReadChangeStreamResponse dataChangeResponse = - ReadChangeStreamResponse.newBuilder().setDataChange(dataChange).build(); - FakeStreamingApi.ServerStreamingStashCallable - inner = new ServerStreamingStashCallable<>(Collections.singletonList(dataChangeResponse)); - ChangeStreamRecordMergingCallable mergingCallable = - new ChangeStreamRecordMergingCallable<>(inner, new DefaultChangeStreamRecordAdapter()); - - // Actual results. - List results = - mergingCallable.all().call(ReadChangeStreamRequest.getDefaultInstance()); - - // Expected results. Note that it shouldn't have source cluster id. - Timestamp fakeCommitTimestamp = Timestamp.newBuilder().setSeconds(100).build(); - ChangeStreamMutation changeStreamMutation = - ChangeStreamMutation.createGcMutation( - ByteString.copyFromUtf8("key"), fakeCommitTimestamp, 100) - .deleteFamily("fake-family") - .setToken("fake-token") - .setLowWatermark(fakeLowWatermark) - .build(); - Truth.assertThat(results).containsExactly(changeStreamMutation); - } - - // [{SetCell_chunk1}, {SetCell_chunk2}] -> [ChangeStreamMutation{SetCell}] - @Test - public void oneChunkedCellTest() { - String firstCellValue = "chunk1-value"; - String secondCellValue = "chunk2-value"; - String totalCellValue = firstCellValue + secondCellValue; - Timestamp fakeLowWatermark = Timestamp.newBuilder().setSeconds(100).build(); - // Build the first ReadChangeStreamResponse which contains the first chunk of a chunked SetCell. - Mutation chunkedCell1 = - Mutation.newBuilder() - .setSetCell( - Mutation.SetCell.newBuilder() - .setFamilyName("fake-family") - .setColumnQualifier(ByteString.copyFromUtf8("fake-qualifier")) - .setTimestampMicros(1000L) - .setValue(ByteString.copyFromUtf8(firstCellValue)) - .build()) - .build(); - ReadChangeStreamResponse.MutationChunk.ChunkInfo chunkInfo1 = - ReadChangeStreamResponse.MutationChunk.ChunkInfo.newBuilder() - .setChunkedValueSize(totalCellValue.length()) - .build(); - ReadChangeStreamResponse.DataChange dataChange1 = - ReadChangeStreamResponse.DataChange.newBuilder() - .setType(ReadChangeStreamResponse.DataChange.Type.USER) - .setSourceClusterId("fake-source-cluster-id") - .setRowKey(ByteString.copyFromUtf8("key")) - .setCommitTimestamp(Timestamp.newBuilder().setSeconds(100).build()) - .setTiebreaker(100) - .addChunks( - ReadChangeStreamResponse.MutationChunk.newBuilder() - .setMutation(chunkedCell1) - .setChunkInfo(chunkInfo1)) - .build(); - ReadChangeStreamResponse dataChangeResponse1 = - ReadChangeStreamResponse.newBuilder().setDataChange(dataChange1).build(); - - // Build the second ReadChangeStreamResponse which contains the second chunk of a chunked - // SetCell. - Mutation chunkedCell2 = - Mutation.newBuilder() - .setSetCell( - Mutation.SetCell.newBuilder() - .setValue(ByteString.copyFromUtf8(secondCellValue)) - .build()) - .build(); - ReadChangeStreamResponse.MutationChunk.ChunkInfo chunkInfo2 = - ReadChangeStreamResponse.MutationChunk.ChunkInfo.newBuilder() - .setChunkedValueOffset(firstCellValue.length()) - .setChunkedValueSize(totalCellValue.length()) - .setLastChunk(true) - .build(); - ReadChangeStreamResponse.DataChange dataChange2 = - ReadChangeStreamResponse.DataChange.newBuilder() - .setType(ReadChangeStreamResponse.DataChange.Type.CONTINUATION) - .addChunks( - ReadChangeStreamResponse.MutationChunk.newBuilder() - .setMutation(chunkedCell2) - .setChunkInfo(chunkInfo2)) - .setDone(true) - .setToken("fake-token") - .setLowWatermark(fakeLowWatermark) - .build(); - ReadChangeStreamResponse dataChangeResponse2 = - ReadChangeStreamResponse.newBuilder().setDataChange(dataChange2).build(); - - // Feed the two ReadChangeStream responses to the merging callable. - FakeStreamingApi.ServerStreamingStashCallable - inner = - new ServerStreamingStashCallable<>( - Arrays.asList(dataChangeResponse1, dataChangeResponse2)); - ChangeStreamRecordMergingCallable mergingCallable = - new ChangeStreamRecordMergingCallable<>(inner, new DefaultChangeStreamRecordAdapter()); - // Actual results. - List results = - mergingCallable.all().call(ReadChangeStreamRequest.getDefaultInstance()); - - // Expected results. - Timestamp fakeCommitTimestamp = Timestamp.newBuilder().setSeconds(100).build(); - ChangeStreamMutation changeStreamMutation = - ChangeStreamMutation.createUserMutation( - ByteString.copyFromUtf8("key"), "fake-source-cluster-id", fakeCommitTimestamp, 100) - .setCell( - "fake-family", - ByteString.copyFromUtf8("fake-qualifier"), - 1000L, - ByteString.copyFromUtf8(totalCellValue)) - .setToken("fake-token") - .setLowWatermark(fakeLowWatermark) - .build(); - Truth.assertThat(results).containsExactly(changeStreamMutation); - } - - // [{SetCell1_chunk1}, {SetCell1_chunk2, SetCell2_chunk1}, {SetCell2_chunk2}] - // -> [ChangeStreamMutation{SetCell1, SetCell2}] - @Test - public void twoChunkedCellsTest() { - Timestamp fakeLowWatermark = Timestamp.newBuilder().setSeconds(100).build(); - String setCell1Chunk1val = "cell1-chunk1"; - String setCell1Chunk2val = "cell1-chunk2"; - String setCell2Chunk1val = "cell2-chunk1"; - String setCell2Chunk2val = "cell2-chunk2"; - String totalCellValue1 = setCell1Chunk1val + setCell1Chunk2val; - String totalCellValue2 = setCell2Chunk1val + setCell2Chunk2val; - // Build the 1st ReadChangeStreamResponse which has 1/2 of SetCell1. - Mutation setCell1Chunk1 = - Mutation.newBuilder() - .setSetCell( - Mutation.SetCell.newBuilder() - .setFamilyName("fake-family") - .setColumnQualifier(ByteString.copyFromUtf8("fake-qualifier")) - .setTimestampMicros(1000L) - .setValue(ByteString.copyFromUtf8(setCell1Chunk1val)) - .build()) - .build(); - ReadChangeStreamResponse.MutationChunk.ChunkInfo setCell1ChunkInfo1 = - ReadChangeStreamResponse.MutationChunk.ChunkInfo.newBuilder() - .setChunkedValueSize(totalCellValue1.length()) - .build(); - ReadChangeStreamResponse.DataChange dataChange1 = - ReadChangeStreamResponse.DataChange.newBuilder() - .setType(ReadChangeStreamResponse.DataChange.Type.USER) - .setSourceClusterId("fake-source-cluster-id") - .setRowKey(ByteString.copyFromUtf8("key")) - .setCommitTimestamp(Timestamp.newBuilder().setSeconds(100).build()) - .setTiebreaker(100) - .addChunks( - ReadChangeStreamResponse.MutationChunk.newBuilder() - .setMutation(setCell1Chunk1) - .setChunkInfo(setCell1ChunkInfo1)) - .build(); - ReadChangeStreamResponse dataChangeResponse1 = - ReadChangeStreamResponse.newBuilder().setDataChange(dataChange1).build(); - - // Build the 2nd ReadChangeStreamResponse which has 2/2 of SetCell1 and 1/2 of SetCell2. - Mutation setCell1Chunk2 = - Mutation.newBuilder() - .setSetCell( - Mutation.SetCell.newBuilder() - .setValue(ByteString.copyFromUtf8(setCell1Chunk2val)) - .build()) - .build(); - ReadChangeStreamResponse.MutationChunk.ChunkInfo setCell1ChunkInfo2 = - ReadChangeStreamResponse.MutationChunk.ChunkInfo.newBuilder() - .setChunkedValueOffset(setCell1Chunk1val.length()) - .setChunkedValueSize(totalCellValue1.length()) - .setLastChunk(true) - .build(); - Mutation setCell2Chunk1 = - Mutation.newBuilder() - .setSetCell( - Mutation.SetCell.newBuilder() - .setFamilyName("fake-family") - .setColumnQualifier(ByteString.copyFromUtf8("fake-qualifier")) - .setTimestampMicros(1000L) - .setValue(ByteString.copyFromUtf8(setCell2Chunk1val)) - .build()) - .build(); - ReadChangeStreamResponse.MutationChunk.ChunkInfo setCell2ChunkInfo1 = - ReadChangeStreamResponse.MutationChunk.ChunkInfo.newBuilder() - .setChunkedValueSize(totalCellValue2.length()) - .build(); - ReadChangeStreamResponse.DataChange dataChange2 = - ReadChangeStreamResponse.DataChange.newBuilder() - .setType(ReadChangeStreamResponse.DataChange.Type.CONTINUATION) - .addChunks( - ReadChangeStreamResponse.MutationChunk.newBuilder() - .setMutation(setCell1Chunk2) - .setChunkInfo(setCell1ChunkInfo2)) - .addChunks( - ReadChangeStreamResponse.MutationChunk.newBuilder() - .setMutation(setCell2Chunk1) - .setChunkInfo(setCell2ChunkInfo1)) - .build(); - ReadChangeStreamResponse dataChangeResponse2 = - ReadChangeStreamResponse.newBuilder().setDataChange(dataChange2).build(); - - // Build the 3rd ReadChangeStreamResponse which has 2/2 of SetCell2. - Mutation setCell2Chunk2 = - Mutation.newBuilder() - .setSetCell( - Mutation.SetCell.newBuilder() - .setValue(ByteString.copyFromUtf8(setCell2Chunk2val)) - .build()) - .build(); - ReadChangeStreamResponse.MutationChunk.ChunkInfo setCell2ChunkInfo2 = - ReadChangeStreamResponse.MutationChunk.ChunkInfo.newBuilder() - .setChunkedValueOffset(setCell2Chunk1val.length()) - .setChunkedValueSize(totalCellValue2.length()) - .setLastChunk(true) - .build(); - ReadChangeStreamResponse.DataChange dataChange3 = - ReadChangeStreamResponse.DataChange.newBuilder() - .setType(ReadChangeStreamResponse.DataChange.Type.CONTINUATION) - .addChunks( - ReadChangeStreamResponse.MutationChunk.newBuilder() - .setMutation(setCell2Chunk2) - .setChunkInfo(setCell2ChunkInfo2)) - .setDone(true) - .setToken("fake-token") - .setLowWatermark(fakeLowWatermark) - .build(); - ReadChangeStreamResponse dataChangeResponse3 = - ReadChangeStreamResponse.newBuilder().setDataChange(dataChange3).build(); - - // Feed the 3 ReadChangeStream responses to the merging callable. - FakeStreamingApi.ServerStreamingStashCallable - inner = - new ServerStreamingStashCallable<>( - Arrays.asList(dataChangeResponse1, dataChangeResponse2, dataChangeResponse3)); - ChangeStreamRecordMergingCallable mergingCallable = - new ChangeStreamRecordMergingCallable<>(inner, new DefaultChangeStreamRecordAdapter()); - // Actual results. - List results = - mergingCallable.all().call(ReadChangeStreamRequest.getDefaultInstance()); - - // Expected results. - Timestamp fakeCommitTimestamp = Timestamp.newBuilder().setSeconds(100).build(); - ChangeStreamMutation changeStreamMutation = - ChangeStreamMutation.createUserMutation( - ByteString.copyFromUtf8("key"), "fake-source-cluster-id", fakeCommitTimestamp, 100) - .setCell( - "fake-family", - ByteString.copyFromUtf8("fake-qualifier"), - 1000L, - ByteString.copyFromUtf8(totalCellValue1)) - .setCell( - "fake-family", - ByteString.copyFromUtf8("fake-qualifier"), - 1000L, - ByteString.copyFromUtf8(totalCellValue2)) - .setToken("fake-token") - .setLowWatermark(fakeLowWatermark) - .build(); - Truth.assertThat(results).containsExactly(changeStreamMutation); - } - - // [{SetCell1_chunk1}, {SetCell1_chunk2, DeleteFamily, DeleteCells, SetCell2}] - // -> [ChangeStreamMutation{SetCell1, DeleteFamily, DeleteCells, SetCell2}] - @Test - public void oneChunkedCell_manyNonChunkedModsTest() { - Timestamp fakeLowWatermark = Timestamp.newBuilder().setSeconds(100).build(); - String firstCellValue = "chunk1-value"; - String secondCellValue = "chunk2-value"; - String totalCellValue = firstCellValue + secondCellValue; - // Build the 1st ReadChangeStreamResponse which contains 1/2 of a chunked SetCell. - Mutation chunkedCell1 = - Mutation.newBuilder() - .setSetCell( - Mutation.SetCell.newBuilder() - .setFamilyName("fake-family") - .setColumnQualifier(ByteString.copyFromUtf8("fake-qualifier")) - .setTimestampMicros(1000L) - .setValue(ByteString.copyFromUtf8(firstCellValue)) - .build()) - .build(); - ReadChangeStreamResponse.MutationChunk.ChunkInfo chunkInfo1 = - ReadChangeStreamResponse.MutationChunk.ChunkInfo.newBuilder() - .setChunkedValueSize(totalCellValue.length()) - .build(); - ReadChangeStreamResponse.DataChange dataChange1 = - ReadChangeStreamResponse.DataChange.newBuilder() - .setType(ReadChangeStreamResponse.DataChange.Type.USER) - .setSourceClusterId("fake-source-cluster-id") - .setRowKey(ByteString.copyFromUtf8("key")) - .setCommitTimestamp(Timestamp.newBuilder().setSeconds(100).build()) - .setTiebreaker(100) - .addChunks( - ReadChangeStreamResponse.MutationChunk.newBuilder() - .setMutation(chunkedCell1) - .setChunkInfo(chunkInfo1)) - .build(); - ReadChangeStreamResponse dataChangeResponse1 = - ReadChangeStreamResponse.newBuilder().setDataChange(dataChange1).build(); - - // Build the 2nd ReadChangeStreamResponse which contains 2/2 of a chunked SetCell, a - // deleteFromFamily, a deleteFromColumn, and a non-chunked SetCell. - Mutation chunkedCell2 = - Mutation.newBuilder() - .setSetCell( - Mutation.SetCell.newBuilder() - .setValue(ByteString.copyFromUtf8(secondCellValue)) - .build()) - .build(); - ReadChangeStreamResponse.MutationChunk.ChunkInfo chunkInfo2 = - ReadChangeStreamResponse.MutationChunk.ChunkInfo.newBuilder() - .setChunkedValueOffset(firstCellValue.length()) - .setChunkedValueSize(totalCellValue.length()) - .setLastChunk(true) - .build(); - Mutation deleteFromFamily = - Mutation.newBuilder() - .setDeleteFromFamily( - Mutation.DeleteFromFamily.newBuilder().setFamilyName("fake-family").build()) - .build(); - Mutation deleteFromColumn = - Mutation.newBuilder() - .setDeleteFromColumn( - Mutation.DeleteFromColumn.newBuilder() - .setFamilyName("fake-family") - .setColumnQualifier(ByteString.copyFromUtf8("fake-qualifier")) - .setTimeRange( - TimestampRange.newBuilder() - .setStartTimestampMicros(1000L) - .setEndTimestampMicros(2000L) - .build()) - .build()) - .build(); - Mutation nonChunkedCell = - Mutation.newBuilder() - .setSetCell( - Mutation.SetCell.newBuilder() - .setFamilyName("fake-family") - .setColumnQualifier(ByteString.copyFromUtf8("fake-qualifier")) - .setTimestampMicros(1000L) - .setValue(ByteString.copyFromUtf8("fake-value")) - .build()) - .build(); - ReadChangeStreamResponse.DataChange dataChange2 = - ReadChangeStreamResponse.DataChange.newBuilder() - .setType(ReadChangeStreamResponse.DataChange.Type.CONTINUATION) - .addChunks( - ReadChangeStreamResponse.MutationChunk.newBuilder() - .setMutation(chunkedCell2) - .setChunkInfo(chunkInfo2)) - .addChunks( - ReadChangeStreamResponse.MutationChunk.newBuilder().setMutation(deleteFromFamily)) - .addChunks( - ReadChangeStreamResponse.MutationChunk.newBuilder().setMutation(deleteFromColumn)) - .addChunks( - ReadChangeStreamResponse.MutationChunk.newBuilder().setMutation(nonChunkedCell)) - .setDone(true) - .setToken("fake-token") - .setLowWatermark(fakeLowWatermark) - .build(); - ReadChangeStreamResponse dataChangeResponse2 = - ReadChangeStreamResponse.newBuilder().setDataChange(dataChange2).build(); - - // Feed the two ReadChangeStream responses to the merging callable. - FakeStreamingApi.ServerStreamingStashCallable - inner = - new ServerStreamingStashCallable<>( - Arrays.asList(dataChangeResponse1, dataChangeResponse2)); - ChangeStreamRecordMergingCallable mergingCallable = - new ChangeStreamRecordMergingCallable<>(inner, new DefaultChangeStreamRecordAdapter()); - // Actual results. - List results = - mergingCallable.all().call(ReadChangeStreamRequest.getDefaultInstance()); - - // Expected results. - Timestamp fakeCommitTimestamp = Timestamp.newBuilder().setSeconds(100).build(); - ChangeStreamMutation changeStreamMutation = - ChangeStreamMutation.createUserMutation( - ByteString.copyFromUtf8("key"), "fake-source-cluster-id", fakeCommitTimestamp, 100) - .setCell( - "fake-family", - ByteString.copyFromUtf8("fake-qualifier"), - 1000L, - ByteString.copyFromUtf8(totalCellValue)) - .deleteFamily("fake-family") - .deleteCells( - "fake-family", - ByteString.copyFromUtf8("fake-qualifier"), - Range.TimestampRange.create(1000L, 2000L)) - .setCell( - "fake-family", - ByteString.copyFromUtf8("fake-qualifier"), - 1000L, - ByteString.copyFromUtf8("fake-value")) - .setToken("fake-token") - .setLowWatermark(fakeLowWatermark) - .build(); - Truth.assertThat(results).containsExactly(changeStreamMutation); - } - - // [{SetCell1, SetCell2_chunk1}, {SetCell2_chunk2}] - // -> [ChangeStreamMutation{SetCell1, SetCell2}] - @Test - public void oneUnChunkedCell_oneChunkedCellTest() { - Timestamp fakeLowWatermark = Timestamp.newBuilder().setSeconds(100).build(); - Mutation nonChunkedCell = - Mutation.newBuilder() - .setSetCell( - Mutation.SetCell.newBuilder() - .setFamilyName("fake-family") - .setColumnQualifier(ByteString.copyFromUtf8("fake-qualifier")) - .setTimestampMicros(1000L) - .setValue(ByteString.copyFromUtf8("fake-value")) - .build()) - .build(); - String firstCellValue = "chunk1-value"; - String secondCellValue = "chunk2-value"; - String totalCellValue = firstCellValue + secondCellValue; - // Build the 1st ReadChangeStreamResponse which contains a nonChunkedCell and 1/2 of the chunked - // SetCell. - Mutation chunkedCell1 = - Mutation.newBuilder() - .setSetCell( - Mutation.SetCell.newBuilder() - .setFamilyName("fake-family") - .setColumnQualifier(ByteString.copyFromUtf8("fake-qualifier")) - .setTimestampMicros(1000L) - .setValue(ByteString.copyFromUtf8(firstCellValue)) - .build()) - .build(); - ReadChangeStreamResponse.MutationChunk.ChunkInfo chunkInfo1 = - ReadChangeStreamResponse.MutationChunk.ChunkInfo.newBuilder() - .setChunkedValueSize(totalCellValue.length()) - .build(); - ReadChangeStreamResponse.DataChange dataChange1 = - ReadChangeStreamResponse.DataChange.newBuilder() - .setType(ReadChangeStreamResponse.DataChange.Type.USER) - .setSourceClusterId("fake-source-cluster-id") - .setRowKey(ByteString.copyFromUtf8("key")) - .setCommitTimestamp(Timestamp.newBuilder().setSeconds(100).build()) - .setTiebreaker(100) - .addChunks( - ReadChangeStreamResponse.MutationChunk.newBuilder().setMutation(nonChunkedCell)) - .addChunks( - ReadChangeStreamResponse.MutationChunk.newBuilder() - .setMutation(chunkedCell1) - .setChunkInfo(chunkInfo1)) - .build(); - ReadChangeStreamResponse dataChangeResponse1 = - ReadChangeStreamResponse.newBuilder().setDataChange(dataChange1).build(); - - // Build the 2nd ReadChangeStreamResponse which contains 2/2 of the chunked SetCell. - Mutation chunkedCell2 = - Mutation.newBuilder() - .setSetCell( - Mutation.SetCell.newBuilder() - .setValue(ByteString.copyFromUtf8(secondCellValue)) - .build()) - .build(); - ReadChangeStreamResponse.MutationChunk.ChunkInfo chunkInfo2 = - ReadChangeStreamResponse.MutationChunk.ChunkInfo.newBuilder() - .setChunkedValueOffset(firstCellValue.length()) - .setChunkedValueSize(totalCellValue.length()) - .setLastChunk(true) - .build(); - ReadChangeStreamResponse.DataChange dataChange2 = - ReadChangeStreamResponse.DataChange.newBuilder() - .setType(ReadChangeStreamResponse.DataChange.Type.CONTINUATION) - .addChunks( - ReadChangeStreamResponse.MutationChunk.newBuilder() - .setMutation(chunkedCell2) - .setChunkInfo(chunkInfo2)) - .setDone(true) - .setToken("fake-token") - .setLowWatermark(fakeLowWatermark) - .build(); - ReadChangeStreamResponse dataChangeResponse2 = - ReadChangeStreamResponse.newBuilder().setDataChange(dataChange2).build(); - - // Feed the two ReadChangeStream responses to the merging callable. - FakeStreamingApi.ServerStreamingStashCallable - inner = - new ServerStreamingStashCallable<>( - Arrays.asList(dataChangeResponse1, dataChangeResponse2)); - ChangeStreamRecordMergingCallable mergingCallable = - new ChangeStreamRecordMergingCallable<>(inner, new DefaultChangeStreamRecordAdapter()); - // Actual results. - List results = - mergingCallable.all().call(ReadChangeStreamRequest.getDefaultInstance()); - - // Expected results. - Timestamp fakeCommitTimestamp = Timestamp.newBuilder().setSeconds(100).build(); - ChangeStreamMutation changeStreamMutation = - ChangeStreamMutation.createUserMutation( - ByteString.copyFromUtf8("key"), "fake-source-cluster-id", fakeCommitTimestamp, 100) - .setCell( - "fake-family", - ByteString.copyFromUtf8("fake-qualifier"), - 1000L, - ByteString.copyFromUtf8("fake-value")) - .setCell( - "fake-family", - ByteString.copyFromUtf8("fake-qualifier"), - 1000L, - ByteString.copyFromUtf8(totalCellValue)) - .setToken("fake-token") - .setLowWatermark(fakeLowWatermark) - .build(); - Truth.assertThat(results).containsExactly(changeStreamMutation); - } - - // [{DeleteFamily1, DeleteCells}, {DeleteFamily2}] - // -> [ChangeStreamMutation{DeleteFamily1, DeleteCells, DeleteFamily2}] - @Test - public void nonSetCellChunkingTest() { - Timestamp fakeLowWatermark = Timestamp.newBuilder().setSeconds(100).build(); - Mutation deleteFromFamily1 = - Mutation.newBuilder() - .setDeleteFromFamily( - Mutation.DeleteFromFamily.newBuilder().setFamilyName("fake-family").build()) - .build(); - Mutation deleteFromColumn = - Mutation.newBuilder() - .setDeleteFromColumn( - Mutation.DeleteFromColumn.newBuilder() - .setFamilyName("fake-family") - .setColumnQualifier(ByteString.copyFromUtf8("fake-qualifier")) - .setTimeRange( - TimestampRange.newBuilder() - .setStartTimestampMicros(1000L) - .setEndTimestampMicros(2000L) - .build()) - .build()) - .build(); - Mutation deleteFromFamily2 = - Mutation.newBuilder() - .setDeleteFromFamily( - Mutation.DeleteFromFamily.newBuilder().setFamilyName("fake-family2").build()) - .build(); - // Build the 1st ReadChangeStreamResponse which contains deleteFromFamily1 and deleteFromColumn. - ReadChangeStreamResponse.DataChange dataChange1 = - ReadChangeStreamResponse.DataChange.newBuilder() - .setType(ReadChangeStreamResponse.DataChange.Type.USER) - .setSourceClusterId("fake-source-cluster-id") - .setRowKey(ByteString.copyFromUtf8("key")) - .setCommitTimestamp(Timestamp.newBuilder().setSeconds(100).build()) - .setTiebreaker(100) - .addChunks( - ReadChangeStreamResponse.MutationChunk.newBuilder().setMutation(deleteFromFamily1)) - .addChunks( - ReadChangeStreamResponse.MutationChunk.newBuilder().setMutation(deleteFromColumn)) - .build(); - ReadChangeStreamResponse dataChangeResponse1 = - ReadChangeStreamResponse.newBuilder().setDataChange(dataChange1).build(); - - // Build the 2nd ReadChangeStreamResponse which contains deleteFromFamily2. - ReadChangeStreamResponse.DataChange dataChange2 = - ReadChangeStreamResponse.DataChange.newBuilder() - .setType(ReadChangeStreamResponse.DataChange.Type.CONTINUATION) - .addChunks( - ReadChangeStreamResponse.MutationChunk.newBuilder().setMutation(deleteFromFamily2)) - .setDone(true) - .setToken("fake-token") - .setLowWatermark(fakeLowWatermark) - .build(); - ReadChangeStreamResponse dataChangeResponse2 = - ReadChangeStreamResponse.newBuilder().setDataChange(dataChange2).build(); - - // Feed the two ReadChangeStream responses to the merging callable. - FakeStreamingApi.ServerStreamingStashCallable - inner = - new ServerStreamingStashCallable<>( - Arrays.asList(dataChangeResponse1, dataChangeResponse2)); - ChangeStreamRecordMergingCallable mergingCallable = - new ChangeStreamRecordMergingCallable<>(inner, new DefaultChangeStreamRecordAdapter()); - // Actual results. - List results = - mergingCallable.all().call(ReadChangeStreamRequest.getDefaultInstance()); - - // Expected results. - Timestamp fakeCommitTimestamp = Timestamp.newBuilder().setSeconds(100).build(); - ChangeStreamMutation changeStreamMutation = - ChangeStreamMutation.createUserMutation( - ByteString.copyFromUtf8("key"), "fake-source-cluster-id", fakeCommitTimestamp, 100) - .deleteFamily("fake-family") - .deleteCells( - "fake-family", - ByteString.copyFromUtf8("fake-qualifier"), - Range.TimestampRange.create(1000L, 2000L)) - .deleteFamily("fake-family2") - .setToken("fake-token") - .setLowWatermark(fakeLowWatermark) - .build(); - Truth.assertThat(results).containsExactly(changeStreamMutation); + // Validate the result. + Assert.assertEquals(results.size(), 1); + ChangeStreamRecord record = results.get(0); + Assert.assertTrue(record instanceof CloseStream); + CloseStream closeStream = (CloseStream) record; + Assert.assertEquals(closeStream.getStatus(), closeStreamProto.getStatus()); + Assert.assertEquals(closeStream.getChangeStreamContinuationTokens().size(), 1); + ChangeStreamContinuationToken changeStreamContinuationToken = + closeStream.getChangeStreamContinuationTokens().get(0); + Assert.assertEquals( + changeStreamContinuationToken.getRowRange(), + streamContinuationToken.getPartition().getRowRange()); + Assert.assertEquals( + changeStreamContinuationToken.getToken(), streamContinuationToken.getToken()); } } diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java index 78745bc484..1d6233cc10 100644 --- a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java +++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java @@ -21,19 +21,9 @@ import com.google.api.client.util.Lists; import com.google.api.gax.rpc.ServerStream; import com.google.api.gax.rpc.ServerStreamingCallable; +import com.google.bigtable.v2.*; import com.google.bigtable.v2.Mutation; -import com.google.bigtable.v2.ReadChangeStreamRequest; -import com.google.bigtable.v2.ReadChangeStreamResponse; -import com.google.bigtable.v2.TimestampRange; -import com.google.cloud.bigtable.data.v2.models.ChangeStreamMutation; -import com.google.cloud.bigtable.data.v2.models.ChangeStreamRecord; -import com.google.cloud.bigtable.data.v2.models.CloseStream; -import com.google.cloud.bigtable.data.v2.models.DefaultChangeStreamRecordAdapter; -import com.google.cloud.bigtable.data.v2.models.DeleteCells; -import com.google.cloud.bigtable.data.v2.models.DeleteFamily; -import com.google.cloud.bigtable.data.v2.models.Entry; -import com.google.cloud.bigtable.data.v2.models.Heartbeat; -import com.google.cloud.bigtable.data.v2.models.SetCell; +import com.google.cloud.bigtable.data.v2.models.*; import com.google.cloud.bigtable.gaxx.testing.FakeStreamingApi; import com.google.cloud.conformance.bigtable.v2.ChangeStreamTestDefinition.ChangeStreamTestFile; import com.google.cloud.conformance.bigtable.v2.ChangeStreamTestDefinition.ReadChangeStreamTest; @@ -95,7 +85,6 @@ public static Collection data() throws IOException { @Test public void test() throws Exception { List responses = testCase.getApiResponsesList(); - System.out.println("testCase: " + responses); // Wrap the responses in a callable. ServerStreamingCallable source = @@ -113,19 +102,46 @@ public void test() throws Exception { try { for (ChangeStreamRecord record : stream) { if (record instanceof Heartbeat) { + Heartbeat heartbeat = (Heartbeat) record; + ReadChangeStreamResponse.Heartbeat heartbeatProto = + ReadChangeStreamResponse.Heartbeat.newBuilder() + .setContinuationToken( + StreamContinuationToken.newBuilder() + .setPartition( + StreamPartition.newBuilder() + .setRowRange( + heartbeat.getChangeStreamContinuationToken().getRowRange()) + .build()) + .setToken(heartbeat.getChangeStreamContinuationToken().getToken()) + .build()) + .setLowWatermark(heartbeat.getLowWatermark()) + .build(); actualResults.add( ReadChangeStreamTest.Result.newBuilder() .setRecord( ReadChangeStreamTest.TestChangeStreamRecord.newBuilder() - .setHeartbeat(((Heartbeat) record).toProto()) + .setHeartbeat(heartbeatProto) .build()) .build()); } else if (record instanceof CloseStream) { + CloseStream closeStream = (CloseStream) record; + ReadChangeStreamResponse.CloseStream.Builder builder = + ReadChangeStreamResponse.CloseStream.newBuilder().setStatus(closeStream.getStatus()); + for (ChangeStreamContinuationToken token : + closeStream.getChangeStreamContinuationTokens()) { + builder.addContinuationTokens( + StreamContinuationToken.newBuilder() + .setPartition( + StreamPartition.newBuilder().setRowRange(token.getRowRange()).build()) + .setToken(token.getToken()) + .build()); + } + ReadChangeStreamResponse.CloseStream closeStreamProto = builder.build(); actualResults.add( ReadChangeStreamTest.Result.newBuilder() .setRecord( ReadChangeStreamTest.TestChangeStreamRecord.newBuilder() - .setCloseStream(((CloseStream) record).toProto()) + .setCloseStream(closeStreamProto) .build()) .build()); } else if (record instanceof ChangeStreamMutation) { From 7e763c66a6e6d346e503f81ceaad0f3a8ba92f1a Mon Sep 17 00:00:00 2001 From: Teng Zhong Date: Thu, 4 Aug 2022 18:56:59 -0400 Subject: [PATCH 04/10] fix: Address comments --- .../data/v2/stub/EnhancedBigtableStub.java | 63 ++++++------------- .../ReadChangeStreamUserCallable.java | 7 ++- 2 files changed, 26 insertions(+), 44 deletions(-) diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStub.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStub.java index 630d55c2bb..c2485fd371 100644 --- a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStub.java +++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStub.java @@ -895,8 +895,9 @@ public Map extract( * dispatch the RPC. *

  • Upon receiving the response stream, it will produce a stream of ChangeStreamRecordT. In * case of mutations, it will merge the {@link ReadChangeStreamResponse.DataChange}s into - * logical mutations. The actual change stream record implementation can be configured by - * the {@code changeStreamRecordAdapter} parameter. + * {@link com.google.cloud.bigtable.data.v2.models.ChangeStreamMutation}. The actual change + * stream record implementation can be configured by the {@code changeStreamRecordAdapter} + * parameter. *
  • TODO: Retry/resume on failure. *
  • Add tracing & metrics. * @@ -905,42 +906,6 @@ public Map extract( ServerStreamingCallable createReadChangeStreamCallable( ChangeStreamRecordAdapter changeStreamRecordAdapter) { - ServerStreamingCallable readChangeStreamCallable = - createReadChangeStreamBaseCallable( - settings.readChangeStreamSettings(), changeStreamRecordAdapter); - - ServerStreamingCallable - readChangeStreamUserCallable = - new ReadChangeStreamUserCallable<>(readChangeStreamCallable, requestContext); - - SpanName span = getSpanName("ReadChangeStream"); - ServerStreamingCallable traced = - new TracedServerStreamingCallable<>( - readChangeStreamUserCallable, clientContext.getTracerFactory(), span); - - return traced.withDefaultCallContext(clientContext.getDefaultCallContext()); - } - - /** - * Creates a callable chain to handle ReadRows RPCs. The chain will: - * - *
      - *
    • Dispatch the RPC with {@link ReadChangeStreamRequest}. - *
    • Upon receiving the response stream, it will produce a stream of ChangeStreamRecordT. In - * case of mutations, it will merge the {@link ReadChangeStreamResponse.DataChange}s into - * logical mutations. The actual change stream record implementation can be configured by - * the {@code changeStreamRecordAdapter} parameter. - *
    • Add header tracer for tracking GFE metrics. - *
    • TODO: Retry/resume on failure. - *
    - * - *

    NOTE: the caller is responsible for adding tracing & metrics. - */ - private - ServerStreamingCallable - createReadChangeStreamBaseCallable( - ServerStreamingCallSettings readChangeStreamSettings, - ChangeStreamRecordAdapter changeStreamRecordAdapter) { ServerStreamingCallable base = GrpcRawCallableFactory.createServerStreamingCallable( GrpcCallSettings.newBuilder() @@ -956,7 +921,7 @@ public Map extract( } }) .build(), - readChangeStreamSettings.getRetryableCodes()); + settings.readChangeStreamSettings().getRetryableCodes()); ServerStreamingCallable withStatsHeaders = new StatsHeadersServerStreamingCallable<>(base); @@ -975,9 +940,9 @@ public Map extract( ServerStreamingCallSettings innerSettings = ServerStreamingCallSettings.newBuilder() // TODO: setResumptionStrategy. - .setRetryableCodes(readChangeStreamSettings.getRetryableCodes()) - .setRetrySettings(readChangeStreamSettings.getRetrySettings()) - .setIdleTimeout(readChangeStreamSettings.getIdleTimeout()) + .setRetryableCodes(settings.readChangeStreamSettings().getRetryableCodes()) + .setRetrySettings(settings.readChangeStreamSettings().getRetrySettings()) + .setIdleTimeout(settings.readChangeStreamSettings().getIdleTimeout()) .build(); ServerStreamingCallable watched = @@ -988,7 +953,19 @@ public Map extract( // TODO: Add ReadChangeStreamRetryCompletedCallable. - return Callables.retrying(withBigtableTracer, innerSettings, clientContext); + ServerStreamingCallable readChangeStreamCallable = + Callables.retrying(withBigtableTracer, innerSettings, clientContext); + + ServerStreamingCallable + readChangeStreamUserCallable = + new ReadChangeStreamUserCallable<>(readChangeStreamCallable, requestContext); + + SpanName span = getSpanName("ReadChangeStream"); + ServerStreamingCallable traced = + new TracedServerStreamingCallable<>( + readChangeStreamUserCallable, clientContext.getTracerFactory(), span); + + return traced.withDefaultCallContext(clientContext.getDefaultCallContext()); } /** diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamUserCallable.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamUserCallable.java index 1ed1e2359b..0c78199ccd 100644 --- a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamUserCallable.java +++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamUserCallable.java @@ -15,6 +15,7 @@ */ package com.google.cloud.bigtable.data.v2.stub.changestream; +import com.google.api.core.InternalApi; import com.google.api.gax.rpc.ApiCallContext; import com.google.api.gax.rpc.ResponseObserver; import com.google.api.gax.rpc.ServerStreamingCallable; @@ -22,7 +23,11 @@ import com.google.cloud.bigtable.data.v2.internal.RequestContext; import com.google.cloud.bigtable.data.v2.models.ReadChangeStreamQuery; -/** A ServerStreamingCallable that reads a change stream given the {@link ReadChangeStreamQuery}. */ +/** + * A ServerStreamingCallable that converts a {@link ReadChangeStreamQuery} to a {@link + * ReadChangeStreamRequest}. + */ +@InternalApi("Used in Changestream beam pipeline.") public class ReadChangeStreamUserCallable extends ServerStreamingCallable { private final ServerStreamingCallable inner; From 4e3fa84eed62201d871af5968e8bea420bf51ccb Mon Sep 17 00:00:00 2001 From: Teng Zhong Date: Fri, 5 Aug 2022 13:07:10 -0400 Subject: [PATCH 05/10] fix: Address some comments --- .../cloud/bigtable/data/v2/stub/EnhancedBigtableStub.java | 6 +++--- .../data/v2/stub/changestream/ChangeStreamRecordMerger.java | 2 +- google-cloud-bigtable/src/main/resources/changestream.json | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStub.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStub.java index c2485fd371..7872b1e07e 100644 --- a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStub.java +++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/EnhancedBigtableStub.java @@ -66,6 +66,7 @@ import com.google.cloud.bigtable.data.v2.internal.JwtCredentialsWithAudience; 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.ChangeStreamMutation; import com.google.cloud.bigtable.data.v2.models.ChangeStreamRecord; import com.google.cloud.bigtable.data.v2.models.ChangeStreamRecordAdapter; import com.google.cloud.bigtable.data.v2.models.ConditionalRowMutation; @@ -895,9 +896,8 @@ public Map extract( * dispatch the RPC. *

  • Upon receiving the response stream, it will produce a stream of ChangeStreamRecordT. In * case of mutations, it will merge the {@link ReadChangeStreamResponse.DataChange}s into - * {@link com.google.cloud.bigtable.data.v2.models.ChangeStreamMutation}. The actual change - * stream record implementation can be configured by the {@code changeStreamRecordAdapter} - * parameter. + * {@link ChangeStreamMutation}. The actual change stream record implementation can be + * configured by the {@code changeStreamRecordAdapter} parameter. *
  • TODO: Retry/resume on failure. *
  • Add tracing & metrics. * diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamRecordMerger.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamRecordMerger.java index 6c77610676..0b4bf5acdb 100644 --- a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamRecordMerger.java +++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamRecordMerger.java @@ -36,7 +36,7 @@ * *
    {@code
      * ChangeStreamRecordMerger changeStreamRecordMerger =
    - *     new ChangeStreamRecordMerger<>(myChangeStreamRecordBuilder);
    + *     new ChangeStreamRecordMerger<>(myChangeStreamRecordAdaptor);
      *
      * while(responseIterator.hasNext()) {
      *   ReadChangeStreamResponse response = responseIterator.next();
    diff --git a/google-cloud-bigtable/src/main/resources/changestream.json b/google-cloud-bigtable/src/main/resources/changestream.json
    index b48056d402..397c9b170f 100644
    --- a/google-cloud-bigtable/src/main/resources/changestream.json
    +++ b/google-cloud-bigtable/src/main/resources/changestream.json
    @@ -273,7 +273,7 @@
           ]
         },
         {
    -      "description": "1 incomplete logical mutation",
    +      "description": "1 incomplete logical mutation(missing `done: true`)",
           "api_responses": [
             {
               "data_change": {
    
    From ad01c394fa1de0c7faae1d0181b24630f4d40af6 Mon Sep 17 00:00:00 2001
    From: Teng Zhong 
    Date: Fri, 5 Aug 2022 14:43:36 -0400
    Subject: [PATCH 06/10] fix: Add test for [{SC_chunk1}, {SC_chunk2},
     {SC_chunk3}]->ChangeStreamMutation{SC}
    
    ---
     .../src/main/resources/changestream.json      | 155 ++++++++++++++----
     ...ReadChangeStreamMergingAcceptanceTest.java |  17 +-
     2 files changed, 138 insertions(+), 34 deletions(-)
    
    diff --git a/google-cloud-bigtable/src/main/resources/changestream.json b/google-cloud-bigtable/src/main/resources/changestream.json
    index 397c9b170f..e6f52e32f0 100644
    --- a/google-cloud-bigtable/src/main/resources/changestream.json
    +++ b/google-cloud-bigtable/src/main/resources/changestream.json
    @@ -384,12 +384,12 @@
                   {
                     "chunk_info": {
                       "chunked_value_offset": 1,
    -                  "chunked_value_size": 1,
    +                  "chunked_value_size": 9,
                       "last_chunk": true
                     },
                     "mutation": {
                       "set_cell": {
    -                    "value": ""
    +                    "value": "YWx1ZS1WQUw="
                       }
                     }
                   }
    @@ -411,6 +411,36 @@
                   "low_watermark": "2022-07-01T00:00:00Z",
                   "mutations": [
                     {
    +                  "set_cell": {
    +                    "family_name": "family",
    +                    "column_qualifier": "0000000000000000",
    +                    "timestamp_micros": 1000,
    +                    "value": "dmFsdWUtVkFM"
    +                  }
    +                }
    +              ]
    +            }
    +          },
    +          "error": false
    +        }
    +      ]
    +    },
    +    {
    +      "description": "1 chunked SetCell([{SC_chunk1}, {SC_chunk2}, {SC_chunk3}]->ChangeStreamMutation{SC})",
    +      "api_responses": [
    +        {
    +          "data_change": {
    +            "row_key": "0000000000000000",
    +            "type": "USER",
    +            "source_cluster_id": "source-cluster-id",
    +            "commit_timestamp": "2022-07-01T00:00:00Z",
    +            "tiebreaker": 100,
    +            "chunks": [
    +              {
    +                "chunk_info": {
    +                  "chunked_value_size": 1
    +                },
    +                "mutation": {
                       "set_cell": {
                         "family_name": "family",
                         "column_qualifier": "0000000000000000",
    @@ -418,6 +448,73 @@
                         "value": "dg=="
                       }
                     }
    +              }
    +            ]
    +          }
    +        },
    +        {
    +          "data_change": {
    +            "type": "CONTINUATION",
    +            "token": "data-change-token",
    +            "low_watermark": "2022-07-01T00:00:00Z",
    +            "chunks": [
    +              {
    +                "chunk_info": {
    +                  "chunked_value_offset": 1,
    +                  "chunked_value_size": 13
    +                },
    +                "mutation": {
    +                  "set_cell": {
    +                    "value": "YWx1ZS1WQUw="
    +                  }
    +                }
    +              }
    +            ]
    +          }
    +        },
    +        {
    +          "data_change": {
    +            "type": "CONTINUATION",
    +            "token": "data-change-token",
    +            "low_watermark": "2022-07-01T00:00:00Z",
    +            "chunks": [
    +              {
    +                "chunk_info": {
    +                  "chunked_value_offset": 9,
    +                  "chunked_value_size": 13,
    +                  "last_chunk": true
    +                },
    +                "mutation": {
    +                  "set_cell": {
    +                    "value": "LVZBTA=="
    +                  }
    +                }
    +              }
    +            ],
    +            "done": true
    +          }
    +        }
    +      ],
    +      "results": [
    +        {
    +          "record": {
    +            "change_stream_mutation": {
    +              "row_key": "0000000000000000",
    +              "type": "USER",
    +              "source_cluster_id": "source-cluster-id",
    +              "commit_timestamp": "2022-07-01T00:00:00Z",
    +              "tiebreaker": 100,
    +              "token": "data-change-token",
    +              "low_watermark": "2022-07-01T00:00:00Z",
    +              "mutations": [
    +                {
    +                  "set_cell": {
    +                    "family_name": "family",
    +                    "column_qualifier": "0000000000000000",
    +                    "timestamp_micros": 1000,
    +                    "value": "dmFsdWUtVkFMLVZBTA=="
    +                  }
    +                }
                   ]
                 }
               },
    @@ -459,12 +556,12 @@
                   {
                     "chunk_info": {
                       "chunked_value_offset": 1,
    -                  "chunked_value_size": 1,
    +                  "chunked_value_size": 9,
                       "last_chunk": true
                     },
                     "mutation": {
                       "set_cell": {
    -                    "value": ""
    +                    "value": "YWx1ZS1WQUw="
                       }
                     }
                   },
    @@ -493,12 +590,12 @@
                   {
                     "chunk_info": {
                       "chunked_value_offset": 1,
    -                  "chunked_value_size": 1,
    +                  "chunked_value_size": 9,
                       "last_chunk": true
                     },
                     "mutation": {
                       "set_cell": {
    -                    "value": ""
    +                    "value": "YWx1ZS1WQUw="
                       }
                     }
                   }
    @@ -524,7 +621,7 @@
                         "family_name": "family",
                         "column_qualifier": "0000000000000000",
                         "timestamp_micros": 1000,
    -                    "value": "dg=="
    +                    "value": "dmFsdWUtVkFM"
                       }
                     },
                     {
    @@ -532,7 +629,7 @@
                         "family_name": "family",
                         "column_qualifier": "0000000000000000",
                         "timestamp_micros": 1000,
    -                    "value": "dg=="
    +                    "value": "dmFsdWUtVkFM"
                       }
                     }
                   ]
    @@ -578,12 +675,12 @@
                   {
                     "chunk_info": {
                       "chunked_value_offset": 1,
    -                  "chunked_value_size": 1,
    +                  "chunked_value_size": 9,
                       "last_chunk": true
                     },
                     "mutation": {
                       "set_cell": {
    -                    "value": ""
    +                    "value": "YWx1ZS1WQUw="
                       }
                     }
                   },
    @@ -593,7 +690,7 @@
                         "family_name": "family",
                         "column_qualifier": "0000000000000000",
                         "timestamp_micros": 1000,
    -                    "value": "dg=="
    +                    "value": "dmFsdWUtVkFM"
                       }
                     }
                   }
    @@ -619,7 +716,7 @@
                         "family_name": "family",
                         "column_qualifier": "0000000000000000",
                         "timestamp_micros": 1000,
    -                    "value": "dg=="
    +                    "value": "dmFsdWUtVkFM"
                       }
                     },
                     {
    @@ -627,7 +724,7 @@
                         "family_name": "family",
                         "column_qualifier": "0000000000000000",
                         "timestamp_micros": 1000,
    -                    "value": "dg=="
    +                    "value": "dmFsdWUtVkFM"
                       }
                     }
                   ]
    @@ -683,12 +780,12 @@
                   {
                     "chunk_info": {
                       "chunked_value_offset": 1,
    -                  "chunked_value_size": 1,
    +                  "chunked_value_size": 9,
                       "last_chunk": true
                     },
                     "mutation": {
                       "set_cell": {
    -                    "value": ""
    +                    "value": "YWx1ZS1WQUw="
                       }
                     }
                   }
    @@ -722,7 +819,7 @@
                         "family_name": "family",
                         "column_qualifier": "0000000000000000",
                         "timestamp_micros": 1000,
    -                    "value": "dg=="
    +                    "value": "dmFsdWUtVkFM"
                       }
                     }
                   ]
    @@ -775,12 +872,12 @@
                   {
                     "chunk_info": {
                       "chunked_value_offset": 1,
    -                  "chunked_value_size": 1,
    +                  "chunked_value_size": 9,
                       "last_chunk": true
                     },
                     "mutation": {
                       "set_cell": {
    -                    "value": ""
    +                    "value": "YWx1ZS1WQUw="
                       }
                     }
                   },
    @@ -818,7 +915,7 @@
                         "family_name": "family",
                         "column_qualifier": "0000000000000000",
                         "timestamp_micros": 1000,
    -                    "value": "dg=="
    +                    "value": "dmFsdWUtVkFM"
                       }
                     },
                     {
    @@ -869,12 +966,12 @@
                   {
                     "chunk_info": {
                       "chunked_value_offset": 1,
    -                  "chunked_value_size": 1,
    +                  "chunked_value_size": 9,
                       "last_chunk": true
                     },
                     "mutation": {
                       "set_cell": {
    -                    "value": ""
    +                    "value": "YWx1ZS1WQUw="
                       }
                     }
                   },
    @@ -892,11 +989,8 @@
                   },
                   {
                     "mutation": {
    -                  "set_cell": {
    -                    "family_name": "family",
    -                    "column_qualifier": "0000000000000000",
    -                    "timestamp_micros": 1000,
    -                    "value": "dg=="
    +                  "delete_from_family": {
    +                    "family_name": "family"
                       }
                     }
                   }
    @@ -922,7 +1016,7 @@
                         "family_name": "family",
                         "column_qualifier": "0000000000000000",
                         "timestamp_micros": 1000,
    -                    "value": "dg=="
    +                    "value": "dmFsdWUtVkFM"
                       }
                     },
                     {
    @@ -936,11 +1030,8 @@
                       }
                     },
                     {
    -                  "set_cell": {
    -                    "family_name": "family",
    -                    "column_qualifier": "0000000000000000",
    -                    "timestamp_micros": 1000,
    -                    "value": "dg=="
    +                  "delete_from_family": {
    +                    "family_name": "family"
                       }
                     }
                   ]
    diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java
    index 1d6233cc10..a207589a2c 100644
    --- a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java
    +++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java
    @@ -21,9 +21,22 @@
     import com.google.api.client.util.Lists;
     import com.google.api.gax.rpc.ServerStream;
     import com.google.api.gax.rpc.ServerStreamingCallable;
    -import com.google.bigtable.v2.*;
     import com.google.bigtable.v2.Mutation;
    -import com.google.cloud.bigtable.data.v2.models.*;
    +import com.google.bigtable.v2.ReadChangeStreamRequest;
    +import com.google.bigtable.v2.ReadChangeStreamResponse;
    +import com.google.bigtable.v2.StreamContinuationToken;
    +import com.google.bigtable.v2.StreamPartition;
    +import com.google.bigtable.v2.TimestampRange;
    +import com.google.cloud.bigtable.data.v2.models.ChangeStreamContinuationToken;
    +import com.google.cloud.bigtable.data.v2.models.ChangeStreamMutation;
    +import com.google.cloud.bigtable.data.v2.models.ChangeStreamRecord;
    +import com.google.cloud.bigtable.data.v2.models.CloseStream;
    +import com.google.cloud.bigtable.data.v2.models.DefaultChangeStreamRecordAdapter;
    +import com.google.cloud.bigtable.data.v2.models.DeleteCells;
    +import com.google.cloud.bigtable.data.v2.models.DeleteFamily;
    +import com.google.cloud.bigtable.data.v2.models.Entry;
    +import com.google.cloud.bigtable.data.v2.models.Heartbeat;
    +import com.google.cloud.bigtable.data.v2.models.SetCell;
     import com.google.cloud.bigtable.gaxx.testing.FakeStreamingApi;
     import com.google.cloud.conformance.bigtable.v2.ChangeStreamTestDefinition.ChangeStreamTestFile;
     import com.google.cloud.conformance.bigtable.v2.ChangeStreamTestDefinition.ReadChangeStreamTest;
    
    From 03a2dfe0fa811779711eef0e37b4849b996c07a8 Mon Sep 17 00:00:00 2001
    From: Teng Zhong 
    Date: Mon, 8 Aug 2022 11:24:56 -0400
    Subject: [PATCH 07/10] fix: Update the changestream.json file for better
     description
    
    ---
     .../src/main/resources/changestream.json      | 26 +++++++++----------
     1 file changed, 13 insertions(+), 13 deletions(-)
    
    diff --git a/google-cloud-bigtable/src/main/resources/changestream.json b/google-cloud-bigtable/src/main/resources/changestream.json
    index e6f52e32f0..4708947db3 100644
    --- a/google-cloud-bigtable/src/main/resources/changestream.json
    +++ b/google-cloud-bigtable/src/main/resources/changestream.json
    @@ -349,7 +349,7 @@
           ]
         },
         {
    -      "description": "1 chunked SetCell([{SC_chunk1}, {SC_chunk2}]->ChangeStreamMutation{SC})",
    +      "description": "1 chunked SetCell([{SC_chunk1(v)}, {SC_chunk2(alue-VAL)}]->ChangeStreamMutation{SC(value-VAL)})",
           "api_responses": [
             {
               "data_change": {
    @@ -361,7 +361,7 @@
                 "chunks": [
                   {
                     "chunk_info": {
    -                  "chunked_value_size": 1
    +                  "chunked_value_size": 9
                     },
                     "mutation": {
                       "set_cell": {
    @@ -426,7 +426,7 @@
           ]
         },
         {
    -      "description": "1 chunked SetCell([{SC_chunk1}, {SC_chunk2}, {SC_chunk3}]->ChangeStreamMutation{SC})",
    +      "description": "1 chunked SetCell([{SC_chunk1(v)}, {SC_chunk2(alue-VAL)}, {SC_chunk3(-VAL)}]->ChangeStreamMutation{SC(value-VAL-VAL)})",
           "api_responses": [
             {
               "data_change": {
    @@ -438,7 +438,7 @@
                 "chunks": [
                   {
                     "chunk_info": {
    -                  "chunked_value_size": 1
    +                  "chunked_value_size": 13
                     },
                     "mutation": {
                       "set_cell": {
    @@ -523,7 +523,7 @@
           ]
         },
         {
    -      "description": "2 chunked SetCells([{SC1_chunk1}, {SC1_chunk2, SC2_chunk1}, {SC2_chunk2}]->ChangeStreamMutation{SC1,SC2})",
    +      "description": "2 chunked SetCells([{SC1_chunk1(v)}, {SC1_chunk2(alue-VAL), SC2_chunk1(v)}, {SC2_chunk2(alue-VAL)}]->ChangeStreamMutation{SC1(value-VAL),SC2(value-VAL)})",
           "api_responses": [
             {
               "data_change": {
    @@ -640,7 +640,7 @@
           ]
         },
         {
    -      "description": "1 chunked SetCell + 1 unchunked SetCell([{SC1_chunk1}, {SC1_chunk2, SC2}]->ChangeStreamMutation{SC1,SC2})",
    +      "description": "1 chunked SetCell + 1 unchunked SetCell([{SC1_chunk1(v)}, {SC1_chunk2(alue-VAL), SC2(value-VAL)}]->ChangeStreamMutation{SC1(value-VAL),SC2(value-VAL)})",
           "api_responses": [
             {
               "data_change": {
    @@ -652,7 +652,7 @@
                 "chunks": [
                   {
                     "chunk_info": {
    -                  "chunked_value_size": 1
    +                  "chunked_value_size": 9
                     },
                     "mutation": {
                       "set_cell": {
    @@ -735,7 +735,7 @@
           ]
         },
         {
    -      "description": "1 unchunked SetCell + 1 chunked SetCell([{SC1, SC2_chunk1}, {SC2_chunk2}]->ChangeStreamMutation{SC1,SC2})",
    +      "description": "1 unchunked SetCell + 1 chunked SetCell([{SC1(v), SC2_chunk1(v)}, {SC2_chunk2(alue-VAL)}]->ChangeStreamMutation{SC1(v),SC2(value-VAL)})",
           "api_responses": [
             {
               "data_change": {
    @@ -757,7 +757,7 @@
                   },
                   {
                     "chunk_info": {
    -                  "chunked_value_size": 1
    +                  "chunked_value_size": 9
                     },
                     "mutation": {
                       "set_cell": {
    @@ -830,7 +830,7 @@
           ]
         },
         {
    -      "description": "1 mod + 1 chunked SetCell + 1 mod([{DF1,SC_chunk1}, {SC_chunk2, DF2}]->ChangeStreamMutation{DF1,SC,DF2})",
    +      "description": "1 mod + 1 chunked SetCell + 1 mod([{DF1,SC_chunk1(v)}, {SC_chunk2(alue-VAL), DF2}]->ChangeStreamMutation{DF1,SC(value-VAL),DF2})",
           "api_responses": [
             {
               "data_change": {
    @@ -849,7 +849,7 @@
                   },
                   {
                     "chunk_info": {
    -                  "chunked_value_size": 1
    +                  "chunked_value_size": 9
                     },
                     "mutation": {
                       "set_cell": {
    @@ -931,7 +931,7 @@
           ]
         },
         {
    -      "description": "1 chunked SetCell + many nonchunked mods([{SC_chunk1}, {SC_chunk2,DF,DC}]->ChangeStreamMutation{SC,DF,DC})",
    +      "description": "1 chunked SetCell + many nonchunked mods([{SC_chunk1(v)}, {SC_chunk2(alue-VAL),DF,DC}]->ChangeStreamMutation{SC(value-VAL),DF,DC})",
           "api_responses": [
             {
               "data_change": {
    @@ -943,7 +943,7 @@
                 "chunks": [
                   {
                     "chunk_info": {
    -                  "chunked_value_size": 1
    +                  "chunked_value_size": 9
                     },
                     "mutation": {
                       "set_cell": {
    
    From a8e0176251243392ae61147379dc669a93a3758d Mon Sep 17 00:00:00 2001
    From: Teng Zhong 
    Date: Mon, 8 Aug 2022 13:02:49 -0400
    Subject: [PATCH 08/10] fix: Update code comments to make style-check happy
    
    ---
     .../google/cloud/bigtable/data/v2/BigtableDataClient.java   | 6 +++---
     1 file changed, 3 insertions(+), 3 deletions(-)
    
    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 ca32c4a6a5..2de6e9b94a 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
    @@ -1640,7 +1640,7 @@ public ServerStreamingCallable listChangeStreamPartitionsCalla
        *   String tableId = "[TABLE]";
        *
        *   ReadChangeStreamQuery query = ReadChangeStreamQuery.create(tableId)
    -   *          .streamPartition("[START KEY]", "[END KEY]")
    +   *          .streamPartition("START_KEY", "END_KEY")
        *          .startTime(Timestamp.newBuilder().setSeconds(100).build());
        *
        *   try {
    @@ -1681,7 +1681,7 @@ public ServerStream readChangeStream(ReadChangeStreamQuery q
        *   String tableId = "[TABLE]";
        *
        *   ReadChangeStreamQuery query = ReadChangeStreamQuery.create(tableId)
    -   *          .streamPartition("[START KEY]", "[END KEY]")
    +   *          .streamPartition("START_KEY", "END_KEY")
        *          .startTime(Timestamp.newBuilder().setSeconds(100).build());
        *
        *   bigtableDataClient.readChangeStreamAsync(query, new ResponseObserver() {
    @@ -1729,7 +1729,7 @@ public void readChangeStreamAsync(
        *   String tableId = "[TABLE]";
        *
        *   ReadChangeStreamQuery query = ReadChangeStreamQuery.create(tableId)
    -   *          .streamPartition("[START KEY]", "[END KEY]")
    +   *          .streamPartition("START_KEY", "END_KEY")
        *          .startTime(Timestamp.newBuilder().setSeconds(100).build());
        *
        *   // Iterator style
    
    From 56d9c4eec1a010a32001eeff8d61b425f1288a3d Mon Sep 17 00:00:00 2001
    From: Teng Zhong 
    Date: Mon, 8 Aug 2022 14:02:22 -0400
    Subject: [PATCH 09/10] fix: Add sanity check for ChunkedValueSize.
    
    Add comments to explain why we can put the AcceptanceTest in the
    google-cloud-bigtable repo
    ---
     .../ChangeStreamStateMachine.java             | 25 ++++++--
     .../src/main/resources/changestream.json      | 60 ++++++++++++++++++-
     ...ReadChangeStreamMergingAcceptanceTest.java |  8 ++-
     3 files changed, 86 insertions(+), 7 deletions(-)
    
    diff --git a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamStateMachine.java b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamStateMachine.java
    index 7ab7fa2b7b..ea555ea445 100644
    --- a/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamStateMachine.java
    +++ b/google-cloud-bigtable/src/main/java/com/google/cloud/bigtable/data/v2/stub/changestream/ChangeStreamStateMachine.java
    @@ -98,6 +98,13 @@ final class ChangeStreamStateMachine {
       private int numDataChanges = 0;
       private int numNonCellMods = 0;
       private int numCellChunks = 0; // 1 for non-chunked cell.
    +  /**
    +   * Expected total size of a chunked SetCell value, given by the {@link
    +   * ReadChangeStreamResponse.MutationChunk.ChunkInfo}. This value should be the same for all chunks
    +   * of a SetCell.
    +   */
    +  private int expectedTotalSizeOfChunkedSetCell = 0;
    +
       private int actualTotalSizeOfChunkedSetCell = 0;
       private ChangeStreamRecordT completeChangeStreamRecord;
     
    @@ -219,6 +226,7 @@ private void reset() {
         numDataChanges = 0;
         numNonCellMods = 0;
         numCellChunks = 0;
    +    expectedTotalSizeOfChunkedSetCell = 0;
         actualTotalSizeOfChunkedSetCell = 0;
         completeChangeStreamRecord = null;
     
    @@ -383,7 +391,11 @@ State handleMod(ReadChangeStreamResponse.DataChange dataChange, int index) {
                   // If it has chunk info, it must be the first chunk of a chunked SetCell.
                   validate(
                       chunk.getChunkInfo().getChunkedValueOffset() == 0,
    -                  "First chunk of a chunked cell must start with offset==0.");
    +                  "AWAITING_NEW_MOD: First chunk of a chunked cell must start with offset==0.");
    +              validate(
    +                  chunk.getChunkInfo().getChunkedValueSize() > 0,
    +                  "AWAITING_NEW_MOD: First chunk of a chunked cell must have a positive chunked value size.");
    +              expectedTotalSizeOfChunkedSetCell = chunk.getChunkInfo().getChunkedValueSize();
                   actualTotalSizeOfChunkedSetCell = 0;
                 }
                 builder.startCell(
    @@ -459,15 +471,18 @@ State handleMod(ReadChangeStreamResponse.DataChange dataChange, int index) {
                 validate(
                     chunk.getChunkInfo().getChunkedValueSize() > 0,
                     "AWAITING_CELL_VALUE: Chunked value size must be positive.");
    +            validate(
    +                chunk.getChunkInfo().getChunkedValueSize() == expectedTotalSizeOfChunkedSetCell,
    +                "AWAITING_CELL_VALUE: Chunked value size must be the same for all chunks.");
                 actualTotalSizeOfChunkedSetCell += setCell.getValue().size();
                 // If it's the last chunk of the chunked SetCell, finish the cell.
                 if (chunk.getChunkInfo().getLastChunk()) {
                   builder.finishCell();
                   validate(
    -                  actualTotalSizeOfChunkedSetCell == chunk.getChunkInfo().getChunkedValueSize(),
    +                  actualTotalSizeOfChunkedSetCell == expectedTotalSizeOfChunkedSetCell,
                       "Chunked value size in ChunkInfo doesn't match the actual total size. "
    -                      + "ChunkInfo: "
    -                      + chunk.getChunkInfo().getChunkedValueSize()
    +                      + "Expected total size: "
    +                      + expectedTotalSizeOfChunkedSetCell
                           + "; actual total size: "
                           + actualTotalSizeOfChunkedSetCell);
                   return checkAndFinishMutationIfNeeded(dataChange, index + 1);
    @@ -569,6 +584,8 @@ private void validate(boolean condition, String message) {
                   + numNonCellMods
                   + ", numCellChunks: "
                   + numCellChunks
    +              + ", expectedTotalSizeOfChunkedSetCell: "
    +              + expectedTotalSizeOfChunkedSetCell
                   + ", actualTotalSizeOfChunkedSetCell: "
                   + actualTotalSizeOfChunkedSetCell);
         }
    diff --git a/google-cloud-bigtable/src/main/resources/changestream.json b/google-cloud-bigtable/src/main/resources/changestream.json
    index 4708947db3..2dd886f805 100644
    --- a/google-cloud-bigtable/src/main/resources/changestream.json
    +++ b/google-cloud-bigtable/src/main/resources/changestream.json
    @@ -425,6 +425,62 @@
             }
           ]
         },
    +    {
    +      "description": "ChunkedValueSize mismatch for a chunked SetCell([{SC_chunk1(v)}, {SC_chunk2(alue-VAL)}]->error)",
    +      "api_responses": [
    +        {
    +          "data_change": {
    +            "row_key": "0000000000000000",
    +            "type": "USER",
    +            "source_cluster_id": "source-cluster-id",
    +            "commit_timestamp": "2022-07-01T00:00:00Z",
    +            "tiebreaker": 100,
    +            "chunks": [
    +              {
    +                "chunk_info": {
    +                  "chunked_value_size": 1
    +                },
    +                "mutation": {
    +                  "set_cell": {
    +                    "family_name": "family",
    +                    "column_qualifier": "0000000000000000",
    +                    "timestamp_micros": 1000,
    +                    "value": "dg=="
    +                  }
    +                }
    +              }
    +            ]
    +          }
    +        },
    +        {
    +          "data_change": {
    +            "type": "CONTINUATION",
    +            "token": "data-change-token",
    +            "low_watermark": "2022-07-01T00:00:00Z",
    +            "chunks": [
    +              {
    +                "chunk_info": {
    +                  "chunked_value_offset": 1,
    +                  "chunked_value_size": 9,
    +                  "last_chunk": true
    +                },
    +                "mutation": {
    +                  "set_cell": {
    +                    "value": "YWx1ZS1WQUw="
    +                  }
    +                }
    +              }
    +            ],
    +            "done": true
    +          }
    +        }
    +      ],
    +      "results": [
    +        {
    +          "error": true
    +        }
    +      ]
    +    },
         {
           "description": "1 chunked SetCell([{SC_chunk1(v)}, {SC_chunk2(alue-VAL)}, {SC_chunk3(-VAL)}]->ChangeStreamMutation{SC(value-VAL-VAL)})",
           "api_responses": [
    @@ -535,7 +591,7 @@
                 "chunks": [
                   {
                     "chunk_info": {
    -                  "chunked_value_size": 1
    +                  "chunked_value_size": 9
                     },
                     "mutation": {
                       "set_cell": {
    @@ -567,7 +623,7 @@
                   },
                   {
                     "chunk_info": {
    -                  "chunked_value_size": 1
    +                  "chunked_value_size": 9
                     },
                     "mutation": {
                       "set_cell": {
    diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java
    index a207589a2c..d847d06ba7 100644
    --- a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java
    +++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java
    @@ -52,7 +52,13 @@
     import org.junit.runner.RunWith;
     import org.junit.runners.Parameterized;
     
    -/** Parses and runs the acceptance tests for read rows */
    +/**
    + * Parses and runs the acceptance tests for read rows. Currently, this test is only used by the JAVA
    + * library. If in the future we need cross-language support, we should move the test proto to https://github.com/googleapis/conformance-tests/tree/main/bigtable/v2/proto/google/cloud/conformance/bigtable/v2
    + * and the test data to https://github.com/googleapis/conformance-tests/blob/main/bigtable/v2/readrows.json
    + */
     @RunWith(Parameterized.class)
     public class ReadChangeStreamMergingAcceptanceTest {
       // Location: `google-cloud-bigtable/src/main/resources/changestream.json`
    
    From a181c36f856314730d7d66f804c0f8c3cc1e3fad Mon Sep 17 00:00:00 2001
    From: Teng Zhong 
    Date: Mon, 8 Aug 2022 14:13:41 -0400
    Subject: [PATCH 10/10] fix: Fix comment
    
    ---
     .../changestream/ReadChangeStreamMergingAcceptanceTest.java  | 5 +++--
     1 file changed, 3 insertions(+), 2 deletions(-)
    
    diff --git a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java
    index d847d06ba7..5ae88a7f9f 100644
    --- a/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java
    +++ b/google-cloud-bigtable/src/test/java/com/google/cloud/bigtable/data/v2/stub/changestream/ReadChangeStreamMergingAcceptanceTest.java
    @@ -53,8 +53,9 @@
     import org.junit.runners.Parameterized;
     
     /**
    - * Parses and runs the acceptance tests for read rows. Currently, this test is only used by the JAVA
    - * library. If in the future we need cross-language support, we should move the test proto to https://github.com/googleapis/conformance-tests/tree/main/bigtable/v2/proto/google/cloud/conformance/bigtable/v2
      * and the test data to https://github.com/googleapis/conformance-tests/blob/main/bigtable/v2/readrows.json