adixitconfluent commented on code in PR #17573:
URL: https://github.com/apache/kafka/pull/17573#discussion_r1816648309


##########
core/src/test/java/kafka/server/share/SharePartitionTest.java:
##########
@@ -189,6 +194,176 @@ public void testMaybeInitialize() {
         assertNull(sharePartition.cachedState().get(11L).offsetState());
     }
 
+    @Test
+    public void 
testMaybeInitializeDefaultStartEpochGroupConfigReturnsEarliest() {
+        Persister persister = Mockito.mock(Persister.class);
+        ReadShareGroupStateResult readShareGroupStateResult = 
Mockito.mock(ReadShareGroupStateResult.class);
+        
Mockito.when(readShareGroupStateResult.topicsData()).thenReturn(Collections.singletonList(
+            new TopicData<>(TOPIC_ID_PARTITION.topicId(), 
Collections.singletonList(
+                PartitionFactory.newPartitionAllData(
+                    0, PartitionFactory.DEFAULT_STATE_EPOCH,
+                    PartitionFactory.UNINITIALIZED_START_OFFSET,
+                    PartitionFactory.DEFAULT_ERROR_CODE,
+                    PartitionFactory.DEFAULT_ERR_MESSAGE,
+                    Collections.emptyList())))));
+        
Mockito.when(persister.readState(Mockito.any())).thenReturn(CompletableFuture.completedFuture(readShareGroupStateResult));
+
+        GroupConfigManager groupConfigManager = 
Mockito.mock(GroupConfigManager.class);
+        GroupConfig groupConfig = Mockito.mock(GroupConfig.class);
+        
Mockito.when(groupConfigManager.groupConfig(GROUP_ID)).thenReturn(Optional.of(groupConfig));
+        
Mockito.when(groupConfig.shareAutoOffsetReset()).thenReturn(GroupConfig.ShareGroupAutoOffsetReset.EARLIEST);
+
+        ReplicaManager replicaManager = Mockito.mock(ReplicaManager.class);
+
+        FileRecords.TimestampAndOffset timestampAndOffset = new 
FileRecords.TimestampAndOffset(-1L, 0L, Optional.empty());
+        Mockito.doReturn(new 
OffsetResultHolder(Option.apply(timestampAndOffset), Option.empty())).
+            
when(replicaManager).fetchOffsetForTimestamp(Mockito.any(TopicPartition.class), 
Mockito.anyLong(), Mockito.any(), Mockito.any(), Mockito.anyBoolean());
+
+        SharePartition sharePartition = SharePartitionBuilder.builder()
+            .withPersister(persister)
+            .withGroupConfigManager(groupConfigManager)
+            .withReplicaManager(replicaManager)
+            .build();
+
+        CompletableFuture<Void> result = sharePartition.maybeInitialize();
+        assertTrue(result.isDone());
+        assertFalse(result.isCompletedExceptionally());
+
+        assertEquals(SharePartitionState.ACTIVE, 
sharePartition.partitionState());
+        assertTrue(sharePartition.cachedState().isEmpty());
+        assertEquals(0, sharePartition.startOffset());

Review Comment:
   How is this verifying whether we are getting the LATEST or EARLIEST 
timestamp? I think we should check number of calls to 
`offsetForLatestTimestamp` and `offsetForEarliestTimestamp`. Same for the rest 
of the tests



##########
core/src/test/java/kafka/server/share/SharePartitionTest.java:
##########
@@ -189,6 +194,176 @@ public void testMaybeInitialize() {
         assertNull(sharePartition.cachedState().get(11L).offsetState());
     }
 
+    @Test
+    public void 
testMaybeInitializeDefaultStartEpochGroupConfigReturnsEarliest() {
+        Persister persister = Mockito.mock(Persister.class);
+        ReadShareGroupStateResult readShareGroupStateResult = 
Mockito.mock(ReadShareGroupStateResult.class);
+        
Mockito.when(readShareGroupStateResult.topicsData()).thenReturn(Collections.singletonList(
+            new TopicData<>(TOPIC_ID_PARTITION.topicId(), 
Collections.singletonList(
+                PartitionFactory.newPartitionAllData(
+                    0, PartitionFactory.DEFAULT_STATE_EPOCH,
+                    PartitionFactory.UNINITIALIZED_START_OFFSET,
+                    PartitionFactory.DEFAULT_ERROR_CODE,
+                    PartitionFactory.DEFAULT_ERR_MESSAGE,
+                    Collections.emptyList())))));
+        
Mockito.when(persister.readState(Mockito.any())).thenReturn(CompletableFuture.completedFuture(readShareGroupStateResult));
+
+        GroupConfigManager groupConfigManager = 
Mockito.mock(GroupConfigManager.class);
+        GroupConfig groupConfig = Mockito.mock(GroupConfig.class);
+        
Mockito.when(groupConfigManager.groupConfig(GROUP_ID)).thenReturn(Optional.of(groupConfig));
+        
Mockito.when(groupConfig.shareAutoOffsetReset()).thenReturn(GroupConfig.ShareGroupAutoOffsetReset.EARLIEST);
+
+        ReplicaManager replicaManager = Mockito.mock(ReplicaManager.class);
+
+        FileRecords.TimestampAndOffset timestampAndOffset = new 
FileRecords.TimestampAndOffset(-1L, 0L, Optional.empty());
+        Mockito.doReturn(new 
OffsetResultHolder(Option.apply(timestampAndOffset), Option.empty())).
+            
when(replicaManager).fetchOffsetForTimestamp(Mockito.any(TopicPartition.class), 
Mockito.anyLong(), Mockito.any(), Mockito.any(), Mockito.anyBoolean());
+
+        SharePartition sharePartition = SharePartitionBuilder.builder()
+            .withPersister(persister)
+            .withGroupConfigManager(groupConfigManager)
+            .withReplicaManager(replicaManager)
+            .build();
+
+        CompletableFuture<Void> result = sharePartition.maybeInitialize();
+        assertTrue(result.isDone());
+        assertFalse(result.isCompletedExceptionally());
+
+        assertEquals(SharePartitionState.ACTIVE, 
sharePartition.partitionState());
+        assertTrue(sharePartition.cachedState().isEmpty());

Review Comment:
   Why do we want to verify this in this test's context? Same for 
`assertEquals(0, sharePartition.nextFetchOffset())`



##########
core/src/test/java/kafka/test/api/ShareConsumerTest.java:
##########
@@ -209,11 +224,12 @@ public void testSubscribeSubscribeEmptyPollFails() {
 
     @Test
     public void testSubscriptionAndPoll() {
+        KafkaShareConsumer<byte[], byte[]> shareConsumer = 
createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), 
"group1");
+        shareConsumer.subscribe(Collections.singleton(tp.topic()));
+        alterShareAutoOffsetReset("group1", "earliest");

Review Comment:
   is there a way by which we can change the default only for our tests? That 
would be better, I guess



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to