jeffkbkim commented on code in PR #14849:
URL: https://github.com/apache/kafka/pull/14849#discussion_r1412701656


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java:
##########
@@ -1659,4 +1680,117 @@ public void testPartitionLoadSensor() {
 
         verify(runtimeMetrics, 
times(1)).recordPartitionLoadSensor(startTimeMs, startTimeMs + 1000);
     }
+
+    @Test
+    public void testPartitionLoadGeneratesSnapshotAtHighWatermark() {
+        MockTimer timer = new MockTimer();
+        MockPartitionWriter writer = mock(MockPartitionWriter.class);
+        MockCoordinatorShardBuilderSupplier supplier = 
mock(MockCoordinatorShardBuilderSupplier.class);
+        MockCoordinatorShardBuilder builder = 
mock(MockCoordinatorShardBuilder.class);
+        MockCoordinatorShard coordinator = mock(MockCoordinatorShard.class);
+        GroupCoordinatorRuntimeMetrics runtimeMetrics = 
mock(GroupCoordinatorRuntimeMetrics.class);
+
+        CoordinatorRuntime<MockCoordinatorShard, String> runtime =
+            new CoordinatorRuntime.Builder<MockCoordinatorShard, String>()
+                .withTime(Time.SYSTEM)
+                .withTimer(timer)
+                .withLoader(new MockCoordinatorLoader(
+                    new CoordinatorLoader.LoadSummary(
+                        1000,
+                        2000,
+                        30,
+                        3000),
+                    Arrays.asList(5L, 15L, 27L),
+                    Arrays.asList(5L, 15L)))
+                .withEventProcessor(new DirectEventProcessor())
+                .withPartitionWriter(writer)
+                .withCoordinatorShardBuilderSupplier(supplier)
+                .withCoordinatorRuntimeMetrics(runtimeMetrics)
+                .withCoordinatorMetrics(mock(GroupCoordinatorMetrics.class))
+                .build();
+
+        when(builder.withSnapshotRegistry(any())).thenReturn(builder);
+        when(builder.withLogContext(any())).thenReturn(builder);
+        when(builder.withTime(any())).thenReturn(builder);
+        when(builder.withTimer(any())).thenReturn(builder);
+        when(builder.withCoordinatorMetrics(any())).thenReturn(builder);
+        when(builder.withTopicPartition(any())).thenReturn(builder);
+        when(builder.build()).thenReturn(coordinator);
+        when(supplier.get()).thenReturn(builder);
+
+        // Getting the coordinator context fails because the coordinator
+        // does not exist until scheduleLoadOperation is called.
+        assertThrows(NotCoordinatorException.class, () -> 
runtime.contextOrThrow(TP));
+
+        // Schedule the loading.
+        runtime.scheduleLoadOperation(TP, 0);
+
+        // Getting the coordinator context succeeds now.
+        CoordinatorRuntime<MockCoordinatorShard, String>.CoordinatorContext 
ctx = runtime.contextOrThrow(TP);
+
+        // When the loading completes, the coordinator transitions to active.
+        assertEquals(ACTIVE, ctx.state);
+
+        assertEquals(27L, ctx.stateMachine.lastWrittenOffset());
+        assertEquals(15L, ctx.stateMachine.lastCommittedOffset());
+        assertFalse(ctx.stateMachine.snapshotRegistry().hasSnapshot(0L));

Review Comment:
   i'm not sure what you mean. this is to confirm that previous snapshots were 
deleted



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to