jsancio commented on a change in pull request #10593: URL: https://github.com/apache/kafka/pull/10593#discussion_r727462578
########## File path: raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java ########## @@ -1533,6 +1508,112 @@ public void testFetchSnapshotRequestClusterIdValidation() throws Exception { context.assertSentFetchSnapshotResponse(Errors.INCONSISTENT_CLUSTER_ID); } + @Test + public void testCreateSnapshotAsLeaderWithInvalidSnapshotId() throws Exception { + int localId = 0; + int otherNodeId = localId + 1; + Set<Integer> voters = Utils.mkSet(localId, otherNodeId); + int epoch = 2; + + List<String> appendRecords = Arrays.asList("a", "b", "c"); + OffsetAndEpoch invalidSnapshotId1 = new OffsetAndEpoch(3, epoch); + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .appendToLog(epoch, appendRecords) + .withAppendLingerMs(1) + .build(); + + context.becomeLeader(); + int currentEpoch = context.currentEpoch(); + + // When leader creating snapshot: + // 1.1 high watermark cannot be empty + assertEquals(OptionalLong.empty(), context.client.highWatermark()); + assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId1.offset, invalidSnapshotId1.epoch)); + + // 1.2 high watermark must larger than or equal to the snapshotId's endOffset + context.advanceLocalLeaderHighWatermarkToLogEndOffset(); + // append some more records to make the LEO > high watermark + List<String> newRecords = Arrays.asList("d", "e", "f"); + context.client.scheduleAppend(currentEpoch, newRecords); + context.time.sleep(context.appendLingerMs()); + context.client.poll(); + assertEquals(context.log.endOffset().offset, context.client.highWatermark().getAsLong() + newRecords.size()); + + OffsetAndEpoch invalidSnapshotId2 = new OffsetAndEpoch(context.client.highWatermark().getAsLong() + 1, currentEpoch); + assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId2.offset, invalidSnapshotId2.epoch)); + + // 2 the quorum epoch must larger than or equal to the snapshotId's epoch + OffsetAndEpoch invalidSnapshotId3 = new OffsetAndEpoch(context.client.highWatermark().getAsLong() - 2, currentEpoch + 1); + assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId3.offset, invalidSnapshotId3.epoch)); + + // 3 the snapshotId should be validated against endOffsetForEpoch + OffsetAndEpoch endOffsetForEpoch = context.log.endOffsetForEpoch(epoch); + assertEquals(epoch, endOffsetForEpoch.epoch); + OffsetAndEpoch invalidSnapshotId4 = new OffsetAndEpoch(endOffsetForEpoch.offset + 1, epoch); + assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId4.offset, invalidSnapshotId4.epoch)); + } + + @Test + public void testCreateSnapshotAsFollowerWithInvalidSnapshotId() throws Exception { + int localId = 0; + int otherNodeId = 1; + int epoch = 5; + Set<Integer> voters = Utils.mkSet(localId, otherNodeId); + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .withElectedLeader(epoch, otherNodeId) + .build(); + context.assertElectedLeader(epoch, otherNodeId); + + // When follower creating snapshot: + // 1.1) high watermark cannot be empty + assertEquals(OptionalLong.empty(), context.client.highWatermark()); + OffsetAndEpoch invalidSnapshotId1 = new OffsetAndEpoch(0, 0); + assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId1.offset, invalidSnapshotId1.epoch)); + + // Poll for our first fetch request + context.pollUntilRequest(); + RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); + assertTrue(voters.contains(fetchRequest.destinationId())); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + + // The response does not advance the high watermark + List<String> records1 = Arrays.asList("a", "b", "c"); + MemoryRecords batch1 = context.buildBatch(0L, 3, records1); + context.deliverResponse(fetchRequest.correlationId, fetchRequest.destinationId(), + context.fetchResponse(epoch, otherNodeId, batch1, 0L, Errors.NONE)); + context.client.poll(); + + // 1.2) high watermark must larger than or equal to the snapshotId's endOffset + int currentEpoch = context.currentEpoch(); + OffsetAndEpoch invalidSnapshotId2 = new OffsetAndEpoch(context.client.highWatermark().getAsLong() + 1, currentEpoch); + assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId2.offset, invalidSnapshotId2.epoch)); + + // 2) the quorum epoch must larger than or equal to the snapshotId's epoch + OffsetAndEpoch invalidSnapshotId3 = new OffsetAndEpoch(context.client.highWatermark().getAsLong() - 2, currentEpoch + 1); + assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId3.offset, invalidSnapshotId3.epoch)); + + // The high watermark advances to be larger than log.endOffsetForEpoch(3), to test the case 3 + context.pollUntilRequest(); + fetchRequest = context.assertSentFetchRequest(); + assertTrue(voters.contains(fetchRequest.destinationId())); + context.assertFetchRequestData(fetchRequest, epoch, 3L, 3); + + List<String> records2 = Arrays.asList("d", "e", "f"); + MemoryRecords batch2 = context.buildBatch(3L, 4, records2); + context.deliverResponse(fetchRequest.correlationId, fetchRequest.destinationId(), + context.fetchResponse(epoch, otherNodeId, batch2, 4L, Errors.NONE)); + context.client.poll(); + assertEquals(4L, context.client.highWatermark().getAsLong()); Review comment: This is minor but so we don't confuse future readers of this code, I think the watermark is suppose to be `6L` instead of `4L`. The high watermark should always be at batch boundaries. -- 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