From 7163479e1887562ca022d5acc56e8fd304cd0627 Mon Sep 17 00:00:00 2001 From: Yuto Kawamura Date: Wed, 2 Oct 2024 19:07:12 +0900 Subject: [PATCH 01/12] Introduce OutOfOrderCommitControl V4 --- .../OutOfOrderCommitControlBenchmark.java | 45 ++++- .../runtime/OutOfOrderCommitControlV3.java | 157 ++++++++++++++++++ .../runtime/ProcessorSubscription.java | 7 +- .../runtime/internal/CommitManager.java | 2 +- .../runtime/internal/ConcurrentBitMap.java | 71 ++++++++ .../runtime/internal/OffsetState.java | 2 +- .../internal/OffsetStorageComplex.java | 155 +++++++++++++++++ .../internal/OutOfOrderCommitControl.java | 113 +++++++------ .../internal/ConcurrentBitMapTest.java | 67 ++++++++ .../internal/OffsetStorageComplexTest.java | 95 +++++++++++ 10 files changed, 651 insertions(+), 63 deletions(-) create mode 100644 processor/src/jmh/java/com/linecorp/decaton/processor/runtime/OutOfOrderCommitControlV3.java create mode 100644 processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConcurrentBitMap.java create mode 100644 processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java create mode 100644 processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ConcurrentBitMapTest.java create mode 100644 processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplexTest.java diff --git a/processor/src/jmh/java/com/linecorp/decaton/processor/runtime/OutOfOrderCommitControlBenchmark.java b/processor/src/jmh/java/com/linecorp/decaton/processor/runtime/OutOfOrderCommitControlBenchmark.java index a192c4e1..761f5e8c 100644 --- a/processor/src/jmh/java/com/linecorp/decaton/processor/runtime/OutOfOrderCommitControlBenchmark.java +++ b/processor/src/jmh/java/com/linecorp/decaton/processor/runtime/OutOfOrderCommitControlBenchmark.java @@ -169,7 +169,48 @@ public void outOfOrderCommitControlV2(BmStateV2 state) throws InterruptedExcepti } @State(Scope.Thread) - public static class BmStateV3 extends BmState { + public static class BmStateV3 extends BmState { + @Override + OutOfOrderCommitControlV3 createCommitControl() { + OffsetStateReaper reaper = new OffsetStateReaper( + Property.ofStatic(ProcessorProperties.CONFIG_DEFERRED_COMPLETE_TIMEOUT_MS), + Metrics.withTags("subscription", "subsc", "topic", "topic", "partition", "1") + .new CommitControlMetrics()); + return new OutOfOrderCommitControlV3(topicPartition, CAPACITY, reaper); + } + } + + @Benchmark + public void outOfOrderCommitControlV3(BmStateV3 state) throws InterruptedException { + OutOfOrderCommitControlV3 control = state.control; + + for (long offset = 1; offset <= NUM_OFFSETS; ) { + boolean noProgress = true; + + for (int i = 0; i < BATCH_SIZE; i++, offset++) { + if (control.pendingOffsetsCount() >= CAPACITY) { + break; + } + noProgress = false; + OffsetState offsetState = control.reportFetchedOffset(offset); + + state.workers.execute(offsetState.completion()::complete); + } + if (noProgress) { + Thread.yield(); + } + control.updateHighWatermark(); + } + + control.updateHighWatermark(); + while (control.commitReadyOffset() < NUM_OFFSETS) { + Thread.yield(); + control.updateHighWatermark(); + } + } + + @State(Scope.Thread) + public static class BmStateV4 extends BmState { @Override OutOfOrderCommitControl createCommitControl() { OffsetStateReaper reaper = new OffsetStateReaper( @@ -181,7 +222,7 @@ OutOfOrderCommitControl createCommitControl() { } @Benchmark - public void outOfOrderCommitControlV3(BmStateV3 state) throws InterruptedException { + public void outOfOrderCommitControlV4(BmStateV4 state) throws InterruptedException { OutOfOrderCommitControl control = state.control; for (long offset = 1; offset <= NUM_OFFSETS; ) { diff --git a/processor/src/jmh/java/com/linecorp/decaton/processor/runtime/OutOfOrderCommitControlV3.java b/processor/src/jmh/java/com/linecorp/decaton/processor/runtime/OutOfOrderCommitControlV3.java new file mode 100644 index 00000000..67796cf3 --- /dev/null +++ b/processor/src/jmh/java/com/linecorp/decaton/processor/runtime/OutOfOrderCommitControlV3.java @@ -0,0 +1,157 @@ +/* + * Copyright 2020 LINE Corporation + * + * LINE Corporation 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: + * + * https://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 com.linecorp.decaton.processor.runtime; + +import java.util.ArrayDeque; +import java.util.Deque; + +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.linecorp.decaton.processor.runtime.internal.OffsetState; +import com.linecorp.decaton.processor.runtime.internal.OffsetStateReaper; + +/** + * Represents consumption processing progress of records consumed from a single partition. + * This class manages sequence of offsets and a flag which represents if each of them was completed or not. + */ +public class OutOfOrderCommitControlV3 implements AutoCloseable { + private static final Logger logger = LoggerFactory.getLogger(OutOfOrderCommitControlV3.class); + + private final TopicPartition topicPartition; + private final int capacity; + private final Deque states; + private final OffsetStateReaper offsetStateReaper; + + /** + * The current smallest offset that has been reported but not completed. + */ + private volatile long earliest; + /** + * The current largest offset that has been reported. + */ + private volatile long latest; + /** + * The current maximum offset which it and all it's previous offsets were committed. + */ + private volatile long highWatermark; + + public OutOfOrderCommitControlV3(TopicPartition topicPartition, int capacity, + OffsetStateReaper offsetStateReaper) { + this.topicPartition = topicPartition; + states = new ArrayDeque<>(capacity); + this.capacity = capacity; + this.offsetStateReaper = offsetStateReaper; + earliest = latest = 0; + highWatermark = -1; + } + + public TopicPartition topicPartition() { + return topicPartition; + } + + public synchronized OffsetState reportFetchedOffset(long offset) { + if (isRegressing(offset)) { + throw new RuntimeException(String.format( + "offset regression %s: %d > %d", topicPartition, offset, latest)); + } + + if (states.size() == capacity) { + throw new IllegalArgumentException( + String.format("offsets count overflow: cap=%d, offset=%d", capacity, offset)); + } + + OffsetState state = new OffsetState(offset); + states.addLast(state); + latest = state.offset(); + + state.completion().asFuture().whenComplete((unused, throwable) -> onComplete(offset)); // TODO okay in this order? + return state; + } + + static void onComplete(long offset) { + if (logger.isDebugEnabled()) { + logger.debug("Offset complete: {}", offset); + } + } + + public synchronized void updateHighWatermark() { + if (logger.isTraceEnabled()) { + StringBuilder sb = new StringBuilder("["); + + boolean first = true; + for (OffsetState st : states) { + if (first) { + first = false; + } else { + sb.append(", "); + } + sb.append(String.valueOf(st.offset()) + ':' + (st.completion().isComplete() ? 'c' : 'n')); + } + sb.append(']'); + logger.trace("Begin updateHighWatermark earliest={} latest={} hw={} states={}", + earliest, latest, highWatermark, sb); + } + + long lastHighWatermark = highWatermark; + + OffsetState state; + while ((state = states.peekFirst()) != null) { + earliest = state.offset(); + if (state.completion().isComplete()) { + highWatermark = state.offset(); + states.pollFirst(); + } else { + offsetStateReaper.maybeReapOffset(state); + break; + } + } + + if (highWatermark != lastHighWatermark) { + logger.debug("High watermark updated for {}: {} => {}", + topicPartition, lastHighWatermark, highWatermark); + } + } + + public synchronized int pendingOffsetsCount() { + return states.size(); + } + + public long commitReadyOffset() { + return highWatermark; + } + + public boolean isRegressing(long offset) { + return offset < latest; + } + + @Override + public String toString() { + return "OutOfOrderCommitControl{" + + "topicPartition=" + topicPartition + + ", earliest=" + earliest + + ", latest=" + latest + + ", highWatermark=" + highWatermark + + '}'; + } + + @Override + public void close() throws Exception { + offsetStateReaper.close(); + } +} diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java index 54513c47..05537878 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java @@ -23,7 +23,6 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionStage; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -108,13 +107,17 @@ public void receive(ConsumerRecord record) { OffsetState offsetState; try { offsetState = context.registerOffset(record.offset()); - } catch (OffsetRegressionException e) { + } catch (OffsetRegressionException ignored) { log.warn("Offset regression at partition {}", tp); assignManager.repair(tp); context = contexts.get(tp); // If it fails even at 2nd attempt... no idea let it die. offsetState = context.registerOffset(record.offset()); } + if (offsetState == null) { + // Means this offset has already been processed + return; + } TracingProvider provider = scope.tracingProvider(); RecordTraceHandle trace = provider.traceFor(record, scope.subscriptionId()); diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/CommitManager.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/CommitManager.java index 6e982497..5bdd4719 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/CommitManager.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/CommitManager.java @@ -157,7 +157,7 @@ public void commitAsync() { // 2. When dynamic processor reload is triggered and the context is renewed by // PartitionContexts.maybeHandlePropertyReload. // The case 2 is safe (safe to keep updating committed offset in renewed PartitionContext) because - // it caries previously consuming offset without reset. + // it carries previously consuming offset without reset. // The case 1 is somewhat suspicious but should still be safe, because whenever partition revoke // happens it calls commitSync() through onPartitionsRevoked(). According to the document of // commitAsync(), it is guaranteed that its callback is called before subsequent commitSync() diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConcurrentBitMap.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConcurrentBitMap.java new file mode 100644 index 00000000..95adcb71 --- /dev/null +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConcurrentBitMap.java @@ -0,0 +1,71 @@ +/* + * Copyright 2024 LY Corporation + * + * LY Corporation 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: + * + * https://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 com.linecorp.decaton.processor.runtime.internal; + +import java.util.concurrent.atomic.AtomicLongArray; + +import lombok.Getter; +import lombok.experimental.Accessors; + +@Accessors(fluent = true) +class ConcurrentBitMap { + @Getter + private final int size; + private final AtomicLongArray buckets; + + ConcurrentBitMap(int size) { + this.size = size; + int nbuckets = size / Long.SIZE; + if (size % Long.SIZE != 0) { + nbuckets++; + } + buckets = new AtomicLongArray(nbuckets); + } + + private static int bucketOf(int index) { + return index / Long.SIZE; + } + + private static int localIndex(int index) { + return index % Long.SIZE; + } + + private void ensureBound(int index) { + if (index < 0 || index >= size) { + throw new IndexOutOfBoundsException(index); + } + } + + public void set(int index, boolean on) { + ensureBound(index); + int bucket = bucketOf(index); + int li = localIndex(index); + while (true) { + long bits = buckets.get(bucket); + long newBits = bits & ~(1L << li) | (on ? 1L : 0L) << li; + if (buckets.compareAndSet(bucket, bits, newBits)) { + break; + } + } + } + + public boolean get(int index) { + ensureBound(index); + int bucket = bucketOf(index); + return (buckets.get(bucket) >> localIndex(index) & 1) == 1; + } +} diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetState.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetState.java index 7ee2d802..dcd9e7d0 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetState.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetState.java @@ -28,7 +28,7 @@ public class OffsetState { @Getter private final CompletionImpl completion; - OffsetState(long offset) { + public OffsetState(long offset) { this.offset = offset; timeoutAt = -1; completion = new CompletionImpl(); diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java new file mode 100644 index 00000000..f2e49702 --- /dev/null +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java @@ -0,0 +1,155 @@ +/* + * Copyright 2024 LY Corporation + * + * LY Corporation 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: + * + * https://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 com.linecorp.decaton.processor.runtime.internal; + +import java.util.Map.Entry; +import java.util.TreeMap; + +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.experimental.Accessors; + +@Accessors(fluent = true) +public class OffsetStorageComplex { + static class OffsetIndex { + @AllArgsConstructor + static class BlockInfo { + long index; + int length; + } + + // Map of block's base offset -> block's starting index + private final TreeMap blockIndex; + @Getter + private long firstIndex; + private long nextIndex; + + OffsetIndex() { + blockIndex = new TreeMap<>(); + } + + public int indexSize() { + return (int) (nextIndex - firstIndex); + } + + public long firstOffset() { + return blockIndex.isEmpty() ? -1 : blockIndex.firstKey(); + } + + public long pollFirst() { + Entry first = blockIndex.pollFirstEntry(); + final BlockInfo firstBlock = first.getValue(); + long removedIndex = firstBlock.index++; + firstIndex = firstBlock.index; + firstBlock.length--; + if (firstBlock.length > 0) { + long nextOffset = first.getKey() + 1; + blockIndex.put(nextOffset, firstBlock); // TODO: inefficient, needs to handle by batch + } + return removedIndex; + } + + public long indexOf(long offset) { + Entry e = blockIndex.floorEntry(offset); + if (e == null) { + return -1; + } + BlockInfo blockInfo = e.getValue(); + if (offset - e.getKey() >= blockInfo.length) { + return -1; // This offset is out of managed bounds + } + return blockInfo.index + offset - e.getKey(); + } + + public long addOffset(long offset) { + Entry e = blockIndex.lastEntry(); + long offsetIndex = nextIndex++; + if (e != null) { + BlockInfo blockInfo = e.getValue(); + if (offset < e.getKey()) { + throw new IllegalArgumentException("can't regress"); + } + long nextOffset = e.getKey() + blockInfo.length; + if (offset == nextOffset) { + // No offset gap, can extend the last block + blockInfo.length++; + return offsetIndex; + } + } + // Offset gap or first entry after cleanup, needs to create a new block + blockIndex.put(offset, new BlockInfo(offsetIndex, 1)); + return offsetIndex; + } + } + + private final OffsetIndex index; + private final ConcurrentBitMap compFlags; + private final OffsetState[] states; + + public OffsetStorageComplex(int capacity) { + index = new OffsetIndex(); + compFlags = new ConcurrentBitMap(capacity); + states = new OffsetState[capacity]; + } + + public int size() { + return index.indexSize(); + } + + public long firstOffset() { + return index.firstOffset(); + } + + public void pollFirst() { + int firstIndex = (int) (index.pollFirst() % states.length); + compFlags.set(firstIndex, false); + states[firstIndex] = null; + } + + public int addOffset(long offset, boolean complete, OffsetState state) { + int nextIndex = (int) (index.addOffset(offset) % states.length); + compFlags.set(nextIndex, complete); + states[nextIndex] = state; + return nextIndex; + } + + public void complete(int ringIndex) { + // Q. Don't we need to guard against multiple-complete of the same offset, which, if it happens + // after the watermark progress, it can set complete flag on 1 or more round forward offset resulting + // a bug? + // A. This method itself contains that weakness, so this method assume the caller to take care of + // external control and prevent a same offset being completed more than once. + // In current OOOCC implementation, it is implemented by using CompletableFuture and call this method + // from the callback of CF on completion, which is guaranteed to be triggered just once even though + // the same CF completed more than once. + compFlags.set(ringIndex, true); + } + + public OffsetState firstState() { + return states[(int) (index.firstIndex() % states.length)]; + } + + public boolean isComplete(long offset) { + int ringIndex = (int) (index.indexOf(offset) % states.length); + if (ringIndex == -1) { + // By contract we expect the offset-out-of-range case to be just the offset being too large against + // managed range, not lower. + return false; + } + return compFlags.get(ringIndex); + } +} diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java index 4987036e..c8bffe5c 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java @@ -16,33 +16,25 @@ package com.linecorp.decaton.processor.runtime.internal; -import java.util.ArrayDeque; -import java.util.Deque; - import org.apache.kafka.common.TopicPartition; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; + +import lombok.Getter; +import lombok.experimental.Accessors; +import lombok.extern.slf4j.Slf4j; /** * Represents consumption processing progress of records consumed from a single partition. * This class manages sequence of offsets and a flag which represents if each of them was completed or not. */ +@Slf4j +@Accessors(fluent = true) public class OutOfOrderCommitControl implements AutoCloseable { - private static final Logger logger = LoggerFactory.getLogger(OutOfOrderCommitControl.class); - + @Getter private final TopicPartition topicPartition; private final int capacity; - private final Deque states; + private final OffsetStorageComplex complex; private final OffsetStateReaper offsetStateReaper; - /** - * The current smallest offset that has been reported but not completed. - */ - private volatile long earliest; - /** - * The current largest offset that has been reported. - */ - private volatile long latest; /** * The current maximum offset which it and all it's previous offsets were committed. */ @@ -51,82 +43,86 @@ public class OutOfOrderCommitControl implements AutoCloseable { public OutOfOrderCommitControl(TopicPartition topicPartition, int capacity, OffsetStateReaper offsetStateReaper) { this.topicPartition = topicPartition; - states = new ArrayDeque<>(capacity); + complex = new OffsetStorageComplex(capacity); this.capacity = capacity; this.offsetStateReaper = offsetStateReaper; - earliest = latest = 0; highWatermark = -1; } - public TopicPartition topicPartition() { - return topicPartition; - } - public synchronized OffsetState reportFetchedOffset(long offset) { if (isRegressing(offset)) { throw new OffsetRegressionException(String.format( - "offset regression %s: %d > %d", topicPartition, offset, latest)); + "offset regression %s: %d < %d", topicPartition, offset, highWatermark)); } - if (states.size() == capacity) { + if (complex.size() == capacity) { throw new IllegalArgumentException( String.format("offsets count overflow: cap=%d, offset=%d", capacity, offset)); } + if (complex.isComplete(offset)) { + // There are two cases for this. + // 1. Offset bigger than that complex's managing bounds. Reasonable to consider as not completed + // because it is the offset to coming in future (and will be added to complex in line below). + // 2. Offset has been processed in the past, marked as completed and now the consumer's consuming + // it again from the point of watermark. + return null; + } + OffsetState state = new OffsetState(offset); - states.addLast(state); - latest = state.offset(); + int ringIndex = complex.addOffset(offset, false, state); - state.completion().asFuture().whenComplete((unused, throwable) -> onComplete(offset)); // TODO okay in this order? + state.completion().asFuture().whenComplete((unused, throwable) -> onComplete(offset, ringIndex)); // TODO okay in this order? return state; } - static void onComplete(long offset) { - if (logger.isDebugEnabled()) { - logger.debug("Offset complete: {}", offset); + void onComplete(long offset, int ringIndex) { + if (log.isDebugEnabled()) { + log.debug("Offset complete: {}", offset); } + complex.complete(ringIndex); } public synchronized void updateHighWatermark() { - if (logger.isTraceEnabled()) { - StringBuilder sb = new StringBuilder("["); - - boolean first = true; - for (OffsetState st : states) { - if (first) { - first = false; - } else { - sb.append(", "); - } - sb.append(String.valueOf(st.offset()) + ':' + (st.completion().isComplete() ? 'c' : 'n')); - } - sb.append(']'); - logger.trace("Begin updateHighWatermark earliest={} latest={} hw={} states={}", - earliest, latest, highWatermark, sb); + if (log.isTraceEnabled()) { +// StringBuilder sb = new StringBuilder("["); +// +// boolean first = true; +// for (OffsetState st : states) { +// if (first) { +// first = false; +// } else { +// sb.append(", "); +// } +// sb.append(String.valueOf(st.offset()) + ':' + (st.completion().isComplete() ? 'c' : 'n')); +// } +// sb.append(']'); +// log.trace("Begin updateHighWatermark earliest={} latest={} hw={} states={}", +// earliest, latest, highWatermark, sb); } long lastHighWatermark = highWatermark; - OffsetState state; - while ((state = states.peekFirst()) != null) { - earliest = state.offset(); - if (state.completion().isComplete()) { - highWatermark = state.offset(); - states.pollFirst(); + while (complex.size() > 0) { + long offset = complex.firstOffset(); + if (complex.isComplete(offset)) { + highWatermark = offset; + complex.pollFirst(); } else { + OffsetState state = complex.firstState(); offsetStateReaper.maybeReapOffset(state); break; } } if (highWatermark != lastHighWatermark) { - logger.debug("High watermark updated for {}: {} => {}", - topicPartition, lastHighWatermark, highWatermark); + log.debug("High watermark updated for {}: {} => {}", + topicPartition, lastHighWatermark, highWatermark); } } public synchronized int pendingOffsetsCount() { - return states.size(); + return complex.size(); } public long commitReadyOffset() { @@ -134,15 +130,18 @@ public long commitReadyOffset() { } public boolean isRegressing(long offset) { - return offset < latest; + long firstOffset = complex.firstOffset(); + if (firstOffset < 0) { + return offset <= highWatermark; + } else { + return offset < firstOffset; + } } @Override public String toString() { return "OutOfOrderCommitControl{" + "topicPartition=" + topicPartition + - ", earliest=" + earliest + - ", latest=" + latest + ", highWatermark=" + highWatermark + '}'; } diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ConcurrentBitMapTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ConcurrentBitMapTest.java new file mode 100644 index 00000000..4298e341 --- /dev/null +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/ConcurrentBitMapTest.java @@ -0,0 +1,67 @@ +/* + * Copyright 2024 LY Corporation + * + * LY Corporation 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: + * + * https://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 com.linecorp.decaton.processor.runtime.internal; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; + +import java.util.Set; + +import org.junit.jupiter.api.Test; +import org.mockito.internal.util.collections.Sets; + +class ConcurrentBitMapTest { + private static void setGetCheck(ConcurrentBitMap map, Set trueIndicies) { + for (int index : trueIndicies) { + map.set(index, true); + } + for (int i = 0; i < map.size(); i++) { + assertEquals(trueIndicies.contains(i), map.get(i)); + } + for (int index : trueIndicies) { + map.set(index, false); + } + for (int i = 0; i < map.size(); i++) { + assertFalse(map.get(i)); + } + } + + @Test + void testSmall() { + ConcurrentBitMap map = new ConcurrentBitMap(10); + assertThrows(IndexOutOfBoundsException.class, () -> map.get(-1)); + assertThrows(IndexOutOfBoundsException.class, () -> map.get(10)); + setGetCheck(map, Sets.newSet(0, 3, 8)); + } + + @Test + void test64() { + ConcurrentBitMap map = new ConcurrentBitMap(Long.SIZE); + assertThrows(IndexOutOfBoundsException.class, () -> map.get(-1)); + assertThrows(IndexOutOfBoundsException.class, () -> map.get(Long.SIZE)); + setGetCheck(map, Sets.newSet(0, 10, Long.SIZE - 1)); + } + + @Test + void testLarge() { + ConcurrentBitMap map = new ConcurrentBitMap(Long.SIZE + 10); + assertThrows(IndexOutOfBoundsException.class, () -> map.get(-1)); + assertThrows(IndexOutOfBoundsException.class, () -> map.get(Long.SIZE + 10)); + setGetCheck(map, Sets.newSet(0, 10, Long.SIZE, Long.SIZE + 3, Long.SIZE + 8)); + } +} diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplexTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplexTest.java new file mode 100644 index 00000000..2ef8efdf --- /dev/null +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplexTest.java @@ -0,0 +1,95 @@ +/* + * Copyright 2024 LY Corporation + * + * LY Corporation 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: + * + * https://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 com.linecorp.decaton.processor.runtime.internal; + +import static org.junit.jupiter.api.Assertions.*; + +import org.junit.jupiter.api.Test; + +import com.linecorp.decaton.processor.runtime.internal.OffsetStorageComplex.OffsetIndex; + +class OffsetStorageComplexTest { + + @Test + void testOffsetIndex() { + OffsetIndex index = new OffsetIndex(); + + index.addOffset(10); // index 0 + index.addOffset(11); // index 1 + index.addOffset(12); // index 2 + index.addOffset(15); // index 3 + index.addOffset(16); // index 4 + index.addOffset(17); // index 5 + + assertEquals(0, index.firstIndex()); + assertEquals(0, index.indexOf(10)); + assertEquals(2, index.indexOf(12)); + assertEquals(3, index.indexOf(15)); + assertEquals(5, index.indexOf(17)); + assertEquals(-1, index.indexOf(9)); + assertEquals(-1, index.indexOf(18)); + + assertEquals(0, index.pollFirst()); + assertEquals(1, index.firstIndex()); + assertEquals(1, index.indexOf(11)); + assertEquals(2, index.indexOf(12)); + assertEquals(3, index.indexOf(15)); + assertEquals(5, index.indexOf(17)); + assertEquals(-1, index.indexOf(10)); + assertEquals(-1, index.indexOf(18)); + + index.pollFirst(); // 11 out + index.pollFirst(); // 12 out + assertEquals(3, index.pollFirst()); + assertEquals(4, index.firstIndex()); + assertEquals(4, index.indexOf(16)); + assertEquals(5, index.indexOf(17)); + assertEquals(-1, index.indexOf(15)); + assertEquals(-1, index.indexOf(18)); + + index.pollFirst(); // 14 out + index.pollFirst(); // 15 out + + index.addOffset(18); + assertEquals(6, index.firstIndex()); + assertEquals(6, index.indexOf(18)); + assertEquals(-1, index.indexOf(17)); + assertEquals(-1, index.indexOf(19)); + } + + private static void addOffset(OffsetStorageComplex complex, long offset) { + complex.addOffset(offset, false, new OffsetState(offset)); + } + + @Test + void test() { + OffsetStorageComplex complex = new OffsetStorageComplex(10); + + addOffset(complex, 10); + addOffset(complex, 11); + addOffset(complex, 12); + addOffset(complex, 15); + addOffset(complex, 16); + addOffset(complex, 17); + + assertEquals(6, complex.size()); + assertFalse(complex.isComplete(10)); + assertFalse(complex.isComplete(17)); + + complex.pollFirst(); + } +} From c2a8a8516fd04a781007f3abd658e192d91a15a1 Mon Sep 17 00:00:00 2001 From: Yuto Kawamura Date: Wed, 23 Oct 2024 18:48:25 +0900 Subject: [PATCH 02/12] Save/restore per-offset completion status as metadata --- processor/build.gradle | 1 + .../processor/metrics/MetricsTest.java | 5 +- .../runtime/ProcessorSubscription.java | 4 +- .../runtime/internal/AssignmentManager.java | 27 ++-- .../runtime/internal/ConcurrentBitMap.java | 35 +++- .../runtime/internal/ConsumeManager.java | 8 +- .../internal/OffsetStorageComplex.java | 66 +++++++- .../internal/OutOfOrderCommitControl.java | 55 ++++++- .../runtime/internal/PartitionContext.java | 26 +-- .../runtime/internal/PartitionContexts.java | 23 +-- .../runtime/ProcessorSubscriptionTest.java | 22 ++- .../internal/AssignmentManagerTest.java | 12 +- .../internal/OutOfOrderCommitControlTest.java | 151 +++++++++++------- .../internal/PartitionContextTest.java | 19 ++- .../internal/PartitionContextsTest.java | 38 ++--- protocol/src/main/proto/decaton.proto | 22 +++ 16 files changed, 372 insertions(+), 142 deletions(-) diff --git a/processor/build.gradle b/processor/build.gradle index 764cde3e..d48f28fb 100644 --- a/processor/build.gradle +++ b/processor/build.gradle @@ -8,6 +8,7 @@ dependencies { implementation "net.openhft:zero-allocation-hashing:0.16" implementation "org.slf4j:slf4j-api:$slf4jVersion" + implementation "com.google.protobuf:protobuf-java-util:$protobufVersion" testImplementation project(":protobuf") diff --git a/processor/src/it/java/com/linecorp/decaton/processor/metrics/MetricsTest.java b/processor/src/it/java/com/linecorp/decaton/processor/metrics/MetricsTest.java index df433e68..88f47242 100644 --- a/processor/src/it/java/com/linecorp/decaton/processor/metrics/MetricsTest.java +++ b/processor/src/it/java/com/linecorp/decaton/processor/metrics/MetricsTest.java @@ -246,11 +246,12 @@ public void testDeferredCompletionLeak() throws Exception { () -> Metrics.registry() .find("decaton.offset.last.committed") .tags("topic", topicName, "partition", "0") - .gauge().value() == 1.0); + .gauge().value() == 2.0); TestUtils.awaitCondition("latest consumed offset should becomes 9", () -> Metrics.registry() .find("decaton.offset.latest.consumed") .tags("topic", topicName, "partition", "0") .gauge().value() == 9.0); - }} + } + } } diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java index 05537878..f9a7a47a 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java @@ -20,6 +20,7 @@ import java.time.Duration; import java.util.Collection; +import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -29,6 +30,7 @@ import org.apache.kafka.clients.consumer.CommitFailedException; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.RebalanceInProgressException; import org.apache.kafka.common.errors.TimeoutException; @@ -93,7 +95,7 @@ public void prepareForRebalance(Collection revokingPartitions) { } @Override - public void updateAssignment(Collection newAssignment) { + public void updateAssignment(Map newAssignment) { assignManager.assign(newAssignment); updateState(SubscriptionStateListener.State.RUNNING); diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/AssignmentManager.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/AssignmentManager.java index 7deadbcc..575654d0 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/AssignmentManager.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/AssignmentManager.java @@ -21,12 +21,14 @@ import java.util.Collection; import java.util.Collections; -import java.util.HashSet; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Set; import java.util.function.Function; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import com.linecorp.decaton.processor.runtime.internal.Utils.Timer; @@ -44,6 +46,7 @@ public class AssignmentManager { @Accessors(fluent = true) static class AssignmentConfig { boolean paused; + OffsetAndMetadata offsetMeta; } /** @@ -94,11 +97,13 @@ public AssignmentManager(AssignmentStore store) { * from/to the store. * @param newAssignment new set of topic-partitions to assign. */ - public void assign(Collection newAssignment) { - Set newSet = new HashSet<>(newAssignment); + public void assign(Map newAssignment) { + Set newSet = newAssignment.keySet(); Set oldSet = store.assignedPartitions(); List removed = computeRemovedPartitions(oldSet, newSet); - List added = computeAddedPartitions(oldSet, newSet); + Map added = computeAddedPartitions(oldSet, newSet) + .stream() + .collect(HashMap::new, (m, v) -> m.put(v, newAssignment.get(v)), HashMap::putAll); log.debug("Assignment update: removed:{}, added:{}, assignment:{}", removed, added, newSet); partitionsRevoked(removed); @@ -113,9 +118,8 @@ public void assign(Collection newAssignment) { */ public void repair(TopicPartition tp) { log.info("Repairing partition: {}", tp); - List target = Collections.singletonList(tp); - partitionsRevoked(target); - partitionsAssigned(target); + partitionsRevoked(Collections.singletonList(tp)); + partitionsAssigned(Collections.singletonMap(tp, null)); } private static List computeRemovedPartitions( @@ -140,18 +144,19 @@ private void partitionsRevoked(Collection partitions) { } } - private void partitionsAssigned(Collection partitions) { - if (partitions.isEmpty()) { + private void partitionsAssigned(Map partitionCommits) { + if (partitionCommits.isEmpty()) { return; } Timer timer = Utils.timer(); Map configs = - partitions.stream().collect(toMap(Function.identity(), ignored -> new AssignmentConfig(false))); + partitionCommits.entrySet().stream() + .collect(toMap(Entry::getKey, e -> new AssignmentConfig(false, e.getValue()))); store.addPartitions(configs); if (log.isInfoEnabled()) { log.info("Added {} partitions in {} ms", - partitions.size(), Utils.formatNum(timer.elapsedMillis())); + partitionCommits.size(), Utils.formatNum(timer.elapsedMillis())); } } } diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConcurrentBitMap.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConcurrentBitMap.java index 95adcb71..8dd81cfa 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConcurrentBitMap.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConcurrentBitMap.java @@ -18,22 +18,31 @@ import java.util.concurrent.atomic.AtomicLongArray; +import com.linecorp.decaton.protocol.Decaton.BitMapProto; + +import lombok.AllArgsConstructor; import lombok.Getter; +import lombok.ToString; import lombok.experimental.Accessors; @Accessors(fluent = true) +@AllArgsConstructor +@ToString class ConcurrentBitMap { @Getter private final int size; private final AtomicLongArray buckets; ConcurrentBitMap(int size) { - this.size = size; + this(size, new AtomicLongArray(bucketSize(size))); + } + + static int bucketSize(int size) { int nbuckets = size / Long.SIZE; if (size % Long.SIZE != 0) { nbuckets++; } - buckets = new AtomicLongArray(nbuckets); + return nbuckets; } private static int bucketOf(int index) { @@ -66,6 +75,26 @@ public void set(int index, boolean on) { public boolean get(int index) { ensureBound(index); int bucket = bucketOf(index); - return (buckets.get(bucket) >> localIndex(index) & 1) == 1; + long bits = buckets.get(bucket); + boolean x = (bits >> localIndex(index) & 1) == 1; + System.err.println("BITMAP GET of " + index + " is " + x + ", bits = " + bits); + return x; + } + + public BitMapProto toProto() { + BitMapProto.Builder builder = BitMapProto.newBuilder() + .setSize(size); + for (int i = 0; i < buckets.length(); i++) { + builder.addBuckets(buckets.get(i)); + } + return builder.build(); + } + + public static ConcurrentBitMap fromProto(BitMapProto proto) { + AtomicLongArray buckets = new AtomicLongArray(proto.getBucketsCount()); + for (int i = 0; i < proto.getBucketsCount(); i++) { + buckets.set(i, proto.getBuckets(i)); + } + return new ConcurrentBitMap(proto.getSize(), buckets); } } diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConsumeManager.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConsumeManager.java index 87b49b44..50645b33 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConsumeManager.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConsumeManager.java @@ -20,6 +20,7 @@ import java.util.Collection; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; @@ -27,6 +28,7 @@ import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import com.linecorp.decaton.processor.metrics.Metrics.SubscriptionMetrics; @@ -88,7 +90,7 @@ public interface ConsumerHandler { * Update assignment with given partitions list. * @param newAssignment list of partitions that are now actively assigned. */ - void updateAssignment(Collection newAssignment); + void updateAssignment(Map newAssignment); /** * Process a {@link ConsumerRecord} that has been consumed from the topic. @@ -136,7 +138,9 @@ public void onPartitionsRevoked(Collection partitions) { @Override public void onPartitionsAssigned(Collection ignored) { - handler.updateAssignment(consumer.assignment()); + Map partitionCommits + = consumer.committed(consumer.assignment()); + handler.updateAssignment(partitionCommits); // Consumer rebalance resets all pause states of assigned partitions even though they // haven't moved over from/to different consumer instance. diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java index f2e49702..67e5f09e 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java @@ -19,14 +19,26 @@ import java.util.Map.Entry; import java.util.TreeMap; +import com.linecorp.decaton.protocol.Decaton.BitMapProto; +import com.linecorp.decaton.protocol.Decaton.OffsetIndexEntryProto; +import com.linecorp.decaton.protocol.Decaton.OffsetIndexProto; +import com.linecorp.decaton.protocol.Decaton.OffsetStorageComplexProto; + +import lombok.AccessLevel; import lombok.AllArgsConstructor; import lombok.Getter; +import lombok.ToString; import lombok.experimental.Accessors; @Accessors(fluent = true) +@AllArgsConstructor(access = AccessLevel.PRIVATE) +@ToString public class OffsetStorageComplex { + @AllArgsConstructor(access = AccessLevel.PACKAGE) + @ToString static class OffsetIndex { @AllArgsConstructor + @ToString static class BlockInfo { long index; int length; @@ -39,7 +51,7 @@ static class BlockInfo { private long nextIndex; OffsetIndex() { - blockIndex = new TreeMap<>(); + this(new TreeMap<>(), 0, 0); } public int indexSize() { @@ -80,10 +92,11 @@ public long addOffset(long offset) { long offsetIndex = nextIndex++; if (e != null) { BlockInfo blockInfo = e.getValue(); - if (offset < e.getKey()) { - throw new IllegalArgumentException("can't regress"); - } long nextOffset = e.getKey() + blockInfo.length; + if (offset < nextOffset) { + // throw new IllegalArgumentException("can't regress"); + throw new OffsetRegressionException("offset regression at " + offset); + } if (offset == nextOffset) { // No offset gap, can extend the last block blockInfo.length++; @@ -94,6 +107,31 @@ public long addOffset(long offset) { blockIndex.put(offset, new BlockInfo(offsetIndex, 1)); return offsetIndex; } + + public OffsetIndexProto toProto() { + OffsetIndexProto.Builder builder = OffsetIndexProto.newBuilder() + .setFirstIndex(firstIndex) + .setNextIndex(nextIndex); + for (Entry entry : blockIndex.entrySet()) { + long offset = entry.getKey(); + BlockInfo blockInfo = entry.getValue(); + OffsetIndexEntryProto entryProto = OffsetIndexEntryProto.newBuilder() + .setStartOffset(offset) + .setStartIndex(blockInfo.index) + .setLength(blockInfo.length).build(); + builder.addEntries(entryProto); + } + return builder.build(); + } + + public static OffsetIndex fromProto(OffsetIndexProto proto) { + TreeMap blockIndex = new TreeMap<>(); + for (int i = 0; i < proto.getEntriesCount(); i++) { + OffsetIndexEntryProto entry = proto.getEntries(i); + blockIndex.put(entry.getStartOffset(), new BlockInfo(entry.getStartIndex(), entry.getLength())); + } + return new OffsetIndex(blockIndex, proto.getFirstIndex(), proto.getNextIndex()); + } } private final OffsetIndex index; @@ -101,9 +139,7 @@ public long addOffset(long offset) { private final OffsetState[] states; public OffsetStorageComplex(int capacity) { - index = new OffsetIndex(); - compFlags = new ConcurrentBitMap(capacity); - states = new OffsetState[capacity]; + this(new OffsetIndex(), new ConcurrentBitMap(capacity), new OffsetState[capacity]); } public int size() { @@ -152,4 +188,20 @@ public boolean isComplete(long offset) { } return compFlags.get(ringIndex); } + + public OffsetStorageComplexProto toProto() { + OffsetIndexProto indexProto = index.toProto(); + BitMapProto compFlagsProto = compFlags.toProto(); + + return OffsetStorageComplexProto.newBuilder() + .setIndex(indexProto) + .setCompFlags(compFlagsProto) + .build(); + } + + public static OffsetStorageComplex fromProto(OffsetStorageComplexProto proto) { + OffsetIndex index = OffsetIndex.fromProto(proto.getIndex()); + ConcurrentBitMap compFlags = ConcurrentBitMap.fromProto(proto.getCompFlags()); + return new OffsetStorageComplex(index, compFlags, new OffsetState[compFlags.size()]); + } } diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java index c8bffe5c..4e0654c4 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java @@ -16,8 +16,15 @@ package com.linecorp.decaton.processor.runtime.internal; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; +import com.google.protobuf.InvalidProtocolBufferException; +import com.google.protobuf.util.JsonFormat; + +import com.linecorp.decaton.protocol.Decaton.OffsetStorageComplexProto; + +import lombok.AllArgsConstructor; import lombok.Getter; import lombok.experimental.Accessors; import lombok.extern.slf4j.Slf4j; @@ -28,12 +35,13 @@ */ @Slf4j @Accessors(fluent = true) +@AllArgsConstructor public class OutOfOrderCommitControl implements AutoCloseable { @Getter private final TopicPartition topicPartition; private final int capacity; - private final OffsetStorageComplex complex; private final OffsetStateReaper offsetStateReaper; + final OffsetStorageComplex complex; /** * The current maximum offset which it and all it's previous offsets were committed. @@ -49,6 +57,14 @@ public OutOfOrderCommitControl(TopicPartition topicPartition, int capacity, highWatermark = -1; } + public static OutOfOrderCommitControl fromOffsetMeta(TopicPartition tp, + int capacity, + OffsetStateReaper offsetStateReaper, + OffsetAndMetadata offsetMeta) { + OffsetStorageComplex complex = complexFromMeta(offsetMeta.metadata()); + return new OutOfOrderCommitControl(tp, capacity, offsetStateReaper, complex, offsetMeta.offset()); + } + public synchronized OffsetState reportFetchedOffset(long offset) { if (isRegressing(offset)) { throw new OffsetRegressionException(String.format( @@ -57,7 +73,7 @@ public synchronized OffsetState reportFetchedOffset(long offset) { if (complex.size() == capacity) { throw new IllegalArgumentException( - String.format("offsets count overflow: cap=%d, offset=%d", capacity, offset)); + String.format("offsets count overflow: size=%d, cap=%d", complex.size(), capacity)); } if (complex.isComplete(offset)) { @@ -105,12 +121,15 @@ public synchronized void updateHighWatermark() { while (complex.size() > 0) { long offset = complex.firstOffset(); - if (complex.isComplete(offset)) { + boolean complete = complex.isComplete(offset); + if (complete) { highWatermark = offset; complex.pollFirst(); } else { OffsetState state = complex.firstState(); - offsetStateReaper.maybeReapOffset(state); + if (state != null) { + offsetStateReaper.maybeReapOffset(state); + } break; } } @@ -125,8 +144,32 @@ public synchronized int pendingOffsetsCount() { return complex.size(); } - public long commitReadyOffset() { - return highWatermark; + public OffsetAndMetadata commitReadyOffset() { + if (highWatermark < 0) { + return null; + } + OffsetStorageComplexProto complexProto = complex.toProto(); + try { + String meta = JsonFormat.printer().print(complexProto); // TODO: 64-bit ints are dumped as strings + return new OffsetAndMetadata(highWatermark + 1, meta); + } catch (InvalidProtocolBufferException e) { + throw new RuntimeException("failed to serialize offset metadata into proto", e); + } + } + + static OffsetStorageComplex complexFromMeta(String metadata) { + OffsetStorageComplexProto proto = parseOffsetMeta(metadata); + return OffsetStorageComplex.fromProto(proto); + } + + static OffsetStorageComplexProto parseOffsetMeta(String metadata) { + OffsetStorageComplexProto.Builder builder = OffsetStorageComplexProto.newBuilder(); + try { + JsonFormat.parser().merge(metadata, builder); + } catch (InvalidProtocolBufferException e) { + throw new IllegalArgumentException(e); + } + return builder.build(); } public boolean isRegressing(long offset) { diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/PartitionContext.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/PartitionContext.java index 3aff3419..cf08d624 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/PartitionContext.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/PartitionContext.java @@ -17,11 +17,12 @@ package com.linecorp.decaton.processor.runtime.internal; import java.util.Collection; -import java.util.OptionalLong; +import java.util.Optional; import java.util.concurrent.TimeUnit; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import com.linecorp.decaton.processor.metrics.Metrics; @@ -70,8 +71,8 @@ public class PartitionContext implements AutoCloseable { @Setter private volatile boolean reloadRequested; - public PartitionContext(PartitionScope scope, Processors processors) { - this(scope, processors, createSubPartitions(scope, processors)); + public PartitionContext(PartitionScope scope, OffsetAndMetadata offsetMeta, Processors processors) { + this(scope, offsetMeta, processors, createSubPartitions(scope, processors)); } private static SubPartitions createSubPartitions(PartitionScope scope, Processors processors) { @@ -87,6 +88,7 @@ private static SubPartitions createSubPartitions(PartitionScope scope, Processor // visible for testing PartitionContext(PartitionScope scope, + OffsetAndMetadata offsetMeta, Processors processors, SubPartitions subPartitions) { this.scope = scope; @@ -102,7 +104,13 @@ private static SubPartitions createSubPartitions(PartitionScope scope, Processor OffsetStateReaper offsetStateReaper = new OffsetStateReaper( scope.props().get(ProcessorProperties.CONFIG_DEFERRED_COMPLETE_TIMEOUT_MS), metricsCtor.new CommitControlMetrics()); - commitControl = new OutOfOrderCommitControl(scope.topicPartition(), capacity, offsetStateReaper); + if (offsetMeta == null) { + commitControl = new OutOfOrderCommitControl(scope.topicPartition(), capacity, offsetStateReaper); + } else { + System.err.println("Creating OOOCC from offsetMeta: " + offsetMeta); + commitControl = OutOfOrderCommitControl.fromOffsetMeta( + scope.topicPartition(), capacity, offsetStateReaper, offsetMeta); + } if (scope.perKeyQuotaConfig().isPresent() && scope.originTopic().equals(scope.topicPartition().topic())) { perKeyQuotaManager = PerKeyQuotaManager.create(scope); } else { @@ -125,12 +133,12 @@ private static SubPartitions createSubPartitions(PartitionScope scope, Processor * * @return optional long value representing an offset waiting for commit. */ - public OptionalLong offsetWaitingCommit() { - long readyOffset = commitControl.commitReadyOffset(); - if (readyOffset > lastCommittedOffset) { - return OptionalLong.of(readyOffset); + public Optional offsetWaitingCommit() { + OffsetAndMetadata offsetMeta = commitControl.commitReadyOffset(); + if (offsetMeta != null && offsetMeta.offset() > lastCommittedOffset) { + return Optional.of(offsetMeta); } - return OptionalLong.empty(); + return Optional.empty(); } /** diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/PartitionContexts.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/PartitionContexts.java index 1901fe06..78860d5b 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/PartitionContexts.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/PartitionContexts.java @@ -81,7 +81,7 @@ public void addPartitions(Map partitions) { for (Entry entry : partitions.entrySet()) { TopicPartition tp = entry.getKey(); AssignmentConfig conf = entry.getValue(); - initContext(tp, conf.paused()); + initContext(tp, conf.offsetMeta(), conf.paused()); } } finally { propertyReloadLock.unlock(); @@ -122,10 +122,10 @@ private void cleanupPartitions(Collection partitions) { * @return instantiated context */ // visible for testing - PartitionContext initContext(TopicPartition tp, boolean paused) { + PartitionContext initContext(TopicPartition tp, OffsetAndMetadata offsetMeta, boolean paused) { propertyReloadLock.lock(); try { - PartitionContext context = instantiateContext(tp); + PartitionContext context = instantiateContext(tp, offsetMeta); if (paused) { context.pause(); } @@ -150,8 +150,7 @@ public Map commitReadyOffsets() { for (PartitionContext context : contexts.values()) { if (!context.revoking()) { context.offsetWaitingCommit().ifPresent( - offset -> offsets.put(context.topicPartition(), - new OffsetAndMetadata(offset + 1, null))); + offsetMeta -> offsets.put(context.topicPartition(), offsetMeta)); } } return offsets; @@ -162,9 +161,7 @@ public void storeCommittedOffsets(Map offsets for (Entry entry : offsets.entrySet()) { TopicPartition tp = entry.getKey(); long offset = entry.getValue().offset(); - // PartitionContext manages their "completed" offset so its minus 1 from committed offset - // which indicates the offset to "fetch next". - contexts.get(tp).updateCommittedOffset(offset - 1); + contexts.get(tp).updateCommittedOffset(offset); } } @@ -197,9 +194,9 @@ public void updateHighWatermarks() { } // visible for testing - PartitionContext instantiateContext(TopicPartition tp) { + PartitionContext instantiateContext(TopicPartition tp, OffsetAndMetadata offsetMeta) { PartitionScope partitionScope = new PartitionScope(scope, tp); - return new PartitionContext(partitionScope, processors); + return new PartitionContext(partitionScope, offsetMeta, processors); } // visible for testing @@ -306,8 +303,12 @@ private void reloadContexts(Collection topicPartitions) { logger.info("Start dropping partition contexts({})", topicPartitions); removePartition(topicPartitions); logger.info("Finished dropping partition contexts. Start recreating partition contexts"); + // It is OK to skip passing correct offsetMeta at this point, because when flow reaches this point + // partition has all the record processed and committed already, hence the watermark should already + // be up-to-date and committed. + AssignmentConfig config = new AssignmentConfig(true, null); Map configs = topicPartitions.stream().collect( - toMap(Function.identity(), tp -> new AssignmentConfig(true))); + toMap(Function.identity(), tp -> config)); addPartitions(configs); logger.info("Completed reloading partition contexts({})", topicPartitions); } diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/ProcessorSubscriptionTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/ProcessorSubscriptionTest.java index 04076617..ac1bd9da 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/ProcessorSubscriptionTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/ProcessorSubscriptionTest.java @@ -18,7 +18,9 @@ import static java.util.Collections.singleton; import static java.util.Collections.singletonMap; +import static java.util.stream.Collectors.toMap; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyCollection; @@ -37,6 +39,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; @@ -47,6 +50,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.stream.Collectors; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; @@ -101,6 +106,17 @@ public synchronized void subscribe(Collection topics, ConsumerRebalanceL super.subscribe(topics, listener); } + // Override this method to mimic actual behavier produced by KafkaConsumer. + // The real producer returns a map with null value associated to partitions w/ no committed record + // while MockConsumer omits entry itself from response. + @Override + public synchronized Map committed(Set partitions) { + Map allValues = + partitions.stream().collect(HashMap::new, (m, v) -> m.put(v, null), HashMap::putAll); + allValues.putAll(super.committed(partitions)); + return allValues; + } + @Override public void close(Duration timeout) {} } @@ -191,6 +207,10 @@ public void testOffsetRegression() throws Exception { listener.set(invocation.getArgument(1)); return null; }).when(consumer).subscribe(anyCollection(), any(ConsumerRebalanceListener.class)); + doAnswer(invocation -> { + Set partitions = invocation.getArgument(0); + return partitions.stream().collect(HashMap::new, (m, v) -> m.put(v, null), HashMap::putAll); + }).when(consumer).committed(any(Set.class)); BlockingQueue feedOffsets = new ArrayBlockingQueue<>(4); feedOffsets.add(100L); @@ -298,7 +318,7 @@ public synchronized ConsumerRecords poll(Duration timeout) { consumer.addRecord(new ConsumerRecord<>(tp.topic(), tp.partition(), 12, new byte[0], NO_DATA)); asyncProcessingStarted.await(); CompletableFuture shutdownFut = subscription.asyncClose(); - assertTrue(consumer.committed(singleton(tp)).isEmpty()); + assertNull(consumer.committed(singleton(tp)).get(tp)); assertEquals(3, subscription.contexts.totalPendingTasks()); letTasksComplete.countDown(); letTaskFinishBlocking.release(2); diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/AssignmentManagerTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/AssignmentManagerTest.java index 636ae7ec..77d8e4e7 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/AssignmentManagerTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/AssignmentManagerTest.java @@ -26,10 +26,13 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.stream.Stream; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -64,8 +67,10 @@ private static TopicPartition tp(int partition) { @Test public void testAssign() { doReturn(emptySet()).when(store).assignedPartitions(); + Map partitionCommits = + Stream.of(tp(1), tp(2), tp(3)).collect(HashMap::new, (m, v) -> m.put(v, null), HashMap::putAll); List partitions = asList(tp(1), tp(2), tp(3)); - assignManager.assign(partitions); + assignManager.assign(partitionCommits); verify(store, times(1)).addPartitions(captor.capture()); HashSet newAssign = new HashSet<>(partitions); @@ -74,8 +79,9 @@ public void testAssign() { verify(store, times(1)).unmarkRevoking(newAssign); doReturn(newAssign).when(store).assignedPartitions(); - partitions = asList(tp(2), tp(3), tp(4), tp(5)); - assignManager.assign(partitions); + partitionCommits = Stream.of(tp(2), tp(3), tp(4), tp(5)).collect( + HashMap::new, (m, v) -> m.put(v, null), HashMap::putAll); + assignManager.assign(partitionCommits); verify(store, times(2)).addPartitions(captor.capture()); assertEquals(new HashSet<>(asList(tp(4), tp(5))), captor.getValue().keySet()); diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControlTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControlTest.java index 39322ec2..d981ef97 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControlTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControlTest.java @@ -17,12 +17,16 @@ package com.linecorp.decaton.processor.runtime.internal; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import java.time.Clock; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -35,122 +39,122 @@ public class OutOfOrderCommitControlTest { private static final int STATES_CAPACITY = 1000; private final TopicPartition topicPartition = new TopicPartition("topic", 0); - private final OutOfOrderCommitControl partitionState = + private final OutOfOrderCommitControl commitControl = new OutOfOrderCommitControl(topicPartition, STATES_CAPACITY, mock(OffsetStateReaper.class)); @Test public void testInOrderOffsetCompletion() { - OffsetState state1 = partitionState.reportFetchedOffset(1); - OffsetState state2 = partitionState.reportFetchedOffset(2); - OffsetState state3 = partitionState.reportFetchedOffset(3); + OffsetState state1 = commitControl.reportFetchedOffset(1); + OffsetState state2 = commitControl.reportFetchedOffset(2); + OffsetState state3 = commitControl.reportFetchedOffset(3); state1.completion().complete(); - partitionState.updateHighWatermark(); - assertEquals(2, partitionState.pendingOffsetsCount()); - assertEquals(1, partitionState.commitReadyOffset()); + commitControl.updateHighWatermark(); + assertEquals(2, commitControl.pendingOffsetsCount()); + assertEquals(2, commitControl.commitReadyOffset().offset()); state2.completion().complete(); - partitionState.updateHighWatermark(); - assertEquals(1, partitionState.pendingOffsetsCount()); - assertEquals(2, partitionState.commitReadyOffset()); + commitControl.updateHighWatermark(); + assertEquals(1, commitControl.pendingOffsetsCount()); + assertEquals(3, commitControl.commitReadyOffset().offset()); state3.completion().complete(); - partitionState.updateHighWatermark(); - assertEquals(0, partitionState.pendingOffsetsCount()); - assertEquals(3, partitionState.commitReadyOffset()); + commitControl.updateHighWatermark(); + assertEquals(0, commitControl.pendingOffsetsCount()); + assertEquals(4, commitControl.commitReadyOffset().offset()); } @Test public void testOutOfOrderOffsetCompletion() { - OffsetState state1 = partitionState.reportFetchedOffset(1); - OffsetState state2 = partitionState.reportFetchedOffset(2); - OffsetState state3 = partitionState.reportFetchedOffset(3); - OffsetState state4 = partitionState.reportFetchedOffset(4); + OffsetState state1 = commitControl.reportFetchedOffset(1); + OffsetState state2 = commitControl.reportFetchedOffset(2); + OffsetState state3 = commitControl.reportFetchedOffset(3); + OffsetState state4 = commitControl.reportFetchedOffset(4); state3.completion().complete(); - partitionState.updateHighWatermark(); - assertEquals(4, partitionState.pendingOffsetsCount()); - assertEquals(-1, partitionState.commitReadyOffset()); + commitControl.updateHighWatermark(); + assertEquals(4, commitControl.pendingOffsetsCount()); + assertNull(commitControl.commitReadyOffset()); state2.completion().complete(); - partitionState.updateHighWatermark(); - assertEquals(4, partitionState.pendingOffsetsCount()); - assertEquals(-1, partitionState.commitReadyOffset()); + commitControl.updateHighWatermark(); + assertEquals(4, commitControl.pendingOffsetsCount()); + assertNull(commitControl.commitReadyOffset()); state1.completion().complete(); - partitionState.updateHighWatermark(); - assertEquals(1, partitionState.pendingOffsetsCount()); - assertEquals(3, partitionState.commitReadyOffset()); + commitControl.updateHighWatermark(); + assertEquals(1, commitControl.pendingOffsetsCount()); + assertEquals(4, commitControl.commitReadyOffset().offset()); state4.completion().complete(); - partitionState.updateHighWatermark(); - assertEquals(0, partitionState.pendingOffsetsCount()); - assertEquals(4, partitionState.commitReadyOffset()); + commitControl.updateHighWatermark(); + assertEquals(0, commitControl.pendingOffsetsCount()); + assertEquals(5, commitControl.commitReadyOffset().offset()); } @Test public void testDoubleCompletingSameOffset() { - OffsetState state1 = partitionState.reportFetchedOffset(1); + OffsetState state1 = commitControl.reportFetchedOffset(1); state1.completion().complete(); - assertEquals(-1, partitionState.commitReadyOffset()); + assertNull(commitControl.commitReadyOffset()); state1.completion().complete(); // nothing happens - partitionState.updateHighWatermark(); - assertEquals(1, partitionState.commitReadyOffset()); + commitControl.updateHighWatermark(); + assertEquals(2, commitControl.commitReadyOffset().offset()); state1.completion().complete(); // nothing happens } @Test public void testReportingTooLargeOffset() { - partitionState.reportFetchedOffset(1); // now earliest=1 + commitControl.reportFetchedOffset(1); // now earliest=1 for (int i = 0; i < STATES_CAPACITY - 1; i++) { - partitionState.reportFetchedOffset(1 + i); + commitControl.reportFetchedOffset(1 + i); } - assertThrows(IllegalArgumentException.class, () -> partitionState.reportFetchedOffset(STATES_CAPACITY)); + assertThrows(IllegalArgumentException.class, () -> commitControl.reportFetchedOffset(STATES_CAPACITY)); } @Test public void testDoubleCompletingSameOffsetCaseDuplicateInCommitted() { - partitionState.reportFetchedOffset(1); - OffsetState state2 = partitionState.reportFetchedOffset(2); + commitControl.reportFetchedOffset(1); + OffsetState state2 = commitControl.reportFetchedOffset(2); state2.completion().complete(); // now committedOffsets contains 2 - partitionState.updateHighWatermark(); - assertEquals(-1, partitionState.commitReadyOffset()); + commitControl.updateHighWatermark(); + assertNull(commitControl.commitReadyOffset()); state2.completion().complete(); // commit again - partitionState.updateHighWatermark(); - assertEquals(-1, partitionState.commitReadyOffset()); + commitControl.updateHighWatermark(); + assertNull(commitControl.commitReadyOffset()); } @Test public void testPendingRecordsCountWithGaps() { - OffsetState state1 = partitionState.reportFetchedOffset(1); - assertEquals(1, partitionState.pendingOffsetsCount()); + OffsetState state1 = commitControl.reportFetchedOffset(1); + assertEquals(1, commitControl.pendingOffsetsCount()); - OffsetState state3 = partitionState.reportFetchedOffset(3); - assertEquals(2, partitionState.pendingOffsetsCount()); + OffsetState state3 = commitControl.reportFetchedOffset(3); + assertEquals(2, commitControl.pendingOffsetsCount()); state1.completion().complete(); state3.completion().complete(); - partitionState.updateHighWatermark(); - assertEquals(3, partitionState.commitReadyOffset()); - assertEquals(0, partitionState.pendingOffsetsCount()); + commitControl.updateHighWatermark(); + assertEquals(4, commitControl.commitReadyOffset().offset()); + assertEquals(0, commitControl.pendingOffsetsCount()); } @Test public void testPendingRecordsCountWithLargeGap() { - OffsetState state1 = partitionState.reportFetchedOffset(1); - assertEquals(1, partitionState.pendingOffsetsCount()); + OffsetState state1 = commitControl.reportFetchedOffset(1); + assertEquals(1, commitControl.pendingOffsetsCount()); long largeGapOffset = 1 + STATES_CAPACITY; - OffsetState stateLarge = partitionState.reportFetchedOffset(largeGapOffset); - assertEquals(2, partitionState.pendingOffsetsCount()); + OffsetState stateLarge = commitControl.reportFetchedOffset(largeGapOffset); + assertEquals(2, commitControl.pendingOffsetsCount()); state1.completion().complete(); stateLarge.completion().complete(); - partitionState.updateHighWatermark(); - assertEquals(largeGapOffset, partitionState.commitReadyOffset()); - assertEquals(0, partitionState.pendingOffsetsCount()); + commitControl.updateHighWatermark(); + assertEquals(largeGapOffset + 1, commitControl.commitReadyOffset().offset()); + assertEquals(0, commitControl.pendingOffsetsCount()); } @Test @@ -174,15 +178,44 @@ public void testTimeoutOffsetReaping() { // 1 is blocking watermark to progress state2.completion().complete(); ooocc.updateHighWatermark(); - assertEquals(-1, ooocc.commitReadyOffset()); + assertNull(ooocc.commitReadyOffset()); doReturn(20L).when(clock).millis(); // offset reaping performed but does not proceed watermark yet ooocc.updateHighWatermark(); state1.completion().asFuture().toCompletableFuture().join(); - assertEquals(-1, ooocc.commitReadyOffset()); + assertNull(ooocc.commitReadyOffset()); // offset should progress as offset 1 has reaped in previous call ooocc.updateHighWatermark(); - assertEquals(2, ooocc.commitReadyOffset()); + assertEquals(3, ooocc.commitReadyOffset().offset()); + } + + @Test + void perOffsetCompleteTest() { + OffsetState state1 = commitControl.reportFetchedOffset(101); + OffsetState state2 = commitControl.reportFetchedOffset(102); + OffsetState state3 = commitControl.reportFetchedOffset(103); + + state1.completion().complete(); + state3.completion().complete(); + + commitControl.updateHighWatermark(); + OffsetAndMetadata om = commitControl.commitReadyOffset(); + System.err.println("ooocc.complex = " + commitControl.complex); + + assertEquals(om.offset(), 102L); + OffsetStorageComplex complex = OutOfOrderCommitControl.complexFromMeta(om.metadata()); + System.err.println("meta = " + om.metadata()); + assertEquals(2, complex.size()); + assertFalse(complex.isComplete(102)); + assertTrue(complex.isComplete(103)); + + state2.completion().complete(); + commitControl.updateHighWatermark(); + om = commitControl.commitReadyOffset(); + + assertEquals(om.offset(), 104L); + complex = OutOfOrderCommitControl.complexFromMeta(om.metadata()); + assertEquals(0, complex.size()); } } diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/PartitionContextTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/PartitionContextTest.java index e5f5db79..b041a385 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/PartitionContextTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/PartitionContextTest.java @@ -31,6 +31,7 @@ import java.util.OptionalLong; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -72,7 +73,7 @@ private static PartitionScope scope(String topic, Optional pe @Test public void testOffsetWaitingCommit() { - PartitionContext context = new PartitionContext(scope("topic", Optional.empty()), processors); + PartitionContext context = new PartitionContext(scope("topic", Optional.empty()), null, processors); assertFalse(context.offsetWaitingCommit().isPresent()); OffsetState state = context.registerOffset(100); @@ -80,15 +81,15 @@ public void testOffsetWaitingCommit() { state.completion().complete(); context.updateHighWatermark(); - assertEquals(OptionalLong.of(100), context.offsetWaitingCommit()); + assertEquals(101, context.offsetWaitingCommit().map(OffsetAndMetadata::offset).get()); - context.updateCommittedOffset(100); + context.updateCommittedOffset(101); assertFalse(context.offsetWaitingCommit().isPresent()); } @Test public void testShouldPausePartition() { - PartitionContext context = new PartitionContext(scope("topic", Optional.empty()), processors); + PartitionContext context = new PartitionContext(scope("topic", Optional.empty()), null, processors); assertFalse(context.shouldPausePartition()); // Register MAX-1 records, which should not pause the partition. @@ -121,25 +122,25 @@ public void testShouldPausePartition() { @Test public void testQuotaUsage() { PartitionContext context = new PartitionContext( - scope("topic", Optional.of(PerKeyQuotaConfig.shape())), processors); + scope("topic", Optional.of(PerKeyQuotaConfig.shape())), null, processors); assertEquals(UsageType.COMPLY, context.maybeRecordQuotaUsage(new byte[0]).type()); } @Test public void testQuotaUsageWhenDisabled() { PartitionContext context = new PartitionContext( - scope("topic", Optional.empty()), processors); + scope("topic", Optional.empty()), null, processors); assertNull(context.maybeRecordQuotaUsage(new byte[0])); } @Test public void testQuotaUsageNonTargetTopic() { PartitionContext context = new PartitionContext( - scope("topic-shaping", Optional.of(PerKeyQuotaConfig.shape())), processors); + scope("topic-shaping", Optional.of(PerKeyQuotaConfig.shape())), null, processors); assertNull(context.maybeRecordQuotaUsage(new byte[0])); context = new PartitionContext( - scope("topic-retry", Optional.of(PerKeyQuotaConfig.shape())), processors); + scope("topic-retry", Optional.of(PerKeyQuotaConfig.shape())), null, processors); assertNull(context.maybeRecordQuotaUsage(new byte[0])); } @@ -147,6 +148,7 @@ public void testQuotaUsageNonTargetTopic() { public void testQuotaApplied() { PartitionContext context = new PartitionContext( scope("topic-shaping", Optional.of(PerKeyQuotaConfig.shape())), + null, processors, subPartitions); @@ -158,6 +160,7 @@ public void testQuotaApplied() { public void testQuotaNotApplied() { PartitionContext context = new PartitionContext( scope("topic-shaping", Optional.of(PerKeyQuotaConfig.shape())), + null, processors, subPartitions); diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/PartitionContextsTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/PartitionContextsTest.java index c40a2cd1..74382d78 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/PartitionContextsTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/PartitionContextsTest.java @@ -26,6 +26,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.isNull; import static org.mockito.Mockito.clearInvocations; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -98,8 +99,8 @@ private List putContexts(int count) { TopicPartition tp = tp(i); PartitionContext context = mock(PartitionContext.class); doReturn(tp).when(context).topicPartition(); - doReturn(context).when(contexts).instantiateContext(tp); - cts.add(contexts.initContext(tp, false)); + doReturn(context).when(contexts).instantiateContext(tp, null); + cts.add(contexts.initContext(tp, null, false)); } return cts; } @@ -120,24 +121,24 @@ public void testCommittedOffsetsValue() { PartitionContext context = putContexts(1).get(0); long offset = 1L; - doReturn(OptionalLong.of(offset)).when(context).offsetWaitingCommit(); + doReturn(Optional.of(new OffsetAndMetadata(offset))).when(context).offsetWaitingCommit(); Map committedOffsets = contexts.commitReadyOffsets(); - assertEquals(offset + 1, committedOffsets.get(context.topicPartition()).offset()); + assertEquals(offset, committedOffsets.get(context.topicPartition()).offset()); } @Test public void testCommittedOffsetsNeverReturnsZero() { List cts = putContexts(2); - doReturn(OptionalLong.empty()).when(cts.get(0)).offsetWaitingCommit(); - doReturn(OptionalLong.empty()).when(cts.get(1)).offsetWaitingCommit(); + doReturn(Optional.empty()).when(cts.get(0)).offsetWaitingCommit(); + doReturn(Optional.empty()).when(cts.get(1)).offsetWaitingCommit(); Map committedOffsets = contexts.commitReadyOffsets(); // No record has been committed so returned map should be empty assertTrue(committedOffsets.isEmpty()); - doReturn(OptionalLong.of(1)).when(cts.get(0)).offsetWaitingCommit(); + doReturn(Optional.of(new OffsetAndMetadata(1))).when(cts.get(0)).offsetWaitingCommit(); committedOffsets = contexts.commitReadyOffsets(); // No record has been committed for tp1 so the returned map shouldn't contain entry for it. assertEquals(1, committedOffsets.size()); @@ -157,8 +158,8 @@ public void testUpdateCommittedOffset() { // PartitionContext manages their "completed" offset so its minus 1 from committed offset // which indicates the offset to "fetch next". - verify(ctxs.get(0), times(1)).updateCommittedOffset(100); - verify(ctxs.get(1), times(1)).updateCommittedOffset(200); + verify(ctxs.get(0), times(1)).updateCommittedOffset(101); + verify(ctxs.get(1), times(1)).updateCommittedOffset(201); } @Test @@ -305,7 +306,7 @@ public void testMaybeHandlePropertyReload() { contexts.maybeHandlePropertyReload(); // property reload is not requested yet - verify(contexts, never()).instantiateContext(any()); + verify(contexts, never()).instantiateContext(any(), isNull()); partitionConcurrencyProperty.set(42); for (PartitionContext context: allContexts) { @@ -314,7 +315,7 @@ public void testMaybeHandlePropertyReload() { contexts.maybeHandlePropertyReload(); // property reload is requested, but there are pending tasks - verify(contexts, times(reloadableContexts.size())).instantiateContext(any()); + verify(contexts, times(reloadableContexts.size())).instantiateContext(any(), isNull()); for (PartitionContext context: reloadableContexts) { doReturn(false).when(context).reloadRequested(); } @@ -324,7 +325,7 @@ public void testMaybeHandlePropertyReload() { } contexts.maybeHandlePropertyReload(); // completed reloading request - verify(contexts, times(count)).instantiateContext(any()); + verify(contexts, times(count)).instantiateContext(any(), isNull()); } @Test @@ -372,19 +373,18 @@ public void testCommitReadyOnlyNonRevoking() { // mark revoking doReturn(true).when(cts.get(0)).revoking(); - doReturn(OptionalLong.of(0)).when(cts.get(0)).offsetWaitingCommit(); - doReturn(OptionalLong.of(1)).when(cts.get(1)).offsetWaitingCommit(); - doReturn(OptionalLong.empty()).when(cts.get(2)).offsetWaitingCommit(); + doReturn(Optional.empty()).when(cts.get(0)).offsetWaitingCommit(); + doReturn(Optional.of(new OffsetAndMetadata(1))).when(cts.get(1)).offsetWaitingCommit(); + doReturn(Optional.empty()).when(cts.get(2)).offsetWaitingCommit(); Map readyOffsets = contexts.commitReadyOffsets(); assertEquals(1, readyOffsets.size()); - assertEquals(2L, readyOffsets.get(tp(1)).offset()); + assertEquals(1L, readyOffsets.get(tp(1)).offset()); // unmark revoking doReturn(false).when(cts.get(0)).revoking(); - doReturn(OptionalLong.empty()).when(cts.get(1)).offsetWaitingCommit(); + doReturn(Optional.empty()).when(cts.get(1)).offsetWaitingCommit(); readyOffsets = contexts.commitReadyOffsets(); - assertEquals(1, readyOffsets.size()); - assertEquals(1L, readyOffsets.get(tp(0)).offset()); + assertTrue(readyOffsets.isEmpty()); } } diff --git a/protocol/src/main/proto/decaton.proto b/protocol/src/main/proto/decaton.proto index f1cc49b9..f868c682 100644 --- a/protocol/src/main/proto/decaton.proto +++ b/protocol/src/main/proto/decaton.proto @@ -20,3 +20,25 @@ message TaskMetadataProto { // but it isn't guaranteed to be executed exactly at the time. int64 scheduled_time_millis = 5; } + +message OffsetIndexEntryProto { + uint64 start_offset = 1; + uint64 start_index = 2; + uint32 length = 3; +} + +message OffsetIndexProto { + uint64 first_index = 1; + uint64 next_index = 2; + repeated OffsetIndexEntryProto entries = 3; +} + +message BitMapProto { + uint32 size = 1; + repeated uint64 buckets = 2; +} + +message OffsetStorageComplexProto { + OffsetIndexProto index = 1; + BitMapProto comp_flags = 2; +} From 2ae89c3b22246883cc1692c3519bc328013bd96a Mon Sep 17 00:00:00 2001 From: Yuto Kawamura Date: Wed, 6 Nov 2024 19:28:16 +0900 Subject: [PATCH 03/12] (fixup) improve logging, bugfixes --- .../processor/CoreFunctionalityTest.java | 14 ++++++++++++-- processor/src/it/resources/logback.xml | 2 +- .../runtime/ProcessorSubscription.java | 2 +- .../runtime/internal/AssignmentManager.java | 3 ++- .../runtime/internal/CommitManager.java | 2 ++ .../runtime/internal/OffsetStorageComplex.java | 5 ++--- .../internal/OutOfOrderCommitControl.java | 11 ++++++----- .../internal/OffsetStorageComplexTest.java | 17 ++++++++++++----- .../internal/OutOfOrderCommitControlTest.java | 4 ++-- protocol/src/main/proto/decaton.proto | 3 +-- 10 files changed, 41 insertions(+), 22 deletions(-) diff --git a/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java b/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java index a3b4bfc6..b9473a98 100644 --- a/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java +++ b/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java @@ -16,10 +16,12 @@ package com.linecorp.decaton.processor; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.ArrayList; import java.util.List; +import java.util.Map.Entry; import java.util.Properties; import java.util.Random; import java.util.concurrent.ConcurrentHashMap; @@ -165,10 +167,12 @@ public void testSingleThreadProcessing() throws Exception { ProcessingGuarantee noDuplicates = new ProcessingGuarantee() { private final ConcurrentMap> produced = new ConcurrentHashMap<>(); private final ConcurrentMap> processed = new ConcurrentHashMap<>(); + private final ConcurrentMap taskToOffset = new ConcurrentHashMap<>(); @Override public void onProduce(ProducedRecord record) { produced.computeIfAbsent(new HashableByteArray(record.key()), key -> new ArrayList<>()).add(record.task()); + taskToOffset.put(record.task(), record.offset()); } @Override @@ -180,7 +184,12 @@ public void onProcess(TaskMetadata metadata, ProcessedRecord record) { public void doAssert() { // use assertTrue instead of assertEquals not to cause error message explosion //noinspection SimplifiableJUnitAssertion - assertTrue(produced.equals(processed)); + for (Entry> e : produced.entrySet()) { + List producedTasks = e.getValue().stream().map(taskToOffset::get).toList(); + List processedTasks = processed.get(e.getKey()).stream().map(taskToOffset::get).toList(); + assertEquals(producedTasks, processedTasks); + } +// assertTrue(produced.equals(processed)); } }; @@ -192,7 +201,8 @@ public void doAssert() { (ctx, task) -> Thread.sleep(rand.nextInt(10)))) .propertySupplier(StaticPropertySupplier.of( Property.ofStatic(ProcessorProperties.CONFIG_PARTITION_CONCURRENCY, 1), - Property.ofStatic(ProcessorProperties.CONFIG_MAX_PENDING_RECORDS, 100) + Property.ofStatic(ProcessorProperties.CONFIG_MAX_PENDING_RECORDS, 100), + Property.ofStatic(ProcessorProperties.CONFIG_SHUTDOWN_TIMEOUT_MS, 1000L) )) .customSemantics(noDuplicates) .build() diff --git a/processor/src/it/resources/logback.xml b/processor/src/it/resources/logback.xml index e0491539..5c919ba9 100644 --- a/processor/src/it/resources/logback.xml +++ b/processor/src/it/resources/logback.xml @@ -17,7 +17,7 @@ - %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n + %d{HH:mm:ss.SSS} [%thread] tp=%X{dt_topic}-%X{dt_partition} off=%X{dt_offset} %-5level %logger{36} - %msg%n diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java index f9a7a47a..4d0112ff 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java @@ -110,7 +110,7 @@ public void receive(ConsumerRecord record) { try { offsetState = context.registerOffset(record.offset()); } catch (OffsetRegressionException ignored) { - log.warn("Offset regression at partition {}", tp); + log.warn("Offset regression at partition {}, by offset {}", tp, record.offset()); assignManager.repair(tp); context = contexts.get(tp); // If it fails even at 2nd attempt... no idea let it die. diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/AssignmentManager.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/AssignmentManager.java index 575654d0..05e8d78a 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/AssignmentManager.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/AssignmentManager.java @@ -104,7 +104,8 @@ public void assign(Map newAssignment) { Map added = computeAddedPartitions(oldSet, newSet) .stream() .collect(HashMap::new, (m, v) -> m.put(v, newAssignment.get(v)), HashMap::putAll); - log.debug("Assignment update: removed:{}, added:{}, assignment:{}", removed, added, newSet); + log.debug("Assignment update: consumer assignment: {}, removed:{}, added:{}, assignment:{}", + newAssignment, removed, added, newSet); partitionsRevoked(removed); partitionsAssigned(added); diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/CommitManager.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/CommitManager.java index 5bdd4719..027f2325 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/CommitManager.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/CommitManager.java @@ -124,6 +124,7 @@ public void commitAsync() { log.debug("Skipping commit due to another async commit is currently in-flight"); return; } + log.debug("Committing offsets ASYNC: {}", offsets); Thread callingThread = Thread.currentThread(); consumer.commitAsync(offsets, (ignored, exception) -> { asyncCommitInFlight = false; @@ -131,6 +132,7 @@ public void commitAsync() { log.warn("Offset commit failed asynchronously", exception); return; } + log.debug("Successful async offset commit: {}", offsets); if (Thread.currentThread() != callingThread) { // This isn't expected to happen (see below comment) but we check it with cheap cost // just in case to avoid silent corruption. diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java index 67e5f09e..8cd0cd2a 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java @@ -110,8 +110,7 @@ public long addOffset(long offset) { public OffsetIndexProto toProto() { OffsetIndexProto.Builder builder = OffsetIndexProto.newBuilder() - .setFirstIndex(firstIndex) - .setNextIndex(nextIndex); + .setFirstIndex(firstIndex); for (Entry entry : blockIndex.entrySet()) { long offset = entry.getKey(); BlockInfo blockInfo = entry.getValue(); @@ -130,7 +129,7 @@ public static OffsetIndex fromProto(OffsetIndexProto proto) { OffsetIndexEntryProto entry = proto.getEntries(i); blockIndex.put(entry.getStartOffset(), new BlockInfo(entry.getStartIndex(), entry.getLength())); } - return new OffsetIndex(blockIndex, proto.getFirstIndex(), proto.getNextIndex()); + return new OffsetIndex(blockIndex, proto.getFirstIndex(), proto.getFirstIndex()); } } diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java index 4e0654c4..c94b098c 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java @@ -62,7 +62,7 @@ public static OutOfOrderCommitControl fromOffsetMeta(TopicPartition tp, OffsetStateReaper offsetStateReaper, OffsetAndMetadata offsetMeta) { OffsetStorageComplex complex = complexFromMeta(offsetMeta.metadata()); - return new OutOfOrderCommitControl(tp, capacity, offsetStateReaper, complex, offsetMeta.offset()); + return new OutOfOrderCommitControl(tp, capacity, offsetStateReaper, complex, offsetMeta.offset() - 1); } public synchronized OffsetState reportFetchedOffset(long offset) { @@ -94,7 +94,7 @@ public synchronized OffsetState reportFetchedOffset(long offset) { void onComplete(long offset, int ringIndex) { if (log.isDebugEnabled()) { - log.debug("Offset complete: {}", offset); + log.debug("Offset complete on {}: {}", topicPartition, offset); } complex.complete(ringIndex); } @@ -134,9 +134,10 @@ public synchronized void updateHighWatermark() { } } - if (highWatermark != lastHighWatermark) { - log.debug("High watermark updated for {}: {} => {}", - topicPartition, lastHighWatermark, highWatermark); + if (highWatermark != lastHighWatermark && log.isDebugEnabled()) { + int pending = pendingOffsetsCount(); + log.debug("High watermark updated {}: {} => {}, pending={}, first={}", + topicPartition, lastHighWatermark, highWatermark, pending, pending > 0 ? complex.firstOffset() : -1); } } diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplexTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplexTest.java index 2ef8efdf..5a9d2050 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplexTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplexTest.java @@ -71,8 +71,8 @@ void testOffsetIndex() { assertEquals(-1, index.indexOf(19)); } - private static void addOffset(OffsetStorageComplex complex, long offset) { - complex.addOffset(offset, false, new OffsetState(offset)); + private static int addOffset(OffsetStorageComplex complex, long offset) { + return complex.addOffset(offset, false, new OffsetState(offset)); } @Test @@ -81,8 +81,8 @@ void test() { addOffset(complex, 10); addOffset(complex, 11); - addOffset(complex, 12); - addOffset(complex, 15); + int ri12 = addOffset(complex, 12); + int ri15 = addOffset(complex, 15); addOffset(complex, 16); addOffset(complex, 17); @@ -90,6 +90,13 @@ void test() { assertFalse(complex.isComplete(10)); assertFalse(complex.isComplete(17)); - complex.pollFirst(); + complex.complete(ri12); + complex.complete(ri15); + assertTrue(complex.isComplete(12)); + assertTrue(complex.isComplete(15)); + assertFalse(complex.isComplete(10)); + assertFalse(complex.isComplete(17)); +// +// addOffset(complex, 10); } } diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControlTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControlTest.java index d981ef97..4ec715db 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControlTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControlTest.java @@ -108,7 +108,7 @@ public void testDoubleCompletingSameOffset() { public void testReportingTooLargeOffset() { commitControl.reportFetchedOffset(1); // now earliest=1 for (int i = 0; i < STATES_CAPACITY - 1; i++) { - commitControl.reportFetchedOffset(1 + i); + commitControl.reportFetchedOffset(2 + i); } assertThrows(IllegalArgumentException.class, () -> commitControl.reportFetchedOffset(STATES_CAPACITY)); } @@ -159,7 +159,7 @@ public void testPendingRecordsCountWithLargeGap() { @Test @Timeout(5) - public void testTimeoutOffsetReaping() { + public void testTimeoutOffsetReaping() throws InterruptedException { Clock clock = mock(Clock.class); doReturn(10L).when(clock).millis(); OffsetStateReaper reaper = new OffsetStateReaper( diff --git a/protocol/src/main/proto/decaton.proto b/protocol/src/main/proto/decaton.proto index f868c682..50adf480 100644 --- a/protocol/src/main/proto/decaton.proto +++ b/protocol/src/main/proto/decaton.proto @@ -29,8 +29,7 @@ message OffsetIndexEntryProto { message OffsetIndexProto { uint64 first_index = 1; - uint64 next_index = 2; - repeated OffsetIndexEntryProto entries = 3; + repeated OffsetIndexEntryProto entries = 2; } message BitMapProto { From 6786d429ea15c263d246e1676a1106ee73321398 Mon Sep 17 00:00:00 2001 From: Yuto Kawamura Date: Mon, 16 Dec 2024 19:23:52 +0900 Subject: [PATCH 04/12] (fixup) follow-up --- .../processor/CoreFunctionalityTest.java | 2 + .../runtime/ProcessorSubscription.java | 2 + .../runtime/internal/CompletionImpl.java | 12 ++- .../runtime/internal/OffsetState.java | 6 +- .../internal/OffsetStorageComplex.java | 90 ++++++++++++++----- .../internal/OutOfOrderCommitControl.java | 30 ++----- .../internal/OffsetStorageComplexTest.java | 46 +++++++++- .../internal/OutOfOrderCommitControlTest.java | 2 +- 8 files changed, 145 insertions(+), 45 deletions(-) diff --git a/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java b/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java index b9473a98..b8dec802 100644 --- a/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java +++ b/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java @@ -44,6 +44,7 @@ import com.linecorp.decaton.testing.RandomExtension; import com.linecorp.decaton.testing.processor.ProcessedRecord; import com.linecorp.decaton.testing.processor.ProcessingGuarantee; +import com.linecorp.decaton.testing.processor.ProcessingGuarantee.GuaranteeType; import com.linecorp.decaton.testing.processor.ProcessorTestSuite; import com.linecorp.decaton.testing.processor.ProducedRecord; import com.linecorp.decaton.testing.processor.TestTask; @@ -224,6 +225,7 @@ public void testAsyncCompletionWithLeakAndTimeout() throws Exception { ctx.deferCompletion(); } })) + .excludeSemantics(GuaranteeType.PROCESS_ORDERING) .build() .run(); } diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java index 4d0112ff..96548d4c 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java @@ -238,6 +238,8 @@ public void run() { consumeManager.init(subscribeTopics()); consumeLoop(); } + } catch (RuntimeException e) { + log.error("Unexpected exception in subscription thread, terminating", e); } finally { loopTerminateFuture.complete(null); } diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/CompletionImpl.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/CompletionImpl.java index de84d39c..ecbdd185 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/CompletionImpl.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/CompletionImpl.java @@ -31,6 +31,7 @@ @Accessors(fluent = true) public class CompletionImpl implements Completion { private final CompletableFuture future; + private final CompletionStage stage; @Setter private volatile Function expireCallback; private volatile Completion dependency; @@ -48,7 +49,16 @@ static CompletionImpl failedCompletion(Throwable ex) { } public CompletionImpl() { + this(null); + } + + public CompletionImpl(Runnable firstAction) { future = new CompletableFuture<>(); + if (firstAction == null) { + stage = future; + } else { + stage = future.whenComplete((ignored, ignored2) -> firstAction.run()); + } } @Override @@ -58,7 +68,7 @@ public boolean isComplete() { @Override public CompletionStage asFuture() { - return future; + return stage; } @Override diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetState.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetState.java index dcd9e7d0..d52fa326 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetState.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetState.java @@ -29,9 +29,13 @@ public class OffsetState { private final CompletionImpl completion; public OffsetState(long offset) { + this(offset, null); + } + + public OffsetState(long offset, Runnable firstAction) { this.offset = offset; timeoutAt = -1; - completion = new CompletionImpl(); + completion = new CompletionImpl(firstAction); } public void setTimeout(long timeoutAt) { diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java index 8cd0cd2a..95e4e98f 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java @@ -16,6 +16,7 @@ package com.linecorp.decaton.processor.runtime.internal; +import java.util.Iterator; import java.util.Map.Entry; import java.util.TreeMap; @@ -27,6 +28,7 @@ import lombok.AccessLevel; import lombok.AllArgsConstructor; import lombok.Getter; +import lombok.RequiredArgsConstructor; import lombok.ToString; import lombok.experimental.Accessors; @@ -88,24 +90,28 @@ public long indexOf(long offset) { } public long addOffset(long offset) { - Entry e = blockIndex.lastEntry(); - long offsetIndex = nextIndex++; - if (e != null) { - BlockInfo blockInfo = e.getValue(); - long nextOffset = e.getKey() + blockInfo.length; - if (offset < nextOffset) { - // throw new IllegalArgumentException("can't regress"); - throw new OffsetRegressionException("offset regression at " + offset); - } - if (offset == nextOffset) { - // No offset gap, can extend the last block + if (blockIndex.isEmpty()) { + blockIndex.put(offset, new BlockInfo(nextIndex, 1)); + return nextIndex++; + } + Entry floorEntry = blockIndex.floorEntry(offset); + if (floorEntry == null) { + throw new OffsetRegressionException("offset regression at " + offset); + } + BlockInfo blockInfo = floorEntry.getValue(); + long indexOffset = nextIndex - blockInfo.index; + long expectedOffset = floorEntry.getKey() + indexOffset; + if (offset < expectedOffset) { + throw new OffsetRegressionException("offset regression at " + offset); + } + if (offset == expectedOffset) { + if (indexOffset == blockInfo.length) { blockInfo.length++; - return offsetIndex; } + return nextIndex++; } - // Offset gap or first entry after cleanup, needs to create a new block - blockIndex.put(offset, new BlockInfo(offsetIndex, 1)); - return offsetIndex; + blockIndex.put(offset, new BlockInfo(nextIndex, 1)); + return nextIndex++; } public OffsetIndexProto toProto() { @@ -131,6 +137,31 @@ public static OffsetIndex fromProto(OffsetIndexProto proto) { } return new OffsetIndex(blockIndex, proto.getFirstIndex(), proto.getFirstIndex()); } + + public Iterator offsetsIterator() { + return new OffsetsIterator(blockIndex.entrySet().iterator()); + } + + @RequiredArgsConstructor + static class OffsetsIterator implements Iterator { + private final Iterator> entries; + private Entry curEntry; + private long localIndex; + + @Override + public boolean hasNext() { + return curEntry != null && localIndex < curEntry.getValue().length || entries.hasNext(); + } + + @Override + public Long next() { + while (curEntry == null || localIndex == curEntry.getValue().length) { + curEntry = entries.next(); + localIndex = 0; + } + return curEntry.getKey() + localIndex++; + } + } } private final OffsetIndex index; @@ -155,11 +186,13 @@ public void pollFirst() { states[firstIndex] = null; } - public int addOffset(long offset, boolean complete, OffsetState state) { - int nextIndex = (int) (index.addOffset(offset) % states.length); - compFlags.set(nextIndex, complete); - states[nextIndex] = state; - return nextIndex; + public int allocNextIndex(long offset) { + return (int) (index.addOffset(offset) % states.length); + } + + public void setIndex(int index, boolean complete, OffsetState state) { + compFlags.set(index, complete); + states[index] = state; } public void complete(int ringIndex) { @@ -203,4 +236,21 @@ public static OffsetStorageComplex fromProto(OffsetStorageComplexProto proto) { ConcurrentBitMap compFlags = ConcurrentBitMap.fromProto(proto.getCompFlags()); return new OffsetStorageComplex(index, compFlags, new OffsetState[compFlags.size()]); } + + public String compDebugDump() { + Iterator offsets = index.offsetsIterator(); + StringBuilder sb = new StringBuilder("["); + boolean first = true; + while (offsets.hasNext()) { + long offset = offsets.next(); + if (first) { + first = false; + } else { + sb.append(", "); + } + sb.append(String.valueOf(offset) + ':' + (isComplete(offset) ? 'c' : 'n')); + } + sb.append(']'); + return sb.toString(); + } } diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java index c94b098c..f5df8012 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java @@ -51,9 +51,9 @@ public class OutOfOrderCommitControl implements AutoCloseable { public OutOfOrderCommitControl(TopicPartition topicPartition, int capacity, OffsetStateReaper offsetStateReaper) { this.topicPartition = topicPartition; - complex = new OffsetStorageComplex(capacity); this.capacity = capacity; this.offsetStateReaper = offsetStateReaper; + complex = new OffsetStorageComplex(capacity); highWatermark = -1; } @@ -76,6 +76,7 @@ public synchronized OffsetState reportFetchedOffset(long offset) { String.format("offsets count overflow: size=%d, cap=%d", complex.size(), capacity)); } + int ringIndex = complex.allocNextIndex(offset); if (complex.isComplete(offset)) { // There are two cases for this. // 1. Offset bigger than that complex's managing bounds. Reasonable to consider as not completed @@ -85,10 +86,8 @@ public synchronized OffsetState reportFetchedOffset(long offset) { return null; } - OffsetState state = new OffsetState(offset); - int ringIndex = complex.addOffset(offset, false, state); - - state.completion().asFuture().whenComplete((unused, throwable) -> onComplete(offset, ringIndex)); // TODO okay in this order? + OffsetState state = new OffsetState(offset, () -> onComplete(offset, ringIndex)); + complex.setIndex(ringIndex, false, state); return state; } @@ -101,20 +100,8 @@ void onComplete(long offset, int ringIndex) { public synchronized void updateHighWatermark() { if (log.isTraceEnabled()) { -// StringBuilder sb = new StringBuilder("["); -// -// boolean first = true; -// for (OffsetState st : states) { -// if (first) { -// first = false; -// } else { -// sb.append(", "); -// } -// sb.append(String.valueOf(st.offset()) + ':' + (st.completion().isComplete() ? 'c' : 'n')); -// } -// sb.append(']'); -// log.trace("Begin updateHighWatermark earliest={} latest={} hw={} states={}", -// earliest, latest, highWatermark, sb); + log.trace("Begin updateHighWatermark tp={} pending={} hw={} states={}", + topicPartition, pendingOffsetsCount(), highWatermark, complex.compDebugDump()); } long lastHighWatermark = highWatermark; @@ -151,8 +138,9 @@ public OffsetAndMetadata commitReadyOffset() { } OffsetStorageComplexProto complexProto = complex.toProto(); try { - String meta = JsonFormat.printer().print(complexProto); // TODO: 64-bit ints are dumped as strings - return new OffsetAndMetadata(highWatermark + 1, meta); + String meta = JsonFormat.printer().omittingInsignificantWhitespace().print(complexProto); + long commitOffset = highWatermark + 1; + return new OffsetAndMetadata(commitOffset, meta); } catch (InvalidProtocolBufferException e) { throw new RuntimeException("failed to serialize offset metadata into proto", e); } diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplexTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplexTest.java index 5a9d2050..0a7d3cd9 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplexTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplexTest.java @@ -21,6 +21,7 @@ import org.junit.jupiter.api.Test; import com.linecorp.decaton.processor.runtime.internal.OffsetStorageComplex.OffsetIndex; +import com.linecorp.decaton.protocol.Decaton.OffsetStorageComplexProto; class OffsetStorageComplexTest { @@ -72,7 +73,9 @@ void testOffsetIndex() { } private static int addOffset(OffsetStorageComplex complex, long offset) { - return complex.addOffset(offset, false, new OffsetState(offset)); + int index = complex.allocNextIndex(offset); + complex.setIndex(index, false, new OffsetState(offset)); + return index; } @Test @@ -99,4 +102,45 @@ void test() { // // addOffset(complex, 10); } + + @Test + void testRecoveryState() { + OffsetStorageComplex origComplex = new OffsetStorageComplex(10); + + addOffset(origComplex, 10); + addOffset(origComplex, 11); + int ri12 = addOffset(origComplex, 12); + int ri15 = addOffset(origComplex, 15); + addOffset(origComplex, 16); + addOffset(origComplex, 17); + origComplex.complete(ri12); + origComplex.complete(ri15); + + OffsetStorageComplexProto proto = origComplex.toProto(); + OffsetStorageComplex complex = OffsetStorageComplex.fromProto(proto); + + assertEquals(0, complex.size()); + assertFalse(complex.isComplete(10)); + assertFalse(complex.isComplete(11)); + assertTrue(complex.isComplete(12)); + assertTrue(complex.isComplete(15)); + assertFalse(complex.isComplete(16)); + assertFalse(complex.isComplete(17)); + + assertThrows(OffsetRegressionException.class, () -> complex.allocNextIndex(9)); + complex.complete(addOffset(complex, 10)); + complex.complete(addOffset(complex, 11)); + complex.allocNextIndex(12); + complex.allocNextIndex(15); + complex.complete(addOffset(complex, 16)); + complex.complete(addOffset(complex, 17)); + + assertEquals(10, complex.firstOffset()); + assertTrue(complex.isComplete(10)); + assertTrue(complex.isComplete(11)); + assertTrue(complex.isComplete(12)); + assertTrue(complex.isComplete(15)); + assertTrue(complex.isComplete(16)); + assertTrue(complex.isComplete(17)); + } } diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControlTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControlTest.java index 4ec715db..6c2b24f0 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControlTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControlTest.java @@ -206,7 +206,7 @@ void perOffsetCompleteTest() { assertEquals(om.offset(), 102L); OffsetStorageComplex complex = OutOfOrderCommitControl.complexFromMeta(om.metadata()); System.err.println("meta = " + om.metadata()); - assertEquals(2, complex.size()); + assertEquals(0, complex.size()); assertFalse(complex.isComplete(102)); assertTrue(complex.isComplete(103)); From 8f9f6e32fd967400f51e2b0eb744e59b47211dca Mon Sep 17 00:00:00 2001 From: Yuto Kawamura Date: Wed, 18 Dec 2024 14:54:01 +0900 Subject: [PATCH 05/12] (fixup) complement unit tests --- .../internal/OffsetStorageComplex.java | 13 +- .../internal/OffsetStorageComplexTest.java | 224 ++++++++++++++++-- 2 files changed, 208 insertions(+), 29 deletions(-) diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java index 95e4e98f..21438e91 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java @@ -18,6 +18,7 @@ import java.util.Iterator; import java.util.Map.Entry; +import java.util.NoSuchElementException; import java.util.TreeMap; import com.linecorp.decaton.protocol.Decaton.BitMapProto; @@ -61,11 +62,14 @@ public int indexSize() { } public long firstOffset() { - return blockIndex.isEmpty() ? -1 : blockIndex.firstKey(); + return indexSize() == 0 ? -1 : blockIndex.firstKey(); } public long pollFirst() { Entry first = blockIndex.pollFirstEntry(); + if (first == null) { + throw new NoSuchElementException(); + } final BlockInfo firstBlock = first.getValue(); long removedIndex = firstBlock.index++; firstIndex = firstBlock.index; @@ -101,7 +105,7 @@ public long addOffset(long offset) { BlockInfo blockInfo = floorEntry.getValue(); long indexOffset = nextIndex - blockInfo.index; long expectedOffset = floorEntry.getKey() + indexOffset; - if (offset < expectedOffset) { + if (indexOffset < 0 || offset < expectedOffset) { throw new OffsetRegressionException("offset regression at " + offset); } if (offset == expectedOffset) { @@ -187,6 +191,9 @@ public void pollFirst() { } public int allocNextIndex(long offset) { + if (size() == states.length) { + throw new IllegalStateException("complex reached its capacity: " + states.length); + } return (int) (index.addOffset(offset) % states.length); } @@ -214,7 +221,7 @@ public OffsetState firstState() { public boolean isComplete(long offset) { int ringIndex = (int) (index.indexOf(offset) % states.length); if (ringIndex == -1) { - // By contract we expect the offset-out-of-range case to be just the offset being too large against + // By contract, we expect the offset-out-of-range case to be just the offset being too large against // managed range, not lower. return false; } diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplexTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplexTest.java index 0a7d3cd9..98e0c166 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplexTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplexTest.java @@ -16,7 +16,14 @@ package com.linecorp.decaton.processor.runtime.internal; -import static org.junit.jupiter.api.Assertions.*; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.util.NoSuchElementException; import org.junit.jupiter.api.Test; @@ -78,31 +85,6 @@ private static int addOffset(OffsetStorageComplex complex, long offset) { return index; } - @Test - void test() { - OffsetStorageComplex complex = new OffsetStorageComplex(10); - - addOffset(complex, 10); - addOffset(complex, 11); - int ri12 = addOffset(complex, 12); - int ri15 = addOffset(complex, 15); - addOffset(complex, 16); - addOffset(complex, 17); - - assertEquals(6, complex.size()); - assertFalse(complex.isComplete(10)); - assertFalse(complex.isComplete(17)); - - complex.complete(ri12); - complex.complete(ri15); - assertTrue(complex.isComplete(12)); - assertTrue(complex.isComplete(15)); - assertFalse(complex.isComplete(10)); - assertFalse(complex.isComplete(17)); -// -// addOffset(complex, 10); - } - @Test void testRecoveryState() { OffsetStorageComplex origComplex = new OffsetStorageComplex(10); @@ -143,4 +125,194 @@ void testRecoveryState() { assertTrue(complex.isComplete(16)); assertTrue(complex.isComplete(17)); } + + @Test + void size() { + OffsetStorageComplex complex = new OffsetStorageComplex(3); + assertEquals(0, complex.size()); + // By allocating one offset, it increments size. + int index = complex.allocNextIndex(10); + assertEquals(1, complex.size()); + // Setting the allocated index doesn't impact size + complex.setIndex(index, false, null); + // Another addition + complex.allocNextIndex(11); + assertEquals(2, complex.size()); + // Polling the first offset decrements size. + complex.pollFirst(); + assertEquals(1, complex.size()); + // Alloc a few more offsets, so the ring-buffer go over one cycle at least. + complex.allocNextIndex(20); + complex.allocNextIndex(21); + assertEquals(3, complex.size()); + } + + @Test + void firstOffset() { + OffsetStorageComplex complex = new OffsetStorageComplex(3); + // Without any entry it is -1 + assertEquals(-1, complex.firstOffset()); + // Add first offset, it should return as-is + complex.allocNextIndex(10); + assertEquals(10, complex.firstOffset()); + // Adding the second offset, should still return the first. + complex.allocNextIndex(11); + assertEquals(10, complex.firstOffset()); + // Popping the first offset out now the first offset should become the second added. + complex.pollFirst(); + assertEquals(11, complex.firstOffset()); + // Add a few more offsets with gaps, also making the ring-buffer to go over one cycle. + complex.allocNextIndex(20); + complex.allocNextIndex(30); + complex.pollFirst(); + assertEquals(20, complex.firstOffset()); + complex.pollFirst(); + assertEquals(30, complex.firstOffset()); + // Finally there are no more offsets in storage, should come back to return -1 again. + complex.pollFirst(); + assertEquals(-1, complex.firstOffset()); + } + + @Test + void pollFirst() { + OffsetStorageComplex complex = new OffsetStorageComplex(1); + assertThrows(NoSuchElementException.class, complex::pollFirst); + // Add an entry, and remove it then it should become back to empty. + complex.allocNextIndex(10); + complex.pollFirst(); + assertThrows(NoSuchElementException.class, complex::pollFirst); + } + + @Test + void allocNextIndex() { + OffsetStorageComplex complex = new OffsetStorageComplex(2); + complex.allocNextIndex(10); // OK + complex.allocNextIndex(11); // OK + assertThrows(IllegalStateException.class, () -> complex.allocNextIndex(20)); + } + + @Test + void firstState() { + OffsetStorageComplex complex = new OffsetStorageComplex(3); + // Without any entry it is null + assertNull(complex.firstState()); + // Try adding the first offset, now the first is it. + OffsetState s10 = new OffsetState(10); + complex.setIndex(complex.allocNextIndex(10), false, s10); + assertSame(s10, complex.firstState()); + // Adding the second offset, should still return the first. + OffsetState s11 = new OffsetState(11); + complex.setIndex(complex.allocNextIndex(11), false, s11); + assertSame(s10, complex.firstState()); + // Popping the first offset out now the first offset should become the second added. + complex.pollFirst(); + assertSame(s11, complex.firstState()); + // Add a few more offsets with gaps, also making the ring-buffer to go over one cycle. + OffsetState s20 = new OffsetState(20); + complex.setIndex(complex.allocNextIndex(20), false, s20); + OffsetState s30 = new OffsetState(30); + complex.setIndex(complex.allocNextIndex(30), false, s30); + complex.pollFirst(); + assertSame(s20, complex.firstState()); + complex.pollFirst(); + assertSame(s30, complex.firstState()); + // Finally there are no more offsets in storage, should come back to return -1 again. + complex.pollFirst(); + assertNull(complex.firstState()); + } + + @Test + void isComplete() { + OffsetStorageComplex complex = new OffsetStorageComplex(3); + // Despite no offsets has been added, it returns false for whatever offset that it doesn't + // recognize. + assertFalse(complex.isComplete(1)); + assertFalse(complex.isComplete(100)); + // Add first offset + int i10 = complex.allocNextIndex(10); + complex.setIndex(i10, false, new OffsetState(10)); + assertFalse(complex.isComplete(10)); + // Now complete the first + complex.complete(i10); + assertTrue(complex.isComplete(10)); + // By popping out the offset, it turns back to return false + complex.pollFirst(); + assertFalse(complex.isComplete(10)); + // Add another offset with completion true + int i11 = complex.allocNextIndex(11); + complex.setIndex(i11, true, new OffsetState(11)); + assertTrue(complex.isComplete(11)); + // Add a few more offsets to let ring-buffer go over one cycle. + int i20 = complex.allocNextIndex(20); + complex.setIndex(i20, false, new OffsetState(20)); + int i30 = complex.allocNextIndex(30); + complex.setIndex(i30, false, new OffsetState(30)); + complex.complete(i30); + // Now the state should be 11=c, 20=n, 30=c + assertTrue(complex.isComplete(11)); + assertFalse(complex.isComplete(20)); + assertTrue(complex.isComplete(30)); + // Popping them all should make them all false + complex.pollFirst(); + assertFalse(complex.isComplete(11)); + complex.pollFirst(); + assertFalse(complex.isComplete(20)); + complex.pollFirst(); + assertFalse(complex.isComplete(30)); + } + + @Test + void fromProto() { + final OffsetStorageComplex recov; + { + OffsetStorageComplex complex = new OffsetStorageComplex(3); + int i10 = complex.allocNextIndex(10); + complex.setIndex(i10, false, new OffsetState(10)); + int i11 = complex.allocNextIndex(11); + complex.setIndex(i11, false, new OffsetState(11)); + int i20 = complex.allocNextIndex(20); + complex.setIndex(i20, true, new OffsetState(20)); + + recov = OffsetStorageComplex.fromProto(complex.toProto()); + } + // Recovered complex has no offsets, hence no first state as well + assertEquals(0, recov.size()); + assertEquals(-1, recov.firstOffset()); + assertNull(recov.firstState()); + + // The recovered complex expects exactly same sequence of offsets to receive after recovery. + assertThrows(OffsetRegressionException.class, () -> recov.allocNextIndex(20)); + int ri10 = recov.allocNextIndex(10); + assertEquals(1, recov.size()); // Size should be incremented only after allocation + assertFalse(recov.isComplete(10)); + recov.setIndex(ri10, false, new OffsetState(10)); + recov.complete(ri10); + assertTrue(recov.isComplete(10)); + int ri11 = recov.allocNextIndex(11); + assertFalse(recov.isComplete(11)); + recov.setIndex(ri11, false, new OffsetState(11)); + recov.complete(ri11); + assertTrue(recov.isComplete(11)); + recov.allocNextIndex(20); + // No setIndex for offset=20 because it's complete already. + assertTrue(recov.isComplete(20)); + // firstOffset always returns the first offset, while firstState might return null + assertEquals(10, recov.firstOffset()); + assertEquals(10, recov.firstState().offset()); + recov.pollFirst(); + assertEquals(11, recov.firstOffset()); + assertEquals(11, recov.firstState().offset()); + recov.pollFirst(); + assertEquals(20, recov.firstOffset()); + assertNull(recov.firstState()); + recov.pollFirst(); + + // After recovery and added all sequence previously seen, it could be used continually + int ri30 = recov.allocNextIndex(30); + recov.setIndex(ri30, false, new OffsetState(30)); + recov.complete(ri30); + assertTrue(recov.isComplete(30)); + assertEquals(30, recov.firstOffset()); + assertEquals(30, recov.firstState().offset()); + } } From cd5e6ef77f9e95e758b30e78bfb9b42cd0396c12 Mon Sep 17 00:00:00 2001 From: Yuto Kawamura Date: Wed, 25 Dec 2024 15:01:33 +0900 Subject: [PATCH 06/12] (fixup) fix integration tests to follow-up new-spec --- .../processor/CoreFunctionalityTest.java | 86 +++++++++--------- .../VThreadCoreFunctionalityTest.java | 88 ++++++++++--------- .../processors/BatchingProcessor.java | 4 +- .../runtime/ProcessorSubscription.java | 4 +- .../runtime/internal/AssignmentManager.java | 2 +- .../runtime/internal/ConsumeManager.java | 5 ++ .../internal/OffsetStorageComplex.java | 17 ++++ .../internal/OutOfOrderCommitControl.java | 32 ++++--- .../processor/KeyedExecutorService.java | 42 +++++++++ .../testing/processor/ProcessorTestSuite.java | 2 +- .../decaton/testing/processor/TestTask.java | 4 + 11 files changed, 187 insertions(+), 99 deletions(-) create mode 100644 testing/src/main/java/com/linecorp/decaton/testing/processor/KeyedExecutorService.java diff --git a/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java b/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java index b8dec802..81df29b8 100644 --- a/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java +++ b/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java @@ -17,17 +17,15 @@ package com.linecorp.decaton.processor; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Map.Entry; import java.util.Properties; import java.util.Random; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.CooperativeStickyAssignor; @@ -42,6 +40,7 @@ import com.linecorp.decaton.processor.runtime.StaticPropertySupplier; import com.linecorp.decaton.testing.KafkaClusterExtension; import com.linecorp.decaton.testing.RandomExtension; +import com.linecorp.decaton.testing.processor.KeyedExecutorService; import com.linecorp.decaton.testing.processor.ProcessedRecord; import com.linecorp.decaton.testing.processor.ProcessingGuarantee; import com.linecorp.decaton.testing.processor.ProcessingGuarantee.GuaranteeType; @@ -49,6 +48,9 @@ import com.linecorp.decaton.testing.processor.ProducedRecord; import com.linecorp.decaton.testing.processor.TestTask; +import lombok.extern.slf4j.Slf4j; + +@Slf4j public class CoreFunctionalityTest { @RegisterExtension public static KafkaClusterExtension rule = new KafkaClusterExtension(); @@ -107,25 +109,26 @@ public void testProcessConcurrent_ThreadScopeProcessor() throws Exception { @Test @Timeout(30) public void testAsyncTaskCompletion() throws Exception { - ExecutorService executorService = Executors.newFixedThreadPool(16); - Random rand = randomExtension.random(); - ProcessorTestSuite - .builder(rule) - .configureProcessorsBuilder(builder -> builder.thenProcess((ctx, task) -> { - DeferredCompletion completion = ctx.deferCompletion(); - executorService.execute(() -> { - try { - Thread.sleep(rand.nextInt(10)); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } finally { - completion.complete(); - } - }); - })) - .build() - .run(); + try (KeyedExecutorService executor = new KeyedExecutorService(16)) { + Random rand = randomExtension.random(); + ProcessorTestSuite + .builder(rule) + .configureProcessorsBuilder(builder -> builder.thenProcess((ctx, task) -> { + DeferredCompletion completion = ctx.deferCompletion(); + executor.execute(Arrays.hashCode(task.getKey()), () -> { + try { + Thread.sleep(rand.nextInt(10)); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } finally { + completion.complete(); + } + }); + })) + .build() + .run(); + } } /* @@ -139,25 +142,26 @@ public void testAsyncTaskCompletion() throws Exception { @Test @Timeout(30) public void testGetCompletionInstanceLater() throws Exception { - ExecutorService executorService = Executors.newFixedThreadPool(16); - Random rand = randomExtension.random(); - ProcessorTestSuite - .builder(rule) - .configureProcessorsBuilder(builder -> builder.thenProcess((ctx, task) -> { - ctx.deferCompletion(); - executorService.execute(() -> { - try { - Thread.sleep(rand.nextInt(10)); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } finally { - ctx.deferCompletion().complete(); - } - }); - })) - .build() - .run(); + try (KeyedExecutorService executor = new KeyedExecutorService(16)) { + Random rand = randomExtension.random(); + ProcessorTestSuite + .builder(rule) + .configureProcessorsBuilder(builder -> builder.thenProcess((ctx, task) -> { + ctx.deferCompletion(); + executor.execute(Arrays.hashCode(task.getKey()), () -> { + try { + Thread.sleep(rand.nextInt(10)); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } finally { + ctx.deferCompletion().complete(); + } + }); + })) + .build() + .run(); + } } @Test diff --git a/processor/src/it/java/com/linecorp/decaton/processor/VThreadCoreFunctionalityTest.java b/processor/src/it/java/com/linecorp/decaton/processor/VThreadCoreFunctionalityTest.java index f6e09008..1bdb1e33 100644 --- a/processor/src/it/java/com/linecorp/decaton/processor/VThreadCoreFunctionalityTest.java +++ b/processor/src/it/java/com/linecorp/decaton/processor/VThreadCoreFunctionalityTest.java @@ -16,9 +16,8 @@ package com.linecorp.decaton.processor; +import java.util.Arrays; import java.util.Random; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -33,6 +32,8 @@ import com.linecorp.decaton.processor.runtime.SubPartitionRuntime; import com.linecorp.decaton.testing.KafkaClusterExtension; import com.linecorp.decaton.testing.RandomExtension; +import com.linecorp.decaton.testing.processor.KeyedExecutorService; +import com.linecorp.decaton.testing.processor.ProcessingGuarantee.GuaranteeType; import com.linecorp.decaton.testing.processor.ProcessorTestSuite; @EnabledForJreRange(min = JRE.JAVA_21) @@ -88,26 +89,27 @@ public void testProcessConcurrent_ThreadScopeProcessor() throws Exception { @Test @Timeout(30) public void testAsyncTaskCompletion() throws Exception { - ExecutorService executorService = Executors.newFixedThreadPool(16); - Random rand = randomExtension.random(); - ProcessorTestSuite - .builder(rule) - .subPartitionRuntime(SubPartitionRuntime.VIRTUAL_THREAD) - .configureProcessorsBuilder(builder -> builder.thenProcess((ctx, task) -> { - DeferredCompletion completion = ctx.deferCompletion(); - executorService.execute(() -> { - try { - Thread.sleep(rand.nextInt(10)); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } finally { - completion.complete(); - } - }); - })) - .build() - .run(); + try (KeyedExecutorService executor = new KeyedExecutorService(16)) { + Random rand = randomExtension.random(); + ProcessorTestSuite + .builder(rule) + .subPartitionRuntime(SubPartitionRuntime.VIRTUAL_THREAD) + .configureProcessorsBuilder(builder -> builder.thenProcess((ctx, task) -> { + DeferredCompletion completion = ctx.deferCompletion(); + executor.execute(Arrays.hashCode(task.getKey()), () -> { + try { + Thread.sleep(rand.nextInt(10)); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } finally { + completion.complete(); + } + }); + })) + .build() + .run(); + } } /* @@ -121,26 +123,27 @@ public void testAsyncTaskCompletion() throws Exception { @Test @Timeout(30) public void testGetCompletionInstanceLater() throws Exception { - ExecutorService executorService = Executors.newFixedThreadPool(16); - Random rand = randomExtension.random(); - ProcessorTestSuite - .builder(rule) - .subPartitionRuntime(SubPartitionRuntime.VIRTUAL_THREAD) - .configureProcessorsBuilder(builder -> builder.thenProcess((ctx, task) -> { - ctx.deferCompletion(); - executorService.execute(() -> { - try { - Thread.sleep(rand.nextInt(10)); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } finally { - ctx.deferCompletion().complete(); - } - }); - })) - .build() - .run(); + try (KeyedExecutorService executor = new KeyedExecutorService(16)) { + Random rand = randomExtension.random(); + ProcessorTestSuite + .builder(rule) + .subPartitionRuntime(SubPartitionRuntime.VIRTUAL_THREAD) + .configureProcessorsBuilder(builder -> builder.thenProcess((ctx, task) -> { + ctx.deferCompletion(); + executor.execute(Arrays.hashCode(task.getKey()), () -> { + try { + Thread.sleep(rand.nextInt(10)); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } finally { + ctx.deferCompletion().complete(); + } + }); + })) + .build() + .run(); + } } @Test @@ -159,6 +162,7 @@ public void testAsyncCompletionWithLeakAndTimeout() throws Exception { ctx.deferCompletion(); } })) + .excludeSemantics(GuaranteeType.PROCESS_ORDERING) .build() .run(); } diff --git a/processor/src/main/java/com/linecorp/decaton/processor/processors/BatchingProcessor.java b/processor/src/main/java/com/linecorp/decaton/processor/processors/BatchingProcessor.java index ac236fbd..5e2bed41 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/processors/BatchingProcessor.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/processors/BatchingProcessor.java @@ -57,9 +57,9 @@ public static class BatchingTask { /** * Instantiate {@link BatchingProcessor}. * @param lingerMillis time limit for this processor. On every lingerMillis milliseconds, - * tasks in past lingerMillis milliseconds are pushed to {@link BatchingTask#processBatchingTasks(List)}. + * tasks in past lingerMillis milliseconds are pushed to {@link BatchingProcessor#processBatchingTasks(List)}. * @param capacity size limit for this processor. Every time tasks’size reaches capacity, - * tasks in past before reaching capacity are pushed to {@link BatchingTask#processBatchingTasks(List)}. + * tasks in past before reaching capacity are pushed to {@link BatchingProcessor#processBatchingTasks(List)}. */ protected BatchingProcessor(long lingerMillis, int capacity) { this.lingerMillis = lingerMillis; diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java index 96548d4c..b0c2f715 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/ProcessorSubscription.java @@ -109,8 +109,8 @@ public void receive(ConsumerRecord record) { OffsetState offsetState; try { offsetState = context.registerOffset(record.offset()); - } catch (OffsetRegressionException ignored) { - log.warn("Offset regression at partition {}, by offset {}", tp, record.offset()); + } catch (OffsetRegressionException e) { + log.warn("Offset regression at partition {}, by offset {}", tp, record.offset(), e); assignManager.repair(tp); context = contexts.get(tp); // If it fails even at 2nd attempt... no idea let it die. diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/AssignmentManager.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/AssignmentManager.java index 05e8d78a..38b375bf 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/AssignmentManager.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/AssignmentManager.java @@ -120,7 +120,7 @@ public void assign(Map newAssignment) { public void repair(TopicPartition tp) { log.info("Repairing partition: {}", tp); partitionsRevoked(Collections.singletonList(tp)); - partitionsAssigned(Collections.singletonMap(tp, null)); + partitionsAssigned(Collections.singletonMap(tp, null)); // TODO: ok to not regard offset metadata? } private static List computeRemovedPartitions( diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConsumeManager.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConsumeManager.java index 50645b33..e9be76f7 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConsumeManager.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConsumeManager.java @@ -17,6 +17,7 @@ package com.linecorp.decaton.processor.runtime.internal; import java.time.Duration; +import java.util.ArrayList; import java.util.Collection; import java.util.HashSet; import java.util.List; @@ -32,6 +33,7 @@ import org.apache.kafka.common.TopicPartition; import com.linecorp.decaton.processor.metrics.Metrics.SubscriptionMetrics; +import com.linecorp.decaton.processor.runtime.ConsumedRecord; import com.linecorp.decaton.processor.runtime.internal.Utils.Timer; import lombok.extern.slf4j.Slf4j; @@ -141,6 +143,9 @@ public void onPartitionsAssigned(Collection ignored) { Map partitionCommits = consumer.committed(consumer.assignment()); handler.updateAssignment(partitionCommits); + for (TopicPartition tp : consumer.assignment()) { + log.debug("Consumer position of {}: {}", tp, consumer.position(tp)); + } // Consumer rebalance resets all pause states of assigned partitions even though they // haven't moved over from/to different consumer instance. diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java index 21438e91..a93cbfae 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OffsetStorageComplex.java @@ -93,6 +93,13 @@ public long indexOf(long offset) { return blockInfo.index + offset - e.getKey(); } + public long activeIndexOf(long offset) { + long index = indexOf(offset); + // The index is active (has been registered after initialization) only if the nextIndex + // already went beyond that. + return index < nextIndex ? index : -1; + } + public long addOffset(long offset) { if (blockIndex.isEmpty()) { blockIndex.put(offset, new BlockInfo(nextIndex, 1)); @@ -191,6 +198,16 @@ public void pollFirst() { } public int allocNextIndex(long offset) { + long activeIndex = index.activeIndexOf(offset); + if (activeIndex >= 0) { + // TODO: I'm not sure if this addressing is right approach... + // maybe instead of trying to avoid dups w/ eager rebalance, we should completely + // based on cooperative rebalance so that w/o taking care of possibility of + // offset regression caused by timing difference between onPartitionsRevoked() + // and onPartitionsAssigned(), we can simply assume HW won't progress beyond the + // committed offset during rebalance. + return (int) (activeIndex % states.length); + } if (size() == states.length) { throw new IllegalStateException("complex reached its capacity: " + states.length); } diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java index f5df8012..84931210 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java @@ -62,13 +62,18 @@ public static OutOfOrderCommitControl fromOffsetMeta(TopicPartition tp, OffsetStateReaper offsetStateReaper, OffsetAndMetadata offsetMeta) { OffsetStorageComplex complex = complexFromMeta(offsetMeta.metadata()); - return new OutOfOrderCommitControl(tp, capacity, offsetStateReaper, complex, offsetMeta.offset() - 1); + OutOfOrderCommitControl cc = new OutOfOrderCommitControl( + tp, capacity, offsetStateReaper, complex, offsetMeta.offset() - 1); + log.debug("Restoring OOOCC from offsetMeta for {}: pending={} hw={} states={}", + cc.topicPartition, cc.pendingOffsetsCount(), cc.highWatermark, cc.complex.compDebugDump()); + return cc; } public synchronized OffsetState reportFetchedOffset(long offset) { if (isRegressing(offset)) { - throw new OffsetRegressionException(String.format( - "offset regression %s: %d < %d", topicPartition, offset, highWatermark)); + log.debug("Not newly adding offset state on {} for offset={} because its lower than HW", + topicPartition, offset); + return null; } if (complex.size() == capacity) { @@ -79,10 +84,12 @@ public synchronized OffsetState reportFetchedOffset(long offset) { int ringIndex = complex.allocNextIndex(offset); if (complex.isComplete(offset)) { // There are two cases for this. - // 1. Offset bigger than that complex's managing bounds. Reasonable to consider as not completed + // 1. Offset is bigger than the complex's managing bounds. Reasonable to consider as not completed // because it is the offset to coming in future (and will be added to complex in line below). // 2. Offset has been processed in the past, marked as completed and now the consumer's consuming // it again from the point of watermark. + log.debug("Not newly adding offset state on {} for offset={}, ringIndex={} because it's complete", + topicPartition, offset, ringIndex); return null; } @@ -162,12 +169,17 @@ static OffsetStorageComplexProto parseOffsetMeta(String metadata) { } public boolean isRegressing(long offset) { - long firstOffset = complex.firstOffset(); - if (firstOffset < 0) { - return offset <= highWatermark; - } else { - return offset < firstOffset; - } + // There's a case that regressing offset gets reported into. + // A most typical case may happen in the following process sequence. + // 1. Consumer commits offset X + // 2. Consumer starts rebalancing + // 3. The remaining tasks in queue, having offset X+1 and later kept processing + // 4. updateHighWatermark() => HW becomes X+1 or later. + // 5. Consumer rebalance complements, the consuming offset resets to X. + // 6. reportFetchedOffset(X) => regression. + // As long as the reported offset is lower than the highWatermark, we should be able to + // respond as the offset has already been processed. + return offset <= highWatermark; } @Override diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/KeyedExecutorService.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/KeyedExecutorService.java new file mode 100644 index 00000000..33bfb644 --- /dev/null +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/KeyedExecutorService.java @@ -0,0 +1,42 @@ +/* + * Copyright 2024 LY Corporation + * + * LY Corporation 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: + * + * https://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 com.linecorp.decaton.testing.processor; + +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +public class KeyedExecutorService implements AutoCloseable { + private final ExecutorService[] executors; + + public KeyedExecutorService(int partitions) { + executors = new ExecutorService[partitions]; + for (int i = 0; i < partitions; i++) { + executors[i] = Executors.newSingleThreadExecutor(); + } + } + public void execute(int key, Runnable runnable) { + int partition = Math.abs(key) % executors.length; + executors[partition].execute(runnable); + } + + @Override + public void close() throws Exception { + for (ExecutorService executor : executors) { + executor.close(); + } + } +} diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessorTestSuite.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessorTestSuite.java index d5e3aa79..cc7e6760 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessorTestSuite.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/ProcessorTestSuite.java @@ -427,8 +427,8 @@ private CompletableFuture> produceTasks( TestTaskSerializer serializer = new TestTaskSerializer(); for (int i = 0; i < produceFutures.length; i++) { - TestTask task = new TestTask(String.valueOf(i)); byte[] key = String.valueOf(i % NUM_KEYS).getBytes(StandardCharsets.UTF_8); + TestTask task = new TestTask(String.valueOf(i), key); TaskMetadataProto taskMetadata = TaskMetadataProto.newBuilder() .setTimestampMillis(System.currentTimeMillis()) diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/TestTask.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/TestTask.java index f9c4389e..bc181e3f 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/TestTask.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/TestTask.java @@ -38,6 +38,10 @@ public class TestTask { * Unique id of the task */ String id; + /** + * The key to use for partitioning. + */ + byte[] key; private static final ObjectMapper mapper = new ObjectMapper(); From f5a5a89706c4ac375fd8d2181ccb74ebb909cbeb Mon Sep 17 00:00:00 2001 From: Yuto Kawamura Date: Wed, 25 Dec 2024 15:16:09 +0900 Subject: [PATCH 07/12] (fixup) drop println --- .../decaton/processor/runtime/internal/ConcurrentBitMap.java | 1 - .../decaton/processor/runtime/internal/PartitionContext.java | 1 - .../processor/runtime/internal/OutOfOrderCommitControlTest.java | 2 -- 3 files changed, 4 deletions(-) diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConcurrentBitMap.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConcurrentBitMap.java index 8dd81cfa..600a1224 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConcurrentBitMap.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConcurrentBitMap.java @@ -77,7 +77,6 @@ public boolean get(int index) { int bucket = bucketOf(index); long bits = buckets.get(bucket); boolean x = (bits >> localIndex(index) & 1) == 1; - System.err.println("BITMAP GET of " + index + " is " + x + ", bits = " + bits); return x; } diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/PartitionContext.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/PartitionContext.java index cf08d624..e97bc30b 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/PartitionContext.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/PartitionContext.java @@ -107,7 +107,6 @@ private static SubPartitions createSubPartitions(PartitionScope scope, Processor if (offsetMeta == null) { commitControl = new OutOfOrderCommitControl(scope.topicPartition(), capacity, offsetStateReaper); } else { - System.err.println("Creating OOOCC from offsetMeta: " + offsetMeta); commitControl = OutOfOrderCommitControl.fromOffsetMeta( scope.topicPartition(), capacity, offsetStateReaper, offsetMeta); } diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControlTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControlTest.java index 6c2b24f0..2d5e96da 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControlTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControlTest.java @@ -201,11 +201,9 @@ void perOffsetCompleteTest() { commitControl.updateHighWatermark(); OffsetAndMetadata om = commitControl.commitReadyOffset(); - System.err.println("ooocc.complex = " + commitControl.complex); assertEquals(om.offset(), 102L); OffsetStorageComplex complex = OutOfOrderCommitControl.complexFromMeta(om.metadata()); - System.err.println("meta = " + om.metadata()); assertEquals(0, complex.size()); assertFalse(complex.isComplete(102)); assertTrue(complex.isComplete(103)); From 4a1a249b044467fc3cf6d694eb8c992488025306 Mon Sep 17 00:00:00 2001 From: Yuto Kawamura Date: Wed, 25 Dec 2024 17:34:02 +0900 Subject: [PATCH 08/12] (fixup) make commit whenever update even though the HW doesn't progress --- processor/src/it/resources/logback.xml | 1 + .../OutOfOrderCommitControlBenchmark.java | 6 ++-- .../internal/OutOfOrderCommitControl.java | 25 ++++++++----- .../runtime/internal/PartitionContext.java | 8 ++--- .../internal/OutOfOrderCommitControlTest.java | 36 +++++++++---------- 5 files changed, 41 insertions(+), 35 deletions(-) diff --git a/processor/src/it/resources/logback.xml b/processor/src/it/resources/logback.xml index 5c919ba9..a417ea74 100644 --- a/processor/src/it/resources/logback.xml +++ b/processor/src/it/resources/logback.xml @@ -25,6 +25,7 @@ + diff --git a/processor/src/jmh/java/com/linecorp/decaton/processor/runtime/OutOfOrderCommitControlBenchmark.java b/processor/src/jmh/java/com/linecorp/decaton/processor/runtime/OutOfOrderCommitControlBenchmark.java index 761f5e8c..7341b7eb 100644 --- a/processor/src/jmh/java/com/linecorp/decaton/processor/runtime/OutOfOrderCommitControlBenchmark.java +++ b/processor/src/jmh/java/com/linecorp/decaton/processor/runtime/OutOfOrderCommitControlBenchmark.java @@ -57,8 +57,8 @@ * it. * - They are passed into threads pool consists of {@link #NUM_WORKER_THREADS}, and then completed immediately. * - Consumer loop thread calls {@link OutOfOrderCommitControl#updateHighWatermark()} after feeding - * {@link #BATCH_SIZE} and calls {@link OutOfOrderCommitControl#commitReadyOffset()}. - * - Loop the above steps until {@link OutOfOrderCommitControl#commitReadyOffset()} returns the value equals to + * {@link #BATCH_SIZE} and calls {@link OutOfOrderCommitControl#commitReadyOffset(long)}. + * - Loop the above steps until {@link OutOfOrderCommitControl#commitReadyOffset(long)} returns the value equals to * {@link #NUM_OFFSETS}. * - Measure entire execution duration as a performance indicator. */ @@ -244,7 +244,7 @@ public void outOfOrderCommitControlV4(BmStateV4 state) throws InterruptedExcepti } control.updateHighWatermark(); - while (control.commitReadyOffset() < NUM_OFFSETS) { + while (control.commitReadyOffset(0).offset() < NUM_OFFSETS + 1) { Thread.yield(); control.updateHighWatermark(); } diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java index 84931210..10707a6d 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControl.java @@ -24,7 +24,6 @@ import com.linecorp.decaton.protocol.Decaton.OffsetStorageComplexProto; -import lombok.AllArgsConstructor; import lombok.Getter; import lombok.experimental.Accessors; import lombok.extern.slf4j.Slf4j; @@ -35,7 +34,6 @@ */ @Slf4j @Accessors(fluent = true) -@AllArgsConstructor public class OutOfOrderCommitControl implements AutoCloseable { @Getter private final TopicPartition topicPartition; @@ -47,14 +45,21 @@ public class OutOfOrderCommitControl implements AutoCloseable { * The current maximum offset which it and all it's previous offsets were committed. */ private volatile long highWatermark; + private volatile boolean anyOffsetsUpdated; public OutOfOrderCommitControl(TopicPartition topicPartition, int capacity, - OffsetStateReaper offsetStateReaper) { + OffsetStateReaper offsetStateReaper, + OffsetStorageComplex complex, long highWatermark) { this.topicPartition = topicPartition; this.capacity = capacity; this.offsetStateReaper = offsetStateReaper; - complex = new OffsetStorageComplex(capacity); - highWatermark = -1; + this.complex = complex; + this.highWatermark = highWatermark; + } + + public OutOfOrderCommitControl(TopicPartition topicPartition, int capacity, + OffsetStateReaper offsetStateReaper) { + this(topicPartition, capacity, offsetStateReaper, new OffsetStorageComplex(capacity), -1); } public static OutOfOrderCommitControl fromOffsetMeta(TopicPartition tp, @@ -103,6 +108,7 @@ void onComplete(long offset, int ringIndex) { log.debug("Offset complete on {}: {}", topicPartition, offset); } complex.complete(ringIndex); + anyOffsetsUpdated = true; } public synchronized void updateHighWatermark() { @@ -139,18 +145,21 @@ public synchronized int pendingOffsetsCount() { return complex.size(); } - public OffsetAndMetadata commitReadyOffset() { - if (highWatermark < 0) { + public OffsetAndMetadata commitReadyOffset(long lastCommittedOffset) { + if (highWatermark < lastCommittedOffset && !anyOffsetsUpdated) { return null; } OffsetStorageComplexProto complexProto = complex.toProto(); + final OffsetAndMetadata offsetMeta; try { String meta = JsonFormat.printer().omittingInsignificantWhitespace().print(complexProto); long commitOffset = highWatermark + 1; - return new OffsetAndMetadata(commitOffset, meta); + offsetMeta = new OffsetAndMetadata(commitOffset, meta); } catch (InvalidProtocolBufferException e) { throw new RuntimeException("failed to serialize offset metadata into proto", e); } + anyOffsetsUpdated = false; + return offsetMeta; } static OffsetStorageComplex complexFromMeta(String metadata) { diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/PartitionContext.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/PartitionContext.java index e97bc30b..98865c25 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/PartitionContext.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/PartitionContext.java @@ -119,7 +119,7 @@ private static SubPartitions createSubPartitions(PartitionScope scope, Processor metrics = metricsCtor.new PartitionStateMetrics( commitControl::pendingOffsetsCount, () -> paused() ? 1 : 0, () -> lastCommittedOffset, () -> latestConsumedOffset); - lastCommittedOffset = -1; + lastCommittedOffset = 0; pausedTimeNanos = -1; lastQueueStarvedTime = -1; } @@ -133,11 +133,7 @@ private static SubPartitions createSubPartitions(PartitionScope scope, Processor * @return optional long value representing an offset waiting for commit. */ public Optional offsetWaitingCommit() { - OffsetAndMetadata offsetMeta = commitControl.commitReadyOffset(); - if (offsetMeta != null && offsetMeta.offset() > lastCommittedOffset) { - return Optional.of(offsetMeta); - } - return Optional.empty(); + return Optional.ofNullable(commitControl.commitReadyOffset(lastCommittedOffset)); } /** diff --git a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControlTest.java b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControlTest.java index 2d5e96da..a9a0bfff 100644 --- a/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControlTest.java +++ b/processor/src/test/java/com/linecorp/decaton/processor/runtime/internal/OutOfOrderCommitControlTest.java @@ -51,17 +51,17 @@ public void testInOrderOffsetCompletion() { state1.completion().complete(); commitControl.updateHighWatermark(); assertEquals(2, commitControl.pendingOffsetsCount()); - assertEquals(2, commitControl.commitReadyOffset().offset()); + assertEquals(2, commitControl.commitReadyOffset(0).offset()); state2.completion().complete(); commitControl.updateHighWatermark(); assertEquals(1, commitControl.pendingOffsetsCount()); - assertEquals(3, commitControl.commitReadyOffset().offset()); + assertEquals(3, commitControl.commitReadyOffset(0).offset()); state3.completion().complete(); commitControl.updateHighWatermark(); assertEquals(0, commitControl.pendingOffsetsCount()); - assertEquals(4, commitControl.commitReadyOffset().offset()); + assertEquals(4, commitControl.commitReadyOffset(0).offset()); } @Test @@ -74,22 +74,22 @@ public void testOutOfOrderOffsetCompletion() { state3.completion().complete(); commitControl.updateHighWatermark(); assertEquals(4, commitControl.pendingOffsetsCount()); - assertNull(commitControl.commitReadyOffset()); + assertEquals(0, commitControl.commitReadyOffset(0).offset()); state2.completion().complete(); commitControl.updateHighWatermark(); assertEquals(4, commitControl.pendingOffsetsCount()); - assertNull(commitControl.commitReadyOffset()); + assertEquals(0, commitControl.commitReadyOffset(0).offset()); state1.completion().complete(); commitControl.updateHighWatermark(); assertEquals(1, commitControl.pendingOffsetsCount()); - assertEquals(4, commitControl.commitReadyOffset().offset()); + assertEquals(4, commitControl.commitReadyOffset(0).offset()); state4.completion().complete(); commitControl.updateHighWatermark(); assertEquals(0, commitControl.pendingOffsetsCount()); - assertEquals(5, commitControl.commitReadyOffset().offset()); + assertEquals(5, commitControl.commitReadyOffset(0).offset()); } @Test @@ -97,10 +97,10 @@ public void testDoubleCompletingSameOffset() { OffsetState state1 = commitControl.reportFetchedOffset(1); state1.completion().complete(); - assertNull(commitControl.commitReadyOffset()); + assertEquals(0, commitControl.commitReadyOffset(0).offset()); state1.completion().complete(); // nothing happens commitControl.updateHighWatermark(); - assertEquals(2, commitControl.commitReadyOffset().offset()); + assertEquals(2, commitControl.commitReadyOffset(0).offset()); state1.completion().complete(); // nothing happens } @@ -120,10 +120,10 @@ public void testDoubleCompletingSameOffsetCaseDuplicateInCommitted() { state2.completion().complete(); // now committedOffsets contains 2 commitControl.updateHighWatermark(); - assertNull(commitControl.commitReadyOffset()); + assertEquals(0, commitControl.commitReadyOffset(0).offset()); state2.completion().complete(); // commit again commitControl.updateHighWatermark(); - assertNull(commitControl.commitReadyOffset()); + assertNull(commitControl.commitReadyOffset(0)); } @Test @@ -137,7 +137,7 @@ public void testPendingRecordsCountWithGaps() { state1.completion().complete(); state3.completion().complete(); commitControl.updateHighWatermark(); - assertEquals(4, commitControl.commitReadyOffset().offset()); + assertEquals(4, commitControl.commitReadyOffset(0).offset()); assertEquals(0, commitControl.pendingOffsetsCount()); } @@ -153,7 +153,7 @@ public void testPendingRecordsCountWithLargeGap() { state1.completion().complete(); stateLarge.completion().complete(); commitControl.updateHighWatermark(); - assertEquals(largeGapOffset + 1, commitControl.commitReadyOffset().offset()); + assertEquals(largeGapOffset + 1, commitControl.commitReadyOffset(0).offset()); assertEquals(0, commitControl.pendingOffsetsCount()); } @@ -178,16 +178,16 @@ public void testTimeoutOffsetReaping() throws InterruptedException { // 1 is blocking watermark to progress state2.completion().complete(); ooocc.updateHighWatermark(); - assertNull(ooocc.commitReadyOffset()); + assertEquals(0, ooocc.commitReadyOffset(0).offset()); doReturn(20L).when(clock).millis(); // offset reaping performed but does not proceed watermark yet ooocc.updateHighWatermark(); state1.completion().asFuture().toCompletableFuture().join(); - assertNull(ooocc.commitReadyOffset()); + assertEquals(0, ooocc.commitReadyOffset(0).offset()); // offset should progress as offset 1 has reaped in previous call ooocc.updateHighWatermark(); - assertEquals(3, ooocc.commitReadyOffset().offset()); + assertEquals(3, ooocc.commitReadyOffset(0).offset()); } @Test @@ -200,7 +200,7 @@ void perOffsetCompleteTest() { state3.completion().complete(); commitControl.updateHighWatermark(); - OffsetAndMetadata om = commitControl.commitReadyOffset(); + OffsetAndMetadata om = commitControl.commitReadyOffset(0); assertEquals(om.offset(), 102L); OffsetStorageComplex complex = OutOfOrderCommitControl.complexFromMeta(om.metadata()); @@ -210,7 +210,7 @@ void perOffsetCompleteTest() { state2.completion().complete(); commitControl.updateHighWatermark(); - om = commitControl.commitReadyOffset(); + om = commitControl.commitReadyOffset(0); assertEquals(om.offset(), 104L); complex = OutOfOrderCommitControl.complexFromMeta(om.metadata()); From f2dbef0e75b5a23bbe129d1fc738c27378954ee7 Mon Sep 17 00:00:00 2001 From: Yuto Kawamura Date: Thu, 26 Dec 2024 14:56:55 +0900 Subject: [PATCH 09/12] (fixup) incidentally used too new feature --- .../linecorp/decaton/processor/CoreFunctionalityTest.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java b/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java index 81df29b8..8f0710cf 100644 --- a/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java +++ b/processor/src/it/java/com/linecorp/decaton/processor/CoreFunctionalityTest.java @@ -26,6 +26,7 @@ import java.util.Random; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.stream.Collectors; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.CooperativeStickyAssignor; @@ -190,8 +191,8 @@ public void doAssert() { // use assertTrue instead of assertEquals not to cause error message explosion //noinspection SimplifiableJUnitAssertion for (Entry> e : produced.entrySet()) { - List producedTasks = e.getValue().stream().map(taskToOffset::get).toList(); - List processedTasks = processed.get(e.getKey()).stream().map(taskToOffset::get).toList(); + List producedTasks = e.getValue().stream().map(taskToOffset::get).collect(Collectors.toList()); + List processedTasks = processed.get(e.getKey()).stream().map(taskToOffset::get).collect(Collectors.toList()); assertEquals(producedTasks, processedTasks); } // assertTrue(produced.equals(processed)); From e075ffaffa097c4f7575e53bbf675ca3a724e50d Mon Sep 17 00:00:00 2001 From: Yuto Kawamura Date: Thu, 26 Dec 2024 15:03:25 +0900 Subject: [PATCH 10/12] (fixup) and more... --- .../decaton/processor/runtime/internal/ConcurrentBitMap.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConcurrentBitMap.java b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConcurrentBitMap.java index 600a1224..5d0b3e37 100644 --- a/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConcurrentBitMap.java +++ b/processor/src/main/java/com/linecorp/decaton/processor/runtime/internal/ConcurrentBitMap.java @@ -55,7 +55,7 @@ private static int localIndex(int index) { private void ensureBound(int index) { if (index < 0 || index >= size) { - throw new IndexOutOfBoundsException(index); + throw new IndexOutOfBoundsException("Index out of range: " + index); } } From 4ea69512c4becd516f98d736c9db529b086886ce Mon Sep 17 00:00:00 2001 From: Yuto Kawamura Date: Thu, 26 Dec 2024 15:27:14 +0900 Subject: [PATCH 11/12] (fixup) and more... --- .../decaton/testing/processor/KeyedExecutorService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/testing/src/main/java/com/linecorp/decaton/testing/processor/KeyedExecutorService.java b/testing/src/main/java/com/linecorp/decaton/testing/processor/KeyedExecutorService.java index 33bfb644..7746ea1b 100644 --- a/testing/src/main/java/com/linecorp/decaton/testing/processor/KeyedExecutorService.java +++ b/testing/src/main/java/com/linecorp/decaton/testing/processor/KeyedExecutorService.java @@ -36,7 +36,7 @@ public void execute(int key, Runnable runnable) { @Override public void close() throws Exception { for (ExecutorService executor : executors) { - executor.close(); + executor.shutdown(); } } } From 5abfbe7e010217cdb6ffea5c70dc229708a4e17d Mon Sep 17 00:00:00 2001 From: Yuto Kawamura Date: Thu, 26 Dec 2024 16:02:21 +0900 Subject: [PATCH 12/12] (fixup) make BatchingProcessor to comply w/ new sematnics --- .../decaton/processor/BatchingProcessorTest.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/processor/src/it/java/com/linecorp/decaton/processor/BatchingProcessorTest.java b/processor/src/it/java/com/linecorp/decaton/processor/BatchingProcessorTest.java index 216ea48b..e6c9a3b4 100644 --- a/processor/src/it/java/com/linecorp/decaton/processor/BatchingProcessorTest.java +++ b/processor/src/it/java/com/linecorp/decaton/processor/BatchingProcessorTest.java @@ -18,6 +18,7 @@ import java.util.List; import java.util.Random; +import java.util.concurrent.locks.ReentrantLock; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -42,20 +43,27 @@ public class BatchingProcessorTest { @Timeout(30) public void testBatchingProcessor() throws Exception { Random rand = randomExtension.random(); + ReentrantLock serializer = new ReentrantLock(true); ProcessorTestSuite .builder(rule) .configureProcessorsBuilder(builder -> builder.thenProcess( new BatchingProcessor(1000, 100) { @Override protected void processBatchingTasks(List> batchingTasks) { - // adding some random delay to simulate realistic usage + // Since multiple calls to this method might be executed concurrently, + // it is implementation-side responsibility to ensure completion of each batch + // to happen in-order. + serializer.lock(); try { + // adding some random delay to simulate realistic usage Thread.sleep(rand.nextInt(10)); + batchingTasks.forEach(batchingTask -> batchingTask.completion().complete()); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new RuntimeException(e); + } finally { + serializer.unlock(); } - batchingTasks.forEach(batchingTask -> batchingTask.completion().complete()); } } ))