From b714b84897a2cd7912e8278a609453516cf0a9fb Mon Sep 17 00:00:00 2001 From: Radek Stankiewicz Date: Thu, 12 Feb 2026 15:09:22 +0100 Subject: [PATCH 1/7] feat: Make Spanner change stream query parameters configurable with `cdcTimeIncrement` and `heartbeatMillis`; add a low-latency option; and update heartbeat record action to explicitly resume processing. --- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 34 +- .../changestreams/action/ActionFactory.java | 181 +- .../action/HeartbeatRecordAction.java | 2 +- .../action/QueryChangeStreamAction.java | 332 +-- .../changestreams/dofn/InitializeDoFn.java | 12 +- .../dofn/ReadChangeStreamPartitionDoFn.java | 179 +- .../action/HeartbeatRecordActionTest.java | 2 +- .../action/QueryChangeStreamActionTest.java | 1924 ++++++++--------- .../dofn/InitializeDoFnTest.java | 3 +- .../ReadChangeStreamPartitionDoFnTest.java | 271 +-- 10 files changed, 1519 insertions(+), 1421 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index 3a69d1177f4a..9b6c22b024bf 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -537,6 +537,8 @@ public static ReadChangeStream readChangeStream() { .setRpcPriority(DEFAULT_RPC_PRIORITY) .setInclusiveStartAt(DEFAULT_INCLUSIVE_START_AT) .setInclusiveEndAt(DEFAULT_INCLUSIVE_END_AT) + .setCdcTimeIncrement(Duration.standardMinutes(2)) + .setHeartbeatMillis(2000) .build(); } @@ -1761,6 +1763,10 @@ public abstract static class ReadChangeStream abstract @Nullable ValueProvider getPlainText(); + abstract Duration getCdcTimeIncrement(); + + abstract Integer getHeartbeatMillis(); + abstract Builder toBuilder(); @AutoValue.Builder @@ -1790,6 +1796,17 @@ abstract static class Builder { abstract Builder setPlainText(ValueProvider plainText); + abstract Builder setCdcTimeIncrement(Duration cdcTimeIncrement); + + /** + * Heartbeat interval for all change stream queries. + * + *

Be careful when changing this interval, as it needs to be less than the checkpointing + * interval in Dataflow. Otherwise, if there are no records within checkpoint intervals, the + * consuming of a change stream query might get stuck. + */ + abstract Builder setHeartbeatMillis(Integer heartbeatMillis); + abstract ReadChangeStream build(); } @@ -1912,6 +1929,13 @@ public ReadChangeStream withUsingPlainTextChannel(boolean plainText) { return withUsingPlainTextChannel(ValueProvider.StaticValueProvider.of(plainText)); } + public ReadChangeStream withLowLatency() { + return toBuilder() + .setCdcTimeIncrement(Duration.standardSeconds(1)) + .setHeartbeatMillis(100) + .build(); + } + @Override public PCollection expand(PBegin input) { checkArgument( @@ -2018,13 +2042,19 @@ && getInclusiveStartAt().toSqlTimestamp().after(getInclusiveEndAt().toSqlTimesta MoreObjects.firstNonNull(getWatermarkRefreshRate(), DEFAULT_WATERMARK_REFRESH_RATE); final CacheFactory cacheFactory = new CacheFactory(daoFactory, watermarkRefreshRate); + final long heartbeatMillis = getHeartbeatMillis().longValue(); + final InitializeDoFn initializeDoFn = - new InitializeDoFn(daoFactory, mapperFactory, startTimestamp, endTimestamp); + new InitializeDoFn( + daoFactory, mapperFactory, startTimestamp, endTimestamp, heartbeatMillis); final DetectNewPartitionsDoFn detectNewPartitionsDoFn = new DetectNewPartitionsDoFn( daoFactory, mapperFactory, actionFactory, cacheFactory, metrics); + final Duration cdcTimeIncrement = getCdcTimeIncrement(); + final ReadChangeStreamPartitionDoFn readChangeStreamPartitionDoFn = - new ReadChangeStreamPartitionDoFn(daoFactory, mapperFactory, actionFactory, metrics); + new ReadChangeStreamPartitionDoFn( + daoFactory, mapperFactory, actionFactory, metrics, cdcTimeIncrement); final PostProcessingMetricsDoFn postProcessingMetricsDoFn = new PostProcessingMetricsDoFn(metrics); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ActionFactory.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ActionFactory.java index cd84168b23f7..8b8ba6c4f51f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ActionFactory.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ActionFactory.java @@ -29,10 +29,12 @@ import org.joda.time.Duration; /** - * Factory class for creating instances that will handle each type of record within a change stream + * Factory class for creating instances that will handle each type of record + * within a change stream * query. The instances created are all singletons. */ -// transient fields are un-initialized, because we start them during the first fetch call (with the +// transient fields are un-initialized, because we start them during the first +// fetch call (with the // singleton pattern). @SuppressWarnings("initialization.field.uninitialized") public class ActionFactory implements Serializable { @@ -48,10 +50,12 @@ public class ActionFactory implements Serializable { private transient DetectNewPartitionsAction detectNewPartitionsActionInstance; /** - * Creates and returns a singleton instance of an action class capable of processing {@link + * Creates and returns a singleton instance of an action class capable of + * processing {@link * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.DataChangeRecord}s. * - *

This method is thread safe. + *

+ * This method is thread safe. * * @return singleton instance of the {@link DataChangeRecordAction} */ @@ -64,8 +68,10 @@ public synchronized DataChangeRecordAction dataChangeRecordAction( } /** - * Creates and returns a singleton instance of an action class capable of processing {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.HeartbeatRecord}s. This method is thread + * Creates and returns a singleton instance of an action class capable of + * processing {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.HeartbeatRecord}s. + * This method is thread * safe. * * @param metrics metrics gathering class @@ -79,48 +85,55 @@ public synchronized HeartbeatRecordAction heartbeatRecordAction(ChangeStreamMetr } /** - * Creates and returns a singleton instance of an action class capable of process {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.ChildPartitionsRecord}s. This method is + * Creates and returns a singleton instance of an action class capable of + * process {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.ChildPartitionsRecord}s. + * This method is * thread safe. * - * @param partitionMetadataDao DAO class to access the Connector's metadata tables - * @param metrics metrics gathering class + * @param partitionMetadataDao DAO class to access the Connector's metadata + * tables + * @param metrics metrics gathering class * @return singleton instance of the {@link ChildPartitionsRecordAction} */ public synchronized ChildPartitionsRecordAction childPartitionsRecordAction( PartitionMetadataDao partitionMetadataDao, ChangeStreamMetrics metrics) { if (childPartitionsRecordActionInstance == null) { - childPartitionsRecordActionInstance = - new ChildPartitionsRecordAction(partitionMetadataDao, metrics); + childPartitionsRecordActionInstance = new ChildPartitionsRecordAction(partitionMetadataDao, metrics); } return childPartitionsRecordActionInstance; } /** - * Creates and returns a singleton instance of an action class capable of process {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionStartRecord}s. This method is + * Creates and returns a singleton instance of an action class capable of + * process {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionStartRecord}s. + * This method is * thread safe. * - * @param partitionMetadataDao DAO class to access the Connector's metadata tables - * @param metrics metrics gathering class + * @param partitionMetadataDao DAO class to access the Connector's metadata + * tables + * @param metrics metrics gathering class * @return singleton instance of the {@link PartitionStartRecordAction} */ public synchronized PartitionStartRecordAction partitionStartRecordAction( PartitionMetadataDao partitionMetadataDao, ChangeStreamMetrics metrics) { if (partitionStartRecordActionInstance == null) { - partitionStartRecordActionInstance = - new PartitionStartRecordAction(partitionMetadataDao, metrics); + partitionStartRecordActionInstance = new PartitionStartRecordAction(partitionMetadataDao, metrics); } return partitionStartRecordActionInstance; } /** - * Creates and returns a singleton instance of an action class capable of process {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionEndRecord}s. This method is + * Creates and returns a singleton instance of an action class capable of + * process {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionEndRecord}s. + * This method is * thread safe. * - * @param partitionMetadataDao DAO class to access the Connector's metadata tables - * @param metrics metrics gathering class + * @param partitionMetadataDao DAO class to access the Connector's metadata + * tables + * @param metrics metrics gathering class * @return singleton instance of the {@link PartitionEndRecordAction} */ public synchronized PartitionEndRecordAction partitionEndRecordAction( @@ -132,12 +145,15 @@ public synchronized PartitionEndRecordAction partitionEndRecordAction( } /** - * Creates and returns a singleton instance of an action class capable of process {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionEventRecord}s. This method is + * Creates and returns a singleton instance of an action class capable of + * process {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionEventRecord}s. + * This method is * thread safe. * - * @param partitionMetadataDao DAO class to access the Connector's metadata tables - * @param metrics metrics gathering class + * @param partitionMetadataDao DAO class to access the Connector's metadata + * tables + * @param metrics metrics gathering class * @return singleton instance of the {@link PartitionEventRecordAction} */ public synchronized PartitionEventRecordAction partitionEventRecordAction( @@ -149,31 +165,40 @@ public synchronized PartitionEventRecordAction partitionEventRecordAction( } /** - * Creates and returns a single instance of an action class capable of performing a change stream - * query for a given partition. It uses the {@link DataChangeRecordAction}, {@link - * HeartbeatRecordAction}, {@link ChildPartitionsRecordAction}, {@link PartitionStartRecordAction} - * ,{@link PartitionEndRecordAction} and {@link PartitionEventRecordAction} to dispatch the + * Creates and returns a single instance of an action class capable of + * performing a change stream + * query for a given partition. It uses the {@link DataChangeRecordAction}, + * {@link + * HeartbeatRecordAction}, {@link ChildPartitionsRecordAction}, + * {@link PartitionStartRecordAction} + * ,{@link PartitionEndRecordAction} and {@link PartitionEventRecordAction} to + * dispatch the * necessary processing depending on the type of record received. * - * @param changeStreamDao DAO class to perform a change stream query - * @param partitionMetadataDao DAO class to access the Connector's metadata tables - * @param changeStreamRecordMapper mapper class to transform change stream records into the - * Connector's domain models - * @param partitionMetadataMapper mapper class to transform partition metadata rows into the - * Connector's domain models - * @param dataChangeRecordAction action class to process {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.DataChangeRecord}s - * @param heartbeatRecordAction action class to process {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.HeartbeatRecord}s + * @param changeStreamDao DAO class to perform a change stream query + * @param partitionMetadataDao DAO class to access the Connector's + * metadata tables + * @param changeStreamRecordMapper mapper class to transform change stream + * records into the + * Connector's domain models + * @param partitionMetadataMapper mapper class to transform partition + * metadata rows into the + * Connector's domain models + * @param dataChangeRecordAction action class to process {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.DataChangeRecord}s + * @param heartbeatRecordAction action class to process {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.HeartbeatRecord}s * @param childPartitionsRecordAction action class to process {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.ChildPartitionsRecord}s - * @param partitionStartRecordAction action class to process {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionStartRecord}s - * @param partitionEndRecordAction action class to process {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionEndRecord}s - * @param partitionEventRecordAction action class to process {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionEventRecord}s - * @param metrics metrics gathering class + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.ChildPartitionsRecord}s + * @param partitionStartRecordAction action class to process {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionStartRecord}s + * @param partitionEndRecordAction action class to process {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionEndRecord}s + * @param partitionEventRecordAction action class to process {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionEventRecord}s + * @param metrics metrics gathering class + * @param cdcTimeIncrement the duration added to current time for the + * end timestamp * @return single instance of the {@link QueryChangeStreamAction} */ public synchronized QueryChangeStreamAction queryChangeStreamAction( @@ -188,35 +213,40 @@ public synchronized QueryChangeStreamAction queryChangeStreamAction( PartitionEndRecordAction partitionEndRecordAction, PartitionEventRecordAction partitionEventRecordAction, ChangeStreamMetrics metrics, - boolean isMutableChangeStream) { + boolean isMutableChangeStream, + Duration cdcTimeIncrement) { if (queryChangeStreamActionInstance == null) { - queryChangeStreamActionInstance = - new QueryChangeStreamAction( - changeStreamDao, - partitionMetadataDao, - changeStreamRecordMapper, - partitionMetadataMapper, - dataChangeRecordAction, - heartbeatRecordAction, - childPartitionsRecordAction, - partitionStartRecordAction, - partitionEndRecordAction, - partitionEventRecordAction, - metrics, - isMutableChangeStream); + queryChangeStreamActionInstance = new QueryChangeStreamAction( + changeStreamDao, + partitionMetadataDao, + changeStreamRecordMapper, + partitionMetadataMapper, + dataChangeRecordAction, + heartbeatRecordAction, + childPartitionsRecordAction, + partitionStartRecordAction, + partitionEndRecordAction, + partitionEventRecordAction, + metrics, + isMutableChangeStream, + cdcTimeIncrement); } return queryChangeStreamActionInstance; } /** - * Creates and returns a single instance of an action class capable of detecting and scheduling + * Creates and returns a single instance of an action class capable of detecting + * and scheduling * new partitions to be queried. * - * @param partitionMetadataDao DAO class to access the Connector's metadata tables - * @param partitionMetadataMapper mapper class to transform partition metadata table rows into the - * Connector's domain models - * @param metrics metrics gathering class - * @param resumeDuration specifies the periodic schedule to re-execute the action + * @param partitionMetadataDao DAO class to access the Connector's metadata + * tables + * @param partitionMetadataMapper mapper class to transform partition metadata + * table rows into the + * Connector's domain models + * @param metrics metrics gathering class + * @param resumeDuration specifies the periodic schedule to re-execute + * the action * @return single instance of the {@link DetectNewPartitionsAction} */ public synchronized DetectNewPartitionsAction detectNewPartitionsAction( @@ -226,13 +256,12 @@ public synchronized DetectNewPartitionsAction detectNewPartitionsAction( ChangeStreamMetrics metrics, Duration resumeDuration) { if (detectNewPartitionsActionInstance == null) { - detectNewPartitionsActionInstance = - new DetectNewPartitionsAction( - partitionMetadataDao, - partitionMetadataMapper, - watermarkCache, - metrics, - resumeDuration); + detectNewPartitionsActionInstance = new DetectNewPartitionsAction( + partitionMetadataDao, + partitionMetadataMapper, + watermarkCache, + metrics, + resumeDuration); } return detectNewPartitionsActionInstance; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/HeartbeatRecordAction.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/HeartbeatRecordAction.java index 0937e896fbf1..2a6ceb311c75 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/HeartbeatRecordAction.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/HeartbeatRecordAction.java @@ -96,6 +96,6 @@ public Optional run( watermarkEstimator.setWatermark(timestampInstant); LOG.debug("[{}] Heartbeat record action completed successfully", token); - return Optional.empty(); + return Optional.of(ProcessContinuation.resume()); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamAction.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamAction.java index 69e89e74a38b..cedc172a4ee6 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamAction.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamAction.java @@ -54,16 +54,24 @@ import org.slf4j.LoggerFactory; /** - * Main action class for querying a partition change stream. This class will perform the change - * stream query and depending on the record type received, it will dispatch the processing of it to - * one of the following: {@link ChildPartitionsRecordAction}, {@link HeartbeatRecordAction}, {@link - * DataChangeRecordAction}, {@link PartitionStartRecordAction}, {@link PartitionEndRecordAction} or + * Main action class for querying a partition change stream. This class will + * perform the change + * stream query and depending on the record type received, it will dispatch the + * processing of it to + * one of the following: {@link ChildPartitionsRecordAction}, + * {@link HeartbeatRecordAction}, {@link + * DataChangeRecordAction}, {@link PartitionStartRecordAction}, + * {@link PartitionEndRecordAction} or * {@link PartitionEventRecordAction}. * - *

This class will also make sure to mirror the current watermark (event timestamp processed) in + *

+ * This class will also make sure to mirror the current watermark (event + * timestamp processed) in * the Connector's metadata tables, by registering a bundle after commit action. * - *

When the change stream query for the partition is finished, this class will update the state + *

+ * When the change stream query for the partition is finished, this class will + * update the state * of the partition in the metadata tables as FINISHED, indicating completion. */ public class QueryChangeStreamAction { @@ -71,9 +79,12 @@ public class QueryChangeStreamAction { private static final Logger LOG = LoggerFactory.getLogger(QueryChangeStreamAction.class); private static final Duration BUNDLE_FINALIZER_TIMEOUT = Duration.standardMinutes(5); /* - * Corresponds to the best effort timeout in case the restriction tracker cannot split the processing - * interval before the hard deadline. When reached it will assure that the already processed timestamps - * will be committed instead of thrown away (DEADLINE_EXCEEDED). The value should be less than + * Corresponds to the best effort timeout in case the restriction tracker cannot + * split the processing + * interval before the hard deadline. When reached it will assure that the + * already processed timestamps + * will be committed instead of thrown away (DEADLINE_EXCEEDED). The value + * should be less than * the RetrySetting RPC timeout setting of SpannerIO#ReadChangeStream. */ private static final Duration RESTRICTION_TRACKER_TIMEOUT = Duration.standardSeconds(40); @@ -91,24 +102,37 @@ public class QueryChangeStreamAction { private final PartitionEventRecordAction partitionEventRecordAction; private final ChangeStreamMetrics metrics; private final boolean isMutableChangeStream; + private final Duration cdcTimeIncrement; /** - * Constructs an action class for performing a change stream query for a given partition. + * Constructs an action class for performing a change stream query for a given + * partition. * - * @param changeStreamDao DAO class to perform a change stream query - * @param partitionMetadataDao DAO class to access the Connector's metadata tables - * @param changeStreamRecordMapper mapper class to transform change stream records into the - * Connector's domain models - * @param partitionMetadataMapper mapper class to transform partition metadata rows into the - * Connector's domain models - * @param dataChangeRecordAction action class to process {@link DataChangeRecord}s - * @param heartbeatRecordAction action class to process {@link HeartbeatRecord}s - * @param childPartitionsRecordAction action class to process {@link ChildPartitionsRecord}s - * @param PartitionStartRecordAction action class to process {@link PartitionStartRecord}s - * @param PartitionEndRecordAction action class to process {@link PartitionEndRecord}s - * @param PartitionEventRecordAction action class to process {@link PartitionEventRecord}s - * @param metrics metrics gathering class - * @param isMutableChangeStream whether the change stream is mutable or not + * @param changeStreamDao DAO class to perform a change stream query + * @param partitionMetadataDao DAO class to access the Connector's + * metadata tables + * @param changeStreamRecordMapper mapper class to transform change stream + * records into the + * Connector's domain models + * @param partitionMetadataMapper mapper class to transform partition + * metadata rows into the + * Connector's domain models + * @param dataChangeRecordAction action class to process + * {@link DataChangeRecord}s + * @param heartbeatRecordAction action class to process + * {@link HeartbeatRecord}s + * @param childPartitionsRecordAction action class to process + * {@link ChildPartitionsRecord}s + * @param PartitionStartRecordAction action class to process + * {@link PartitionStartRecord}s + * @param PartitionEndRecordAction action class to process + * {@link PartitionEndRecord}s + * @param PartitionEventRecordAction action class to process + * {@link PartitionEventRecord}s + * @param metrics metrics gathering class + * @param isMutableChangeStream whether the change stream is mutable or + * not + * @param cdcTimeIncrement duration to add to current time */ QueryChangeStreamAction( ChangeStreamDao changeStreamDao, @@ -122,56 +146,69 @@ public class QueryChangeStreamAction { PartitionEndRecordAction partitionEndRecordAction, PartitionEventRecordAction partitionEventRecordAction, ChangeStreamMetrics metrics, - boolean isMutableChangeStream) { + boolean isMutableChangeStream, + Duration cdcTimeIncrement) { this.changeStreamDao = changeStreamDao; - this.partitionMetadataDao = partitionMetadataDao; this.changeStreamRecordMapper = changeStreamRecordMapper; - this.partitionMetadataMapper = partitionMetadataMapper; this.dataChangeRecordAction = dataChangeRecordAction; - this.heartbeatRecordAction = heartbeatRecordAction; this.childPartitionsRecordAction = childPartitionsRecordAction; this.partitionStartRecordAction = partitionStartRecordAction; this.partitionEndRecordAction = partitionEndRecordAction; this.partitionEventRecordAction = partitionEventRecordAction; this.metrics = metrics; this.isMutableChangeStream = isMutableChangeStream; + this.cdcTimeIncrement = cdcTimeIncrement; } /** - * This method will dispatch a change stream query for the given partition, it delegate the - * processing of the records to one of the corresponding action classes registered and it will + * This method will dispatch a change stream query for the given partition, it + * delegate the + * processing of the records to one of the corresponding action classes + * registered and it will * keep the state of the partition up to date in the Connector's metadata table. * - *

The algorithm is as follows: + *

+ * The algorithm is as follows: * *

    - *
  1. A change stream query for the partition is performed. - *
  2. For each record, we check the type of the record and dispatch the processing to one of - * the actions registered. - *
  3. If an {@link Optional} with a {@link ProcessContinuation#stop()} is returned from the - * actions, we stop processing and return. - *
  4. Before returning we register a bundle finalizer callback to update the watermark of the - * partition in the metadata tables to the latest processed timestamp. - *
  5. When a change stream query finishes successfully (no more records) we update the - * partition state to FINISHED. + *
  6. A change stream query for the partition is performed. + *
  7. For each record, we check the type of the record and dispatch the + * processing to one of + * the actions registered. + *
  8. If an {@link Optional} with a {@link ProcessContinuation#stop()} is + * returned from the + * actions, we stop processing and return. + *
  9. Before returning we register a bundle finalizer callback to update the + * watermark of the + * partition in the metadata tables to the latest processed timestamp. + *
  10. When a change stream query finishes successfully (no more records) we + * update the + * partition state to FINISHED. *
* - * There might be cases where due to a split at the exact end timestamp of a partition's change - * stream query, this function could process a residual with an invalid timestamp. In this case, + * There might be cases where due to a split at the exact end timestamp of a + * partition's change + * stream query, this function could process a residual with an invalid + * timestamp. In this case, * the error is ignored and no work is done for the residual. * - * @param partition the current partition being processed - * @param tracker the restriction tracker of the {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.dofn.ReadChangeStreamPartitionDoFn} SDF - * @param receiver the output receiver of the {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.dofn.ReadChangeStreamPartitionDoFn} SDF + * @param partition the current partition being processed + * @param tracker the restriction tracker of the {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.dofn.ReadChangeStreamPartitionDoFn} + * SDF + * @param receiver the output receiver of the {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.dofn.ReadChangeStreamPartitionDoFn} + * SDF * @param watermarkEstimator the watermark estimator of the {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.dofn.ReadChangeStreamPartitionDoFn} SDF - * @param bundleFinalizer the bundle finalizer for {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.dofn.ReadChangeStreamPartitionDoFn} SDF - * bundles - * @return a {@link ProcessContinuation#stop()} if a record timestamp could not be claimed or if - * the partition processing has finished + * org.apache.beam.sdk.io.gcp.spanner.changestreams.dofn.ReadChangeStreamPartitionDoFn} + * SDF + * @param bundleFinalizer the bundle finalizer for {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.dofn.ReadChangeStreamPartitionDoFn} + * SDF + * bundles + * @return a {@link ProcessContinuation#stop()} if a record timestamp could not + * be claimed or if + * the partition processing has finished */ @VisibleForTesting public ProcessContinuation run( @@ -182,108 +219,101 @@ public ProcessContinuation run( BundleFinalizer bundleFinalizer) { final String token = partition.getPartitionToken(); - // TODO: Potentially we can avoid this fetch, by enriching the runningAt timestamp when the + // TODO: Potentially we can avoid this fetch, by enriching the runningAt + // timestamp when the // ReadChangeStreamPartitionDoFn#processElement is called - final PartitionMetadata updatedPartition = - Optional.ofNullable(partitionMetadataDao.getPartition(token)) - .map(partitionMetadataMapper::from) - .orElseThrow( - () -> - new IllegalStateException( - "Partition " + token + " not found in metadata table")); + final PartitionMetadata updatedPartition = Optional.ofNullable(partitionMetadataDao.getPartition(token)) + .map(partitionMetadataMapper::from) + .orElseThrow( + () -> new IllegalStateException( + "Partition " + token + " not found in metadata table")); - // Interrupter with soft timeout to commit the work if any records have been processed. - RestrictionInterrupter interrupter = - RestrictionInterrupter.withSoftTimeout(RESTRICTION_TRACKER_TIMEOUT); + // Interrupter with soft timeout to commit the work if any records have been + // processed. + RestrictionInterrupter interrupter = RestrictionInterrupter.withSoftTimeout(RESTRICTION_TRACKER_TIMEOUT); final Timestamp startTimestamp = tracker.currentRestriction().getFrom(); final Timestamp endTimestamp = partition.getEndTimestamp(); final boolean isBoundedRestriction = !endTimestamp.equals(MAX_INCLUSIVE_END_AT); - final Timestamp changeStreamQueryEndTimestamp = - isBoundedRestriction - ? getBoundedQueryEndTimestamp(endTimestamp) - : getNextReadChangeStreamEndTimestamp(); + final Timestamp changeStreamQueryEndTimestamp = isBoundedRestriction + ? getBoundedQueryEndTimestamp(endTimestamp) + : getNextReadChangeStreamEndTimestamp(); - // Once the changeStreamQuery completes we may need to resume reading from the partition if we - // had an unbounded restriction for which we set an arbitrary query end timestamp and for which - // we didn't encounter any indications that the partition is done (explicit end records or - // exceptions about being out of timestamp range). We also special case the InitialPartition, + // Once the changeStreamQuery completes we may need to resume reading from the + // partition if we + // had an unbounded restriction for which we set an arbitrary query end + // timestamp and for which + // we didn't encounter any indications that the partition is done (explicit end + // records or + // exceptions about being out of timestamp range). We also special case the + // InitialPartition, // which always stops after the query succeeds. boolean stopAfterQuerySucceeds = false; if (InitialPartition.isInitialPartition(partition.getPartitionToken())) { stopAfterQuerySucceeds = true; } else { - stopAfterQuerySucceeds = - isBoundedRestriction && changeStreamQueryEndTimestamp.equals(endTimestamp); + stopAfterQuerySucceeds = isBoundedRestriction && changeStreamQueryEndTimestamp.equals(endTimestamp); } - try (ChangeStreamResultSet resultSet = - changeStreamDao.changeStreamQuery( - token, startTimestamp, changeStreamQueryEndTimestamp, partition.getHeartbeatMillis())) { + try (ChangeStreamResultSet resultSet = changeStreamDao.changeStreamQuery( + token, startTimestamp, changeStreamQueryEndTimestamp, partition.getHeartbeatMillis())) { metrics.incQueryCounter(); while (resultSet.next()) { - final List records = - changeStreamRecordMapper.toChangeStreamRecords( - updatedPartition, resultSet, resultSet.getMetadata()); + final List records = changeStreamRecordMapper.toChangeStreamRecords( + updatedPartition, resultSet, resultSet.getMetadata()); Optional maybeContinuation; for (final ChangeStreamRecord record : records) { if (record instanceof DataChangeRecord) { - maybeContinuation = - dataChangeRecordAction.run( - updatedPartition, - (DataChangeRecord) record, - tracker, - interrupter, - receiver, - watermarkEstimator); + maybeContinuation = dataChangeRecordAction.run( + updatedPartition, + (DataChangeRecord) record, + tracker, + interrupter, + receiver, + watermarkEstimator); } else if (record instanceof HeartbeatRecord) { - maybeContinuation = - heartbeatRecordAction.run( - updatedPartition, - (HeartbeatRecord) record, - tracker, - interrupter, - watermarkEstimator); + maybeContinuation = heartbeatRecordAction.run( + updatedPartition, + (HeartbeatRecord) record, + tracker, + interrupter, + watermarkEstimator); } else if (record instanceof ChildPartitionsRecord) { - maybeContinuation = - childPartitionsRecordAction.run( - updatedPartition, - (ChildPartitionsRecord) record, - tracker, - interrupter, - watermarkEstimator); + maybeContinuation = childPartitionsRecordAction.run( + updatedPartition, + (ChildPartitionsRecord) record, + tracker, + interrupter, + watermarkEstimator); // Child Partition records indicate that the partition has ended. There may be // additional ChildPartitionRecords but they will share the same timestamp and // will be returned by the query and processed if it finishes successfully. stopAfterQuerySucceeds = true; } else if (record instanceof PartitionStartRecord) { - maybeContinuation = - partitionStartRecordAction.run( - updatedPartition, - (PartitionStartRecord) record, - tracker, - interrupter, - watermarkEstimator); + maybeContinuation = partitionStartRecordAction.run( + updatedPartition, + (PartitionStartRecord) record, + tracker, + interrupter, + watermarkEstimator); } else if (record instanceof PartitionEndRecord) { - maybeContinuation = - partitionEndRecordAction.run( - updatedPartition, - (PartitionEndRecord) record, - tracker, - interrupter, - watermarkEstimator); + maybeContinuation = partitionEndRecordAction.run( + updatedPartition, + (PartitionEndRecord) record, + tracker, + interrupter, + watermarkEstimator); // The PartitionEndRecord indicates that there are no more records expected // for this partition. stopAfterQuerySucceeds = true; } else if (record instanceof PartitionEventRecord) { - maybeContinuation = - partitionEventRecordAction.run( - updatedPartition, - (PartitionEventRecord) record, - tracker, - interrupter, - watermarkEstimator); + maybeContinuation = partitionEventRecordAction.run( + updatedPartition, + (PartitionEventRecord) record, + tracker, + interrupter, + watermarkEstimator); } else { LOG.error("[{}] Unknown record type {}", token, record.getClass()); throw new IllegalArgumentException("Unknown record type " + record.getClass()); @@ -300,11 +330,14 @@ public ProcessContinuation run( } } catch (SpannerException e) { /* - If there is a split when a partition is supposed to be finished, the residual will try - to perform a change stream query for an out of range interval. We ignore this error - here, and the residual should be able to claim the end of the timestamp range, finishing - the partition. - */ + * If there is a split when a partition is supposed to be finished, the residual + * will try + * to perform a change stream query for an out of range interval. We ignore this + * error + * here, and the residual should be able to claim the end of the timestamp + * range, finishing + * the partition. + */ if (!isTimestampOutOfRange(e)) { throw e; } @@ -329,8 +362,10 @@ public ProcessContinuation run( "[{}] change stream completed successfully up to {}", token, changeStreamQueryEndTimestamp); if (!stopAfterQuerySucceeds) { - // Records stopped being returned for the query due to our artificial query end timestamp but - // we want to continue processing the partition, resuming from changeStreamQueryEndTimestamp. + // Records stopped being returned for the query due to our artificial query end + // timestamp but + // we want to continue processing the partition, resuming from + // changeStreamQueryEndTimestamp. if (!tracker.tryClaim(changeStreamQueryEndTimestamp)) { return ProcessContinuation.stop(); } @@ -342,20 +377,25 @@ public ProcessContinuation run( } // Otherwise we have finished processing the partition, either due to: - // 1. reading to the bounded restriction end timestamp - // 2. encountering a ChildPartitionRecord or EndPartitionRecord indicating there are no more - // elements in the partition - // 3. encountering a exception indicating the start timestamp is out of bounds of the - // partition - // We claim the restriction completely to satisfy internal sanity checks and do not reschedule + // 1. reading to the bounded restriction end timestamp + // 2. encountering a ChildPartitionRecord or EndPartitionRecord indicating there + // are no more + // elements in the partition + // 3. encountering a exception indicating the start timestamp is out of bounds + // of the + // partition + // We claim the restriction completely to satisfy internal sanity checks and do + // not reschedule // the restriction. if (!tracker.tryClaim(endTimestamp)) { return ProcessContinuation.stop(); } LOG.debug("[{}] Finishing partition", token); - // TODO: This should be performed after the commit succeeds. Since bundle finalizers are not - // guaranteed to be called, this needs to be performed in a subsequent fused stage. + // TODO: This should be performed after the commit succeeds. Since bundle + // finalizers are not + // guaranteed to be called, this needs to be performed in a subsequent fused + // stage. partitionMetadataDao.updateToFinished(token); metrics.decActivePartitionReadCounter(); LOG.info("[{}] After attempting to finish the partition", token); @@ -382,20 +422,26 @@ private BundleFinalizer.Callback updateWatermarkCallback( private boolean isTimestampOutOfRange(SpannerException e) { return (e.getErrorCode() == ErrorCode.INVALID_ARGUMENT - || e.getErrorCode() == ErrorCode.OUT_OF_RANGE) + || e.getErrorCode() == ErrorCode.OUT_OF_RANGE) && e.getMessage() != null && e.getMessage().contains(OUT_OF_RANGE_ERROR_MESSAGE); } - // Return (now + 2 mins) as the end timestamp for reading change streams. This is only used if - // users want to run the connector forever. If the end timestamp is reached, we will resume + // Return (now + config duration) as the end timestamp for reading change + // streams. This is only + // used if + // users want to run the connector forever. If the end timestamp is reached, we + // will resume // processing from that timestamp on a subsequent DoFn execution. private Timestamp getNextReadChangeStreamEndTimestamp() { final Timestamp current = Timestamp.now(); - return Timestamp.ofTimeSecondsAndNanos(current.getSeconds() + 2 * 60, current.getNanos()); + long seconds = current.getSeconds() + cdcTimeIncrement.getStandardSeconds(); + int nanos = current.getNanos() + (int) ((cdcTimeIncrement.getMillis() % 1000) * 1_000_000); + return Timestamp.ofTimeSecondsAndNanos(seconds, nanos); } - // For Mutable Change Stream bounded queries, update the query end timestamp to be within 2 + // For Mutable Change Stream bounded queries, update the query end timestamp to + // be within 2 // minutes in the future. private Timestamp getBoundedQueryEndTimestamp(Timestamp endTimestamp) { if (this.isMutableChangeStream) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/InitializeDoFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/InitializeDoFn.java index 60eb96ca3387..4191f2d93594 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/InitializeDoFn.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/InitializeDoFn.java @@ -36,11 +36,7 @@ public class InitializeDoFn extends DoFn implements S private static final long serialVersionUID = -8921188388649003102L; - /** Heartbeat interval for all change stream queries will be of 2 seconds. */ - // Be careful when changing this interval, as it needs to be less than the checkpointing interval - // in Dataflow. Otherwise, if there are no records within checkpoint intervals, the consuming of - // a change stream query might get stuck. - private static final long DEFAULT_HEARTBEAT_MILLIS = 2000; + private final long heartbeatMillis; private final DaoFactory daoFactory; private final MapperFactory mapperFactory; @@ -53,11 +49,13 @@ public InitializeDoFn( DaoFactory daoFactory, MapperFactory mapperFactory, com.google.cloud.Timestamp startTimestamp, - com.google.cloud.Timestamp endTimestamp) { + com.google.cloud.Timestamp endTimestamp, + long heartbeatMillis) { this.daoFactory = daoFactory; this.mapperFactory = mapperFactory; this.startTimestamp = startTimestamp; this.endTimestamp = endTimestamp; + this.heartbeatMillis = heartbeatMillis; } @ProcessElement @@ -88,7 +86,7 @@ private void createFakeParentPartition() { .setPartitionToken(InitialPartition.PARTITION_TOKEN) .setStartTimestamp(startTimestamp) .setEndTimestamp(endTimestamp) - .setHeartbeatMillis(DEFAULT_HEARTBEAT_MILLIS) + .setHeartbeatMillis(heartbeatMillis) .setState(State.CREATED) .setWatermark(startTimestamp) .build(); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFn.java index c3650b42761b..dfee2cf077d3 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFn.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFn.java @@ -53,11 +53,16 @@ import org.slf4j.LoggerFactory; /** - * A SDF (Splittable DoFn) class which is responsible for performing a change stream query for a - * given partition. A different action will be taken depending on the type of record received from - * the query. This component will also reflect the partition state in the partition metadata tables. + * A SDF (Splittable DoFn) class which is responsible for performing a change + * stream query for a + * given partition. A different action will be taken depending on the type of + * record received from + * the query. This component will also reflect the partition state in the + * partition metadata tables. * - *

The processing of a partition is delegated to the {@link QueryChangeStreamAction}. + *

+ * The processing of a partition is delegated to the + * {@link QueryChangeStreamAction}. */ // Allows for transient QueryChangeStreamAction @SuppressWarnings("initialization.fields.uninitialized") @@ -76,36 +81,47 @@ public class ReadChangeStreamPartitionDoFn extends DoFn throughputEstimator; + private final Duration cdcTimeIncrement; + private transient QueryChangeStreamAction queryChangeStreamAction; /** - * This class needs a {@link DaoFactory} to build DAOs to access the partition metadata tables and - * to perform the change streams query. It uses mappers to transform database rows into the {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.ChangeStreamRecord} model. It uses the - * {@link ActionFactory} to construct the action dispatchers, which will perform the change stream - * query and process each type of record received. It emits metrics for the partition using the + * This class needs a {@link DaoFactory} to build DAOs to access the partition + * metadata tables and + * to perform the change streams query. It uses mappers to transform database + * rows into the {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.ChangeStreamRecord} + * model. It uses the + * {@link ActionFactory} to construct the action dispatchers, which will perform + * the change stream + * query and process each type of record received. It emits metrics for the + * partition using the * {@link ChangeStreamMetrics}. * - * @param daoFactory the {@link DaoFactory} to construct {@link PartitionMetadataDao}s and {@link - * ChangeStreamDao}s - * @param mapperFactory the {@link MapperFactory} to construct {@link ChangeStreamRecordMapper}s - * @param actionFactory the {@link ActionFactory} to construct actions - * @param metrics the {@link ChangeStreamMetrics} to emit partition related metrics + * @param daoFactory the {@link DaoFactory} to construct + * {@link PartitionMetadataDao}s and {@link + * ChangeStreamDao}s + * @param mapperFactory the {@link MapperFactory} to construct + * {@link ChangeStreamRecordMapper}s + * @param actionFactory the {@link ActionFactory} to construct actions + * @param metrics the {@link ChangeStreamMetrics} to emit partition + * related metrics + * @param cdcTimeIncrement duration to be used for the next end timestamp */ public ReadChangeStreamPartitionDoFn( DaoFactory daoFactory, MapperFactory mapperFactory, ActionFactory actionFactory, - ChangeStreamMetrics metrics) { - this.daoFactory = daoFactory; + Duration cdcTimeIncrement) { this.mapperFactory = mapperFactory; - this.actionFactory = actionFactory; this.metrics = metrics; this.isMutableChangeStream = daoFactory.isMutableChangeStream(); + this.cdcTimeIncrement = cdcTimeIncrement; this.throughputEstimator = new NullThroughputEstimator<>(); } @@ -121,28 +137,31 @@ public ManualWatermarkEstimator newWatermarkEstimator( } /** - * The restriction for a partition will be defined from the start and end timestamp to query the - * partition for. The {@link TimestampRange} restriction represents a closed-open interval, while - * the start / end timestamps represent a closed-closed interval, so we add 1 nanosecond to the + * The restriction for a partition will be defined from the start and end + * timestamp to query the + * partition for. The {@link TimestampRange} restriction represents a + * closed-open interval, while + * the start / end timestamps represent a closed-closed interval, so we add 1 + * nanosecond to the * end timestamp to convert it to closed-open. * - *

In this function we also update the partition state to {@link + *

+ * In this function we also update the partition state to {@link * PartitionMetadata.State#RUNNING}. * * @param partition the partition to be queried - * @return the timestamp range from the partition start timestamp to the partition end timestamp + - * 1 nanosecond + * @return the timestamp range from the partition start timestamp to the + * partition end timestamp + + * 1 nanosecond */ @GetInitialRestriction public TimestampRange initialRestriction(@Element PartitionMetadata partition) { final String token = partition.getPartitionToken(); final com.google.cloud.Timestamp startTimestamp = partition.getStartTimestamp(); // Range represents closed-open interval - final com.google.cloud.Timestamp endTimestamp = - TimestampUtils.next(partition.getEndTimestamp()); + final com.google.cloud.Timestamp endTimestamp = TimestampUtils.next(partition.getEndTimestamp()); final com.google.cloud.Timestamp partitionScheduledAt = partition.getScheduledAt(); - final com.google.cloud.Timestamp partitionRunningAt = - daoFactory.getPartitionMetadataDao().updateToRunning(token); + final com.google.cloud.Timestamp partitionRunningAt = daoFactory.getPartitionMetadataDao().updateToRunning(token); if (partitionScheduledAt != null && partitionRunningAt != null) { metrics.updatePartitionScheduledToRunning( @@ -158,15 +177,14 @@ public TimestampRange initialRestriction(@Element PartitionMetadata partition) { @GetSize public double getSize(@Element PartitionMetadata partition, @Restriction TimestampRange range) throws Exception { - final BigDecimal timeGapInSeconds = - BigDecimal.valueOf(newTracker(partition, range).getProgress().getWorkRemaining()); + final BigDecimal timeGapInSeconds = BigDecimal + .valueOf(newTracker(partition, range).getProgress().getWorkRemaining()); final BigDecimal throughput = BigDecimal.valueOf(this.throughputEstimator.get()); - final double size = - timeGapInSeconds - .multiply(throughput) - // Cap it at Double.MAX_VALUE to avoid an overflow. - .min(MAX_DOUBLE) - .doubleValue(); + final double size = timeGapInSeconds + .multiply(throughput) + // Cap it at Double.MAX_VALUE to avoid an overflow. + .min(MAX_DOUBLE) + .doubleValue(); LOG.debug( "getSize() = {} ({} timeGapInSeconds * {} throughput)", size, timeGapInSeconds, throughput); return size; @@ -179,8 +197,10 @@ public ReadChangeStreamPartitionRangeTracker newTracker( } /** - * Constructs instances for the {@link PartitionMetadataDao}, {@link ChangeStreamDao}, {@link - * ChangeStreamRecordMapper}, {@link PartitionMetadataMapper}, {@link DataChangeRecordAction}, + * Constructs instances for the {@link PartitionMetadataDao}, + * {@link ChangeStreamDao}, {@link + * ChangeStreamRecordMapper}, {@link PartitionMetadataMapper}, + * {@link DataChangeRecordAction}, * {@link HeartbeatRecordAction}, {@link ChildPartitionsRecordAction}, {@link * PartitionStartRecordAction}, {@link PartitionEndRecordAction}, {@link * PartitionEventRecordAction} and {@link QueryChangeStreamAction}. @@ -189,52 +209,56 @@ public ReadChangeStreamPartitionRangeTracker newTracker( public void setup() { final PartitionMetadataDao partitionMetadataDao = daoFactory.getPartitionMetadataDao(); final ChangeStreamDao changeStreamDao = daoFactory.getChangeStreamDao(); - final ChangeStreamRecordMapper changeStreamRecordMapper = - mapperFactory.changeStreamRecordMapper(); + final ChangeStreamRecordMapper changeStreamRecordMapper = mapperFactory.changeStreamRecordMapper(); final PartitionMetadataMapper partitionMetadataMapper = mapperFactory.partitionMetadataMapper(); - final DataChangeRecordAction dataChangeRecordAction = - actionFactory.dataChangeRecordAction(throughputEstimator); - final HeartbeatRecordAction heartbeatRecordAction = - actionFactory.heartbeatRecordAction(metrics); - final ChildPartitionsRecordAction childPartitionsRecordAction = - actionFactory.childPartitionsRecordAction(partitionMetadataDao, metrics); - final PartitionStartRecordAction partitionStartRecordAction = - actionFactory.partitionStartRecordAction(partitionMetadataDao, metrics); - final PartitionEndRecordAction partitionEndRecordAction = - actionFactory.partitionEndRecordAction(partitionMetadataDao, metrics); - final PartitionEventRecordAction partitionEventRecordAction = - actionFactory.partitionEventRecordAction(partitionMetadataDao, metrics); + final DataChangeRecordAction dataChangeRecordAction = actionFactory.dataChangeRecordAction(throughputEstimator); + final HeartbeatRecordAction heartbeatRecordAction = actionFactory.heartbeatRecordAction(metrics); + final ChildPartitionsRecordAction childPartitionsRecordAction = actionFactory + .childPartitionsRecordAction(partitionMetadataDao, metrics); + final PartitionStartRecordAction partitionStartRecordAction = actionFactory + .partitionStartRecordAction(partitionMetadataDao, metrics); + final PartitionEndRecordAction partitionEndRecordAction = actionFactory + .partitionEndRecordAction(partitionMetadataDao, metrics); + final PartitionEventRecordAction partitionEventRecordAction = actionFactory + .partitionEventRecordAction(partitionMetadataDao, metrics); - this.queryChangeStreamAction = - actionFactory.queryChangeStreamAction( - changeStreamDao, - partitionMetadataDao, - changeStreamRecordMapper, - partitionMetadataMapper, - dataChangeRecordAction, - heartbeatRecordAction, - childPartitionsRecordAction, - partitionStartRecordAction, - partitionEndRecordAction, - partitionEventRecordAction, - metrics, - isMutableChangeStream); + this.queryChangeStreamAction = actionFactory.queryChangeStreamAction( + changeStreamDao, + partitionMetadataDao, + changeStreamRecordMapper, + partitionMetadataMapper, + dataChangeRecordAction, + heartbeatRecordAction, + childPartitionsRecordAction, + partitionStartRecordAction, + partitionEndRecordAction, + partitionEventRecordAction, + metrics, + isMutableChangeStream, + cdcTimeIncrement); } /** - * Performs a change stream query for a given partition. A different action will be taken - * depending on the type of record received from the query. This component will also reflect the + * Performs a change stream query for a given partition. A different action will + * be taken + * depending on the type of record received from the query. This component will + * also reflect the * partition state in the partition metadata tables. * - *

The processing of a partition is delegated to the {@link QueryChangeStreamAction}. + *

+ * The processing of a partition is delegated to the + * {@link QueryChangeStreamAction}. * - * @param partition the partition to be queried - * @param tracker an instance of {@link ReadChangeStreamPartitionRangeTracker} - * @param receiver a {@link DataChangeRecord} {@link OutputReceiver} - * @param watermarkEstimator a {@link ManualWatermarkEstimator} of {@link Instant} - * @param bundleFinalizer the bundle finalizer - * @return a {@link ProcessContinuation#stop()} if a record timestamp could not be claimed or if - * the partition processing has finished + * @param partition the partition to be queried + * @param tracker an instance of + * {@link ReadChangeStreamPartitionRangeTracker} + * @param receiver a {@link DataChangeRecord} {@link OutputReceiver} + * @param watermarkEstimator a {@link ManualWatermarkEstimator} of + * {@link Instant} + * @param bundleFinalizer the bundle finalizer + * @return a {@link ProcessContinuation#stop()} if a record timestamp could not + * be claimed or if + * the partition processing has finished */ @ProcessElement public ProcessContinuation processElement( @@ -253,7 +277,8 @@ public ProcessContinuation processElement( } /** - * Sets the estimator to calculate the backlog of this function. Must be called after the + * Sets the estimator to calculate the backlog of this function. Must be called + * after the * initialization of this DoFn. * * @param throughputEstimator an estimator to calculate local throughput. diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/HeartbeatRecordActionTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/HeartbeatRecordActionTest.java index 56d1825c8a18..39453956e290 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/HeartbeatRecordActionTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/HeartbeatRecordActionTest.java @@ -72,7 +72,7 @@ public void testRestrictionClaimed() { interrupter, watermarkEstimator); - assertEquals(Optional.empty(), maybeContinuation); + assertEquals(Optional.of(ProcessContinuation.resume()), maybeContinuation); verify(watermarkEstimator).setWatermark(new Instant(timestamp.toSqlTimestamp().getTime())); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamActionTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamActionTest.java index 26ab41dff878..067b2e9e9786 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamActionTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamActionTest.java @@ -64,984 +64,950 @@ import org.mockito.ArgumentCaptor; public class QueryChangeStreamActionTest { - private static final String PARTITION_TOKEN = "partitionToken"; - private static final Timestamp PARTITION_START_TIMESTAMP = Timestamp.ofTimeMicroseconds(10L); - private static final Timestamp RECORD_TIMESTAMP = Timestamp.ofTimeMicroseconds(20L); - private static final Timestamp PARTITION_END_TIMESTAMP = Timestamp.ofTimeMicroseconds(30L); - private static final long PARTITION_HEARTBEAT_MILLIS = 30_000L; - private static final Instant WATERMARK = Instant.now(); - private static final Timestamp WATERMARK_TIMESTAMP = - Timestamp.ofTimeMicroseconds(WATERMARK.getMillis() * 1_000L); - - private ChangeStreamDao changeStreamDao; - private PartitionMetadataDao partitionMetadataDao; - private PartitionMetadata partition; - private ChangeStreamMetrics metrics; - private TimestampRange restriction; - private RestrictionTracker restrictionTracker; - private OutputReceiver outputReceiver; - private ChangeStreamRecordMapper changeStreamRecordMapper; - private PartitionMetadataMapper partitionMetadataMapper; - private ManualWatermarkEstimator watermarkEstimator; - private BundleFinalizer bundleFinalizer; - private DataChangeRecordAction dataChangeRecordAction; - private HeartbeatRecordAction heartbeatRecordAction; - private ChildPartitionsRecordAction childPartitionsRecordAction; - private PartitionStartRecordAction partitionStartRecordAction; - private PartitionEndRecordAction partitionEndRecordAction; - private PartitionEventRecordAction partitionEventRecordAction; - private QueryChangeStreamAction action; - - @Before - public void setUp() throws Exception { - changeStreamDao = mock(ChangeStreamDao.class); - partitionMetadataDao = mock(PartitionMetadataDao.class); - changeStreamRecordMapper = mock(ChangeStreamRecordMapper.class); - partitionMetadataMapper = mock(PartitionMetadataMapper.class); - dataChangeRecordAction = mock(DataChangeRecordAction.class); - heartbeatRecordAction = mock(HeartbeatRecordAction.class); - childPartitionsRecordAction = mock(ChildPartitionsRecordAction.class); - partitionStartRecordAction = mock(PartitionStartRecordAction.class); - partitionEndRecordAction = mock(PartitionEndRecordAction.class); - partitionEventRecordAction = mock(PartitionEventRecordAction.class); - metrics = mock(ChangeStreamMetrics.class); - - action = - new QueryChangeStreamAction( - changeStreamDao, - partitionMetadataDao, - changeStreamRecordMapper, - partitionMetadataMapper, - dataChangeRecordAction, - heartbeatRecordAction, - childPartitionsRecordAction, - partitionStartRecordAction, - partitionEndRecordAction, - partitionEventRecordAction, - metrics, - false); - final Struct row = mock(Struct.class); - partition = - PartitionMetadata.newBuilder() - .setPartitionToken(PARTITION_TOKEN) - .setParentTokens(Sets.newHashSet("parentToken")) - .setStartTimestamp(PARTITION_START_TIMESTAMP) - .setEndTimestamp(PARTITION_END_TIMESTAMP) - .setHeartbeatMillis(PARTITION_HEARTBEAT_MILLIS) - .setState(SCHEDULED) - .setWatermark(WATERMARK_TIMESTAMP) - .setScheduledAt(Timestamp.now()) - .build(); - restriction = mock(TimestampRange.class); - restrictionTracker = mock(RestrictionTracker.class); - outputReceiver = mock(OutputReceiver.class); - watermarkEstimator = mock(ManualWatermarkEstimator.class); - bundleFinalizer = new BundleFinalizerStub(); - - when(restrictionTracker.currentRestriction()).thenReturn(restriction); - when(restriction.getFrom()).thenReturn(PARTITION_START_TIMESTAMP); - when(restriction.getTo()).thenReturn(PARTITION_END_TIMESTAMP); - when(partitionMetadataDao.getPartition(PARTITION_TOKEN)).thenReturn(row); - when(partitionMetadataMapper.from(row)).thenReturn(partition); - } - - void setupUnboundedPartition() { - partition = - PartitionMetadata.newBuilder() - .setPartitionToken(PARTITION_TOKEN) - .setParentTokens(Sets.newHashSet("parentToken")) - .setStartTimestamp(PARTITION_START_TIMESTAMP) - .setEndTimestamp(MAX_INCLUSIVE_END_AT) - .setHeartbeatMillis(PARTITION_HEARTBEAT_MILLIS) - .setState(SCHEDULED) - .setWatermark(WATERMARK_TIMESTAMP) - .setScheduledAt(Timestamp.now()) - .build(); - when(partitionMetadataMapper.from(any())).thenReturn(partition); - when(restriction.getTo()).thenReturn(MAX_INCLUSIVE_END_AT); - } - - @Test - public void testQueryChangeStreamWithDataChangeRecord() { - final Struct rowAsStruct = mock(Struct.class); - final ChangeStreamResultSetMetadata resultSetMetadata = - mock(ChangeStreamResultSetMetadata.class); - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final DataChangeRecord record1 = mock(DataChangeRecord.class); - final DataChangeRecord record2 = mock(DataChangeRecord.class); - when(record1.getRecordTimestamp()).thenReturn(PARTITION_START_TIMESTAMP); - when(record2.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); - when(changeStreamDao.changeStreamQuery( - PARTITION_TOKEN, - PARTITION_START_TIMESTAMP, - PARTITION_END_TIMESTAMP, - PARTITION_HEARTBEAT_MILLIS)) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(true); - when(resultSet.getCurrentRowAsStruct()).thenReturn(rowAsStruct); - when(resultSet.getMetadata()).thenReturn(resultSetMetadata); - when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) - .thenReturn(Arrays.asList(record1, record2)); - when(dataChangeRecordAction.run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(outputReceiver), - eq(watermarkEstimator))) - .thenReturn(Optional.empty()); - when(dataChangeRecordAction.run( - eq(partition), - eq(record2), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(outputReceiver), - eq(watermarkEstimator))) - .thenReturn(Optional.of(ProcessContinuation.stop())); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - - final ProcessContinuation result = - action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - verify(dataChangeRecordAction) - .run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(outputReceiver), - eq(watermarkEstimator)); - verify(dataChangeRecordAction) - .run( - eq(partition), - eq(record2), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(outputReceiver), - eq(watermarkEstimator)); - verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); - - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(restrictionTracker, never()).tryClaim(any()); - } - - @Test - public void testQueryChangeStreamWithHeartbeatRecord() { - final Struct rowAsStruct = mock(Struct.class); - final ChangeStreamResultSetMetadata resultSetMetadata = - mock(ChangeStreamResultSetMetadata.class); - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final HeartbeatRecord record1 = mock(HeartbeatRecord.class); - final HeartbeatRecord record2 = mock(HeartbeatRecord.class); - when(record1.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); - when(record2.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); - when(changeStreamDao.changeStreamQuery( - PARTITION_TOKEN, - PARTITION_START_TIMESTAMP, - PARTITION_END_TIMESTAMP, - PARTITION_HEARTBEAT_MILLIS)) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(true); - when(resultSet.getCurrentRowAsStruct()).thenReturn(rowAsStruct); - when(resultSet.getMetadata()).thenReturn(resultSetMetadata); - when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) - .thenReturn(Arrays.asList(record1, record2)); - when(heartbeatRecordAction.run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.empty()); - when(heartbeatRecordAction.run( - eq(partition), - eq(record2), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.of(ProcessContinuation.stop())); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - - final ProcessContinuation result = - action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - verify(heartbeatRecordAction) - .run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(heartbeatRecordAction) - .run( - eq(partition), - eq(record2), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(restrictionTracker, never()).tryClaim(any()); - } - - @Test - public void testQueryChangeStreamWithChildPartitionsRecord() { - final Struct rowAsStruct = mock(Struct.class); - final ChangeStreamResultSetMetadata resultSetMetadata = - mock(ChangeStreamResultSetMetadata.class); - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final ChildPartitionsRecord record1 = mock(ChildPartitionsRecord.class); - final ChildPartitionsRecord record2 = mock(ChildPartitionsRecord.class); - when(record1.getRecordTimestamp()).thenReturn(PARTITION_START_TIMESTAMP); - when(record2.getRecordTimestamp()).thenReturn(PARTITION_START_TIMESTAMP); - when(changeStreamDao.changeStreamQuery( - PARTITION_TOKEN, - PARTITION_START_TIMESTAMP, - PARTITION_END_TIMESTAMP, - PARTITION_HEARTBEAT_MILLIS)) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(true); - when(resultSet.getCurrentRowAsStruct()).thenReturn(rowAsStruct); - when(resultSet.getMetadata()).thenReturn(resultSetMetadata); - when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) - .thenReturn(Arrays.asList(record1, record2)); - when(childPartitionsRecordAction.run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.empty()); - when(childPartitionsRecordAction.run( - eq(partition), - eq(record2), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.of(ProcessContinuation.stop())); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - - final ProcessContinuation result = - action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - verify(childPartitionsRecordAction) - .run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(childPartitionsRecordAction) - .run( - eq(partition), - eq(record2), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(restrictionTracker, never()).tryClaim(any()); - } - - @Test - public void testQueryChangeStreamWithRestrictionFromAfterPartitionStart() { - final Struct rowAsStruct = mock(Struct.class); - final ChangeStreamResultSetMetadata resultSetMetadata = - mock(ChangeStreamResultSetMetadata.class); - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final ChildPartitionsRecord record1 = mock(ChildPartitionsRecord.class); - final ChildPartitionsRecord record2 = mock(ChildPartitionsRecord.class); - - // From is after Partition start at - when(restriction.getFrom()).thenReturn(Timestamp.ofTimeMicroseconds(15L)); - // Both records should be included - when(record1.getRecordTimestamp()).thenReturn(Timestamp.ofTimeMicroseconds(15L)); - when(record2.getRecordTimestamp()).thenReturn(Timestamp.ofTimeMicroseconds(25L)); - when(changeStreamDao.changeStreamQuery( - PARTITION_TOKEN, - Timestamp.ofTimeMicroseconds(15L), - PARTITION_END_TIMESTAMP, - PARTITION_HEARTBEAT_MILLIS)) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(true); - when(resultSet.getCurrentRowAsStruct()).thenReturn(rowAsStruct); - when(resultSet.getMetadata()).thenReturn(resultSetMetadata); - when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) - .thenReturn(Arrays.asList(record1, record2)); - when(childPartitionsRecordAction.run( - eq(partition), - eq(record2), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.of(ProcessContinuation.stop())); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - - final ProcessContinuation result = - action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - verify(childPartitionsRecordAction) - .run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(childPartitionsRecordAction) - .run( - eq(partition), - eq(record2), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(restrictionTracker, never()).tryClaim(any()); - } - - @Test - public void testQueryChangeStreamWithPartitionStartRecord() { - final ChangeStreamResultSetMetadata resultSetMetadata = - mock(ChangeStreamResultSetMetadata.class); - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final PartitionStartRecord record1 = mock(PartitionStartRecord.class); - when(record1.getRecordTimestamp()).thenReturn(PARTITION_START_TIMESTAMP); - when(changeStreamDao.changeStreamQuery( - PARTITION_TOKEN, - PARTITION_START_TIMESTAMP, - PARTITION_END_TIMESTAMP, - PARTITION_HEARTBEAT_MILLIS)) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(true); - when(resultSet.getMetadata()).thenReturn(resultSetMetadata); - when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) - .thenReturn(Arrays.asList(record1)); - when(partitionStartRecordAction.run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.of(ProcessContinuation.stop())); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - - final ProcessContinuation result = - action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - verify(partitionStartRecordAction) - .run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(restrictionTracker, never()).tryClaim(any()); - } - - @Test - public void testQueryChangeStreamWithRestrictionFromAfterPartitionStartForPartitionStartRecord() { - final ChangeStreamResultSetMetadata resultSetMetadata = - mock(ChangeStreamResultSetMetadata.class); - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final PartitionStartRecord record1 = mock(PartitionStartRecord.class); - - // From is after Partition start at - when(restriction.getFrom()).thenReturn(Timestamp.ofTimeMicroseconds(15L)); - // This record should be included. - when(record1.getRecordTimestamp()).thenReturn(Timestamp.ofTimeMicroseconds(15L)); - when(changeStreamDao.changeStreamQuery( - PARTITION_TOKEN, - Timestamp.ofTimeMicroseconds(15L), - PARTITION_END_TIMESTAMP, - PARTITION_HEARTBEAT_MILLIS)) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(true); - when(resultSet.getMetadata()).thenReturn(resultSetMetadata); - when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) - .thenReturn(Arrays.asList(record1)); - when(partitionStartRecordAction.run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.of(ProcessContinuation.stop())); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - - final ProcessContinuation result = - action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - verify(partitionStartRecordAction) - .run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(restrictionTracker, never()).tryClaim(any()); - } - - @Test - public void testQueryChangeStreamWithPartitionEndRecordBoundedRestriction() { - final ChangeStreamResultSetMetadata resultSetMetadata = - mock(ChangeStreamResultSetMetadata.class); - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final PartitionEndRecord record1 = mock(PartitionEndRecord.class); - when(record1.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); - when(changeStreamDao.changeStreamQuery( - PARTITION_TOKEN, - PARTITION_START_TIMESTAMP, - PARTITION_END_TIMESTAMP, - PARTITION_HEARTBEAT_MILLIS)) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(true, false); - when(resultSet.getMetadata()).thenReturn(resultSetMetadata); - when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) - .thenReturn(Arrays.asList(record1)); - when(partitionEndRecordAction.run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.empty()); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); - - final ProcessContinuation result = - action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - verify(partitionEndRecordAction) - .run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(restrictionTracker).tryClaim(PARTITION_END_TIMESTAMP); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionMetadataDao, never()).updateWatermark(any(), any()); - } - - @Test - public void testQueryChangeStreamWithPartitionEndRecordUnboundedRestriction() { - setupUnboundedPartition(); - - final ChangeStreamResultSetMetadata resultSetMetadata = - mock(ChangeStreamResultSetMetadata.class); - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final PartitionEndRecord record1 = mock(PartitionEndRecord.class); - when(record1.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); - final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); - when(changeStreamDao.changeStreamQuery( - eq(PARTITION_TOKEN), - eq(PARTITION_START_TIMESTAMP), - timestampCaptor.capture(), - eq(PARTITION_HEARTBEAT_MILLIS))) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(true, false); - when(resultSet.getMetadata()).thenReturn(resultSetMetadata); - when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) - .thenReturn(Arrays.asList(record1)); - when(partitionEndRecordAction.run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.empty()); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); - - final ProcessContinuation result = - action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - assertNotEquals(MAX_INCLUSIVE_END_AT, timestampCaptor.getValue()); - verify(partitionEndRecordAction) - .run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(restrictionTracker).tryClaim(MAX_INCLUSIVE_END_AT); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionMetadataDao, never()).updateWatermark(any(), any()); - } - - @Test - public void testQueryChangeStreamWithPartitionEventRecord() { - final ChangeStreamResultSetMetadata resultSetMetadata = - mock(ChangeStreamResultSetMetadata.class); - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final PartitionEventRecord record1 = mock(PartitionEventRecord.class); - when(record1.getRecordTimestamp()).thenReturn(PARTITION_START_TIMESTAMP); - when(changeStreamDao.changeStreamQuery( - PARTITION_TOKEN, - PARTITION_START_TIMESTAMP, - PARTITION_END_TIMESTAMP, - PARTITION_HEARTBEAT_MILLIS)) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(true); - when(resultSet.getMetadata()).thenReturn(resultSetMetadata); - when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) - .thenReturn(Arrays.asList(record1)); - when(partitionEventRecordAction.run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.of(ProcessContinuation.stop())); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - - final ProcessContinuation result = - action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - verify(partitionEventRecordAction) - .run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(restrictionTracker, never()).tryClaim(any()); - } - - @Test - public void testQueryChangeStreamWithStreamFinished() { - final ChangeStreamResultSet changeStreamResultSet = mock(ChangeStreamResultSet.class); - when(changeStreamDao.changeStreamQuery( - PARTITION_TOKEN, - PARTITION_START_TIMESTAMP, - PARTITION_END_TIMESTAMP, - PARTITION_HEARTBEAT_MILLIS)) - .thenReturn(changeStreamResultSet); - when(changeStreamResultSet.next()).thenReturn(false); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - when(restrictionTracker.tryClaim(PARTITION_END_TIMESTAMP)).thenReturn(true); - - final ProcessContinuation result = - action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - verify(partitionMetadataDao).updateToFinished(PARTITION_TOKEN); - verify(metrics).decActivePartitionReadCounter(); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - } - - @Test - public void testQueryChangeStreamFinishedWithResume() { - partition = - PartitionMetadata.newBuilder() - .setPartitionToken(PARTITION_TOKEN) - .setParentTokens(Sets.newHashSet("parentToken")) - .setStartTimestamp(PARTITION_START_TIMESTAMP) - .setEndTimestamp(MAX_INCLUSIVE_END_AT) - .setHeartbeatMillis(PARTITION_HEARTBEAT_MILLIS) - .setState(SCHEDULED) - .setWatermark(WATERMARK_TIMESTAMP) - .setScheduledAt(Timestamp.now()) - .build(); - when(partitionMetadataMapper.from(any())).thenReturn(partition); - - final ChangeStreamResultSet changeStreamResultSet = mock(ChangeStreamResultSet.class); - final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); - when(changeStreamDao.changeStreamQuery( - eq(PARTITION_TOKEN), - eq(PARTITION_START_TIMESTAMP), - timestampCaptor.capture(), - eq(PARTITION_HEARTBEAT_MILLIS))) - .thenReturn(changeStreamResultSet); - when(changeStreamResultSet.next()).thenReturn(false); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); - - final ProcessContinuation result = - action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - assertEquals(ProcessContinuation.resume(), result); - assertNotEquals(MAX_INCLUSIVE_END_AT, timestampCaptor.getValue()); - - verify(restrictionTracker).tryClaim(timestampCaptor.getValue()); - verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); - verify(partitionMetadataDao, never()).updateToFinished(PARTITION_TOKEN); - verify(metrics, never()).decActivePartitionReadCounter(); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - } - - // Out of range indicates that we're beyond the end of the partition and should stop - // processing. - @Test - public void testQueryChangeStreamWithOutOfRangeErrorOnUnboundedPartition() { - setupUnboundedPartition(); - - final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); - when(changeStreamDao.changeStreamQuery( - eq(PARTITION_TOKEN), - eq(PARTITION_START_TIMESTAMP), - timestampCaptor.capture(), - eq(PARTITION_HEARTBEAT_MILLIS))) - .thenThrow( - SpannerExceptionFactory.newSpannerException( - ErrorCode.OUT_OF_RANGE, "Specified start_timestamp is invalid")); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); - - final ProcessContinuation result = - action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - assertEquals(ProcessContinuation.stop(), result); - assertNotEquals(MAX_INCLUSIVE_END_AT, timestampCaptor.getValue()); - - verify(restrictionTracker).tryClaim(MAX_INCLUSIVE_END_AT); - verify(partitionMetadataDao).updateToFinished(PARTITION_TOKEN); - verify(metrics).decActivePartitionReadCounter(); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionMetadataDao, never()).updateWatermark(any(), any()); - } - - // Out of range indicates that we're beyond the end of the partition and should stop - // processing. - @Test - public void testQueryChangeStreamWithOutOfRangeErrorOnBoundedPartition() { - when(changeStreamDao.changeStreamQuery( - eq(PARTITION_TOKEN), - eq(PARTITION_START_TIMESTAMP), - eq(PARTITION_END_TIMESTAMP), - eq(PARTITION_HEARTBEAT_MILLIS))) - .thenThrow( - SpannerExceptionFactory.newSpannerException( - ErrorCode.OUT_OF_RANGE, "Specified start_timestamp is invalid")); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); - - final ProcessContinuation result = - action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - assertEquals(ProcessContinuation.stop(), result); - - verify(restrictionTracker).tryClaim(PARTITION_END_TIMESTAMP); - verify(partitionMetadataDao).updateToFinished(PARTITION_TOKEN); - verify(metrics).decActivePartitionReadCounter(); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionMetadataDao, never()).updateWatermark(any(), any()); - } - - @Test - public void testQueryChangeStreamWithChildPartitionsRecordBoundedRestriction() { - final ChangeStreamResultSetMetadata resultSetMetadata = - mock(ChangeStreamResultSetMetadata.class); - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final ChildPartitionsRecord record1 = mock(ChildPartitionsRecord.class); - when(record1.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); - when(changeStreamDao.changeStreamQuery( - PARTITION_TOKEN, - PARTITION_START_TIMESTAMP, - PARTITION_END_TIMESTAMP, - PARTITION_HEARTBEAT_MILLIS)) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(true, false); - when(resultSet.getMetadata()).thenReturn(resultSetMetadata); - when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) - .thenReturn(Arrays.asList(record1)); - when(childPartitionsRecordAction.run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.empty()); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); - - final ProcessContinuation result = - action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - verify(childPartitionsRecordAction) - .run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(restrictionTracker).tryClaim(PARTITION_END_TIMESTAMP); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionMetadataDao, never()).updateWatermark(any(), any()); - } - - @Test - public void testQueryChangeStreamWithChildPartitionsRecordUnboundedRestriction() { - setupUnboundedPartition(); - - final ChangeStreamResultSetMetadata resultSetMetadata = - mock(ChangeStreamResultSetMetadata.class); - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final ChildPartitionsRecord record1 = mock(ChildPartitionsRecord.class); - when(record1.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); - final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); - when(changeStreamDao.changeStreamQuery( - eq(PARTITION_TOKEN), - eq(PARTITION_START_TIMESTAMP), - timestampCaptor.capture(), - eq(PARTITION_HEARTBEAT_MILLIS))) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(true, false); - when(resultSet.getMetadata()).thenReturn(resultSetMetadata); - when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) - .thenReturn(Arrays.asList(record1)); - when(childPartitionsRecordAction.run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.empty()); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); - - final ProcessContinuation result = - action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - assertNotEquals(MAX_INCLUSIVE_END_AT, timestampCaptor.getValue()); - verify(childPartitionsRecordAction) - .run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(restrictionTracker).tryClaim(MAX_INCLUSIVE_END_AT); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionMetadataDao, never()).updateWatermark(any(), any()); - } - - @Test - public void testQueryChangeStreamWithMutableChangeStreamCappedEndTimestamp() { - // Initialize action with isMutableChangeStream = true - action = - new QueryChangeStreamAction( - changeStreamDao, - partitionMetadataDao, - changeStreamRecordMapper, - partitionMetadataMapper, - dataChangeRecordAction, - heartbeatRecordAction, - childPartitionsRecordAction, - partitionStartRecordAction, - partitionEndRecordAction, - partitionEventRecordAction, - metrics, - true); - - // Set endTimestamp to 60 minutes in the future - Timestamp now = Timestamp.now(); - Timestamp endTimestamp = - Timestamp.ofTimeSecondsAndNanos(now.getSeconds() + 60 * 60, now.getNanos()); - - partition = partition.toBuilder().setEndTimestamp(endTimestamp).build(); - when(restriction.getTo()).thenReturn(endTimestamp); - when(partitionMetadataMapper.from(any())).thenReturn(partition); - - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); - when(changeStreamDao.changeStreamQuery( - eq(PARTITION_TOKEN), eq(PARTITION_START_TIMESTAMP), - timestampCaptor.capture(), eq(PARTITION_HEARTBEAT_MILLIS))) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(false); // Query finishes (reaches cap) - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); - - final ProcessContinuation result = - action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - // Verify query was capped at ~2 minutes - long diff = timestampCaptor.getValue().getSeconds() - now.getSeconds(); - assertTrue("Query should be capped at approx 2 minutes (120s)", Math.abs(diff - 120) < 10); - - // Crucial: Should RESUME to process the rest later - assertEquals(ProcessContinuation.resume(), result); - } - - @Test - public void testQueryChangeStreamWithMutableChangeStreamUncappedEndTimestamp() { - action = - new QueryChangeStreamAction( - changeStreamDao, - partitionMetadataDao, - changeStreamRecordMapper, - partitionMetadataMapper, - dataChangeRecordAction, - heartbeatRecordAction, - childPartitionsRecordAction, - partitionStartRecordAction, - partitionEndRecordAction, - partitionEventRecordAction, - metrics, - true); - - // Set endTimestamp to only 10 seconds in the future - Timestamp now = Timestamp.now(); - Timestamp endTimestamp = Timestamp.ofTimeSecondsAndNanos(now.getSeconds() + 10, now.getNanos()); - - partition = partition.toBuilder().setEndTimestamp(endTimestamp).build(); - when(restriction.getTo()).thenReturn(endTimestamp); - when(partitionMetadataMapper.from(any())).thenReturn(partition); - - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); - when(changeStreamDao.changeStreamQuery( - eq(PARTITION_TOKEN), eq(PARTITION_START_TIMESTAMP), - timestampCaptor.capture(), eq(PARTITION_HEARTBEAT_MILLIS))) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(false); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - when(restrictionTracker.tryClaim(endTimestamp)).thenReturn(true); - - final ProcessContinuation result = - action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - // Should use the exact endTimestamp since it is within the limit (10s < 2m) - assertEquals(endTimestamp, timestampCaptor.getValue()); - - // Should STOP because we reached the actual requested endTimestamp - assertEquals(ProcessContinuation.stop(), result); - } - - @Test - public void testQueryChangeStreamUnboundedResumesCorrectly() { - // Unbounded restriction (streaming forever) - setupUnboundedPartition(); - - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - when(changeStreamDao.changeStreamQuery(any(), any(), any(), anyLong())).thenReturn(resultSet); - when(resultSet.next()).thenReturn(false); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); - - final ProcessContinuation result = - action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - // Should return RESUME to continue reading the stream every 2 minutes - assertEquals(ProcessContinuation.resume(), result); - verify(metrics).incQueryCounter(); - } - - private static class BundleFinalizerStub implements BundleFinalizer { - @Override - public void afterBundleCommit(Instant callbackExpiry, Callback callback) { - try { - callback.onBundleSuccess(); - } catch (Exception e) { - throw new RuntimeException(e); - } + private static final String PARTITION_TOKEN = "partitionToken"; + private static final Timestamp PARTITION_START_TIMESTAMP = Timestamp.ofTimeMicroseconds(10L); + private static final Timestamp RECORD_TIMESTAMP = Timestamp.ofTimeMicroseconds(20L); + private static final Timestamp PARTITION_END_TIMESTAMP = Timestamp.ofTimeMicroseconds(30L); + private static final long PARTITION_HEARTBEAT_MILLIS = 30_000L; + private static final Instant WATERMARK = Instant.now(); + private static final Timestamp WATERMARK_TIMESTAMP = Timestamp.ofTimeMicroseconds(WATERMARK.getMillis() * 1_000L); + + private ChangeStreamDao changeStreamDao; + private PartitionMetadataDao partitionMetadataDao; + private PartitionMetadata partition; + private ChangeStreamMetrics metrics; + private TimestampRange restriction; + private RestrictionTracker restrictionTracker; + private OutputReceiver outputReceiver; + private ChangeStreamRecordMapper changeStreamRecordMapper; + private PartitionMetadataMapper partitionMetadataMapper; + private ManualWatermarkEstimator watermarkEstimator; + private BundleFinalizer bundleFinalizer; + private DataChangeRecordAction dataChangeRecordAction; + private HeartbeatRecordAction heartbeatRecordAction; + private ChildPartitionsRecordAction childPartitionsRecordAction; + private PartitionStartRecordAction partitionStartRecordAction; + private PartitionEndRecordAction partitionEndRecordAction; + private PartitionEventRecordAction partitionEventRecordAction; + private QueryChangeStreamAction action; + + @Before + public void setUp() throws Exception { + changeStreamDao = mock(ChangeStreamDao.class); + partitionMetadataDao = mock(PartitionMetadataDao.class); + changeStreamRecordMapper = mock(ChangeStreamRecordMapper.class); + partitionMetadataMapper = mock(PartitionMetadataMapper.class); + dataChangeRecordAction = mock(DataChangeRecordAction.class); + heartbeatRecordAction = mock(HeartbeatRecordAction.class); + childPartitionsRecordAction = mock(ChildPartitionsRecordAction.class); + partitionStartRecordAction = mock(PartitionStartRecordAction.class); + partitionEndRecordAction = mock(PartitionEndRecordAction.class); + partitionEventRecordAction = mock(PartitionEventRecordAction.class); + metrics = mock(ChangeStreamMetrics.class); + + action = new QueryChangeStreamAction( + changeStreamDao, + partitionMetadataDao, + changeStreamRecordMapper, + partitionMetadataMapper, + dataChangeRecordAction, + heartbeatRecordAction, + childPartitionsRecordAction, + partitionStartRecordAction, + partitionEndRecordAction, + partitionEventRecordAction, + metrics, + false, + org.joda.time.Duration.standardMinutes(2)); + final Struct row = mock(Struct.class); + partition = PartitionMetadata.newBuilder() + .setPartitionToken(PARTITION_TOKEN) + .setParentTokens(Sets.newHashSet("parentToken")) + .setStartTimestamp(PARTITION_START_TIMESTAMP) + .setEndTimestamp(PARTITION_END_TIMESTAMP) + .setHeartbeatMillis(PARTITION_HEARTBEAT_MILLIS) + .setState(SCHEDULED) + .setWatermark(WATERMARK_TIMESTAMP) + .setScheduledAt(Timestamp.now()) + .build(); + restriction = mock(TimestampRange.class); + restrictionTracker = mock(RestrictionTracker.class); + outputReceiver = mock(OutputReceiver.class); + watermarkEstimator = mock(ManualWatermarkEstimator.class); + bundleFinalizer = new BundleFinalizerStub(); + + when(restrictionTracker.currentRestriction()).thenReturn(restriction); + when(restriction.getFrom()).thenReturn(PARTITION_START_TIMESTAMP); + when(restriction.getTo()).thenReturn(PARTITION_END_TIMESTAMP); + when(partitionMetadataDao.getPartition(PARTITION_TOKEN)).thenReturn(row); + when(partitionMetadataMapper.from(row)).thenReturn(partition); + } + + void setupUnboundedPartition() { + partition = PartitionMetadata.newBuilder() + .setPartitionToken(PARTITION_TOKEN) + .setParentTokens(Sets.newHashSet("parentToken")) + .setStartTimestamp(PARTITION_START_TIMESTAMP) + .setEndTimestamp(MAX_INCLUSIVE_END_AT) + .setHeartbeatMillis(PARTITION_HEARTBEAT_MILLIS) + .setState(SCHEDULED) + .setWatermark(WATERMARK_TIMESTAMP) + .setScheduledAt(Timestamp.now()) + .build(); + when(partitionMetadataMapper.from(any())).thenReturn(partition); + when(restriction.getTo()).thenReturn(MAX_INCLUSIVE_END_AT); + } + + @Test + public void testQueryChangeStreamWithDataChangeRecord() { + final Struct rowAsStruct = mock(Struct.class); + final ChangeStreamResultSetMetadata resultSetMetadata = mock(ChangeStreamResultSetMetadata.class); + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final DataChangeRecord record1 = mock(DataChangeRecord.class); + final DataChangeRecord record2 = mock(DataChangeRecord.class); + when(record1.getRecordTimestamp()).thenReturn(PARTITION_START_TIMESTAMP); + when(record2.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); + when(changeStreamDao.changeStreamQuery( + PARTITION_TOKEN, + PARTITION_START_TIMESTAMP, + PARTITION_END_TIMESTAMP, + PARTITION_HEARTBEAT_MILLIS)) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(true); + when(resultSet.getCurrentRowAsStruct()).thenReturn(rowAsStruct); + when(resultSet.getMetadata()).thenReturn(resultSetMetadata); + when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) + .thenReturn(Arrays.asList(record1, record2)); + when(dataChangeRecordAction.run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(outputReceiver), + eq(watermarkEstimator))) + .thenReturn(Optional.empty()); + when(dataChangeRecordAction.run( + eq(partition), + eq(record2), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(outputReceiver), + eq(watermarkEstimator))) + .thenReturn(Optional.of(ProcessContinuation.stop())); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + + final ProcessContinuation result = action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + verify(dataChangeRecordAction) + .run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(outputReceiver), + eq(watermarkEstimator)); + verify(dataChangeRecordAction) + .run( + eq(partition), + eq(record2), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(outputReceiver), + eq(watermarkEstimator)); + verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); + + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(restrictionTracker, never()).tryClaim(any()); + } + + @Test + public void testQueryChangeStreamWithHeartbeatRecord() { + final Struct rowAsStruct = mock(Struct.class); + final ChangeStreamResultSetMetadata resultSetMetadata = mock(ChangeStreamResultSetMetadata.class); + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final HeartbeatRecord record1 = mock(HeartbeatRecord.class); + final HeartbeatRecord record2 = mock(HeartbeatRecord.class); + when(record1.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); + when(record2.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); + when(changeStreamDao.changeStreamQuery( + PARTITION_TOKEN, + PARTITION_START_TIMESTAMP, + PARTITION_END_TIMESTAMP, + PARTITION_HEARTBEAT_MILLIS)) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(true); + when(resultSet.getCurrentRowAsStruct()).thenReturn(rowAsStruct); + when(resultSet.getMetadata()).thenReturn(resultSetMetadata); + when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) + .thenReturn(Arrays.asList(record1, record2)); + when(heartbeatRecordAction.run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.empty()); + when(heartbeatRecordAction.run( + eq(partition), + eq(record2), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.of(ProcessContinuation.stop())); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + + final ProcessContinuation result = action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + verify(heartbeatRecordAction) + .run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(heartbeatRecordAction) + .run( + eq(partition), + eq(record2), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(restrictionTracker, never()).tryClaim(any()); + } + + @Test + public void testQueryChangeStreamWithChildPartitionsRecord() { + final Struct rowAsStruct = mock(Struct.class); + final ChangeStreamResultSetMetadata resultSetMetadata = mock(ChangeStreamResultSetMetadata.class); + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final ChildPartitionsRecord record1 = mock(ChildPartitionsRecord.class); + final ChildPartitionsRecord record2 = mock(ChildPartitionsRecord.class); + when(record1.getRecordTimestamp()).thenReturn(PARTITION_START_TIMESTAMP); + when(record2.getRecordTimestamp()).thenReturn(PARTITION_START_TIMESTAMP); + when(changeStreamDao.changeStreamQuery( + PARTITION_TOKEN, + PARTITION_START_TIMESTAMP, + PARTITION_END_TIMESTAMP, + PARTITION_HEARTBEAT_MILLIS)) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(true); + when(resultSet.getCurrentRowAsStruct()).thenReturn(rowAsStruct); + when(resultSet.getMetadata()).thenReturn(resultSetMetadata); + when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) + .thenReturn(Arrays.asList(record1, record2)); + when(childPartitionsRecordAction.run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.empty()); + when(childPartitionsRecordAction.run( + eq(partition), + eq(record2), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.of(ProcessContinuation.stop())); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + + final ProcessContinuation result = action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + verify(childPartitionsRecordAction) + .run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(childPartitionsRecordAction) + .run( + eq(partition), + eq(record2), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(restrictionTracker, never()).tryClaim(any()); + } + + @Test + public void testQueryChangeStreamWithRestrictionFromAfterPartitionStart() { + final Struct rowAsStruct = mock(Struct.class); + final ChangeStreamResultSetMetadata resultSetMetadata = mock(ChangeStreamResultSetMetadata.class); + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final ChildPartitionsRecord record1 = mock(ChildPartitionsRecord.class); + final ChildPartitionsRecord record2 = mock(ChildPartitionsRecord.class); + + // From is after Partition start at + when(restriction.getFrom()).thenReturn(Timestamp.ofTimeMicroseconds(15L)); + // Both records should be included + when(record1.getRecordTimestamp()).thenReturn(Timestamp.ofTimeMicroseconds(15L)); + when(record2.getRecordTimestamp()).thenReturn(Timestamp.ofTimeMicroseconds(25L)); + when(changeStreamDao.changeStreamQuery( + PARTITION_TOKEN, + Timestamp.ofTimeMicroseconds(15L), + PARTITION_END_TIMESTAMP, + PARTITION_HEARTBEAT_MILLIS)) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(true); + when(resultSet.getCurrentRowAsStruct()).thenReturn(rowAsStruct); + when(resultSet.getMetadata()).thenReturn(resultSetMetadata); + when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) + .thenReturn(Arrays.asList(record1, record2)); + when(childPartitionsRecordAction.run( + eq(partition), + eq(record2), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.of(ProcessContinuation.stop())); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + + final ProcessContinuation result = action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + verify(childPartitionsRecordAction) + .run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(childPartitionsRecordAction) + .run( + eq(partition), + eq(record2), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(restrictionTracker, never()).tryClaim(any()); + } + + @Test + public void testQueryChangeStreamWithPartitionStartRecord() { + final ChangeStreamResultSetMetadata resultSetMetadata = mock(ChangeStreamResultSetMetadata.class); + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final PartitionStartRecord record1 = mock(PartitionStartRecord.class); + when(record1.getRecordTimestamp()).thenReturn(PARTITION_START_TIMESTAMP); + when(changeStreamDao.changeStreamQuery( + PARTITION_TOKEN, + PARTITION_START_TIMESTAMP, + PARTITION_END_TIMESTAMP, + PARTITION_HEARTBEAT_MILLIS)) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(true); + when(resultSet.getMetadata()).thenReturn(resultSetMetadata); + when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) + .thenReturn(Arrays.asList(record1)); + when(partitionStartRecordAction.run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.of(ProcessContinuation.stop())); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + + final ProcessContinuation result = action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + verify(partitionStartRecordAction) + .run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(restrictionTracker, never()).tryClaim(any()); + } + + @Test + public void testQueryChangeStreamWithRestrictionFromAfterPartitionStartForPartitionStartRecord() { + final ChangeStreamResultSetMetadata resultSetMetadata = mock(ChangeStreamResultSetMetadata.class); + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final PartitionStartRecord record1 = mock(PartitionStartRecord.class); + + // From is after Partition start at + when(restriction.getFrom()).thenReturn(Timestamp.ofTimeMicroseconds(15L)); + // This record should be included. + when(record1.getRecordTimestamp()).thenReturn(Timestamp.ofTimeMicroseconds(15L)); + when(changeStreamDao.changeStreamQuery( + PARTITION_TOKEN, + Timestamp.ofTimeMicroseconds(15L), + PARTITION_END_TIMESTAMP, + PARTITION_HEARTBEAT_MILLIS)) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(true); + when(resultSet.getMetadata()).thenReturn(resultSetMetadata); + when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) + .thenReturn(Arrays.asList(record1)); + when(partitionStartRecordAction.run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.of(ProcessContinuation.stop())); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + + final ProcessContinuation result = action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + verify(partitionStartRecordAction) + .run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(restrictionTracker, never()).tryClaim(any()); + } + + @Test + public void testQueryChangeStreamWithPartitionEndRecordBoundedRestriction() { + final ChangeStreamResultSetMetadata resultSetMetadata = mock(ChangeStreamResultSetMetadata.class); + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final PartitionEndRecord record1 = mock(PartitionEndRecord.class); + when(record1.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); + when(changeStreamDao.changeStreamQuery( + PARTITION_TOKEN, + PARTITION_START_TIMESTAMP, + PARTITION_END_TIMESTAMP, + PARTITION_HEARTBEAT_MILLIS)) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(true, false); + when(resultSet.getMetadata()).thenReturn(resultSetMetadata); + when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) + .thenReturn(Arrays.asList(record1)); + when(partitionEndRecordAction.run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.empty()); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); + + final ProcessContinuation result = action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + verify(partitionEndRecordAction) + .run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(restrictionTracker).tryClaim(PARTITION_END_TIMESTAMP); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionMetadataDao, never()).updateWatermark(any(), any()); + } + + @Test + public void testQueryChangeStreamWithPartitionEndRecordUnboundedRestriction() { + setupUnboundedPartition(); + + final ChangeStreamResultSetMetadata resultSetMetadata = mock(ChangeStreamResultSetMetadata.class); + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final PartitionEndRecord record1 = mock(PartitionEndRecord.class); + when(record1.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); + final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); + when(changeStreamDao.changeStreamQuery( + eq(PARTITION_TOKEN), + eq(PARTITION_START_TIMESTAMP), + timestampCaptor.capture(), + eq(PARTITION_HEARTBEAT_MILLIS))) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(true, false); + when(resultSet.getMetadata()).thenReturn(resultSetMetadata); + when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) + .thenReturn(Arrays.asList(record1)); + when(partitionEndRecordAction.run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.empty()); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); + + final ProcessContinuation result = action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + assertNotEquals(MAX_INCLUSIVE_END_AT, timestampCaptor.getValue()); + verify(partitionEndRecordAction) + .run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(restrictionTracker).tryClaim(MAX_INCLUSIVE_END_AT); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionMetadataDao, never()).updateWatermark(any(), any()); + } + + @Test + public void testQueryChangeStreamWithPartitionEventRecord() { + final ChangeStreamResultSetMetadata resultSetMetadata = mock(ChangeStreamResultSetMetadata.class); + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final PartitionEventRecord record1 = mock(PartitionEventRecord.class); + when(record1.getRecordTimestamp()).thenReturn(PARTITION_START_TIMESTAMP); + when(changeStreamDao.changeStreamQuery( + PARTITION_TOKEN, + PARTITION_START_TIMESTAMP, + PARTITION_END_TIMESTAMP, + PARTITION_HEARTBEAT_MILLIS)) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(true); + when(resultSet.getMetadata()).thenReturn(resultSetMetadata); + when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) + .thenReturn(Arrays.asList(record1)); + when(partitionEventRecordAction.run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.of(ProcessContinuation.stop())); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + + final ProcessContinuation result = action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + verify(partitionEventRecordAction) + .run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(restrictionTracker, never()).tryClaim(any()); + } + + @Test + public void testQueryChangeStreamWithStreamFinished() { + final ChangeStreamResultSet changeStreamResultSet = mock(ChangeStreamResultSet.class); + when(changeStreamDao.changeStreamQuery( + PARTITION_TOKEN, + PARTITION_START_TIMESTAMP, + PARTITION_END_TIMESTAMP, + PARTITION_HEARTBEAT_MILLIS)) + .thenReturn(changeStreamResultSet); + when(changeStreamResultSet.next()).thenReturn(false); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + when(restrictionTracker.tryClaim(PARTITION_END_TIMESTAMP)).thenReturn(true); + + final ProcessContinuation result = action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + verify(partitionMetadataDao).updateToFinished(PARTITION_TOKEN); + verify(metrics).decActivePartitionReadCounter(); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + } + + @Test + public void testQueryChangeStreamFinishedWithResume() { + partition = PartitionMetadata.newBuilder() + .setPartitionToken(PARTITION_TOKEN) + .setParentTokens(Sets.newHashSet("parentToken")) + .setStartTimestamp(PARTITION_START_TIMESTAMP) + .setEndTimestamp(MAX_INCLUSIVE_END_AT) + .setHeartbeatMillis(PARTITION_HEARTBEAT_MILLIS) + .setState(SCHEDULED) + .setWatermark(WATERMARK_TIMESTAMP) + .setScheduledAt(Timestamp.now()) + .build(); + when(partitionMetadataMapper.from(any())).thenReturn(partition); + + final ChangeStreamResultSet changeStreamResultSet = mock(ChangeStreamResultSet.class); + final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); + when(changeStreamDao.changeStreamQuery( + eq(PARTITION_TOKEN), + eq(PARTITION_START_TIMESTAMP), + timestampCaptor.capture(), + eq(PARTITION_HEARTBEAT_MILLIS))) + .thenReturn(changeStreamResultSet); + when(changeStreamResultSet.next()).thenReturn(false); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); + + final ProcessContinuation result = action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + assertEquals(ProcessContinuation.resume(), result); + assertNotEquals(MAX_INCLUSIVE_END_AT, timestampCaptor.getValue()); + + verify(restrictionTracker).tryClaim(timestampCaptor.getValue()); + verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); + verify(partitionMetadataDao, never()).updateToFinished(PARTITION_TOKEN); + verify(metrics, never()).decActivePartitionReadCounter(); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + } + + // Out of range indicates that we're beyond the end of the partition and should + // stop + // processing. + @Test + public void testQueryChangeStreamWithOutOfRangeErrorOnUnboundedPartition() { + setupUnboundedPartition(); + + final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); + when(changeStreamDao.changeStreamQuery( + eq(PARTITION_TOKEN), + eq(PARTITION_START_TIMESTAMP), + timestampCaptor.capture(), + eq(PARTITION_HEARTBEAT_MILLIS))) + .thenThrow( + SpannerExceptionFactory.newSpannerException( + ErrorCode.OUT_OF_RANGE, "Specified start_timestamp is invalid")); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); + + final ProcessContinuation result = action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + assertEquals(ProcessContinuation.stop(), result); + assertNotEquals(MAX_INCLUSIVE_END_AT, timestampCaptor.getValue()); + + verify(restrictionTracker).tryClaim(MAX_INCLUSIVE_END_AT); + verify(partitionMetadataDao).updateToFinished(PARTITION_TOKEN); + verify(metrics).decActivePartitionReadCounter(); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionMetadataDao, never()).updateWatermark(any(), any()); + } + + // Out of range indicates that we're beyond the end of the partition and should + // stop + // processing. + @Test + public void testQueryChangeStreamWithOutOfRangeErrorOnBoundedPartition() { + when(changeStreamDao.changeStreamQuery( + eq(PARTITION_TOKEN), + eq(PARTITION_START_TIMESTAMP), + eq(PARTITION_END_TIMESTAMP), + eq(PARTITION_HEARTBEAT_MILLIS))) + .thenThrow( + SpannerExceptionFactory.newSpannerException( + ErrorCode.OUT_OF_RANGE, "Specified start_timestamp is invalid")); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); + + final ProcessContinuation result = action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + assertEquals(ProcessContinuation.stop(), result); + + verify(restrictionTracker).tryClaim(PARTITION_END_TIMESTAMP); + verify(partitionMetadataDao).updateToFinished(PARTITION_TOKEN); + verify(metrics).decActivePartitionReadCounter(); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionMetadataDao, never()).updateWatermark(any(), any()); + } + + @Test + public void testQueryChangeStreamWithChildPartitionsRecordBoundedRestriction() { + final ChangeStreamResultSetMetadata resultSetMetadata = mock(ChangeStreamResultSetMetadata.class); + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final ChildPartitionsRecord record1 = mock(ChildPartitionsRecord.class); + when(record1.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); + when(changeStreamDao.changeStreamQuery( + PARTITION_TOKEN, + PARTITION_START_TIMESTAMP, + PARTITION_END_TIMESTAMP, + PARTITION_HEARTBEAT_MILLIS)) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(true, false); + when(resultSet.getMetadata()).thenReturn(resultSetMetadata); + when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) + .thenReturn(Arrays.asList(record1)); + when(childPartitionsRecordAction.run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.empty()); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); + + final ProcessContinuation result = action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + verify(childPartitionsRecordAction) + .run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(restrictionTracker).tryClaim(PARTITION_END_TIMESTAMP); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionMetadataDao, never()).updateWatermark(any(), any()); + } + + @Test + public void testQueryChangeStreamWithChildPartitionsRecordUnboundedRestriction() { + setupUnboundedPartition(); + + final ChangeStreamResultSetMetadata resultSetMetadata = mock(ChangeStreamResultSetMetadata.class); + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final ChildPartitionsRecord record1 = mock(ChildPartitionsRecord.class); + when(record1.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); + final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); + when(changeStreamDao.changeStreamQuery( + eq(PARTITION_TOKEN), + eq(PARTITION_START_TIMESTAMP), + timestampCaptor.capture(), + eq(PARTITION_HEARTBEAT_MILLIS))) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(true, false); + when(resultSet.getMetadata()).thenReturn(resultSetMetadata); + when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) + .thenReturn(Arrays.asList(record1)); + when(childPartitionsRecordAction.run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.empty()); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); + + final ProcessContinuation result = action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + assertNotEquals(MAX_INCLUSIVE_END_AT, timestampCaptor.getValue()); + verify(childPartitionsRecordAction) + .run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(restrictionTracker).tryClaim(MAX_INCLUSIVE_END_AT); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionMetadataDao, never()).updateWatermark(any(), any()); + } + + @Test + public void testQueryChangeStreamWithMutableChangeStreamCappedEndTimestamp() { + // Initialize action with isMutableChangeStream = true + action = new QueryChangeStreamAction( + changeStreamDao, + partitionMetadataDao, + changeStreamRecordMapper, + partitionMetadataMapper, + dataChangeRecordAction, + heartbeatRecordAction, + childPartitionsRecordAction, + partitionStartRecordAction, + partitionEndRecordAction, + partitionEventRecordAction, + metrics, + true); + + // Set endTimestamp to 60 minutes in the future + Timestamp now = Timestamp.now(); + Timestamp endTimestamp = Timestamp.ofTimeSecondsAndNanos(now.getSeconds() + 60 * 60, now.getNanos()); + + partition = partition.toBuilder().setEndTimestamp(endTimestamp).build(); + when(restriction.getTo()).thenReturn(endTimestamp); + when(partitionMetadataMapper.from(any())).thenReturn(partition); + + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); + when(changeStreamDao.changeStreamQuery( + eq(PARTITION_TOKEN), eq(PARTITION_START_TIMESTAMP), + timestampCaptor.capture(), eq(PARTITION_HEARTBEAT_MILLIS))) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(false); // Query finishes (reaches cap) + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); + + final ProcessContinuation result = action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + // Verify query was capped at ~2 minutes + long diff = timestampCaptor.getValue().getSeconds() - now.getSeconds(); + assertTrue("Query should be capped at approx 2 minutes (120s)", Math.abs(diff - 120) < 10); + + // Crucial: Should RESUME to process the rest later + assertEquals(ProcessContinuation.resume(), result); + } + + @Test + public void testQueryChangeStreamWithMutableChangeStreamUncappedEndTimestamp() { + action = new QueryChangeStreamAction( + changeStreamDao, + partitionMetadataDao, + changeStreamRecordMapper, + partitionMetadataMapper, + dataChangeRecordAction, + heartbeatRecordAction, + childPartitionsRecordAction, + partitionStartRecordAction, + partitionEndRecordAction, + partitionEventRecordAction, + metrics, + true); + + // Set endTimestamp to only 10 seconds in the future + Timestamp now = Timestamp.now(); + Timestamp endTimestamp = Timestamp.ofTimeSecondsAndNanos(now.getSeconds() + 10, now.getNanos()); + + partition = partition.toBuilder().setEndTimestamp(endTimestamp).build(); + when(restriction.getTo()).thenReturn(endTimestamp); + when(partitionMetadataMapper.from(any())).thenReturn(partition); + + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); + when(changeStreamDao.changeStreamQuery( + eq(PARTITION_TOKEN), eq(PARTITION_START_TIMESTAMP), + timestampCaptor.capture(), eq(PARTITION_HEARTBEAT_MILLIS))) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(false); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + when(restrictionTracker.tryClaim(endTimestamp)).thenReturn(true); + + final ProcessContinuation result = action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + // Should use the exact endTimestamp since it is within the limit (10s < 2m) + assertEquals(endTimestamp, timestampCaptor.getValue()); + + // Should STOP because we reached the actual requested endTimestamp + assertEquals(ProcessContinuation.stop(), result); + } + + @Test + public void testQueryChangeStreamUnboundedResumesCorrectly() { + // Unbounded restriction (streaming forever) + setupUnboundedPartition(); + + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + when(changeStreamDao.changeStreamQuery(any(), any(), any(), anyLong())).thenReturn(resultSet); + when(resultSet.next()).thenReturn(false); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); + + final ProcessContinuation result = action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + // Should return RESUME to continue reading the stream every 2 minutes + assertEquals(ProcessContinuation.resume(), result); + verify(metrics).incQueryCounter(); + } + + private static class BundleFinalizerStub implements BundleFinalizer { + @Override + public void afterBundleCommit(Instant callbackExpiry, Callback callback) { + try { + callback.onBundleSuccess(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } } - } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/InitializeDoFnTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/InitializeDoFnTest.java index 9672e23b16d7..c3bee10f8e14 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/InitializeDoFnTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/InitializeDoFnTest.java @@ -62,7 +62,8 @@ public void setUp() { daoFactory, mapperFactory, Timestamp.ofTimeMicroseconds(1L), - Timestamp.ofTimeMicroseconds(2L)); + Timestamp.ofTimeMicroseconds(2L), + 2000L); } @Test diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFnTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFnTest.java index 9e588de77a03..a64259049dc3 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFnTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFnTest.java @@ -62,139 +62,142 @@ @RunWith(JUnit4.class) public class ReadChangeStreamPartitionDoFnTest { - private static final String PARTITION_TOKEN = "partitionToken"; - private static final Timestamp PARTITION_START_TIMESTAMP = - Timestamp.ofTimeSecondsAndNanos(10, 20); - private static final Timestamp PARTITION_END_TIMESTAMP = Timestamp.ofTimeSecondsAndNanos(30, 40); - private static final long PARTITION_HEARTBEAT_MILLIS = 30_000L; - - private ReadChangeStreamPartitionDoFn doFn; - private PartitionMetadata partition; - private TimestampRange restriction; - private RestrictionTracker tracker; - private OutputReceiver receiver; - private ManualWatermarkEstimator watermarkEstimator; - private BundleFinalizer bundleFinalizer; - private DataChangeRecordAction dataChangeRecordAction; - private HeartbeatRecordAction heartbeatRecordAction; - private ChildPartitionsRecordAction childPartitionsRecordAction; - private PartitionStartRecordAction partitionStartRecordAction; - private PartitionEndRecordAction partitionEndRecordAction; - private PartitionEventRecordAction partitionEventRecordAction; - private QueryChangeStreamAction queryChangeStreamAction; - - @Before - public void setUp() { - final DaoFactory daoFactory = mock(DaoFactory.class); - final MapperFactory mapperFactory = mock(MapperFactory.class); - final ChangeStreamMetrics metrics = mock(ChangeStreamMetrics.class); - final BytesThroughputEstimator throughputEstimator = - mock(BytesThroughputEstimator.class); - final ActionFactory actionFactory = mock(ActionFactory.class); - final PartitionMetadataDao partitionMetadataDao = mock(PartitionMetadataDao.class); - final ChangeStreamDao changeStreamDao = mock(ChangeStreamDao.class); - final ChangeStreamRecordMapper changeStreamRecordMapper = mock(ChangeStreamRecordMapper.class); - final PartitionMetadataMapper partitionMetadataMapper = mock(PartitionMetadataMapper.class); - dataChangeRecordAction = mock(DataChangeRecordAction.class); - heartbeatRecordAction = mock(HeartbeatRecordAction.class); - childPartitionsRecordAction = mock(ChildPartitionsRecordAction.class); - partitionStartRecordAction = mock(PartitionStartRecordAction.class); - partitionEndRecordAction = mock(PartitionEndRecordAction.class); - partitionEventRecordAction = mock(PartitionEventRecordAction.class); - queryChangeStreamAction = mock(QueryChangeStreamAction.class); - - doFn = new ReadChangeStreamPartitionDoFn(daoFactory, mapperFactory, actionFactory, metrics); - doFn.setThroughputEstimator(throughputEstimator); - - partition = - PartitionMetadata.newBuilder() - .setPartitionToken(PARTITION_TOKEN) - .setParentTokens(Sets.newHashSet("parentToken")) - .setStartTimestamp(PARTITION_START_TIMESTAMP) - .setEndTimestamp(PARTITION_END_TIMESTAMP) - .setHeartbeatMillis(PARTITION_HEARTBEAT_MILLIS) - .setState(SCHEDULED) - .setWatermark(PARTITION_START_TIMESTAMP) - .setScheduledAt(Timestamp.now()) - .build(); - restriction = mock(TimestampRange.class); - tracker = mock(RestrictionTracker.class); - receiver = mock(OutputReceiver.class); - watermarkEstimator = mock(ManualWatermarkEstimator.class); - bundleFinalizer = mock(BundleFinalizer.class); - - when(tracker.currentRestriction()).thenReturn(restriction); - when(daoFactory.getPartitionMetadataDao()).thenReturn(partitionMetadataDao); - when(daoFactory.getChangeStreamDao()).thenReturn(changeStreamDao); - when(mapperFactory.changeStreamRecordMapper()).thenReturn(changeStreamRecordMapper); - when(mapperFactory.partitionMetadataMapper()).thenReturn(partitionMetadataMapper); - - when(actionFactory.dataChangeRecordAction(throughputEstimator)) - .thenReturn(dataChangeRecordAction); - when(actionFactory.heartbeatRecordAction(metrics)).thenReturn(heartbeatRecordAction); - when(actionFactory.childPartitionsRecordAction(partitionMetadataDao, metrics)) - .thenReturn(childPartitionsRecordAction); - when(actionFactory.partitionStartRecordAction(partitionMetadataDao, metrics)) - .thenReturn(partitionStartRecordAction); - when(actionFactory.partitionEndRecordAction(partitionMetadataDao, metrics)) - .thenReturn(partitionEndRecordAction); - when(actionFactory.partitionEventRecordAction(partitionMetadataDao, metrics)) - .thenReturn(partitionEventRecordAction); - when(actionFactory.queryChangeStreamAction( - eq(changeStreamDao), - eq(partitionMetadataDao), - eq(changeStreamRecordMapper), - eq(partitionMetadataMapper), - eq(dataChangeRecordAction), - eq(heartbeatRecordAction), - eq(childPartitionsRecordAction), - eq(partitionStartRecordAction), - eq(partitionEndRecordAction), - eq(partitionEventRecordAction), - eq(metrics), - anyBoolean())) - .thenReturn(queryChangeStreamAction); - - doFn.setup(); - } - - @Test - public void testQueryChangeStreamMode() { - when(queryChangeStreamAction.run(any(), any(), any(), any(), any())) - .thenReturn(ProcessContinuation.stop()); - - final ProcessContinuation result = - doFn.processElement(partition, tracker, receiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - verify(queryChangeStreamAction) - .run(partition, tracker, receiver, watermarkEstimator, bundleFinalizer); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(tracker, never()).tryClaim(any()); - } - - // -------------------------- - // Sad Paths - - // Client library errors: - // 1. RESOURCE_EXHAUSTED error on client library - // 2. DEADLINE_EXCEEDED error on client library - // 3. INTERNAL error on client library - // 4. UNAVAILABLE error on client library - // 5. UNKNOWN error on client library (transaction outcome unknown) - // 6. ABORTED error on client library - // 7. UNAUTHORIZED error on client library - - // Metadata table - // - Table is deleted - // - Database is deleted - // - No permissions for the metadata table - // -------------------------- + private static final String PARTITION_TOKEN = "partitionToken"; + private static final Timestamp PARTITION_START_TIMESTAMP = Timestamp.ofTimeSecondsAndNanos(10, 20); + private static final Timestamp PARTITION_END_TIMESTAMP = Timestamp.ofTimeSecondsAndNanos(30, 40); + private static final long PARTITION_HEARTBEAT_MILLIS = 30_000L; + + private ReadChangeStreamPartitionDoFn doFn; + private PartitionMetadata partition; + private TimestampRange restriction; + private RestrictionTracker tracker; + private OutputReceiver receiver; + private ManualWatermarkEstimator watermarkEstimator; + private BundleFinalizer bundleFinalizer; + private DataChangeRecordAction dataChangeRecordAction; + private HeartbeatRecordAction heartbeatRecordAction; + private ChildPartitionsRecordAction childPartitionsRecordAction; + private PartitionStartRecordAction partitionStartRecordAction; + private PartitionEndRecordAction partitionEndRecordAction; + private PartitionEventRecordAction partitionEventRecordAction; + private QueryChangeStreamAction queryChangeStreamAction; + + @Before + public void setUp() { + final DaoFactory daoFactory = mock(DaoFactory.class); + final MapperFactory mapperFactory = mock(MapperFactory.class); + final ChangeStreamMetrics metrics = mock(ChangeStreamMetrics.class); + final BytesThroughputEstimator throughputEstimator = mock(BytesThroughputEstimator.class); + final ActionFactory actionFactory = mock(ActionFactory.class); + final PartitionMetadataDao partitionMetadataDao = mock(PartitionMetadataDao.class); + final ChangeStreamDao changeStreamDao = mock(ChangeStreamDao.class); + final ChangeStreamRecordMapper changeStreamRecordMapper = mock(ChangeStreamRecordMapper.class); + final PartitionMetadataMapper partitionMetadataMapper = mock(PartitionMetadataMapper.class); + dataChangeRecordAction = mock(DataChangeRecordAction.class); + heartbeatRecordAction = mock(HeartbeatRecordAction.class); + childPartitionsRecordAction = mock(ChildPartitionsRecordAction.class); + partitionStartRecordAction = mock(PartitionStartRecordAction.class); + partitionEndRecordAction = mock(PartitionEndRecordAction.class); + partitionEventRecordAction = mock(PartitionEventRecordAction.class); + queryChangeStreamAction = mock(QueryChangeStreamAction.class); + + doFn = new ReadChangeStreamPartitionDoFn( + daoFactory, + mapperFactory, + actionFactory, + metrics, + org.joda.time.Duration.standardMinutes(2)); + doFn.setThroughputEstimator(throughputEstimator); + + partition = PartitionMetadata.newBuilder() + .setPartitionToken(PARTITION_TOKEN) + .setParentTokens(Sets.newHashSet("parentToken")) + .setStartTimestamp(PARTITION_START_TIMESTAMP) + .setEndTimestamp(PARTITION_END_TIMESTAMP) + .setHeartbeatMillis(PARTITION_HEARTBEAT_MILLIS) + .setState(SCHEDULED) + .setWatermark(PARTITION_START_TIMESTAMP) + .setScheduledAt(Timestamp.now()) + .build(); + restriction = mock(TimestampRange.class); + tracker = mock(RestrictionTracker.class); + receiver = mock(OutputReceiver.class); + watermarkEstimator = mock(ManualWatermarkEstimator.class); + bundleFinalizer = mock(BundleFinalizer.class); + + when(tracker.currentRestriction()).thenReturn(restriction); + when(daoFactory.getPartitionMetadataDao()).thenReturn(partitionMetadataDao); + when(daoFactory.getChangeStreamDao()).thenReturn(changeStreamDao); + when(mapperFactory.changeStreamRecordMapper()).thenReturn(changeStreamRecordMapper); + when(mapperFactory.partitionMetadataMapper()).thenReturn(partitionMetadataMapper); + + when(actionFactory.dataChangeRecordAction(throughputEstimator)) + .thenReturn(dataChangeRecordAction); + when(actionFactory.heartbeatRecordAction(metrics)).thenReturn(heartbeatRecordAction); + when(actionFactory.childPartitionsRecordAction(partitionMetadataDao, metrics)) + .thenReturn(childPartitionsRecordAction); + when(actionFactory.partitionStartRecordAction(partitionMetadataDao, metrics)) + .thenReturn(partitionStartRecordAction); + when(actionFactory.partitionEndRecordAction(partitionMetadataDao, metrics)) + .thenReturn(partitionEndRecordAction); + when(actionFactory.partitionEventRecordAction(partitionMetadataDao, metrics)) + .thenReturn(partitionEventRecordAction); + when(actionFactory.queryChangeStreamAction( + eq(changeStreamDao), + eq(partitionMetadataDao), + eq(changeStreamRecordMapper), + eq(partitionMetadataMapper), + eq(dataChangeRecordAction), + eq(heartbeatRecordAction), + eq(childPartitionsRecordAction), + eq(partitionStartRecordAction), + eq(partitionEndRecordAction), + eq(partitionEventRecordAction), + eq(metrics), + anyBoolean(), + org.joda.time.Duration.standardMinutes(2))) + .thenReturn(queryChangeStreamAction); + + doFn.setup(); + } + + @Test + public void testQueryChangeStreamMode() { + when(queryChangeStreamAction.run(any(), any(), any(), any(), any())) + .thenReturn(ProcessContinuation.stop()); + + final ProcessContinuation result = doFn.processElement(partition, tracker, receiver, watermarkEstimator, + bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + verify(queryChangeStreamAction) + .run(partition, tracker, receiver, watermarkEstimator, bundleFinalizer); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(tracker, never()).tryClaim(any()); + } + + // -------------------------- + // Sad Paths + + // Client library errors: + // 1. RESOURCE_EXHAUSTED error on client library + // 2. DEADLINE_EXCEEDED error on client library + // 3. INTERNAL error on client library + // 4. UNAVAILABLE error on client library + // 5. UNKNOWN error on client library (transaction outcome unknown) + // 6. ABORTED error on client library + // 7. UNAUTHORIZED error on client library + + // Metadata table + // - Table is deleted + // - Database is deleted + // - No permissions for the metadata table + // -------------------------- } From fc4cf57b66f2aeb52990f750dd406aeaa6138d5c Mon Sep 17 00:00:00 2001 From: Radek Stankiewicz Date: Thu, 12 Feb 2026 15:17:19 +0100 Subject: [PATCH 2/7] Introduce and use constants for Spanner Change Streams CDC time increment and heartbeat milliseconds. --- .../apache/beam/sdk/io/gcp/spanner/SpannerIO.java | 12 ++++++++---- .../changestreams/ChangeStreamsConstants.java | 8 ++++++++ 2 files changed, 16 insertions(+), 4 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index 9b6c22b024bf..1239c5c19c15 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -19,9 +19,13 @@ import static java.util.stream.Collectors.toList; import static org.apache.beam.sdk.io.gcp.spanner.MutationUtils.isPointDelete; +import static org.apache.beam.sdk.io.gcp.spanner.changestreams.ChangeStreamsConstants.DEFAULT_CDC_TIME_INCREMENT; import static org.apache.beam.sdk.io.gcp.spanner.changestreams.ChangeStreamsConstants.DEFAULT_CHANGE_STREAM_NAME; +import static org.apache.beam.sdk.io.gcp.spanner.changestreams.ChangeStreamsConstants.DEFAULT_HEARTBEAT_MILLIS; import static org.apache.beam.sdk.io.gcp.spanner.changestreams.ChangeStreamsConstants.DEFAULT_INCLUSIVE_END_AT; import static org.apache.beam.sdk.io.gcp.spanner.changestreams.ChangeStreamsConstants.DEFAULT_INCLUSIVE_START_AT; +import static org.apache.beam.sdk.io.gcp.spanner.changestreams.ChangeStreamsConstants.DEFAULT_LOW_LATENCY_CDC_TIME_INCREMENT; +import static org.apache.beam.sdk.io.gcp.spanner.changestreams.ChangeStreamsConstants.DEFAULT_LOW_LATENCY_HEARTBEAT_MILLIS; import static org.apache.beam.sdk.io.gcp.spanner.changestreams.ChangeStreamsConstants.DEFAULT_RPC_PRIORITY; import static org.apache.beam.sdk.io.gcp.spanner.changestreams.ChangeStreamsConstants.DEFAULT_WATERMARK_REFRESH_RATE; import static org.apache.beam.sdk.io.gcp.spanner.changestreams.ChangeStreamsConstants.MAX_INCLUSIVE_END_AT; @@ -537,8 +541,8 @@ public static ReadChangeStream readChangeStream() { .setRpcPriority(DEFAULT_RPC_PRIORITY) .setInclusiveStartAt(DEFAULT_INCLUSIVE_START_AT) .setInclusiveEndAt(DEFAULT_INCLUSIVE_END_AT) - .setCdcTimeIncrement(Duration.standardMinutes(2)) - .setHeartbeatMillis(2000) + .setCdcTimeIncrement(DEFAULT_CDC_TIME_INCREMENT) + .setHeartbeatMillis(DEFAULT_HEARTBEAT_MILLIS) .build(); } @@ -1931,8 +1935,8 @@ public ReadChangeStream withUsingPlainTextChannel(boolean plainText) { public ReadChangeStream withLowLatency() { return toBuilder() - .setCdcTimeIncrement(Duration.standardSeconds(1)) - .setHeartbeatMillis(100) + .setCdcTimeIncrement(DEFAULT_LOW_LATENCY_CDC_TIME_INCREMENT) + .setHeartbeatMillis(DEFAULT_LOW_LATENCY_HEARTBEAT_MILLIS) .build(); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/ChangeStreamsConstants.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/ChangeStreamsConstants.java index db09adb0f27e..191f120da196 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/ChangeStreamsConstants.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/ChangeStreamsConstants.java @@ -49,6 +49,14 @@ public class ChangeStreamsConstants { */ public static final Timestamp DEFAULT_INCLUSIVE_END_AT = MAX_INCLUSIVE_END_AT; + public static final Duration DEFAULT_CDC_TIME_INCREMENT = Duration.standardMinutes(2); + + public static final int DEFAULT_HEARTBEAT_MILLIS = 2000; + + public static final Duration DEFAULT_LOW_LATENCY_CDC_TIME_INCREMENT = Duration.standardSeconds(1); + + public static final int DEFAULT_LOW_LATENCY_HEARTBEAT_MILLIS = 100; + /** The default priority for a change stream query is {@link RpcPriority#HIGH}. */ public static final RpcPriority DEFAULT_RPC_PRIORITY = RpcPriority.HIGH; From 02035786e29a27ea1cc8678b76ab17afdc10c243 Mon Sep 17 00:00:00 2001 From: Radek Stankiewicz Date: Thu, 12 Feb 2026 15:28:54 +0100 Subject: [PATCH 3/7] spotless --- .../changestreams/action/ActionFactory.java | 177 +- .../action/QueryChangeStreamAction.java | 244 +-- .../dofn/ReadChangeStreamPartitionDoFn.java | 172 +- .../action/QueryChangeStreamActionTest.java | 1927 +++++++++-------- .../ReadChangeStreamPartitionDoFnTest.java | 278 +-- 5 files changed, 1386 insertions(+), 1412 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ActionFactory.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ActionFactory.java index 8b8ba6c4f51f..11eba7e5223d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ActionFactory.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ActionFactory.java @@ -29,8 +29,7 @@ import org.joda.time.Duration; /** - * Factory class for creating instances that will handle each type of record - * within a change stream + * Factory class for creating instances that will handle each type of record within a change stream * query. The instances created are all singletons. */ // transient fields are un-initialized, because we start them during the first @@ -50,12 +49,10 @@ public class ActionFactory implements Serializable { private transient DetectNewPartitionsAction detectNewPartitionsActionInstance; /** - * Creates and returns a singleton instance of an action class capable of - * processing {@link + * Creates and returns a singleton instance of an action class capable of processing {@link * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.DataChangeRecord}s. * - *

- * This method is thread safe. + *

This method is thread safe. * * @return singleton instance of the {@link DataChangeRecordAction} */ @@ -68,10 +65,8 @@ public synchronized DataChangeRecordAction dataChangeRecordAction( } /** - * Creates and returns a singleton instance of an action class capable of - * processing {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.HeartbeatRecord}s. - * This method is thread + * Creates and returns a singleton instance of an action class capable of processing {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.HeartbeatRecord}s. This method is thread * safe. * * @param metrics metrics gathering class @@ -85,55 +80,48 @@ public synchronized HeartbeatRecordAction heartbeatRecordAction(ChangeStreamMetr } /** - * Creates and returns a singleton instance of an action class capable of - * process {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.ChildPartitionsRecord}s. - * This method is + * Creates and returns a singleton instance of an action class capable of process {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.ChildPartitionsRecord}s. This method is * thread safe. * - * @param partitionMetadataDao DAO class to access the Connector's metadata - * tables - * @param metrics metrics gathering class + * @param partitionMetadataDao DAO class to access the Connector's metadata tables + * @param metrics metrics gathering class * @return singleton instance of the {@link ChildPartitionsRecordAction} */ public synchronized ChildPartitionsRecordAction childPartitionsRecordAction( PartitionMetadataDao partitionMetadataDao, ChangeStreamMetrics metrics) { if (childPartitionsRecordActionInstance == null) { - childPartitionsRecordActionInstance = new ChildPartitionsRecordAction(partitionMetadataDao, metrics); + childPartitionsRecordActionInstance = + new ChildPartitionsRecordAction(partitionMetadataDao, metrics); } return childPartitionsRecordActionInstance; } /** - * Creates and returns a singleton instance of an action class capable of - * process {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionStartRecord}s. - * This method is + * Creates and returns a singleton instance of an action class capable of process {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionStartRecord}s. This method is * thread safe. * - * @param partitionMetadataDao DAO class to access the Connector's metadata - * tables - * @param metrics metrics gathering class + * @param partitionMetadataDao DAO class to access the Connector's metadata tables + * @param metrics metrics gathering class * @return singleton instance of the {@link PartitionStartRecordAction} */ public synchronized PartitionStartRecordAction partitionStartRecordAction( PartitionMetadataDao partitionMetadataDao, ChangeStreamMetrics metrics) { if (partitionStartRecordActionInstance == null) { - partitionStartRecordActionInstance = new PartitionStartRecordAction(partitionMetadataDao, metrics); + partitionStartRecordActionInstance = + new PartitionStartRecordAction(partitionMetadataDao, metrics); } return partitionStartRecordActionInstance; } /** - * Creates and returns a singleton instance of an action class capable of - * process {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionEndRecord}s. - * This method is + * Creates and returns a singleton instance of an action class capable of process {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionEndRecord}s. This method is * thread safe. * - * @param partitionMetadataDao DAO class to access the Connector's metadata - * tables - * @param metrics metrics gathering class + * @param partitionMetadataDao DAO class to access the Connector's metadata tables + * @param metrics metrics gathering class * @return singleton instance of the {@link PartitionEndRecordAction} */ public synchronized PartitionEndRecordAction partitionEndRecordAction( @@ -145,15 +133,12 @@ public synchronized PartitionEndRecordAction partitionEndRecordAction( } /** - * Creates and returns a singleton instance of an action class capable of - * process {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionEventRecord}s. - * This method is + * Creates and returns a singleton instance of an action class capable of process {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionEventRecord}s. This method is * thread safe. * - * @param partitionMetadataDao DAO class to access the Connector's metadata - * tables - * @param metrics metrics gathering class + * @param partitionMetadataDao DAO class to access the Connector's metadata tables + * @param metrics metrics gathering class * @return singleton instance of the {@link PartitionEventRecordAction} */ public synchronized PartitionEventRecordAction partitionEventRecordAction( @@ -165,40 +150,32 @@ public synchronized PartitionEventRecordAction partitionEventRecordAction( } /** - * Creates and returns a single instance of an action class capable of - * performing a change stream - * query for a given partition. It uses the {@link DataChangeRecordAction}, - * {@link - * HeartbeatRecordAction}, {@link ChildPartitionsRecordAction}, - * {@link PartitionStartRecordAction} - * ,{@link PartitionEndRecordAction} and {@link PartitionEventRecordAction} to - * dispatch the + * Creates and returns a single instance of an action class capable of performing a change stream + * query for a given partition. It uses the {@link DataChangeRecordAction}, {@link + * HeartbeatRecordAction}, {@link ChildPartitionsRecordAction}, {@link PartitionStartRecordAction} + * ,{@link PartitionEndRecordAction} and {@link PartitionEventRecordAction} to dispatch the * necessary processing depending on the type of record received. * - * @param changeStreamDao DAO class to perform a change stream query - * @param partitionMetadataDao DAO class to access the Connector's - * metadata tables - * @param changeStreamRecordMapper mapper class to transform change stream - * records into the - * Connector's domain models - * @param partitionMetadataMapper mapper class to transform partition - * metadata rows into the - * Connector's domain models - * @param dataChangeRecordAction action class to process {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.DataChangeRecord}s - * @param heartbeatRecordAction action class to process {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.HeartbeatRecord}s + * @param changeStreamDao DAO class to perform a change stream query + * @param partitionMetadataDao DAO class to access the Connector's metadata tables + * @param changeStreamRecordMapper mapper class to transform change stream records into the + * Connector's domain models + * @param partitionMetadataMapper mapper class to transform partition metadata rows into the + * Connector's domain models + * @param dataChangeRecordAction action class to process {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.DataChangeRecord}s + * @param heartbeatRecordAction action class to process {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.HeartbeatRecord}s * @param childPartitionsRecordAction action class to process {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.ChildPartitionsRecord}s - * @param partitionStartRecordAction action class to process {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionStartRecord}s - * @param partitionEndRecordAction action class to process {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionEndRecord}s - * @param partitionEventRecordAction action class to process {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionEventRecord}s - * @param metrics metrics gathering class - * @param cdcTimeIncrement the duration added to current time for the - * end timestamp + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.ChildPartitionsRecord}s + * @param partitionStartRecordAction action class to process {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionStartRecord}s + * @param partitionEndRecordAction action class to process {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionEndRecord}s + * @param partitionEventRecordAction action class to process {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionEventRecord}s + * @param metrics metrics gathering class + * @param cdcTimeIncrement the duration added to current time for the end timestamp * @return single instance of the {@link QueryChangeStreamAction} */ public synchronized QueryChangeStreamAction queryChangeStreamAction( @@ -216,37 +193,34 @@ public synchronized QueryChangeStreamAction queryChangeStreamAction( boolean isMutableChangeStream, Duration cdcTimeIncrement) { if (queryChangeStreamActionInstance == null) { - queryChangeStreamActionInstance = new QueryChangeStreamAction( - changeStreamDao, - partitionMetadataDao, - changeStreamRecordMapper, - partitionMetadataMapper, - dataChangeRecordAction, - heartbeatRecordAction, - childPartitionsRecordAction, - partitionStartRecordAction, - partitionEndRecordAction, - partitionEventRecordAction, - metrics, - isMutableChangeStream, - cdcTimeIncrement); + queryChangeStreamActionInstance = + new QueryChangeStreamAction( + changeStreamDao, + partitionMetadataDao, + changeStreamRecordMapper, + partitionMetadataMapper, + dataChangeRecordAction, + heartbeatRecordAction, + childPartitionsRecordAction, + partitionStartRecordAction, + partitionEndRecordAction, + partitionEventRecordAction, + metrics, + isMutableChangeStream, + cdcTimeIncrement); } return queryChangeStreamActionInstance; } /** - * Creates and returns a single instance of an action class capable of detecting - * and scheduling + * Creates and returns a single instance of an action class capable of detecting and scheduling * new partitions to be queried. * - * @param partitionMetadataDao DAO class to access the Connector's metadata - * tables - * @param partitionMetadataMapper mapper class to transform partition metadata - * table rows into the - * Connector's domain models - * @param metrics metrics gathering class - * @param resumeDuration specifies the periodic schedule to re-execute - * the action + * @param partitionMetadataDao DAO class to access the Connector's metadata tables + * @param partitionMetadataMapper mapper class to transform partition metadata table rows into the + * Connector's domain models + * @param metrics metrics gathering class + * @param resumeDuration specifies the periodic schedule to re-execute the action * @return single instance of the {@link DetectNewPartitionsAction} */ public synchronized DetectNewPartitionsAction detectNewPartitionsAction( @@ -256,12 +230,13 @@ public synchronized DetectNewPartitionsAction detectNewPartitionsAction( ChangeStreamMetrics metrics, Duration resumeDuration) { if (detectNewPartitionsActionInstance == null) { - detectNewPartitionsActionInstance = new DetectNewPartitionsAction( - partitionMetadataDao, - partitionMetadataMapper, - watermarkCache, - metrics, - resumeDuration); + detectNewPartitionsActionInstance = + new DetectNewPartitionsAction( + partitionMetadataDao, + partitionMetadataMapper, + watermarkCache, + metrics, + resumeDuration); } return detectNewPartitionsActionInstance; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamAction.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamAction.java index cedc172a4ee6..7a9ee9134ebc 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamAction.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamAction.java @@ -54,24 +54,16 @@ import org.slf4j.LoggerFactory; /** - * Main action class for querying a partition change stream. This class will - * perform the change - * stream query and depending on the record type received, it will dispatch the - * processing of it to - * one of the following: {@link ChildPartitionsRecordAction}, - * {@link HeartbeatRecordAction}, {@link - * DataChangeRecordAction}, {@link PartitionStartRecordAction}, - * {@link PartitionEndRecordAction} or + * Main action class for querying a partition change stream. This class will perform the change + * stream query and depending on the record type received, it will dispatch the processing of it to + * one of the following: {@link ChildPartitionsRecordAction}, {@link HeartbeatRecordAction}, {@link + * DataChangeRecordAction}, {@link PartitionStartRecordAction}, {@link PartitionEndRecordAction} or * {@link PartitionEventRecordAction}. * - *

- * This class will also make sure to mirror the current watermark (event - * timestamp processed) in + *

This class will also make sure to mirror the current watermark (event timestamp processed) in * the Connector's metadata tables, by registering a bundle after commit action. * - *

- * When the change stream query for the partition is finished, this class will - * update the state + *

When the change stream query for the partition is finished, this class will update the state * of the partition in the metadata tables as FINISHED, indicating completion. */ public class QueryChangeStreamAction { @@ -105,34 +97,23 @@ public class QueryChangeStreamAction { private final Duration cdcTimeIncrement; /** - * Constructs an action class for performing a change stream query for a given - * partition. + * Constructs an action class for performing a change stream query for a given partition. * - * @param changeStreamDao DAO class to perform a change stream query - * @param partitionMetadataDao DAO class to access the Connector's - * metadata tables - * @param changeStreamRecordMapper mapper class to transform change stream - * records into the - * Connector's domain models - * @param partitionMetadataMapper mapper class to transform partition - * metadata rows into the - * Connector's domain models - * @param dataChangeRecordAction action class to process - * {@link DataChangeRecord}s - * @param heartbeatRecordAction action class to process - * {@link HeartbeatRecord}s - * @param childPartitionsRecordAction action class to process - * {@link ChildPartitionsRecord}s - * @param PartitionStartRecordAction action class to process - * {@link PartitionStartRecord}s - * @param PartitionEndRecordAction action class to process - * {@link PartitionEndRecord}s - * @param PartitionEventRecordAction action class to process - * {@link PartitionEventRecord}s - * @param metrics metrics gathering class - * @param isMutableChangeStream whether the change stream is mutable or - * not - * @param cdcTimeIncrement duration to add to current time + * @param changeStreamDao DAO class to perform a change stream query + * @param partitionMetadataDao DAO class to access the Connector's metadata tables + * @param changeStreamRecordMapper mapper class to transform change stream records into the + * Connector's domain models + * @param partitionMetadataMapper mapper class to transform partition metadata rows into the + * Connector's domain models + * @param dataChangeRecordAction action class to process {@link DataChangeRecord}s + * @param heartbeatRecordAction action class to process {@link HeartbeatRecord}s + * @param childPartitionsRecordAction action class to process {@link ChildPartitionsRecord}s + * @param PartitionStartRecordAction action class to process {@link PartitionStartRecord}s + * @param PartitionEndRecordAction action class to process {@link PartitionEndRecord}s + * @param PartitionEventRecordAction action class to process {@link PartitionEventRecord}s + * @param metrics metrics gathering class + * @param isMutableChangeStream whether the change stream is mutable or not + * @param cdcTimeIncrement duration to add to current time */ QueryChangeStreamAction( ChangeStreamDao changeStreamDao, @@ -161,54 +142,40 @@ public class QueryChangeStreamAction { } /** - * This method will dispatch a change stream query for the given partition, it - * delegate the - * processing of the records to one of the corresponding action classes - * registered and it will + * This method will dispatch a change stream query for the given partition, it delegate the + * processing of the records to one of the corresponding action classes registered and it will * keep the state of the partition up to date in the Connector's metadata table. * - *

- * The algorithm is as follows: + *

The algorithm is as follows: * *

    - *
  1. A change stream query for the partition is performed. - *
  2. For each record, we check the type of the record and dispatch the - * processing to one of - * the actions registered. - *
  3. If an {@link Optional} with a {@link ProcessContinuation#stop()} is - * returned from the - * actions, we stop processing and return. - *
  4. Before returning we register a bundle finalizer callback to update the - * watermark of the - * partition in the metadata tables to the latest processed timestamp. - *
  5. When a change stream query finishes successfully (no more records) we - * update the - * partition state to FINISHED. + *
  6. A change stream query for the partition is performed. + *
  7. For each record, we check the type of the record and dispatch the processing to one of + * the actions registered. + *
  8. If an {@link Optional} with a {@link ProcessContinuation#stop()} is returned from the + * actions, we stop processing and return. + *
  9. Before returning we register a bundle finalizer callback to update the watermark of the + * partition in the metadata tables to the latest processed timestamp. + *
  10. When a change stream query finishes successfully (no more records) we update the + * partition state to FINISHED. *
* - * There might be cases where due to a split at the exact end timestamp of a - * partition's change - * stream query, this function could process a residual with an invalid - * timestamp. In this case, + * There might be cases where due to a split at the exact end timestamp of a partition's change + * stream query, this function could process a residual with an invalid timestamp. In this case, * the error is ignored and no work is done for the residual. * - * @param partition the current partition being processed - * @param tracker the restriction tracker of the {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.dofn.ReadChangeStreamPartitionDoFn} - * SDF - * @param receiver the output receiver of the {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.dofn.ReadChangeStreamPartitionDoFn} - * SDF + * @param partition the current partition being processed + * @param tracker the restriction tracker of the {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.dofn.ReadChangeStreamPartitionDoFn} SDF + * @param receiver the output receiver of the {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.dofn.ReadChangeStreamPartitionDoFn} SDF * @param watermarkEstimator the watermark estimator of the {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.dofn.ReadChangeStreamPartitionDoFn} - * SDF - * @param bundleFinalizer the bundle finalizer for {@link - * org.apache.beam.sdk.io.gcp.spanner.changestreams.dofn.ReadChangeStreamPartitionDoFn} - * SDF - * bundles - * @return a {@link ProcessContinuation#stop()} if a record timestamp could not - * be claimed or if - * the partition processing has finished + * org.apache.beam.sdk.io.gcp.spanner.changestreams.dofn.ReadChangeStreamPartitionDoFn} SDF + * @param bundleFinalizer the bundle finalizer for {@link + * org.apache.beam.sdk.io.gcp.spanner.changestreams.dofn.ReadChangeStreamPartitionDoFn} SDF + * bundles + * @return a {@link ProcessContinuation#stop()} if a record timestamp could not be claimed or if + * the partition processing has finished */ @VisibleForTesting public ProcessContinuation run( @@ -222,22 +189,26 @@ public ProcessContinuation run( // TODO: Potentially we can avoid this fetch, by enriching the runningAt // timestamp when the // ReadChangeStreamPartitionDoFn#processElement is called - final PartitionMetadata updatedPartition = Optional.ofNullable(partitionMetadataDao.getPartition(token)) - .map(partitionMetadataMapper::from) - .orElseThrow( - () -> new IllegalStateException( - "Partition " + token + " not found in metadata table")); + final PartitionMetadata updatedPartition = + Optional.ofNullable(partitionMetadataDao.getPartition(token)) + .map(partitionMetadataMapper::from) + .orElseThrow( + () -> + new IllegalStateException( + "Partition " + token + " not found in metadata table")); // Interrupter with soft timeout to commit the work if any records have been // processed. - RestrictionInterrupter interrupter = RestrictionInterrupter.withSoftTimeout(RESTRICTION_TRACKER_TIMEOUT); + RestrictionInterrupter interrupter = + RestrictionInterrupter.withSoftTimeout(RESTRICTION_TRACKER_TIMEOUT); final Timestamp startTimestamp = tracker.currentRestriction().getFrom(); final Timestamp endTimestamp = partition.getEndTimestamp(); final boolean isBoundedRestriction = !endTimestamp.equals(MAX_INCLUSIVE_END_AT); - final Timestamp changeStreamQueryEndTimestamp = isBoundedRestriction - ? getBoundedQueryEndTimestamp(endTimestamp) - : getNextReadChangeStreamEndTimestamp(); + final Timestamp changeStreamQueryEndTimestamp = + isBoundedRestriction + ? getBoundedQueryEndTimestamp(endTimestamp) + : getNextReadChangeStreamEndTimestamp(); // Once the changeStreamQuery completes we may need to resume reading from the // partition if we @@ -252,68 +223,77 @@ public ProcessContinuation run( if (InitialPartition.isInitialPartition(partition.getPartitionToken())) { stopAfterQuerySucceeds = true; } else { - stopAfterQuerySucceeds = isBoundedRestriction && changeStreamQueryEndTimestamp.equals(endTimestamp); + stopAfterQuerySucceeds = + isBoundedRestriction && changeStreamQueryEndTimestamp.equals(endTimestamp); } - try (ChangeStreamResultSet resultSet = changeStreamDao.changeStreamQuery( - token, startTimestamp, changeStreamQueryEndTimestamp, partition.getHeartbeatMillis())) { + try (ChangeStreamResultSet resultSet = + changeStreamDao.changeStreamQuery( + token, startTimestamp, changeStreamQueryEndTimestamp, partition.getHeartbeatMillis())) { metrics.incQueryCounter(); while (resultSet.next()) { - final List records = changeStreamRecordMapper.toChangeStreamRecords( - updatedPartition, resultSet, resultSet.getMetadata()); + final List records = + changeStreamRecordMapper.toChangeStreamRecords( + updatedPartition, resultSet, resultSet.getMetadata()); Optional maybeContinuation; for (final ChangeStreamRecord record : records) { if (record instanceof DataChangeRecord) { - maybeContinuation = dataChangeRecordAction.run( - updatedPartition, - (DataChangeRecord) record, - tracker, - interrupter, - receiver, - watermarkEstimator); + maybeContinuation = + dataChangeRecordAction.run( + updatedPartition, + (DataChangeRecord) record, + tracker, + interrupter, + receiver, + watermarkEstimator); } else if (record instanceof HeartbeatRecord) { - maybeContinuation = heartbeatRecordAction.run( - updatedPartition, - (HeartbeatRecord) record, - tracker, - interrupter, - watermarkEstimator); + maybeContinuation = + heartbeatRecordAction.run( + updatedPartition, + (HeartbeatRecord) record, + tracker, + interrupter, + watermarkEstimator); } else if (record instanceof ChildPartitionsRecord) { - maybeContinuation = childPartitionsRecordAction.run( - updatedPartition, - (ChildPartitionsRecord) record, - tracker, - interrupter, - watermarkEstimator); + maybeContinuation = + childPartitionsRecordAction.run( + updatedPartition, + (ChildPartitionsRecord) record, + tracker, + interrupter, + watermarkEstimator); // Child Partition records indicate that the partition has ended. There may be // additional ChildPartitionRecords but they will share the same timestamp and // will be returned by the query and processed if it finishes successfully. stopAfterQuerySucceeds = true; } else if (record instanceof PartitionStartRecord) { - maybeContinuation = partitionStartRecordAction.run( - updatedPartition, - (PartitionStartRecord) record, - tracker, - interrupter, - watermarkEstimator); + maybeContinuation = + partitionStartRecordAction.run( + updatedPartition, + (PartitionStartRecord) record, + tracker, + interrupter, + watermarkEstimator); } else if (record instanceof PartitionEndRecord) { - maybeContinuation = partitionEndRecordAction.run( - updatedPartition, - (PartitionEndRecord) record, - tracker, - interrupter, - watermarkEstimator); + maybeContinuation = + partitionEndRecordAction.run( + updatedPartition, + (PartitionEndRecord) record, + tracker, + interrupter, + watermarkEstimator); // The PartitionEndRecord indicates that there are no more records expected // for this partition. stopAfterQuerySucceeds = true; } else if (record instanceof PartitionEventRecord) { - maybeContinuation = partitionEventRecordAction.run( - updatedPartition, - (PartitionEventRecord) record, - tracker, - interrupter, - watermarkEstimator); + maybeContinuation = + partitionEventRecordAction.run( + updatedPartition, + (PartitionEventRecord) record, + tracker, + interrupter, + watermarkEstimator); } else { LOG.error("[{}] Unknown record type {}", token, record.getClass()); throw new IllegalArgumentException("Unknown record type " + record.getClass()); @@ -422,7 +402,7 @@ private BundleFinalizer.Callback updateWatermarkCallback( private boolean isTimestampOutOfRange(SpannerException e) { return (e.getErrorCode() == ErrorCode.INVALID_ARGUMENT - || e.getErrorCode() == ErrorCode.OUT_OF_RANGE) + || e.getErrorCode() == ErrorCode.OUT_OF_RANGE) && e.getMessage() != null && e.getMessage().contains(OUT_OF_RANGE_ERROR_MESSAGE); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFn.java index dfee2cf077d3..1a33cdbe35e6 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFn.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFn.java @@ -53,16 +53,11 @@ import org.slf4j.LoggerFactory; /** - * A SDF (Splittable DoFn) class which is responsible for performing a change - * stream query for a - * given partition. A different action will be taken depending on the type of - * record received from - * the query. This component will also reflect the partition state in the - * partition metadata tables. + * A SDF (Splittable DoFn) class which is responsible for performing a change stream query for a + * given partition. A different action will be taken depending on the type of record received from + * the query. This component will also reflect the partition state in the partition metadata tables. * - *

- * The processing of a partition is delegated to the - * {@link QueryChangeStreamAction}. + *

The processing of a partition is delegated to the {@link QueryChangeStreamAction}. */ // Allows for transient QueryChangeStreamAction @SuppressWarnings("initialization.fields.uninitialized") @@ -81,8 +76,7 @@ public class ReadChangeStreamPartitionDoFn extends DoFn throughputEstimator; @@ -91,26 +85,18 @@ public class ReadChangeStreamPartitionDoFn extends DoFn newWatermarkEstimator( } /** - * The restriction for a partition will be defined from the start and end - * timestamp to query the - * partition for. The {@link TimestampRange} restriction represents a - * closed-open interval, while - * the start / end timestamps represent a closed-closed interval, so we add 1 - * nanosecond to the + * The restriction for a partition will be defined from the start and end timestamp to query the + * partition for. The {@link TimestampRange} restriction represents a closed-open interval, while + * the start / end timestamps represent a closed-closed interval, so we add 1 nanosecond to the * end timestamp to convert it to closed-open. * - *

- * In this function we also update the partition state to {@link + *

In this function we also update the partition state to {@link * PartitionMetadata.State#RUNNING}. * * @param partition the partition to be queried - * @return the timestamp range from the partition start timestamp to the - * partition end timestamp + - * 1 nanosecond + * @return the timestamp range from the partition start timestamp to the partition end timestamp + + * 1 nanosecond */ @GetInitialRestriction public TimestampRange initialRestriction(@Element PartitionMetadata partition) { final String token = partition.getPartitionToken(); final com.google.cloud.Timestamp startTimestamp = partition.getStartTimestamp(); // Range represents closed-open interval - final com.google.cloud.Timestamp endTimestamp = TimestampUtils.next(partition.getEndTimestamp()); + final com.google.cloud.Timestamp endTimestamp = + TimestampUtils.next(partition.getEndTimestamp()); final com.google.cloud.Timestamp partitionScheduledAt = partition.getScheduledAt(); - final com.google.cloud.Timestamp partitionRunningAt = daoFactory.getPartitionMetadataDao().updateToRunning(token); + final com.google.cloud.Timestamp partitionRunningAt = + daoFactory.getPartitionMetadataDao().updateToRunning(token); if (partitionScheduledAt != null && partitionRunningAt != null) { metrics.updatePartitionScheduledToRunning( @@ -177,14 +160,15 @@ public TimestampRange initialRestriction(@Element PartitionMetadata partition) { @GetSize public double getSize(@Element PartitionMetadata partition, @Restriction TimestampRange range) throws Exception { - final BigDecimal timeGapInSeconds = BigDecimal - .valueOf(newTracker(partition, range).getProgress().getWorkRemaining()); + final BigDecimal timeGapInSeconds = + BigDecimal.valueOf(newTracker(partition, range).getProgress().getWorkRemaining()); final BigDecimal throughput = BigDecimal.valueOf(this.throughputEstimator.get()); - final double size = timeGapInSeconds - .multiply(throughput) - // Cap it at Double.MAX_VALUE to avoid an overflow. - .min(MAX_DOUBLE) - .doubleValue(); + final double size = + timeGapInSeconds + .multiply(throughput) + // Cap it at Double.MAX_VALUE to avoid an overflow. + .min(MAX_DOUBLE) + .doubleValue(); LOG.debug( "getSize() = {} ({} timeGapInSeconds * {} throughput)", size, timeGapInSeconds, throughput); return size; @@ -197,10 +181,8 @@ public ReadChangeStreamPartitionRangeTracker newTracker( } /** - * Constructs instances for the {@link PartitionMetadataDao}, - * {@link ChangeStreamDao}, {@link - * ChangeStreamRecordMapper}, {@link PartitionMetadataMapper}, - * {@link DataChangeRecordAction}, + * Constructs instances for the {@link PartitionMetadataDao}, {@link ChangeStreamDao}, {@link + * ChangeStreamRecordMapper}, {@link PartitionMetadataMapper}, {@link DataChangeRecordAction}, * {@link HeartbeatRecordAction}, {@link ChildPartitionsRecordAction}, {@link * PartitionStartRecordAction}, {@link PartitionEndRecordAction}, {@link * PartitionEventRecordAction} and {@link QueryChangeStreamAction}. @@ -209,56 +191,53 @@ public ReadChangeStreamPartitionRangeTracker newTracker( public void setup() { final PartitionMetadataDao partitionMetadataDao = daoFactory.getPartitionMetadataDao(); final ChangeStreamDao changeStreamDao = daoFactory.getChangeStreamDao(); - final ChangeStreamRecordMapper changeStreamRecordMapper = mapperFactory.changeStreamRecordMapper(); + final ChangeStreamRecordMapper changeStreamRecordMapper = + mapperFactory.changeStreamRecordMapper(); final PartitionMetadataMapper partitionMetadataMapper = mapperFactory.partitionMetadataMapper(); - final DataChangeRecordAction dataChangeRecordAction = actionFactory.dataChangeRecordAction(throughputEstimator); - final HeartbeatRecordAction heartbeatRecordAction = actionFactory.heartbeatRecordAction(metrics); - final ChildPartitionsRecordAction childPartitionsRecordAction = actionFactory - .childPartitionsRecordAction(partitionMetadataDao, metrics); - final PartitionStartRecordAction partitionStartRecordAction = actionFactory - .partitionStartRecordAction(partitionMetadataDao, metrics); - final PartitionEndRecordAction partitionEndRecordAction = actionFactory - .partitionEndRecordAction(partitionMetadataDao, metrics); - final PartitionEventRecordAction partitionEventRecordAction = actionFactory - .partitionEventRecordAction(partitionMetadataDao, metrics); + final DataChangeRecordAction dataChangeRecordAction = + actionFactory.dataChangeRecordAction(throughputEstimator); + final HeartbeatRecordAction heartbeatRecordAction = + actionFactory.heartbeatRecordAction(metrics); + final ChildPartitionsRecordAction childPartitionsRecordAction = + actionFactory.childPartitionsRecordAction(partitionMetadataDao, metrics); + final PartitionStartRecordAction partitionStartRecordAction = + actionFactory.partitionStartRecordAction(partitionMetadataDao, metrics); + final PartitionEndRecordAction partitionEndRecordAction = + actionFactory.partitionEndRecordAction(partitionMetadataDao, metrics); + final PartitionEventRecordAction partitionEventRecordAction = + actionFactory.partitionEventRecordAction(partitionMetadataDao, metrics); - this.queryChangeStreamAction = actionFactory.queryChangeStreamAction( - changeStreamDao, - partitionMetadataDao, - changeStreamRecordMapper, - partitionMetadataMapper, - dataChangeRecordAction, - heartbeatRecordAction, - childPartitionsRecordAction, - partitionStartRecordAction, - partitionEndRecordAction, - partitionEventRecordAction, - metrics, - isMutableChangeStream, - cdcTimeIncrement); + this.queryChangeStreamAction = + actionFactory.queryChangeStreamAction( + changeStreamDao, + partitionMetadataDao, + changeStreamRecordMapper, + partitionMetadataMapper, + dataChangeRecordAction, + heartbeatRecordAction, + childPartitionsRecordAction, + partitionStartRecordAction, + partitionEndRecordAction, + partitionEventRecordAction, + metrics, + isMutableChangeStream, + cdcTimeIncrement); } /** - * Performs a change stream query for a given partition. A different action will - * be taken - * depending on the type of record received from the query. This component will - * also reflect the + * Performs a change stream query for a given partition. A different action will be taken + * depending on the type of record received from the query. This component will also reflect the * partition state in the partition metadata tables. * - *

- * The processing of a partition is delegated to the - * {@link QueryChangeStreamAction}. + *

The processing of a partition is delegated to the {@link QueryChangeStreamAction}. * - * @param partition the partition to be queried - * @param tracker an instance of - * {@link ReadChangeStreamPartitionRangeTracker} - * @param receiver a {@link DataChangeRecord} {@link OutputReceiver} - * @param watermarkEstimator a {@link ManualWatermarkEstimator} of - * {@link Instant} - * @param bundleFinalizer the bundle finalizer - * @return a {@link ProcessContinuation#stop()} if a record timestamp could not - * be claimed or if - * the partition processing has finished + * @param partition the partition to be queried + * @param tracker an instance of {@link ReadChangeStreamPartitionRangeTracker} + * @param receiver a {@link DataChangeRecord} {@link OutputReceiver} + * @param watermarkEstimator a {@link ManualWatermarkEstimator} of {@link Instant} + * @param bundleFinalizer the bundle finalizer + * @return a {@link ProcessContinuation#stop()} if a record timestamp could not be claimed or if + * the partition processing has finished */ @ProcessElement public ProcessContinuation processElement( @@ -277,8 +256,7 @@ public ProcessContinuation processElement( } /** - * Sets the estimator to calculate the backlog of this function. Must be called - * after the + * Sets the estimator to calculate the backlog of this function. Must be called after the * initialization of this DoFn. * * @param throughputEstimator an estimator to calculate local throughput. diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamActionTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamActionTest.java index 067b2e9e9786..9d060f4805e0 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamActionTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamActionTest.java @@ -64,950 +64,987 @@ import org.mockito.ArgumentCaptor; public class QueryChangeStreamActionTest { - private static final String PARTITION_TOKEN = "partitionToken"; - private static final Timestamp PARTITION_START_TIMESTAMP = Timestamp.ofTimeMicroseconds(10L); - private static final Timestamp RECORD_TIMESTAMP = Timestamp.ofTimeMicroseconds(20L); - private static final Timestamp PARTITION_END_TIMESTAMP = Timestamp.ofTimeMicroseconds(30L); - private static final long PARTITION_HEARTBEAT_MILLIS = 30_000L; - private static final Instant WATERMARK = Instant.now(); - private static final Timestamp WATERMARK_TIMESTAMP = Timestamp.ofTimeMicroseconds(WATERMARK.getMillis() * 1_000L); - - private ChangeStreamDao changeStreamDao; - private PartitionMetadataDao partitionMetadataDao; - private PartitionMetadata partition; - private ChangeStreamMetrics metrics; - private TimestampRange restriction; - private RestrictionTracker restrictionTracker; - private OutputReceiver outputReceiver; - private ChangeStreamRecordMapper changeStreamRecordMapper; - private PartitionMetadataMapper partitionMetadataMapper; - private ManualWatermarkEstimator watermarkEstimator; - private BundleFinalizer bundleFinalizer; - private DataChangeRecordAction dataChangeRecordAction; - private HeartbeatRecordAction heartbeatRecordAction; - private ChildPartitionsRecordAction childPartitionsRecordAction; - private PartitionStartRecordAction partitionStartRecordAction; - private PartitionEndRecordAction partitionEndRecordAction; - private PartitionEventRecordAction partitionEventRecordAction; - private QueryChangeStreamAction action; - - @Before - public void setUp() throws Exception { - changeStreamDao = mock(ChangeStreamDao.class); - partitionMetadataDao = mock(PartitionMetadataDao.class); - changeStreamRecordMapper = mock(ChangeStreamRecordMapper.class); - partitionMetadataMapper = mock(PartitionMetadataMapper.class); - dataChangeRecordAction = mock(DataChangeRecordAction.class); - heartbeatRecordAction = mock(HeartbeatRecordAction.class); - childPartitionsRecordAction = mock(ChildPartitionsRecordAction.class); - partitionStartRecordAction = mock(PartitionStartRecordAction.class); - partitionEndRecordAction = mock(PartitionEndRecordAction.class); - partitionEventRecordAction = mock(PartitionEventRecordAction.class); - metrics = mock(ChangeStreamMetrics.class); - - action = new QueryChangeStreamAction( - changeStreamDao, - partitionMetadataDao, - changeStreamRecordMapper, - partitionMetadataMapper, - dataChangeRecordAction, - heartbeatRecordAction, - childPartitionsRecordAction, - partitionStartRecordAction, - partitionEndRecordAction, - partitionEventRecordAction, - metrics, - false, - org.joda.time.Duration.standardMinutes(2)); - final Struct row = mock(Struct.class); - partition = PartitionMetadata.newBuilder() - .setPartitionToken(PARTITION_TOKEN) - .setParentTokens(Sets.newHashSet("parentToken")) - .setStartTimestamp(PARTITION_START_TIMESTAMP) - .setEndTimestamp(PARTITION_END_TIMESTAMP) - .setHeartbeatMillis(PARTITION_HEARTBEAT_MILLIS) - .setState(SCHEDULED) - .setWatermark(WATERMARK_TIMESTAMP) - .setScheduledAt(Timestamp.now()) - .build(); - restriction = mock(TimestampRange.class); - restrictionTracker = mock(RestrictionTracker.class); - outputReceiver = mock(OutputReceiver.class); - watermarkEstimator = mock(ManualWatermarkEstimator.class); - bundleFinalizer = new BundleFinalizerStub(); - - when(restrictionTracker.currentRestriction()).thenReturn(restriction); - when(restriction.getFrom()).thenReturn(PARTITION_START_TIMESTAMP); - when(restriction.getTo()).thenReturn(PARTITION_END_TIMESTAMP); - when(partitionMetadataDao.getPartition(PARTITION_TOKEN)).thenReturn(row); - when(partitionMetadataMapper.from(row)).thenReturn(partition); - } - - void setupUnboundedPartition() { - partition = PartitionMetadata.newBuilder() - .setPartitionToken(PARTITION_TOKEN) - .setParentTokens(Sets.newHashSet("parentToken")) - .setStartTimestamp(PARTITION_START_TIMESTAMP) - .setEndTimestamp(MAX_INCLUSIVE_END_AT) - .setHeartbeatMillis(PARTITION_HEARTBEAT_MILLIS) - .setState(SCHEDULED) - .setWatermark(WATERMARK_TIMESTAMP) - .setScheduledAt(Timestamp.now()) - .build(); - when(partitionMetadataMapper.from(any())).thenReturn(partition); - when(restriction.getTo()).thenReturn(MAX_INCLUSIVE_END_AT); - } - - @Test - public void testQueryChangeStreamWithDataChangeRecord() { - final Struct rowAsStruct = mock(Struct.class); - final ChangeStreamResultSetMetadata resultSetMetadata = mock(ChangeStreamResultSetMetadata.class); - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final DataChangeRecord record1 = mock(DataChangeRecord.class); - final DataChangeRecord record2 = mock(DataChangeRecord.class); - when(record1.getRecordTimestamp()).thenReturn(PARTITION_START_TIMESTAMP); - when(record2.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); - when(changeStreamDao.changeStreamQuery( - PARTITION_TOKEN, - PARTITION_START_TIMESTAMP, - PARTITION_END_TIMESTAMP, - PARTITION_HEARTBEAT_MILLIS)) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(true); - when(resultSet.getCurrentRowAsStruct()).thenReturn(rowAsStruct); - when(resultSet.getMetadata()).thenReturn(resultSetMetadata); - when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) - .thenReturn(Arrays.asList(record1, record2)); - when(dataChangeRecordAction.run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(outputReceiver), - eq(watermarkEstimator))) - .thenReturn(Optional.empty()); - when(dataChangeRecordAction.run( - eq(partition), - eq(record2), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(outputReceiver), - eq(watermarkEstimator))) - .thenReturn(Optional.of(ProcessContinuation.stop())); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - - final ProcessContinuation result = action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - verify(dataChangeRecordAction) - .run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(outputReceiver), - eq(watermarkEstimator)); - verify(dataChangeRecordAction) - .run( - eq(partition), - eq(record2), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(outputReceiver), - eq(watermarkEstimator)); - verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); - - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(restrictionTracker, never()).tryClaim(any()); - } - - @Test - public void testQueryChangeStreamWithHeartbeatRecord() { - final Struct rowAsStruct = mock(Struct.class); - final ChangeStreamResultSetMetadata resultSetMetadata = mock(ChangeStreamResultSetMetadata.class); - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final HeartbeatRecord record1 = mock(HeartbeatRecord.class); - final HeartbeatRecord record2 = mock(HeartbeatRecord.class); - when(record1.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); - when(record2.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); - when(changeStreamDao.changeStreamQuery( - PARTITION_TOKEN, - PARTITION_START_TIMESTAMP, - PARTITION_END_TIMESTAMP, - PARTITION_HEARTBEAT_MILLIS)) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(true); - when(resultSet.getCurrentRowAsStruct()).thenReturn(rowAsStruct); - when(resultSet.getMetadata()).thenReturn(resultSetMetadata); - when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) - .thenReturn(Arrays.asList(record1, record2)); - when(heartbeatRecordAction.run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.empty()); - when(heartbeatRecordAction.run( - eq(partition), - eq(record2), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.of(ProcessContinuation.stop())); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - - final ProcessContinuation result = action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - verify(heartbeatRecordAction) - .run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(heartbeatRecordAction) - .run( - eq(partition), - eq(record2), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(restrictionTracker, never()).tryClaim(any()); - } - - @Test - public void testQueryChangeStreamWithChildPartitionsRecord() { - final Struct rowAsStruct = mock(Struct.class); - final ChangeStreamResultSetMetadata resultSetMetadata = mock(ChangeStreamResultSetMetadata.class); - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final ChildPartitionsRecord record1 = mock(ChildPartitionsRecord.class); - final ChildPartitionsRecord record2 = mock(ChildPartitionsRecord.class); - when(record1.getRecordTimestamp()).thenReturn(PARTITION_START_TIMESTAMP); - when(record2.getRecordTimestamp()).thenReturn(PARTITION_START_TIMESTAMP); - when(changeStreamDao.changeStreamQuery( - PARTITION_TOKEN, - PARTITION_START_TIMESTAMP, - PARTITION_END_TIMESTAMP, - PARTITION_HEARTBEAT_MILLIS)) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(true); - when(resultSet.getCurrentRowAsStruct()).thenReturn(rowAsStruct); - when(resultSet.getMetadata()).thenReturn(resultSetMetadata); - when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) - .thenReturn(Arrays.asList(record1, record2)); - when(childPartitionsRecordAction.run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.empty()); - when(childPartitionsRecordAction.run( - eq(partition), - eq(record2), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.of(ProcessContinuation.stop())); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - - final ProcessContinuation result = action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - verify(childPartitionsRecordAction) - .run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(childPartitionsRecordAction) - .run( - eq(partition), - eq(record2), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(restrictionTracker, never()).tryClaim(any()); - } - - @Test - public void testQueryChangeStreamWithRestrictionFromAfterPartitionStart() { - final Struct rowAsStruct = mock(Struct.class); - final ChangeStreamResultSetMetadata resultSetMetadata = mock(ChangeStreamResultSetMetadata.class); - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final ChildPartitionsRecord record1 = mock(ChildPartitionsRecord.class); - final ChildPartitionsRecord record2 = mock(ChildPartitionsRecord.class); - - // From is after Partition start at - when(restriction.getFrom()).thenReturn(Timestamp.ofTimeMicroseconds(15L)); - // Both records should be included - when(record1.getRecordTimestamp()).thenReturn(Timestamp.ofTimeMicroseconds(15L)); - when(record2.getRecordTimestamp()).thenReturn(Timestamp.ofTimeMicroseconds(25L)); - when(changeStreamDao.changeStreamQuery( - PARTITION_TOKEN, - Timestamp.ofTimeMicroseconds(15L), - PARTITION_END_TIMESTAMP, - PARTITION_HEARTBEAT_MILLIS)) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(true); - when(resultSet.getCurrentRowAsStruct()).thenReturn(rowAsStruct); - when(resultSet.getMetadata()).thenReturn(resultSetMetadata); - when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) - .thenReturn(Arrays.asList(record1, record2)); - when(childPartitionsRecordAction.run( - eq(partition), - eq(record2), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.of(ProcessContinuation.stop())); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - - final ProcessContinuation result = action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - verify(childPartitionsRecordAction) - .run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(childPartitionsRecordAction) - .run( - eq(partition), - eq(record2), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(restrictionTracker, never()).tryClaim(any()); - } - - @Test - public void testQueryChangeStreamWithPartitionStartRecord() { - final ChangeStreamResultSetMetadata resultSetMetadata = mock(ChangeStreamResultSetMetadata.class); - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final PartitionStartRecord record1 = mock(PartitionStartRecord.class); - when(record1.getRecordTimestamp()).thenReturn(PARTITION_START_TIMESTAMP); - when(changeStreamDao.changeStreamQuery( - PARTITION_TOKEN, - PARTITION_START_TIMESTAMP, - PARTITION_END_TIMESTAMP, - PARTITION_HEARTBEAT_MILLIS)) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(true); - when(resultSet.getMetadata()).thenReturn(resultSetMetadata); - when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) - .thenReturn(Arrays.asList(record1)); - when(partitionStartRecordAction.run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.of(ProcessContinuation.stop())); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - - final ProcessContinuation result = action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - verify(partitionStartRecordAction) - .run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(restrictionTracker, never()).tryClaim(any()); - } - - @Test - public void testQueryChangeStreamWithRestrictionFromAfterPartitionStartForPartitionStartRecord() { - final ChangeStreamResultSetMetadata resultSetMetadata = mock(ChangeStreamResultSetMetadata.class); - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final PartitionStartRecord record1 = mock(PartitionStartRecord.class); - - // From is after Partition start at - when(restriction.getFrom()).thenReturn(Timestamp.ofTimeMicroseconds(15L)); - // This record should be included. - when(record1.getRecordTimestamp()).thenReturn(Timestamp.ofTimeMicroseconds(15L)); - when(changeStreamDao.changeStreamQuery( - PARTITION_TOKEN, - Timestamp.ofTimeMicroseconds(15L), - PARTITION_END_TIMESTAMP, - PARTITION_HEARTBEAT_MILLIS)) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(true); - when(resultSet.getMetadata()).thenReturn(resultSetMetadata); - when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) - .thenReturn(Arrays.asList(record1)); - when(partitionStartRecordAction.run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.of(ProcessContinuation.stop())); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - - final ProcessContinuation result = action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - verify(partitionStartRecordAction) - .run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(restrictionTracker, never()).tryClaim(any()); - } - - @Test - public void testQueryChangeStreamWithPartitionEndRecordBoundedRestriction() { - final ChangeStreamResultSetMetadata resultSetMetadata = mock(ChangeStreamResultSetMetadata.class); - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final PartitionEndRecord record1 = mock(PartitionEndRecord.class); - when(record1.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); - when(changeStreamDao.changeStreamQuery( - PARTITION_TOKEN, - PARTITION_START_TIMESTAMP, - PARTITION_END_TIMESTAMP, - PARTITION_HEARTBEAT_MILLIS)) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(true, false); - when(resultSet.getMetadata()).thenReturn(resultSetMetadata); - when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) - .thenReturn(Arrays.asList(record1)); - when(partitionEndRecordAction.run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.empty()); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); - - final ProcessContinuation result = action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - verify(partitionEndRecordAction) - .run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(restrictionTracker).tryClaim(PARTITION_END_TIMESTAMP); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionMetadataDao, never()).updateWatermark(any(), any()); - } - - @Test - public void testQueryChangeStreamWithPartitionEndRecordUnboundedRestriction() { - setupUnboundedPartition(); - - final ChangeStreamResultSetMetadata resultSetMetadata = mock(ChangeStreamResultSetMetadata.class); - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final PartitionEndRecord record1 = mock(PartitionEndRecord.class); - when(record1.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); - final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); - when(changeStreamDao.changeStreamQuery( - eq(PARTITION_TOKEN), - eq(PARTITION_START_TIMESTAMP), - timestampCaptor.capture(), - eq(PARTITION_HEARTBEAT_MILLIS))) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(true, false); - when(resultSet.getMetadata()).thenReturn(resultSetMetadata); - when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) - .thenReturn(Arrays.asList(record1)); - when(partitionEndRecordAction.run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.empty()); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); - - final ProcessContinuation result = action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - assertNotEquals(MAX_INCLUSIVE_END_AT, timestampCaptor.getValue()); - verify(partitionEndRecordAction) - .run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(restrictionTracker).tryClaim(MAX_INCLUSIVE_END_AT); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionMetadataDao, never()).updateWatermark(any(), any()); - } - - @Test - public void testQueryChangeStreamWithPartitionEventRecord() { - final ChangeStreamResultSetMetadata resultSetMetadata = mock(ChangeStreamResultSetMetadata.class); - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final PartitionEventRecord record1 = mock(PartitionEventRecord.class); - when(record1.getRecordTimestamp()).thenReturn(PARTITION_START_TIMESTAMP); - when(changeStreamDao.changeStreamQuery( - PARTITION_TOKEN, - PARTITION_START_TIMESTAMP, - PARTITION_END_TIMESTAMP, - PARTITION_HEARTBEAT_MILLIS)) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(true); - when(resultSet.getMetadata()).thenReturn(resultSetMetadata); - when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) - .thenReturn(Arrays.asList(record1)); - when(partitionEventRecordAction.run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.of(ProcessContinuation.stop())); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - - final ProcessContinuation result = action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - verify(partitionEventRecordAction) - .run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(restrictionTracker, never()).tryClaim(any()); - } - - @Test - public void testQueryChangeStreamWithStreamFinished() { - final ChangeStreamResultSet changeStreamResultSet = mock(ChangeStreamResultSet.class); - when(changeStreamDao.changeStreamQuery( - PARTITION_TOKEN, - PARTITION_START_TIMESTAMP, - PARTITION_END_TIMESTAMP, - PARTITION_HEARTBEAT_MILLIS)) - .thenReturn(changeStreamResultSet); - when(changeStreamResultSet.next()).thenReturn(false); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - when(restrictionTracker.tryClaim(PARTITION_END_TIMESTAMP)).thenReturn(true); - - final ProcessContinuation result = action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - verify(partitionMetadataDao).updateToFinished(PARTITION_TOKEN); - verify(metrics).decActivePartitionReadCounter(); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - } - - @Test - public void testQueryChangeStreamFinishedWithResume() { - partition = PartitionMetadata.newBuilder() - .setPartitionToken(PARTITION_TOKEN) - .setParentTokens(Sets.newHashSet("parentToken")) - .setStartTimestamp(PARTITION_START_TIMESTAMP) - .setEndTimestamp(MAX_INCLUSIVE_END_AT) - .setHeartbeatMillis(PARTITION_HEARTBEAT_MILLIS) - .setState(SCHEDULED) - .setWatermark(WATERMARK_TIMESTAMP) - .setScheduledAt(Timestamp.now()) - .build(); - when(partitionMetadataMapper.from(any())).thenReturn(partition); - - final ChangeStreamResultSet changeStreamResultSet = mock(ChangeStreamResultSet.class); - final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); - when(changeStreamDao.changeStreamQuery( - eq(PARTITION_TOKEN), - eq(PARTITION_START_TIMESTAMP), - timestampCaptor.capture(), - eq(PARTITION_HEARTBEAT_MILLIS))) - .thenReturn(changeStreamResultSet); - when(changeStreamResultSet.next()).thenReturn(false); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); - - final ProcessContinuation result = action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - assertEquals(ProcessContinuation.resume(), result); - assertNotEquals(MAX_INCLUSIVE_END_AT, timestampCaptor.getValue()); - - verify(restrictionTracker).tryClaim(timestampCaptor.getValue()); - verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); - verify(partitionMetadataDao, never()).updateToFinished(PARTITION_TOKEN); - verify(metrics, never()).decActivePartitionReadCounter(); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - } - - // Out of range indicates that we're beyond the end of the partition and should - // stop - // processing. - @Test - public void testQueryChangeStreamWithOutOfRangeErrorOnUnboundedPartition() { - setupUnboundedPartition(); - - final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); - when(changeStreamDao.changeStreamQuery( - eq(PARTITION_TOKEN), - eq(PARTITION_START_TIMESTAMP), - timestampCaptor.capture(), - eq(PARTITION_HEARTBEAT_MILLIS))) - .thenThrow( - SpannerExceptionFactory.newSpannerException( - ErrorCode.OUT_OF_RANGE, "Specified start_timestamp is invalid")); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); - - final ProcessContinuation result = action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - assertEquals(ProcessContinuation.stop(), result); - assertNotEquals(MAX_INCLUSIVE_END_AT, timestampCaptor.getValue()); - - verify(restrictionTracker).tryClaim(MAX_INCLUSIVE_END_AT); - verify(partitionMetadataDao).updateToFinished(PARTITION_TOKEN); - verify(metrics).decActivePartitionReadCounter(); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionMetadataDao, never()).updateWatermark(any(), any()); - } - - // Out of range indicates that we're beyond the end of the partition and should - // stop - // processing. - @Test - public void testQueryChangeStreamWithOutOfRangeErrorOnBoundedPartition() { - when(changeStreamDao.changeStreamQuery( - eq(PARTITION_TOKEN), - eq(PARTITION_START_TIMESTAMP), - eq(PARTITION_END_TIMESTAMP), - eq(PARTITION_HEARTBEAT_MILLIS))) - .thenThrow( - SpannerExceptionFactory.newSpannerException( - ErrorCode.OUT_OF_RANGE, "Specified start_timestamp is invalid")); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); - - final ProcessContinuation result = action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - assertEquals(ProcessContinuation.stop(), result); - - verify(restrictionTracker).tryClaim(PARTITION_END_TIMESTAMP); - verify(partitionMetadataDao).updateToFinished(PARTITION_TOKEN); - verify(metrics).decActivePartitionReadCounter(); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionMetadataDao, never()).updateWatermark(any(), any()); - } - - @Test - public void testQueryChangeStreamWithChildPartitionsRecordBoundedRestriction() { - final ChangeStreamResultSetMetadata resultSetMetadata = mock(ChangeStreamResultSetMetadata.class); - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final ChildPartitionsRecord record1 = mock(ChildPartitionsRecord.class); - when(record1.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); - when(changeStreamDao.changeStreamQuery( - PARTITION_TOKEN, - PARTITION_START_TIMESTAMP, - PARTITION_END_TIMESTAMP, - PARTITION_HEARTBEAT_MILLIS)) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(true, false); - when(resultSet.getMetadata()).thenReturn(resultSetMetadata); - when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) - .thenReturn(Arrays.asList(record1)); - when(childPartitionsRecordAction.run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.empty()); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); - - final ProcessContinuation result = action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - verify(childPartitionsRecordAction) - .run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(restrictionTracker).tryClaim(PARTITION_END_TIMESTAMP); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionMetadataDao, never()).updateWatermark(any(), any()); - } - - @Test - public void testQueryChangeStreamWithChildPartitionsRecordUnboundedRestriction() { - setupUnboundedPartition(); - - final ChangeStreamResultSetMetadata resultSetMetadata = mock(ChangeStreamResultSetMetadata.class); - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final ChildPartitionsRecord record1 = mock(ChildPartitionsRecord.class); - when(record1.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); - final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); - when(changeStreamDao.changeStreamQuery( - eq(PARTITION_TOKEN), - eq(PARTITION_START_TIMESTAMP), - timestampCaptor.capture(), - eq(PARTITION_HEARTBEAT_MILLIS))) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(true, false); - when(resultSet.getMetadata()).thenReturn(resultSetMetadata); - when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) - .thenReturn(Arrays.asList(record1)); - when(childPartitionsRecordAction.run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator))) - .thenReturn(Optional.empty()); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); - - final ProcessContinuation result = action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - assertNotEquals(MAX_INCLUSIVE_END_AT, timestampCaptor.getValue()); - verify(childPartitionsRecordAction) - .run( - eq(partition), - eq(record1), - eq(restrictionTracker), - any(RestrictionInterrupter.class), - eq(watermarkEstimator)); - verify(restrictionTracker).tryClaim(MAX_INCLUSIVE_END_AT); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionMetadataDao, never()).updateWatermark(any(), any()); - } - - @Test - public void testQueryChangeStreamWithMutableChangeStreamCappedEndTimestamp() { - // Initialize action with isMutableChangeStream = true - action = new QueryChangeStreamAction( - changeStreamDao, - partitionMetadataDao, - changeStreamRecordMapper, - partitionMetadataMapper, - dataChangeRecordAction, - heartbeatRecordAction, - childPartitionsRecordAction, - partitionStartRecordAction, - partitionEndRecordAction, - partitionEventRecordAction, - metrics, - true); - - // Set endTimestamp to 60 minutes in the future - Timestamp now = Timestamp.now(); - Timestamp endTimestamp = Timestamp.ofTimeSecondsAndNanos(now.getSeconds() + 60 * 60, now.getNanos()); - - partition = partition.toBuilder().setEndTimestamp(endTimestamp).build(); - when(restriction.getTo()).thenReturn(endTimestamp); - when(partitionMetadataMapper.from(any())).thenReturn(partition); - - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); - when(changeStreamDao.changeStreamQuery( - eq(PARTITION_TOKEN), eq(PARTITION_START_TIMESTAMP), - timestampCaptor.capture(), eq(PARTITION_HEARTBEAT_MILLIS))) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(false); // Query finishes (reaches cap) - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); - - final ProcessContinuation result = action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - // Verify query was capped at ~2 minutes - long diff = timestampCaptor.getValue().getSeconds() - now.getSeconds(); - assertTrue("Query should be capped at approx 2 minutes (120s)", Math.abs(diff - 120) < 10); - - // Crucial: Should RESUME to process the rest later - assertEquals(ProcessContinuation.resume(), result); - } - - @Test - public void testQueryChangeStreamWithMutableChangeStreamUncappedEndTimestamp() { - action = new QueryChangeStreamAction( - changeStreamDao, - partitionMetadataDao, - changeStreamRecordMapper, - partitionMetadataMapper, - dataChangeRecordAction, - heartbeatRecordAction, - childPartitionsRecordAction, - partitionStartRecordAction, - partitionEndRecordAction, - partitionEventRecordAction, - metrics, - true); - - // Set endTimestamp to only 10 seconds in the future - Timestamp now = Timestamp.now(); - Timestamp endTimestamp = Timestamp.ofTimeSecondsAndNanos(now.getSeconds() + 10, now.getNanos()); - - partition = partition.toBuilder().setEndTimestamp(endTimestamp).build(); - when(restriction.getTo()).thenReturn(endTimestamp); - when(partitionMetadataMapper.from(any())).thenReturn(partition); - - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); - when(changeStreamDao.changeStreamQuery( - eq(PARTITION_TOKEN), eq(PARTITION_START_TIMESTAMP), - timestampCaptor.capture(), eq(PARTITION_HEARTBEAT_MILLIS))) - .thenReturn(resultSet); - when(resultSet.next()).thenReturn(false); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - when(restrictionTracker.tryClaim(endTimestamp)).thenReturn(true); - - final ProcessContinuation result = action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - // Should use the exact endTimestamp since it is within the limit (10s < 2m) - assertEquals(endTimestamp, timestampCaptor.getValue()); - - // Should STOP because we reached the actual requested endTimestamp - assertEquals(ProcessContinuation.stop(), result); - } - - @Test - public void testQueryChangeStreamUnboundedResumesCorrectly() { - // Unbounded restriction (streaming forever) - setupUnboundedPartition(); - - final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); - when(changeStreamDao.changeStreamQuery(any(), any(), any(), anyLong())).thenReturn(resultSet); - when(resultSet.next()).thenReturn(false); - when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); - when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); - - final ProcessContinuation result = action.run( - partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); - - // Should return RESUME to continue reading the stream every 2 minutes - assertEquals(ProcessContinuation.resume(), result); - verify(metrics).incQueryCounter(); - } - - private static class BundleFinalizerStub implements BundleFinalizer { - @Override - public void afterBundleCommit(Instant callbackExpiry, Callback callback) { - try { - callback.onBundleSuccess(); - } catch (Exception e) { - throw new RuntimeException(e); - } - } + private static final String PARTITION_TOKEN = "partitionToken"; + private static final Timestamp PARTITION_START_TIMESTAMP = Timestamp.ofTimeMicroseconds(10L); + private static final Timestamp RECORD_TIMESTAMP = Timestamp.ofTimeMicroseconds(20L); + private static final Timestamp PARTITION_END_TIMESTAMP = Timestamp.ofTimeMicroseconds(30L); + private static final long PARTITION_HEARTBEAT_MILLIS = 30_000L; + private static final Instant WATERMARK = Instant.now(); + private static final Timestamp WATERMARK_TIMESTAMP = + Timestamp.ofTimeMicroseconds(WATERMARK.getMillis() * 1_000L); + + private ChangeStreamDao changeStreamDao; + private PartitionMetadataDao partitionMetadataDao; + private PartitionMetadata partition; + private ChangeStreamMetrics metrics; + private TimestampRange restriction; + private RestrictionTracker restrictionTracker; + private OutputReceiver outputReceiver; + private ChangeStreamRecordMapper changeStreamRecordMapper; + private PartitionMetadataMapper partitionMetadataMapper; + private ManualWatermarkEstimator watermarkEstimator; + private BundleFinalizer bundleFinalizer; + private DataChangeRecordAction dataChangeRecordAction; + private HeartbeatRecordAction heartbeatRecordAction; + private ChildPartitionsRecordAction childPartitionsRecordAction; + private PartitionStartRecordAction partitionStartRecordAction; + private PartitionEndRecordAction partitionEndRecordAction; + private PartitionEventRecordAction partitionEventRecordAction; + private QueryChangeStreamAction action; + + @Before + public void setUp() throws Exception { + changeStreamDao = mock(ChangeStreamDao.class); + partitionMetadataDao = mock(PartitionMetadataDao.class); + changeStreamRecordMapper = mock(ChangeStreamRecordMapper.class); + partitionMetadataMapper = mock(PartitionMetadataMapper.class); + dataChangeRecordAction = mock(DataChangeRecordAction.class); + heartbeatRecordAction = mock(HeartbeatRecordAction.class); + childPartitionsRecordAction = mock(ChildPartitionsRecordAction.class); + partitionStartRecordAction = mock(PartitionStartRecordAction.class); + partitionEndRecordAction = mock(PartitionEndRecordAction.class); + partitionEventRecordAction = mock(PartitionEventRecordAction.class); + metrics = mock(ChangeStreamMetrics.class); + + action = + new QueryChangeStreamAction( + changeStreamDao, + partitionMetadataDao, + changeStreamRecordMapper, + partitionMetadataMapper, + dataChangeRecordAction, + heartbeatRecordAction, + childPartitionsRecordAction, + partitionStartRecordAction, + partitionEndRecordAction, + partitionEventRecordAction, + metrics, + false, + org.joda.time.Duration.standardMinutes(2)); + final Struct row = mock(Struct.class); + partition = + PartitionMetadata.newBuilder() + .setPartitionToken(PARTITION_TOKEN) + .setParentTokens(Sets.newHashSet("parentToken")) + .setStartTimestamp(PARTITION_START_TIMESTAMP) + .setEndTimestamp(PARTITION_END_TIMESTAMP) + .setHeartbeatMillis(PARTITION_HEARTBEAT_MILLIS) + .setState(SCHEDULED) + .setWatermark(WATERMARK_TIMESTAMP) + .setScheduledAt(Timestamp.now()) + .build(); + restriction = mock(TimestampRange.class); + restrictionTracker = mock(RestrictionTracker.class); + outputReceiver = mock(OutputReceiver.class); + watermarkEstimator = mock(ManualWatermarkEstimator.class); + bundleFinalizer = new BundleFinalizerStub(); + + when(restrictionTracker.currentRestriction()).thenReturn(restriction); + when(restriction.getFrom()).thenReturn(PARTITION_START_TIMESTAMP); + when(restriction.getTo()).thenReturn(PARTITION_END_TIMESTAMP); + when(partitionMetadataDao.getPartition(PARTITION_TOKEN)).thenReturn(row); + when(partitionMetadataMapper.from(row)).thenReturn(partition); + } + + void setupUnboundedPartition() { + partition = + PartitionMetadata.newBuilder() + .setPartitionToken(PARTITION_TOKEN) + .setParentTokens(Sets.newHashSet("parentToken")) + .setStartTimestamp(PARTITION_START_TIMESTAMP) + .setEndTimestamp(MAX_INCLUSIVE_END_AT) + .setHeartbeatMillis(PARTITION_HEARTBEAT_MILLIS) + .setState(SCHEDULED) + .setWatermark(WATERMARK_TIMESTAMP) + .setScheduledAt(Timestamp.now()) + .build(); + when(partitionMetadataMapper.from(any())).thenReturn(partition); + when(restriction.getTo()).thenReturn(MAX_INCLUSIVE_END_AT); + } + + @Test + public void testQueryChangeStreamWithDataChangeRecord() { + final Struct rowAsStruct = mock(Struct.class); + final ChangeStreamResultSetMetadata resultSetMetadata = + mock(ChangeStreamResultSetMetadata.class); + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final DataChangeRecord record1 = mock(DataChangeRecord.class); + final DataChangeRecord record2 = mock(DataChangeRecord.class); + when(record1.getRecordTimestamp()).thenReturn(PARTITION_START_TIMESTAMP); + when(record2.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); + when(changeStreamDao.changeStreamQuery( + PARTITION_TOKEN, + PARTITION_START_TIMESTAMP, + PARTITION_END_TIMESTAMP, + PARTITION_HEARTBEAT_MILLIS)) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(true); + when(resultSet.getCurrentRowAsStruct()).thenReturn(rowAsStruct); + when(resultSet.getMetadata()).thenReturn(resultSetMetadata); + when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) + .thenReturn(Arrays.asList(record1, record2)); + when(dataChangeRecordAction.run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(outputReceiver), + eq(watermarkEstimator))) + .thenReturn(Optional.empty()); + when(dataChangeRecordAction.run( + eq(partition), + eq(record2), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(outputReceiver), + eq(watermarkEstimator))) + .thenReturn(Optional.of(ProcessContinuation.stop())); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + + final ProcessContinuation result = + action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + verify(dataChangeRecordAction) + .run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(outputReceiver), + eq(watermarkEstimator)); + verify(dataChangeRecordAction) + .run( + eq(partition), + eq(record2), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(outputReceiver), + eq(watermarkEstimator)); + verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); + + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(restrictionTracker, never()).tryClaim(any()); + } + + @Test + public void testQueryChangeStreamWithHeartbeatRecord() { + final Struct rowAsStruct = mock(Struct.class); + final ChangeStreamResultSetMetadata resultSetMetadata = + mock(ChangeStreamResultSetMetadata.class); + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final HeartbeatRecord record1 = mock(HeartbeatRecord.class); + final HeartbeatRecord record2 = mock(HeartbeatRecord.class); + when(record1.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); + when(record2.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); + when(changeStreamDao.changeStreamQuery( + PARTITION_TOKEN, + PARTITION_START_TIMESTAMP, + PARTITION_END_TIMESTAMP, + PARTITION_HEARTBEAT_MILLIS)) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(true); + when(resultSet.getCurrentRowAsStruct()).thenReturn(rowAsStruct); + when(resultSet.getMetadata()).thenReturn(resultSetMetadata); + when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) + .thenReturn(Arrays.asList(record1, record2)); + when(heartbeatRecordAction.run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.empty()); + when(heartbeatRecordAction.run( + eq(partition), + eq(record2), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.of(ProcessContinuation.stop())); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + + final ProcessContinuation result = + action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + verify(heartbeatRecordAction) + .run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(heartbeatRecordAction) + .run( + eq(partition), + eq(record2), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(restrictionTracker, never()).tryClaim(any()); + } + + @Test + public void testQueryChangeStreamWithChildPartitionsRecord() { + final Struct rowAsStruct = mock(Struct.class); + final ChangeStreamResultSetMetadata resultSetMetadata = + mock(ChangeStreamResultSetMetadata.class); + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final ChildPartitionsRecord record1 = mock(ChildPartitionsRecord.class); + final ChildPartitionsRecord record2 = mock(ChildPartitionsRecord.class); + when(record1.getRecordTimestamp()).thenReturn(PARTITION_START_TIMESTAMP); + when(record2.getRecordTimestamp()).thenReturn(PARTITION_START_TIMESTAMP); + when(changeStreamDao.changeStreamQuery( + PARTITION_TOKEN, + PARTITION_START_TIMESTAMP, + PARTITION_END_TIMESTAMP, + PARTITION_HEARTBEAT_MILLIS)) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(true); + when(resultSet.getCurrentRowAsStruct()).thenReturn(rowAsStruct); + when(resultSet.getMetadata()).thenReturn(resultSetMetadata); + when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) + .thenReturn(Arrays.asList(record1, record2)); + when(childPartitionsRecordAction.run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.empty()); + when(childPartitionsRecordAction.run( + eq(partition), + eq(record2), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.of(ProcessContinuation.stop())); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + + final ProcessContinuation result = + action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + verify(childPartitionsRecordAction) + .run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(childPartitionsRecordAction) + .run( + eq(partition), + eq(record2), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(restrictionTracker, never()).tryClaim(any()); + } + + @Test + public void testQueryChangeStreamWithRestrictionFromAfterPartitionStart() { + final Struct rowAsStruct = mock(Struct.class); + final ChangeStreamResultSetMetadata resultSetMetadata = + mock(ChangeStreamResultSetMetadata.class); + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final ChildPartitionsRecord record1 = mock(ChildPartitionsRecord.class); + final ChildPartitionsRecord record2 = mock(ChildPartitionsRecord.class); + + // From is after Partition start at + when(restriction.getFrom()).thenReturn(Timestamp.ofTimeMicroseconds(15L)); + // Both records should be included + when(record1.getRecordTimestamp()).thenReturn(Timestamp.ofTimeMicroseconds(15L)); + when(record2.getRecordTimestamp()).thenReturn(Timestamp.ofTimeMicroseconds(25L)); + when(changeStreamDao.changeStreamQuery( + PARTITION_TOKEN, + Timestamp.ofTimeMicroseconds(15L), + PARTITION_END_TIMESTAMP, + PARTITION_HEARTBEAT_MILLIS)) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(true); + when(resultSet.getCurrentRowAsStruct()).thenReturn(rowAsStruct); + when(resultSet.getMetadata()).thenReturn(resultSetMetadata); + when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) + .thenReturn(Arrays.asList(record1, record2)); + when(childPartitionsRecordAction.run( + eq(partition), + eq(record2), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.of(ProcessContinuation.stop())); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + + final ProcessContinuation result = + action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + verify(childPartitionsRecordAction) + .run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(childPartitionsRecordAction) + .run( + eq(partition), + eq(record2), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(restrictionTracker, never()).tryClaim(any()); + } + + @Test + public void testQueryChangeStreamWithPartitionStartRecord() { + final ChangeStreamResultSetMetadata resultSetMetadata = + mock(ChangeStreamResultSetMetadata.class); + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final PartitionStartRecord record1 = mock(PartitionStartRecord.class); + when(record1.getRecordTimestamp()).thenReturn(PARTITION_START_TIMESTAMP); + when(changeStreamDao.changeStreamQuery( + PARTITION_TOKEN, + PARTITION_START_TIMESTAMP, + PARTITION_END_TIMESTAMP, + PARTITION_HEARTBEAT_MILLIS)) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(true); + when(resultSet.getMetadata()).thenReturn(resultSetMetadata); + when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) + .thenReturn(Arrays.asList(record1)); + when(partitionStartRecordAction.run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.of(ProcessContinuation.stop())); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + + final ProcessContinuation result = + action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + verify(partitionStartRecordAction) + .run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(restrictionTracker, never()).tryClaim(any()); + } + + @Test + public void testQueryChangeStreamWithRestrictionFromAfterPartitionStartForPartitionStartRecord() { + final ChangeStreamResultSetMetadata resultSetMetadata = + mock(ChangeStreamResultSetMetadata.class); + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final PartitionStartRecord record1 = mock(PartitionStartRecord.class); + + // From is after Partition start at + when(restriction.getFrom()).thenReturn(Timestamp.ofTimeMicroseconds(15L)); + // This record should be included. + when(record1.getRecordTimestamp()).thenReturn(Timestamp.ofTimeMicroseconds(15L)); + when(changeStreamDao.changeStreamQuery( + PARTITION_TOKEN, + Timestamp.ofTimeMicroseconds(15L), + PARTITION_END_TIMESTAMP, + PARTITION_HEARTBEAT_MILLIS)) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(true); + when(resultSet.getMetadata()).thenReturn(resultSetMetadata); + when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) + .thenReturn(Arrays.asList(record1)); + when(partitionStartRecordAction.run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.of(ProcessContinuation.stop())); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + + final ProcessContinuation result = + action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + verify(partitionStartRecordAction) + .run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(restrictionTracker, never()).tryClaim(any()); + } + + @Test + public void testQueryChangeStreamWithPartitionEndRecordBoundedRestriction() { + final ChangeStreamResultSetMetadata resultSetMetadata = + mock(ChangeStreamResultSetMetadata.class); + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final PartitionEndRecord record1 = mock(PartitionEndRecord.class); + when(record1.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); + when(changeStreamDao.changeStreamQuery( + PARTITION_TOKEN, + PARTITION_START_TIMESTAMP, + PARTITION_END_TIMESTAMP, + PARTITION_HEARTBEAT_MILLIS)) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(true, false); + when(resultSet.getMetadata()).thenReturn(resultSetMetadata); + when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) + .thenReturn(Arrays.asList(record1)); + when(partitionEndRecordAction.run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.empty()); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); + + final ProcessContinuation result = + action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + verify(partitionEndRecordAction) + .run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(restrictionTracker).tryClaim(PARTITION_END_TIMESTAMP); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionMetadataDao, never()).updateWatermark(any(), any()); + } + + @Test + public void testQueryChangeStreamWithPartitionEndRecordUnboundedRestriction() { + setupUnboundedPartition(); + + final ChangeStreamResultSetMetadata resultSetMetadata = + mock(ChangeStreamResultSetMetadata.class); + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final PartitionEndRecord record1 = mock(PartitionEndRecord.class); + when(record1.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); + final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); + when(changeStreamDao.changeStreamQuery( + eq(PARTITION_TOKEN), + eq(PARTITION_START_TIMESTAMP), + timestampCaptor.capture(), + eq(PARTITION_HEARTBEAT_MILLIS))) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(true, false); + when(resultSet.getMetadata()).thenReturn(resultSetMetadata); + when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) + .thenReturn(Arrays.asList(record1)); + when(partitionEndRecordAction.run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.empty()); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); + + final ProcessContinuation result = + action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + assertNotEquals(MAX_INCLUSIVE_END_AT, timestampCaptor.getValue()); + verify(partitionEndRecordAction) + .run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(restrictionTracker).tryClaim(MAX_INCLUSIVE_END_AT); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionMetadataDao, never()).updateWatermark(any(), any()); + } + + @Test + public void testQueryChangeStreamWithPartitionEventRecord() { + final ChangeStreamResultSetMetadata resultSetMetadata = + mock(ChangeStreamResultSetMetadata.class); + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final PartitionEventRecord record1 = mock(PartitionEventRecord.class); + when(record1.getRecordTimestamp()).thenReturn(PARTITION_START_TIMESTAMP); + when(changeStreamDao.changeStreamQuery( + PARTITION_TOKEN, + PARTITION_START_TIMESTAMP, + PARTITION_END_TIMESTAMP, + PARTITION_HEARTBEAT_MILLIS)) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(true); + when(resultSet.getMetadata()).thenReturn(resultSetMetadata); + when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) + .thenReturn(Arrays.asList(record1)); + when(partitionEventRecordAction.run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.of(ProcessContinuation.stop())); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + + final ProcessContinuation result = + action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + verify(partitionEventRecordAction) + .run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(restrictionTracker, never()).tryClaim(any()); + } + + @Test + public void testQueryChangeStreamWithStreamFinished() { + final ChangeStreamResultSet changeStreamResultSet = mock(ChangeStreamResultSet.class); + when(changeStreamDao.changeStreamQuery( + PARTITION_TOKEN, + PARTITION_START_TIMESTAMP, + PARTITION_END_TIMESTAMP, + PARTITION_HEARTBEAT_MILLIS)) + .thenReturn(changeStreamResultSet); + when(changeStreamResultSet.next()).thenReturn(false); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + when(restrictionTracker.tryClaim(PARTITION_END_TIMESTAMP)).thenReturn(true); + + final ProcessContinuation result = + action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + verify(partitionMetadataDao).updateToFinished(PARTITION_TOKEN); + verify(metrics).decActivePartitionReadCounter(); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + } + + @Test + public void testQueryChangeStreamFinishedWithResume() { + partition = + PartitionMetadata.newBuilder() + .setPartitionToken(PARTITION_TOKEN) + .setParentTokens(Sets.newHashSet("parentToken")) + .setStartTimestamp(PARTITION_START_TIMESTAMP) + .setEndTimestamp(MAX_INCLUSIVE_END_AT) + .setHeartbeatMillis(PARTITION_HEARTBEAT_MILLIS) + .setState(SCHEDULED) + .setWatermark(WATERMARK_TIMESTAMP) + .setScheduledAt(Timestamp.now()) + .build(); + when(partitionMetadataMapper.from(any())).thenReturn(partition); + + final ChangeStreamResultSet changeStreamResultSet = mock(ChangeStreamResultSet.class); + final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); + when(changeStreamDao.changeStreamQuery( + eq(PARTITION_TOKEN), + eq(PARTITION_START_TIMESTAMP), + timestampCaptor.capture(), + eq(PARTITION_HEARTBEAT_MILLIS))) + .thenReturn(changeStreamResultSet); + when(changeStreamResultSet.next()).thenReturn(false); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); + + final ProcessContinuation result = + action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + assertEquals(ProcessContinuation.resume(), result); + assertNotEquals(MAX_INCLUSIVE_END_AT, timestampCaptor.getValue()); + + verify(restrictionTracker).tryClaim(timestampCaptor.getValue()); + verify(partitionMetadataDao).updateWatermark(PARTITION_TOKEN, WATERMARK_TIMESTAMP); + verify(partitionMetadataDao, never()).updateToFinished(PARTITION_TOKEN); + verify(metrics, never()).decActivePartitionReadCounter(); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + } + + // Out of range indicates that we're beyond the end of the partition and should + // stop + // processing. + @Test + public void testQueryChangeStreamWithOutOfRangeErrorOnUnboundedPartition() { + setupUnboundedPartition(); + + final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); + when(changeStreamDao.changeStreamQuery( + eq(PARTITION_TOKEN), + eq(PARTITION_START_TIMESTAMP), + timestampCaptor.capture(), + eq(PARTITION_HEARTBEAT_MILLIS))) + .thenThrow( + SpannerExceptionFactory.newSpannerException( + ErrorCode.OUT_OF_RANGE, "Specified start_timestamp is invalid")); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); + + final ProcessContinuation result = + action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + assertEquals(ProcessContinuation.stop(), result); + assertNotEquals(MAX_INCLUSIVE_END_AT, timestampCaptor.getValue()); + + verify(restrictionTracker).tryClaim(MAX_INCLUSIVE_END_AT); + verify(partitionMetadataDao).updateToFinished(PARTITION_TOKEN); + verify(metrics).decActivePartitionReadCounter(); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionMetadataDao, never()).updateWatermark(any(), any()); + } + + // Out of range indicates that we're beyond the end of the partition and should + // stop + // processing. + @Test + public void testQueryChangeStreamWithOutOfRangeErrorOnBoundedPartition() { + when(changeStreamDao.changeStreamQuery( + eq(PARTITION_TOKEN), + eq(PARTITION_START_TIMESTAMP), + eq(PARTITION_END_TIMESTAMP), + eq(PARTITION_HEARTBEAT_MILLIS))) + .thenThrow( + SpannerExceptionFactory.newSpannerException( + ErrorCode.OUT_OF_RANGE, "Specified start_timestamp is invalid")); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); + + final ProcessContinuation result = + action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + assertEquals(ProcessContinuation.stop(), result); + + verify(restrictionTracker).tryClaim(PARTITION_END_TIMESTAMP); + verify(partitionMetadataDao).updateToFinished(PARTITION_TOKEN); + verify(metrics).decActivePartitionReadCounter(); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionMetadataDao, never()).updateWatermark(any(), any()); + } + + @Test + public void testQueryChangeStreamWithChildPartitionsRecordBoundedRestriction() { + final ChangeStreamResultSetMetadata resultSetMetadata = + mock(ChangeStreamResultSetMetadata.class); + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final ChildPartitionsRecord record1 = mock(ChildPartitionsRecord.class); + when(record1.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); + when(changeStreamDao.changeStreamQuery( + PARTITION_TOKEN, + PARTITION_START_TIMESTAMP, + PARTITION_END_TIMESTAMP, + PARTITION_HEARTBEAT_MILLIS)) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(true, false); + when(resultSet.getMetadata()).thenReturn(resultSetMetadata); + when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) + .thenReturn(Arrays.asList(record1)); + when(childPartitionsRecordAction.run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.empty()); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); + + final ProcessContinuation result = + action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + verify(childPartitionsRecordAction) + .run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(restrictionTracker).tryClaim(PARTITION_END_TIMESTAMP); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionMetadataDao, never()).updateWatermark(any(), any()); + } + + @Test + public void testQueryChangeStreamWithChildPartitionsRecordUnboundedRestriction() { + setupUnboundedPartition(); + + final ChangeStreamResultSetMetadata resultSetMetadata = + mock(ChangeStreamResultSetMetadata.class); + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final ChildPartitionsRecord record1 = mock(ChildPartitionsRecord.class); + when(record1.getRecordTimestamp()).thenReturn(RECORD_TIMESTAMP); + final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); + when(changeStreamDao.changeStreamQuery( + eq(PARTITION_TOKEN), + eq(PARTITION_START_TIMESTAMP), + timestampCaptor.capture(), + eq(PARTITION_HEARTBEAT_MILLIS))) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(true, false); + when(resultSet.getMetadata()).thenReturn(resultSetMetadata); + when(changeStreamRecordMapper.toChangeStreamRecords(partition, resultSet, resultSetMetadata)) + .thenReturn(Arrays.asList(record1)); + when(childPartitionsRecordAction.run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator))) + .thenReturn(Optional.empty()); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); + + final ProcessContinuation result = + action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + assertNotEquals(MAX_INCLUSIVE_END_AT, timestampCaptor.getValue()); + verify(childPartitionsRecordAction) + .run( + eq(partition), + eq(record1), + eq(restrictionTracker), + any(RestrictionInterrupter.class), + eq(watermarkEstimator)); + verify(restrictionTracker).tryClaim(MAX_INCLUSIVE_END_AT); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionMetadataDao, never()).updateWatermark(any(), any()); + } + + @Test + public void testQueryChangeStreamWithMutableChangeStreamCappedEndTimestamp() { + // Initialize action with isMutableChangeStream = true + action = + new QueryChangeStreamAction( + changeStreamDao, + partitionMetadataDao, + changeStreamRecordMapper, + partitionMetadataMapper, + dataChangeRecordAction, + heartbeatRecordAction, + childPartitionsRecordAction, + partitionStartRecordAction, + partitionEndRecordAction, + partitionEventRecordAction, + metrics, + true); + + // Set endTimestamp to 60 minutes in the future + Timestamp now = Timestamp.now(); + Timestamp endTimestamp = + Timestamp.ofTimeSecondsAndNanos(now.getSeconds() + 60 * 60, now.getNanos()); + + partition = partition.toBuilder().setEndTimestamp(endTimestamp).build(); + when(restriction.getTo()).thenReturn(endTimestamp); + when(partitionMetadataMapper.from(any())).thenReturn(partition); + + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); + when(changeStreamDao.changeStreamQuery( + eq(PARTITION_TOKEN), eq(PARTITION_START_TIMESTAMP), + timestampCaptor.capture(), eq(PARTITION_HEARTBEAT_MILLIS))) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(false); // Query finishes (reaches cap) + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); + + final ProcessContinuation result = + action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + // Verify query was capped at ~2 minutes + long diff = timestampCaptor.getValue().getSeconds() - now.getSeconds(); + assertTrue("Query should be capped at approx 2 minutes (120s)", Math.abs(diff - 120) < 10); + + // Crucial: Should RESUME to process the rest later + assertEquals(ProcessContinuation.resume(), result); + } + + @Test + public void testQueryChangeStreamWithMutableChangeStreamUncappedEndTimestamp() { + action = + new QueryChangeStreamAction( + changeStreamDao, + partitionMetadataDao, + changeStreamRecordMapper, + partitionMetadataMapper, + dataChangeRecordAction, + heartbeatRecordAction, + childPartitionsRecordAction, + partitionStartRecordAction, + partitionEndRecordAction, + partitionEventRecordAction, + metrics, + true); + + // Set endTimestamp to only 10 seconds in the future + Timestamp now = Timestamp.now(); + Timestamp endTimestamp = Timestamp.ofTimeSecondsAndNanos(now.getSeconds() + 10, now.getNanos()); + + partition = partition.toBuilder().setEndTimestamp(endTimestamp).build(); + when(restriction.getTo()).thenReturn(endTimestamp); + when(partitionMetadataMapper.from(any())).thenReturn(partition); + + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + final ArgumentCaptor timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); + when(changeStreamDao.changeStreamQuery( + eq(PARTITION_TOKEN), eq(PARTITION_START_TIMESTAMP), + timestampCaptor.capture(), eq(PARTITION_HEARTBEAT_MILLIS))) + .thenReturn(resultSet); + when(resultSet.next()).thenReturn(false); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + when(restrictionTracker.tryClaim(endTimestamp)).thenReturn(true); + + final ProcessContinuation result = + action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + // Should use the exact endTimestamp since it is within the limit (10s < 2m) + assertEquals(endTimestamp, timestampCaptor.getValue()); + + // Should STOP because we reached the actual requested endTimestamp + assertEquals(ProcessContinuation.stop(), result); + } + + @Test + public void testQueryChangeStreamUnboundedResumesCorrectly() { + // Unbounded restriction (streaming forever) + setupUnboundedPartition(); + + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); + when(changeStreamDao.changeStreamQuery(any(), any(), any(), anyLong())).thenReturn(resultSet); + when(resultSet.next()).thenReturn(false); + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); + when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); + + final ProcessContinuation result = + action.run( + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); + + // Should return RESUME to continue reading the stream every 2 minutes + assertEquals(ProcessContinuation.resume(), result); + verify(metrics).incQueryCounter(); + } + + private static class BundleFinalizerStub implements BundleFinalizer { + @Override + public void afterBundleCommit(Instant callbackExpiry, Callback callback) { + try { + callback.onBundleSuccess(); + } catch (Exception e) { + throw new RuntimeException(e); + } } + } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFnTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFnTest.java index a64259049dc3..e0ce161a8eae 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFnTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFnTest.java @@ -62,142 +62,146 @@ @RunWith(JUnit4.class) public class ReadChangeStreamPartitionDoFnTest { - private static final String PARTITION_TOKEN = "partitionToken"; - private static final Timestamp PARTITION_START_TIMESTAMP = Timestamp.ofTimeSecondsAndNanos(10, 20); - private static final Timestamp PARTITION_END_TIMESTAMP = Timestamp.ofTimeSecondsAndNanos(30, 40); - private static final long PARTITION_HEARTBEAT_MILLIS = 30_000L; - - private ReadChangeStreamPartitionDoFn doFn; - private PartitionMetadata partition; - private TimestampRange restriction; - private RestrictionTracker tracker; - private OutputReceiver receiver; - private ManualWatermarkEstimator watermarkEstimator; - private BundleFinalizer bundleFinalizer; - private DataChangeRecordAction dataChangeRecordAction; - private HeartbeatRecordAction heartbeatRecordAction; - private ChildPartitionsRecordAction childPartitionsRecordAction; - private PartitionStartRecordAction partitionStartRecordAction; - private PartitionEndRecordAction partitionEndRecordAction; - private PartitionEventRecordAction partitionEventRecordAction; - private QueryChangeStreamAction queryChangeStreamAction; - - @Before - public void setUp() { - final DaoFactory daoFactory = mock(DaoFactory.class); - final MapperFactory mapperFactory = mock(MapperFactory.class); - final ChangeStreamMetrics metrics = mock(ChangeStreamMetrics.class); - final BytesThroughputEstimator throughputEstimator = mock(BytesThroughputEstimator.class); - final ActionFactory actionFactory = mock(ActionFactory.class); - final PartitionMetadataDao partitionMetadataDao = mock(PartitionMetadataDao.class); - final ChangeStreamDao changeStreamDao = mock(ChangeStreamDao.class); - final ChangeStreamRecordMapper changeStreamRecordMapper = mock(ChangeStreamRecordMapper.class); - final PartitionMetadataMapper partitionMetadataMapper = mock(PartitionMetadataMapper.class); - dataChangeRecordAction = mock(DataChangeRecordAction.class); - heartbeatRecordAction = mock(HeartbeatRecordAction.class); - childPartitionsRecordAction = mock(ChildPartitionsRecordAction.class); - partitionStartRecordAction = mock(PartitionStartRecordAction.class); - partitionEndRecordAction = mock(PartitionEndRecordAction.class); - partitionEventRecordAction = mock(PartitionEventRecordAction.class); - queryChangeStreamAction = mock(QueryChangeStreamAction.class); - - doFn = new ReadChangeStreamPartitionDoFn( - daoFactory, - mapperFactory, - actionFactory, - metrics, - org.joda.time.Duration.standardMinutes(2)); - doFn.setThroughputEstimator(throughputEstimator); - - partition = PartitionMetadata.newBuilder() - .setPartitionToken(PARTITION_TOKEN) - .setParentTokens(Sets.newHashSet("parentToken")) - .setStartTimestamp(PARTITION_START_TIMESTAMP) - .setEndTimestamp(PARTITION_END_TIMESTAMP) - .setHeartbeatMillis(PARTITION_HEARTBEAT_MILLIS) - .setState(SCHEDULED) - .setWatermark(PARTITION_START_TIMESTAMP) - .setScheduledAt(Timestamp.now()) - .build(); - restriction = mock(TimestampRange.class); - tracker = mock(RestrictionTracker.class); - receiver = mock(OutputReceiver.class); - watermarkEstimator = mock(ManualWatermarkEstimator.class); - bundleFinalizer = mock(BundleFinalizer.class); - - when(tracker.currentRestriction()).thenReturn(restriction); - when(daoFactory.getPartitionMetadataDao()).thenReturn(partitionMetadataDao); - when(daoFactory.getChangeStreamDao()).thenReturn(changeStreamDao); - when(mapperFactory.changeStreamRecordMapper()).thenReturn(changeStreamRecordMapper); - when(mapperFactory.partitionMetadataMapper()).thenReturn(partitionMetadataMapper); - - when(actionFactory.dataChangeRecordAction(throughputEstimator)) - .thenReturn(dataChangeRecordAction); - when(actionFactory.heartbeatRecordAction(metrics)).thenReturn(heartbeatRecordAction); - when(actionFactory.childPartitionsRecordAction(partitionMetadataDao, metrics)) - .thenReturn(childPartitionsRecordAction); - when(actionFactory.partitionStartRecordAction(partitionMetadataDao, metrics)) - .thenReturn(partitionStartRecordAction); - when(actionFactory.partitionEndRecordAction(partitionMetadataDao, metrics)) - .thenReturn(partitionEndRecordAction); - when(actionFactory.partitionEventRecordAction(partitionMetadataDao, metrics)) - .thenReturn(partitionEventRecordAction); - when(actionFactory.queryChangeStreamAction( - eq(changeStreamDao), - eq(partitionMetadataDao), - eq(changeStreamRecordMapper), - eq(partitionMetadataMapper), - eq(dataChangeRecordAction), - eq(heartbeatRecordAction), - eq(childPartitionsRecordAction), - eq(partitionStartRecordAction), - eq(partitionEndRecordAction), - eq(partitionEventRecordAction), - eq(metrics), - anyBoolean(), - org.joda.time.Duration.standardMinutes(2))) - .thenReturn(queryChangeStreamAction); - - doFn.setup(); - } - - @Test - public void testQueryChangeStreamMode() { - when(queryChangeStreamAction.run(any(), any(), any(), any(), any())) - .thenReturn(ProcessContinuation.stop()); - - final ProcessContinuation result = doFn.processElement(partition, tracker, receiver, watermarkEstimator, - bundleFinalizer); - - assertEquals(ProcessContinuation.stop(), result); - verify(queryChangeStreamAction) - .run(partition, tracker, receiver, watermarkEstimator, bundleFinalizer); - - verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); - verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); - verify(tracker, never()).tryClaim(any()); - } - - // -------------------------- - // Sad Paths - - // Client library errors: - // 1. RESOURCE_EXHAUSTED error on client library - // 2. DEADLINE_EXCEEDED error on client library - // 3. INTERNAL error on client library - // 4. UNAVAILABLE error on client library - // 5. UNKNOWN error on client library (transaction outcome unknown) - // 6. ABORTED error on client library - // 7. UNAUTHORIZED error on client library - - // Metadata table - // - Table is deleted - // - Database is deleted - // - No permissions for the metadata table - // -------------------------- + private static final String PARTITION_TOKEN = "partitionToken"; + private static final Timestamp PARTITION_START_TIMESTAMP = + Timestamp.ofTimeSecondsAndNanos(10, 20); + private static final Timestamp PARTITION_END_TIMESTAMP = Timestamp.ofTimeSecondsAndNanos(30, 40); + private static final long PARTITION_HEARTBEAT_MILLIS = 30_000L; + + private ReadChangeStreamPartitionDoFn doFn; + private PartitionMetadata partition; + private TimestampRange restriction; + private RestrictionTracker tracker; + private OutputReceiver receiver; + private ManualWatermarkEstimator watermarkEstimator; + private BundleFinalizer bundleFinalizer; + private DataChangeRecordAction dataChangeRecordAction; + private HeartbeatRecordAction heartbeatRecordAction; + private ChildPartitionsRecordAction childPartitionsRecordAction; + private PartitionStartRecordAction partitionStartRecordAction; + private PartitionEndRecordAction partitionEndRecordAction; + private PartitionEventRecordAction partitionEventRecordAction; + private QueryChangeStreamAction queryChangeStreamAction; + + @Before + public void setUp() { + final DaoFactory daoFactory = mock(DaoFactory.class); + final MapperFactory mapperFactory = mock(MapperFactory.class); + final ChangeStreamMetrics metrics = mock(ChangeStreamMetrics.class); + final BytesThroughputEstimator throughputEstimator = + mock(BytesThroughputEstimator.class); + final ActionFactory actionFactory = mock(ActionFactory.class); + final PartitionMetadataDao partitionMetadataDao = mock(PartitionMetadataDao.class); + final ChangeStreamDao changeStreamDao = mock(ChangeStreamDao.class); + final ChangeStreamRecordMapper changeStreamRecordMapper = mock(ChangeStreamRecordMapper.class); + final PartitionMetadataMapper partitionMetadataMapper = mock(PartitionMetadataMapper.class); + dataChangeRecordAction = mock(DataChangeRecordAction.class); + heartbeatRecordAction = mock(HeartbeatRecordAction.class); + childPartitionsRecordAction = mock(ChildPartitionsRecordAction.class); + partitionStartRecordAction = mock(PartitionStartRecordAction.class); + partitionEndRecordAction = mock(PartitionEndRecordAction.class); + partitionEventRecordAction = mock(PartitionEventRecordAction.class); + queryChangeStreamAction = mock(QueryChangeStreamAction.class); + + doFn = + new ReadChangeStreamPartitionDoFn( + daoFactory, + mapperFactory, + actionFactory, + metrics, + org.joda.time.Duration.standardMinutes(2)); + doFn.setThroughputEstimator(throughputEstimator); + + partition = + PartitionMetadata.newBuilder() + .setPartitionToken(PARTITION_TOKEN) + .setParentTokens(Sets.newHashSet("parentToken")) + .setStartTimestamp(PARTITION_START_TIMESTAMP) + .setEndTimestamp(PARTITION_END_TIMESTAMP) + .setHeartbeatMillis(PARTITION_HEARTBEAT_MILLIS) + .setState(SCHEDULED) + .setWatermark(PARTITION_START_TIMESTAMP) + .setScheduledAt(Timestamp.now()) + .build(); + restriction = mock(TimestampRange.class); + tracker = mock(RestrictionTracker.class); + receiver = mock(OutputReceiver.class); + watermarkEstimator = mock(ManualWatermarkEstimator.class); + bundleFinalizer = mock(BundleFinalizer.class); + + when(tracker.currentRestriction()).thenReturn(restriction); + when(daoFactory.getPartitionMetadataDao()).thenReturn(partitionMetadataDao); + when(daoFactory.getChangeStreamDao()).thenReturn(changeStreamDao); + when(mapperFactory.changeStreamRecordMapper()).thenReturn(changeStreamRecordMapper); + when(mapperFactory.partitionMetadataMapper()).thenReturn(partitionMetadataMapper); + + when(actionFactory.dataChangeRecordAction(throughputEstimator)) + .thenReturn(dataChangeRecordAction); + when(actionFactory.heartbeatRecordAction(metrics)).thenReturn(heartbeatRecordAction); + when(actionFactory.childPartitionsRecordAction(partitionMetadataDao, metrics)) + .thenReturn(childPartitionsRecordAction); + when(actionFactory.partitionStartRecordAction(partitionMetadataDao, metrics)) + .thenReturn(partitionStartRecordAction); + when(actionFactory.partitionEndRecordAction(partitionMetadataDao, metrics)) + .thenReturn(partitionEndRecordAction); + when(actionFactory.partitionEventRecordAction(partitionMetadataDao, metrics)) + .thenReturn(partitionEventRecordAction); + when(actionFactory.queryChangeStreamAction( + eq(changeStreamDao), + eq(partitionMetadataDao), + eq(changeStreamRecordMapper), + eq(partitionMetadataMapper), + eq(dataChangeRecordAction), + eq(heartbeatRecordAction), + eq(childPartitionsRecordAction), + eq(partitionStartRecordAction), + eq(partitionEndRecordAction), + eq(partitionEventRecordAction), + eq(metrics), + anyBoolean(), + org.joda.time.Duration.standardMinutes(2))) + .thenReturn(queryChangeStreamAction); + + doFn.setup(); + } + + @Test + public void testQueryChangeStreamMode() { + when(queryChangeStreamAction.run(any(), any(), any(), any(), any())) + .thenReturn(ProcessContinuation.stop()); + + final ProcessContinuation result = + doFn.processElement(partition, tracker, receiver, watermarkEstimator, bundleFinalizer); + + assertEquals(ProcessContinuation.stop(), result); + verify(queryChangeStreamAction) + .run(partition, tracker, receiver, watermarkEstimator, bundleFinalizer); + + verify(dataChangeRecordAction, never()).run(any(), any(), any(), any(), any(), any()); + verify(heartbeatRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(childPartitionsRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionStartRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEndRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); + verify(tracker, never()).tryClaim(any()); + } + + // -------------------------- + // Sad Paths + + // Client library errors: + // 1. RESOURCE_EXHAUSTED error on client library + // 2. DEADLINE_EXCEEDED error on client library + // 3. INTERNAL error on client library + // 4. UNAVAILABLE error on client library + // 5. UNKNOWN error on client library (transaction outcome unknown) + // 6. ABORTED error on client library + // 7. UNAUTHORIZED error on client library + + // Metadata table + // - Table is deleted + // - Database is deleted + // - No permissions for the metadata table + // -------------------------- } From 54478a1ddaf12476ed148bc4d7de8f764f05cbd5 Mon Sep 17 00:00:00 2001 From: Radek Stankiewicz Date: Thu, 12 Feb 2026 15:40:06 +0100 Subject: [PATCH 4/7] Fix compilation issue --- .../changestreams/dofn/ReadChangeStreamPartitionDoFn.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFn.java index 1a33cdbe35e6..6be085d99a75 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFn.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFn.java @@ -103,7 +103,10 @@ public ReadChangeStreamPartitionDoFn( DaoFactory daoFactory, MapperFactory mapperFactory, ActionFactory actionFactory, + ChangeStreamMetrics metrics, Duration cdcTimeIncrement) { + this.daoFactory = daoFactory; + this.actionFactory = actionFactory; this.mapperFactory = mapperFactory; this.metrics = metrics; this.isMutableChangeStream = daoFactory.isMutableChangeStream(); From dadd770b23a0862e10baf9b9392627f9cb0629bf Mon Sep 17 00:00:00 2001 From: Radek Stankiewicz Date: Thu, 12 Feb 2026 16:03:19 +0100 Subject: [PATCH 5/7] fix comments --- .../changestreams/action/ActionFactory.java | 3 +- .../action/QueryChangeStreamAction.java | 73 +++++++------------ .../action/QueryChangeStreamActionTest.java | 13 ++-- .../ReadChangeStreamPartitionDoFnTest.java | 20 ++--- 4 files changed, 46 insertions(+), 63 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ActionFactory.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ActionFactory.java index 11eba7e5223d..5b11650dbb45 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ActionFactory.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ActionFactory.java @@ -32,8 +32,7 @@ * Factory class for creating instances that will handle each type of record within a change stream * query. The instances created are all singletons. */ -// transient fields are un-initialized, because we start them during the first -// fetch call (with the +// transient fields are un-initialized, because we start them during the first fetch call (with the // singleton pattern). @SuppressWarnings("initialization.field.uninitialized") public class ActionFactory implements Serializable { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamAction.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamAction.java index 7a9ee9134ebc..cfc797445d37 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamAction.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamAction.java @@ -71,12 +71,9 @@ public class QueryChangeStreamAction { private static final Logger LOG = LoggerFactory.getLogger(QueryChangeStreamAction.class); private static final Duration BUNDLE_FINALIZER_TIMEOUT = Duration.standardMinutes(5); /* - * Corresponds to the best effort timeout in case the restriction tracker cannot - * split the processing - * interval before the hard deadline. When reached it will assure that the - * already processed timestamps - * will be committed instead of thrown away (DEADLINE_EXCEEDED). The value - * should be less than + * Corresponds to the best effort timeout in case the restriction tracker cannot split the processing + * interval before the hard deadline. When reached it will assure that the already processed timestamps + * will be committed instead of thrown away (DEADLINE_EXCEEDED). The value should be less than * the RetrySetting RPC timeout setting of SpannerIO#ReadChangeStream. */ private static final Duration RESTRICTION_TRACKER_TIMEOUT = Duration.standardSeconds(40); @@ -130,8 +127,11 @@ public class QueryChangeStreamAction { boolean isMutableChangeStream, Duration cdcTimeIncrement) { this.changeStreamDao = changeStreamDao; + this.partitionMetadataDao = partitionMetadataDao; this.changeStreamRecordMapper = changeStreamRecordMapper; + this.partitionMetadataMapper = partitionMetadataMapper; this.dataChangeRecordAction = dataChangeRecordAction; + this.heartbeatRecordAction = heartbeatRecordAction; this.childPartitionsRecordAction = childPartitionsRecordAction; this.partitionStartRecordAction = partitionStartRecordAction; this.partitionEndRecordAction = partitionEndRecordAction; @@ -186,8 +186,7 @@ public ProcessContinuation run( BundleFinalizer bundleFinalizer) { final String token = partition.getPartitionToken(); - // TODO: Potentially we can avoid this fetch, by enriching the runningAt - // timestamp when the + // TODO: Potentially we can avoid this fetch, by enriching the runningAt timestamp when the // ReadChangeStreamPartitionDoFn#processElement is called final PartitionMetadata updatedPartition = Optional.ofNullable(partitionMetadataDao.getPartition(token)) @@ -197,8 +196,7 @@ public ProcessContinuation run( new IllegalStateException( "Partition " + token + " not found in metadata table")); - // Interrupter with soft timeout to commit the work if any records have been - // processed. + // Interrupter with soft timeout to commit the work if any records have been processed. RestrictionInterrupter interrupter = RestrictionInterrupter.withSoftTimeout(RESTRICTION_TRACKER_TIMEOUT); @@ -210,14 +208,10 @@ public ProcessContinuation run( ? getBoundedQueryEndTimestamp(endTimestamp) : getNextReadChangeStreamEndTimestamp(); - // Once the changeStreamQuery completes we may need to resume reading from the - // partition if we - // had an unbounded restriction for which we set an arbitrary query end - // timestamp and for which - // we didn't encounter any indications that the partition is done (explicit end - // records or - // exceptions about being out of timestamp range). We also special case the - // InitialPartition, + // Once the changeStreamQuery completes we may need to resume reading from the partition if we + // had an unbounded restriction for which we set an arbitrary query end timestamp and for which + // we didn't encounter any indications that the partition is done (explicit end records or + // exceptions about being out of timestamp range). We also special case the InitialPartition, // which always stops after the query succeeds. boolean stopAfterQuerySucceeds = false; if (InitialPartition.isInitialPartition(partition.getPartitionToken())) { @@ -310,14 +304,11 @@ public ProcessContinuation run( } } catch (SpannerException e) { /* - * If there is a split when a partition is supposed to be finished, the residual - * will try - * to perform a change stream query for an out of range interval. We ignore this - * error - * here, and the residual should be able to claim the end of the timestamp - * range, finishing - * the partition. - */ + If there is a split when a partition is supposed to be finished, the residual will try + to perform a change stream query for an out of range interval. We ignore this error + here, and the residual should be able to claim the end of the timestamp range, finishing + the partition. + */ if (!isTimestampOutOfRange(e)) { throw e; } @@ -342,10 +333,8 @@ public ProcessContinuation run( "[{}] change stream completed successfully up to {}", token, changeStreamQueryEndTimestamp); if (!stopAfterQuerySucceeds) { - // Records stopped being returned for the query due to our artificial query end - // timestamp but - // we want to continue processing the partition, resuming from - // changeStreamQueryEndTimestamp. + // Records stopped being returned for the query due to our artificial query end timestamp but + // we want to continue processing the partition, resuming from changeStreamQueryEndTimestamp. if (!tracker.tryClaim(changeStreamQueryEndTimestamp)) { return ProcessContinuation.stop(); } @@ -357,25 +346,20 @@ public ProcessContinuation run( } // Otherwise we have finished processing the partition, either due to: - // 1. reading to the bounded restriction end timestamp - // 2. encountering a ChildPartitionRecord or EndPartitionRecord indicating there - // are no more - // elements in the partition - // 3. encountering a exception indicating the start timestamp is out of bounds - // of the - // partition - // We claim the restriction completely to satisfy internal sanity checks and do - // not reschedule + // 1. reading to the bounded restriction end timestamp + // 2. encountering a ChildPartitionRecord or EndPartitionRecord indicating there are no more + // elements in the partition + // 3. encountering a exception indicating the start timestamp is out of bounds of the + // partition + // We claim the restriction completely to satisfy internal sanity checks and do not reschedule // the restriction. if (!tracker.tryClaim(endTimestamp)) { return ProcessContinuation.stop(); } LOG.debug("[{}] Finishing partition", token); - // TODO: This should be performed after the commit succeeds. Since bundle - // finalizers are not - // guaranteed to be called, this needs to be performed in a subsequent fused - // stage. + // TODO: This should be performed after the commit succeeds. Since bundle finalizers are not + // guaranteed to be called, this needs to be performed in a subsequent fused stage. partitionMetadataDao.updateToFinished(token); metrics.decActivePartitionReadCounter(); LOG.info("[{}] After attempting to finish the partition", token); @@ -420,8 +404,7 @@ private Timestamp getNextReadChangeStreamEndTimestamp() { return Timestamp.ofTimeSecondsAndNanos(seconds, nanos); } - // For Mutable Change Stream bounded queries, update the query end timestamp to - // be within 2 + // For Mutable Change Stream bounded queries, update the query end timestamp to be within 2 // minutes in the future. private Timestamp getBoundedQueryEndTimestamp(Timestamp endTimestamp) { if (this.isMutableChangeStream) { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamActionTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamActionTest.java index 9d060f4805e0..e3d22a466ea3 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamActionTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamActionTest.java @@ -58,6 +58,7 @@ import org.apache.beam.sdk.transforms.splittabledofn.ManualWatermarkEstimator; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; +import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; import org.junit.Test; @@ -748,8 +749,7 @@ public void testQueryChangeStreamFinishedWithResume() { verify(partitionEventRecordAction, never()).run(any(), any(), any(), any(), any()); } - // Out of range indicates that we're beyond the end of the partition and should - // stop + // Out of range indicates that we're beyond the end of the partition and should stop // processing. @Test public void testQueryChangeStreamWithOutOfRangeErrorOnUnboundedPartition() { @@ -786,8 +786,7 @@ public void testQueryChangeStreamWithOutOfRangeErrorOnUnboundedPartition() { verify(partitionMetadataDao, never()).updateWatermark(any(), any()); } - // Out of range indicates that we're beyond the end of the partition and should - // stop + // Out of range indicates that we're beyond the end of the partition and should stop // processing. @Test public void testQueryChangeStreamWithOutOfRangeErrorOnBoundedPartition() { @@ -938,7 +937,8 @@ public void testQueryChangeStreamWithMutableChangeStreamCappedEndTimestamp() { partitionEndRecordAction, partitionEventRecordAction, metrics, - true); + true, + Duration.standardMinutes(2)); // Set endTimestamp to 60 minutes in the future Timestamp now = Timestamp.now(); @@ -986,7 +986,8 @@ public void testQueryChangeStreamWithMutableChangeStreamUncappedEndTimestamp() { partitionEndRecordAction, partitionEventRecordAction, metrics, - true); + true, + Duration.standardMinutes(2)); // Set endTimestamp to only 10 seconds in the future Timestamp now = Timestamp.now(); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFnTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFnTest.java index e0ce161a8eae..def919aa9de8 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFnTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFnTest.java @@ -190,18 +190,18 @@ public void testQueryChangeStreamMode() { // Sad Paths // Client library errors: - // 1. RESOURCE_EXHAUSTED error on client library - // 2. DEADLINE_EXCEEDED error on client library - // 3. INTERNAL error on client library - // 4. UNAVAILABLE error on client library - // 5. UNKNOWN error on client library (transaction outcome unknown) - // 6. ABORTED error on client library - // 7. UNAUTHORIZED error on client library + // 1. RESOURCE_EXHAUSTED error on client library + // 2. DEADLINE_EXCEEDED error on client library + // 3. INTERNAL error on client library + // 4. UNAVAILABLE error on client library + // 5. UNKNOWN error on client library (transaction outcome unknown) + // 6. ABORTED error on client library + // 7. UNAUTHORIZED error on client library // Metadata table - // - Table is deleted - // - Database is deleted - // - No permissions for the metadata table + // - Table is deleted + // - Database is deleted + // - No permissions for the metadata table // -------------------------- } From a7eda6a1c1a7e132ac3a63b49391f2e06855b859 Mon Sep 17 00:00:00 2001 From: Radek Stankiewicz Date: Thu, 12 Feb 2026 19:17:47 +0100 Subject: [PATCH 6/7] fix test --- .../dofn/ReadChangeStreamPartitionDoFnTest.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFnTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFnTest.java index def919aa9de8..f5236596a0fd 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFnTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFnTest.java @@ -53,6 +53,7 @@ import org.apache.beam.sdk.transforms.splittabledofn.ManualWatermarkEstimator; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; +import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; import org.junit.Test; @@ -109,7 +110,7 @@ public void setUp() { mapperFactory, actionFactory, metrics, - org.joda.time.Duration.standardMinutes(2)); + Duration.standardMinutes(2)); doFn.setThroughputEstimator(throughputEstimator); partition = @@ -159,7 +160,7 @@ public void setUp() { eq(partitionEventRecordAction), eq(metrics), anyBoolean(), - org.joda.time.Duration.standardMinutes(2))) + eq(Duration.standardMinutes(2)))) .thenReturn(queryChangeStreamAction); doFn.setup(); From f57f7f52bfd224d353263648e05bf9244397350d Mon Sep 17 00:00:00 2001 From: Radek Stankiewicz Date: Thu, 12 Feb 2026 20:01:05 +0100 Subject: [PATCH 7/7] spotless --- .../dofn/ReadChangeStreamPartitionDoFnTest.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFnTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFnTest.java index f5236596a0fd..a4172ec05ed9 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFnTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/ReadChangeStreamPartitionDoFnTest.java @@ -106,11 +106,7 @@ public void setUp() { doFn = new ReadChangeStreamPartitionDoFn( - daoFactory, - mapperFactory, - actionFactory, - metrics, - Duration.standardMinutes(2)); + daoFactory, mapperFactory, actionFactory, metrics, Duration.standardMinutes(2)); doFn.setThroughputEstimator(throughputEstimator); partition =