|
21 | 21 | import static org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionMetadata.State.SCHEDULED; |
22 | 22 | import static org.junit.Assert.assertEquals; |
23 | 23 | import static org.junit.Assert.assertNotEquals; |
| 24 | +import static org.junit.Assert.assertTrue; |
24 | 25 | import static org.mockito.ArgumentMatchers.any; |
| 26 | +import static org.mockito.ArgumentMatchers.anyLong; |
25 | 27 | import static org.mockito.ArgumentMatchers.eq; |
26 | 28 | import static org.mockito.Mockito.mock; |
27 | 29 | import static org.mockito.Mockito.never; |
@@ -116,7 +118,8 @@ public void setUp() throws Exception { |
116 | 118 | partitionStartRecordAction, |
117 | 119 | partitionEndRecordAction, |
118 | 120 | partitionEventRecordAction, |
119 | | - metrics); |
| 121 | + metrics, |
| 122 | + false); |
120 | 123 | final Struct row = mock(Struct.class); |
121 | 124 | partition = |
122 | 125 | PartitionMetadata.newBuilder() |
@@ -916,6 +919,118 @@ public void testQueryChangeStreamWithChildPartitionsRecordUnboundedRestriction() |
916 | 919 | verify(partitionMetadataDao, never()).updateWatermark(any(), any()); |
917 | 920 | } |
918 | 921 |
|
| 922 | + @Test |
| 923 | + public void testQueryChangeStreamWithMutableChangeStreamCappedEndTimestamp() { |
| 924 | + // Initialize action with isMutableChangeStream = true |
| 925 | + action = |
| 926 | + new QueryChangeStreamAction( |
| 927 | + changeStreamDao, |
| 928 | + partitionMetadataDao, |
| 929 | + changeStreamRecordMapper, |
| 930 | + partitionMetadataMapper, |
| 931 | + dataChangeRecordAction, |
| 932 | + heartbeatRecordAction, |
| 933 | + childPartitionsRecordAction, |
| 934 | + partitionStartRecordAction, |
| 935 | + partitionEndRecordAction, |
| 936 | + partitionEventRecordAction, |
| 937 | + metrics, |
| 938 | + true); |
| 939 | + |
| 940 | + // Set endTimestamp to 60 minutes in the future |
| 941 | + Timestamp now = Timestamp.now(); |
| 942 | + Timestamp endTimestamp = |
| 943 | + Timestamp.ofTimeSecondsAndNanos(now.getSeconds() + 60 * 60, now.getNanos()); |
| 944 | + |
| 945 | + partition = partition.toBuilder().setEndTimestamp(endTimestamp).build(); |
| 946 | + when(restriction.getTo()).thenReturn(endTimestamp); |
| 947 | + when(partitionMetadataMapper.from(any())).thenReturn(partition); |
| 948 | + |
| 949 | + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); |
| 950 | + final ArgumentCaptor<Timestamp> timestampCaptor = ArgumentCaptor.forClass(Timestamp.class); |
| 951 | + when(changeStreamDao.changeStreamQuery( |
| 952 | + eq(PARTITION_TOKEN), eq(PARTITION_START_TIMESTAMP), |
| 953 | + timestampCaptor.capture(), eq(PARTITION_HEARTBEAT_MILLIS))) |
| 954 | + .thenReturn(resultSet); |
| 955 | + when(resultSet.next()).thenReturn(false); // Query finishes (reaches cap) |
| 956 | + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); |
| 957 | + when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); |
| 958 | + |
| 959 | + final ProcessContinuation result = |
| 960 | + action.run( |
| 961 | + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); |
| 962 | + |
| 963 | + // Verify query was capped at ~30 minutes |
| 964 | + long diff = timestampCaptor.getValue().getSeconds() - now.getSeconds(); |
| 965 | + assertTrue("Query should be capped at approx 30 minutes (1800s)", Math.abs(diff - 1800) < 10); |
| 966 | + |
| 967 | + // Crucial: Should RESUME to process the rest later |
| 968 | + assertEquals(ProcessContinuation.resume(), result); |
| 969 | + } |
| 970 | + |
| 971 | + @Test |
| 972 | + public void testQueryChangeStreamWithMutableChangeStreamUncappedEndTimestamp() { |
| 973 | + action = |
| 974 | + new QueryChangeStreamAction( |
| 975 | + changeStreamDao, |
| 976 | + partitionMetadataDao, |
| 977 | + changeStreamRecordMapper, |
| 978 | + partitionMetadataMapper, |
| 979 | + dataChangeRecordAction, |
| 980 | + heartbeatRecordAction, |
| 981 | + childPartitionsRecordAction, |
| 982 | + partitionStartRecordAction, |
| 983 | + partitionEndRecordAction, |
| 984 | + partitionEventRecordAction, |
| 985 | + metrics, |
| 986 | + true); |
| 987 | + |
| 988 | + // Set endTimestamp to only 10 minutes in the future |
| 989 | + Timestamp now = Timestamp.now(); |
| 990 | + Timestamp endTimestamp = |
| 991 | + Timestamp.ofTimeSecondsAndNanos(now.getSeconds() + 10 * 60, now.getNanos()); |
| 992 | + |
| 993 | + partition = partition.toBuilder().setEndTimestamp(endTimestamp).build(); |
| 994 | + when(restriction.getTo()).thenReturn(endTimestamp); |
| 995 | + when(partitionMetadataMapper.from(any())).thenReturn(partition); |
| 996 | + |
| 997 | + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); |
| 998 | + when(changeStreamDao.changeStreamQuery( |
| 999 | + eq(PARTITION_TOKEN), eq(PARTITION_START_TIMESTAMP), |
| 1000 | + eq(endTimestamp), eq(PARTITION_HEARTBEAT_MILLIS))) |
| 1001 | + .thenReturn(resultSet); |
| 1002 | + when(resultSet.next()).thenReturn(false); |
| 1003 | + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); |
| 1004 | + when(restrictionTracker.tryClaim(endTimestamp)).thenReturn(true); |
| 1005 | + |
| 1006 | + final ProcessContinuation result = |
| 1007 | + action.run( |
| 1008 | + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); |
| 1009 | + |
| 1010 | + // Should STOP because we reached the actual requested endTimestamp |
| 1011 | + assertEquals(ProcessContinuation.stop(), result); |
| 1012 | + } |
| 1013 | + |
| 1014 | + @Test |
| 1015 | + public void testQueryChangeStreamUnboundedResumesCorrectly() { |
| 1016 | + // Unbounded restriction (streaming forever) |
| 1017 | + setupUnboundedPartition(); |
| 1018 | + |
| 1019 | + final ChangeStreamResultSet resultSet = mock(ChangeStreamResultSet.class); |
| 1020 | + when(changeStreamDao.changeStreamQuery(any(), any(), any(), anyLong())).thenReturn(resultSet); |
| 1021 | + when(resultSet.next()).thenReturn(false); |
| 1022 | + when(watermarkEstimator.currentWatermark()).thenReturn(WATERMARK); |
| 1023 | + when(restrictionTracker.tryClaim(any(Timestamp.class))).thenReturn(true); |
| 1024 | + |
| 1025 | + final ProcessContinuation result = |
| 1026 | + action.run( |
| 1027 | + partition, restrictionTracker, outputReceiver, watermarkEstimator, bundleFinalizer); |
| 1028 | + |
| 1029 | + // Should return RESUME to continue reading the stream every 2 minutes |
| 1030 | + assertEquals(ProcessContinuation.resume(), result); |
| 1031 | + verify(metrics).incQueryCounter(); |
| 1032 | + } |
| 1033 | + |
919 | 1034 | private static class BundleFinalizerStub implements BundleFinalizer { |
920 | 1035 | @Override |
921 | 1036 | public void afterBundleCommit(Instant callbackExpiry, Callback callback) { |
|
0 commit comments