From 4b96c797b6e2ca469aba78d23b39466c5bebe946 Mon Sep 17 00:00:00 2001 From: Sean Quah Date: Mon, 27 Oct 2025 21:18:24 +0000 Subject: [PATCH 01/10] KAFKA-19826: Implement adaptive append linger in the coordinator runtime --- .../common/runtime/CoordinatorRuntime.java | 88 ++++++++++++++----- 1 file changed, 68 insertions(+), 20 deletions(-) diff --git a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java index 552e7e72effd8..a4864e9b8134b 100644 --- a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java +++ b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java @@ -117,7 +117,7 @@ public static class Builder, U> { private CoordinatorMetrics coordinatorMetrics; private Serializer serializer; private Compression compression; - private int appendLingerMs; + private OptionalInt appendLingerMs; private ExecutorService executorService; public Builder withLogPrefix(String logPrefix) { @@ -186,6 +186,11 @@ public Builder withCompression(Compression compression) { } public Builder withAppendLingerMs(int appendLingerMs) { + this.appendLingerMs = OptionalInt.of(appendLingerMs); + return this; + } + + public Builder withAppendLingerMs(OptionalInt appendLingerMs) { this.appendLingerMs = appendLingerMs; return this; } @@ -195,6 +200,7 @@ public Builder withExecutorService(ExecutorService executorService) { return this; } + @SuppressWarnings("checkstyle:CyclomaticComplexity") public CoordinatorRuntime build() { if (logPrefix == null) logPrefix = ""; @@ -220,8 +226,10 @@ public CoordinatorRuntime build() { throw new IllegalArgumentException("Serializer must be set."); if (compression == null) compression = Compression.NONE; - if (appendLingerMs < 0) - throw new IllegalArgumentException("AppendLinger must be >= 0"); + if (appendLingerMs == null) + appendLingerMs = OptionalInt.of(0); + if (appendLingerMs.isPresent() && appendLingerMs.getAsInt() < -1) + throw new IllegalArgumentException("AppendLinger must be empty or >= 0"); if (executorService == null) throw new IllegalArgumentException("ExecutorService must be set."); @@ -599,6 +607,12 @@ class CoordinatorContext { */ CoordinatorBatch currentBatch; + /** + * The batch epoch. Incremented every time a new batch is started. + * Only valid for the lifetime of the CoordinatorContext. The first batch has an epoch of 1. + */ + int batchEpoch; + /** * Constructor. * @@ -769,6 +783,24 @@ private void freeCurrentBatch() { currentBatch = null; } + /** + * Adds a flush event to the end of the event queue, after any existing writes in the queue. + * + * @param expectedBatchEpoch The epoch of the batch to flush. + */ + private void enqueueAdaptiveFlush(int expectedBatchEpoch) { + enqueueLast(new CoordinatorInternalEvent("FlushBatch", tp, () -> { + withActiveContextOrThrow(tp, context -> { + // The batch could have already been flushed because it reached the maximum + // batch size or a transactional write came in. When this happens, we want + // to avoid flushing the next batch early. + if (context.currentBatch != null && context.batchEpoch == expectedBatchEpoch) { + context.flushCurrentBatch(); + } + }); + })); + } + /** * Flushes the current (or pending) batch to the log. When the batch is written * locally, a new snapshot is created in the snapshot registry and the events @@ -833,7 +865,11 @@ private void flushCurrentBatch() { */ private void maybeFlushCurrentBatch(long currentTimeMs) { if (currentBatch != null) { - if (currentBatch.builder.isTransactional() || (currentTimeMs - currentBatch.appendTimeMs) >= appendLingerMs || !currentBatch.builder.hasRoomFor(0)) { + if (currentBatch.builder.isTransactional() || + // When adaptive linger time is enabled, we avoid flushing here. + // Instead, we rely on the flush event enqueued at the back of the event queue. + (appendLingerMs.isPresent() && (currentTimeMs - currentBatch.appendTimeMs) >= appendLingerMs.getAsInt()) || + !currentBatch.builder.hasRoomFor(0)) { flushCurrentBatch(); } } @@ -882,20 +918,31 @@ private void maybeAllocateNewBatch( maxBatchSize ); + batchEpoch++; + Optional lingerTimeoutTask = Optional.empty(); - if (appendLingerMs > 0) { - lingerTimeoutTask = Optional.of(new TimerTask(appendLingerMs) { - @Override - public void run() { - // An event to flush the batch is pushed to the front of the queue - // to ensure that the linger time is respected. - enqueueFirst(new CoordinatorInternalEvent("FlushBatch", tp, () -> { - if (this.isCancelled()) return; - withActiveContextOrThrow(tp, CoordinatorContext::flushCurrentBatch); - })); - } - }); - CoordinatorRuntime.this.timer.add(lingerTimeoutTask.get()); + if (appendLingerMs.isPresent()) { + if (appendLingerMs.getAsInt() > 0) { + lingerTimeoutTask = Optional.of(new TimerTask(appendLingerMs.getAsInt()) { + @Override + public void run() { + // An event to flush the batch is pushed to the front of the queue + // to ensure that the linger time is respected. + enqueueFirst(new CoordinatorInternalEvent("FlushBatch", tp, () -> { + if (this.isCancelled()) return; + withActiveContextOrThrow(tp, CoordinatorContext::flushCurrentBatch); + })); + } + }); + CoordinatorRuntime.this.timer.add(lingerTimeoutTask.get()); + } + } else { + // Always queue a flush immediately at the end of the queue, unless the batch is + // transactional. Transactional batches are flushed immediately at the end of + // the write, so a flush event is never needed. + if (!builder.isTransactional()) { + enqueueAdaptiveFlush(batchEpoch); + } } currentBatch = new CoordinatorBatch( @@ -1991,9 +2038,10 @@ public void onHighWatermarkUpdated( /** * The duration in milliseconds that the coordinator will wait for writes to - * accumulate before flushing them to disk. + * accumulate before flushing them to disk. {@code OptionalInt.empty()} indicates + * an adaptive linger time based on the workload. */ - private final int appendLingerMs; + private final OptionalInt appendLingerMs; /** * The executor service used by the coordinator runtime to schedule @@ -2045,7 +2093,7 @@ private CoordinatorRuntime( CoordinatorMetrics coordinatorMetrics, Serializer serializer, Compression compression, - int appendLingerMs, + OptionalInt appendLingerMs, ExecutorService executorService ) { this.logPrefix = logPrefix; From 33a13a5f8f5b561daa4a1e4bc447dc45aca69445 Mon Sep 17 00:00:00 2001 From: Sean Quah Date: Mon, 27 Oct 2025 21:20:43 +0000 Subject: [PATCH 02/10] KAFKA-19826: Add tests for adaptive append linger time --- .../runtime/CoordinatorRuntimeTest.java | 447 ++++++++++++++++++ 1 file changed, 447 insertions(+) diff --git a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java index de1aa21f3f116..e004e1e6e4e34 100644 --- a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java +++ b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java @@ -4942,6 +4942,453 @@ public void testLingerTimeComparisonInMaybeFlushCurrentBatch() throws Exception assertEquals(0, schedulerTimer.size()); } + @Test + public void testAdaptiveAppendLingerTime() { + MockTimer timer = new MockTimer(); + ManualEventProcessor processor = new ManualEventProcessor(); + MockPartitionWriter writer = new MockPartitionWriter(); + + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(timer.time()) + .withTimer(timer) + .withDefaultWriteTimeOut(Duration.ofMillis(20)) + .withLoader(new MockCoordinatorLoader()) + .withEventProcessor(processor) + .withPartitionWriter(writer) + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) + .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) + .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) + .withSerializer(new StringSerializer()) + .withAppendLingerMs(OptionalInt.empty()) + .withExecutorService(mock(ExecutorService.class)) + .build(); + + // Loads the coordinator. Poll once to execute the load operation and once + // to complete the load. + runtime.scheduleLoadOperation(TP, 10); + processor.poll(); + processor.poll(); + + // Verify the initial state. + CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertNull(ctx.currentBatch); + + // Write #1. + runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(List.of("record1", "record2"), "response1") + ); + + // Write #2. + runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(List.of("record3"), "response2") + ); + + // Execute write #1. + processor.poll(); + + // A batch has been created. + assertNotNull(ctx.currentBatch); + + // A flush event is queued after write #2. + assertEquals(2, processor.size()); + + // Verify the state. Records are replayed but no batch written. + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of( + new MockCoordinatorShard.RecordAndMetadata(0, "record1"), + new MockCoordinatorShard.RecordAndMetadata(1, "record2") + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of(), writer.entries(TP)); + + // Execute write #2. + processor.poll(); + assertEquals(1, processor.size()); + + // The batch has not been flushed. + assertNotNull(ctx.currentBatch); + + // Verify the state. Records are replayed but no batch written. + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of( + new MockCoordinatorShard.RecordAndMetadata(0, "record1"), + new MockCoordinatorShard.RecordAndMetadata(1, "record2"), + new MockCoordinatorShard.RecordAndMetadata(2, "record3") + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of(), writer.entries(TP)); + + // Flush the batch. + processor.poll(); + + // The batch is flushed. + assertNull(ctx.currentBatch); + + // Verify the state. + assertEquals(3L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L, 3L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of( + new MockCoordinatorShard.RecordAndMetadata(0, "record1"), + new MockCoordinatorShard.RecordAndMetadata(1, "record2"), + new MockCoordinatorShard.RecordAndMetadata(2, "record3") + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of( + TestUtil.records(timer.time().milliseconds(), "record1", "record2", "record3") + ), writer.entries(TP)); + } + + /** + * Tests a flush triggered by the max batch size with an adaptive append linger time. + * + * The flush for the first batch must not flush the second batch. + */ + @Test + @SuppressWarnings("checkstyle:MethodLength") + public void testAdaptiveAppendLingerWithMaxBatchSizeFlush() { + MockTimer timer = new MockTimer(); + ManualEventProcessor processor = new ManualEventProcessor(); + MockPartitionWriter writer = new MockPartitionWriter(); + + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(timer.time()) + .withTimer(timer) + .withDefaultWriteTimeOut(Duration.ofMillis(20)) + .withLoader(new MockCoordinatorLoader()) + .withEventProcessor(processor) + .withPartitionWriter(writer) + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) + .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) + .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) + .withSerializer(new StringSerializer()) + .withAppendLingerMs(OptionalInt.empty()) + .withExecutorService(mock(ExecutorService.class)) + .build(); + + // Loads the coordinator. Poll once to execute the load operation and once + // to complete the load. + runtime.scheduleLoadOperation(TP, 10); + processor.poll(); + processor.poll(); + + // Verify the initial state. + CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertNull(ctx.currentBatch); + + // Get the max batch size. + int maxBatchSize = writer.config(TP).maxMessageSize(); + + // Create records with a quarter of the max batch size each. Keep in mind that + // each batch has a header so it is not possible to have those four records + // in one single batch. + List records = Stream.of('1', '2', '3', '4', '5').map(c -> { + char[] payload = new char[maxBatchSize / 4]; + Arrays.fill(payload, c); + return new String(payload); + }).collect(Collectors.toList()); + + // Write #1. + runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(0, 1), "response1") + ); + + // Write #2. + runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(1, 2), "response2") + ); + + // Write #3. + runtime.scheduleWriteOperation("write#3", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(2, 3), "response3") + ); + + // Write #4. + runtime.scheduleWriteOperation("write#4", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(3, 4), "response4") + ); + + // Execute write #1, write #2 and write #3. + processor.poll(); + processor.poll(); + processor.poll(); + + // A batch has been created. + assertNotNull(ctx.currentBatch); + + // A flush event is queued after write #4. + assertEquals(2, processor.size()); + + // Verify the state. Records are replayed but no batch written. + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of( + new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), + new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), + new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)) + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of(), writer.entries(TP)); + + // Write #5. + runtime.scheduleWriteOperation("write#5", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(4, 5), "response5") + ); + + // Execute write #4. This one cannot go into the existing batch + // so the existing batch should be flushed and a new one should be created. + processor.poll(); + + // A batch has been created. + assertNotNull(ctx.currentBatch); + + // Another flush event is queued after write #5. + assertEquals(3, processor.size()); + + // Verify the state. + assertEquals(3L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L, 3L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of( + new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), + new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), + new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)), + new MockCoordinatorShard.RecordAndMetadata(3, records.get(3)) + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of( + TestUtil.records(timer.time().milliseconds(), records.subList(0, 3)) + ), writer.entries(TP)); + + // Execute the first flush. + processor.poll(); + assertEquals(2, processor.size()); + + // The flush does not belong to the current batch and is ignored. + assertNotNull(ctx.currentBatch); + + // Verify the state. + assertEquals(3L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L, 3L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of( + new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), + new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), + new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)), + new MockCoordinatorShard.RecordAndMetadata(3, records.get(3)) + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of( + TestUtil.records(timer.time().milliseconds(), records.subList(0, 3)) + ), writer.entries(TP)); + + // Execute write #5. + processor.poll(); + assertEquals(1, processor.size()); + + // Verify the state. + assertEquals(3L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L, 3L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of( + new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), + new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), + new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)), + new MockCoordinatorShard.RecordAndMetadata(3, records.get(3)), + new MockCoordinatorShard.RecordAndMetadata(4, records.get(4)) + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of( + TestUtil.records(timer.time().milliseconds(), records.subList(0, 3)) + ), writer.entries(TP)); + + // Execute the second flush. + processor.poll(); + assertEquals(0, processor.size()); + + // The batch is flushed. + assertNull(ctx.currentBatch); + + // Verify the state. + assertEquals(5L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L, 3L, 5L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of( + new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), + new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), + new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)), + new MockCoordinatorShard.RecordAndMetadata(3, records.get(3)), + new MockCoordinatorShard.RecordAndMetadata(4, records.get(4)) + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of( + TestUtil.records(timer.time().milliseconds(), records.subList(0, 3)), + TestUtil.records(timer.time().milliseconds(), records.subList(3, 5)) + ), writer.entries(TP)); + } + + /** + * Tests a transactional write with an adaptive append linger time. + * + * The transactional write must not enqueue a flush, since it flushes immediately. + * The flush for the batch before the transactional write must not flush the batch after the + * transactional write. + */ + @Test + public void testAdaptiveAppendLingerWithTransactionalWrite() { + MockTimer timer = new MockTimer(); + ManualEventProcessor processor = new ManualEventProcessor(); + MockPartitionWriter writer = new MockPartitionWriter(); + + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(timer.time()) + .withTimer(timer) + .withDefaultWriteTimeOut(Duration.ofMillis(20)) + .withLoader(new MockCoordinatorLoader()) + .withEventProcessor(processor) + .withPartitionWriter(writer) + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) + .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) + .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) + .withSerializer(new StringSerializer()) + .withAppendLingerMs(OptionalInt.empty()) + .withExecutorService(mock(ExecutorService.class)) + .build(); + + // Loads the coordinator. Poll once to execute the load operation and once + // to complete the load. + runtime.scheduleLoadOperation(TP, 10); + processor.poll(); + processor.poll(); + + // Verify the initial state. + CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertNull(ctx.currentBatch); + + // Write #1. + runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(List.of("record1"), "response1") + ); + + // Transactional write #2. This will flush the batch. + runtime.scheduleTransactionalWriteOperation( + "txn-write#1", + TP, + "transactional-id", + 100L, + (short) 50, + Duration.ofMillis(20), + state -> new CoordinatorResult<>(List.of("record2"), "response2"), + TXN_OFFSET_COMMIT_LATEST_VERSION + ); + + // Write #3. + runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(List.of("record3"), "response3") + ); + + assertEquals(3, processor.size()); + + // Execute write #1. + processor.poll(); + + // A batch has been created. + assertNotNull(ctx.currentBatch); + + // A flush event is queued after write #3. + assertEquals(3, processor.size()); + + // Verify the state. Records are replayed but no batch written. + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of( + new MockCoordinatorShard.RecordAndMetadata(0, "record1") + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of(), writer.entries(TP)); + + // Execute transactional write #2. + processor.poll(); + + // The batch is flushed. + assertNull(ctx.currentBatch); + + // No flush event is queued. + assertEquals(2, processor.size()); + + // Verify the state. The current batch and the transactional records are + // written to the log. + assertEquals(2L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L, 1L, 2L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of( + new MockCoordinatorShard.RecordAndMetadata(0, "record1") + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of( + TestUtil.records(timer.time().milliseconds(), "record1"), + TestUtil.transactionalRecords(100L, (short) 50, timer.time().milliseconds(), "record2") + ), writer.entries(TP)); + + // Execute write #3. + processor.poll(); + + // A batch has been created. + assertNotNull(ctx.currentBatch); + + // A flush event is queued after the first flush. + assertEquals(2, processor.size()); + + // Verify the state. Records are replayed but no batch written. + assertEquals(2L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L, 1L, 2L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of( + new MockCoordinatorShard.RecordAndMetadata(0, "record1"), + new MockCoordinatorShard.RecordAndMetadata(2, "record3") + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of( + TestUtil.records(timer.time().milliseconds(), "record1"), + TestUtil.transactionalRecords(100L, (short) 50, timer.time().milliseconds(), "record2") + ), writer.entries(TP)); + + // Execute the first flush. + processor.poll(); + assertEquals(1, processor.size()); + + // The flush does not belong to the current batch and is ignored. + assertNotNull(ctx.currentBatch); + + // Execute the second flush. + processor.poll(); + assertEquals(0, processor.size()); + + // The batch is flushed. + assertNull(ctx.currentBatch); + + // Verify the state. + assertEquals(3L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L, 1L, 2L, 3L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of( + new MockCoordinatorShard.RecordAndMetadata(0, "record1"), + new MockCoordinatorShard.RecordAndMetadata(2, "record3") + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of( + TestUtil.records(timer.time().milliseconds(), "record1"), + TestUtil.transactionalRecords(100L, (short) 50, timer.time().milliseconds(), "record2"), + TestUtil.records(timer.time().milliseconds(), "record3") + ), writer.entries(TP)); + } + private static , U> ArgumentMatcher> coordinatorMatcher( CoordinatorRuntime runtime, TopicPartition tp From b93f79051f01716b3255539b0a39d05b5f90a726 Mon Sep 17 00:00:00 2001 From: Sean Quah Date: Mon, 27 Oct 2025 21:22:56 +0000 Subject: [PATCH 03/10] KAFKA-19826: Update config to allow append.linger.ms = -1 --- .../unit/kafka/server/KafkaConfigTest.scala | 5 +++++ .../group/GroupCoordinatorConfig.java | 18 +++++++++++------ .../share/ShareCoordinatorConfig.java | 20 ++++++++++++++----- 3 files changed, 32 insertions(+), 11 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index ab7f45ae6895e..d0900a63ce08e 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -34,6 +34,7 @@ import org.apache.kafka.coordinator.group.ConsumerGroupMigrationPolicy import org.apache.kafka.coordinator.group.Group.GroupType import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.modern.share.ShareGroupConfig +import org.apache.kafka.coordinator.share.ShareCoordinatorConfig import org.apache.kafka.coordinator.transaction.{TransactionLogConfig, TransactionStateManagerConfig} import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.raft.{MetadataLogConfig, QuorumConfig} @@ -1038,6 +1039,7 @@ class KafkaConfigTest { /** New group coordinator configs */ case GroupCoordinatorConfig.GROUP_COORDINATOR_NUM_THREADS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", 0, -1) + case GroupCoordinatorConfig.GROUP_COORDINATOR_APPEND_LINGER_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", -2) /** Consumer groups configs */ case GroupCoordinatorConfig.CONSUMER_GROUP_SESSION_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", 0, -1) @@ -1077,6 +1079,9 @@ class KafkaConfigTest { case GroupCoordinatorConfig.STREAMS_GROUP_NUM_STANDBY_REPLICAS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", -1) case GroupCoordinatorConfig.STREAMS_GROUP_MAX_STANDBY_REPLICAS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", -1) + /** Share coordinator configs */ + case ShareCoordinatorConfig.APPEND_LINGER_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", -2) + case _ => assertPropertyInvalid(baseProperties, name, "not_a_number", "-1") } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java index e8a2f49663955..fb3e820ea8856 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java @@ -36,6 +36,7 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.OptionalInt; import java.util.function.Function; import java.util.stream.Collectors; @@ -73,8 +74,9 @@ public class GroupCoordinatorConfig { public static final String GROUP_COORDINATOR_APPEND_LINGER_MS_DOC = "The duration in milliseconds that the coordinator will " + "wait for writes to accumulate before flushing them to disk. Increasing this value improves write efficiency and batch size, " + "but also increases the response latency for requests, as the coordinator must wait for batches to be flushed to " + - "disk before completing request processing. Transactional writes are not accumulated."; - public static final int GROUP_COORDINATOR_APPEND_LINGER_MS_DEFAULT = 5; + "disk before completing request processing. Transactional writes are not accumulated. " + + "Set to -1 for an adaptive linger time that minimizes latency based on the workload."; + public static final int GROUP_COORDINATOR_APPEND_LINGER_MS_DEFAULT = -1; public static final String GROUP_COORDINATOR_NUM_THREADS_CONFIG = "group.coordinator.threads"; public static final String GROUP_COORDINATOR_NUM_THREADS_DOC = "The number of threads used by the group coordinator."; @@ -300,7 +302,7 @@ public class GroupCoordinatorConfig { .define(GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, LIST, GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DEFAULT, ConfigDef.ValidList.in(false, Group.GroupType.documentValidValues()), MEDIUM, GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DOC) .define(GROUP_COORDINATOR_NUM_THREADS_CONFIG, INT, GROUP_COORDINATOR_NUM_THREADS_DEFAULT, atLeast(1), HIGH, GROUP_COORDINATOR_NUM_THREADS_DOC) - .define(GROUP_COORDINATOR_APPEND_LINGER_MS_CONFIG, INT, GROUP_COORDINATOR_APPEND_LINGER_MS_DEFAULT, atLeast(0), MEDIUM, GROUP_COORDINATOR_APPEND_LINGER_MS_DOC) + .define(GROUP_COORDINATOR_APPEND_LINGER_MS_CONFIG, INT, GROUP_COORDINATOR_APPEND_LINGER_MS_DEFAULT, atLeast(-1), MEDIUM, GROUP_COORDINATOR_APPEND_LINGER_MS_DOC) .define(OFFSET_COMMIT_TIMEOUT_MS_CONFIG, INT, OFFSET_COMMIT_TIMEOUT_MS_DEFAULT, atLeast(1), HIGH, OFFSET_COMMIT_TIMEOUT_MS_DOC) .define(OFFSETS_LOAD_BUFFER_SIZE_CONFIG, INT, OFFSETS_LOAD_BUFFER_SIZE_DEFAULT, atLeast(1), HIGH, OFFSETS_LOAD_BUFFER_SIZE_DOC) .define(OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, SHORT, OFFSETS_TOPIC_REPLICATION_FACTOR_DEFAULT, atLeast(1), HIGH, OFFSETS_TOPIC_REPLICATION_FACTOR_DOC) @@ -647,10 +649,14 @@ public int numThreads() { /** * The duration in milliseconds that the coordinator will wait for writes to - * accumulate before flushing them to disk. + * accumulate before flushing them to disk. {@code OptionalInt.empty()} indicates + * an adaptive linger time based on the workload. */ - public int appendLingerMs() { - return appendLingerMs; + public OptionalInt appendLingerMs() { + if (appendLingerMs == -1) { + return OptionalInt.empty(); + } + return OptionalInt.of(appendLingerMs); } /** diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorConfig.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorConfig.java index b92947af9cd60..98e0a2a5f25c4 100644 --- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorConfig.java +++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorConfig.java @@ -22,6 +22,7 @@ import org.apache.kafka.common.utils.Utils; import java.util.Optional; +import java.util.OptionalInt; import static org.apache.kafka.common.config.ConfigDef.Importance.HIGH; import static org.apache.kafka.common.config.ConfigDef.Importance.LOW; @@ -69,8 +70,9 @@ public class ShareCoordinatorConfig { public static final String STATE_TOPIC_COMPRESSION_CODEC_DOC = "Compression codec for the share-group state topic."; public static final String APPEND_LINGER_MS_CONFIG = "share.coordinator.append.linger.ms"; - public static final int APPEND_LINGER_MS_DEFAULT = 5; - public static final String APPEND_LINGER_MS_DOC = "The duration in milliseconds that the share coordinator will wait for writes to accumulate before flushing them to disk."; + public static final int APPEND_LINGER_MS_DEFAULT = -1; + public static final String APPEND_LINGER_MS_DOC = "The duration in milliseconds that the share coordinator will wait for writes to accumulate before flushing them to disk. " + + "Set to -1 for an adaptive linger time that minimizes latency based on the workload."; public static final String STATE_TOPIC_PRUNE_INTERVAL_MS_CONFIG = "share.coordinator.state.topic.prune.interval.ms"; public static final int STATE_TOPIC_PRUNE_INTERVAL_MS_DEFAULT = 5 * 60 * 1000; // 5 minutes @@ -89,7 +91,7 @@ public class ShareCoordinatorConfig { .define(SNAPSHOT_UPDATE_RECORDS_PER_SNAPSHOT_CONFIG, INT, SNAPSHOT_UPDATE_RECORDS_PER_SNAPSHOT_DEFAULT, atLeast(0), MEDIUM, SNAPSHOT_UPDATE_RECORDS_PER_SNAPSHOT_DOC) .define(LOAD_BUFFER_SIZE_CONFIG, INT, LOAD_BUFFER_SIZE_DEFAULT, atLeast(1), HIGH, LOAD_BUFFER_SIZE_DOC) .define(STATE_TOPIC_COMPRESSION_CODEC_CONFIG, INT, (int) STATE_TOPIC_COMPRESSION_CODEC_DEFAULT.id, HIGH, STATE_TOPIC_COMPRESSION_CODEC_DOC) - .define(APPEND_LINGER_MS_CONFIG, INT, APPEND_LINGER_MS_DEFAULT, atLeast(0), MEDIUM, APPEND_LINGER_MS_DOC) + .define(APPEND_LINGER_MS_CONFIG, INT, APPEND_LINGER_MS_DEFAULT, atLeast(-1), MEDIUM, APPEND_LINGER_MS_DOC) .define(WRITE_TIMEOUT_MS_CONFIG, INT, WRITE_TIMEOUT_MS_DEFAULT, atLeast(1), HIGH, WRITE_TIMEOUT_MS_DOC) .defineInternal(STATE_TOPIC_PRUNE_INTERVAL_MS_CONFIG, INT, STATE_TOPIC_PRUNE_INTERVAL_MS_DEFAULT, atLeast(1), LOW, STATE_TOPIC_PRUNE_INTERVAL_MS_DOC) .defineInternal(COLD_PARTITION_SNAPSHOT_INTERVAL_MS_CONFIG, INT, COLD_PARTITION_SNAPSHOT_INTERVAL_MS_DEFAULT, atLeast(1), LOW, COLD_PARTITION_SNAPSHOT_INTERVAL_MS_DOC); @@ -157,8 +159,16 @@ public int shareCoordinatorLoadBufferSize() { return loadBufferSize; } - public int shareCoordinatorAppendLingerMs() { - return appendLingerMs; + /** + * The duration in milliseconds that the coordinator will wait for writes to + * accumulate before flushing them to disk. {@code OptionalInt.empty()} indicates + * an adaptive linger time based on the workload. + */ + public OptionalInt shareCoordinatorAppendLingerMs() { + if (appendLingerMs == -1) { + return OptionalInt.empty(); + } + return OptionalInt.of(appendLingerMs); } public CompressionType shareCoordinatorStateTopicCompressionType() { From f13c2659a4275d54e876e267b45f2430c7d7c8f7 Mon Sep 17 00:00:00 2001 From: Sean Quah Date: Mon, 27 Oct 2025 21:26:04 +0000 Subject: [PATCH 04/10] KAFKA-19826: Set default CoordinatorRuntime append linger to adaptive NB: This refers to the CoordinatorRuntimeBuilder's default append linger time, not the config-level default. --- .../common/runtime/CoordinatorRuntime.java | 2 +- .../runtime/CoordinatorRuntimeTest.java | 5 +++ .../common/runtime/DirectEventProcessor.java | 41 ++++++++++++++----- 3 files changed, 37 insertions(+), 11 deletions(-) diff --git a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java index a4864e9b8134b..102ec8b8436ea 100644 --- a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java +++ b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java @@ -227,7 +227,7 @@ public CoordinatorRuntime build() { if (compression == null) compression = Compression.NONE; if (appendLingerMs == null) - appendLingerMs = OptionalInt.of(0); + appendLingerMs = OptionalInt.empty(); if (appendLingerMs.isPresent() && appendLingerMs.getAsInt() < -1) throw new IllegalArgumentException("AppendLinger must be empty or >= 0"); if (executorService == null) diff --git a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java index e004e1e6e4e34..c7d71ea46d8be 100644 --- a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java +++ b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java @@ -2615,6 +2615,7 @@ public void testHighWatermarkUpdate() { .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) + .withAppendLingerMs(0) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -2766,6 +2767,7 @@ public void testWriteEventWriteTimeoutTaskIsCancelledWhenHighWatermarkIsUpdated( .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) + .withAppendLingerMs(0) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -4576,6 +4578,7 @@ public void testRecordEventPurgatoryTime() throws Exception { .withCoordinatorRuntimeMetrics(runtimeMetrics) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) + .withAppendLingerMs(0) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -4662,6 +4665,7 @@ public void testWriteEventCompletesOnlyOnce() throws Exception { .withCoordinatorRuntimeMetrics(runtimeMetrics) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) + .withAppendLingerMs(0) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -4807,6 +4811,7 @@ public void testCoordinatorExecutor() { .withCoordinatorRuntimeMetrics(runtimeMetrics) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) + .withAppendLingerMs(0) .withExecutorService(executorService) .build(); diff --git a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/DirectEventProcessor.java b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/DirectEventProcessor.java index 60b74c3a12f72..19c79e267147f 100644 --- a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/DirectEventProcessor.java +++ b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/DirectEventProcessor.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.coordinator.common.runtime; +import java.util.Deque; +import java.util.LinkedList; import java.util.concurrent.RejectedExecutionException; /** @@ -23,24 +25,43 @@ * useful in unit tests where execution in threads is not required. */ public class DirectEventProcessor implements CoordinatorEventProcessor { + private final Deque queue; + private boolean inEvent; + + public DirectEventProcessor() { + this.queue = new LinkedList<>(); + this.inEvent = false; + } + @Override public void enqueueLast(CoordinatorEvent event) throws RejectedExecutionException { - try { - event.run(); - } catch (Throwable ex) { - event.complete(ex); - } + queue.addLast(event); + processQueue(); } @Override public void enqueueFirst(CoordinatorEvent event) throws RejectedExecutionException { - try { - event.run(); - } catch (Throwable ex) { - event.complete(ex); - } + queue.addFirst(event); + processQueue(); } @Override public void close() {} + + private void processQueue() { + if (inEvent) { + return; + } + + inEvent = true; + while (!queue.isEmpty()) { + CoordinatorEvent event = queue.removeFirst(); + try { + event.run(); + } catch (Throwable ex) { + event.complete(ex); + } + } + inEvent = false; + } } From 578b13a8a63619d8aaa76ed715baf768356c3769 Mon Sep 17 00:00:00 2001 From: Sean Quah Date: Mon, 27 Oct 2025 21:31:06 +0000 Subject: [PATCH 05/10] KAFKA-19826: Move new tests to just after the batching tests --- .../runtime/CoordinatorRuntimeTest.java | 2134 ++++++++--------- 1 file changed, 1067 insertions(+), 1067 deletions(-) diff --git a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java index c7d71ea46d8be..89758254f869d 100644 --- a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java +++ b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java @@ -3652,183 +3652,117 @@ public void testScheduleTransactionalWriteOperationWithBatching() throws Executi } @Test - public void testStateMachineIsReloadedWhenOutOfSync() { + public void testAdaptiveAppendLingerTime() { MockTimer timer = new MockTimer(); - MockCoordinatorLoader loader = spy(new MockCoordinatorLoader()); - MockPartitionWriter writer = new MockPartitionWriter() { - @Override - public long append( - TopicPartition tp, - VerificationGuard verificationGuard, - MemoryRecords batch - ) { - // Add 1 to the returned offsets. - return super.append(tp, verificationGuard, batch) + 1; - } - }; + ManualEventProcessor processor = new ManualEventProcessor(); + MockPartitionWriter writer = new MockPartitionWriter(); CoordinatorRuntime runtime = new CoordinatorRuntime.Builder() .withTime(timer.time()) .withTimer(timer) .withDefaultWriteTimeOut(Duration.ofMillis(20)) - .withLoader(loader) - .withEventProcessor(new DirectEventProcessor()) + .withLoader(new MockCoordinatorLoader()) + .withEventProcessor(processor) .withPartitionWriter(writer) .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(10) + .withAppendLingerMs(OptionalInt.empty()) .withExecutorService(mock(ExecutorService.class)) .build(); - // Schedule the loading. + // Loads the coordinator. Poll once to execute the load operation and once + // to complete the load. runtime.scheduleLoadOperation(TP, 10); + processor.poll(); + processor.poll(); // Verify the initial state. CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); - assertEquals(ACTIVE, ctx.state); assertEquals(0L, ctx.coordinator.lastWrittenOffset()); assertEquals(0L, ctx.coordinator.lastCommittedOffset()); assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); assertNull(ctx.currentBatch); - // Keep a reference to the current coordinator. - SnapshottableCoordinator coordinator = ctx.coordinator; - - // Get the max batch size. - int maxBatchSize = writer.config(TP).maxMessageSize(); - - // Create records with a quarter of the max batch size each. Keep in mind that - // each batch has a header so it is not possible to have those four records - // in one single batch. - List records = Stream.of('1', '2', '3', '4').map(c -> { - char[] payload = new char[maxBatchSize / 4]; - Arrays.fill(payload, c); - return new String(payload); - }).collect(Collectors.toList()); - // Write #1. - CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(records.subList(0, 1), "response1")); + runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(List.of("record1", "record2"), "response1") + ); // Write #2. - CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(records.subList(1, 2), "response2")); - - // Write #3. - CompletableFuture write3 = runtime.scheduleWriteOperation("write#3", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(records.subList(2, 3), "response3")); - - // Write #4. This write cannot make it in the current batch. So the current batch - // is flushed. It will fail. So we expect all writes to fail. - CompletableFuture write4 = runtime.scheduleWriteOperation("write#4", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(records.subList(3, 4), "response4")); - - // Verify the futures. - assertFutureThrows(NotCoordinatorException.class, write1); - assertFutureThrows(NotCoordinatorException.class, write2); - assertFutureThrows(NotCoordinatorException.class, write3); - // Write #4 is also expected to fail. - assertFutureThrows(NotCoordinatorException.class, write4); - - // Verify that the state machine was loaded twice. - verify(loader, times(2)).load(eq(TP), any()); - - // Verify that the state is active and that the state machine - // is actually a new one. - assertEquals(ACTIVE, ctx.state); - assertNotEquals(coordinator, ctx.coordinator); - } - - @Test - public void testWriteOpIsNotReleasedWhenStateMachineIsNotCaughtUpAfterLoad() throws ExecutionException, InterruptedException, TimeoutException { - MockTimer timer = new MockTimer(); - MockPartitionWriter writer = new MockPartitionWriter(); - CoordinatorLoader loader = new CoordinatorLoader() { - @Override - public CompletableFuture load( - TopicPartition tp, - CoordinatorPlayback coordinator - ) { - coordinator.replay( - 0, - RecordBatch.NO_PRODUCER_ID, - RecordBatch.NO_PRODUCER_EPOCH, - "record#0" - ); - - coordinator.replay( - 0, - RecordBatch.NO_PRODUCER_ID, - RecordBatch.NO_PRODUCER_EPOCH, - "record#1" - ); + runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(List.of("record3"), "response2") + ); - coordinator.updateLastWrittenOffset(2L); - coordinator.updateLastCommittedOffset(1L); + // Execute write #1. + processor.poll(); - return CompletableFuture.completedFuture(new LoadSummary( - 0L, - 0L, - 0L, - 2, - 1 - )); - } + // A batch has been created. + assertNotNull(ctx.currentBatch); - @Override - public void close() {} - }; + // A flush event is queued after write #2. + assertEquals(2, processor.size()); - CoordinatorRuntime runtime = - new CoordinatorRuntime.Builder() - .withTime(timer.time()) - .withTimer(timer) - .withDefaultWriteTimeOut(Duration.ofMillis(20)) - .withLoader(loader) - .withEventProcessor(new DirectEventProcessor()) - .withPartitionWriter(writer) - .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) - .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) - .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) - .withSerializer(new StringSerializer()) - .withAppendLingerMs(10) - .withExecutorService(mock(ExecutorService.class)) - .build(); + // Verify the state. Records are replayed but no batch written. + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of( + new MockCoordinatorShard.RecordAndMetadata(0, "record1"), + new MockCoordinatorShard.RecordAndMetadata(1, "record2") + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of(), writer.entries(TP)); - // Schedule the loading. - runtime.scheduleLoadOperation(TP, 10); + // Execute write #2. + processor.poll(); + assertEquals(1, processor.size()); - // Verify the initial state. - CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); - assertEquals(2L, ctx.coordinator.lastWrittenOffset()); - assertEquals(1L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of(2L), ctx.coordinator.snapshotRegistry().epochsList()); + // The batch has not been flushed. + assertNotNull(ctx.currentBatch); - // Schedule a write operation that does not generate any records. - CompletableFuture write = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(List.of(), "response1")); + // Verify the state. Records are replayed but no batch written. + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of( + new MockCoordinatorShard.RecordAndMetadata(0, "record1"), + new MockCoordinatorShard.RecordAndMetadata(1, "record2"), + new MockCoordinatorShard.RecordAndMetadata(2, "record3") + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of(), writer.entries(TP)); - // The write operation should not be done. - assertFalse(write.isDone()); + // Flush the batch. + processor.poll(); - // Advance the last committed offset. - ctx.highWatermarklistener.onHighWatermarkUpdated(TP, 2L); + // The batch is flushed. + assertNull(ctx.currentBatch); // Verify the state. - assertEquals(2L, ctx.coordinator.lastWrittenOffset()); - assertEquals(2L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of(2L), ctx.coordinator.snapshotRegistry().epochsList()); - - // The write operation should be completed. - assertEquals("response1", write.get(5, TimeUnit.SECONDS)); + assertEquals(3L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L, 3L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of( + new MockCoordinatorShard.RecordAndMetadata(0, "record1"), + new MockCoordinatorShard.RecordAndMetadata(1, "record2"), + new MockCoordinatorShard.RecordAndMetadata(2, "record3") + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of( + TestUtil.records(timer.time().milliseconds(), "record1", "record2", "record3") + ), writer.entries(TP)); } + /** + * Tests a flush triggered by the max batch size with an adaptive append linger time. + * + * The flush for the first batch must not flush the second batch. + */ @Test - public void testScheduleNonAtomicWriteOperation() throws ExecutionException, InterruptedException, TimeoutException { + @SuppressWarnings("checkstyle:MethodLength") + public void testAdaptiveAppendLingerWithMaxBatchSizeFlush() { MockTimer timer = new MockTimer(); + ManualEventProcessor processor = new ManualEventProcessor(); MockPartitionWriter writer = new MockPartitionWriter(); CoordinatorRuntime runtime = @@ -3837,18 +3771,21 @@ public void testScheduleNonAtomicWriteOperation() throws ExecutionException, Int .withTimer(timer) .withDefaultWriteTimeOut(Duration.ofMillis(20)) .withLoader(new MockCoordinatorLoader()) - .withEventProcessor(new DirectEventProcessor()) + .withEventProcessor(processor) .withPartitionWriter(writer) .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(10) + .withAppendLingerMs(OptionalInt.empty()) .withExecutorService(mock(ExecutorService.class)) .build(); - // Schedule the loading. + // Loads the coordinator. Poll once to execute the load operation and once + // to complete the load. runtime.scheduleLoadOperation(TP, 10); + processor.poll(); + processor.poll(); // Verify the initial state. CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); @@ -3863,59 +3800,73 @@ public void testScheduleNonAtomicWriteOperation() throws ExecutionException, Int // Create records with a quarter of the max batch size each. Keep in mind that // each batch has a header so it is not possible to have those four records // in one single batch. - List records = Stream.of('1', '2', '3', '4').map(c -> { + List records = Stream.of('1', '2', '3', '4', '5').map(c -> { char[] payload = new char[maxBatchSize / 4]; Arrays.fill(payload, c); return new String(payload); }).collect(Collectors.toList()); - // Let's try to write all the records atomically (the default) to ensure - // that it fails. - CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(records, "write#1") + // Write #1. + runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(0, 1), "response1") ); - assertFutureThrows(RecordTooLargeException.class, write1); + // Write #2. + runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(1, 2), "response2") + ); - // Let's try to write the same records non-atomically. - CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(records, "write#2", null, true, false) + // Write #3. + runtime.scheduleWriteOperation("write#3", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(2, 3), "response3") ); - // The write is pending. - assertFalse(write2.isDone()); + // Write #4. + runtime.scheduleWriteOperation("write#4", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(3, 4), "response4") + ); - // Verify the state. + // Execute write #1, write #2 and write #3. + processor.poll(); + processor.poll(); + processor.poll(); + + // A batch has been created. assertNotNull(ctx.currentBatch); - // The last written offset is 3L because one batch was written to the log with - // the first three records. The 4th one is pending. - assertEquals(3L, ctx.coordinator.lastWrittenOffset()); + + // A flush event is queued after write #4. + assertEquals(2, processor.size()); + + // Verify the state. Records are replayed but no batch written. + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); assertEquals(0L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of(0L, 3L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); assertEquals(List.of( new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), - new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)), - new MockCoordinatorShard.RecordAndMetadata(3, records.get(3)) + new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)) ), ctx.coordinator.coordinator().fullRecords()); - assertEquals(List.of( - records(timer.time().milliseconds(), records.subList(0, 3)) - ), writer.entries(TP)); + assertEquals(List.of(), writer.entries(TP)); - // Commit up to 3L. - writer.commit(TP, 3L); + // Write #5. + runtime.scheduleWriteOperation("write#5", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(4, 5), "response5") + ); - // The write is still pending. - assertFalse(write2.isDone()); + // Execute write #4. This one cannot go into the existing batch + // so the existing batch should be flushed and a new one should be created. + processor.poll(); - // Advance past the linger time to flush the pending batch. - timer.advanceClock(11); + // A batch has been created. + assertNotNull(ctx.currentBatch); + + // Another flush event is queued after write #5. + assertEquals(3, processor.size()); // Verify the state. - assertNull(ctx.currentBatch); - assertEquals(4L, ctx.coordinator.lastWrittenOffset()); - assertEquals(3L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of(3L, 4L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(3L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L, 3L), ctx.coordinator.snapshotRegistry().epochsList()); assertEquals(List.of( new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), @@ -3923,118 +3874,85 @@ public void testScheduleNonAtomicWriteOperation() throws ExecutionException, Int new MockCoordinatorShard.RecordAndMetadata(3, records.get(3)) ), ctx.coordinator.coordinator().fullRecords()); assertEquals(List.of( - records(timer.time().milliseconds() - 11, records.subList(0, 3)), - records(timer.time().milliseconds() - 11, records.subList(3, 4)) + TestUtil.records(timer.time().milliseconds(), records.subList(0, 3)) ), writer.entries(TP)); - // Commit up to 4L. - writer.commit(TP, 4L); - - // Verify that the write is completed. - assertTrue(write2.isDone()); - assertEquals("write#2", write2.get(5, TimeUnit.SECONDS)); - } - - @Test - public void testScheduleNonAtomicWriteOperationWithRecordTooLarge() throws InterruptedException { - MockTimer timer = new MockTimer(); - MockPartitionWriter writer = new MockPartitionWriter(); - - CoordinatorRuntime runtime = - new CoordinatorRuntime.Builder() - .withTime(timer.time()) - .withTimer(timer) - .withDefaultWriteTimeOut(Duration.ofMillis(20)) - .withLoader(new MockCoordinatorLoader()) - .withEventProcessor(new DirectEventProcessor()) - .withPartitionWriter(writer) - .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) - .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) - .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) - .withSerializer(new StringSerializer()) - .withAppendLingerMs(10) - .withExecutorService(mock(ExecutorService.class)) - .build(); + // Execute the first flush. + processor.poll(); + assertEquals(2, processor.size()); - // Schedule the loading. - runtime.scheduleLoadOperation(TP, 10); + // The flush does not belong to the current batch and is ignored. + assertNotNull(ctx.currentBatch); - // Verify the initial state. - CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); - assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + // Verify the state. + assertEquals(3L, ctx.coordinator.lastWrittenOffset()); assertEquals(0L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); - assertNull(ctx.currentBatch); - - // Get the max batch size. - int maxBatchSize = writer.config(TP).maxMessageSize(); - - // Create records with a quarter of the max batch size each. Keep in mind that - // each batch has a header so it is not possible to have those four records - // in one single batch. - List records = Stream.of('1', '2', '3').map(c -> { - char[] payload = new char[maxBatchSize / 4]; - Arrays.fill(payload, c); - return new String(payload); - }).collect(Collectors.toList()); - - // Create another record larger than the max batch size. - char[] payload = new char[maxBatchSize]; - Arrays.fill(payload, '4'); - String record = new String(payload); + assertEquals(List.of(0L, 3L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of( + new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), + new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), + new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)), + new MockCoordinatorShard.RecordAndMetadata(3, records.get(3)) + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of( + TestUtil.records(timer.time().milliseconds(), records.subList(0, 3)) + ), writer.entries(TP)); - // Let's write the first three records. - CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(records, "write#1", null, true, false) - ); + // Execute write #5. + processor.poll(); + assertEquals(1, processor.size()); // Verify the state. - assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(3L, ctx.coordinator.lastWrittenOffset()); assertEquals(0L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of(0L, 3L), ctx.coordinator.snapshotRegistry().epochsList()); assertEquals(List.of( new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), - new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)) + new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)), + new MockCoordinatorShard.RecordAndMetadata(3, records.get(3)), + new MockCoordinatorShard.RecordAndMetadata(4, records.get(4)) ), ctx.coordinator.coordinator().fullRecords()); - assertEquals(List.of(), writer.entries(TP)); - - // Let's write the 4th record which is too large. This will flush the current - // pending batch, allocate a new batch, and put the record into it. - // Note that the batch will fail only when the batch is written because the - // MemoryBatchBuilder always accept one record. - CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(List.of(record), "write#2", null, true, false) - ); - - // Advance past the linger time to flush the pending batch. - timer.advanceClock(11); + assertEquals(List.of( + TestUtil.records(timer.time().milliseconds(), records.subList(0, 3)) + ), writer.entries(TP)); - // The write should have failed... - assertFutureThrows(RecordTooLargeException.class, write2); + // Execute the second flush. + processor.poll(); + assertEquals(0, processor.size()); - // ... but write#1 should be left intact. - assertFalse(write1.isDone()); + // The batch is flushed. + assertNull(ctx.currentBatch); // Verify the state. - assertEquals(3L, ctx.coordinator.lastWrittenOffset()); + assertEquals(5L, ctx.coordinator.lastWrittenOffset()); assertEquals(0L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of(0L, 3L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of(0L, 3L, 5L), ctx.coordinator.snapshotRegistry().epochsList()); assertEquals(List.of( new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), - new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)) + new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)), + new MockCoordinatorShard.RecordAndMetadata(3, records.get(3)), + new MockCoordinatorShard.RecordAndMetadata(4, records.get(4)) ), ctx.coordinator.coordinator().fullRecords()); assertEquals(List.of( - records(timer.time().milliseconds() - 11, records.subList(0, 3)) + TestUtil.records(timer.time().milliseconds(), records.subList(0, 3)), + TestUtil.records(timer.time().milliseconds(), records.subList(3, 5)) ), writer.entries(TP)); } + /** + * Tests a transactional write with an adaptive append linger time. + * + * The transactional write must not enqueue a flush, since it flushes immediately. + * The flush for the batch before the transactional write must not flush the batch after the + * transactional write. + */ @Test - public void testScheduleNonAtomicWriteOperationWhenWriteFails() { + public void testAdaptiveAppendLingerWithTransactionalWrite() { MockTimer timer = new MockTimer(); - // The partition writer does not accept any writes. - MockPartitionWriter writer = new MockPartitionWriter(0); + ManualEventProcessor processor = new ManualEventProcessor(); + MockPartitionWriter writer = new MockPartitionWriter(); CoordinatorRuntime runtime = new CoordinatorRuntime.Builder() @@ -4042,18 +3960,21 @@ public void testScheduleNonAtomicWriteOperationWhenWriteFails() { .withTimer(timer) .withDefaultWriteTimeOut(Duration.ofMillis(20)) .withLoader(new MockCoordinatorLoader()) - .withEventProcessor(new DirectEventProcessor()) + .withEventProcessor(processor) .withPartitionWriter(writer) .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(10) + .withAppendLingerMs(OptionalInt.empty()) .withExecutorService(mock(ExecutorService.class)) .build(); - // Schedule the loading. + // Loads the coordinator. Poll once to execute the load operation and once + // to complete the load. runtime.scheduleLoadOperation(TP, 10); + processor.poll(); + processor.poll(); // Verify the initial state. CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); @@ -4062,79 +3983,149 @@ public void testScheduleNonAtomicWriteOperationWhenWriteFails() { assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); assertNull(ctx.currentBatch); - // Get the max batch size. - int maxBatchSize = writer.config(TP).maxMessageSize(); - - // Create records with a quarter of the max batch size each. Keep in mind that - // each batch has a header so it is not possible to have those four records - // in one single batch. - List records = Stream.of('1', '2', '3', '4').map(c -> { - char[] payload = new char[maxBatchSize / 4]; - Arrays.fill(payload, c); - return new String(payload); - }).collect(Collectors.toList()); - // Write #1. - CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(records.subList(0, 1), "response1", null, true, false)); + runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(List.of("record1"), "response1") + ); - // Write #2. - CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(records.subList(1, 2), "response2", null, true, false)); + // Transactional write #2. This will flush the batch. + runtime.scheduleTransactionalWriteOperation( + "txn-write#1", + TP, + "transactional-id", + 100L, + (short) 50, + Duration.ofMillis(20), + state -> new CoordinatorResult<>(List.of("record2"), "response2"), + TXN_OFFSET_COMMIT_LATEST_VERSION + ); // Write #3. - CompletableFuture write3 = runtime.scheduleWriteOperation("write#3", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(records.subList(2, 3), "response3", null, true, false)); + runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(List.of("record3"), "response3") + ); - // Verify the state. + assertEquals(3, processor.size()); + + // Execute write #1. + processor.poll(); + + // A batch has been created. + assertNotNull(ctx.currentBatch); + + // A flush event is queued after write #3. + assertEquals(3, processor.size()); + + // Verify the state. Records are replayed but no batch written. assertEquals(0L, ctx.coordinator.lastWrittenOffset()); assertEquals(0L, ctx.coordinator.lastCommittedOffset()); assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); assertEquals(List.of( - new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), - new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), - new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)) + new MockCoordinatorShard.RecordAndMetadata(0, "record1") ), ctx.coordinator.coordinator().fullRecords()); assertEquals(List.of(), writer.entries(TP)); - // Write #4. This write cannot make it in the current batch. So the current batch - // is flushed. It will fail. So we expect all writes to fail. - CompletableFuture write4 = runtime.scheduleWriteOperation("write#4", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(records.subList(3, 4), "response4", null, true, false)); + // Execute transactional write #2. + processor.poll(); - // Verify the futures. - assertFutureThrows(KafkaException.class, write1); - assertFutureThrows(KafkaException.class, write2); - assertFutureThrows(KafkaException.class, write3); - // Write #4 is also expected to fail. - assertFutureThrows(KafkaException.class, write4); + // The batch is flushed. + assertNull(ctx.currentBatch); - // Verify the state. The state should be reverted to the initial state. - assertEquals(0L, ctx.coordinator.lastWrittenOffset()); - assertEquals(0L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); - assertEquals(List.of(), ctx.coordinator.coordinator().fullRecords()); - assertEquals(List.of(), writer.entries(TP)); - } + // No flush event is queued. + assertEquals(2, processor.size()); - @Test - public void testEmptyBatch() throws Exception { - MockTimer timer = new MockTimer(); - MockPartitionWriter writer = new MockPartitionWriter(); - ThrowingSerializer serializer = new ThrowingSerializer(new StringSerializer()); + // Verify the state. The current batch and the transactional records are + // written to the log. + assertEquals(2L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L, 1L, 2L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of( + new MockCoordinatorShard.RecordAndMetadata(0, "record1") + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of( + TestUtil.records(timer.time().milliseconds(), "record1"), + TestUtil.transactionalRecords(100L, (short) 50, timer.time().milliseconds(), "record2") + ), writer.entries(TP)); + + // Execute write #3. + processor.poll(); + + // A batch has been created. + assertNotNull(ctx.currentBatch); + + // A flush event is queued after the first flush. + assertEquals(2, processor.size()); + + // Verify the state. Records are replayed but no batch written. + assertEquals(2L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L, 1L, 2L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of( + new MockCoordinatorShard.RecordAndMetadata(0, "record1"), + new MockCoordinatorShard.RecordAndMetadata(2, "record3") + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of( + TestUtil.records(timer.time().milliseconds(), "record1"), + TestUtil.transactionalRecords(100L, (short) 50, timer.time().milliseconds(), "record2") + ), writer.entries(TP)); + + // Execute the first flush. + processor.poll(); + assertEquals(1, processor.size()); + + // The flush does not belong to the current batch and is ignored. + assertNotNull(ctx.currentBatch); + + // Execute the second flush. + processor.poll(); + assertEquals(0, processor.size()); + + // The batch is flushed. + assertNull(ctx.currentBatch); + + // Verify the state. + assertEquals(3L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L, 1L, 2L, 3L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of( + new MockCoordinatorShard.RecordAndMetadata(0, "record1"), + new MockCoordinatorShard.RecordAndMetadata(2, "record3") + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of( + TestUtil.records(timer.time().milliseconds(), "record1"), + TestUtil.transactionalRecords(100L, (short) 50, timer.time().milliseconds(), "record2"), + TestUtil.records(timer.time().milliseconds(), "record3") + ), writer.entries(TP)); + } + + @Test + public void testStateMachineIsReloadedWhenOutOfSync() { + MockTimer timer = new MockTimer(); + MockCoordinatorLoader loader = spy(new MockCoordinatorLoader()); + MockPartitionWriter writer = new MockPartitionWriter() { + @Override + public long append( + TopicPartition tp, + VerificationGuard verificationGuard, + MemoryRecords batch + ) { + // Add 1 to the returned offsets. + return super.append(tp, verificationGuard, batch) + 1; + } + }; CoordinatorRuntime runtime = new CoordinatorRuntime.Builder() .withTime(timer.time()) .withTimer(timer) .withDefaultWriteTimeOut(Duration.ofMillis(20)) - .withLoader(new MockCoordinatorLoader()) + .withLoader(loader) .withEventProcessor(new DirectEventProcessor()) .withPartitionWriter(writer) .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) - .withSerializer(serializer) + .withSerializer(new StringSerializer()) .withAppendLingerMs(10) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -4144,53 +4135,148 @@ public void testEmptyBatch() throws Exception { // Verify the initial state. CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertEquals(ACTIVE, ctx.state); + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); assertNull(ctx.currentBatch); - // Write #1, which fails. - serializer.throwOnNextOperation(); - CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(List.of("1"), "response1")); + // Keep a reference to the current coordinator. + SnapshottableCoordinator coordinator = ctx.coordinator; - // Write #1 should fail and leave an empty batch. - assertFutureThrows(BufferOverflowException.class, write1); - assertNotNull(ctx.currentBatch); + // Get the max batch size. + int maxBatchSize = writer.config(TP).maxMessageSize(); - // Write #2, with no records. + // Create records with a quarter of the max batch size each. Keep in mind that + // each batch has a header so it is not possible to have those four records + // in one single batch. + List records = Stream.of('1', '2', '3', '4').map(c -> { + char[] payload = new char[maxBatchSize / 4]; + Arrays.fill(payload, c); + return new String(payload); + }).collect(Collectors.toList()); + + // Write #1. + CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(0, 1), "response1")); + + // Write #2. CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(List.of(), "response2")); + state -> new CoordinatorResult<>(records.subList(1, 2), "response2")); - // Write #2 should not be attached to the empty batch. - assertTrue(write2.isDone()); - assertEquals("response2", write2.get(5, TimeUnit.SECONDS)); + // Write #3. + CompletableFuture write3 = runtime.scheduleWriteOperation("write#3", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(2, 3), "response3")); - // Complete transaction #1. It will flush the current empty batch. - // The coordinator must not try to write an empty batch, otherwise the mock partition writer - // will throw an exception. - CompletableFuture complete1 = runtime.scheduleTransactionCompletion( - "complete#1", - TP, - 100L, - (short) 50, - 10, - TransactionResult.COMMIT, - DEFAULT_WRITE_TIMEOUT - ); + // Write #4. This write cannot make it in the current batch. So the current batch + // is flushed. It will fail. So we expect all writes to fail. + CompletableFuture write4 = runtime.scheduleWriteOperation("write#4", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(3, 4), "response4")); - // Verify that the completion is not committed yet. - assertFalse(complete1.isDone()); + // Verify the futures. + assertFutureThrows(NotCoordinatorException.class, write1); + assertFutureThrows(NotCoordinatorException.class, write2); + assertFutureThrows(NotCoordinatorException.class, write3); + // Write #4 is also expected to fail. + assertFutureThrows(NotCoordinatorException.class, write4); - // Commit and verify that writes are completed. - writer.commit(TP); - assertNull(complete1.get(5, TimeUnit.SECONDS)); + // Verify that the state machine was loaded twice. + verify(loader, times(2)).load(eq(TP), any()); + + // Verify that the state is active and that the state machine + // is actually a new one. + assertEquals(ACTIVE, ctx.state); + assertNotEquals(coordinator, ctx.coordinator); } @Test - public void testRecordFlushTime() throws Exception { + public void testWriteOpIsNotReleasedWhenStateMachineIsNotCaughtUpAfterLoad() throws ExecutionException, InterruptedException, TimeoutException { MockTimer timer = new MockTimer(); + MockPartitionWriter writer = new MockPartitionWriter(); + CoordinatorLoader loader = new CoordinatorLoader() { + @Override + public CompletableFuture load( + TopicPartition tp, + CoordinatorPlayback coordinator + ) { + coordinator.replay( + 0, + RecordBatch.NO_PRODUCER_ID, + RecordBatch.NO_PRODUCER_EPOCH, + "record#0" + ); - // Writer sleeps for 10ms before appending records. - MockPartitionWriter writer = new MockPartitionWriter(timer.time(), Integer.MAX_VALUE, false); - CoordinatorRuntimeMetrics runtimeMetrics = mock(CoordinatorRuntimeMetrics.class); + coordinator.replay( + 0, + RecordBatch.NO_PRODUCER_ID, + RecordBatch.NO_PRODUCER_EPOCH, + "record#1" + ); + + coordinator.updateLastWrittenOffset(2L); + coordinator.updateLastCommittedOffset(1L); + + return CompletableFuture.completedFuture(new LoadSummary( + 0L, + 0L, + 0L, + 2, + 1 + )); + } + + @Override + public void close() {} + }; + + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(timer.time()) + .withTimer(timer) + .withDefaultWriteTimeOut(Duration.ofMillis(20)) + .withLoader(loader) + .withEventProcessor(new DirectEventProcessor()) + .withPartitionWriter(writer) + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) + .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) + .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) + .withSerializer(new StringSerializer()) + .withAppendLingerMs(10) + .withExecutorService(mock(ExecutorService.class)) + .build(); + + // Schedule the loading. + runtime.scheduleLoadOperation(TP, 10); + + // Verify the initial state. + CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertEquals(2L, ctx.coordinator.lastWrittenOffset()); + assertEquals(1L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(2L), ctx.coordinator.snapshotRegistry().epochsList()); + + // Schedule a write operation that does not generate any records. + CompletableFuture write = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(List.of(), "response1")); + + // The write operation should not be done. + assertFalse(write.isDone()); + + // Advance the last committed offset. + ctx.highWatermarklistener.onHighWatermarkUpdated(TP, 2L); + + // Verify the state. + assertEquals(2L, ctx.coordinator.lastWrittenOffset()); + assertEquals(2L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(2L), ctx.coordinator.snapshotRegistry().epochsList()); + + // The write operation should be completed. + assertEquals("response1", write.get(5, TimeUnit.SECONDS)); + } + + @Test + public void testScheduleNonAtomicWriteOperation() throws ExecutionException, InterruptedException, TimeoutException { + MockTimer timer = new MockTimer(); + MockPartitionWriter writer = new MockPartitionWriter(); CoordinatorRuntime runtime = new CoordinatorRuntime.Builder() @@ -4201,7 +4287,7 @@ public void testRecordFlushTime() throws Exception { .withEventProcessor(new DirectEventProcessor()) .withPartitionWriter(writer) .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) - .withCoordinatorRuntimeMetrics(runtimeMetrics) + .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) .withAppendLingerMs(10) @@ -4213,6 +4299,9 @@ public void testRecordFlushTime() throws Exception { // Verify the initial state. CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); assertNull(ctx.currentBatch); // Get the max batch size. @@ -4227,35 +4316,29 @@ public void testRecordFlushTime() throws Exception { return new String(payload); }).collect(Collectors.toList()); - // Write #1 with two records. - long firstBatchTimestamp = timer.time().milliseconds(); - CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(50), - state -> new CoordinatorResult<>(records.subList(0, 2), "response1") - ); - - // A batch has been created. - assertNotNull(ctx.currentBatch); - - // Write #2 with one record. - CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(50), - state -> new CoordinatorResult<>(records.subList(2, 3), "response2") + // Let's try to write all the records atomically (the default) to ensure + // that it fails. + CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records, "write#1") ); - // Verify the state. Records are replayed but no batch written. - assertEquals(List.of(), writer.entries(TP)); - verify(runtimeMetrics, times(0)).recordFlushTime(10); + assertFutureThrows(RecordTooLargeException.class, write1); - // Write #3 with one record. This one cannot go into the existing batch - // so the existing batch should be flushed and a new one should be created. - long secondBatchTimestamp = timer.time().milliseconds(); - CompletableFuture write3 = runtime.scheduleWriteOperation("write#3", TP, Duration.ofMillis(50), - state -> new CoordinatorResult<>(records.subList(3, 4), "response3") + // Let's try to write the same records non-atomically. + CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records, "write#2", null, true, false) ); - // Verify the state. Records are replayed. The previous batch - // got flushed with all the records but the new one from #3. + // The write is pending. + assertFalse(write2.isDone()); + + // Verify the state. + assertNotNull(ctx.currentBatch); + // The last written offset is 3L because one batch was written to the log with + // the first three records. The 4th one is pending. assertEquals(3L, ctx.coordinator.lastWrittenOffset()); assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L, 3L), ctx.coordinator.snapshotRegistry().epochsList()); assertEquals(List.of( new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), @@ -4263,16 +4346,23 @@ public void testRecordFlushTime() throws Exception { new MockCoordinatorShard.RecordAndMetadata(3, records.get(3)) ), ctx.coordinator.coordinator().fullRecords()); assertEquals(List.of( - records(firstBatchTimestamp, records.subList(0, 3)) + records(timer.time().milliseconds(), records.subList(0, 3)) ), writer.entries(TP)); - verify(runtimeMetrics, times(1)).recordFlushTime(10); - // Advance past the linger time. + // Commit up to 3L. + writer.commit(TP, 3L); + + // The write is still pending. + assertFalse(write2.isDone()); + + // Advance past the linger time to flush the pending batch. timer.advanceClock(11); - // Verify the state. The pending batch is flushed. + // Verify the state. + assertNull(ctx.currentBatch); assertEquals(4L, ctx.coordinator.lastWrittenOffset()); - assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(3L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(3L, 4L), ctx.coordinator.snapshotRegistry().epochsList()); assertEquals(List.of( new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), @@ -4280,27 +4370,22 @@ public void testRecordFlushTime() throws Exception { new MockCoordinatorShard.RecordAndMetadata(3, records.get(3)) ), ctx.coordinator.coordinator().fullRecords()); assertEquals(List.of( - records(secondBatchTimestamp, records.subList(0, 3)), - records(secondBatchTimestamp, records.subList(3, 4)) + records(timer.time().milliseconds() - 11, records.subList(0, 3)), + records(timer.time().milliseconds() - 11, records.subList(3, 4)) ), writer.entries(TP)); - verify(runtimeMetrics, times(2)).recordFlushTime(10); - // Commit and verify that writes are completed. - writer.commit(TP); - assertTrue(write1.isDone()); + // Commit up to 4L. + writer.commit(TP, 4L); + + // Verify that the write is completed. assertTrue(write2.isDone()); - assertTrue(write3.isDone()); - assertEquals(4L, ctx.coordinator.lastCommittedOffset()); - assertEquals("response1", write1.get(5, TimeUnit.SECONDS)); - assertEquals("response2", write2.get(5, TimeUnit.SECONDS)); - assertEquals("response3", write3.get(5, TimeUnit.SECONDS)); + assertEquals("write#2", write2.get(5, TimeUnit.SECONDS)); } @Test - public void testCompressibleRecordTriggersFlushAndSucceeds() throws Exception { + public void testScheduleNonAtomicWriteOperationWithRecordTooLarge() throws InterruptedException { MockTimer timer = new MockTimer(); MockPartitionWriter writer = new MockPartitionWriter(); - Compression compression = Compression.gzip().build(); CoordinatorRuntime runtime = new CoordinatorRuntime.Builder() @@ -4313,7 +4398,6 @@ public void testCompressibleRecordTriggersFlushAndSucceeds() throws Exception { .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) - .withCompression(compression) .withSerializer(new StringSerializer()) .withAppendLingerMs(10) .withExecutorService(mock(ExecutorService.class)) @@ -4324,69 +4408,80 @@ public void testCompressibleRecordTriggersFlushAndSucceeds() throws Exception { // Verify the initial state. CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); assertNull(ctx.currentBatch); // Get the max batch size. int maxBatchSize = writer.config(TP).maxMessageSize(); - // Create 2 records with a quarter of the max batch size each. - List records = Stream.of('1', '2').map(c -> { + // Create records with a quarter of the max batch size each. Keep in mind that + // each batch has a header so it is not possible to have those four records + // in one single batch. + List records = Stream.of('1', '2', '3').map(c -> { char[] payload = new char[maxBatchSize / 4]; Arrays.fill(payload, c); return new String(payload); }).collect(Collectors.toList()); - // Write #1 with the small records, batch will be about half full - long firstBatchTimestamp = timer.time().milliseconds(); - CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(50), - state -> new CoordinatorResult<>(records, "response1") - ); + // Create another record larger than the max batch size. + char[] payload = new char[maxBatchSize]; + Arrays.fill(payload, '4'); + String record = new String(payload); - // A batch has been created. - assertNotNull(ctx.currentBatch); + // Let's write the first three records. + CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records, "write#1", null, true, false) + ); - // Verify the state - batch is not yet flushed + // Verify the state. + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of( + new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), + new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), + new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)) + ), ctx.coordinator.coordinator().fullRecords()); assertEquals(List.of(), writer.entries(TP)); - // Create a record of highly compressible data - List largeRecord = List.of("a".repeat((int) (0.75 * maxBatchSize))); - - // Write #2 with the large record. This record is too large to go into the previous batch - // uncompressed but fits in a new buffer, so we should flush the previous batch and allocate - // a new one. - CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(50), - state -> new CoordinatorResult<>(largeRecord, "response2") + // Let's write the 4th record which is too large. This will flush the current + // pending batch, allocate a new batch, and put the record into it. + // Note that the batch will fail only when the batch is written because the + // MemoryBatchBuilder always accept one record. + CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(List.of(record), "write#2", null, true, false) ); - // Verify the state. The first batch has flushed but the second is pending. - assertEquals(2L, ctx.coordinator.lastWrittenOffset()); + // Advance past the linger time to flush the pending batch. + timer.advanceClock(11); + + // The write should have failed... + assertFutureThrows(RecordTooLargeException.class, write2); + + // ... but write#1 should be left intact. + assertFalse(write1.isDone()); + + // Verify the state. + assertEquals(3L, ctx.coordinator.lastWrittenOffset()); assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L, 3L), ctx.coordinator.snapshotRegistry().epochsList()); assertEquals(List.of( new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), - new MockCoordinatorShard.RecordAndMetadata(2, largeRecord.get(0)) + new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)) ), ctx.coordinator.coordinator().fullRecords()); assertEquals(List.of( - records(firstBatchTimestamp, compression, records) + records(timer.time().milliseconds() - 11, records.subList(0, 3)) ), writer.entries(TP)); - - // Advance past the linger time - timer.advanceClock(11); - - // Commit and verify that the second batch is completed - writer.commit(TP); - assertTrue(write1.isDone()); - assertTrue(write2.isDone()); - assertEquals(3L, ctx.coordinator.lastCommittedOffset()); - assertEquals("response1", write1.get(5, TimeUnit.SECONDS)); - assertEquals("response2", write2.get(5, TimeUnit.SECONDS)); } @Test - public void testLargeCompressibleRecordTriggersFlushAndSucceeds() throws Exception { + public void testScheduleNonAtomicWriteOperationWhenWriteFails() { MockTimer timer = new MockTimer(); - MockPartitionWriter writer = new MockPartitionWriter(); - Compression compression = Compression.gzip().build(); + // The partition writer does not accept any writes. + MockPartitionWriter writer = new MockPartitionWriter(0); CoordinatorRuntime runtime = new CoordinatorRuntime.Builder() @@ -4399,7 +4494,6 @@ public void testLargeCompressibleRecordTriggersFlushAndSucceeds() throws Excepti .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) - .withCompression(compression) .withSerializer(new StringSerializer()) .withAppendLingerMs(10) .withExecutorService(mock(ExecutorService.class)) @@ -4410,69 +4504,71 @@ public void testLargeCompressibleRecordTriggersFlushAndSucceeds() throws Excepti // Verify the initial state. CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); assertNull(ctx.currentBatch); // Get the max batch size. int maxBatchSize = writer.config(TP).maxMessageSize(); - // Create 2 records with a quarter of the max batch size each. - List records = Stream.of('1', '2').map(c -> { + // Create records with a quarter of the max batch size each. Keep in mind that + // each batch has a header so it is not possible to have those four records + // in one single batch. + List records = Stream.of('1', '2', '3', '4').map(c -> { char[] payload = new char[maxBatchSize / 4]; Arrays.fill(payload, c); return new String(payload); }).collect(Collectors.toList()); - // Write #1 with the small records, batch will be about half full - long firstBatchTimestamp = timer.time().milliseconds(); - CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(50), - state -> new CoordinatorResult<>(records, "response1") - ); - - // A batch has been created. - assertNotNull(ctx.currentBatch); - - // Verify the state - batch is not yet flushed - assertEquals(List.of(), writer.entries(TP)); + // Write #1. + CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(0, 1), "response1", null, true, false)); - // Create a large record of highly compressible data - List largeRecord = List.of("a".repeat(3 * maxBatchSize)); + // Write #2. + CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(1, 2), "response2", null, true, false)); - // Write #2 with the large record. This record is too large to go into the previous batch - // uncompressed but will fit in the new buffer once compressed, so we should flush the - // previous batch and successfully allocate a new batch for this record. The new batch - // will also trigger an immediate flush. - long secondBatchTimestamp = timer.time().milliseconds(); - CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(50), - state -> new CoordinatorResult<>(largeRecord, "response2") - ); + // Write #3. + CompletableFuture write3 = runtime.scheduleWriteOperation("write#3", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(2, 3), "response3", null, true, false)); // Verify the state. - assertEquals(3L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); assertEquals(List.of( new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), - new MockCoordinatorShard.RecordAndMetadata(2, largeRecord.get(0)) + new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)) ), ctx.coordinator.coordinator().fullRecords()); - assertEquals(List.of( - records(firstBatchTimestamp, compression, records), - records(secondBatchTimestamp, compression, largeRecord) - ), writer.entries(TP)); + assertEquals(List.of(), writer.entries(TP)); - // Commit and verify that writes are completed. - writer.commit(TP); - assertTrue(write1.isDone()); - assertTrue(write2.isDone()); - assertEquals(3L, ctx.coordinator.lastCommittedOffset()); - assertEquals("response1", write1.get(5, TimeUnit.SECONDS)); - assertEquals("response2", write2.get(5, TimeUnit.SECONDS)); + // Write #4. This write cannot make it in the current batch. So the current batch + // is flushed. It will fail. So we expect all writes to fail. + CompletableFuture write4 = runtime.scheduleWriteOperation("write#4", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(records.subList(3, 4), "response4", null, true, false)); + + // Verify the futures. + assertFutureThrows(KafkaException.class, write1); + assertFutureThrows(KafkaException.class, write2); + assertFutureThrows(KafkaException.class, write3); + // Write #4 is also expected to fail. + assertFutureThrows(KafkaException.class, write4); + + // Verify the state. The state should be reverted to the initial state. + assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); + assertEquals(List.of(), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of(), writer.entries(TP)); } @Test - public void testLargeUncompressibleRecordTriggersFlushAndFails() throws Exception { + public void testEmptyBatch() throws Exception { MockTimer timer = new MockTimer(); MockPartitionWriter writer = new MockPartitionWriter(); - Compression compression = Compression.gzip().build(); + ThrowingSerializer serializer = new ThrowingSerializer(new StringSerializer()); CoordinatorRuntime runtime = new CoordinatorRuntime.Builder() @@ -4485,8 +4581,7 @@ public void testLargeUncompressibleRecordTriggersFlushAndFails() throws Exceptio .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) - .withCompression(compression) - .withSerializer(new StringSerializer()) + .withSerializer(serializer) .withAppendLingerMs(10) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -4498,393 +4593,338 @@ public void testLargeUncompressibleRecordTriggersFlushAndFails() throws Exceptio CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); assertNull(ctx.currentBatch); - // Get the max batch size. - int maxBatchSize = writer.config(TP).maxMessageSize(); + // Write #1, which fails. + serializer.throwOnNextOperation(); + CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(List.of("1"), "response1")); - // Create 2 records with a quarter of the max batch size each. - List records = Stream.of('1', '2').map(c -> { - char[] payload = new char[maxBatchSize / 4]; - Arrays.fill(payload, c); - return new String(payload); - }).collect(Collectors.toList()); + // Write #1 should fail and leave an empty batch. + assertFutureThrows(BufferOverflowException.class, write1); + assertNotNull(ctx.currentBatch); - // Write #1 with the small records, batch will be about half full - long firstBatchTimestamp = timer.time().milliseconds(); - CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(50), - state -> new CoordinatorResult<>(records, "response1") - ); - - // A batch has been created. - assertNotNull(ctx.currentBatch); - - // Verify the state - batch is not yet flushed - assertEquals(List.of(), writer.entries(TP)); + // Write #2, with no records. + CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(List.of(), "response2")); - // Create a large record of not very compressible data - char[] payload = new char[3 * maxBatchSize]; - Random offset = new Random(); - for (int i = 0; i < payload.length; i++) { - payload[i] = (char) ('a' + ((char) offset.nextInt() % 26)); - } - List largeRecord = List.of(new String(payload)); + // Write #2 should not be attached to the empty batch. + assertTrue(write2.isDone()); + assertEquals("response2", write2.get(5, TimeUnit.SECONDS)); - // Write #2 with the large record. This record is too large to go into the previous batch - // and is not compressible so it should be flushed. It is also too large to fit in a new batch - // so the write should fail with RecordTooLargeException - CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(50), - state -> new CoordinatorResult<>(largeRecord, "response2") + // Complete transaction #1. It will flush the current empty batch. + // The coordinator must not try to write an empty batch, otherwise the mock partition writer + // will throw an exception. + CompletableFuture complete1 = runtime.scheduleTransactionCompletion( + "complete#1", + TP, + 100L, + (short) 50, + 10, + TransactionResult.COMMIT, + DEFAULT_WRITE_TIMEOUT ); - // Check that write2 fails with RecordTooLargeException - assertFutureThrows(RecordTooLargeException.class, write2); - - // Verify the state. The first batch was flushed and the largeRecord - // write failed. - assertEquals(2L, ctx.coordinator.lastWrittenOffset()); - assertEquals(0L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of( - new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), - new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)) - ), ctx.coordinator.coordinator().fullRecords()); - assertEquals(List.of( - records(firstBatchTimestamp, compression, records) - ), writer.entries(TP)); + // Verify that the completion is not committed yet. + assertFalse(complete1.isDone()); // Commit and verify that writes are completed. writer.commit(TP); - assertTrue(write1.isDone()); - assertTrue(write2.isDone()); - assertEquals(2L, ctx.coordinator.lastCommittedOffset()); - assertEquals("response1", write1.get(5, TimeUnit.SECONDS)); - } + assertNull(complete1.get(5, TimeUnit.SECONDS)); + } @Test - public void testRecordEventPurgatoryTime() throws Exception { - Duration writeTimeout = Duration.ofMillis(1000); + public void testRecordFlushTime() throws Exception { MockTimer timer = new MockTimer(); - MockPartitionWriter writer = new MockPartitionWriter(); - ManualEventProcessor processor = new ManualEventProcessor(); + + // Writer sleeps for 10ms before appending records. + MockPartitionWriter writer = new MockPartitionWriter(timer.time(), Integer.MAX_VALUE, false); CoordinatorRuntimeMetrics runtimeMetrics = mock(CoordinatorRuntimeMetrics.class); CoordinatorRuntime runtime = new CoordinatorRuntime.Builder() .withTime(timer.time()) .withTimer(timer) - .withDefaultWriteTimeOut(writeTimeout) + .withDefaultWriteTimeOut(Duration.ofMillis(20)) .withLoader(new MockCoordinatorLoader()) - .withEventProcessor(processor) + .withEventProcessor(new DirectEventProcessor()) .withPartitionWriter(writer) .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) .withCoordinatorRuntimeMetrics(runtimeMetrics) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(0) + .withAppendLingerMs(10) .withExecutorService(mock(ExecutorService.class)) .build(); - // Loads the coordinator. Poll once to execute the load operation and once - // to complete the load. + // Schedule the loading. runtime.scheduleLoadOperation(TP, 10); - processor.poll(); - processor.poll(); - // write#1 will be committed and update the high watermark. Record time spent in purgatory. - CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, writeTimeout, - state -> new CoordinatorResult<>(List.of("record1"), "response1") + // Verify the initial state. + CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertNull(ctx.currentBatch); + + // Get the max batch size. + int maxBatchSize = writer.config(TP).maxMessageSize(); + + // Create records with a quarter of the max batch size each. Keep in mind that + // each batch has a header so it is not possible to have those four records + // in one single batch. + List records = Stream.of('1', '2', '3', '4').map(c -> { + char[] payload = new char[maxBatchSize / 4]; + Arrays.fill(payload, c); + return new String(payload); + }).collect(Collectors.toList()); + + // Write #1 with two records. + long firstBatchTimestamp = timer.time().milliseconds(); + CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(50), + state -> new CoordinatorResult<>(records.subList(0, 2), "response1") ); - // write#2 will time out sitting in the purgatory. Record time spent in purgatory. - CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, writeTimeout, - state -> new CoordinatorResult<>(List.of("record2"), "response2") + + // A batch has been created. + assertNotNull(ctx.currentBatch); + + // Write #2 with one record. + CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(50), + state -> new CoordinatorResult<>(records.subList(2, 3), "response2") ); - // write#3 will error while appending. Does not spend time in purgatory. - CompletableFuture write3 = runtime.scheduleWriteOperation("write#3", TP, writeTimeout, - state -> { - throw new KafkaException("write#3 failed."); - }); - processor.poll(); - processor.poll(); - processor.poll(); + // Verify the state. Records are replayed but no batch written. + assertEquals(List.of(), writer.entries(TP)); + verify(runtimeMetrics, times(0)).recordFlushTime(10); - // Confirm we do not record purgatory time for write#3. - assertTrue(write3.isCompletedExceptionally()); - verify(runtimeMetrics, times(0)).recordEventPurgatoryTime(0L); + // Write #3 with one record. This one cannot go into the existing batch + // so the existing batch should be flushed and a new one should be created. + long secondBatchTimestamp = timer.time().milliseconds(); + CompletableFuture write3 = runtime.scheduleWriteOperation("write#3", TP, Duration.ofMillis(50), + state -> new CoordinatorResult<>(records.subList(3, 4), "response3") + ); - // Records have been written to the log. - long writeTimestamp = timer.time().milliseconds(); + // Verify the state. Records are replayed. The previous batch + // got flushed with all the records but the new one from #3. + assertEquals(3L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); assertEquals(List.of( - records(writeTimestamp, "record1"), - records(writeTimestamp, "record2") + new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), + new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), + new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)), + new MockCoordinatorShard.RecordAndMetadata(3, records.get(3)) + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of( + records(firstBatchTimestamp, records.subList(0, 3)) ), writer.entries(TP)); + verify(runtimeMetrics, times(1)).recordFlushTime(10); - // There is no pending high watermark. - assertEquals(NO_OFFSET, runtime.contextOrThrow(TP).highWatermarklistener.lastHighWatermark()); - - // Advance the clock then commit records from write#1. - timer.advanceClock(700); - writer.commit(TP, 1); - - // We should still have one pending event and the pending high watermark should be updated. - assertEquals(1, processor.size()); - assertEquals(1, runtime.contextOrThrow(TP).highWatermarklistener.lastHighWatermark()); - - // Poll once to process the high watermark update and complete the writes. - processor.poll(); - long purgatoryTimeMs = timer.time().milliseconds() - writeTimestamp; + // Advance past the linger time. + timer.advanceClock(11); - // Advance the clock past write timeout. write#2 has now timed out. - timer.advanceClock(300 + 1); - processor.poll(); + // Verify the state. The pending batch is flushed. + assertEquals(4L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of( + new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), + new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), + new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)), + new MockCoordinatorShard.RecordAndMetadata(3, records.get(3)) + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of( + records(secondBatchTimestamp, records.subList(0, 3)), + records(secondBatchTimestamp, records.subList(3, 4)) + ), writer.entries(TP)); + verify(runtimeMetrics, times(2)).recordFlushTime(10); - assertEquals(NO_OFFSET, runtime.contextOrThrow(TP).highWatermarklistener.lastHighWatermark()); - assertEquals(1, runtime.contextOrThrow(TP).coordinator.lastCommittedOffset()); + // Commit and verify that writes are completed. + writer.commit(TP); assertTrue(write1.isDone()); - assertTrue(write2.isCompletedExceptionally()); - verify(runtimeMetrics, times(1)).recordEventPurgatoryTime(purgatoryTimeMs); - verify(runtimeMetrics, times(1)).recordEventPurgatoryTime(writeTimeout.toMillis() + 1); + assertTrue(write2.isDone()); + assertTrue(write3.isDone()); + assertEquals(4L, ctx.coordinator.lastCommittedOffset()); + assertEquals("response1", write1.get(5, TimeUnit.SECONDS)); + assertEquals("response2", write2.get(5, TimeUnit.SECONDS)); + assertEquals("response3", write3.get(5, TimeUnit.SECONDS)); } @Test - public void testWriteEventCompletesOnlyOnce() throws Exception { - // Completes once via timeout, then again with HWM update. - Duration writeTimeout = Duration.ofMillis(1000L); + public void testCompressibleRecordTriggersFlushAndSucceeds() throws Exception { MockTimer timer = new MockTimer(); MockPartitionWriter writer = new MockPartitionWriter(); - ManualEventProcessor processor = new ManualEventProcessor(); - CoordinatorRuntimeMetrics runtimeMetrics = mock(CoordinatorRuntimeMetrics.class); + Compression compression = Compression.gzip().build(); CoordinatorRuntime runtime = new CoordinatorRuntime.Builder() .withTime(timer.time()) .withTimer(timer) - .withDefaultWriteTimeOut(writeTimeout) + .withDefaultWriteTimeOut(Duration.ofMillis(20)) .withLoader(new MockCoordinatorLoader()) - .withEventProcessor(processor) + .withEventProcessor(new DirectEventProcessor()) .withPartitionWriter(writer) .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) - .withCoordinatorRuntimeMetrics(runtimeMetrics) + .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) + .withCompression(compression) .withSerializer(new StringSerializer()) - .withAppendLingerMs(0) + .withAppendLingerMs(10) .withExecutorService(mock(ExecutorService.class)) .build(); - // Loads the coordinator. Poll once to execute the load operation and once - // to complete the load. + // Schedule the loading. runtime.scheduleLoadOperation(TP, 10); - processor.poll(); - processor.poll(); - // write#1 will be committed and update the high watermark. Record time spent in purgatory. - CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, writeTimeout, - state -> new CoordinatorResult<>(List.of("record1"), "response1") - ); + // Verify the initial state. + CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertNull(ctx.currentBatch); - processor.poll(); + // Get the max batch size. + int maxBatchSize = writer.config(TP).maxMessageSize(); - // Records have been written to the log. - long writeTimestamp = timer.time().milliseconds(); - assertEquals(List.of( - records(writeTimestamp, "record1") - ), writer.entries(TP)); + // Create 2 records with a quarter of the max batch size each. + List records = Stream.of('1', '2').map(c -> { + char[] payload = new char[maxBatchSize / 4]; + Arrays.fill(payload, c); + return new String(payload); + }).collect(Collectors.toList()); - // There is no pending high watermark. - assertEquals(NO_OFFSET, runtime.contextOrThrow(TP).highWatermarklistener.lastHighWatermark()); + // Write #1 with the small records, batch will be about half full + long firstBatchTimestamp = timer.time().milliseconds(); + CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(50), + state -> new CoordinatorResult<>(records, "response1") + ); - // Advance the clock to time out the write event. Confirm write#1 is completed with a timeout. - timer.advanceClock(writeTimeout.toMillis() + 1L); - processor.poll(); - verify(runtimeMetrics, times(1)).recordEventPurgatoryTime(writeTimeout.toMillis() + 1); - assertTrue(write1.isCompletedExceptionally()); + // A batch has been created. + assertNotNull(ctx.currentBatch); - // HWM update - writer.commit(TP, 1); - assertEquals(1, processor.size()); - assertEquals(1, runtime.contextOrThrow(TP).highWatermarklistener.lastHighWatermark()); + // Verify the state - batch is not yet flushed + assertEquals(List.of(), writer.entries(TP)); - // Poll once to process the high watermark update and complete write#1. It has already - // been completed and this is a noop. - processor.poll(); - - assertEquals(NO_OFFSET, runtime.contextOrThrow(TP).highWatermarklistener.lastHighWatermark()); - assertEquals(1, runtime.contextOrThrow(TP).coordinator.lastCommittedOffset()); - assertTrue(write1.isCompletedExceptionally()); - verify(runtimeMetrics, times(1)).recordEventPurgatoryTime(writeTimeout.toMillis() + 1L); - } - - @Test - public void testCompleteTransactionEventCompletesOnlyOnce() throws Exception { - // Completes once via timeout, then again with HWM update. - Duration writeTimeout = Duration.ofMillis(1000L); - MockTimer timer = new MockTimer(); - MockPartitionWriter writer = new MockPartitionWriter(); - ManualEventProcessor processor = new ManualEventProcessor(); - CoordinatorRuntimeMetrics runtimeMetrics = mock(CoordinatorRuntimeMetrics.class); - - CoordinatorRuntime runtime = - new CoordinatorRuntime.Builder() - .withTime(timer.time()) - .withTimer(timer) - .withDefaultWriteTimeOut(writeTimeout) - .withLoader(new MockCoordinatorLoader()) - .withEventProcessor(processor) - .withPartitionWriter(writer) - .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) - .withCoordinatorRuntimeMetrics(runtimeMetrics) - .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) - .withSerializer(new StringSerializer()) - .withExecutorService(mock(ExecutorService.class)) - .build(); - - // Loads the coordinator. Poll once to execute the load operation and once - // to complete the load. - runtime.scheduleLoadOperation(TP, 10); - processor.poll(); - processor.poll(); + // Create a record of highly compressible data + List largeRecord = List.of("a".repeat((int) (0.75 * maxBatchSize))); - // transaction completion. - CompletableFuture write1 = runtime.scheduleTransactionCompletion( - "transactional-write", - TP, - 100L, - (short) 50, - 1, - TransactionResult.COMMIT, - writeTimeout + // Write #2 with the large record. This record is too large to go into the previous batch + // uncompressed but fits in a new buffer, so we should flush the previous batch and allocate + // a new one. + CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(50), + state -> new CoordinatorResult<>(largeRecord, "response2") ); - processor.poll(); - // Records have been written to the log. + // Verify the state. The first batch has flushed but the second is pending. + assertEquals(2L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); assertEquals(List.of( - endTransactionMarker(100, (short) 50, timer.time().milliseconds(), 1, ControlRecordType.COMMIT) + new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), + new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), + new MockCoordinatorShard.RecordAndMetadata(2, largeRecord.get(0)) + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of( + records(firstBatchTimestamp, compression, records) ), writer.entries(TP)); - // The write timeout tasks exist. - assertEquals(1, timer.size()); - assertFalse(write1.isDone()); - - // Advance the clock to time out the write event. Confirm write#1 is completed with a timeout. - timer.advanceClock(writeTimeout.toMillis() + 1L); - processor.poll(); - verify(runtimeMetrics, times(1)).recordEventPurgatoryTime(writeTimeout.toMillis() + 1); - assertTrue(write1.isCompletedExceptionally()); - - // HWM update - writer.commit(TP, 1); - assertEquals(1, processor.size()); - assertEquals(1, runtime.contextOrThrow(TP).highWatermarklistener.lastHighWatermark()); - - // Poll once to process the high watermark update and complete write#1. It has already - // been completed and this is a noop. - processor.poll(); + // Advance past the linger time + timer.advanceClock(11); - assertEquals(NO_OFFSET, runtime.contextOrThrow(TP).highWatermarklistener.lastHighWatermark()); - assertEquals(1, runtime.contextOrThrow(TP).coordinator.lastCommittedOffset()); - assertTrue(write1.isCompletedExceptionally()); - verify(runtimeMetrics, times(1)).recordEventPurgatoryTime(writeTimeout.toMillis() + 1L); + // Commit and verify that the second batch is completed + writer.commit(TP); + assertTrue(write1.isDone()); + assertTrue(write2.isDone()); + assertEquals(3L, ctx.coordinator.lastCommittedOffset()); + assertEquals("response1", write1.get(5, TimeUnit.SECONDS)); + assertEquals("response2", write2.get(5, TimeUnit.SECONDS)); } @Test - public void testCoordinatorExecutor() { - Duration writeTimeout = Duration.ofMillis(1000); + public void testLargeCompressibleRecordTriggersFlushAndSucceeds() throws Exception { MockTimer timer = new MockTimer(); MockPartitionWriter writer = new MockPartitionWriter(); - ManualEventProcessor processor = new ManualEventProcessor(); - CoordinatorRuntimeMetrics runtimeMetrics = mock(CoordinatorRuntimeMetrics.class); - ExecutorService executorService = mock(ExecutorService.class); - - when(executorService.submit(any(Runnable.class))).thenAnswer(args -> { - Runnable op = args.getArgument(0); - op.run(); - return CompletableFuture.completedFuture(null); - }); + Compression compression = Compression.gzip().build(); CoordinatorRuntime runtime = new CoordinatorRuntime.Builder() .withTime(timer.time()) .withTimer(timer) - .withDefaultWriteTimeOut(writeTimeout) + .withDefaultWriteTimeOut(Duration.ofMillis(20)) .withLoader(new MockCoordinatorLoader()) - .withEventProcessor(processor) + .withEventProcessor(new DirectEventProcessor()) .withPartitionWriter(writer) .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) - .withCoordinatorRuntimeMetrics(runtimeMetrics) + .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) + .withCompression(compression) .withSerializer(new StringSerializer()) - .withAppendLingerMs(0) - .withExecutorService(executorService) + .withAppendLingerMs(10) + .withExecutorService(mock(ExecutorService.class)) .build(); - // Loads the coordinator. Poll once to execute the load operation and once - // to complete the load. + // Schedule the loading. runtime.scheduleLoadOperation(TP, 10); - processor.poll(); - processor.poll(); - // Schedule a write which schedules an async tasks. - CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, writeTimeout, - state -> { - state.executor().schedule( - "write#1#task", - () -> "task result", - (result, exception) -> { - assertEquals("task result", result); - assertNull(exception); - return new CoordinatorResult<>(List.of("record2"), null); - } - ); - return new CoordinatorResult<>(List.of("record1"), "response1"); - } - ); + // Verify the initial state. + CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertNull(ctx.currentBatch); - // Execute the write. - processor.poll(); + // Get the max batch size. + int maxBatchSize = writer.config(TP).maxMessageSize(); - // We should have a new write event in the queue as a result of the - // task being executed immediately. - assertEquals(1, processor.size()); + // Create 2 records with a quarter of the max batch size each. + List records = Stream.of('1', '2').map(c -> { + char[] payload = new char[maxBatchSize / 4]; + Arrays.fill(payload, c); + return new String(payload); + }).collect(Collectors.toList()); - // Verify the state. - CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); - assertEquals(1L, ctx.coordinator.lastWrittenOffset()); - assertEquals(0L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of( - new MockCoordinatorShard.RecordAndMetadata(0, "record1") - ), ctx.coordinator.coordinator().fullRecords()); + // Write #1 with the small records, batch will be about half full + long firstBatchTimestamp = timer.time().milliseconds(); + CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(50), + state -> new CoordinatorResult<>(records, "response1") + ); - // Execute the pending write. - processor.poll(); + // A batch has been created. + assertNotNull(ctx.currentBatch); - // The processor must be empty now. - assertEquals(0, processor.size()); + // Verify the state - batch is not yet flushed + assertEquals(List.of(), writer.entries(TP)); + + // Create a large record of highly compressible data + List largeRecord = List.of("a".repeat(3 * maxBatchSize)); + + // Write #2 with the large record. This record is too large to go into the previous batch + // uncompressed but will fit in the new buffer once compressed, so we should flush the + // previous batch and successfully allocate a new batch for this record. The new batch + // will also trigger an immediate flush. + long secondBatchTimestamp = timer.time().milliseconds(); + CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(50), + state -> new CoordinatorResult<>(largeRecord, "response2") + ); // Verify the state. - assertEquals(2L, ctx.coordinator.lastWrittenOffset()); + assertEquals(3L, ctx.coordinator.lastWrittenOffset()); assertEquals(0L, ctx.coordinator.lastCommittedOffset()); assertEquals(List.of( - new MockCoordinatorShard.RecordAndMetadata(0, "record1"), - new MockCoordinatorShard.RecordAndMetadata(1, "record2") + new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), + new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), + new MockCoordinatorShard.RecordAndMetadata(2, largeRecord.get(0)) ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of( + records(firstBatchTimestamp, compression, records), + records(secondBatchTimestamp, compression, largeRecord) + ), writer.entries(TP)); - // Commit. + // Commit and verify that writes are completed. writer.commit(TP); - processor.poll(); assertTrue(write1.isDone()); + assertTrue(write2.isDone()); + assertEquals(3L, ctx.coordinator.lastCommittedOffset()); + assertEquals("response1", write1.get(5, TimeUnit.SECONDS)); + assertEquals("response2", write2.get(5, TimeUnit.SECONDS)); } @Test - public void testLingerTimeComparisonInMaybeFlushCurrentBatch() throws Exception { - // Provides the runtime clock; we will advance it. - MockTimer clockTimer = new MockTimer(); - // Used for scheduling timer tasks; we won't advance it to avoid a timer-triggered batch flush. - MockTimer schedulerTimer = new MockTimer(); - + public void testLargeUncompressibleRecordTriggersFlushAndFails() throws Exception { + MockTimer timer = new MockTimer(); MockPartitionWriter writer = new MockPartitionWriter(); + Compression compression = Compression.gzip().build(); CoordinatorRuntime runtime = new CoordinatorRuntime.Builder() - .withTime(clockTimer.time()) - .withTimer(schedulerTimer) + .withTime(timer.time()) + .withTimer(timer) .withDefaultWriteTimeOut(Duration.ofMillis(20)) .withLoader(new MockCoordinatorLoader()) .withEventProcessor(new DirectEventProcessor()) @@ -4892,6 +4932,7 @@ public void testLingerTimeComparisonInMaybeFlushCurrentBatch() throws Exception .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) + .withCompression(compression) .withSerializer(new StringSerializer()) .withAppendLingerMs(10) .withExecutorService(mock(ExecutorService.class)) @@ -4902,70 +4943,89 @@ public void testLingerTimeComparisonInMaybeFlushCurrentBatch() throws Exception // Verify the initial state. CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); - assertEquals(ACTIVE, ctx.state); assertNull(ctx.currentBatch); - // Write #1. - CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(List.of("record1"), "response1") + // Get the max batch size. + int maxBatchSize = writer.config(TP).maxMessageSize(); + + // Create 2 records with a quarter of the max batch size each. + List records = Stream.of('1', '2').map(c -> { + char[] payload = new char[maxBatchSize / 4]; + Arrays.fill(payload, c); + return new String(payload); + }).collect(Collectors.toList()); + + // Write #1 with the small records, batch will be about half full + long firstBatchTimestamp = timer.time().milliseconds(); + CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(50), + state -> new CoordinatorResult<>(records, "response1") ); - assertFalse(write1.isDone()); + + // A batch has been created. assertNotNull(ctx.currentBatch); - assertEquals(0, writer.entries(TP).size()); - // Verify that the linger timeout task is created; there will also be a default write timeout task. - assertEquals(2, schedulerTimer.size()); + // Verify the state - batch is not yet flushed + assertEquals(List.of(), writer.entries(TP)); - // Advance past the linger time. - clockTimer.advanceClock(11); + // Create a large record of not very compressible data + char[] payload = new char[3 * maxBatchSize]; + Random offset = new Random(); + for (int i = 0; i < payload.length; i++) { + payload[i] = (char) ('a' + ((char) offset.nextInt() % 26)); + } + List largeRecord = List.of(new String(payload)); - // At this point, there are still two scheduled tasks; the linger task has not fired - // because we did not advance the schedulerTimer. - assertEquals(2, schedulerTimer.size()); - - // Write #2. - CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(List.of("record2"), "response2") + // Write #2 with the large record. This record is too large to go into the previous batch + // and is not compressible so it should be flushed. It is also too large to fit in a new batch + // so the write should fail with RecordTooLargeException + CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(50), + state -> new CoordinatorResult<>(largeRecord, "response2") ); - // The batch should have been flushed. - assertEquals(1, writer.entries(TP).size()); - - // Because flushing the batch cancels the linger task, there should now be two write timeout tasks. - assertEquals(2, schedulerTimer.size()); + // Check that write2 fails with RecordTooLargeException + assertFutureThrows(RecordTooLargeException.class, write2); - // Verify batch contains both two records - MemoryRecords batch = writer.entries(TP).get(0); - RecordBatch recordBatch = batch.firstBatch(); - assertEquals(2, recordBatch.countOrNull()); + // Verify the state. The first batch was flushed and the largeRecord + // write failed. + assertEquals(2L, ctx.coordinator.lastWrittenOffset()); + assertEquals(0L, ctx.coordinator.lastCommittedOffset()); + assertEquals(List.of( + new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), + new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)) + ), ctx.coordinator.coordinator().fullRecords()); + assertEquals(List.of( + records(firstBatchTimestamp, compression, records) + ), writer.entries(TP)); // Commit and verify that writes are completed. writer.commit(TP); assertTrue(write1.isDone()); assertTrue(write2.isDone()); - // Now that all scheduled tasks have been cancelled, the scheduler queue should be empty. - assertEquals(0, schedulerTimer.size()); - } + assertEquals(2L, ctx.coordinator.lastCommittedOffset()); + assertEquals("response1", write1.get(5, TimeUnit.SECONDS)); + } @Test - public void testAdaptiveAppendLingerTime() { + public void testRecordEventPurgatoryTime() throws Exception { + Duration writeTimeout = Duration.ofMillis(1000); MockTimer timer = new MockTimer(); - ManualEventProcessor processor = new ManualEventProcessor(); MockPartitionWriter writer = new MockPartitionWriter(); + ManualEventProcessor processor = new ManualEventProcessor(); + CoordinatorRuntimeMetrics runtimeMetrics = mock(CoordinatorRuntimeMetrics.class); CoordinatorRuntime runtime = new CoordinatorRuntime.Builder() .withTime(timer.time()) .withTimer(timer) - .withDefaultWriteTimeOut(Duration.ofMillis(20)) + .withDefaultWriteTimeOut(writeTimeout) .withLoader(new MockCoordinatorLoader()) .withEventProcessor(processor) .withPartitionWriter(writer) .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) - .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) + .withCoordinatorRuntimeMetrics(runtimeMetrics) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(OptionalInt.empty()) + .withAppendLingerMs(0) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -4975,105 +5035,84 @@ public void testAdaptiveAppendLingerTime() { processor.poll(); processor.poll(); - // Verify the initial state. - CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); - assertEquals(0L, ctx.coordinator.lastWrittenOffset()); - assertEquals(0L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); - assertNull(ctx.currentBatch); - - // Write #1. - runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(List.of("record1", "record2"), "response1") + // write#1 will be committed and update the high watermark. Record time spent in purgatory. + CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, writeTimeout, + state -> new CoordinatorResult<>(List.of("record1"), "response1") ); - - // Write #2. - runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(List.of("record3"), "response2") + // write#2 will time out sitting in the purgatory. Record time spent in purgatory. + CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, writeTimeout, + state -> new CoordinatorResult<>(List.of("record2"), "response2") ); + // write#3 will error while appending. Does not spend time in purgatory. + CompletableFuture write3 = runtime.scheduleWriteOperation("write#3", TP, writeTimeout, + state -> { + throw new KafkaException("write#3 failed."); + }); - // Execute write #1. + processor.poll(); + processor.poll(); processor.poll(); - // A batch has been created. - assertNotNull(ctx.currentBatch); - - // A flush event is queued after write #2. - assertEquals(2, processor.size()); + // Confirm we do not record purgatory time for write#3. + assertTrue(write3.isCompletedExceptionally()); + verify(runtimeMetrics, times(0)).recordEventPurgatoryTime(0L); - // Verify the state. Records are replayed but no batch written. - assertEquals(0L, ctx.coordinator.lastWrittenOffset()); - assertEquals(0L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); + // Records have been written to the log. + long writeTimestamp = timer.time().milliseconds(); assertEquals(List.of( - new MockCoordinatorShard.RecordAndMetadata(0, "record1"), - new MockCoordinatorShard.RecordAndMetadata(1, "record2") - ), ctx.coordinator.coordinator().fullRecords()); - assertEquals(List.of(), writer.entries(TP)); + records(writeTimestamp, "record1"), + records(writeTimestamp, "record2") + ), writer.entries(TP)); - // Execute write #2. - processor.poll(); - assertEquals(1, processor.size()); + // There is no pending high watermark. + assertEquals(NO_OFFSET, runtime.contextOrThrow(TP).highWatermarklistener.lastHighWatermark()); - // The batch has not been flushed. - assertNotNull(ctx.currentBatch); + // Advance the clock then commit records from write#1. + timer.advanceClock(700); + writer.commit(TP, 1); - // Verify the state. Records are replayed but no batch written. - assertEquals(0L, ctx.coordinator.lastWrittenOffset()); - assertEquals(0L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); - assertEquals(List.of( - new MockCoordinatorShard.RecordAndMetadata(0, "record1"), - new MockCoordinatorShard.RecordAndMetadata(1, "record2"), - new MockCoordinatorShard.RecordAndMetadata(2, "record3") - ), ctx.coordinator.coordinator().fullRecords()); - assertEquals(List.of(), writer.entries(TP)); + // We should still have one pending event and the pending high watermark should be updated. + assertEquals(1, processor.size()); + assertEquals(1, runtime.contextOrThrow(TP).highWatermarklistener.lastHighWatermark()); - // Flush the batch. + // Poll once to process the high watermark update and complete the writes. processor.poll(); + long purgatoryTimeMs = timer.time().milliseconds() - writeTimestamp; - // The batch is flushed. - assertNull(ctx.currentBatch); + // Advance the clock past write timeout. write#2 has now timed out. + timer.advanceClock(300 + 1); + processor.poll(); - // Verify the state. - assertEquals(3L, ctx.coordinator.lastWrittenOffset()); - assertEquals(0L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of(0L, 3L), ctx.coordinator.snapshotRegistry().epochsList()); - assertEquals(List.of( - new MockCoordinatorShard.RecordAndMetadata(0, "record1"), - new MockCoordinatorShard.RecordAndMetadata(1, "record2"), - new MockCoordinatorShard.RecordAndMetadata(2, "record3") - ), ctx.coordinator.coordinator().fullRecords()); - assertEquals(List.of( - TestUtil.records(timer.time().milliseconds(), "record1", "record2", "record3") - ), writer.entries(TP)); + assertEquals(NO_OFFSET, runtime.contextOrThrow(TP).highWatermarklistener.lastHighWatermark()); + assertEquals(1, runtime.contextOrThrow(TP).coordinator.lastCommittedOffset()); + assertTrue(write1.isDone()); + assertTrue(write2.isCompletedExceptionally()); + verify(runtimeMetrics, times(1)).recordEventPurgatoryTime(purgatoryTimeMs); + verify(runtimeMetrics, times(1)).recordEventPurgatoryTime(writeTimeout.toMillis() + 1); } - /** - * Tests a flush triggered by the max batch size with an adaptive append linger time. - * - * The flush for the first batch must not flush the second batch. - */ @Test - @SuppressWarnings("checkstyle:MethodLength") - public void testAdaptiveAppendLingerWithMaxBatchSizeFlush() { + public void testWriteEventCompletesOnlyOnce() throws Exception { + // Completes once via timeout, then again with HWM update. + Duration writeTimeout = Duration.ofMillis(1000L); MockTimer timer = new MockTimer(); - ManualEventProcessor processor = new ManualEventProcessor(); MockPartitionWriter writer = new MockPartitionWriter(); + ManualEventProcessor processor = new ManualEventProcessor(); + CoordinatorRuntimeMetrics runtimeMetrics = mock(CoordinatorRuntimeMetrics.class); CoordinatorRuntime runtime = new CoordinatorRuntime.Builder() .withTime(timer.time()) .withTimer(timer) - .withDefaultWriteTimeOut(Duration.ofMillis(20)) + .withDefaultWriteTimeOut(writeTimeout) .withLoader(new MockCoordinatorLoader()) .withEventProcessor(processor) .withPartitionWriter(writer) .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) - .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) + .withCoordinatorRuntimeMetrics(runtimeMetrics) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(OptionalInt.empty()) + .withAppendLingerMs(0) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -5083,187 +5122,144 @@ public void testAdaptiveAppendLingerWithMaxBatchSizeFlush() { processor.poll(); processor.poll(); - // Verify the initial state. - CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); - assertEquals(0L, ctx.coordinator.lastWrittenOffset()); - assertEquals(0L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); - assertNull(ctx.currentBatch); - - // Get the max batch size. - int maxBatchSize = writer.config(TP).maxMessageSize(); - - // Create records with a quarter of the max batch size each. Keep in mind that - // each batch has a header so it is not possible to have those four records - // in one single batch. - List records = Stream.of('1', '2', '3', '4', '5').map(c -> { - char[] payload = new char[maxBatchSize / 4]; - Arrays.fill(payload, c); - return new String(payload); - }).collect(Collectors.toList()); - - // Write #1. - runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(records.subList(0, 1), "response1") - ); - - // Write #2. - runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(records.subList(1, 2), "response2") - ); - - // Write #3. - runtime.scheduleWriteOperation("write#3", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(records.subList(2, 3), "response3") - ); - - // Write #4. - runtime.scheduleWriteOperation("write#4", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(records.subList(3, 4), "response4") + // write#1 will be committed and update the high watermark. Record time spent in purgatory. + CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, writeTimeout, + state -> new CoordinatorResult<>(List.of("record1"), "response1") ); - // Execute write #1, write #2 and write #3. - processor.poll(); - processor.poll(); processor.poll(); - // A batch has been created. - assertNotNull(ctx.currentBatch); - - // A flush event is queued after write #4. - assertEquals(2, processor.size()); - - // Verify the state. Records are replayed but no batch written. - assertEquals(0L, ctx.coordinator.lastWrittenOffset()); - assertEquals(0L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); + // Records have been written to the log. + long writeTimestamp = timer.time().milliseconds(); assertEquals(List.of( - new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), - new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), - new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)) - ), ctx.coordinator.coordinator().fullRecords()); - assertEquals(List.of(), writer.entries(TP)); + records(writeTimestamp, "record1") + ), writer.entries(TP)); - // Write #5. - runtime.scheduleWriteOperation("write#5", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(records.subList(4, 5), "response5") - ); + // There is no pending high watermark. + assertEquals(NO_OFFSET, runtime.contextOrThrow(TP).highWatermarklistener.lastHighWatermark()); - // Execute write #4. This one cannot go into the existing batch - // so the existing batch should be flushed and a new one should be created. + // Advance the clock to time out the write event. Confirm write#1 is completed with a timeout. + timer.advanceClock(writeTimeout.toMillis() + 1L); processor.poll(); + verify(runtimeMetrics, times(1)).recordEventPurgatoryTime(writeTimeout.toMillis() + 1); + assertTrue(write1.isCompletedExceptionally()); - // A batch has been created. - assertNotNull(ctx.currentBatch); + // HWM update + writer.commit(TP, 1); + assertEquals(1, processor.size()); + assertEquals(1, runtime.contextOrThrow(TP).highWatermarklistener.lastHighWatermark()); - // Another flush event is queued after write #5. - assertEquals(3, processor.size()); + // Poll once to process the high watermark update and complete write#1. It has already + // been completed and this is a noop. + processor.poll(); - // Verify the state. - assertEquals(3L, ctx.coordinator.lastWrittenOffset()); - assertEquals(0L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of(0L, 3L), ctx.coordinator.snapshotRegistry().epochsList()); - assertEquals(List.of( - new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), - new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), - new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)), - new MockCoordinatorShard.RecordAndMetadata(3, records.get(3)) - ), ctx.coordinator.coordinator().fullRecords()); - assertEquals(List.of( - TestUtil.records(timer.time().milliseconds(), records.subList(0, 3)) - ), writer.entries(TP)); + assertEquals(NO_OFFSET, runtime.contextOrThrow(TP).highWatermarklistener.lastHighWatermark()); + assertEquals(1, runtime.contextOrThrow(TP).coordinator.lastCommittedOffset()); + assertTrue(write1.isCompletedExceptionally()); + verify(runtimeMetrics, times(1)).recordEventPurgatoryTime(writeTimeout.toMillis() + 1L); + } - // Execute the first flush. - processor.poll(); - assertEquals(2, processor.size()); + @Test + public void testCompleteTransactionEventCompletesOnlyOnce() throws Exception { + // Completes once via timeout, then again with HWM update. + Duration writeTimeout = Duration.ofMillis(1000L); + MockTimer timer = new MockTimer(); + MockPartitionWriter writer = new MockPartitionWriter(); + ManualEventProcessor processor = new ManualEventProcessor(); + CoordinatorRuntimeMetrics runtimeMetrics = mock(CoordinatorRuntimeMetrics.class); - // The flush does not belong to the current batch and is ignored. - assertNotNull(ctx.currentBatch); + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(timer.time()) + .withTimer(timer) + .withDefaultWriteTimeOut(writeTimeout) + .withLoader(new MockCoordinatorLoader()) + .withEventProcessor(processor) + .withPartitionWriter(writer) + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) + .withCoordinatorRuntimeMetrics(runtimeMetrics) + .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) + .withSerializer(new StringSerializer()) + .withExecutorService(mock(ExecutorService.class)) + .build(); - // Verify the state. - assertEquals(3L, ctx.coordinator.lastWrittenOffset()); - assertEquals(0L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of(0L, 3L), ctx.coordinator.snapshotRegistry().epochsList()); - assertEquals(List.of( - new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), - new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), - new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)), - new MockCoordinatorShard.RecordAndMetadata(3, records.get(3)) - ), ctx.coordinator.coordinator().fullRecords()); - assertEquals(List.of( - TestUtil.records(timer.time().milliseconds(), records.subList(0, 3)) - ), writer.entries(TP)); + // Loads the coordinator. Poll once to execute the load operation and once + // to complete the load. + runtime.scheduleLoadOperation(TP, 10); + processor.poll(); + processor.poll(); - // Execute write #5. + // transaction completion. + CompletableFuture write1 = runtime.scheduleTransactionCompletion( + "transactional-write", + TP, + 100L, + (short) 50, + 1, + TransactionResult.COMMIT, + writeTimeout + ); processor.poll(); - assertEquals(1, processor.size()); - // Verify the state. - assertEquals(3L, ctx.coordinator.lastWrittenOffset()); - assertEquals(0L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of(0L, 3L), ctx.coordinator.snapshotRegistry().epochsList()); - assertEquals(List.of( - new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), - new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), - new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)), - new MockCoordinatorShard.RecordAndMetadata(3, records.get(3)), - new MockCoordinatorShard.RecordAndMetadata(4, records.get(4)) - ), ctx.coordinator.coordinator().fullRecords()); + // Records have been written to the log. assertEquals(List.of( - TestUtil.records(timer.time().milliseconds(), records.subList(0, 3)) + endTransactionMarker(100, (short) 50, timer.time().milliseconds(), 1, ControlRecordType.COMMIT) ), writer.entries(TP)); - // Execute the second flush. + // The write timeout tasks exist. + assertEquals(1, timer.size()); + assertFalse(write1.isDone()); + + // Advance the clock to time out the write event. Confirm write#1 is completed with a timeout. + timer.advanceClock(writeTimeout.toMillis() + 1L); processor.poll(); - assertEquals(0, processor.size()); + verify(runtimeMetrics, times(1)).recordEventPurgatoryTime(writeTimeout.toMillis() + 1); + assertTrue(write1.isCompletedExceptionally()); - // The batch is flushed. - assertNull(ctx.currentBatch); + // HWM update + writer.commit(TP, 1); + assertEquals(1, processor.size()); + assertEquals(1, runtime.contextOrThrow(TP).highWatermarklistener.lastHighWatermark()); - // Verify the state. - assertEquals(5L, ctx.coordinator.lastWrittenOffset()); - assertEquals(0L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of(0L, 3L, 5L), ctx.coordinator.snapshotRegistry().epochsList()); - assertEquals(List.of( - new MockCoordinatorShard.RecordAndMetadata(0, records.get(0)), - new MockCoordinatorShard.RecordAndMetadata(1, records.get(1)), - new MockCoordinatorShard.RecordAndMetadata(2, records.get(2)), - new MockCoordinatorShard.RecordAndMetadata(3, records.get(3)), - new MockCoordinatorShard.RecordAndMetadata(4, records.get(4)) - ), ctx.coordinator.coordinator().fullRecords()); - assertEquals(List.of( - TestUtil.records(timer.time().milliseconds(), records.subList(0, 3)), - TestUtil.records(timer.time().milliseconds(), records.subList(3, 5)) - ), writer.entries(TP)); + // Poll once to process the high watermark update and complete write#1. It has already + // been completed and this is a noop. + processor.poll(); + + assertEquals(NO_OFFSET, runtime.contextOrThrow(TP).highWatermarklistener.lastHighWatermark()); + assertEquals(1, runtime.contextOrThrow(TP).coordinator.lastCommittedOffset()); + assertTrue(write1.isCompletedExceptionally()); + verify(runtimeMetrics, times(1)).recordEventPurgatoryTime(writeTimeout.toMillis() + 1L); } - /** - * Tests a transactional write with an adaptive append linger time. - * - * The transactional write must not enqueue a flush, since it flushes immediately. - * The flush for the batch before the transactional write must not flush the batch after the - * transactional write. - */ @Test - public void testAdaptiveAppendLingerWithTransactionalWrite() { + public void testCoordinatorExecutor() { + Duration writeTimeout = Duration.ofMillis(1000); MockTimer timer = new MockTimer(); - ManualEventProcessor processor = new ManualEventProcessor(); MockPartitionWriter writer = new MockPartitionWriter(); + ManualEventProcessor processor = new ManualEventProcessor(); + CoordinatorRuntimeMetrics runtimeMetrics = mock(CoordinatorRuntimeMetrics.class); + ExecutorService executorService = mock(ExecutorService.class); + + when(executorService.submit(any(Runnable.class))).thenAnswer(args -> { + Runnable op = args.getArgument(0); + op.run(); + return CompletableFuture.completedFuture(null); + }); CoordinatorRuntime runtime = new CoordinatorRuntime.Builder() .withTime(timer.time()) .withTimer(timer) - .withDefaultWriteTimeOut(Duration.ofMillis(20)) + .withDefaultWriteTimeOut(writeTimeout) .withLoader(new MockCoordinatorLoader()) .withEventProcessor(processor) .withPartitionWriter(writer) .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) - .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) + .withCoordinatorRuntimeMetrics(runtimeMetrics) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(OptionalInt.empty()) - .withExecutorService(mock(ExecutorService.class)) + .withAppendLingerMs(0) + .withExecutorService(executorService) .build(); // Loads the coordinator. Poll once to execute the load operation and once @@ -5272,126 +5268,130 @@ public void testAdaptiveAppendLingerWithTransactionalWrite() { processor.poll(); processor.poll(); - // Verify the initial state. - CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); - assertEquals(0L, ctx.coordinator.lastWrittenOffset()); - assertEquals(0L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); - assertNull(ctx.currentBatch); - - // Write #1. - runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(List.of("record1"), "response1") - ); - - // Transactional write #2. This will flush the batch. - runtime.scheduleTransactionalWriteOperation( - "txn-write#1", - TP, - "transactional-id", - 100L, - (short) 50, - Duration.ofMillis(20), - state -> new CoordinatorResult<>(List.of("record2"), "response2"), - TXN_OFFSET_COMMIT_LATEST_VERSION - ); - - // Write #3. - runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), - state -> new CoordinatorResult<>(List.of("record3"), "response3") + // Schedule a write which schedules an async tasks. + CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, writeTimeout, + state -> { + state.executor().schedule( + "write#1#task", + () -> "task result", + (result, exception) -> { + assertEquals("task result", result); + assertNull(exception); + return new CoordinatorResult<>(List.of("record2"), null); + } + ); + return new CoordinatorResult<>(List.of("record1"), "response1"); + } ); - assertEquals(3, processor.size()); - - // Execute write #1. + // Execute the write. processor.poll(); - // A batch has been created. - assertNotNull(ctx.currentBatch); - - // A flush event is queued after write #3. - assertEquals(3, processor.size()); + // We should have a new write event in the queue as a result of the + // task being executed immediately. + assertEquals(1, processor.size()); - // Verify the state. Records are replayed but no batch written. - assertEquals(0L, ctx.coordinator.lastWrittenOffset()); + // Verify the state. + CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertEquals(1L, ctx.coordinator.lastWrittenOffset()); assertEquals(0L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of(0L), ctx.coordinator.snapshotRegistry().epochsList()); assertEquals(List.of( new MockCoordinatorShard.RecordAndMetadata(0, "record1") ), ctx.coordinator.coordinator().fullRecords()); - assertEquals(List.of(), writer.entries(TP)); - // Execute transactional write #2. + // Execute the pending write. processor.poll(); - // The batch is flushed. - assertNull(ctx.currentBatch); - - // No flush event is queued. - assertEquals(2, processor.size()); + // The processor must be empty now. + assertEquals(0, processor.size()); - // Verify the state. The current batch and the transactional records are - // written to the log. + // Verify the state. assertEquals(2L, ctx.coordinator.lastWrittenOffset()); assertEquals(0L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of(0L, 1L, 2L), ctx.coordinator.snapshotRegistry().epochsList()); assertEquals(List.of( - new MockCoordinatorShard.RecordAndMetadata(0, "record1") + new MockCoordinatorShard.RecordAndMetadata(0, "record1"), + new MockCoordinatorShard.RecordAndMetadata(1, "record2") ), ctx.coordinator.coordinator().fullRecords()); - assertEquals(List.of( - TestUtil.records(timer.time().milliseconds(), "record1"), - TestUtil.transactionalRecords(100L, (short) 50, timer.time().milliseconds(), "record2") - ), writer.entries(TP)); - // Execute write #3. + // Commit. + writer.commit(TP); processor.poll(); + assertTrue(write1.isDone()); + } - // A batch has been created. - assertNotNull(ctx.currentBatch); + @Test + public void testLingerTimeComparisonInMaybeFlushCurrentBatch() throws Exception { + // Provides the runtime clock; we will advance it. + MockTimer clockTimer = new MockTimer(); + // Used for scheduling timer tasks; we won't advance it to avoid a timer-triggered batch flush. + MockTimer schedulerTimer = new MockTimer(); - // A flush event is queued after the first flush. - assertEquals(2, processor.size()); + MockPartitionWriter writer = new MockPartitionWriter(); - // Verify the state. Records are replayed but no batch written. - assertEquals(2L, ctx.coordinator.lastWrittenOffset()); - assertEquals(0L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of(0L, 1L, 2L), ctx.coordinator.snapshotRegistry().epochsList()); - assertEquals(List.of( - new MockCoordinatorShard.RecordAndMetadata(0, "record1"), - new MockCoordinatorShard.RecordAndMetadata(2, "record3") - ), ctx.coordinator.coordinator().fullRecords()); - assertEquals(List.of( - TestUtil.records(timer.time().milliseconds(), "record1"), - TestUtil.transactionalRecords(100L, (short) 50, timer.time().milliseconds(), "record2") - ), writer.entries(TP)); + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(clockTimer.time()) + .withTimer(schedulerTimer) + .withDefaultWriteTimeOut(Duration.ofMillis(20)) + .withLoader(new MockCoordinatorLoader()) + .withEventProcessor(new DirectEventProcessor()) + .withPartitionWriter(writer) + .withCoordinatorShardBuilderSupplier(new MockCoordinatorShardBuilderSupplier()) + .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) + .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) + .withSerializer(new StringSerializer()) + .withAppendLingerMs(10) + .withExecutorService(mock(ExecutorService.class)) + .build(); - // Execute the first flush. - processor.poll(); - assertEquals(1, processor.size()); + // Schedule the loading. + runtime.scheduleLoadOperation(TP, 10); - // The flush does not belong to the current batch and is ignored. + // Verify the initial state. + CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertEquals(ACTIVE, ctx.state); + assertNull(ctx.currentBatch); + + // Write #1. + CompletableFuture write1 = runtime.scheduleWriteOperation("write#1", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(List.of("record1"), "response1") + ); + assertFalse(write1.isDone()); assertNotNull(ctx.currentBatch); + assertEquals(0, writer.entries(TP).size()); - // Execute the second flush. - processor.poll(); - assertEquals(0, processor.size()); + // Verify that the linger timeout task is created; there will also be a default write timeout task. + assertEquals(2, schedulerTimer.size()); - // The batch is flushed. - assertNull(ctx.currentBatch); + // Advance past the linger time. + clockTimer.advanceClock(11); - // Verify the state. - assertEquals(3L, ctx.coordinator.lastWrittenOffset()); - assertEquals(0L, ctx.coordinator.lastCommittedOffset()); - assertEquals(List.of(0L, 1L, 2L, 3L), ctx.coordinator.snapshotRegistry().epochsList()); - assertEquals(List.of( - new MockCoordinatorShard.RecordAndMetadata(0, "record1"), - new MockCoordinatorShard.RecordAndMetadata(2, "record3") - ), ctx.coordinator.coordinator().fullRecords()); - assertEquals(List.of( - TestUtil.records(timer.time().milliseconds(), "record1"), - TestUtil.transactionalRecords(100L, (short) 50, timer.time().milliseconds(), "record2"), - TestUtil.records(timer.time().milliseconds(), "record3") - ), writer.entries(TP)); + // At this point, there are still two scheduled tasks; the linger task has not fired + // because we did not advance the schedulerTimer. + assertEquals(2, schedulerTimer.size()); + + // Write #2. + CompletableFuture write2 = runtime.scheduleWriteOperation("write#2", TP, Duration.ofMillis(20), + state -> new CoordinatorResult<>(List.of("record2"), "response2") + ); + + // The batch should have been flushed. + assertEquals(1, writer.entries(TP).size()); + + // Because flushing the batch cancels the linger task, there should now be two write timeout tasks. + assertEquals(2, schedulerTimer.size()); + + // Verify batch contains both two records + MemoryRecords batch = writer.entries(TP).get(0); + RecordBatch recordBatch = batch.firstBatch(); + assertEquals(2, recordBatch.countOrNull()); + + // Commit and verify that writes are completed. + writer.commit(TP); + assertTrue(write1.isDone()); + assertTrue(write2.isDone()); + // Now that all scheduled tasks have been cancelled, the scheduler queue should be empty. + assertEquals(0, schedulerTimer.size()); } private static , U> ArgumentMatcher> coordinatorMatcher( From d4df111bed1c2ecd973e0fc8084c4152efcab7a6 Mon Sep 17 00:00:00 2001 From: Sean Quah Date: Tue, 28 Oct 2025 16:38:38 +0000 Subject: [PATCH 06/10] fixup: fix appendLingerMs range check --- .../kafka/coordinator/common/runtime/CoordinatorRuntime.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java index 102ec8b8436ea..997e09eedfeb9 100644 --- a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java +++ b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java @@ -228,7 +228,7 @@ public CoordinatorRuntime build() { compression = Compression.NONE; if (appendLingerMs == null) appendLingerMs = OptionalInt.empty(); - if (appendLingerMs.isPresent() && appendLingerMs.getAsInt() < -1) + if (appendLingerMs.isPresent() && appendLingerMs.getAsInt() < 0) throw new IllegalArgumentException("AppendLinger must be empty or >= 0"); if (executorService == null) throw new IllegalArgumentException("ExecutorService must be set."); From 9cadfb4872668b6710df3eef7a974303978a9f28 Mon Sep 17 00:00:00 2001 From: Sean Quah Date: Tue, 28 Oct 2025 19:37:04 +0000 Subject: [PATCH 07/10] fixup: fix GroupCoordinatorConfigTest --- .../kafka/coordinator/group/GroupCoordinatorConfigTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java index 491df993e0999..2f5751132dc18 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java @@ -37,6 +37,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.OptionalInt; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertInstanceOf; @@ -218,7 +219,7 @@ public void testConfigs() { assertEquals(Duration.ofMinutes(24 * 60 * 60 * 1000L).toMillis(), config.offsetsRetentionMs()); assertEquals(5000, config.offsetCommitTimeoutMs()); assertEquals(CompressionType.GZIP, config.offsetTopicCompressionType()); - assertEquals(10, config.appendLingerMs()); + assertEquals(OptionalInt.of(10), config.appendLingerMs()); assertEquals(555, config.offsetsLoadBufferSize()); assertEquals(111, config.offsetsTopicPartitions()); assertEquals(11, config.offsetsTopicReplicationFactor()); From 68423f4dc532e08e0df615750f5c6a4e72c357b9 Mon Sep 17 00:00:00 2001 From: Sean Quah Date: Tue, 28 Oct 2025 19:37:17 +0000 Subject: [PATCH 08/10] fixup: add GroupCoordinatorConfigtest for appendLingerMs --- .../group/GroupCoordinatorConfigTest.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java index 2f5751132dc18..60747d811f7fa 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java @@ -326,6 +326,18 @@ public void testInvalidConfigs() { assertThrows(IllegalArgumentException.class, () -> createConfig(configs)).getMessage()); } + @Test + public void testAppendLingerMs() { + GroupCoordinatorConfig config = createConfig(Map.of(GroupCoordinatorConfig.GROUP_COORDINATOR_APPEND_LINGER_MS_CONFIG, -1)); + assertEquals(OptionalInt.empty(), config.appendLingerMs()); + + config = createConfig(Map.of(GroupCoordinatorConfig.GROUP_COORDINATOR_APPEND_LINGER_MS_CONFIG, 0)); + assertEquals(OptionalInt.of(0), config.appendLingerMs()); + + config = createConfig(Map.of(GroupCoordinatorConfig.GROUP_COORDINATOR_APPEND_LINGER_MS_CONFIG, 5)); + assertEquals(OptionalInt.of(5), config.appendLingerMs()); + } + public static GroupCoordinatorConfig createGroupCoordinatorConfig( int offsetMetadataMaxSize, long offsetsRetentionCheckIntervalMs, From 286ef96b6d250d0b0346722833dc62350a6bf699 Mon Sep 17 00:00:00 2001 From: Sean Quah Date: Tue, 28 Oct 2025 19:36:30 +0000 Subject: [PATCH 09/10] fixup: add ShareCoordinatorConfigTest for appendLingerMs --- .../share/ShareCoordinatorConfigTest.java | 49 +++++++++++++++++++ 1 file changed, 49 insertions(+) create mode 100644 share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorConfigTest.java diff --git a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorConfigTest.java b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorConfigTest.java new file mode 100644 index 0000000000000..86ae8c1101a37 --- /dev/null +++ b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorConfigTest.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.coordinator.share; + +import org.apache.kafka.common.config.AbstractConfig; + +import org.junit.jupiter.api.Test; + +import java.util.Map; +import java.util.OptionalInt; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class ShareCoordinatorConfigTest { + + @Test + public void testAppendLingerMs() { + ShareCoordinatorConfig config = createConfig(Map.of(ShareCoordinatorConfig.APPEND_LINGER_MS_CONFIG, -1)); + assertEquals(OptionalInt.empty(), config.shareCoordinatorAppendLingerMs()); + + config = createConfig(Map.of(ShareCoordinatorConfig.APPEND_LINGER_MS_CONFIG, 0)); + assertEquals(OptionalInt.of(0), config.shareCoordinatorAppendLingerMs()); + + config = createConfig(Map.of(ShareCoordinatorConfig.APPEND_LINGER_MS_CONFIG, 5)); + assertEquals(OptionalInt.of(5), config.shareCoordinatorAppendLingerMs()); + } + + public static ShareCoordinatorConfig createConfig(Map configs) { + return new ShareCoordinatorConfig(new AbstractConfig( + ShareCoordinatorConfig.CONFIG_DEF, + configs, + false + )); + } +} From 142f12f166194439f175def802f584473c3a5527 Mon Sep 17 00:00:00 2001 From: Sean Quah Date: Tue, 28 Oct 2025 19:49:16 +0000 Subject: [PATCH 10/10] fixup: remove withAppendLingerMs(int) overload --- .../common/runtime/CoordinatorRuntime.java | 5 -- .../runtime/CoordinatorRuntimeTest.java | 46 +++++++++---------- 2 files changed, 23 insertions(+), 28 deletions(-) diff --git a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java index 997e09eedfeb9..28fc5672f4093 100644 --- a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java +++ b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java @@ -185,11 +185,6 @@ public Builder withCompression(Compression compression) { return this; } - public Builder withAppendLingerMs(int appendLingerMs) { - this.appendLingerMs = OptionalInt.of(appendLingerMs); - return this; - } - public Builder withAppendLingerMs(OptionalInt appendLingerMs) { this.appendLingerMs = appendLingerMs; return this; diff --git a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java index 89758254f869d..1876eb8640f56 100644 --- a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java +++ b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java @@ -634,7 +634,7 @@ public void testScheduleUnloadingWithDeferredEventExceptions() throws ExecutionE .withCoordinatorRuntimeMetrics(metrics) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(10) + .withAppendLingerMs(OptionalInt.of(10)) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -2615,7 +2615,7 @@ public void testHighWatermarkUpdate() { .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(0) + .withAppendLingerMs(OptionalInt.of(0)) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -2690,7 +2690,7 @@ public void testHighWatermarkUpdateWithDeferredEventExceptions() throws Executio .withCoordinatorRuntimeMetrics(metrics) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(10) + .withAppendLingerMs(OptionalInt.of(10)) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -2767,7 +2767,7 @@ public void testWriteEventWriteTimeoutTaskIsCancelledWhenHighWatermarkIsUpdated( .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(0) + .withAppendLingerMs(OptionalInt.of(0)) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -3165,7 +3165,7 @@ public void testScheduleWriteOperationWithBatching() throws ExecutionException, .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(10) + .withAppendLingerMs(OptionalInt.of(10)) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -3300,7 +3300,7 @@ public void testScheduleWriteOperationWithBatchingWhenRecordsTooLarge() { .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(10) + .withAppendLingerMs(OptionalInt.of(10)) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -3352,7 +3352,7 @@ public void testScheduleWriteOperationWithBatchingWhenWriteFails() { .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(10) + .withAppendLingerMs(OptionalInt.of(10)) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -3438,7 +3438,7 @@ public void testScheduleWriteOperationWithBatchingWhenReplayFails() { .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(10) + .withAppendLingerMs(OptionalInt.of(10)) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -3536,7 +3536,7 @@ public void testScheduleTransactionalWriteOperationWithBatching() throws Executi .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(10) + .withAppendLingerMs(OptionalInt.of(10)) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -4126,7 +4126,7 @@ public long append( .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(10) + .withAppendLingerMs(OptionalInt.of(10)) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -4241,7 +4241,7 @@ public void close() {} .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(10) + .withAppendLingerMs(OptionalInt.of(10)) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -4290,7 +4290,7 @@ public void testScheduleNonAtomicWriteOperation() throws ExecutionException, Int .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(10) + .withAppendLingerMs(OptionalInt.of(10)) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -4399,7 +4399,7 @@ public void testScheduleNonAtomicWriteOperationWithRecordTooLarge() throws Inter .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(10) + .withAppendLingerMs(OptionalInt.of(10)) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -4495,7 +4495,7 @@ public void testScheduleNonAtomicWriteOperationWhenWriteFails() { .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(10) + .withAppendLingerMs(OptionalInt.of(10)) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -4582,7 +4582,7 @@ public void testEmptyBatch() throws Exception { .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(serializer) - .withAppendLingerMs(10) + .withAppendLingerMs(OptionalInt.of(10)) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -4651,7 +4651,7 @@ public void testRecordFlushTime() throws Exception { .withCoordinatorRuntimeMetrics(runtimeMetrics) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(10) + .withAppendLingerMs(OptionalInt.of(10)) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -4762,7 +4762,7 @@ public void testCompressibleRecordTriggersFlushAndSucceeds() throws Exception { .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withCompression(compression) .withSerializer(new StringSerializer()) - .withAppendLingerMs(10) + .withAppendLingerMs(OptionalInt.of(10)) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -4848,7 +4848,7 @@ public void testLargeCompressibleRecordTriggersFlushAndSucceeds() throws Excepti .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withCompression(compression) .withSerializer(new StringSerializer()) - .withAppendLingerMs(10) + .withAppendLingerMs(OptionalInt.of(10)) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -4934,7 +4934,7 @@ public void testLargeUncompressibleRecordTriggersFlushAndFails() throws Exceptio .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withCompression(compression) .withSerializer(new StringSerializer()) - .withAppendLingerMs(10) + .withAppendLingerMs(OptionalInt.of(10)) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -5025,7 +5025,7 @@ public void testRecordEventPurgatoryTime() throws Exception { .withCoordinatorRuntimeMetrics(runtimeMetrics) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(0) + .withAppendLingerMs(OptionalInt.of(0)) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -5112,7 +5112,7 @@ public void testWriteEventCompletesOnlyOnce() throws Exception { .withCoordinatorRuntimeMetrics(runtimeMetrics) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(0) + .withAppendLingerMs(OptionalInt.of(0)) .withExecutorService(mock(ExecutorService.class)) .build(); @@ -5258,7 +5258,7 @@ public void testCoordinatorExecutor() { .withCoordinatorRuntimeMetrics(runtimeMetrics) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(0) + .withAppendLingerMs(OptionalInt.of(0)) .withExecutorService(executorService) .build(); @@ -5340,7 +5340,7 @@ public void testLingerTimeComparisonInMaybeFlushCurrentBatch() throws Exception .withCoordinatorRuntimeMetrics(mock(CoordinatorRuntimeMetrics.class)) .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) .withSerializer(new StringSerializer()) - .withAppendLingerMs(10) + .withAppendLingerMs(OptionalInt.of(10)) .withExecutorService(mock(ExecutorService.class)) .build();