From af798af99d7f42510d4ba4268f97766d78165e80 Mon Sep 17 00:00:00 2001 From: Chirag Wadhwa Date: Fri, 24 Oct 2025 03:01:47 +0530 Subject: [PATCH 1/3] KAFKA-19797: Added deliveryCompleteCount to writeState, ShareUpdate and ShareSnapshot schemas --- .../message/WriteShareGroupStateRequest.json | 4 +- .../message/WriteShareGroupStateResponse.json | 2 +- .../common/requests/RequestResponseTest.java | 1 + .../kafka/server/share/SharePartition.java | 2 +- .../unit/kafka/server/KafkaApisTest.scala | 4 +- .../persister/DefaultStatePersister.java | 1 + .../server/share/persister/PartitionData.java | 20 +++++++-- .../share/persister/PartitionFactory.java | 17 +++---- .../persister/PartitionStateBatchData.java | 2 + .../persister/PersisterStateManager.java | 6 +++ .../WriteShareGroupStateParameters.java | 6 ++- .../persister/DefaultStatePersisterTest.java | 5 ++- .../persister/PersisterStateManagerTest.java | 7 +++ .../share/ShareCoordinatorRecordHelpers.java | 2 + .../share/ShareCoordinatorService.java | 1 + .../share/ShareCoordinatorShard.java | 4 ++ .../coordinator/share/ShareGroupOffset.java | 27 +++++++++++- .../common/message/ShareSnapshotValue.json | 2 + .../common/message/ShareUpdateValue.json | 2 + .../ShareCoordinatorRecordHelpersTest.java | 4 ++ .../share/ShareCoordinatorServiceTest.java | 13 ++++-- .../share/ShareCoordinatorShardTest.java | 44 ++++++++++++++++++- 22 files changed, 151 insertions(+), 25 deletions(-) diff --git a/clients/src/main/resources/common/message/WriteShareGroupStateRequest.json b/clients/src/main/resources/common/message/WriteShareGroupStateRequest.json index 9ebe169c8d6ac..af6aadcf6dbea 100644 --- a/clients/src/main/resources/common/message/WriteShareGroupStateRequest.json +++ b/clients/src/main/resources/common/message/WriteShareGroupStateRequest.json @@ -18,7 +18,7 @@ "type": "request", "listeners": ["broker"], "name": "WriteShareGroupStateRequest", - "validVersions": "0", + "validVersions": "0-1", "flexibleVersions": "0+", "fields": [ { "name": "GroupId", "type": "string", "versions": "0+", @@ -37,6 +37,8 @@ "about": "The leader epoch of the share-partition." }, { "name": "StartOffset", "type": "int64", "versions": "0+", "about": "The share-partition start offset, or -1 if the start offset is not being written." }, + { "name": "DeliveryCompleteCount", "type": "int32", "versions": "1+", "ignorable": "true", "default": "-1", + "about": "The number of offsets greater than or equal to share-partition start offset for which delivery has been completed."}, { "name": "StateBatches", "type": "[]StateBatch", "versions": "0+", "about": "The state batches for the share-partition.", "fields": [ { "name": "FirstOffset", "type": "int64", "versions": "0+", diff --git a/clients/src/main/resources/common/message/WriteShareGroupStateResponse.json b/clients/src/main/resources/common/message/WriteShareGroupStateResponse.json index 8d4050476519c..edf5504382eb3 100644 --- a/clients/src/main/resources/common/message/WriteShareGroupStateResponse.json +++ b/clients/src/main/resources/common/message/WriteShareGroupStateResponse.json @@ -17,7 +17,7 @@ "apiKey": 85, "type": "response", "name": "WriteShareGroupStateResponse", - "validVersions": "0", + "validVersions": "0-1", "flexibleVersions": "0+", // - NOT_COORDINATOR (version 0+) // - COORDINATOR_NOT_AVAILABLE (version 0+) diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index d683d287419b3..596828a08391b 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -3737,6 +3737,7 @@ private WriteShareGroupStateRequest createWriteShareGroupStateRequest(short vers .setPartition(0) .setStateEpoch(0) .setStartOffset(0) + .setDeliveryCompleteCount(0) .setStateBatches(singletonList(new WriteShareGroupStateRequestData.StateBatch() .setFirstOffset(0) .setLastOffset(0) diff --git a/core/src/main/java/kafka/server/share/SharePartition.java b/core/src/main/java/kafka/server/share/SharePartition.java index 0bf3eb10b2726..826a7c862db30 100644 --- a/core/src/main/java/kafka/server/share/SharePartition.java +++ b/core/src/main/java/kafka/server/share/SharePartition.java @@ -2504,7 +2504,7 @@ CompletableFuture writeShareGroupState(List stateBatc .setGroupId(this.groupId) .setTopicsData(List.of(new TopicData<>(topicIdPartition.topicId(), List.of(PartitionFactory.newPartitionStateBatchData( - topicIdPartition.partition(), stateEpoch, startOffset(), leaderEpoch, stateBatches)))) + topicIdPartition.partition(), stateEpoch, startOffset(), inFlightTerminalRecords(), leaderEpoch, stateBatches)))) ).build()).build()) .whenComplete((result, exception) -> { if (exception != null) { diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 3d801536072cc..472054a5e5175 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -13299,6 +13299,7 @@ class KafkaApisTest extends Logging { .setLeaderEpoch(1) .setStateEpoch(2) .setStartOffset(10) + .setDeliveryCompleteCount(5) .setStateBatches(util.List.of( new WriteShareGroupStateRequestData.StateBatch() .setFirstOffset(11) @@ -13346,6 +13347,7 @@ class KafkaApisTest extends Logging { .setLeaderEpoch(1) .setStateEpoch(2) .setStartOffset(10) + .setDeliveryCompleteCount(5) .setStateBatches(util.List.of( new WriteShareGroupStateRequestData.StateBatch() .setFirstOffset(11) @@ -13851,7 +13853,7 @@ class KafkaApisTest extends Logging { def getWriteShareGroupStateResponse(requestData: WriteShareGroupStateRequestData, configOverrides: Map[String, String] = Map.empty, verifyNoErr: Boolean = true, authorizer: Authorizer = null, writeStateResult: util.List[WriteShareGroupStateResponseData.WriteStateResult]): WriteShareGroupStateResponse = { - val requestChannelRequest = buildRequest(new WriteShareGroupStateRequest.Builder(requestData).build()) + val requestChannelRequest = buildRequest(new WriteShareGroupStateRequest.Builder(requestData).build(0)) val future = new CompletableFuture[WriteShareGroupStateResponseData]() when(shareCoordinator.writeState( diff --git a/server-common/src/main/java/org/apache/kafka/server/share/persister/DefaultStatePersister.java b/server-common/src/main/java/org/apache/kafka/server/share/persister/DefaultStatePersister.java index fe66efa7f7f56..776c4f73bdf71 100644 --- a/server-common/src/main/java/org/apache/kafka/server/share/persister/DefaultStatePersister.java +++ b/server-common/src/main/java/org/apache/kafka/server/share/persister/DefaultStatePersister.java @@ -150,6 +150,7 @@ stateManager.new WriteStateHandler( partitionData.stateEpoch(), partitionData.leaderEpoch(), partitionData.startOffset(), + partitionData.deliveryCompleteCount(), partitionData.stateBatches(), future, null) ); diff --git a/server-common/src/main/java/org/apache/kafka/server/share/persister/PartitionData.java b/server-common/src/main/java/org/apache/kafka/server/share/persister/PartitionData.java index 25db77380da7e..4d38e67916764 100644 --- a/server-common/src/main/java/org/apache/kafka/server/share/persister/PartitionData.java +++ b/server-common/src/main/java/org/apache/kafka/server/share/persister/PartitionData.java @@ -30,16 +30,18 @@ public class PartitionData implements private final int partition; private final int stateEpoch; private final long startOffset; + private final int deliveryCompleteCount; private final short errorCode; private final String errorMessage; private final int leaderEpoch; private final List stateBatches; - public PartitionData(int partition, int stateEpoch, long startOffset, short errorCode, + public PartitionData(int partition, int stateEpoch, long startOffset, int deliveryCompleteCount, short errorCode, String errorMessage, int leaderEpoch, List stateBatches) { this.partition = partition; this.stateEpoch = stateEpoch; this.startOffset = startOffset; + this.deliveryCompleteCount = deliveryCompleteCount; this.errorCode = errorCode; this.leaderEpoch = leaderEpoch; this.errorMessage = errorMessage; @@ -58,6 +60,10 @@ public long startOffset() { return startOffset; } + public int deliveryCompleteCount() { + return deliveryCompleteCount; + } + public short errorCode() { return errorCode; } @@ -82,6 +88,7 @@ public boolean equals(Object o) { return Objects.equals(partition, that.partition) && Objects.equals(stateEpoch, that.stateEpoch) && Objects.equals(startOffset, that.startOffset) && + Objects.equals(deliveryCompleteCount, that.deliveryCompleteCount) && Objects.equals(errorCode, that.errorCode) && Objects.equals(errorMessage, that.errorMessage) && Objects.equals(leaderEpoch, that.leaderEpoch) && @@ -90,13 +97,14 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(partition, stateEpoch, startOffset, errorCode, leaderEpoch, errorMessage, stateBatches); + return Objects.hash(partition, stateEpoch, startOffset, deliveryCompleteCount, errorCode, leaderEpoch, errorMessage, stateBatches); } public static class Builder { private int partition; private int stateEpoch; private long startOffset; + private int deliveryCompleteCount; private short errorCode; private String errorMessage; private int leaderEpoch; @@ -117,6 +125,11 @@ public Builder setStartOffset(long startOffset) { return this; } + public Builder setDeliveryCompleteCount(int deliveryCompleteCount) { + this.deliveryCompleteCount = deliveryCompleteCount; + return this; + } + public Builder setErrorCode(short errorCode) { this.errorCode = errorCode; return this; @@ -138,7 +151,7 @@ public Builder setStateBatches(List stateBatches) { } public PartitionData build() { - return new PartitionData(partition, stateEpoch, startOffset, errorCode, errorMessage, leaderEpoch, stateBatches); + return new PartitionData(partition, stateEpoch, startOffset, deliveryCompleteCount, errorCode, errorMessage, leaderEpoch, stateBatches); } } @@ -148,6 +161,7 @@ public String toString() { "partition=" + partition + "," + "stateEpoch=" + stateEpoch + "," + "startOffset=" + startOffset + "," + + "deliveryCompleteCount=" + deliveryCompleteCount + "," + "errorCode=" + errorCode + "," + "errorMessage=" + errorMessage + "," + "leaderEpoch=" + leaderEpoch + "," + diff --git a/server-common/src/main/java/org/apache/kafka/server/share/persister/PartitionFactory.java b/server-common/src/main/java/org/apache/kafka/server/share/persister/PartitionFactory.java index 78a6902a170a9..f0612677fc904 100644 --- a/server-common/src/main/java/org/apache/kafka/server/share/persister/PartitionFactory.java +++ b/server-common/src/main/java/org/apache/kafka/server/share/persister/PartitionFactory.java @@ -27,35 +27,36 @@ public class PartitionFactory { public static final int DEFAULT_STATE_EPOCH = 0; public static final int UNINITIALIZED_START_OFFSET = -1; + public static final int UNINITIALIZED_DELIVERY_COMPLETE_COUNT = -1; public static final short DEFAULT_ERROR_CODE = Errors.NONE.code(); public static final int DEFAULT_LEADER_EPOCH = 0; public static final String DEFAULT_ERR_MESSAGE = Errors.NONE.message(); public static PartitionIdData newPartitionIdData(int partition) { - return new PartitionData(partition, DEFAULT_STATE_EPOCH, UNINITIALIZED_START_OFFSET, DEFAULT_ERROR_CODE, DEFAULT_ERR_MESSAGE, DEFAULT_LEADER_EPOCH, null); + return new PartitionData(partition, DEFAULT_STATE_EPOCH, UNINITIALIZED_START_OFFSET, UNINITIALIZED_DELIVERY_COMPLETE_COUNT, DEFAULT_ERROR_CODE, DEFAULT_ERR_MESSAGE, DEFAULT_LEADER_EPOCH, null); } public static PartitionIdLeaderEpochData newPartitionIdLeaderEpochData(int partition, int leaderEpoch) { - return new PartitionData(partition, DEFAULT_STATE_EPOCH, UNINITIALIZED_START_OFFSET, DEFAULT_ERROR_CODE, DEFAULT_ERR_MESSAGE, leaderEpoch, null); + return new PartitionData(partition, DEFAULT_STATE_EPOCH, UNINITIALIZED_START_OFFSET, UNINITIALIZED_DELIVERY_COMPLETE_COUNT, DEFAULT_ERROR_CODE, DEFAULT_ERR_MESSAGE, leaderEpoch, null); } public static PartitionStateData newPartitionStateData(int partition, int stateEpoch, long startOffset) { - return new PartitionData(partition, stateEpoch, startOffset, DEFAULT_ERROR_CODE, DEFAULT_ERR_MESSAGE, DEFAULT_LEADER_EPOCH, null); + return new PartitionData(partition, stateEpoch, startOffset, UNINITIALIZED_DELIVERY_COMPLETE_COUNT, DEFAULT_ERROR_CODE, DEFAULT_ERR_MESSAGE, DEFAULT_LEADER_EPOCH, null); } public static PartitionErrorData newPartitionErrorData(int partition, short errorCode, String errorMessage) { - return new PartitionData(partition, DEFAULT_STATE_EPOCH, UNINITIALIZED_START_OFFSET, errorCode, errorMessage, DEFAULT_LEADER_EPOCH, null); + return new PartitionData(partition, DEFAULT_STATE_EPOCH, UNINITIALIZED_START_OFFSET, UNINITIALIZED_DELIVERY_COMPLETE_COUNT, errorCode, errorMessage, DEFAULT_LEADER_EPOCH, null); } public static PartitionStateSummaryData newPartitionStateSummaryData(int partition, int stateEpoch, long startOffset, int leaderEpoch, short errorCode, String errorMessage) { - return new PartitionData(partition, stateEpoch, startOffset, errorCode, errorMessage, leaderEpoch, null); + return new PartitionData(partition, stateEpoch, startOffset, UNINITIALIZED_DELIVERY_COMPLETE_COUNT, errorCode, errorMessage, leaderEpoch, null); } - public static PartitionStateBatchData newPartitionStateBatchData(int partition, int stateEpoch, long startOffset, int leaderEpoch, List stateBatches) { - return new PartitionData(partition, stateEpoch, startOffset, DEFAULT_ERROR_CODE, DEFAULT_ERR_MESSAGE, leaderEpoch, stateBatches); + public static PartitionStateBatchData newPartitionStateBatchData(int partition, int stateEpoch, long startOffset, int deliveryCompleteCount, int leaderEpoch, List stateBatches) { + return new PartitionData(partition, stateEpoch, startOffset, deliveryCompleteCount, DEFAULT_ERROR_CODE, DEFAULT_ERR_MESSAGE, leaderEpoch, stateBatches); } public static PartitionAllData newPartitionAllData(int partition, int stateEpoch, long startOffset, short errorCode, String errorMessage, List stateBatches) { - return new PartitionData(partition, stateEpoch, startOffset, errorCode, errorMessage, DEFAULT_LEADER_EPOCH, stateBatches); + return new PartitionData(partition, stateEpoch, startOffset, UNINITIALIZED_DELIVERY_COMPLETE_COUNT, errorCode, errorMessage, DEFAULT_LEADER_EPOCH, stateBatches); } } diff --git a/server-common/src/main/java/org/apache/kafka/server/share/persister/PartitionStateBatchData.java b/server-common/src/main/java/org/apache/kafka/server/share/persister/PartitionStateBatchData.java index 326b6e2ca56bc..e4b00aa5e100b 100644 --- a/server-common/src/main/java/org/apache/kafka/server/share/persister/PartitionStateBatchData.java +++ b/server-common/src/main/java/org/apache/kafka/server/share/persister/PartitionStateBatchData.java @@ -28,6 +28,8 @@ public interface PartitionStateBatchData extends PartitionInfoData, PartitionIdD long startOffset(); + int deliveryCompleteCount(); + int leaderEpoch(); List stateBatches(); diff --git a/server-common/src/main/java/org/apache/kafka/server/share/persister/PersisterStateManager.java b/server-common/src/main/java/org/apache/kafka/server/share/persister/PersisterStateManager.java index 6189e8b9b1329..3935ef0dd902f 100644 --- a/server-common/src/main/java/org/apache/kafka/server/share/persister/PersisterStateManager.java +++ b/server-common/src/main/java/org/apache/kafka/server/share/persister/PersisterStateManager.java @@ -656,6 +656,7 @@ public class WriteStateHandler extends PersisterStateManagerHandler { private final int stateEpoch; private final int leaderEpoch; private final long startOffset; + private final int deliveryCompleteCount; private final List batches; private final CompletableFuture result; private final BackoffManager writeStateBackoff; @@ -667,6 +668,7 @@ public WriteStateHandler( int stateEpoch, int leaderEpoch, long startOffset, + int deliveryCompleteCount, List batches, CompletableFuture result, long backoffMs, @@ -677,6 +679,7 @@ public WriteStateHandler( this.stateEpoch = stateEpoch; this.leaderEpoch = leaderEpoch; this.startOffset = startOffset; + this.deliveryCompleteCount = deliveryCompleteCount; this.batches = batches; this.result = result; this.writeStateBackoff = new BackoffManager(maxRPCRetryAttempts, backoffMs, backoffMaxMs); @@ -689,6 +692,7 @@ public WriteStateHandler( int stateEpoch, int leaderEpoch, long startOffset, + int deliveryCompleteCount, List batches, CompletableFuture result, Consumer onCompleteCallback @@ -700,6 +704,7 @@ public WriteStateHandler( stateEpoch, leaderEpoch, startOffset, + deliveryCompleteCount, batches, result, REQUEST_BACKOFF_MS, @@ -1456,6 +1461,7 @@ private static AbstractRequest.Builder coalesceWrites .setStateEpoch(handler.stateEpoch) .setLeaderEpoch(handler.leaderEpoch) .setStartOffset(handler.startOffset) + .setDeliveryCompleteCount(handler.deliveryCompleteCount) .setStateBatches(handler.batches.stream() .map(batch -> new WriteShareGroupStateRequestData.StateBatch() .setFirstOffset(batch.firstOffset()) diff --git a/server-common/src/main/java/org/apache/kafka/server/share/persister/WriteShareGroupStateParameters.java b/server-common/src/main/java/org/apache/kafka/server/share/persister/WriteShareGroupStateParameters.java index 7d4f806542348..c4df621300f7d 100644 --- a/server-common/src/main/java/org/apache/kafka/server/share/persister/WriteShareGroupStateParameters.java +++ b/server-common/src/main/java/org/apache/kafka/server/share/persister/WriteShareGroupStateParameters.java @@ -40,7 +40,11 @@ public static WriteShareGroupStateParameters from(WriteShareGroupStateRequestDat .setGroupTopicPartitionData(new GroupTopicPartitionData<>(data.groupId(), data.topics().stream() .map(writeStateData -> new TopicData<>(writeStateData.topicId(), writeStateData.partitions().stream() - .map(partitionData -> PartitionFactory.newPartitionStateBatchData(partitionData.partition(), partitionData.stateEpoch(), partitionData.startOffset(), + .map(partitionData -> PartitionFactory.newPartitionStateBatchData( + partitionData.partition(), + partitionData.stateEpoch(), + partitionData.startOffset(), + partitionData.deliveryCompleteCount(), partitionData.leaderEpoch(), partitionData.stateBatches().stream() .map(PersisterStateBatch::from) diff --git a/server-common/src/test/java/org/apache/kafka/server/share/persister/DefaultStatePersisterTest.java b/server-common/src/test/java/org/apache/kafka/server/share/persister/DefaultStatePersisterTest.java index 697d958723a1a..02afb10798f0f 100644 --- a/server-common/src/test/java/org/apache/kafka/server/share/persister/DefaultStatePersisterTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/share/persister/DefaultStatePersisterTest.java @@ -178,7 +178,7 @@ public void testWriteStateValidate() { .setGroupId(groupId) .setTopicsData(List.of(new TopicData<>(null, List.of(PartitionFactory.newPartitionStateBatchData( - partition, 1, 0, 0, null))))).build()).build()); + partition, 1, 0, 0, 0, null))))).build()).build()); assertTrue(result.isDone()); assertTrue(result.isCompletedExceptionally()); assertFutureThrows(IllegalArgumentException.class, result); @@ -200,7 +200,7 @@ public void testWriteStateValidate() { .setGroupId(groupId) .setTopicsData(List.of(new TopicData<>(topicId, List.of(PartitionFactory.newPartitionStateBatchData( - incorrectPartition, 1, 0, 0, null))))).build()).build()); + incorrectPartition, 1, 0, 0, 0, null))))).build()).build()); assertTrue(result.isDone()); assertTrue(result.isCompletedExceptionally()); assertFutureThrows(IllegalArgumentException.class, result); @@ -600,6 +600,7 @@ public void testWriteStateSuccess() { .setStateEpoch(0) .setLeaderEpoch(1) .setStartOffset(0) + .setDeliveryCompleteCount(10) .setStateBatches(List.of(new WriteShareGroupStateRequestData.StateBatch() .setFirstOffset(0) .setLastOffset(10) diff --git a/server-common/src/test/java/org/apache/kafka/server/share/persister/PersisterStateManagerTest.java b/server-common/src/test/java/org/apache/kafka/server/share/persister/PersisterStateManagerTest.java index 6de002e518a13..cbbfddac5428d 100644 --- a/server-common/src/test/java/org/apache/kafka/server/share/persister/PersisterStateManagerTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/share/persister/PersisterStateManagerTest.java @@ -741,6 +741,7 @@ public void testWriteStateRequestCoordinatorFoundSuccessfully() { 0, 0, 0, + 0, stateBatches, future, REQUEST_BACKOFF_MS, @@ -865,6 +866,7 @@ public void testWriteStateRequestRetryWithNotCoordinatorSuccessfulOnRetry() thro 0, 0, 0, + 0, stateBatches, future, REQUEST_BACKOFF_MS, @@ -985,6 +987,7 @@ public void testWriteStateRequestCoordinatorFoundOnRetry() { 0, 0, 0, + 0, stateBatches, future, REQUEST_BACKOFF_MS, @@ -1077,6 +1080,7 @@ public void testWriteStateRequestWithCoordinatorNodeLookup() { 0, 0, 0, + 0, stateBatches, future, REQUEST_BACKOFF_MS, @@ -1191,6 +1195,7 @@ public void testWriteStateRequestWithRetryAndCoordinatorNodeLookup() { 0, 0, 0, + 0, stateBatches, future, REQUEST_BACKOFF_MS, @@ -1326,6 +1331,7 @@ public void testWriteStateRequestFailedMaxRetriesExhausted() { 0, 0, 0, + 0, stateBatches, future, REQUEST_BACKOFF_MS, @@ -1430,6 +1436,7 @@ public void testWriteStateRequestBatchingWithCoordinatorNodeLookup() throws Exec 0, 0, 0, + 0, stateBatches, future, REQUEST_BACKOFF_MS, diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorRecordHelpers.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorRecordHelpers.java index 49daa9a43bc8e..e46d4681280e4 100644 --- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorRecordHelpers.java +++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorRecordHelpers.java @@ -37,6 +37,7 @@ public static CoordinatorRecord newShareSnapshotRecord(String groupId, Uuid topi .setStateEpoch(offsetData.stateEpoch()) .setLeaderEpoch(offsetData.leaderEpoch()) .setStartOffset(offsetData.startOffset()) + .setDeliveryCompleteCount(offsetData.deliveryCompleteCount()) .setStateBatches(offsetData.stateBatches().stream() .map(batch -> new ShareSnapshotValue.StateBatch() .setFirstOffset(batch.firstOffset()) @@ -61,6 +62,7 @@ public static CoordinatorRecord newShareUpdateRecord(String groupId, Uuid topicI .setSnapshotEpoch(offsetData.snapshotEpoch()) .setLeaderEpoch(offsetData.leaderEpoch()) .setStartOffset(offsetData.startOffset()) + .setDeliveryCompleteCount(offsetData.deliveryCompleteCount()) .setStateBatches(offsetData.stateBatches().stream() .map(batch -> new ShareUpdateValue.StateBatch() .setFirstOffset(batch.firstOffset()) diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java index 3a85cc3f1feda..d0386e32b9f40 100644 --- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java +++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java @@ -472,6 +472,7 @@ public CompletableFuture writeState(RequestCon .setPartitions(List.of(new WriteShareGroupStateRequestData.PartitionData() .setPartition(partitionData.partition()) .setStartOffset(partitionData.startOffset()) + .setDeliveryCompleteCount(partitionData.deliveryCompleteCount()) .setLeaderEpoch(partitionData.leaderEpoch()) .setStateEpoch(partitionData.stateEpoch()) .setStateBatches(partitionData.stateBatches()))))))) diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java index 9d52780faa5a9..db8819425d645 100644 --- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java +++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java @@ -403,6 +403,7 @@ public CoordinatorResult rea .setLeaderEpoch(leaderEpoch) .setStateBatches(List.of()) .setStartOffset(responseData.results().get(0).partitions().get(0).startOffset()) + .setDeliveryCompleteCount(offsetValue.deliveryCompleteCount()) .setStateEpoch(responseData.results().get(0).partitions().get(0).stateEpoch()); CoordinatorRecord record = generateShareStateRecord(writePartitionData, key, true); @@ -668,6 +669,7 @@ private CoordinatorRecord generateShareStateRecord( new ShareGroupOffset.Builder() .setSnapshotEpoch(currentState.snapshotEpoch() + 1) // We must increment snapshot epoch as this is new snapshot. .setStartOffset(newStartOffset) + .setDeliveryCompleteCount(partitionData.deliveryCompleteCount()) .setLeaderEpoch(newLeaderEpoch) .setStateEpoch(currentState.stateEpoch()) .setStateBatches(mergeBatches(currentState.stateBatches(), partitionData, newStartOffset)) @@ -683,6 +685,7 @@ private CoordinatorRecord generateShareStateRecord( new ShareGroupOffset.Builder() .setSnapshotEpoch(currentState.snapshotEpoch()) // Use same snapshotEpoch as last share snapshot. .setStartOffset(partitionData.startOffset()) + .setDeliveryCompleteCount(partitionData.deliveryCompleteCount()) .setLeaderEpoch(newLeaderEpoch) .setStateBatches(mergeBatches(List.of(), partitionData)) .build()); @@ -995,6 +998,7 @@ private static ShareGroupOffset merge(ShareGroupOffset soFar, ShareUpdateValue n .setSnapshotEpoch(soFar.snapshotEpoch()) .setStateEpoch(soFar.stateEpoch()) .setStartOffset(newStartOffset) + .setDeliveryCompleteCount(newData.deliveryCompleteCount()) .setLeaderEpoch(newLeaderEpoch) .setStateBatches(new PersisterStateBatchCombiner(currentBatches, newData.stateBatches().stream() .map(ShareCoordinatorShard::toPersisterStateBatch) diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareGroupOffset.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareGroupOffset.java index 2ca1a646885c6..ddf286b1888c6 100644 --- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareGroupOffset.java +++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareGroupOffset.java @@ -34,12 +34,14 @@ public class ShareGroupOffset { public static final int NO_TIMESTAMP = 0; public static final int UNINITIALIZED_EPOCH = 0; + public static final int UNINITIALIZED_DELIVERY_COMPLETE_COUNT = -1; public static final int DEFAULT_EPOCH = 0; private final int snapshotEpoch; private final int stateEpoch; private final int leaderEpoch; private final long startOffset; + private final int deliveryCompleteCount; private final List stateBatches; private final long createTimestamp; private final long writeTimestamp; @@ -49,6 +51,7 @@ private ShareGroupOffset( int stateEpoch, int leaderEpoch, long startOffset, + int deliveryCompleteCount, List stateBatches, long createTimestamp, long writeTimestamp @@ -57,6 +60,7 @@ private ShareGroupOffset( this.stateEpoch = stateEpoch; this.leaderEpoch = leaderEpoch; this.startOffset = startOffset; + this.deliveryCompleteCount = deliveryCompleteCount; this.stateBatches = stateBatches; this.createTimestamp = createTimestamp; this.writeTimestamp = writeTimestamp; @@ -78,6 +82,10 @@ public long startOffset() { return startOffset; } + public int deliveryCompleteCount() { + return deliveryCompleteCount; + } + public long createTimestamp() { return createTimestamp; } @@ -104,6 +112,7 @@ public static ShareGroupOffset fromRecord(ShareSnapshotValue record) { record.stateEpoch(), record.leaderEpoch(), record.startOffset(), + record.deliveryCompleteCount(), record.stateBatches().stream() .map(ShareGroupOffset::toPersisterOffsetsStateBatch) .toList(), @@ -118,6 +127,7 @@ public static ShareGroupOffset fromRecord(ShareUpdateValue record) { UNINITIALIZED_EPOCH, record.leaderEpoch(), record.startOffset(), + record.deliveryCompleteCount(), record.stateBatches().stream() .map(ShareGroupOffset::toPersisterOffsetsStateBatch) .toList(), @@ -136,6 +146,7 @@ public static ShareGroupOffset fromRequest(WriteShareGroupStateRequestData.Parti data.stateEpoch(), data.leaderEpoch(), data.startOffset(), + data.deliveryCompleteCount(), data.stateBatches().stream() .map(PersisterStateBatch::from) .toList(), @@ -149,11 +160,14 @@ public static ShareGroupOffset fromRequest(InitializeShareGroupStateRequestData. } public static ShareGroupOffset fromRequest(InitializeShareGroupStateRequestData.PartitionData data, int snapshotEpoch, long timestamp) { + // Since initialization changes the start offset, and hence the in flight state is forgotten, the end offset is set + // to be the same as the start offset, and the in flight record count is set to 0. return new ShareGroupOffset( snapshotEpoch, data.stateEpoch(), UNINITIALIZED_EPOCH, data.startOffset(), + UNINITIALIZED_DELIVERY_COMPLETE_COUNT, List.of(), timestamp, timestamp @@ -165,6 +179,7 @@ public static class Builder { private int stateEpoch; private int leaderEpoch; private long startOffset; + private int deliveryCompleteCount; private List stateBatches; private long createTimestamp = NO_TIMESTAMP; private long writeTimestamp = NO_TIMESTAMP; @@ -189,6 +204,11 @@ public Builder setStartOffset(long startOffset) { return this; } + public Builder setDeliveryCompleteCount(int deliveryCompleteCount) { + this.deliveryCompleteCount = deliveryCompleteCount; + return this; + } + public Builder setStateBatches(List stateBatches) { this.stateBatches = stateBatches == null ? List.of() : stateBatches.stream().toList(); return this; @@ -205,7 +225,7 @@ public Builder setWriteTimestamp(long writeTimestamp) { } public ShareGroupOffset build() { - return new ShareGroupOffset(snapshotEpoch, stateEpoch, leaderEpoch, startOffset, stateBatches, createTimestamp, writeTimestamp); + return new ShareGroupOffset(snapshotEpoch, stateEpoch, leaderEpoch, startOffset, deliveryCompleteCount, stateBatches, createTimestamp, writeTimestamp); } } @@ -218,6 +238,7 @@ public boolean equals(Object o) { stateEpoch == that.stateEpoch && leaderEpoch == that.leaderEpoch && startOffset == that.startOffset && + deliveryCompleteCount == that.deliveryCompleteCount && Objects.equals(stateBatches, that.stateBatches) && createTimestamp == that.createTimestamp && writeTimestamp == that.writeTimestamp; @@ -225,7 +246,7 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(snapshotEpoch, stateEpoch, leaderEpoch, startOffset, stateBatches, createTimestamp, writeTimestamp); + return Objects.hash(snapshotEpoch, stateEpoch, leaderEpoch, startOffset, deliveryCompleteCount, stateBatches, createTimestamp, writeTimestamp); } @Override @@ -235,6 +256,7 @@ public String toString() { ", stateEpoch=" + stateEpoch + ", leaderEpoch=" + leaderEpoch + ", startOffset=" + startOffset + + ", inFlightRecordCount=" + deliveryCompleteCount + ", createTimestamp=" + createTimestamp + ", writeTimestamp=" + writeTimestamp + ", stateBatches=" + stateBatches + @@ -247,6 +269,7 @@ public Builder builderSupplier() { .setStateEpoch(stateEpoch()) .setLeaderEpoch(leaderEpoch()) .setStartOffset(startOffset()) + .setDeliveryCompleteCount(deliveryCompleteCount()) .setStateBatches(stateBatches()) .setCreateTimestamp(createTimestamp()) .setWriteTimestamp(writeTimestamp()); diff --git a/share-coordinator/src/main/resources/common/message/ShareSnapshotValue.json b/share-coordinator/src/main/resources/common/message/ShareSnapshotValue.json index 46cd4f6c489da..6eff1564b7b08 100644 --- a/share-coordinator/src/main/resources/common/message/ShareSnapshotValue.json +++ b/share-coordinator/src/main/resources/common/message/ShareSnapshotValue.json @@ -28,6 +28,8 @@ "about": "The leader epoch of the share-partition." }, { "name": "StartOffset", "type": "int64", "versions": "0+", "about": "The share-partition start offset." }, + { "name": "DeliveryCompleteCount", "type": "int32", "versions": "0+", "taggedVersions": "0+", "tag": 0, "default": "-1", + "about": "The number of offsets greater than or equal to share-partition start offset for which delivery has been completed."}, { "name": "CreateTimestamp", "type": "int64", "versions": "0+", "about": "The time at which the state was created." }, { "name": "WriteTimestamp", "type": "int64", "versions": "0+", diff --git a/share-coordinator/src/main/resources/common/message/ShareUpdateValue.json b/share-coordinator/src/main/resources/common/message/ShareUpdateValue.json index 5cd20c0d15923..3955520022c1a 100644 --- a/share-coordinator/src/main/resources/common/message/ShareUpdateValue.json +++ b/share-coordinator/src/main/resources/common/message/ShareUpdateValue.json @@ -26,6 +26,8 @@ "about": "The leader epoch of the share-partition." }, { "name": "StartOffset", "type": "int64", "versions": "0+", "about": "The share-partition start offset, or -1 if the start offset is not being updated." }, + { "name": "DeliveryCompleteCount", "type": "int32", "versions": "0+", "taggedVersions": "0+", "tag": 0, "default": "-1", + "about": "The number of offsets greater than or equal to share-partition start offset for which delivery has been completed."}, { "name": "StateBatches", "type": "[]StateBatch", "versions": "0+", "about": "The state batches that have been updated.", "fields": [ { "name": "FirstOffset", "type": "int64", "versions": "0+", diff --git a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorRecordHelpersTest.java b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorRecordHelpersTest.java index 630318399d8af..924a83d23314d 100644 --- a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorRecordHelpersTest.java +++ b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorRecordHelpersTest.java @@ -48,6 +48,7 @@ public void testNewShareSnapshotRecord() { .setStateEpoch(1) .setLeaderEpoch(5) .setStartOffset(0) + .setDeliveryCompleteCount(10) .setCreateTimestamp(timestamp) .setWriteTimestamp(timestamp) .setStateBatches(List.of(batch)) @@ -65,6 +66,7 @@ public void testNewShareSnapshotRecord() { .setStateEpoch(1) .setLeaderEpoch(5) .setStartOffset(0) + .setDeliveryCompleteCount(10) .setCreateTimestamp(timestamp) .setWriteTimestamp(timestamp) .setStateBatches(List.of( @@ -93,6 +95,7 @@ public void testNewShareUpdateRecord() { .setStateEpoch(-1) // ignored for share update .setLeaderEpoch(5) .setStartOffset(0) + .setDeliveryCompleteCount(10) .setStateBatches(List.of(batch)) .build() ); @@ -107,6 +110,7 @@ public void testNewShareUpdateRecord() { .setSnapshotEpoch(0) .setLeaderEpoch(5) .setStartOffset(0) + .setDeliveryCompleteCount(10) .setStateBatches(List.of( new ShareUpdateValue.StateBatch() .setFirstOffset(1L) diff --git a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java index d44735f048fa9..0546dcce9c66b 100644 --- a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java +++ b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java @@ -154,11 +154,12 @@ public void testWriteStateSuccess() throws ExecutionException, InterruptedExcept new WriteShareGroupStateRequestData.PartitionData() .setPartition(partition1) .setStartOffset(0) + .setDeliveryCompleteCount(10) .setStateEpoch(1) .setLeaderEpoch(1) .setStateBatches(List.of(new WriteShareGroupStateRequestData.StateBatch() .setFirstOffset(0) - .setLastOffset(10) + .setLastOffset(9) .setDeliveryCount((short) 1) .setDeliveryState((byte) 0)) ) @@ -169,11 +170,12 @@ public void testWriteStateSuccess() throws ExecutionException, InterruptedExcept new WriteShareGroupStateRequestData.PartitionData() .setPartition(partition2) .setStartOffset(0) + .setDeliveryCompleteCount(10) .setStateEpoch(1) .setLeaderEpoch(1) .setStateBatches(List.of(new WriteShareGroupStateRequestData.StateBatch() .setFirstOffset(0) - .setLastOffset(10) + .setLastOffset(9) .setDeliveryCount((short) 1) .setDeliveryState((byte) 0)) ) @@ -857,11 +859,12 @@ public void testWriteStateWhenNotStarted() throws ExecutionException, Interrupte new WriteShareGroupStateRequestData.PartitionData() .setPartition(partition1) .setStartOffset(0) + .setDeliveryCompleteCount(10) .setStateEpoch(1) .setLeaderEpoch(1) .setStateBatches(List.of(new WriteShareGroupStateRequestData.StateBatch() .setFirstOffset(0) - .setLastOffset(10) + .setLastOffset(9) .setDeliveryCount((short) 1) .setDeliveryState((byte) 0)) ) @@ -872,11 +875,12 @@ public void testWriteStateWhenNotStarted() throws ExecutionException, Interrupte new WriteShareGroupStateRequestData.PartitionData() .setPartition(partition2) .setStartOffset(0) + .setDeliveryCompleteCount(10) .setStateEpoch(1) .setLeaderEpoch(1) .setStateBatches(List.of(new WriteShareGroupStateRequestData.StateBatch() .setFirstOffset(0) - .setLastOffset(10) + .setLastOffset(9) .setDeliveryCount((short) 1) .setDeliveryState((byte) 0)) ) @@ -1197,6 +1201,7 @@ public void testWriteFutureReturnsError() throws ExecutionException, Interrupted .setPartition(partition) .setLeaderEpoch(1) .setStartOffset(1) + .setDeliveryCompleteCount(9) .setStateEpoch(1) .setStateBatches(List.of(new WriteShareGroupStateRequestData.StateBatch() .setFirstOffset(2) diff --git a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorShardTest.java b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorShardTest.java index 08990cd8510b9..f82843f2de55c 100644 --- a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorShardTest.java +++ b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorShardTest.java @@ -208,6 +208,8 @@ public void testReplayWithShareSnapshot() { new ShareSnapshotValue() .setSnapshotEpoch(0) .setStateEpoch(0) + .setStartOffset(0) + .setDeliveryCompleteCount(11) .setLeaderEpoch(leaderEpoch) .setStateBatches(List.of( new ShareSnapshotValue.StateBatch() @@ -228,6 +230,8 @@ public void testReplayWithShareSnapshot() { new ShareSnapshotValue() .setSnapshotEpoch(1) .setStateEpoch(1) + .setStartOffset(0) + .setDeliveryCompleteCount(13) .setLeaderEpoch(leaderEpoch + 1) .setStateBatches(List.of( new ShareSnapshotValue.StateBatch() @@ -263,6 +267,7 @@ public void testWriteFailsOnUninitializedPartition() { .setPartitions(List.of(new WriteShareGroupStateRequestData.PartitionData() .setPartition(PARTITION) .setStartOffset(0) + .setDeliveryCompleteCount(11) .setStateEpoch(0) .setLeaderEpoch(0) .setStateBatches(List.of(new WriteShareGroupStateRequestData.StateBatch() @@ -297,6 +302,7 @@ public void testWriteStateSuccess() { .setPartitions(List.of(new WriteShareGroupStateRequestData.PartitionData() .setPartition(PARTITION) .setStartOffset(0) + .setDeliveryCompleteCount(11) .setStateEpoch(0) .setLeaderEpoch(0) .setStateBatches(List.of(new WriteShareGroupStateRequestData.StateBatch() @@ -336,6 +342,7 @@ public void testSubsequentWriteStateSnapshotEpochUpdatesSuccessfully() { .setPartitions(List.of(new WriteShareGroupStateRequestData.PartitionData() .setPartition(PARTITION) .setStartOffset(0) + .setDeliveryCompleteCount(11) .setStateEpoch(0) .setLeaderEpoch(0) .setStateBatches(List.of(new WriteShareGroupStateRequestData.StateBatch() @@ -351,6 +358,7 @@ public void testSubsequentWriteStateSnapshotEpochUpdatesSuccessfully() { .setPartitions(List.of(new WriteShareGroupStateRequestData.PartitionData() .setPartition(PARTITION) .setStartOffset(0) + .setDeliveryCompleteCount(21) .setStateEpoch(0) .setLeaderEpoch(0) .setStateBatches(List.of(new WriteShareGroupStateRequestData.StateBatch() @@ -411,6 +419,7 @@ public void testWriteStateInvalidRequestData() { .setPartitions(List.of(new WriteShareGroupStateRequestData.PartitionData() .setPartition(partition) .setStartOffset(0) + .setDeliveryCompleteCount(11) .setStateEpoch(0) .setLeaderEpoch(0) .setStateBatches(List.of(new WriteShareGroupStateRequestData.StateBatch() @@ -445,6 +454,7 @@ public void testWriteNullMetadataImage() { .setPartitions(List.of(new WriteShareGroupStateRequestData.PartitionData() .setPartition(0) .setStartOffset(0) + .setDeliveryCompleteCount(11) .setStateEpoch(0) .setLeaderEpoch(0) .setStateBatches(List.of(new WriteShareGroupStateRequestData.StateBatch() @@ -476,6 +486,7 @@ public void testWriteStateFencedLeaderEpochError() { .setPartitions(List.of(new WriteShareGroupStateRequestData.PartitionData() .setPartition(PARTITION) .setStartOffset(0) + .setDeliveryCompleteCount(11) .setStateEpoch(0) .setLeaderEpoch(0) .setStateBatches(List.of(new WriteShareGroupStateRequestData.StateBatch() @@ -506,6 +517,7 @@ public void testWriteStateFencedStateEpochError() { .setPartitions(List.of(new WriteShareGroupStateRequestData.PartitionData() .setPartition(PARTITION) .setStartOffset(0) + .setDeliveryCompleteCount(10) .setStateEpoch(0) // Lower state epoch in the second request. .setLeaderEpoch(5) .setStateBatches(List.of(new WriteShareGroupStateRequestData.StateBatch() @@ -753,6 +765,7 @@ public void testNonSequentialBatchUpdates() { .setPartitions(List.of(new WriteShareGroupStateRequestData.PartitionData() .setPartition(PARTITION) .setStartOffset(100) + .setDeliveryCompleteCount(0) .setStateEpoch(0) .setLeaderEpoch(0) .setStateBatches(List.of( @@ -800,6 +813,7 @@ public void testNonSequentialBatchUpdates() { .setPartitions(List.of(new WriteShareGroupStateRequestData.PartitionData() .setPartition(PARTITION) .setStartOffset(-1) + .setDeliveryCompleteCount(10) .setStateEpoch(0) .setLeaderEpoch(0) .setStateBatches(List.of( @@ -822,6 +836,7 @@ public void testNonSequentialBatchUpdates() { .setPartitions(List.of(new WriteShareGroupStateRequestData.PartitionData() .setPartition(PARTITION) .setStartOffset(110) // 100 -> 110 + .setDeliveryCompleteCount(10) .setStateEpoch(0) .setLeaderEpoch(0) .setStateBatches(List.of( @@ -839,6 +854,7 @@ public void testNonSequentialBatchUpdates() { WriteShareGroupStateResponseData expectedDataFinal = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION); ShareGroupOffset offsetFinal = new ShareGroupOffset.Builder() .setStartOffset(110) + .setDeliveryCompleteCount(10) .setLeaderEpoch(0) .setStateEpoch(0) .setSnapshotEpoch(5) // since subsequent share snapshot @@ -899,6 +915,7 @@ public void testReadStateLeaderEpochUpdateSuccess() { List expectedRecords = List.of(ShareCoordinatorRecordHelpers.newShareUpdateRecord( GROUP_ID, TOPIC_ID, PARTITION, new ShareGroupOffset.Builder() .setStartOffset(PartitionFactory.UNINITIALIZED_START_OFFSET) + .setDeliveryCompleteCount(PartitionFactory.UNINITIALIZED_DELIVERY_COMPLETE_COUNT) .setLeaderEpoch(2) .setStateBatches(List.of()) .setSnapshotEpoch(0) @@ -1339,6 +1356,8 @@ public void testSnapshotColdPartitionsNoEligiblePartitions() { new ShareSnapshotValue() .setSnapshotEpoch(0) .setStateEpoch(0) + .setStartOffset(0) + .setDeliveryCompleteCount(11) .setLeaderEpoch(leaderEpoch) .setCreateTimestamp(timestamp) .setWriteTimestamp(timestamp) @@ -1361,6 +1380,8 @@ public void testSnapshotColdPartitionsNoEligiblePartitions() { new ShareSnapshotValue() .setSnapshotEpoch(0) .setStateEpoch(0) + .setStartOffset(0) + .setDeliveryCompleteCount(11) .setLeaderEpoch(leaderEpoch) .setCreateTimestamp(timestamp) .setWriteTimestamp(timestamp) @@ -1405,6 +1426,8 @@ public void testSnapshotColdPartitionsSnapshotUpdateNotConsidered() { new ShareSnapshotValue() .setSnapshotEpoch(0) .setStateEpoch(0) + .setStartOffset(0) + .setDeliveryCompleteCount(11) .setLeaderEpoch(leaderEpoch) .setCreateTimestamp(timestamp) .setWriteTimestamp(timestamp) @@ -1436,6 +1459,8 @@ public void testSnapshotColdPartitionsSnapshotUpdateNotConsidered() { new ShareSnapshotValue() .setSnapshotEpoch(1) .setStateEpoch(0) + .setStartOffset(0) + .setDeliveryCompleteCount(11) .setLeaderEpoch(leaderEpoch) .setCreateTimestamp(timestamp) .setWriteTimestamp(timestamp + sleep) @@ -1463,6 +1488,8 @@ public void testSnapshotColdPartitionsSnapshotUpdateNotConsidered() { new ApiMessageAndVersion( new ShareUpdateValue() .setSnapshotEpoch(0) + .setStartOffset(0) + .setDeliveryCompleteCount(11) .setLeaderEpoch(leaderEpoch) .setStateBatches(List.of( new ShareUpdateValue.StateBatch() @@ -1502,6 +1529,8 @@ public void testSnapshotColdPartitionsDoesNotPerpetuallySnapshot() { new ShareSnapshotValue() .setSnapshotEpoch(0) .setStateEpoch(0) + .setStartOffset(0) + .setDeliveryCompleteCount(11) .setLeaderEpoch(leaderEpoch) .setCreateTimestamp(timestamp) .setWriteTimestamp(timestamp) @@ -1531,6 +1560,8 @@ public void testSnapshotColdPartitionsDoesNotPerpetuallySnapshot() { new ShareSnapshotValue() .setSnapshotEpoch(1) .setStateEpoch(0) + .setStartOffset(0) + .setDeliveryCompleteCount(11) .setLeaderEpoch(leaderEpoch) .setCreateTimestamp(timestamp) .setWriteTimestamp(timestamp + sleep) @@ -1579,6 +1610,8 @@ public void testSnapshotColdPartitionsPartialEligiblePartitions() { new ShareSnapshotValue() .setSnapshotEpoch(record1SnapshotEpoch) .setStateEpoch(0) + .setStartOffset(0) + .setDeliveryCompleteCount(11) .setLeaderEpoch(leaderEpoch) .setCreateTimestamp(timestamp) .setWriteTimestamp(timestamp) @@ -1603,6 +1636,8 @@ public void testSnapshotColdPartitionsPartialEligiblePartitions() { new ShareSnapshotValue() .setSnapshotEpoch(0) .setStateEpoch(0) + .setStartOffset(0) + .setDeliveryCompleteCount(11) .setLeaderEpoch(leaderEpoch) .setCreateTimestamp(timestamp + delta) .setWriteTimestamp(timestamp + delta) @@ -1637,6 +1672,8 @@ public void testSnapshotColdPartitionsPartialEligiblePartitions() { new ShareSnapshotValue() .setSnapshotEpoch(record1SnapshotEpoch + 1) .setStateEpoch(0) + .setStartOffset(0) + .setDeliveryCompleteCount(11) .setLeaderEpoch(leaderEpoch) .setCreateTimestamp(timestamp) .setWriteTimestamp(timestamp + sleep) @@ -1693,6 +1730,8 @@ public void testOnTopicsDeletedTopicIds() { new ShareSnapshotValue() .setSnapshotEpoch(record1SnapshotEpoch) .setStateEpoch(0) + .setStartOffset(0) + .setDeliveryCompleteCount(11) .setLeaderEpoch(leaderEpoch) .setCreateTimestamp(timestamp) .setWriteTimestamp(timestamp) @@ -1715,6 +1754,8 @@ public void testOnTopicsDeletedTopicIds() { new ShareSnapshotValue() .setSnapshotEpoch(record1SnapshotEpoch) .setStateEpoch(0) + .setStartOffset(0) + .setDeliveryCompleteCount(11) .setLeaderEpoch(leaderEpoch) .setCreateTimestamp(timestamp) .setWriteTimestamp(timestamp) @@ -1758,7 +1799,8 @@ private void initSharePartition(ShareCoordinatorShard shard, SharePartitionKey k new ShareSnapshotValue() .setStateEpoch(stateEpoch) .setLeaderEpoch(-1) - .setStartOffset(-1), + .setStartOffset(-1) + .setDeliveryCompleteCount(-1), (short) 0 ) )); From 1efe5de05c4e7d1680955bde08702a1778115b35 Mon Sep 17 00:00:00 2001 From: Chirag Wadhwa Date: Thu, 30 Oct 2025 23:46:22 +0530 Subject: [PATCH 2/3] KAFKA-19797: Minor change for better readability --- .../org/apache/kafka/coordinator/share/ShareGroupOffset.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareGroupOffset.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareGroupOffset.java index ddf286b1888c6..e2f274209b320 100644 --- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareGroupOffset.java +++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareGroupOffset.java @@ -256,7 +256,7 @@ public String toString() { ", stateEpoch=" + stateEpoch + ", leaderEpoch=" + leaderEpoch + ", startOffset=" + startOffset + - ", inFlightRecordCount=" + deliveryCompleteCount + + ", deliveryCompleteCount=" + deliveryCompleteCount + ", createTimestamp=" + createTimestamp + ", writeTimestamp=" + writeTimestamp + ", stateBatches=" + stateBatches + From 0c9eef59f75de8c0235dcc5740c47b1a18a0a6d8 Mon Sep 17 00:00:00 2001 From: Chirag Wadhwa Date: Sun, 2 Nov 2025 21:45:54 +0530 Subject: [PATCH 3/3] KAFKA-19797: Minor changes for better readability --- .../message/WriteShareGroupStateRequest.json | 3 + .../message/WriteShareGroupStateResponse.json | 3 + .../kafka/server/share/SharePartition.java | 80 +-- .../server/share/SharePartitionTest.java | 626 +++++++++--------- .../unit/kafka/server/KafkaApisTest.scala | 2 +- .../coordinator/share/ShareGroupOffset.java | 4 +- .../common/message/ShareSnapshotValue.json | 1 + .../common/message/ShareUpdateValue.json | 1 + 8 files changed, 364 insertions(+), 356 deletions(-) diff --git a/clients/src/main/resources/common/message/WriteShareGroupStateRequest.json b/clients/src/main/resources/common/message/WriteShareGroupStateRequest.json index af6aadcf6dbea..4bfdb46e96a14 100644 --- a/clients/src/main/resources/common/message/WriteShareGroupStateRequest.json +++ b/clients/src/main/resources/common/message/WriteShareGroupStateRequest.json @@ -18,6 +18,9 @@ "type": "request", "listeners": ["broker"], "name": "WriteShareGroupStateRequest", + // Version 0 is the initial version (KIP-932). + // + // Version 1 introduces DeliveryCompleteCount (KIP-1226). "validVersions": "0-1", "flexibleVersions": "0+", "fields": [ diff --git a/clients/src/main/resources/common/message/WriteShareGroupStateResponse.json b/clients/src/main/resources/common/message/WriteShareGroupStateResponse.json index edf5504382eb3..b1afd09d209bb 100644 --- a/clients/src/main/resources/common/message/WriteShareGroupStateResponse.json +++ b/clients/src/main/resources/common/message/WriteShareGroupStateResponse.json @@ -17,6 +17,9 @@ "apiKey": 85, "type": "response", "name": "WriteShareGroupStateResponse", + // Version 0 is the initial version (KIP-932). + // + // Version 1 introduces DeliveryCompleteCount in the request (KIP-1226). "validVersions": "0-1", "flexibleVersions": "0+", // - NOT_COORDINATOR (version 0+) diff --git a/core/src/main/java/kafka/server/share/SharePartition.java b/core/src/main/java/kafka/server/share/SharePartition.java index 826a7c862db30..900ce8ca68461 100644 --- a/core/src/main/java/kafka/server/share/SharePartition.java +++ b/core/src/main/java/kafka/server/share/SharePartition.java @@ -278,10 +278,10 @@ enum SharePartitionState { private final DelayedShareFetchKey delayedShareFetchKey; /** - * The inFlightTerminalRecords tracks the number of terminal (ACKNOWLEDGED / ARCHIVED) records within the + * The deliveryCompleteCount tracks the number of terminal (ACKNOWLEDGED / ARCHIVED) records within the * cachedState. This is used in the calculations for determining the current Share Partition lag. */ - private final AtomicInteger inFlightTerminalRecords; + private final AtomicInteger deliveryCompleteCount; /** * The state epoch is used to track the version of the state of the share partition. @@ -368,7 +368,7 @@ enum SharePartitionState { this.sharePartitionMetrics = sharePartitionMetrics; this.timeoutHandler = releaseAcquisitionLockOnTimeout(); this.registerGaugeMetrics(); - this.inFlightTerminalRecords = new AtomicInteger(0); + this.deliveryCompleteCount = new AtomicInteger(0); } /** @@ -475,10 +475,10 @@ public CompletableFuture maybeInitialize() { stateBatch.lastOffset(), RecordState.forId(stateBatch.deliveryState()), stateBatch.deliveryCount(), null, timeoutHandler, sharePartitionMetrics); cachedState.put(stateBatch.firstOffset(), inFlightBatch); - // During initialization, inFlightTerminalRecords is updated with the number of records that are in the + // During initialization, deliveryCompleteCount is updated with the number of records that are in the // ACKNOWLEDGED or ARCHIVED state. if (isStateTerminal(RecordState.forId(stateBatch.deliveryState()))) { - inFlightTerminalRecords.addAndGet((int) (stateBatch.lastOffset() - stateBatch.firstOffset() + 1)); + deliveryCompleteCount.addAndGet((int) (stateBatch.lastOffset() - stateBatch.firstOffset() + 1)); } sharePartitionMetrics.recordInFlightBatchMessageCount(stateBatch.lastOffset() - stateBatch.firstOffset() + 1); } @@ -1056,7 +1056,7 @@ private Optional releaseAcquiredRecordsForPerOffsetBatch(String membe persisterBatches.add(new PersisterBatch(updateResult, new PersisterStateBatch(offsetState.getKey(), offsetState.getKey(), updateResult.state().id(), (short) updateResult.deliveryCount()))); if (offsetState.getKey() >= startOffset && isStateTerminal(updateResult.state())) { - inFlightTerminalRecords.incrementAndGet(); + deliveryCompleteCount.incrementAndGet(); } // Do not update the next fetch offset as the offset has not completed the transition yet. } @@ -1097,7 +1097,7 @@ private Optional releaseAcquiredRecordsForCompleteBatch(String member persisterBatches.add(new PersisterBatch(updateResult, new PersisterStateBatch(inFlightBatch.firstOffset(), inFlightBatch.lastOffset(), updateResult.state().id(), (short) updateResult.deliveryCount()))); if (isStateTerminal(updateResult.state())) { - inFlightTerminalRecords.addAndGet(numInFlightRecordsInBatch(inFlightBatch.firstOffset(), inFlightBatch.lastOffset())); + deliveryCompleteCount.addAndGet(numInFlightRecordsInBatch(inFlightBatch.firstOffset(), inFlightBatch.lastOffset())); } // Do not update the next fetch offset as the batch has not completed the transition yet. } @@ -1139,7 +1139,7 @@ void updateCacheAndOffsets(long logStartOffset) { // This can occur during the initialization of share partition if LSO has moved. startOffset = logStartOffset; endOffset = logStartOffset; - inFlightTerminalRecords.set(0); + deliveryCompleteCount.set(0); return; } @@ -1158,8 +1158,8 @@ void updateCacheAndOffsets(long logStartOffset) { // The new startOffset will be the log start offset. startOffset = logStartOffset; - // After the start offset has moved, update the inFlightTerminalRecords. - inFlightTerminalRecords.addAndGet((-1) * numTerminalRecordsRemoved); + // After the start offset has moved, update the deliveryCompleteCount. + deliveryCompleteCount.addAndGet((-1) * numTerminalRecordsRemoved); if (endOffset < startOffset) { // This case means that the cached state is completely fresh now. // Example scenario - batch of 0-10 in acquired state in cached state, then LSO moves to 15, @@ -1294,7 +1294,7 @@ private int countTerminalRecordsUpToNewLSO(long newLSO) { * @param endOffset The offset before which the records should be archived. * @param map The map containing the in-flight records. * @param initialState The initial state of the records to be archived. - * @param updateInFlightTerminalRecords A boolean which indicates whether the inFlightTerminalRecords should be updated. + * @param updateDeliveryCompleteCount A boolean which indicates whether the deliveryCompleteCount should be updated. * @return A boolean which indicates whether any record is archived or not. */ private boolean archiveRecords( @@ -1302,7 +1302,7 @@ private boolean archiveRecords( long endOffset, NavigableMap map, RecordState initialState, - boolean updateInFlightTerminalRecords) { + boolean updateDeliveryCompleteCount) { lock.writeLock().lock(); try { boolean isAnyOffsetArchived = false, isAnyBatchArchived = false; @@ -1328,11 +1328,11 @@ private boolean archiveRecords( } inFlightBatch.maybeInitializeOffsetStateUpdate(); } - isAnyOffsetArchived = archivePerOffsetBatchRecords(inFlightBatch, startOffset, endOffset - 1, initialState, updateInFlightTerminalRecords) || isAnyOffsetArchived; + isAnyOffsetArchived = archivePerOffsetBatchRecords(inFlightBatch, startOffset, endOffset - 1, initialState, updateDeliveryCompleteCount) || isAnyOffsetArchived; continue; } // The in-flight batch is a full match hence change the state of the complete batch. - isAnyBatchArchived = archiveCompleteBatch(inFlightBatch, initialState, updateInFlightTerminalRecords) || isAnyBatchArchived; + isAnyBatchArchived = archiveCompleteBatch(inFlightBatch, initialState, updateDeliveryCompleteCount) || isAnyBatchArchived; } return isAnyOffsetArchived || isAnyBatchArchived; } finally { @@ -1344,7 +1344,7 @@ private boolean archivePerOffsetBatchRecords(InFlightBatch inFlightBatch, long startOffsetToArchive, long endOffsetToArchive, RecordState initialState, - boolean updateInFlightTerminalRecords + boolean updateDeliveryCompleteCount ) { lock.writeLock().lock(); try { @@ -1363,10 +1363,10 @@ private boolean archivePerOffsetBatchRecords(InFlightBatch inFlightBatch, } offsetState.getValue().archive(); - if (updateInFlightTerminalRecords) { - // If the record moves from a non-terminal state to a terminal state (in this case ARCHIVE), inFlightTerminalRecords + if (updateDeliveryCompleteCount) { + // If the record moves from a non-terminal state to a terminal state (in this case ARCHIVE), deliveryCompleteCount // needs to be incremented. - inFlightTerminalRecords.incrementAndGet(); + deliveryCompleteCount.incrementAndGet(); } isAnyOffsetArchived = true; } @@ -1379,17 +1379,17 @@ private boolean archivePerOffsetBatchRecords(InFlightBatch inFlightBatch, private boolean archiveCompleteBatch( InFlightBatch inFlightBatch, RecordState initialState, - boolean updateInFlightTerminalRecords) { + boolean updateDeliveryCompleteCount) { lock.writeLock().lock(); try { log.trace("Archiving complete batch: {} for the share partition: {}-{}", inFlightBatch, groupId, topicIdPartition); if (inFlightBatch.batchState() == initialState) { // Change the state of complete batch since the same state exists for the entire inFlight batch. inFlightBatch.archiveBatch(); - if (updateInFlightTerminalRecords) { - // If the records move from a non-terminal state to a terminal state (in this case ARCHIVE), inFlightTerminalRecords + if (updateDeliveryCompleteCount) { + // If the records move from a non-terminal state to a terminal state (in this case ARCHIVE), deliveryCompleteCount // needs to be incremented by the number of records in the batch. - inFlightTerminalRecords.addAndGet((int) (inFlightBatch.lastOffset() - inFlightBatch.firstOffset() + 1)); + deliveryCompleteCount.addAndGet((int) (inFlightBatch.lastOffset() - inFlightBatch.firstOffset() + 1)); } return true; } @@ -2103,7 +2103,7 @@ private Optional acknowledgePerOffsetBatchRecords( persisterBatches.add(new PersisterBatch(updateResult, new PersisterStateBatch(offsetState.getKey(), offsetState.getKey(), updateResult.state().id(), (short) updateResult.deliveryCount()))); if (isStateTerminal(updateResult.state())) { - inFlightTerminalRecords.incrementAndGet(); + deliveryCompleteCount.incrementAndGet(); } // Do not update the nextFetchOffset as the offset has not completed the transition yet. } @@ -2153,7 +2153,7 @@ private Optional acknowledgeCompleteBatch( persisterBatches.add(new PersisterBatch(updateResult, new PersisterStateBatch(inFlightBatch.firstOffset(), inFlightBatch.lastOffset(), updateResult.state().id(), (short) updateResult.deliveryCount()))); if (isStateTerminal(updateResult.state())) { - inFlightTerminalRecords.addAndGet((int) (inFlightBatch.lastOffset() - inFlightBatch.firstOffset() + 1)); + deliveryCompleteCount.addAndGet((int) (inFlightBatch.lastOffset() - inFlightBatch.firstOffset() + 1)); } // Do not update the next fetch offset as the batch has not completed the transition yet. } finally { @@ -2209,12 +2209,12 @@ void rollbackOrProcessStateUpdates( if (persisterBatch.updatedState.state() == RecordState.AVAILABLE) { updateFindNextFetchOffset(true); } - // If there is a failure, then update the inFlightTerminalRecords only in case there are some records + // If there is a failure, then update the deliveryCompleteCount only in case there are some records // which were in a Terminal state, but after rolling back they are in a non-Terminal state. We also // need to consider only those records that lie after the start offset, because LSO movement can happen // after local state transition begins but before writeState result is obtained. if (isStateTerminal(RecordState.forId(persisterBatch.stateBatch.deliveryState())) && !isStateTerminal(persisterBatch.updatedState.state())) { - inFlightTerminalRecords.addAndGet(-numInFlightRecordsInBatch(persisterBatch.stateBatch.firstOffset(), persisterBatch.stateBatch.lastOffset())); + deliveryCompleteCount.addAndGet(-numInFlightRecordsInBatch(persisterBatch.stateBatch.firstOffset(), persisterBatch.stateBatch.lastOffset())); } }); future.completeExceptionally(throwable); @@ -2247,12 +2247,12 @@ void rollbackOrProcessStateUpdates( if (persisterBatch.updatedState().state() == RecordState.AVAILABLE) { updateFindNextFetchOffset(true); } - // If there is a failure, then update the inFlightTerminalRecords only in case there are some records + // If there is a failure, then update the deliveryCompleteCount only in case there are some records // which were in a Terminal state, but after rolling back they are in a non-Terminal state. We also // need to consider only those records that lie after the start offset, because LSO movement can happen // after local state transition begins but before writeState result is obtained. if (isStateTerminal(RecordState.forId(persisterBatch.stateBatch.deliveryState())) && !isStateTerminal(persisterBatch.updatedState.state())) { - inFlightTerminalRecords.addAndGet(-numInFlightRecordsInBatch(persisterBatch.stateBatch.firstOffset(), persisterBatch.stateBatch.lastOffset())); + deliveryCompleteCount.addAndGet(-numInFlightRecordsInBatch(persisterBatch.stateBatch.firstOffset(), persisterBatch.stateBatch.lastOffset())); } }); future.completeExceptionally(exception); @@ -2311,7 +2311,7 @@ private boolean maybeUpdateCachedStateAndOffsets() { startOffset = lastCachedOffset + 1; // The next offset that will be fetched and acquired in the share partition endOffset = lastCachedOffset + 1; cachedState.clear(); - inFlightTerminalRecords.set(0); + deliveryCompleteCount.set(0); // Nothing further to do. return true; } @@ -2356,7 +2356,7 @@ be removed once all the records (0-99) are acknowledged (ACCEPT or REJECT). lastKeyToRemove = cachedState.lowerKey(entry.getKey()); } } - inFlightTerminalRecords.addAndGet((-1) * result.numTerminalRecords); + deliveryCompleteCount.addAndGet((-1) * result.numTerminalRecords); if (lastKeyToRemove != -1) { cachedState.subMap(firstKeyToRemove, true, lastKeyToRemove, true).clear(); } @@ -2427,7 +2427,7 @@ private boolean isStateTerminal(RecordState recordState) { * the current startOffset which are in terminal state (ACKNOWLEDGED or ARCHIVED), then the startOffset can be moved * past these. The method iterates over the cached state and finds the last offset which is in terminal state which * can be moved out of in-flight. The method also counts the number of records which are in terminal state which would - * be removed from in-flight if the startOffset is moved, in order to update the inFlightTerminalRecords count. + * be removed from in-flight if the startOffset is moved, in order to update the deliveryCompleteCount count. * * @return StartOffsetAdvanceResult, which contains the last offset post which the startOffset can move * and the number of terminal records which be moved out of in-flight if the startOffset is moved. @@ -2504,7 +2504,7 @@ CompletableFuture writeShareGroupState(List stateBatc .setGroupId(this.groupId) .setTopicsData(List.of(new TopicData<>(topicIdPartition.topicId(), List.of(PartitionFactory.newPartitionStateBatchData( - topicIdPartition.partition(), stateEpoch, startOffset(), inFlightTerminalRecords(), leaderEpoch, stateBatches)))) + topicIdPartition.partition(), stateEpoch, startOffset(), deliveryCompleteCount(), leaderEpoch, stateBatches)))) ).build()).build()) .whenComplete((result, exception) -> { if (exception != null) { @@ -2683,12 +2683,12 @@ private void releaseAcquisitionLockOnTimeoutForCompleteBatch(InFlightBatch inFli updateFindNextFetchOffset(true); } // If after acquisition lock timeout, a batch is moved to ARCHIVED state, then we have records moved from - // non-Terminal state to a Terminal state. Hence, we need to update inFlightTerminalRecords. But there could + // non-Terminal state to a Terminal state. Hence, we need to update deliveryCompleteCount. But there could // be a situation where the batch was acquired, and LSO moved past. In that case, we only need to consider the // records from the current batch that are greater than or equal to the new startOffset. For the records that - // lie before the new startOffset, they would have already been counted in inFlightTerminalRecords when LSO moved. + // lie before the new startOffset, they would have already been counted in deliveryCompleteCount when LSO moved. if (isStateTerminal(updateResult.state())) { - inFlightTerminalRecords.addAndGet(numInFlightRecordsInBatch(inFlightBatch.firstOffset(), inFlightBatch.lastOffset())); + deliveryCompleteCount.addAndGet(numInFlightRecordsInBatch(inFlightBatch.firstOffset(), inFlightBatch.lastOffset())); } return; } @@ -2738,11 +2738,11 @@ private void releaseAcquisitionLockOnTimeoutForPerOffsetBatch(InFlightBatch inFl updateFindNextFetchOffset(true); } // If after acquisition lock timeout, an offset state is moved to ARCHIVED, then we have records moved from - // non-Terminal state to a Terminal state. Hence, we need to update inFlightTerminalRecords, but only if the + // non-Terminal state to a Terminal state. Hence, we need to update deliveryCompleteCount, but only if the // records are past the startOffset. If the LSO was moved in between, then we don't need to care about the - // records that lie before the startOffset as they would have already been counted in inFlightTerminalRecords. + // records that lie before the startOffset as they would have already been counted in deliveryCompleteCount. if (offsetState.getKey() >= startOffset && isStateTerminal(updateResult.state())) { - inFlightTerminalRecords.incrementAndGet(); + deliveryCompleteCount.incrementAndGet(); } } } @@ -3035,8 +3035,8 @@ Uuid fetchLock() { } // Visible for testing. - int inFlightTerminalRecords() { - return inFlightTerminalRecords.get(); + int deliveryCompleteCount() { + return deliveryCompleteCount.get(); } /** diff --git a/core/src/test/java/kafka/server/share/SharePartitionTest.java b/core/src/test/java/kafka/server/share/SharePartitionTest.java index 36a2d0a1ac586..149c5e7a34018 100644 --- a/core/src/test/java/kafka/server/share/SharePartitionTest.java +++ b/core/src/test/java/kafka/server/share/SharePartitionTest.java @@ -184,8 +184,8 @@ public void testMaybeInitialize() throws InterruptedException { assertEquals(3, sharePartition.cachedState().get(11L).batchDeliveryCount()); assertNull(sharePartition.cachedState().get(11L).offsetState()); - // inFlightTerminalRecords is incremented by the number of ACKNOWLEDGED and ARCHIVED records in readState result. - assertEquals(5, sharePartition.inFlightTerminalRecords()); + // deliveryCompleteCount is incremented by the number of ACKNOWLEDGED and ARCHIVED records in readState result. + assertEquals(5, sharePartition.deliveryCompleteCount()); TestUtils.waitForCondition(() -> yammerMetricValue(SharePartitionMetrics.IN_FLIGHT_BATCH_COUNT).intValue() == 2, "In-flight batch count should be 2."); @@ -245,7 +245,7 @@ public void testMaybeInitializeDefaultStartEpochGroupConfigReturnsEarliest() { assertEquals(0, sharePartition.startOffset()); assertEquals(0, sharePartition.endOffset()); assertEquals(PartitionFactory.DEFAULT_STATE_EPOCH, sharePartition.stateEpoch()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -296,7 +296,7 @@ public void testMaybeInitializeDefaultStartEpochGroupConfigReturnsLatest() { assertEquals(15, sharePartition.startOffset()); assertEquals(15, sharePartition.endOffset()); assertEquals(PartitionFactory.DEFAULT_STATE_EPOCH, sharePartition.stateEpoch()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -358,7 +358,7 @@ public void testMaybeInitializeDefaultStartEpochGroupConfigReturnsByDuration() assertEquals(15, sharePartition.startOffset()); assertEquals(15, sharePartition.endOffset()); assertEquals(PartitionFactory.DEFAULT_STATE_EPOCH, sharePartition.stateEpoch()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); TestUtils.waitForCondition(() -> yammerMetricValue(SharePartitionMetrics.IN_FLIGHT_BATCH_COUNT).intValue() == 0, "In-flight batch count should be 0."); @@ -641,7 +641,7 @@ public void testMaybeInitializeWithEmptyStateBatches() { assertEquals(10, sharePartition.endOffset()); assertEquals(5, sharePartition.stateEpoch()); assertEquals(10, sharePartition.nextFetchOffset()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -978,8 +978,8 @@ public void testMaybeInitializeStateBatchesWithGapAtBeginning() { assertEquals(10, persisterReadResultGapWindow.gapStartOffset()); assertEquals(30, persisterReadResultGapWindow.endOffset()); - // inFlightTerminalRecords is incremented by the number of ACKNOWLEDGED and ARCHIVED records in readState result. - assertEquals(16, sharePartition.inFlightTerminalRecords()); + // deliveryCompleteCount is incremented by the number of ACKNOWLEDGED and ARCHIVED records in readState result. + assertEquals(16, sharePartition.deliveryCompleteCount()); } @Test @@ -1026,8 +1026,8 @@ public void testMaybeInitializeStateBatchesWithMultipleGaps() { assertEquals(10, persisterReadResultGapWindow.gapStartOffset()); assertEquals(40, persisterReadResultGapWindow.endOffset()); - // inFlightTerminalRecords is incremented by the number of ACKNOWLEDGED and ARCHIVED records in readState result. - assertEquals(17, sharePartition.inFlightTerminalRecords()); + // deliveryCompleteCount is incremented by the number of ACKNOWLEDGED and ARCHIVED records in readState result. + assertEquals(17, sharePartition.deliveryCompleteCount()); } @Test @@ -1069,7 +1069,7 @@ public void testMaybeInitializeStateBatchesWithGapNotAtBeginning() { assertEquals(21, persisterReadResultGapWindow.gapStartOffset()); assertEquals(40, persisterReadResultGapWindow.endOffset()); - assertEquals(11, sharePartition.inFlightTerminalRecords()); + assertEquals(11, sharePartition.deliveryCompleteCount()); } @Test @@ -1095,7 +1095,7 @@ public void testMaybeInitializeStateBatchesWithoutGaps() { assertEquals(31, sharePartition.endOffset()); assertEquals(3, sharePartition.stateEpoch()); assertEquals(31, sharePartition.nextFetchOffset()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); @@ -1136,7 +1136,7 @@ public void testMaybeInitializeAndAcquire() { assertNotNull(sharePartition.persisterReadResultGapWindow()); assertEquals(10L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); assertEquals(30L, sharePartition.persisterReadResultGapWindow().endOffset()); - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); // Create a single batch record that covers the entire range from 10 to 30 of initial read gap. // The records in the batch are from 10 to 49. @@ -1164,7 +1164,7 @@ public void testMaybeInitializeAndAcquire() { assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(26L).batchState()); assertNotNull(sharePartition.persisterReadResultGapWindow()); assertEquals(15L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); // Send the same batch again to acquire the next set of records. acquiredRecordsList = fetchAcquiredRecords(sharePartition.acquire( @@ -1199,7 +1199,7 @@ public void testMaybeInitializeAndAcquire() { assertEquals(30L, sharePartition.endOffset()); // As all the gaps are now filled, the persisterReadResultGapWindow should be null. assertNull(sharePartition.persisterReadResultGapWindow()); - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); // Now initial read gap is filled, so the complete batch can be acquired despite max fetch records being 1. acquiredRecordsList = fetchAcquiredRecords(sharePartition.acquire( @@ -1220,7 +1220,7 @@ public void testMaybeInitializeAndAcquire() { assertEquals(1, sharePartition.cachedState().get(31L).batchDeliveryCount()); assertNull(sharePartition.cachedState().get(31L).offsetState()); assertEquals(49L, sharePartition.endOffset()); - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); } @Test @@ -1256,7 +1256,7 @@ public void testMaybeInitializeAndAcquireWithHigherMaxFetchRecords() { assertNotNull(sharePartition.persisterReadResultGapWindow()); assertEquals(10L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); assertEquals(30L, sharePartition.persisterReadResultGapWindow().endOffset()); - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); // Create a single batch record that covers the entire range from 10 to 30 of initial read gap. // The records in the batch are from 10 to 49. @@ -1300,7 +1300,7 @@ public void testMaybeInitializeAndAcquireWithHigherMaxFetchRecords() { assertEquals(49L, sharePartition.endOffset()); // As all the gaps are now filled, the persisterReadResultGapWindow should be null. assertNull(sharePartition.persisterReadResultGapWindow()); - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); } @Test @@ -1336,7 +1336,7 @@ public void testMaybeInitializeAndAcquireWithFetchBatchLastOffsetWithinCachedBat assertNotNull(sharePartition.persisterReadResultGapWindow()); assertEquals(10L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); assertEquals(30L, sharePartition.persisterReadResultGapWindow().endOffset()); - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); // Create a single batch record that ends in between the cached batch and the fetch offset is // post startOffset. @@ -1381,7 +1381,7 @@ public void testMaybeInitializeAndAcquireWithFetchBatchLastOffsetWithinCachedBat assertEquals(30L, sharePartition.endOffset()); assertNotNull(sharePartition.persisterReadResultGapWindow()); assertEquals(28L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); } @Test @@ -1417,7 +1417,7 @@ public void testMaybeInitializeAndAcquireWithFetchBatchPriorStartOffset() { assertNotNull(sharePartition.persisterReadResultGapWindow()); assertEquals(10L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); assertEquals(30L, sharePartition.persisterReadResultGapWindow().endOffset()); - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); // Create a single batch record where first offset is prior startOffset. MemoryRecords records = memoryRecords(6, 16); @@ -1451,7 +1451,7 @@ public void testMaybeInitializeAndAcquireWithFetchBatchPriorStartOffset() { assertEquals(30L, sharePartition.endOffset()); assertNotNull(sharePartition.persisterReadResultGapWindow()); assertEquals(20L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); } @Test @@ -1487,7 +1487,7 @@ public void testMaybeInitializeAndAcquireWithMultipleBatches() { assertNotNull(sharePartition.persisterReadResultGapWindow()); assertEquals(5L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); assertEquals(30L, sharePartition.persisterReadResultGapWindow().endOffset()); - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); // Create multiple batch records that covers the entire range from 5 to 30 of initial read gap. // The records in the batch are from 5 to 49. @@ -1524,7 +1524,7 @@ public void testMaybeInitializeAndAcquireWithMultipleBatches() { assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(26L).batchState()); assertNotNull(sharePartition.persisterReadResultGapWindow()); assertEquals(7L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); // Remove first batch from the records as the fetch offset has moved forward to 7 offset. List batch = TestUtils.toList(records.batches()); @@ -1553,7 +1553,7 @@ public void testMaybeInitializeAndAcquireWithMultipleBatches() { assertEquals(30L, sharePartition.endOffset()); assertNotNull(sharePartition.persisterReadResultGapWindow()); assertEquals(12L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); // Remove the next 2 batches from the records as the fetch offset has moved forward to 12 offset. int size = batch.get(1).sizeInBytes() + batch.get(2).sizeInBytes(); @@ -1591,7 +1591,7 @@ public void testMaybeInitializeAndAcquireWithMultipleBatches() { assertEquals(30L, sharePartition.endOffset()); assertNotNull(sharePartition.persisterReadResultGapWindow()); assertEquals(26L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); // Remove the next 2 batches from the records as the fetch offset has moved forward to 26 offset. // Do not remove the 5th batch as it's only partially acquired. @@ -1621,7 +1621,7 @@ public void testMaybeInitializeAndAcquireWithMultipleBatches() { assertEquals(49L, sharePartition.endOffset()); // As all the gaps are now filled, the persisterReadResultGapWindow should be null. assertNull(sharePartition.persisterReadResultGapWindow()); - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); } @Test @@ -1657,7 +1657,7 @@ public void testMaybeInitializeAndAcquireWithMultipleBatchesAndLastOffsetWithinC assertNotNull(sharePartition.persisterReadResultGapWindow()); assertEquals(5L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); assertEquals(30L, sharePartition.persisterReadResultGapWindow().endOffset()); - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); // Create multiple batch records that ends in between the cached batch and the fetch offset is // post startOffset. @@ -1709,7 +1709,7 @@ public void testMaybeInitializeAndAcquireWithMultipleBatchesAndLastOffsetWithinC assertEquals(30L, sharePartition.endOffset()); assertNotNull(sharePartition.persisterReadResultGapWindow()); assertEquals(28L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); } @Test @@ -1745,7 +1745,7 @@ public void testMaybeInitializeAndAcquireWithMultipleBatchesPriorStartOffset() { assertNotNull(sharePartition.persisterReadResultGapWindow()); assertEquals(10L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); assertEquals(30L, sharePartition.persisterReadResultGapWindow().endOffset()); - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); // Create multiple batch records where multiple batches base offsets are prior startOffset. ByteBuffer buffer = ByteBuffer.allocate(4096); @@ -1785,7 +1785,7 @@ public void testMaybeInitializeAndAcquireWithMultipleBatchesPriorStartOffset() { assertEquals(30L, sharePartition.endOffset()); assertNotNull(sharePartition.persisterReadResultGapWindow()); assertEquals(20L, sharePartition.persisterReadResultGapWindow().gapStartOffset()); - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); } @Test @@ -1808,8 +1808,8 @@ public void testAcquireSingleRecord() throws InterruptedException { assertEquals(1, sharePartition.cachedState().get(0L).batchDeliveryCount()); assertNull(sharePartition.cachedState().get(0L).offsetState()); - // inFlightTerminalRecords will not be changed because no record went to a Terminal state. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // deliveryCompleteCount will not be changed because no record went to a Terminal state. + assertEquals(0, sharePartition.deliveryCompleteCount()); TestUtils.waitForCondition(() -> yammerMetricValue(SharePartitionMetrics.IN_FLIGHT_BATCH_COUNT).intValue() == 1, "In-flight batch count should be 1."); @@ -1838,8 +1838,8 @@ public void testAcquireMultipleRecords() throws InterruptedException { assertEquals(1, sharePartition.cachedState().get(10L).batchDeliveryCount()); assertNull(sharePartition.cachedState().get(10L).offsetState()); - // inFlightTerminalRecords will not be changed because no record went to a Terminal state. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // deliveryCompleteCount will not be changed because no record went to a Terminal state. + assertEquals(0, sharePartition.deliveryCompleteCount()); TestUtils.waitForCondition(() -> yammerMetricValue(SharePartitionMetrics.IN_FLIGHT_BATCH_COUNT).intValue() == 1, "In-flight batch count should be 1."); @@ -2649,7 +2649,7 @@ public void testAcknowledgeSingleRecordBatch() { // Should not invoke completeDelayedShareFetchRequest as the first offset is not acknowledged yet. Mockito.verify(replicaManager, Mockito.times(0)) .completeDelayedShareFetchRequest(new DelayedShareFetchGroupKey(GROUP_ID, TOPIC_ID_PARTITION)); - assertEquals(1, sharePartition.inFlightTerminalRecords()); + assertEquals(1, sharePartition.deliveryCompleteCount()); } @Test @@ -2675,7 +2675,7 @@ public void testAcknowledgeMultipleRecordBatch() { // Should invoke completeDelayedShareFetchRequest as the start offset is moved. Mockito.verify(replicaManager, Mockito.times(1)) .completeDelayedShareFetchRequest(new DelayedShareFetchGroupKey(GROUP_ID, TOPIC_ID_PARTITION)); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -2730,9 +2730,9 @@ public void testAcknowledgeMultipleRecordBatchWithGapOffsets() { expectedOffsetStateMap.put(18L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); - // Out of the 11 records acquired, 3 are GAP and 1 is ACCEPTED, which are Terminal. Thus inFlightTerminalRecords + // Out of the 11 records acquired, 3 are GAP and 1 is ACCEPTED, which are Terminal. Thus deliveryCompleteCount // will be 4 - assertEquals(4, sharePartition.inFlightTerminalRecords()); + assertEquals(4, sharePartition.deliveryCompleteCount()); } @Test @@ -2797,7 +2797,7 @@ public void testAcknowledgeMultipleSubsetRecordBatchWithGapOffsets() { assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); // After acknowledgements, records at offsets 6, and 10 -> 18 are in Terminal state. - assertEquals(10, sharePartition.inFlightTerminalRecords()); + assertEquals(10, sharePartition.deliveryCompleteCount()); } @Test @@ -2823,7 +2823,7 @@ public void testAcknowledgeOutOfRangeCachedData() { List.of(new ShareAcknowledgementBatch(20, 25, List.of((byte) 3)))); assertTrue(ackResult.isCompletedExceptionally()); assertFutureThrows(InvalidRequestException.class, ackResult); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -2856,7 +2856,7 @@ public void testAcknowledgeOutOfRangeCachedDataFirstBatch() { acquiredRecordsList = fetchAcquiredRecords(sharePartition, records, 6); assertEquals(1, acquiredRecordsList.size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Previous failed acknowledge request should succeed now. ackResult = sharePartition.acknowledge( @@ -2864,8 +2864,8 @@ public void testAcknowledgeOutOfRangeCachedDataFirstBatch() { assertNull(ackResult.join()); assertFalse(ackResult.isCompletedExceptionally()); // After the acknowledgments are successful, the cache is updated. Since all record batches are in Terminal state, - // the cache is cleared and thus inFlightTerminalRecords is set as 0. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // the cache is cleared and thus deliveryCompleteCount is set as 0. + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -2904,7 +2904,7 @@ public void testAcknowledgeWhenOffsetNotAcquired() { assertFalse(ackResult.isCompletedExceptionally()); // All records are RELEASED, so none of them moved to a Terminal state. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Acknowledge the same batch again but with ACCEPT type. ackResult = sharePartition.acknowledge( @@ -2917,14 +2917,14 @@ public void testAcknowledgeWhenOffsetNotAcquired() { acquiredRecordsList = fetchAcquiredRecords(sharePartition, records, 5); assertEquals(1, acquiredRecordsList.size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); ackResult = sharePartition.acknowledge( MEMBER_ID, List.of(new ShareAcknowledgementBatch(6, 8, List.of((byte) 3)))); assertNull(ackResult.join()); assertFalse(ackResult.isCompletedExceptionally()); - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); // Re-acknowledge the subset batch with REJECT type. ackResult = sharePartition.acknowledge( @@ -2932,7 +2932,7 @@ public void testAcknowledgeWhenOffsetNotAcquired() { List.of(new ShareAcknowledgementBatch(6, 8, List.of((byte) 3)))); assertTrue(ackResult.isCompletedExceptionally()); assertFutureThrows(InvalidRecordStateException.class, ackResult); - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); } @Test @@ -2971,7 +2971,7 @@ public void testAcknowledgeRollbackWithFullBatchError() { assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(5L).batchState()); assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(10L).batchState()); assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(15L).batchState()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -3012,7 +3012,7 @@ public void testAcknowledgeRollbackWithSubsetError() { // Though the last batch is subset but the offset state map will not be exploded as the batch is // not in acquired state due to previous batch acknowledgement. assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(15L).batchState()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -3042,14 +3042,14 @@ public void testAcquireReleasedRecord() { expectedOffsetStateMap.put(13L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(14L, new InFlightState(RecordState.ACQUIRED, (short) 1, MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Send the same fetch request batch again but only 2 offsets should come as acquired. acquiredRecordsList = fetchAcquiredRecords(sharePartition, records, 2); assertArrayEquals(expectedAcquiredRecords(12, 13, 2).toArray(), acquiredRecordsList.toArray()); assertEquals(15, sharePartition.nextFetchOffset()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -3186,7 +3186,7 @@ public void testAcquireGapAtBeginningAndRecordsFetchedFromGap() { SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister).build(); sharePartition.maybeInitialize(); - assertEquals(20, sharePartition.inFlightTerminalRecords()); + assertEquals(20, sharePartition.deliveryCompleteCount()); // All records fetched are part of the gap. The gap is from 11 to 20, fetched offsets are 11 to 15. MemoryRecords records = memoryRecords(11, 5); @@ -3201,7 +3201,7 @@ public void testAcquireGapAtBeginningAndRecordsFetchedFromGap() { assertEquals(40, sharePartition.endOffset()); assertEquals(3, sharePartition.stateEpoch()); assertEquals(16, sharePartition.nextFetchOffset()); - assertEquals(20, sharePartition.inFlightTerminalRecords()); + assertEquals(20, sharePartition.deliveryCompleteCount()); GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); assertNotNull(persisterReadResultGapWindow); @@ -3227,7 +3227,7 @@ public void testAcquireGapAtBeginningAndFetchedRecordsOverlapInFlightBatches() { SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister).build(); sharePartition.maybeInitialize(); - assertEquals(20, sharePartition.inFlightTerminalRecords()); + assertEquals(20, sharePartition.deliveryCompleteCount()); // Fetched offsets overlap the inFlight batches. The gap is from 11 to 20, but fetched records are from 11 to 25. MemoryRecords records = memoryRecords(11, 15); @@ -3242,7 +3242,7 @@ public void testAcquireGapAtBeginningAndFetchedRecordsOverlapInFlightBatches() { assertEquals(40, sharePartition.endOffset()); assertEquals(3, sharePartition.stateEpoch()); assertEquals(41, sharePartition.nextFetchOffset()); - assertEquals(20, sharePartition.inFlightTerminalRecords()); + assertEquals(20, sharePartition.deliveryCompleteCount()); GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); assertNotNull(persisterReadResultGapWindow); @@ -3274,7 +3274,7 @@ public void testAcquireGapAtBeginningAndFetchedRecordsOverlapInFlightAvailableBa SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister).build(); sharePartition.maybeInitialize(); - assertEquals(10, sharePartition.inFlightTerminalRecords()); + assertEquals(10, sharePartition.deliveryCompleteCount()); // Fetched offsets overlap the inFlight batches. The gap is from 11 to 20, but fetched records are from 11 to 25. MemoryRecords records = memoryRecords(11, 15); @@ -3297,7 +3297,7 @@ public void testAcquireGapAtBeginningAndFetchedRecordsOverlapInFlightAvailableBa assertEquals(40, sharePartition.endOffset()); assertEquals(3, sharePartition.stateEpoch()); assertEquals(26, sharePartition.nextFetchOffset()); - assertEquals(10, sharePartition.inFlightTerminalRecords()); + assertEquals(10, sharePartition.deliveryCompleteCount()); GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); assertNotNull(persisterReadResultGapWindow); @@ -3323,7 +3323,7 @@ public void testAcquireWhenCachedStateContainsGapsAndRecordsFetchedFromNonGapOff SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister).build(); sharePartition.maybeInitialize(); - assertEquals(10, sharePartition.inFlightTerminalRecords()); + assertEquals(10, sharePartition.deliveryCompleteCount()); // Fetched records are part of inFlightBatch 11-20 with state AVAILABLE. Fetched offsets also overlap the // inFlight batches. The gap is from 11 to 20, but fetched records are from 11 to 25. @@ -3343,7 +3343,7 @@ public void testAcquireWhenCachedStateContainsGapsAndRecordsFetchedFromNonGapOff assertEquals(40, sharePartition.endOffset()); assertEquals(3, sharePartition.stateEpoch()); assertEquals(26, sharePartition.nextFetchOffset()); - assertEquals(10, sharePartition.inFlightTerminalRecords()); + assertEquals(10, sharePartition.deliveryCompleteCount()); GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); assertNotNull(persisterReadResultGapWindow); @@ -3371,7 +3371,7 @@ public void testAcquireGapAtBeginningAndFetchedRecordsOverlapMultipleInFlightBat SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister).build(); sharePartition.maybeInitialize(); - assertEquals(20, sharePartition.inFlightTerminalRecords()); + assertEquals(20, sharePartition.deliveryCompleteCount()); MemoryRecords records = memoryRecords(11, 75); @@ -3406,7 +3406,7 @@ public void testAcquireGapAtBeginningAndFetchedRecordsOverlapMultipleInFlightBat GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); assertNotNull(persisterReadResultGapWindow); - assertEquals(20, sharePartition.inFlightTerminalRecords()); + assertEquals(20, sharePartition.deliveryCompleteCount()); // After records are acquired, the persisterReadResultGapWindow should be updated assertEquals(86, persisterReadResultGapWindow.gapStartOffset()); @@ -3430,7 +3430,7 @@ public void testAcquireGapAtBeginningAndFetchedRecordsEndJustBeforeGap() { SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister).build(); sharePartition.maybeInitialize(); - assertEquals(20, sharePartition.inFlightTerminalRecords()); + assertEquals(20, sharePartition.deliveryCompleteCount()); MemoryRecords records = memoryRecords(11, 20); @@ -3449,7 +3449,7 @@ public void testAcquireGapAtBeginningAndFetchedRecordsEndJustBeforeGap() { assertEquals(70, sharePartition.endOffset()); assertEquals(3, sharePartition.stateEpoch()); assertEquals(31, sharePartition.nextFetchOffset()); - assertEquals(20, sharePartition.inFlightTerminalRecords()); + assertEquals(20, sharePartition.deliveryCompleteCount()); GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); assertNotNull(persisterReadResultGapWindow); @@ -3477,7 +3477,7 @@ public void testAcquireGapAtBeginningAndFetchedRecordsIncludeGapOffsetsAtEnd() { SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister).build(); sharePartition.maybeInitialize(); - assertEquals(20, sharePartition.inFlightTerminalRecords()); + assertEquals(20, sharePartition.deliveryCompleteCount()); MemoryRecords records = memoryRecords(11, 65); @@ -3502,7 +3502,7 @@ public void testAcquireGapAtBeginningAndFetchedRecordsIncludeGapOffsetsAtEnd() { assertEquals(90, sharePartition.endOffset()); assertEquals(3, sharePartition.stateEpoch()); assertEquals(76, sharePartition.nextFetchOffset()); - assertEquals(20, sharePartition.inFlightTerminalRecords()); + assertEquals(20, sharePartition.deliveryCompleteCount()); GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); assertNotNull(persisterReadResultGapWindow); @@ -3531,8 +3531,8 @@ public void testAcquireWhenRecordsFetchedFromGapAndMaxFetchRecordsIsExceeded() { sharePartition.maybeInitialize(); // After initialization is successful, the startOffset can move ahead because the very first batch in cached state // is in a Terminal state (11 -> 20 ACKNOWLEDGED). Thus, start offset will move past it and the only batch remaining - // in cached state will be (31 -> 40) ARCHIVED. This, instead of 20, inFlightTerminalRecords is 10. - assertEquals(10, sharePartition.inFlightTerminalRecords()); + // in cached state will be (31 -> 40) ARCHIVED. This, instead of 20, deliveryCompleteCount is 10. + assertEquals(10, sharePartition.deliveryCompleteCount()); // Creating 3 batches of records with a total of 8 records ByteBuffer buffer = ByteBuffer.allocate(4096); @@ -3584,7 +3584,7 @@ public void testAcquireMaxFetchRecordsExceededAfterAcquiringGaps() { SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister).build(); sharePartition.maybeInitialize(); - assertEquals(10, sharePartition.inFlightTerminalRecords()); + assertEquals(10, sharePartition.deliveryCompleteCount()); // Creating 3 batches of records with a total of 8 records ByteBuffer buffer = ByteBuffer.allocate(4096); @@ -3634,7 +3634,7 @@ public void testAcquireMaxFetchRecordsExceededBeforeAcquiringGaps() { SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister).build(); sharePartition.maybeInitialize(); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Creating 3 batches of records with a total of 8 records ByteBuffer buffer = ByteBuffer.allocate(4096); @@ -3684,7 +3684,7 @@ public void testAcquireWhenRecordsFetchedFromGapAndPartitionContainsNaturalGaps( SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister).build(); sharePartition.maybeInitialize(); - assertEquals(17, sharePartition.inFlightTerminalRecords()); + assertEquals(17, sharePartition.deliveryCompleteCount()); ByteBuffer buffer = ByteBuffer.allocate(4096); memoryRecordsBuilder(buffer, 10, 11).close(); @@ -3734,7 +3734,7 @@ public void testAcquireCachedStateInitialGapMatchesWithActualPartitionGap() { SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister).build(); sharePartition.maybeInitialize(); - assertEquals(20, sharePartition.inFlightTerminalRecords()); + assertEquals(20, sharePartition.deliveryCompleteCount()); // Creating 2 batches starting from 21, such that there is a natural gap from 11 to 20 ByteBuffer buffer = ByteBuffer.allocate(4096); @@ -3779,7 +3779,7 @@ public void testAcquireCachedStateInitialGapOverlapsWithActualPartitionGap() { SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister).build(); sharePartition.maybeInitialize(); - assertEquals(20, sharePartition.inFlightTerminalRecords()); + assertEquals(20, sharePartition.deliveryCompleteCount()); // Creating 2 batches starting from 16, such that there is a natural gap from 11 to 15 ByteBuffer buffer = ByteBuffer.allocate(4096); @@ -3826,7 +3826,7 @@ public void testAcquireCachedStateGapInBetweenOverlapsWithActualPartitionGap() { SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister).build(); sharePartition.maybeInitialize(); - assertEquals(20, sharePartition.inFlightTerminalRecords()); + assertEquals(20, sharePartition.deliveryCompleteCount()); // Creating 3 batches starting from 11, such that there is a natural gap from 26 to 30 ByteBuffer buffer = ByteBuffer.allocate(4096); @@ -3882,7 +3882,7 @@ public void testAcquireWhenRecordsFetchedAfterGapsAreFetched() { SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister).build(); sharePartition.maybeInitialize(); - assertEquals(10, sharePartition.inFlightTerminalRecords()); + assertEquals(10, sharePartition.deliveryCompleteCount()); // Fetched records are from 21 to 35 MemoryRecords records = memoryRecords(21, 15); @@ -3934,7 +3934,7 @@ public void testAcquisitionLockForAcquiringSingleRecord() throws InterruptedExce assertNotNull(sharePartition.cachedState().get(0L).batchAcquisitionLockTimeoutTask()); assertEquals(1, sharePartition.timer().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Allowing acquisition lock to expire. mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); @@ -3949,9 +3949,9 @@ public void testAcquisitionLockForAcquiringSingleRecord() throws InterruptedExce assertEquals(1, sharePartitionMetrics.acquisitionLockTimeoutPerSec().count()); assertTrue(sharePartitionMetrics.acquisitionLockTimeoutPerSec().meanRate() > 0); - // Since the delivery attempts are not exhausted, the inFlightTerminalRecords will still be 0 as the state + // Since the delivery attempts are not exhausted, the deliveryCompleteCount will still be 0 as the state // of the record is AVAILABLE. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -3965,7 +3965,7 @@ public void testAcquisitionLockForAcquiringMultipleRecords() throws InterruptedE assertEquals(1, sharePartition.timer().size()); assertNotNull(sharePartition.cachedState().get(10L).batchAcquisitionLockTimeoutTask()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Allowing acquisition lock to expire. mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); @@ -3980,9 +3980,9 @@ public void testAcquisitionLockForAcquiringMultipleRecords() throws InterruptedE assertEquals(5, sharePartitionMetrics.acquisitionLockTimeoutPerSec().count()); assertTrue(sharePartitionMetrics.acquisitionLockTimeoutPerSec().meanRate() > 0); - // Since the delivery attempts are not exhausted, the inFlightTerminalRecords will still be 0 as the state + // Since the delivery attempts are not exhausted, the deliveryCompleteCount will still be 0 as the state // of the record is AVAILABLE. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -3997,7 +3997,7 @@ public void testAcquisitionLockForAcquiringMultipleRecordsWithOverlapAndNewBatch assertNotNull(sharePartition.cachedState().get(0L).batchAcquisitionLockTimeoutTask()); assertEquals(1, sharePartition.timer().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Add records from 0-9 offsets, 5-9 should be acquired and 0-4 should be ignored. fetchAcquiredRecords(sharePartition, memoryRecords(10), 5); @@ -4020,9 +4020,9 @@ public void testAcquisitionLockForAcquiringMultipleRecordsWithOverlapAndNewBatch assertEquals(10, sharePartitionMetrics.acquisitionLockTimeoutPerSec().count()); assertTrue(sharePartitionMetrics.acquisitionLockTimeoutPerSec().meanRate() > 0); - // Since the delivery attempts are not exhausted, the inFlightTerminalRecords will still be 0 as the state + // Since the delivery attempts are not exhausted, the deliveryCompleteCount will still be 0 as the state // of the record is AVAILABLE. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -4036,7 +4036,7 @@ public void testAcquisitionLockForAcquiringSameBatchAgain() throws InterruptedEx assertNotNull(sharePartition.cachedState().get(10L).batchAcquisitionLockTimeoutTask()); assertEquals(1, sharePartition.timer().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Allowing acquisition lock to expire. mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); @@ -4047,7 +4047,7 @@ public void testAcquisitionLockForAcquiringSameBatchAgain() throws InterruptedEx DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS, () -> assertionFailedMessage(sharePartition, Map.of(10L, List.of()))); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Acquire the same batch again. fetchAcquiredRecords(sharePartition, memoryRecords(10, 5), 5); @@ -4055,7 +4055,7 @@ public void testAcquisitionLockForAcquiringSameBatchAgain() throws InterruptedEx // Acquisition lock timeout task should be created on re-acquire action. assertNotNull(sharePartition.cachedState().get(10L).batchAcquisitionLockTimeoutTask()); assertEquals(1, sharePartition.timer().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -4071,7 +4071,7 @@ public void testAcquisitionLockOnAcknowledgingSingleRecordBatch() throws Interru assertNull(sharePartition.cachedState().get(0L).batchAcquisitionLockTimeoutTask()); assertEquals(0, sharePartition.timer().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); assertEquals(0, sharePartition.nextFetchOffset()); assertEquals(1, sharePartition.cachedState().size()); @@ -4089,7 +4089,7 @@ public void testAcquisitionLockOnAcknowledgingSingleRecordBatch() throws Interru sharePartition.cachedState().get(0L).batchAcquisitionLockTimeoutTask() == null, DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS, () -> assertionFailedMessage(sharePartition, Map.of(0L, List.of()))); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -4108,7 +4108,7 @@ public void testAcquisitionLockOnAcknowledgingMultipleRecordBatch() throws Inter assertEquals(1, sharePartition.cachedState().get(5L).batchDeliveryCount()); assertNull(sharePartition.cachedState().get(5L).batchAcquisitionLockTimeoutTask()); assertEquals(0, sharePartition.timer().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Allowing acquisition lock to expire. This will not cause any change to cached state map since the batch is already acknowledged. // Hence, the acquisition lock timeout task would be cancelled already. @@ -4120,7 +4120,7 @@ public void testAcquisitionLockOnAcknowledgingMultipleRecordBatch() throws Inter sharePartition.cachedState().get(5L).batchAcquisitionLockTimeoutTask() == null, DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS, () -> assertionFailedMessage(sharePartition, Map.of(5L, List.of()))); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -4161,7 +4161,7 @@ public void testAcquisitionLockOnAcknowledgingMultipleRecordBatchWithGapOffsets( assertNotNull(sharePartition.cachedState().get(1L).batchAcquisitionLockTimeoutTask()); assertEquals(1, sharePartition.timer().size()); // All the acquired records except 1 -> 2 have been acknowledged. - assertEquals(11, sharePartition.inFlightTerminalRecords()); + assertEquals(11, sharePartition.deliveryCompleteCount()); // Allowing acquisition lock to expire. The acquisition lock timeout will cause release of records for batch with starting offset 1. // Since, other records have been acknowledged. @@ -4178,7 +4178,7 @@ public void testAcquisitionLockOnAcknowledgingMultipleRecordBatchWithGapOffsets( assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(1L).batchState()); assertEquals(RecordState.ACKNOWLEDGED, sharePartition.cachedState().get(5L).batchState()); assertEquals(RecordState.ACKNOWLEDGED, sharePartition.cachedState().get(10L).batchState()); - assertEquals(11, sharePartition.inFlightTerminalRecords()); + assertEquals(11, sharePartition.deliveryCompleteCount()); } @Test @@ -4192,7 +4192,7 @@ public void testAcquisitionLockForAcquiringSubsetBatchAgain() throws Interrupted assertNotNull(sharePartition.cachedState().get(10L).batchAcquisitionLockTimeoutTask()); assertEquals(1, sharePartition.timer().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Allowing acquisition lock to expire. mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); @@ -4205,7 +4205,7 @@ public void testAcquisitionLockForAcquiringSubsetBatchAgain() throws Interrupted DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS, () -> assertionFailedMessage(sharePartition, Map.of(10L, List.of()))); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Acquire subset of records again. fetchAcquiredRecords(sharePartition, memoryRecords(12, 3), 3); @@ -4249,7 +4249,7 @@ public void testAcquisitionLockForAcquiringSubsetBatchAgain() throws Interrupted assertNull(sharePartition.cachedState().get(10L).offsetState().get(15L).acquisitionLockTimeoutTask()); assertNull(sharePartition.cachedState().get(10L).offsetState().get(16L).acquisitionLockTimeoutTask()); assertNull(sharePartition.cachedState().get(10L).offsetState().get(17L).acquisitionLockTimeoutTask()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -4277,7 +4277,7 @@ public void testAcquisitionLockOnAcknowledgingMultipleSubsetRecordBatchWithGapOf fetchAcquiredRecords(sharePartition, records2, 11); assertNotNull(sharePartition.cachedState().get(10L).batchAcquisitionLockTimeoutTask()); assertEquals(2, sharePartition.timer().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Acknowledging over subset of both batch with subset of gap offsets. sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch( @@ -4303,7 +4303,7 @@ public void testAcquisitionLockOnAcknowledgingMultipleSubsetRecordBatchWithGapOf assertNotNull(sharePartition.cachedState().get(10L).offsetState().get(19L).acquisitionLockTimeoutTask()); assertNotNull(sharePartition.cachedState().get(10L).offsetState().get(20L).acquisitionLockTimeoutTask()); assertEquals(3, sharePartition.timer().size()); - assertEquals(10, sharePartition.inFlightTerminalRecords()); + assertEquals(10, sharePartition.deliveryCompleteCount()); // Allowing acquisition lock to expire for the offsets that have not been acknowledged yet. mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); @@ -4348,7 +4348,7 @@ public void testAcquisitionLockOnAcknowledgingMultipleSubsetRecordBatchWithGapOf assertNull(sharePartition.cachedState().get(10L).offsetState().get(18L).acquisitionLockTimeoutTask()); assertNull(sharePartition.cachedState().get(10L).offsetState().get(19L).acquisitionLockTimeoutTask()); assertNull(sharePartition.cachedState().get(10L).offsetState().get(20L).acquisitionLockTimeoutTask()); - assertEquals(10, sharePartition.inFlightTerminalRecords()); + assertEquals(10, sharePartition.deliveryCompleteCount()); } @Test @@ -4380,7 +4380,7 @@ public void testAcquisitionLockTimeoutCauseMaxDeliveryCountExceed() throws Inter DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS, () -> assertionFailedMessage(sharePartition, Map.of(10L, List.of()))); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); fetchAcquiredRecords(sharePartition, memoryRecords(10, 10), 10); @@ -4401,7 +4401,7 @@ public void testAcquisitionLockTimeoutCauseMaxDeliveryCountExceed() throws Inter DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS, () -> assertionFailedMessage(sharePartition, Map.of(10L, List.of()))); // After the acquisition lock expires for the second time, the records should be archived as the max delivery count is reached. - assertEquals(10, sharePartition.inFlightTerminalRecords()); + assertEquals(10, sharePartition.deliveryCompleteCount()); } @Test @@ -4428,7 +4428,7 @@ public void testAcquisitionLockTimeoutCauseSPSOMoveForward() throws InterruptedE DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS, () -> assertionFailedMessage(sharePartition, Map.of(0L, List.of()))); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); fetchAcquiredRecords(sharePartition, memoryRecords(5), 5); @@ -4479,7 +4479,7 @@ public void testAcquisitionLockTimeoutCauseSPSOMoveForward() throws InterruptedE // Since only first 5 records from the batch are archived, the batch remains in the cachedState, but the // start offset is updated assertEquals(5, sharePartition.startOffset()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -4505,7 +4505,7 @@ public void testAcquisitionLockTimeoutCauseSPSOMoveForwardAndClearCachedState() DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS, () -> assertionFailedMessage(sharePartition, Map.of(0L, List.of()))); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); fetchAcquiredRecords(sharePartition, memoryRecords(10), 10); @@ -4522,7 +4522,7 @@ public void testAcquisitionLockTimeoutCauseSPSOMoveForwardAndClearCachedState() sharePartition.nextFetchOffset() == 10, DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS, () -> assertionFailedMessage(sharePartition, Map.of())); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -4547,7 +4547,7 @@ public void testAcknowledgeAfterAcquisitionLockTimeout() throws InterruptedExcep DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS, () -> assertionFailedMessage(sharePartition, Map.of(5L, List.of()))); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Acknowledge with ACCEPT type should throw InvalidRecordStateException since they've been released due to acquisition lock timeout. CompletableFuture ackResult = sharePartition.acknowledge(MEMBER_ID, @@ -4556,7 +4556,7 @@ public void testAcknowledgeAfterAcquisitionLockTimeout() throws InterruptedExcep assertFutureThrows(InvalidRecordStateException.class, ackResult); assertNull(sharePartition.cachedState().get(5L).batchAcquisitionLockTimeoutTask()); assertEquals(0, sharePartition.timer().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Try acknowledging with REJECT type should throw InvalidRecordStateException since they've been released due to acquisition lock timeout. ackResult = sharePartition.acknowledge(MEMBER_ID, @@ -4565,7 +4565,7 @@ public void testAcknowledgeAfterAcquisitionLockTimeout() throws InterruptedExcep assertFutureThrows(InvalidRecordStateException.class, ackResult); assertNull(sharePartition.cachedState().get(5L).batchAcquisitionLockTimeoutTask()); assertEquals(0, sharePartition.timer().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -4589,7 +4589,7 @@ public void testAcquisitionLockAfterDifferentAcknowledges() throws InterruptedEx assertNotNull(sharePartition.cachedState().get(5L).offsetState().get(8L).acquisitionLockTimeoutTask()); assertNotNull(sharePartition.cachedState().get(5L).offsetState().get(9L).acquisitionLockTimeoutTask()); assertEquals(3, sharePartition.timer().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Acknowledge with ACCEPT type. sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(8, 9, List.of((byte) 1)))); @@ -4600,7 +4600,7 @@ public void testAcquisitionLockAfterDifferentAcknowledges() throws InterruptedEx assertNull(sharePartition.cachedState().get(5L).offsetState().get(8L).acquisitionLockTimeoutTask()); assertNull(sharePartition.cachedState().get(5L).offsetState().get(9L).acquisitionLockTimeoutTask()); assertEquals(1, sharePartition.timer().size()); - assertEquals(2, sharePartition.inFlightTerminalRecords()); + assertEquals(2, sharePartition.deliveryCompleteCount()); // Allowing acquisition lock to expire will only affect the offsets that have not been acknowledged yet. mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); @@ -4625,7 +4625,7 @@ public void testAcquisitionLockAfterDifferentAcknowledges() throws InterruptedEx assertNull(sharePartition.cachedState().get(5L).offsetState().get(7L).acquisitionLockTimeoutTask()); assertNull(sharePartition.cachedState().get(5L).offsetState().get(8L).acquisitionLockTimeoutTask()); assertNull(sharePartition.cachedState().get(5L).offsetState().get(9L).acquisitionLockTimeoutTask()); - assertEquals(2, sharePartition.inFlightTerminalRecords()); + assertEquals(2, sharePartition.deliveryCompleteCount()); } @Test @@ -4662,7 +4662,7 @@ public void testAcquisitionLockOnBatchWithWriteShareGroupStateFailure() throws I } @Test - public void testInFlightTerminalRecordsOnLockExpiryAndWriteFailureOnBatchLastDelivery() throws InterruptedException { + public void testDeliveryCompleteCountOnLockExpiryAndWriteFailureOnBatchLastDelivery() throws InterruptedException { Persister persister = Mockito.mock(Persister.class); mockPersisterReadStateMethod(persister); SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister) @@ -4684,7 +4684,7 @@ public void testInFlightTerminalRecordsOnLockExpiryAndWriteFailureOnBatchLastDel assertEquals(2, sharePartition.timer().size()); assertNotNull(sharePartition.cachedState().get(5L).batchAcquisitionLockTimeoutTask()); assertNotNull(sharePartition.cachedState().get(15L).batchAcquisitionLockTimeoutTask()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Allowing acquisition lock to expire. Even if write share group state RPC fails, state transition still happens. mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); @@ -4696,12 +4696,12 @@ public void testInFlightTerminalRecordsOnLockExpiryAndWriteFailureOnBatchLastDel sharePartition.cachedState().get(15L).batchAcquisitionLockTimeoutTask() == null, DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS, () -> assertionFailedMessage(sharePartition, Map.of(5L, List.of()))); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); fetchAcquiredRecords(sharePartition, memoryRecords(15, 10), 10); assertEquals(1, sharePartition.timer().size()); assertNotNull(sharePartition.cachedState().get(15L).batchAcquisitionLockTimeoutTask()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Mock persister writeState method so that sharePartition.isWriteShareGroupStateSuccessful() returns false. writeShareGroupStateResult = Mockito.mock(WriteShareGroupStateResult.class); @@ -4721,12 +4721,12 @@ public void testInFlightTerminalRecordsOnLockExpiryAndWriteFailureOnBatchLastDel DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS, () -> assertionFailedMessage(sharePartition, Map.of(5L, List.of()))); - // Even though the write state call failed, the records are still archived and inFlightTerminalRecords is updated. - assertEquals(10, sharePartition.inFlightTerminalRecords()); + // Even though the write state call failed, the records are still archived and deliveryCompleteCount is updated. + assertEquals(10, sharePartition.deliveryCompleteCount()); } @Test - public void testInFlightTerminalRecordsOnLockExpiryAndWriteFailureOnOffsetLastDelivery() throws InterruptedException { + public void testDeliveryCompleteCountOnLockExpiryAndWriteFailureOnOffsetLastDelivery() throws InterruptedException { Persister persister = Mockito.mock(Persister.class); mockPersisterReadStateMethod(persister); SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister) @@ -4749,7 +4749,7 @@ public void testInFlightTerminalRecordsOnLockExpiryAndWriteFailureOnOffsetLastDe sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(8, 9, List.of((byte) 1)))); - assertEquals(2, sharePartition.inFlightTerminalRecords()); + assertEquals(2, sharePartition.deliveryCompleteCount()); // Mock persister writeState method so that sharePartition.isWriteShareGroupStateSuccessful() returns false. Mockito.when(writeShareGroupStateResult.topicsData()).thenReturn(List.of( @@ -4780,7 +4780,7 @@ public void testInFlightTerminalRecordsOnLockExpiryAndWriteFailureOnOffsetLastDe assertNull(sharePartition.cachedState().get(5L).offsetState().get(8L).acquisitionLockTimeoutTask()); assertNull(sharePartition.cachedState().get(5L).offsetState().get(9L).acquisitionLockTimeoutTask()); assertNull(sharePartition.cachedState().get(5L).offsetState().get(10L).acquisitionLockTimeoutTask()); - assertEquals(2, sharePartition.inFlightTerminalRecords()); + assertEquals(2, sharePartition.deliveryCompleteCount()); fetchAcquiredRecords(sharePartition, memoryRecords(10, 1), 1); @@ -4810,8 +4810,8 @@ public void testInFlightTerminalRecordsOnLockExpiryAndWriteFailureOnOffsetLastDe DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS, () -> assertionFailedMessage(sharePartition, Map.of(5L, List.of(5L, 6L, 7L, 10L)))); - // Even though the write state call failed, the record is still archived and inFlightTerminalRecords is updated. - assertEquals(3, sharePartition.inFlightTerminalRecords()); + // Even though the write state call failed, the record is still archived and deliveryCompleteCount is updated. + assertEquals(3, sharePartition.deliveryCompleteCount()); } @Test @@ -4873,7 +4873,7 @@ public void testReleaseSingleRecordBatch() { SharePartition sharePartition = SharePartitionBuilder.builder().withState(SharePartitionState.ACTIVE).build(); fetchAcquiredRecords(sharePartition, memoryRecords(1), 1); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); CompletableFuture releaseResult = sharePartition.releaseAcquiredRecords(MEMBER_ID); assertNull(releaseResult.join()); @@ -4885,7 +4885,7 @@ public void testReleaseSingleRecordBatch() { // Release delivery count. assertEquals(0, sharePartition.cachedState().get(0L).batchDeliveryCount()); assertNull(sharePartition.cachedState().get(0L).offsetState()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -4893,7 +4893,7 @@ public void testReleaseMultipleRecordBatch() { SharePartition sharePartition = SharePartitionBuilder.builder().withState(SharePartitionState.ACTIVE).build(); fetchAcquiredRecords(sharePartition, memoryRecords(5, 10), 10); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); CompletableFuture releaseResult = sharePartition.releaseAcquiredRecords(MEMBER_ID); assertNull(releaseResult.join()); @@ -4904,7 +4904,7 @@ public void testReleaseMultipleRecordBatch() { assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(5L).batchState()); assertEquals(0, sharePartition.cachedState().get(5L).batchDeliveryCount()); assertNull(sharePartition.cachedState().get(5L).offsetState()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -4923,7 +4923,7 @@ public void testReleaseMultipleAcknowledgedRecordBatch() { // After the acknowledgements, the cached state has 11 Terminal records -> // (5 -> 6) // (10 -> 18) - assertEquals(11, sharePartition.inFlightTerminalRecords()); + assertEquals(11, sharePartition.deliveryCompleteCount()); CompletableFuture releaseResult = sharePartition.releaseAcquiredRecords(MEMBER_ID); assertNull(releaseResult.join()); @@ -4934,7 +4934,7 @@ public void testReleaseMultipleAcknowledgedRecordBatch() { assertEquals(RecordState.ACKNOWLEDGED, sharePartition.cachedState().get(10L).batchState()); assertNull(sharePartition.cachedState().get(5L).offsetState()); assertNull(sharePartition.cachedState().get(10L).offsetState()); - assertEquals(11, sharePartition.inFlightTerminalRecords()); + assertEquals(11, sharePartition.deliveryCompleteCount()); } @Test @@ -4966,7 +4966,7 @@ public void testReleaseAcknowledgedMultipleSubsetRecordBatch() { // After the acknowledgements, the cached state has 10 Terminal records -> // 6 // (10 -> 18) - assertEquals(10, sharePartition.inFlightTerminalRecords()); + assertEquals(10, sharePartition.deliveryCompleteCount()); CompletableFuture releaseResult = sharePartition.releaseAcquiredRecords(MEMBER_ID); assertNull(releaseResult.join()); @@ -4992,7 +4992,7 @@ public void testReleaseAcknowledgedMultipleSubsetRecordBatch() { expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); - assertEquals(10, sharePartition.inFlightTerminalRecords()); + assertEquals(10, sharePartition.deliveryCompleteCount()); } @Test @@ -5020,7 +5020,7 @@ public void testReleaseAcquiredRecordsWithAnotherMember() { // After the acknowledgements, the cached state has 9 Terminal records -> // (10 -> 18) - assertEquals(9, sharePartition.inFlightTerminalRecords()); + assertEquals(9, sharePartition.deliveryCompleteCount()); // Release acquired records for "member-1". CompletableFuture releaseResult = sharePartition.releaseAcquiredRecords(MEMBER_ID); @@ -5044,7 +5044,7 @@ public void testReleaseAcquiredRecordsWithAnotherMember() { expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); - assertEquals(9, sharePartition.inFlightTerminalRecords()); + assertEquals(9, sharePartition.deliveryCompleteCount()); // Release acquired records for "member-2". releaseResult = sharePartition.releaseAcquiredRecords("member-2"); @@ -5068,7 +5068,7 @@ public void testReleaseAcquiredRecordsWithAnotherMember() { expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); - assertEquals(9, sharePartition.inFlightTerminalRecords()); + assertEquals(9, sharePartition.deliveryCompleteCount()); } @Test @@ -5096,7 +5096,7 @@ public void testReleaseAcquiredRecordsWithAnotherMemberAndSubsetAcknowledged() { // After the acknowledgements, the cached state has 9 Terminal records -> // (10 -> 18) - assertEquals(9, sharePartition.inFlightTerminalRecords()); + assertEquals(9, sharePartition.deliveryCompleteCount()); // Release acquired records for "member-1". CompletableFuture releaseResult = sharePartition.releaseAcquiredRecords(MEMBER_ID); @@ -5120,14 +5120,14 @@ public void testReleaseAcquiredRecordsWithAnotherMemberAndSubsetAcknowledged() { expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); - assertEquals(9, sharePartition.inFlightTerminalRecords()); + assertEquals(9, sharePartition.deliveryCompleteCount()); // Ack subset of records by "member-2". sharePartition.acknowledge("member-2", List.of(new ShareAcknowledgementBatch(5, 5, List.of((byte) 1)))); // After the acknowledgements, the startOffset will be upadated to 6, since offset 5 is Terminal. Hence - // inFlightTerminalRecords will remain 9. - assertEquals(9, sharePartition.inFlightTerminalRecords()); + // deliveryCompleteCount will remain 9. + assertEquals(9, sharePartition.deliveryCompleteCount()); // Release acquired records for "member-2". releaseResult = sharePartition.releaseAcquiredRecords("member-2"); @@ -5153,7 +5153,7 @@ public void testReleaseAcquiredRecordsWithAnotherMemberAndSubsetAcknowledged() { expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); - assertEquals(9, sharePartition.inFlightTerminalRecords()); + assertEquals(9, sharePartition.deliveryCompleteCount()); } @Test @@ -5178,7 +5178,7 @@ public void testReleaseAcquiredRecordsAfterDifferentAcknowledges() { sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(8, 9, List.of((byte) 1)))); - assertEquals(2, sharePartition.inFlightTerminalRecords()); + assertEquals(2, sharePartition.deliveryCompleteCount()); CompletableFuture releaseResult = sharePartition.releaseAcquiredRecords(MEMBER_ID); assertNull(releaseResult.join()); @@ -5192,7 +5192,7 @@ public void testReleaseAcquiredRecordsAfterDifferentAcknowledges() { expectedOffsetStateMap.put(8L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(9L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(5L).offsetState()); - assertEquals(2, sharePartition.inFlightTerminalRecords()); + assertEquals(2, sharePartition.deliveryCompleteCount()); } @Test @@ -5209,7 +5209,7 @@ public void testMaxDeliveryCountLimitNotExceededForRecordsSubsetAfterReleaseAcqu sharePartition.acknowledge(MEMBER_ID, List.of( new ShareAcknowledgementBatch(10, 14, List.of((byte) 2)))); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); fetchAcquiredRecords(sharePartition, records2, 5); @@ -5221,7 +5221,7 @@ public void testMaxDeliveryCountLimitNotExceededForRecordsSubsetAfterReleaseAcqu assertEquals(2, sharePartition.cachedState().size()); assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(10L).batchState()); assertNull(sharePartition.cachedState().get(10L).offsetState()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -5247,7 +5247,7 @@ public void testMaxDeliveryCountLimitNotExceededForRecordsSubsetAfterReleaseAcqu new ShareAcknowledgementBatch(20, 24, List.of((byte) 2)) ))); - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); // Send next batch from offset 13, only 2 records should be acquired. fetchAcquiredRecords(sharePartition, records1, 2); @@ -5285,7 +5285,7 @@ public void testMaxDeliveryCountLimitNotExceededForRecordsSubsetAfterReleaseAcqu expectedOffsetStateMap.put(18L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(19L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(15L).offsetState()); - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); } @Test @@ -5314,7 +5314,7 @@ public void testMaxDeliveryCountLimitExceededForRecordsSubsetCacheCleared() { // After acknowledgements, since offsets 10 -> 12 are at the start of the caches state and are in Terminal state, // the start offset will be updated to 13. From the remaining offstes in flight, only records (17 -> 19) are in Terminal state. - assertEquals(3, sharePartition.inFlightTerminalRecords()); + assertEquals(3, sharePartition.deliveryCompleteCount()); // Send next batch from offset 13, only 2 records should be acquired. fetchAcquiredRecords(sharePartition, records1, 2); @@ -5329,7 +5329,7 @@ public void testMaxDeliveryCountLimitExceededForRecordsSubsetCacheCleared() { assertEquals(25, sharePartition.nextFetchOffset()); assertEquals(0, sharePartition.cachedState().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -5341,7 +5341,7 @@ public void testReleaseAcquiredRecordsSubsetWithAnotherMember() { sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(5, 7, List.of((byte) 1)))); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Release acquired records subset with another member. CompletableFuture releaseResult = sharePartition.releaseAcquiredRecords("member-2"); @@ -5357,7 +5357,7 @@ public void testReleaseAcquiredRecordsSubsetWithAnotherMember() { expectedOffsetStateMap.put(10L, new InFlightState(RecordState.ACQUIRED, (short) 1, MEMBER_ID)); expectedOffsetStateMap.put(11L, new InFlightState(RecordState.ACQUIRED, (short) 1, MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(5L).offsetState()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -5386,7 +5386,7 @@ public void testReleaseBatchWithWriteShareGroupStateFailure() { assertEquals(1, sharePartition.cachedState().size()); assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(5L).batchState()); assertEquals(MEMBER_ID, sharePartition.cachedState().get(5L).batchMemberId()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -5410,7 +5410,7 @@ public void testReleaseOffsetWithWriteShareGroupStateFailure() { sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(8, 9, List.of((byte) 1)))); - assertEquals(2, sharePartition.inFlightTerminalRecords()); + assertEquals(2, sharePartition.deliveryCompleteCount()); // Mock persister writeState method so that sharePartition.isWriteShareGroupStateSuccessful() returns false. Mockito.when(writeShareGroupStateResult.topicsData()).thenReturn(List.of( @@ -5437,7 +5437,7 @@ public void testReleaseOffsetWithWriteShareGroupStateFailure() { assertEquals(EMPTY_MEMBER_ID, sharePartition.cachedState().get(5L).offsetState().get(8L).memberId()); assertEquals(EMPTY_MEMBER_ID, sharePartition.cachedState().get(5L).offsetState().get(9L).memberId()); assertEquals(MEMBER_ID, sharePartition.cachedState().get(5L).offsetState().get(10L).memberId()); - assertEquals(2, sharePartition.inFlightTerminalRecords()); + assertEquals(2, sharePartition.deliveryCompleteCount()); } @Test @@ -5458,7 +5458,7 @@ public void testAcquisitionLockOnReleasingMultipleRecordBatch() { // Acquisition lock timer task would be cancelled by the release acquired records operation. assertNull(sharePartition.cachedState().get(5L).batchAcquisitionLockTimeoutTask()); assertEquals(0, sharePartition.timer().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -5489,7 +5489,7 @@ public void testAcquisitionLockOnReleasingAcknowledgedMultipleSubsetRecordBatchW (byte) 0, (byte) 1, (byte) 0, (byte) 1)))); - assertEquals(10, sharePartition.inFlightTerminalRecords()); + assertEquals(10, sharePartition.deliveryCompleteCount()); CompletableFuture releaseResult = sharePartition.releaseAcquiredRecords(MEMBER_ID); assertNull(releaseResult.join()); @@ -5533,11 +5533,11 @@ public void testAcquisitionLockOnReleasingAcknowledgedMultipleSubsetRecordBatchW assertNull(sharePartition.cachedState().get(10L).offsetState().get(20L).acquisitionLockTimeoutTask()); assertEquals(0, sharePartition.timer().size()); - assertEquals(10, sharePartition.inFlightTerminalRecords()); + assertEquals(10, sharePartition.deliveryCompleteCount()); } @Test - public void testInFlightTerminalRecordsWhenStaleBatchesAreArchived() { + public void testDeliveryCompleteCountWhenStaleBatchesAreArchived() { Persister persister = Mockito.mock(Persister.class); ReadShareGroupStateResult readShareGroupStateResult = Mockito.mock(ReadShareGroupStateResult.class); Mockito.when(readShareGroupStateResult.topicsData()).thenReturn(List.of( @@ -5553,7 +5553,7 @@ public void testInFlightTerminalRecordsWhenStaleBatchesAreArchived() { sharePartition.maybeInitialize(); assertEquals(11, sharePartition.nextFetchOffset()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Fetched records are from 21 to 30 MemoryRecords records = memoryRecords(21, 10); @@ -5574,12 +5574,12 @@ public void testInFlightTerminalRecordsWhenStaleBatchesAreArchived() { assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(11L).batchState()); assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(21L).batchState()); - // Since the records 11 -> 20 are ARCHIVED, inFlightTerminalRecords will be 10. - assertEquals(10, sharePartition.inFlightTerminalRecords()); + // Since the records 11 -> 20 are ARCHIVED, deliveryCompleteCount will be 10. + assertEquals(10, sharePartition.deliveryCompleteCount()); } @Test - public void testInFlightTerminalRecordsWhenStaleOffsetsAreArchived() { + public void testDeliveryCompleteCountWhenStaleOffsetsAreArchived() { Persister persister = Mockito.mock(Persister.class); ReadShareGroupStateResult readShareGroupStateResult = Mockito.mock(ReadShareGroupStateResult.class); Mockito.when(readShareGroupStateResult.topicsData()).thenReturn(List.of( @@ -5595,7 +5595,7 @@ public void testInFlightTerminalRecordsWhenStaleOffsetsAreArchived() { sharePartition.maybeInitialize(); assertEquals(11, sharePartition.nextFetchOffset()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Fetched records are from 21 to 30 MemoryRecords records = memoryRecords(16, 15); @@ -5626,8 +5626,8 @@ public void testInFlightTerminalRecordsWhenStaleOffsetsAreArchived() { assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(11L).offsetState().get(15L).state()); assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(21L).batchState()); - // Since the records 11 -> 15 are ARCHIVED, inFlightTerminalRecords will be 5. - assertEquals(5, sharePartition.inFlightTerminalRecords()); + // Since the records 11 -> 15 are ARCHIVED, deliveryCompleteCount will be 5. + assertEquals(5, sharePartition.deliveryCompleteCount()); } @Test @@ -5668,7 +5668,7 @@ public void testLsoMovementForArchivingBatches() { // After the acknowledgements, the records in Terminal state are -> // 27 -> 31: ARCHIVED // Records 2 -> 6 are ACKNOWLEDGED, but since they are at the start of the cache, the start offset will be moved to 7. - assertEquals(5, sharePartition.inFlightTerminalRecords()); + assertEquals(5, sharePartition.deliveryCompleteCount()); // LSO is at 20. sharePartition.updateCacheAndOffsets(20); @@ -5704,8 +5704,8 @@ public void testLsoMovementForArchivingBatches() { assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(32L).batchState()); assertNotNull(sharePartition.cachedState().get(32L).batchAcquisitionLockTimeoutTask()); // After the LSO is moved, AVAILABLE batches are ARCHIVED. Thus, the records 12 -> 16 will be ARCHIVED. Since - // these are prior to the new startOffset, inFlightTerminalRecords remains the same. - assertEquals(5, sharePartition.inFlightTerminalRecords()); + // these are prior to the new startOffset, deliveryCompleteCount remains the same. + assertEquals(5, sharePartition.deliveryCompleteCount()); } @Test @@ -5731,7 +5731,7 @@ public void testLsoMovementPostArchivedBatches() { // After the acknowledgements, the records in Terminal state are -> // 12 -> 16: ARCHIVED // 27 -> 31: ARCHIVED - assertEquals(10, sharePartition.inFlightTerminalRecords()); + assertEquals(10, sharePartition.deliveryCompleteCount()); // LSO is at 20. sharePartition.updateCacheAndOffsets(20); @@ -5770,8 +5770,8 @@ public void testLsoMovementPostArchivedBatches() { assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(32L).batchState()); assertNotNull(sharePartition.cachedState().get(32L).batchAcquisitionLockTimeoutTask()); // After the LSO is moved, the number of Terminal records between old and new start offsets is calculated. - // In this case it is 5 (for the records 12 -> 16). Thus, inFlightTerminalRecords is decremented by 5. - assertEquals(5, sharePartition.inFlightTerminalRecords()); + // In this case it is 5 (for the records 12 -> 16). Thus, deliveryCompleteCount is decremented by 5. + assertEquals(5, sharePartition.deliveryCompleteCount()); } @Test @@ -5799,7 +5799,7 @@ public void testLsoMovementPostArchivedRecords() { // 12 -> 16: ARCHIVED // 19 -> 21: ARCHIVED // 27 -> 31: ARCHIVED - assertEquals(13, sharePartition.inFlightTerminalRecords()); + assertEquals(13, sharePartition.deliveryCompleteCount()); // LSO is at 20. sharePartition.updateCacheAndOffsets(20); @@ -5840,8 +5840,8 @@ public void testLsoMovementPostArchivedRecords() { assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(32L).batchState()); assertNotNull(sharePartition.cachedState().get(32L).batchAcquisitionLockTimeoutTask()); // After the LSO is moved, the number of Terminal records between old and new start offsets is calculated. - // In this case it is 6 (for the records 12 -> 16 and 19). Thus, inFlightTerminalRecords is decremented by 6. - assertEquals(7, sharePartition.inFlightTerminalRecords()); + // In this case it is 6 (for the records 12 -> 16 and 19). Thus, deliveryCompleteCount is decremented by 6. + assertEquals(7, sharePartition.deliveryCompleteCount()); } @Test @@ -5864,7 +5864,7 @@ public void testLsoMovementForArchivingAllAvailableBatches() { new ShareAcknowledgementBatch(31, 40, List.of((byte) 2)) )); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Move the LSO to 41. When the LSO moves ahead, all batches that are AVAILABLE before the new LSO will be ARCHIVED. // Thus, the state of the share partition will be: @@ -5886,13 +5886,13 @@ public void testLsoMovementForArchivingAllAvailableBatches() { assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(31L).batchState()); assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(41L).batchState()); // There are no records in flight in Terminal state. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // The client acknowledges the batch 21 -> 30. Since this batch is before the LSO, nothing will be done and these // records will remain in the ACQUIRED state. sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(21L, 30L, List.of((byte) 2)))); // The acknowledgements make no difference to in flight records. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // The batch is still in ACQUIRED state. assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(21L).batchState()); @@ -5902,7 +5902,7 @@ public void testLsoMovementForArchivingAllAvailableBatches() { sharePartition.cachedState().get(21L).batchAcquisitionLockTimeoutTask().run(); assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(21L).batchState()); // Even when the acquisition lock expires, this happens for records before the LSO, hence in flight terminal records remain 0. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -5925,7 +5925,7 @@ public void testLsoMovementForArchivingAllAvailableOffsets() { new ShareAcknowledgementBatch(31, 40, List.of((byte) 2)) )); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Move the LSO to 36. When the LSO moves ahead, all records that are AVAILABLE before the new LSO will be ARCHIVED. // Thus, the state of the share partition will be: @@ -5956,22 +5956,22 @@ public void testLsoMovementForArchivingAllAvailableOffsets() { assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(31L).offsetState().get(39L).state()); assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(31L).offsetState().get(40L).state()); assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(41L).batchState()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // The client acknowledges the batch 21 -> 30. Since this batch is before the LSO, nothing will be done and these // records will remain in the ACQUIRED state. sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(21L, 30L, List.of((byte) 2)))); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // The batch is still in ACQUIRED state. assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(21L).batchState()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Once the acquisition lock timer task for the batch 21 -> 30 is expired, these records will directly be // ARCHIVED. sharePartition.cachedState().get(21L).batchAcquisitionLockTimeoutTask().run(); assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(21L).batchState()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -6429,7 +6429,7 @@ public void testTerminalRecordsNotUpdatedWhenBatchesBeforeStartOffsetAreExpired( sharePartition.maybeInitialize(); assertEquals(11, sharePartition.nextFetchOffset()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Fetched records are from 11 to 20 MemoryRecords records = memoryRecords(10, 11); @@ -6452,8 +6452,8 @@ public void testTerminalRecordsNotUpdatedWhenBatchesBeforeStartOffsetAreExpired( sharePartition.updateCacheAndOffsets(21); // After the LSO is moved to 21, all the records after new Start offset are in non-Terminal states. Thus, - // inFlightTerminalRecords is not changed. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // deliveryCompleteCount is not changed. + assertEquals(0, sharePartition.deliveryCompleteCount()); assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(11L).batchState()); assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(21L).batchState()); @@ -6461,8 +6461,8 @@ public void testTerminalRecordsNotUpdatedWhenBatchesBeforeStartOffsetAreExpired( // will simply be ignored. sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(11L, 20L, List.of((byte) 2)))); - // Since the acknowledgements are ignored, the inFlightTerminalRecords should not change. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // Since the acknowledgements are ignored, the deliveryCompleteCount should not change. + assertEquals(0, sharePartition.deliveryCompleteCount()); assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(11L).batchState()); assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(21L).batchState()); @@ -6475,8 +6475,8 @@ public void testTerminalRecordsNotUpdatedWhenBatchesBeforeStartOffsetAreExpired( // moved to the ARCHIVED state. assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(11L).batchState()); assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(21L).batchState()); - // Since these records are present before the start offset, the inFlightTerminalRecords should not change. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // Since these records are present before the start offset, the deliveryCompleteCount should not change. + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -6496,7 +6496,7 @@ public void testTerminalRecordsNotUpdatedWhenOffsetsBeforeStartOffsetAreExpired( sharePartition.maybeInitialize(); assertEquals(11, sharePartition.nextFetchOffset()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Fetched records are from 11 to 20 MemoryRecords records = memoryRecords(10, 11); @@ -6519,8 +6519,8 @@ public void testTerminalRecordsNotUpdatedWhenOffsetsBeforeStartOffsetAreExpired( sharePartition.updateCacheAndOffsets(16); // After the LSO is moved to 21, all the records after new Start offset are in non-Terminal states. Thus, - // inFlightTerminalRecords is not changed. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // deliveryCompleteCount is not changed. + assertEquals(0, sharePartition.deliveryCompleteCount()); assertEquals(16, sharePartition.startOffset()); assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(11L).batchState()); assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(21L).batchState()); @@ -6530,8 +6530,8 @@ public void testTerminalRecordsNotUpdatedWhenOffsetsBeforeStartOffsetAreExpired( // Expiring the acquisition lock timer task of the ACQUIRED batch. sharePartition.cachedState().get(11L).batchAcquisitionLockTimeoutTask().run(); - // Since these records are present before the start offset, the inFlightTerminalRecords should not change. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // Since these records are present before the start offset, the deliveryCompleteCount should not change. + assertEquals(0, sharePartition.deliveryCompleteCount()); assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(11L).offsetState().get(11L).state()); assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(11L).offsetState().get(12L).state()); assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(11L).offsetState().get(13L).state()); @@ -6562,7 +6562,7 @@ public void testTerminalRecordsNotUpdatedWhenOffsetsBeforeStartOffsetAreExpiredA sharePartition.maybeInitialize(); assertEquals(11, sharePartition.nextFetchOffset()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Fetched records are from 11 to 20 MemoryRecords records = memoryRecords(10, 11); @@ -6585,7 +6585,7 @@ public void testTerminalRecordsNotUpdatedWhenOffsetsBeforeStartOffsetAreExpiredA sharePartition.updateCacheAndOffsets(16); // There are no Terminal records between start offset and end offset. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); assertEquals(16, sharePartition.startOffset()); assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(11L).batchState()); assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(21L).batchState()); @@ -6600,7 +6600,7 @@ public void testTerminalRecordsNotUpdatedWhenOffsetsBeforeStartOffsetAreExpiredA // In this case, records 11 -> 15 will remain in the ACQUIRED state, while records 16 -> 20 will be RELEASED. sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(11L, 20L, List.of((byte) 2)))); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(11L).offsetState().get(11L).state()); assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(11L).offsetState().get(12L).state()); assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(11L).offsetState().get(13L).state()); @@ -6622,8 +6622,8 @@ public void testTerminalRecordsNotUpdatedWhenOffsetsBeforeStartOffsetAreExpiredA sharePartition.cachedState().get(11L).offsetState().get(14L).acquisitionLockTimeoutTask().run(); sharePartition.cachedState().get(11L).offsetState().get(15L).acquisitionLockTimeoutTask().run(); - // Since these records are present before the start offset, the inFlightTerminalRecords should not change. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // Since these records are present before the start offset, the deliveryCompleteCount should not change. + assertEquals(0, sharePartition.deliveryCompleteCount()); assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(11L).offsetState().get(11L).state()); assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(11L).offsetState().get(12L).state()); assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(11L).offsetState().get(13L).state()); @@ -6651,7 +6651,7 @@ public void testReleaseAcquiredRecordsBatchesPostStartOffsetMovement() { fetchAcquiredRecords(sharePartition, memoryRecords(30, 5), 5); fetchAcquiredRecords(sharePartition, memoryRecords(35, 5), 5); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Acknowledge records. sharePartition.acknowledge(MEMBER_ID, List.of( @@ -6661,7 +6661,7 @@ public void testReleaseAcquiredRecordsBatchesPostStartOffsetMovement() { new ShareAcknowledgementBatch(35, 37, List.of((byte) 2)) )); - assertEquals(2, sharePartition.inFlightTerminalRecords()); + assertEquals(2, sharePartition.deliveryCompleteCount()); // LSO is at 24. sharePartition.updateCacheAndOffsets(24); @@ -6670,7 +6670,7 @@ public void testReleaseAcquiredRecordsBatchesPostStartOffsetMovement() { assertEquals(24, sharePartition.startOffset()); assertEquals(39, sharePartition.endOffset()); assertEquals(7, sharePartition.cachedState().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Release acquired records for MEMBER_ID. CompletableFuture releaseResult = sharePartition.releaseAcquiredRecords(MEMBER_ID); @@ -6700,7 +6700,7 @@ public void testReleaseAcquiredRecordsBatchesPostStartOffsetMovement() { expectedOffsetStateMap.put(23L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(24L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(20L).offsetState()); @@ -6733,7 +6733,7 @@ public void testReleaseAcquiredRecordsBatchesPostStartOffsetMovementToStartOfBat assertEquals(10, sharePartition.startOffset()); assertEquals(14, sharePartition.endOffset()); assertEquals(2, sharePartition.cachedState().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Release acquired records. CompletableFuture releaseResult = sharePartition.releaseAcquiredRecords(MEMBER_ID); @@ -6746,8 +6746,8 @@ public void testReleaseAcquiredRecordsBatchesPostStartOffsetMovementToStartOfBat assertEquals(EMPTY_MEMBER_ID, sharePartition.cachedState().get(10L).batchMemberId()); assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(10L).batchState()); - // The records after the start offset are in non-Terminal states. Thus, inFlightTerminalRecords is not changed. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // The records after the start offset are in non-Terminal states. Thus, deliveryCompleteCount is not changed. + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -6765,7 +6765,7 @@ public void testReleaseAcquiredRecordsBatchesPostStartOffsetMovementToMiddleOfBa assertEquals(14, sharePartition.endOffset()); assertEquals(2, sharePartition.cachedState().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Release acquired records. CompletableFuture releaseResult = sharePartition.releaseAcquiredRecords(MEMBER_ID); @@ -6784,8 +6784,8 @@ public void testReleaseAcquiredRecordsBatchesPostStartOffsetMovementToMiddleOfBa assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); - // The records after the start offset are in non-Terminal states. Thus, inFlightTerminalRecords is not changed. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // The records after the start offset are in non-Terminal states. Thus, deliveryCompleteCount is not changed. + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -6798,7 +6798,7 @@ public void testReleaseAcquiredRecordsDecreaseDeliveryCount() { sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(12, 13, List.of((byte) 1)))); // Records 12 and 13 are ACKNOWLEDGED. - assertEquals(2, sharePartition.inFlightTerminalRecords()); + assertEquals(2, sharePartition.deliveryCompleteCount()); // LSO is at 11. sharePartition.updateCacheAndOffsets(11); @@ -6807,7 +6807,7 @@ public void testReleaseAcquiredRecordsDecreaseDeliveryCount() { assertEquals(11, sharePartition.startOffset()); assertEquals(14, sharePartition.endOffset()); assertEquals(2, sharePartition.cachedState().size()); - assertEquals(2, sharePartition.inFlightTerminalRecords()); + assertEquals(2, sharePartition.deliveryCompleteCount()); // Before release, the delivery count was incremented. Map expectedOffsetStateMap = new HashMap<>(); @@ -6836,7 +6836,7 @@ public void testReleaseAcquiredRecordsDecreaseDeliveryCount() { expectedOffsetStateMap.put(13L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(14L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); - assertEquals(2, sharePartition.inFlightTerminalRecords()); + assertEquals(2, sharePartition.deliveryCompleteCount()); } @Test @@ -6864,7 +6864,7 @@ public void testAcquisitionLockTimeoutForBatchesPostStartOffsetMovement() throws new ShareAcknowledgementBatch(35, 37, List.of((byte) 2)) )); - assertEquals(2, sharePartition.inFlightTerminalRecords()); + assertEquals(2, sharePartition.deliveryCompleteCount()); // LSO is at 24. sharePartition.updateCacheAndOffsets(24); @@ -6873,7 +6873,7 @@ public void testAcquisitionLockTimeoutForBatchesPostStartOffsetMovement() throws assertEquals(24, sharePartition.startOffset()); assertEquals(39, sharePartition.endOffset()); assertEquals(7, sharePartition.cachedState().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Allowing acquisition lock to expire. mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); @@ -6915,7 +6915,7 @@ public void testAcquisitionLockTimeoutForBatchesPostStartOffsetMovement() throws assertEquals(EMPTY_MEMBER_ID, sharePartition.cachedState().get(15L).batchMemberId()); assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(15L).batchState()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -6935,7 +6935,7 @@ public void testAcquisitionLockTimeoutForBatchesPostStartOffsetMovementToStartOf assertEquals(10, sharePartition.startOffset()); assertEquals(14, sharePartition.endOffset()); assertEquals(2, sharePartition.cachedState().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Allowing acquisition lock to expire. mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); @@ -6947,8 +6947,8 @@ public void testAcquisitionLockTimeoutForBatchesPostStartOffsetMovementToStartOf DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS, () -> assertionFailedMessage(sharePartition, Map.of(5L, List.of(), 10L, List.of()))); - // All records after startOffset are in non-Terminal states. Thus, inFlightTerminalRecords is not changed. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // All records after startOffset are in non-Terminal states. Thus, deliveryCompleteCount is not changed. + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -6968,7 +6968,7 @@ public void testAcquisitionLockTimeoutForBatchesPostStartOffsetMovementToMiddleO assertEquals(11, sharePartition.startOffset()); assertEquals(14, sharePartition.endOffset()); assertEquals(2, sharePartition.cachedState().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Allowing acquisition lock to expire. mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); @@ -6987,8 +6987,8 @@ public void testAcquisitionLockTimeoutForBatchesPostStartOffsetMovementToMiddleO DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS, () -> assertionFailedMessage(sharePartition, Map.of(5L, List.of(), 10L, List.of(10L, 11L, 12L, 13L, 14L)))); - // All records after startOffset are in non-Terminal states. Thus, inFlightTerminalRecords is not changed. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // All records after startOffset are in non-Terminal states. Thus, deliveryCompleteCount is not changed. + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -7050,7 +7050,7 @@ public void testAcknowledgeBatchAndOffsetPostLsoMovement() { assertEquals(12, sharePartition.startOffset()); assertEquals(14, sharePartition.endOffset()); assertEquals(2, sharePartition.cachedState().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Check cached state map. assertEquals(MEMBER_ID, sharePartition.cachedState().get(2L).batchMemberId()); @@ -7066,8 +7066,8 @@ public void testAcknowledgeBatchAndOffsetPostLsoMovement() { assertNull(ackResult.join()); assertFalse(ackResult.isCompletedExceptionally()); - // No record is moved to Terminal state, thus inFlightTerminalRecords is not changed. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // No record is moved to Terminal state, thus deliveryCompleteCount is not changed. + assertEquals(0, sharePartition.deliveryCompleteCount()); assertEquals(12, sharePartition.nextFetchOffset()); assertEquals(12, sharePartition.startOffset()); @@ -7094,7 +7094,7 @@ public void testAcknowledgeBatchAndOffsetPostLsoMovement() { assertNull(sharePartition.cachedState().get(10L).offsetState().get(13L).acquisitionLockTimeoutTask()); assertNull(sharePartition.cachedState().get(10L).offsetState().get(14L).acquisitionLockTimeoutTask()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -7120,15 +7120,15 @@ public void testAcknowledgeBatchPostLsoMovement() { assertEquals(MEMBER_ID, sharePartition.cachedState().get(20L).batchMemberId()); assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(20L).batchState()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Acknowledge with ACCEPT action. CompletableFuture ackResult = sharePartition.acknowledge(MEMBER_ID, List.of( new ShareAcknowledgementBatch(2, 14, List.of((byte) 1)))); assertNull(ackResult.join()); assertFalse(ackResult.isCompletedExceptionally()); - // Only record 14 is post startOffset and in a Terminal state. Thus, only that is considered for inFlightTerminalRecords. - assertEquals(1, sharePartition.inFlightTerminalRecords()); + // Only record 14 is post startOffset and in a Terminal state. Thus, only that is considered for deliveryCompleteCount. + assertEquals(1, sharePartition.deliveryCompleteCount()); assertEquals(25, sharePartition.nextFetchOffset()); // For cached state corresponding to entry 2, the offset states will be ARCHIVED, ARCHIVED, ARCHIVED, ARCHIVED and ACKNOWLEDGED. @@ -7173,7 +7173,7 @@ public void testLsoMovementThenAcquisitionLockTimeoutThenAcknowledge() throws In assertEquals(MEMBER_ID, sharePartition.cachedState().get(2L).batchMemberId()); assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(2L).batchState()); assertNotNull(sharePartition.cachedState().get(2L).batchAcquisitionLockTimeoutTask()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Allowing acquisition lock to expire. mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); @@ -7182,7 +7182,7 @@ public void testLsoMovementThenAcquisitionLockTimeoutThenAcknowledge() throws In sharePartition.startOffset() == 7 && sharePartition.endOffset() == 7, DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS, () -> assertionFailedMessage(sharePartition, Map.of())); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); fetchAcquiredRecords(sharePartition, memoryRecords(10, 5), 5); @@ -7190,8 +7190,8 @@ public void testLsoMovementThenAcquisitionLockTimeoutThenAcknowledge() throws In assertEquals(10, sharePartition.startOffset()); assertEquals(14, sharePartition.endOffset()); assertEquals(1, sharePartition.cachedState().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Acknowledge with RELEASE action. This contains a batch that doesn't exist at all. sharePartition.acknowledge(MEMBER_ID, List.of( @@ -7205,7 +7205,7 @@ public void testLsoMovementThenAcquisitionLockTimeoutThenAcknowledge() throws In assertEquals(EMPTY_MEMBER_ID, sharePartition.cachedState().get(10L).batchMemberId()); assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(10L).batchState()); assertNull(sharePartition.cachedState().get(10L).batchAcquisitionLockTimeoutTask()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -7223,7 +7223,7 @@ public void testLsoMovementThenAcquisitionLockTimeoutThenAcknowledgeBatchLastOff assertEquals(3, sharePartition.startOffset()); assertEquals(3, sharePartition.endOffset()); assertEquals(1, sharePartition.cachedState().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Check cached state map. assertEquals(MEMBER_ID, sharePartition.cachedState().get(1L).batchMemberId()); @@ -7238,7 +7238,7 @@ public void testLsoMovementThenAcquisitionLockTimeoutThenAcknowledgeBatchLastOff DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS, () -> assertionFailedMessage(sharePartition, Map.of())); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); fetchAcquiredRecords(sharePartition, memoryRecords(3, 2), 2); fetchAcquiredRecords(sharePartition, memoryRecords(5, 3), 3); @@ -7247,7 +7247,7 @@ public void testLsoMovementThenAcquisitionLockTimeoutThenAcknowledgeBatchLastOff assertEquals(3, sharePartition.startOffset()); assertEquals(7, sharePartition.endOffset()); assertEquals(2, sharePartition.cachedState().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Acknowledge with RELEASE action. This contains a batch that doesn't exist at all. sharePartition.acknowledge(MEMBER_ID, List.of( @@ -7265,7 +7265,7 @@ public void testLsoMovementThenAcquisitionLockTimeoutThenAcknowledgeBatchLastOff assertEquals(EMPTY_MEMBER_ID, sharePartition.cachedState().get(5L).batchMemberId()); assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(5L).batchState()); assertNull(sharePartition.cachedState().get(5L).batchAcquisitionLockTimeoutTask()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -7511,7 +7511,7 @@ public void testMaybeUpdateCachedStateWhenAcknowledgeTypeAccept() { assertTrue(sharePartition.canAcquireRecords()); // The records have been accepted, thus they are removed from the cached state. assertEquals(0, sharePartition.cachedState().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -7532,7 +7532,7 @@ public void testMaybeUpdateCachedStateWhenAcknowledgeTypeReject() { assertTrue(sharePartition.canAcquireRecords()); // The records have been rejected, thus they are removed from the cached state. assertEquals(0, sharePartition.cachedState().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -7554,7 +7554,7 @@ public void testMaybeUpdateCachedStateWhenAcknowledgeTypeRelease() { assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(0L).batchState()); assertEquals(EMPTY_MEMBER_ID, sharePartition.cachedState().get(0L).batchMemberId()); assertEquals(1, sharePartition.cachedState().get(0L).batchDeliveryCount()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -7581,7 +7581,7 @@ public void testMaybeUpdateCachedStateWhenAcknowledgementsFromBeginningForBatchS assertEquals(13, sharePartition.startOffset()); assertEquals(29, sharePartition.endOffset()); assertEquals(30, sharePartition.nextFetchOffset()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -7607,7 +7607,7 @@ public void testMaybeUpdateCachedStateWhenAcknowledgementsFromBeginningForEntire assertEquals(15, sharePartition.startOffset()); assertEquals(29, sharePartition.endOffset()); assertEquals(30, sharePartition.nextFetchOffset()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -7637,8 +7637,8 @@ public void testMaybeUpdateCachedStateWhenAcknowledgementsInBetween() { assertEquals(0, sharePartition.startOffset()); assertEquals(29, sharePartition.endOffset()); assertEquals(30, sharePartition.nextFetchOffset()); - // Records 10 -> 14 are in ARCHIVED state, and so inFlightTerminalRecords is 5. - assertEquals(5, sharePartition.inFlightTerminalRecords()); + // Records 10 -> 14 are in ARCHIVED state, and so deliveryCompleteCount is 5. + assertEquals(5, sharePartition.deliveryCompleteCount()); } @Test @@ -7662,7 +7662,7 @@ public void testMaybeUpdateCachedStateWhenAllRecordsInCachedStateAreAcknowledged assertEquals(30, sharePartition.endOffset()); assertEquals(30, sharePartition.nextFetchOffset()); // Cache state is empty. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -7689,7 +7689,7 @@ public void testMaybeUpdateCachedStateMultipleAcquisitionsAndAcknowledgements() assertEquals(20, sharePartition.startOffset()); assertEquals(59, sharePartition.endOffset()); assertEquals(60, sharePartition.nextFetchOffset()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); fetchAcquiredRecords(sharePartition, memoryRecords(60, 20), 20); assertTrue(sharePartition.canAcquireRecords()); @@ -7704,7 +7704,7 @@ public void testMaybeUpdateCachedStateMultipleAcquisitionsAndAcknowledgements() assertEquals(50, sharePartition.startOffset()); assertEquals(79, sharePartition.endOffset()); assertEquals(80, sharePartition.nextFetchOffset()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); fetchAcquiredRecords(sharePartition, memoryRecords(80, 100), 100); assertFalse(sharePartition.canAcquireRecords()); @@ -7718,7 +7718,7 @@ public void testMaybeUpdateCachedStateMultipleAcquisitionsAndAcknowledgements() assertEquals(180, sharePartition.startOffset()); assertEquals(180, sharePartition.endOffset()); assertEquals(180, sharePartition.nextFetchOffset()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); fetchAcquiredRecords(sharePartition, memoryRecords(180, 20), 20); @@ -7728,7 +7728,7 @@ public void testMaybeUpdateCachedStateMultipleAcquisitionsAndAcknowledgements() assertEquals(180, sharePartition.startOffset()); assertEquals(199, sharePartition.endOffset()); assertEquals(200, sharePartition.nextFetchOffset()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -7753,7 +7753,7 @@ public void testMaybeUpdateCachedStateGapAfterLastOffsetAcknowledged() { SharePartition sharePartition = SharePartitionBuilder.builder().withPersister(persister).build(); sharePartition.maybeInitialize(); - assertEquals(10, sharePartition.inFlightTerminalRecords()); + assertEquals(10, sharePartition.deliveryCompleteCount()); // Acquiring the first AVAILABLE batch from 11 to 20 fetchAcquiredRecords(sharePartition, memoryRecords(11, 10), 10); @@ -7771,7 +7771,7 @@ public void testMaybeUpdateCachedStateGapAfterLastOffsetAcknowledged() { assertEquals(21, sharePartition.startOffset()); assertEquals(40, sharePartition.endOffset()); assertEquals(21, sharePartition.nextFetchOffset()); - assertEquals(10, sharePartition.inFlightTerminalRecords()); + assertEquals(10, sharePartition.deliveryCompleteCount()); GapWindow persisterReadResultGapWindow = sharePartition.persisterReadResultGapWindow(); assertNotNull(persisterReadResultGapWindow); @@ -8163,7 +8163,7 @@ public void testNextFetchOffsetWithMultipleConsumers() { sharePartition.acknowledge(memberId1, List.of( new ShareAcknowledgementBatch(0, 2, List.of((byte) 2)))); assertEquals(0, sharePartition.nextFetchOffset()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); sharePartition.acquire(memberId2, BATCH_SIZE, MAX_FETCH_RECORDS, 3, fetchPartitionData(memoryRecords(3, 2)), FETCH_ISOLATION_HWM); assertEquals(0, sharePartition.nextFetchOffset()); @@ -8174,7 +8174,7 @@ public void testNextFetchOffsetWithMultipleConsumers() { sharePartition.acknowledge(memberId2, List.of( new ShareAcknowledgementBatch(3, 4, List.of((byte) 2)))); assertEquals(3, sharePartition.nextFetchOffset()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -8210,7 +8210,7 @@ public void testReacquireSubsetWithAnotherMember() { new ShareAcknowledgementBatch(17, 20, List.of((byte) 2)))); // Records 12-13 have been identified as gaps, hence they are kept in the cache as ARCHIVED state. - assertEquals(2, sharePartition.inFlightTerminalRecords()); + assertEquals(2, sharePartition.deliveryCompleteCount()); // Reacquire with another member. sharePartition.acquire("member-2", BATCH_SIZE, MAX_FETCH_RECORDS, 5, fetchPartitionData(records1), FETCH_ISOLATION_HWM); @@ -8270,7 +8270,7 @@ public void testMaybeInitializeWhenReadStateRpcReturnsZeroAvailableRecords() { assertEquals(734, sharePartition.nextFetchOffset()); assertEquals(734, sharePartition.startOffset()); assertEquals(734, sharePartition.endOffset()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -8507,7 +8507,7 @@ public void testAcquireAndAcknowledgeWithRecordsAheadOfRecordBatchStartOffset() assertNotNull(sharePartition.cachedState().get(5L).offsetState()); // after acknowledgements, the start offset moves to 15, and thus there are no Terminal records post that. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Check cached state. Map expectedOffsetStateMap = new HashMap<>(); @@ -8557,7 +8557,7 @@ public void testAcquireWhenBatchesAreRemovedFromBetweenInSubsequentFetchData() { sharePartition.releaseAcquiredRecords(MEMBER_ID); // Validate cache has 4 entries. assertEquals(4, sharePartition.cachedState().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Compact all batches and remove some of the batches from the fetch response. buffer = ByteBuffer.allocate(4096); @@ -8618,7 +8618,7 @@ public void testAcquireWhenBatchesAreRemovedFromBetweenInSubsequentFetchData() { } }); // All in flight records are in a non-Terminal state. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } /** @@ -8644,7 +8644,7 @@ public void testAcquireWhenBatchesRemovedForFetchOffset() { // Validate cache has 3 entries. assertEquals(3, sharePartition.cachedState().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Compact second batch and remove first batch from the fetch response. ByteBuffer buffer = ByteBuffer.allocate(4096); @@ -8676,7 +8676,7 @@ public void testAcquireWhenBatchesRemovedForFetchOffset() { assertEquals(RecordState.AVAILABLE, inFlightState.batchState()); }); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } /** @@ -8700,14 +8700,14 @@ public void testAcquireWhenBatchesRemovedForFetchOffsetWithinBatch() { // Accept the 3 offsets of first batch. new ShareAcknowledgementBatch(5, 7, List.of(AcknowledgeType.ACCEPT.id)))).join(); - // After acknowledgements, the start offset moves past Terminal records, hence inFlightTerminalRecords is 0. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // After acknowledgements, the start offset moves past Terminal records, hence deliveryCompleteCount is 0. + assertEquals(0, sharePartition.deliveryCompleteCount()); // Release the remaining batches/offsets in the cache. sharePartition.releaseAcquiredRecords(MEMBER_ID).join(); // Validate cache has 2 entries. assertEquals(2, sharePartition.cachedState().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Mark fetch offset within the first batch to 8, first available offset. fetchAcquiredRecords(sharePartition, memoryRecords(10, 15), 8, 0, 15); @@ -8718,7 +8718,7 @@ public void testAcquireWhenBatchesRemovedForFetchOffsetWithinBatch() { assertEquals(8, sharePartition.startOffset()); // Since the fetchOffset in the acquire request was prior to the actual records fetched, the records 8 and 9 are marked // as ARCHIVED. Thus, there are 2 Terminal records in the cache. - assertEquals(2, sharePartition.inFlightTerminalRecords()); + assertEquals(2, sharePartition.deliveryCompleteCount()); // Releasing acquired records updates the cache and moves the start offset. sharePartition.releaseAcquiredRecords(MEMBER_ID); @@ -8728,7 +8728,7 @@ public void testAcquireWhenBatchesRemovedForFetchOffsetWithinBatch() { assertEquals(1, sharePartition.cachedState().size()); assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(10L).batchState()); // Since the start offset has moved past all Terminal records, the count is 0. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } /** @@ -8758,7 +8758,7 @@ public void testAcquireWhenBatchesRemovedForFetchOffsetForSameCachedBatch() { sharePartition.releaseAcquiredRecords(MEMBER_ID); // Validate cache has 1 entry. assertEquals(1, sharePartition.cachedState().size()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Compact second batch and remove first batch from the fetch response. buffer = ByteBuffer.allocate(4096); @@ -8778,7 +8778,7 @@ public void testAcquireWhenBatchesRemovedForFetchOffsetForSameCachedBatch() { // the acquire operation only marks offsets as archived. The start offset will be correctly // updated once any records are acknowledged. assertEquals(0, sharePartition.startOffset()); - assertEquals(5, sharePartition.inFlightTerminalRecords()); + assertEquals(5, sharePartition.deliveryCompleteCount()); // Releasing acquired records updates the cache and moves the start offset. sharePartition.releaseAcquiredRecords(MEMBER_ID); @@ -8792,7 +8792,7 @@ public void testAcquireWhenBatchesRemovedForFetchOffsetForSameCachedBatch() { assertEquals(recordState, offsetState.state()); }); }); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } private String assertionFailedMessage(SharePartition sharePartition, Map> offsets) { @@ -9241,8 +9241,8 @@ public void testTerminalRecordsUpdatedWhenAbortedTransactionBatchesAreArchived() assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(19L).batchState()); assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(20L).batchState()); assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(29L).batchState()); - // Records 10 -> 18 are ARCHIVED, hence inFlightTerminalRecords should be 9. - assertEquals(9, sharePartition.inFlightTerminalRecords()); + // Records 10 -> 18 are ARCHIVED, hence deliveryCompleteCount should be 9. + assertEquals(9, sharePartition.deliveryCompleteCount()); } @Test @@ -9294,8 +9294,8 @@ public void testTerminalRecordsUpdatedWhenAbortedTransactionOffsetsAreArchived() assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(0L).offsetState().get(6L).state()); assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(0L).offsetState().get(7L).state()); assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(0L).offsetState().get(8L).state()); - // Records 3 -> 4 are ARCHIVED, hence inFlightTerminalRecords should be 2. - assertEquals(2, sharePartition.inFlightTerminalRecords()); + // Records 3 -> 4 are ARCHIVED, hence deliveryCompleteCount should be 2. + assertEquals(2, sharePartition.deliveryCompleteCount()); } @Test @@ -9341,8 +9341,8 @@ public void testAcquireWhenBatchHasOngoingTransition() { // Acknowledge batch to create ongoing transition. sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(21, 30, List.of(AcknowledgeType.RELEASE.id)))); - // Since the future is not yet completed, inFlightTerminalRecords will not be updated yet. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // Since the future is not yet completed, deliveryCompleteCount will not be updated yet. + assertEquals(0, sharePartition.deliveryCompleteCount()); // Assert the start offset has not moved and batch has ongoing transition. assertEquals(21L, sharePartition.startOffset()); @@ -9359,8 +9359,8 @@ public void testAcquireWhenBatchHasOngoingTransition() { ), 0 ); - // Since no new records are acquired, inFlightTerminalRecords will remain the same. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // Since no new records are acquired, deliveryCompleteCount will remain the same. + assertEquals(0, sharePartition.deliveryCompleteCount()); assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(21L).batchState()); assertEquals(EMPTY_MEMBER_ID, sharePartition.cachedState().get(21L).batchMemberId()); @@ -9372,8 +9372,8 @@ public void testAcquireWhenBatchHasOngoingTransition() { PartitionFactory.newPartitionErrorData(0, Errors.NONE.code(), Errors.NONE.message()))))); future.complete(writeShareGroupStateResult); - // Since the records successfully acknowledged are moved to AVAILABLE state, inFlightTerminalRecords will still not change. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // Since the records successfully acknowledged are moved to AVAILABLE state, deliveryCompleteCount will still not change. + assertEquals(0, sharePartition.deliveryCompleteCount()); // Acquire the same batch with member-2. 10 records will be acquired. fetchAcquiredRecords( @@ -9383,7 +9383,7 @@ public void testAcquireWhenBatchHasOngoingTransition() { ); assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(21L).batchState()); assertEquals("member-2", sharePartition.cachedState().get(21L).batchMemberId()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -9428,8 +9428,8 @@ public void testNextFetchOffsetWhenBatchHasOngoingTransition() { sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(0, 9, List.of(AcknowledgeType.RELEASE.id)))); sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(10, 19, List.of(AcknowledgeType.RELEASE.id)))); - // inFlightTerminalRecords will not be updated, because the acknowledgment type is RELEASE. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // deliveryCompleteCount will not be updated, because the acknowledgment type is RELEASE. + assertEquals(0, sharePartition.deliveryCompleteCount()); // Complete future2 so second acknowledge API can be completed, which updates the cache. WriteShareGroupStateResult writeShareGroupStateResult = Mockito.mock(WriteShareGroupStateResult.class); @@ -9438,8 +9438,8 @@ public void testNextFetchOffsetWhenBatchHasOngoingTransition() { PartitionFactory.newPartitionErrorData(0, Errors.NONE.code(), Errors.NONE.message()))))); future2.complete(writeShareGroupStateResult); - // Since the records successfully acknowledged are moved to AVAILABLE state, inFlightTerminalRecords will still not change. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // Since the records successfully acknowledged are moved to AVAILABLE state, deliveryCompleteCount will still not change. + assertEquals(0, sharePartition.deliveryCompleteCount()); // Offsets 0-9 will have ongoing state transition since future1 is not complete yet. // Offsets 10-19 won't have ongoing state transition since future2 has been completed. @@ -9481,8 +9481,8 @@ public void testNextFetchOffsetWhenOffsetsHaveOngoingTransition() { sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(5, 9, List.of(AcknowledgeType.RELEASE.id)))); sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(20, 24, List.of(AcknowledgeType.RELEASE.id)))); - // inFlightTerminalRecords will not be updated, because the acknowledgment type is RELEASE. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // deliveryCompleteCount will not be updated, because the acknowledgment type is RELEASE. + assertEquals(0, sharePartition.deliveryCompleteCount()); // Complete future2 so second acknowledge API can be completed, which updates the cache. WriteShareGroupStateResult writeShareGroupStateResult = Mockito.mock(WriteShareGroupStateResult.class); @@ -9491,8 +9491,8 @@ public void testNextFetchOffsetWhenOffsetsHaveOngoingTransition() { PartitionFactory.newPartitionErrorData(0, Errors.NONE.code(), Errors.NONE.message()))))); future2.complete(writeShareGroupStateResult); - // Since the records successfully acknowledged are moved to AVAILABLE state, inFlightTerminalRecords will still not change. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // Since the records successfully acknowledged are moved to AVAILABLE state, deliveryCompleteCount will still not change. + assertEquals(0, sharePartition.deliveryCompleteCount()); // Offsets 5-9 will have ongoing state transition since future1 is not complete yet. // Offsets 20-24 won't have ongoing state transition since future2 has been completed. @@ -9570,8 +9570,8 @@ public void testAcquisitionLockTimeoutWithConcurrentAcknowledgement() throws Int assertFalse(sharePartition.cachedState().get(0L).offsetState().get(2L).hasOngoingStateTransition()); assertTrue(sharePartition.cachedState().get(5L).batchHasOngoingStateTransition()); - // Records 1 and 5 -> 19 are acked with ACKNOWLEDGE type, thus inFlightTerminalRecords will account for these. - assertEquals(16, sharePartition.inFlightTerminalRecords()); + // Records 1 and 5 -> 19 are acked with ACKNOWLEDGE type, thus deliveryCompleteCount will account for these. + assertEquals(16, sharePartition.deliveryCompleteCount()); // Validate first timer task is already cancelled. assertTrue(timerTask1.isCancelled()); @@ -9591,8 +9591,8 @@ public void testAcquisitionLockTimeoutWithConcurrentAcknowledgement() throws Int future2.complete(writeShareGroupStateResult); // Now that the futures are completed, offsets 1 and 5 -> 19 are all committed to the final ACKNOWLEDGED state. - // The inFlightTerminalRecords will remain same as before the future is completed. - assertEquals(16, sharePartition.inFlightTerminalRecords()); + // The deliveryCompleteCount will remain same as before the future is completed. + assertEquals(16, sharePartition.deliveryCompleteCount()); // Verify timer tasks are now cancelled, except unacknowledged offsets. assertEquals(2, sharePartition.cachedState().size()); @@ -9622,7 +9622,7 @@ public void testAcquisitionLockTimeoutWithConcurrentAcknowledgement() throws Int timerTaskOffsetState3.run(); assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(0L).offsetState().get(2L).state()); - assertEquals(16, sharePartition.inFlightTerminalRecords()); + assertEquals(16, sharePartition.deliveryCompleteCount()); } @Test @@ -9655,8 +9655,8 @@ public void testLsoMovementWithWriteStateRPCFailuresInAcknowledgement() { // Validate that there is no ongoing transition. assertTrue(sharePartition.cachedState().get(2L).batchHasOngoingStateTransition()); assertTrue(sharePartition.cachedState().get(7L).batchHasOngoingStateTransition()); - // Records 7 -> 11 are acked with ACCEPT type, thus inFlightTerminalRecords will account for these. - assertEquals(5, sharePartition.inFlightTerminalRecords()); + // Records 7 -> 11 are acked with ACCEPT type, thus deliveryCompleteCount will account for these. + assertEquals(5, sharePartition.deliveryCompleteCount()); // Move LSO to 7, so some records/offsets can be marked archived for the first batch. sharePartition.updateCacheAndOffsets(7L); @@ -9671,7 +9671,7 @@ public void testLsoMovementWithWriteStateRPCFailuresInAcknowledgement() { assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(2L).batchState()); assertEquals(RecordState.ACKNOWLEDGED, sharePartition.cachedState().get(7L).batchState()); - assertEquals(5, sharePartition.inFlightTerminalRecords()); + assertEquals(5, sharePartition.deliveryCompleteCount()); // Complete future1 exceptionally so acknowledgement for 2-6 offsets will be completed. WriteShareGroupStateResult writeShareGroupStateResult = Mockito.mock(WriteShareGroupStateResult.class); @@ -9690,7 +9690,7 @@ public void testLsoMovementWithWriteStateRPCFailuresInAcknowledgement() { assertTrue(sharePartition.cachedState().get(7L).batchHasOngoingStateTransition()); assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(2L).batchState()); assertEquals(RecordState.ACKNOWLEDGED, sharePartition.cachedState().get(7L).batchState()); - assertEquals(5, sharePartition.inFlightTerminalRecords()); + assertEquals(5, sharePartition.deliveryCompleteCount()); future2.complete(writeShareGroupStateResult); assertEquals(12L, sharePartition.nextFetchOffset()); @@ -9699,9 +9699,9 @@ public void testLsoMovementWithWriteStateRPCFailuresInAcknowledgement() { assertEquals(2, sharePartition.cachedState().size()); assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(2L).batchState()); assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(7L).batchState()); - // After the write RPC failure, the record states are rolled back and inFlightTerminalRecords is calculated - // from scratch. Since there is no Terminal record now in flight, inFlightTerminalRecords becomes 0. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // After the write RPC failure, the record states are rolled back and deliveryCompleteCount is calculated + // from scratch. Since there is no Terminal record now in flight, deliveryCompleteCount becomes 0. + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -9737,8 +9737,8 @@ public void testAcquisitionLockTimeoutWithWriteStateRPCFailure() throws Interrup assertEquals(EMPTY_MEMBER_ID, sharePartition.cachedState().get(0L).batchMemberId()); // Timer task has not been expired yet. assertFalse(sharePartition.cachedState().get(0L).batchAcquisitionLockTimeoutTask().hasExpired()); - // Record are acked with ACKNOWLEDGED type, thus inFlightTerminalRecords will account for these. - assertEquals(2, sharePartition.inFlightTerminalRecords()); + // Record are acked with ACKNOWLEDGED type, thus deliveryCompleteCount will account for these. + assertEquals(2, sharePartition.deliveryCompleteCount()); // Allowing acquisition lock to expire. This will not cause any change because the record is not in ACQUIRED state. // This will remove the entry of the timer task from timer. @@ -9755,7 +9755,7 @@ public void testAcquisitionLockTimeoutWithWriteStateRPCFailure() throws Interrup // Timer task should be expired now. assertTrue(sharePartition.cachedState().get(0L).batchAcquisitionLockTimeoutTask().hasExpired()); - assertEquals(2, sharePartition.inFlightTerminalRecords()); + assertEquals(2, sharePartition.deliveryCompleteCount()); // Complete future exceptionally so acknowledgement for 0-1 offsets will be completed. WriteShareGroupStateResult writeShareGroupStateResult = Mockito.mock(WriteShareGroupStateResult.class); @@ -9770,9 +9770,9 @@ public void testAcquisitionLockTimeoutWithWriteStateRPCFailure() throws Interrup assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(0L).batchState()); assertEquals(EMPTY_MEMBER_ID, sharePartition.cachedState().get(0L).batchMemberId()); assertNull(sharePartition.cachedState().get(0L).batchAcquisitionLockTimeoutTask()); - // After the write RPC failure, the record states are rolled back and inFlightTerminalRecords is calculated - // from scratch. Since there is no Terminal record now in flight, inFlightTerminalRecords becomes 0. - assertEquals(0, sharePartition.inFlightTerminalRecords()); + // After the write RPC failure, the record states are rolled back and deliveryCompleteCount is calculated + // from scratch. Since there is no Terminal record now in flight, deliveryCompleteCount becomes 0. + assertEquals(0, sharePartition.deliveryCompleteCount()); } @Test @@ -9802,8 +9802,8 @@ public void testRecordArchivedWithWriteStateRPCFailure() throws InterruptedExcep assertEquals(RecordState.ACKNOWLEDGED, sharePartition.cachedState().get(7L).batchState()); assertEquals(1, sharePartition.cachedState().get(7L).batchDeliveryCount()); - // Records 3 and 7 -> 11 are acked with ACKNOWLEDGE type, thus inFlightTerminalRecords will account for these. - assertEquals(6, sharePartition.inFlightTerminalRecords()); + // Records 3 and 7 -> 11 are acked with ACKNOWLEDGE type, thus deliveryCompleteCount will account for these. + assertEquals(6, sharePartition.deliveryCompleteCount()); WriteShareGroupStateResult writeShareGroupStateResult = Mockito.mock(WriteShareGroupStateResult.class); Mockito.when(writeShareGroupStateResult.topicsData()).thenReturn(List.of( @@ -9816,7 +9816,7 @@ public void testRecordArchivedWithWriteStateRPCFailure() throws InterruptedExcep assertEquals(1, sharePartition.cachedState().get(2L).offsetState().get(3L).deliveryCount()); assertEquals(RecordState.ACKNOWLEDGED, sharePartition.cachedState().get(7L).batchState()); assertEquals(1, sharePartition.cachedState().get(7L).batchDeliveryCount()); - assertEquals(5, sharePartition.inFlightTerminalRecords()); + assertEquals(5, sharePartition.deliveryCompleteCount()); future2.complete(writeShareGroupStateResult); assertEquals(12L, sharePartition.nextFetchOffset()); @@ -9825,7 +9825,7 @@ public void testRecordArchivedWithWriteStateRPCFailure() throws InterruptedExcep assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(7L).batchState()); assertEquals(1, sharePartition.cachedState().get(7L).batchDeliveryCount()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Allowing acquisition lock to expire. This will also ensure that acquisition lock timeout task // is run successfully post write state RPC failure. @@ -9840,7 +9840,7 @@ public void testRecordArchivedWithWriteStateRPCFailure() throws InterruptedExcep () -> assertionFailedMessage(sharePartition, Map.of(2L, List.of(3L), 7L, List.of()))); // Acquisition lock timeout task has run already and next fetch offset is moved to 2. assertEquals(2, sharePartition.nextFetchOffset()); - assertEquals(0, sharePartition.inFlightTerminalRecords()); + assertEquals(0, sharePartition.deliveryCompleteCount()); // Send the same batches again. fetchAcquiredRecords(sharePartition, memoryRecords(2, 5), 5); @@ -9853,7 +9853,7 @@ public void testRecordArchivedWithWriteStateRPCFailure() throws InterruptedExcep sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(3, 3, List.of(AcknowledgeType.ACCEPT.id)))); sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(7, 11, List.of(AcknowledgeType.ACCEPT.id)))); - assertEquals(6, sharePartition.inFlightTerminalRecords()); + assertEquals(6, sharePartition.deliveryCompleteCount()); mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); // Verify the timer tasks have run and the state is archived for the offsets which are not acknowledged, @@ -9868,9 +9868,9 @@ public void testRecordArchivedWithWriteStateRPCFailure() throws InterruptedExcep DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS, () -> assertionFailedMessage(sharePartition, Map.of(2L, List.of(3L), 7L, List.of()))); - // After the acquisition lock timeout task has expired, records 2, 4 -> 6 are archived, and thus inFlightTerminalRecords + // After the acquisition lock timeout task has expired, records 2, 4 -> 6 are archived, and thus deliveryCompleteCount // increases by 4. - assertEquals(10, sharePartition.inFlightTerminalRecords()); + assertEquals(10, sharePartition.deliveryCompleteCount()); future1.complete(writeShareGroupStateResult); // Now the state should be archived for the offsets despite the write state RPC failure, as the @@ -9887,8 +9887,8 @@ public void testRecordArchivedWithWriteStateRPCFailure() throws InterruptedExcep assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(7L).batchState()); assertEquals(2, sharePartition.cachedState().get(7L).batchDeliveryCount()); // At this point, the batch 2 -> 6 is removed from the cached state and startOffset is moved to 7. Thus, in flight - // contains records 7 -> 11 which are archived. Therefore, inFlightTerminalRecords becomes 5. - assertEquals(5, sharePartition.inFlightTerminalRecords()); + // contains records 7 -> 11 which are archived. Therefore, deliveryCompleteCount becomes 5. + assertEquals(5, sharePartition.deliveryCompleteCount()); } /** diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 472054a5e5175..01daa19d7a01d 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -13853,7 +13853,7 @@ class KafkaApisTest extends Logging { def getWriteShareGroupStateResponse(requestData: WriteShareGroupStateRequestData, configOverrides: Map[String, String] = Map.empty, verifyNoErr: Boolean = true, authorizer: Authorizer = null, writeStateResult: util.List[WriteShareGroupStateResponseData.WriteStateResult]): WriteShareGroupStateResponse = { - val requestChannelRequest = buildRequest(new WriteShareGroupStateRequest.Builder(requestData).build(0)) + val requestChannelRequest = buildRequest(new WriteShareGroupStateRequest.Builder(requestData).build()) val future = new CompletableFuture[WriteShareGroupStateResponseData]() when(shareCoordinator.writeState( diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareGroupOffset.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareGroupOffset.java index e2f274209b320..9f1a264425701 100644 --- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareGroupOffset.java +++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareGroupOffset.java @@ -160,8 +160,8 @@ public static ShareGroupOffset fromRequest(InitializeShareGroupStateRequestData. } public static ShareGroupOffset fromRequest(InitializeShareGroupStateRequestData.PartitionData data, int snapshotEpoch, long timestamp) { - // Since initialization changes the start offset, and hence the in flight state is forgotten, the end offset is set - // to be the same as the start offset, and the in flight record count is set to 0. + // This method is invoked during InitializeShareGroupStateRequest. Since the deliveryCompleteCount is not yet + // known at this stage, it is initialized to its default value. return new ShareGroupOffset( snapshotEpoch, data.stateEpoch(), diff --git a/share-coordinator/src/main/resources/common/message/ShareSnapshotValue.json b/share-coordinator/src/main/resources/common/message/ShareSnapshotValue.json index 6eff1564b7b08..b6e70bee542f6 100644 --- a/share-coordinator/src/main/resources/common/message/ShareSnapshotValue.json +++ b/share-coordinator/src/main/resources/common/message/ShareSnapshotValue.json @@ -28,6 +28,7 @@ "about": "The leader epoch of the share-partition." }, { "name": "StartOffset", "type": "int64", "versions": "0+", "about": "The share-partition start offset." }, + // DeliveryCompleteCount was added in Apache Kafka 4.2 (KIP-1226). { "name": "DeliveryCompleteCount", "type": "int32", "versions": "0+", "taggedVersions": "0+", "tag": 0, "default": "-1", "about": "The number of offsets greater than or equal to share-partition start offset for which delivery has been completed."}, { "name": "CreateTimestamp", "type": "int64", "versions": "0+", diff --git a/share-coordinator/src/main/resources/common/message/ShareUpdateValue.json b/share-coordinator/src/main/resources/common/message/ShareUpdateValue.json index 3955520022c1a..0714d7d346fff 100644 --- a/share-coordinator/src/main/resources/common/message/ShareUpdateValue.json +++ b/share-coordinator/src/main/resources/common/message/ShareUpdateValue.json @@ -26,6 +26,7 @@ "about": "The leader epoch of the share-partition." }, { "name": "StartOffset", "type": "int64", "versions": "0+", "about": "The share-partition start offset, or -1 if the start offset is not being updated." }, + // DeliveryCompleteCount was added in Apache Kafka 4.2 (KIP-1226). { "name": "DeliveryCompleteCount", "type": "int32", "versions": "0+", "taggedVersions": "0+", "tag": 0, "default": "-1", "about": "The number of offsets greater than or equal to share-partition start offset for which delivery has been completed."}, { "name": "StateBatches", "type": "[]StateBatch", "versions": "0+",