From 5e8bfb6482e523aef409c624809086a0114d34a6 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Fri, 30 Sep 2022 16:28:07 +0100 Subject: [PATCH 01/33] START: fixes #409: Truncate state on offset high reset Track expected offsets returned from broker, and truncate where needed. --- .../offsets/OffsetMapCodecManager.java | 3 +- .../state/PartitionState.java | 59 +++++++++++++++---- .../state/PartitionStateManager.java | 2 +- .../state/PartitionStateTest.java | 49 +++++++++++++++ 4 files changed, 101 insertions(+), 12 deletions(-) create mode 100644 parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/offsets/OffsetMapCodecManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/offsets/OffsetMapCodecManager.java index 7283f9b0b..83c050224 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/offsets/OffsetMapCodecManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/offsets/OffsetMapCodecManager.java @@ -72,6 +72,7 @@ public static class HighestOffsetAndIncompletes { /** * Of the offsets encoded, the incomplete ones. */ + // todo change to List as Sets have no order Set incompleteOffsets; public static HighestOffsetAndIncompletes of(long highestSeenOffset) { @@ -164,7 +165,7 @@ public static HighestOffsetAndIncompletes deserialiseIncompleteOffsetMapFromBase PartitionState decodePartitionState(TopicPartition tp, OffsetAndMetadata offsetData) throws OffsetDecodingError { HighestOffsetAndIncompletes incompletes = deserialiseIncompleteOffsetMapFromBase64(offsetData); log.debug("Loaded incomplete offsets from offset payload {}", incompletes); - return new PartitionState(tp, incompletes); + return new PartitionState<>(tp, incompletes); } public String makeOffsetMetadataPayload(long baseOffsetForPartition, PartitionState state) throws NoEncodingPossibleException { diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java index 849e90f7c..624944a6d 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java @@ -15,10 +15,7 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import java.util.Collections; -import java.util.NavigableMap; -import java.util.Optional; -import java.util.Set; +import java.util.*; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ConcurrentSkipListSet; import java.util.stream.Collectors; @@ -59,7 +56,12 @@ public class PartitionState { * so that the broker polling thread can grab it, if it wants to commit - i.e. the poller would not prepare/query * the data for itself. See also #200 Refactor: Consider a shared nothing architecture. */ - private final ConcurrentSkipListSet incompleteOffsets; + private ConcurrentSkipListSet incompleteOffsets; + + /** + * Marker for the first record to be tracked. Used for some initial analysis. + */ + private boolean noWorkAddedYet = true; /** * Cache view of the state of the partition. Is set dirty when the incomplete state of any offset changes. Is set @@ -126,6 +128,7 @@ public PartitionState(TopicPartition tp, OffsetMapCodecManager.HighestOffsetAndI this.offsetHighestSeen = offsetData.getHighestSeenOffset().orElse(KAFKA_OFFSET_ABSENCE); this.incompleteOffsets = new ConcurrentSkipListSet<>(offsetData.getIncompleteOffsets()); this.offsetHighestSucceeded = this.offsetHighestSeen; + this.nextExpectedPolledOffset = getNextExpectedInitialPolledOffset(); } private void maybeRaiseHighestSeenOffset(final long offset) { @@ -199,11 +202,43 @@ private void updateHighestSucceededOffsetSoFar(WorkContainer work) { public void addWorkContainer(WorkContainer wc) { long newOffset = wc.offset(); + +// if (noWorkAddedYet) { +// noWorkAddedYet = false; +// long bootstrapOffset = wc.offset(); + maybeTruncateBelow(newOffset); +// } + maybeRaiseHighestSeenOffset(newOffset); commitQueue.put(newOffset, wc); incompleteOffsets.add(newOffset); } + /** + * If the offset is higher than expected, according to the previously committed / polled offset, truncate up to it. + * Offsets between have disappeared and will never be polled again. + */ + private void maybeTruncateBelow(long polledOffset) { + long nextExpectedPolledOffset = this.getNextExpectedPolledOffset(); + boolean bootstrapRecordAboveExpected = polledOffset > nextExpectedPolledOffset; + if (bootstrapRecordAboveExpected) { + log.debug("Truncating state - offsets have been removed form the partition by the broker. Polled {} but expected {} - e.g. record retention expiring, with 'auto.offset.reset'", + polledOffset, + nextExpectedPolledOffset); + NavigableSet truncatedIncompletes = incompleteOffsets.tailSet(polledOffset); + ConcurrentSkipListSet wrapped = new ConcurrentSkipListSet<>(truncatedIncompletes); + this.incompleteOffsets = wrapped; + } + + this.nextExpectedPolledOffset = polledOffset + 1; + } + + private long nextExpectedPolledOffset = KAFKA_OFFSET_ABSENCE; + + private long getNextExpectedPolledOffset() { + return nextExpectedPolledOffset; + } + /** * Has this partition been removed? No. * @@ -220,18 +255,22 @@ public Optional getCommitDataIfDirty() { return empty(); } - private OffsetAndMetadata createOffsetAndMetadata() { + // visible for testing + protected OffsetAndMetadata createOffsetAndMetadata() { Optional payloadOpt = tryToEncodeOffsets(); - long nextOffset = getNextExpectedPolledOffset(); + long nextOffset = getNextExpectedInitialPolledOffset(); return payloadOpt .map(s -> new OffsetAndMetadata(nextOffset, s)) .orElseGet(() -> new OffsetAndMetadata(nextOffset)); } /** - * Defines as the offset one below the highest sequentially succeeded offset + * Next offset expected to be polled, upon freshly connecting to a broker. + *

+ * Defines as the offset one below the highest sequentially succeeded offset. */ - private long getNextExpectedPolledOffset() { + // visible for testing + protected long getNextExpectedInitialPolledOffset() { return getOffsetHighestSequentialSucceeded() + 1; } @@ -297,7 +336,7 @@ private Optional tryToEncodeOffsets() { try { // todo refactor use of null shouldn't be needed. Is OffsetMapCodecManager stateful? remove null #233 OffsetMapCodecManager om = new OffsetMapCodecManager<>(null); - long offsetOfNextExpectedMessage = getNextExpectedPolledOffset(); + long offsetOfNextExpectedMessage = getNextExpectedInitialPolledOffset(); String offsetMapPayload = om.makeOffsetMetadataPayload(offsetOfNextExpectedMessage, this); boolean mustStrip = updateBlockFromEncodingResult(offsetMapPayload); if (mustStrip) { diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java index 2aff1eeb8..857c15e25 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java @@ -334,7 +334,7 @@ void maybeRegisterNewRecordAsWork(final EpochAndRecordsMap recordsMap) { log.debug("Incoming {} new records...", recordsMap.count()); for (var partition : recordsMap.partitions()) { var recordsList = recordsMap.records(partition); - var epochOfInboundRecords = recordsList.getEpochOfPartitionAtPoll(); + long epochOfInboundRecords = recordsList.getEpochOfPartitionAtPoll(); for (var rec : recordsList.getRecords()) { maybeRegisterNewRecordAsWork(epochOfInboundRecords, rec); } diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java new file mode 100644 index 000000000..9c55cca6a --- /dev/null +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java @@ -0,0 +1,49 @@ +package io.confluent.parallelconsumer.state; + +import com.google.common.truth.Truth; +import io.confluent.parallelconsumer.internal.PCModuleTestEnv; +import io.confluent.parallelconsumer.offsets.OffsetMapCodecManager.HighestOffsetAndIncompletes; +import org.apache.kafka.common.TopicPartition; +import org.junit.jupiter.api.Test; +import pl.tlinkowski.unij.api.UniLists; + +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +class PartitionStateTest { + + ModelUtils mu = new ModelUtils(new PCModuleTestEnv()); + + /** + * @see PartitionState#maybeTruncateBelow + */ + @Test + void bootstrapTruncation() { + TopicPartition tp = new TopicPartition("topic", 0); + long unexpectedlyHighOffset = 20L; + final long previouslyCommittedOffset = 11L; + List incompletes = UniLists.of(previouslyCommittedOffset, 15L, unexpectedlyHighOffset, 60L, 80L); + List expectedTruncatedIncompletes = incompletes.stream() + .filter(offset -> offset >= unexpectedlyHighOffset) + .collect(Collectors.toList()); + + HighestOffsetAndIncompletes offsetData = new HighestOffsetAndIncompletes(Optional.of(100L), new HashSet<>(incompletes)); // todo fix set/list + + PartitionState state = new PartitionState<>(tp, offsetData); + + + var w20 = mu.createWorkFor(unexpectedlyHighOffset); + + // bootstrap the first record, triggering truncation - it's offset #unexpectedlyHighOffset, but we were expecting #previouslyCommittedOffset + state.addWorkContainer(w20); + + + Truth.assertThat(state.getNextExpectedInitialPolledOffset()).isEqualTo(unexpectedlyHighOffset); + + Truth.assertThat(state.getIncompleteOffsetsBelowHighestSucceeded()).containsExactlyElementsIn(expectedTruncatedIncompletes); + + } + +} \ No newline at end of file From d1c42d692ddf8b3e152914d7012be5f76a576528 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Fri, 30 Sep 2022 16:40:47 +0100 Subject: [PATCH 02/33] review --- CHANGELOG.adoc | 1 + README.adoc | 1 + .../confluent/parallelconsumer/state/PartitionStateTest.java | 4 ++++ 3 files changed, 6 insertions(+) diff --git a/CHANGELOG.adoc b/CHANGELOG.adoc index b049e070a..582124344 100644 --- a/CHANGELOG.adoc +++ b/CHANGELOG.adoc @@ -40,6 +40,7 @@ endif::[] * fixes #419 NoSuchElementException during race condition in PartitionState (#422) * Fixes #412: ClassCastException with retryDelayProvider (#417) +* Fixes #409: Parallel Consumer keeps committing the old offset after OFFSET_OUT_OF_RANGE in auto.offset.reset = latest (#425) == v0.5.2.2 diff --git a/README.adoc b/README.adoc index 62e8edde8..ddeecb185 100644 --- a/README.adoc +++ b/README.adoc @@ -1312,6 +1312,7 @@ endif::[] * fixes #419 NoSuchElementException during race condition in PartitionState (#422) * Fixes #412: ClassCastException with retryDelayProvider (#417) +* Fixes #409: Parallel Consumer keeps committing the old offset after OFFSET_OUT_OF_RANGE in auto.offset.reset = latest (#425) == v0.5.2.2 diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java index 9c55cca6a..40886c4a3 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java @@ -1,5 +1,9 @@ package io.confluent.parallelconsumer.state; +/*- + * Copyright (C) 2020-2022 Confluent, Inc. + */ + import com.google.common.truth.Truth; import io.confluent.parallelconsumer.internal.PCModuleTestEnv; import io.confluent.parallelconsumer.offsets.OffsetMapCodecManager.HighestOffsetAndIncompletes; From 5dca73c03d3dbcac441e161f7f4185d05154000f Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Fri, 30 Sep 2022 16:41:40 +0100 Subject: [PATCH 03/33] review --- .../io/confluent/parallelconsumer/state/PartitionStateTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java index 40886c4a3..8f43fa215 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java @@ -23,6 +23,7 @@ class PartitionStateTest { /** * @see PartitionState#maybeTruncateBelow */ + // parameter test with offsets closer together to check off by one @Test void bootstrapTruncation() { TopicPartition tp = new TopicPartition("topic", 0); From 23c289f42f93e506baa0fbcf5e901bee68f98a1d Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Fri, 30 Sep 2022 17:22:47 +0100 Subject: [PATCH 04/33] step - continuous checking and restructure --- .../io/confluent/csid/utils/JavaUtils.java | 10 ++- .../state/PartitionState.java | 18 +++++- .../state/PartitionStateManager.java | 63 ++++++++++++++----- 3 files changed, 72 insertions(+), 19 deletions(-) diff --git a/parallel-consumer-core/src/main/java/io/confluent/csid/utils/JavaUtils.java b/parallel-consumer-core/src/main/java/io/confluent/csid/utils/JavaUtils.java index bba786ed7..37e56237d 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/csid/utils/JavaUtils.java +++ b/parallel-consumer-core/src/main/java/io/confluent/csid/utils/JavaUtils.java @@ -20,9 +20,13 @@ @UtilityClass public class JavaUtils { - public static Optional getLast(final List commitHistory) { - if (commitHistory.isEmpty()) return Optional.empty(); - return Optional.of(commitHistory.get(commitHistory.size() - 1)); + public static Optional getLast(final List someList) { + if (someList.isEmpty()) return Optional.empty(); + return Optional.of(someList.get(someList.size() - 1)); + } + + public static Optional getFirst(final List someList) { + return someList.isEmpty() ? Optional.empty() : Optional.of(someList.get(0)); } public static Optional getOnlyOne(final Map stringMapMap) { diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java index 624944a6d..73af0a68c 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java @@ -206,7 +206,7 @@ public void addWorkContainer(WorkContainer wc) { // if (noWorkAddedYet) { // noWorkAddedYet = false; // long bootstrapOffset = wc.offset(); - maybeTruncateBelow(newOffset); +// maybeTruncateBelow(newOffset); // } maybeRaiseHighestSeenOffset(newOffset); @@ -233,6 +233,21 @@ private void maybeTruncateBelow(long polledOffset) { this.nextExpectedPolledOffset = polledOffset + 1; } + public void maybeTruncate(long batchStartOffset, long batchEndOffset) { + long nextExpectedPolledOffset = this.getNextExpectedPolledOffset(); + boolean bootstrapRecordAboveExpected = batchStartOffset > nextExpectedPolledOffset; + if (bootstrapRecordAboveExpected) { + log.debug("Truncating state - offsets have been removed form the partition by the broker. Polled {} but expected {} - e.g. record retention expiring, with 'auto.offset.reset'", + batchStartOffset, + nextExpectedPolledOffset); + NavigableSet truncatedIncompletes = incompleteOffsets.tailSet(batchStartOffset); + ConcurrentSkipListSet wrapped = new ConcurrentSkipListSet<>(truncatedIncompletes); + this.incompleteOffsets = wrapped; + } + + this.nextExpectedPolledOffset = batchEndOffset + 1; + } + private long nextExpectedPolledOffset = KAFKA_OFFSET_ABSENCE; private long getNextExpectedPolledOffset() { @@ -398,5 +413,6 @@ public void onPartitionsRemoved(ShardManager sm) { public boolean isBlocked() { return !isAllowedMoreRecords(); } + } diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java index 857c15e25..488ca6b9c 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java @@ -6,6 +6,7 @@ import io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder; import io.confluent.parallelconsumer.internal.*; +import io.confluent.parallelconsumer.internal.EpochAndRecordsMap.RecordsAndEpoch; import io.confluent.parallelconsumer.offsets.OffsetMapCodecManager; import lombok.Getter; import lombok.NonNull; @@ -21,6 +22,8 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; +import static io.confluent.csid.utils.JavaUtils.getFirst; +import static io.confluent.csid.utils.JavaUtils.getLast; import static io.confluent.csid.utils.KafkaUtils.toTopicPartition; import static io.confluent.csid.utils.StringUtils.msg; @@ -244,6 +247,7 @@ boolean checkIfWorkIsStale(final WorkContainer workContainer) { return false; } + // todo move to partition state public boolean isRecordPreviouslyCompleted(ConsumerRecord rec) { var tp = toTopicPartition(rec); var partitionState = getPartitionState(tp); @@ -309,6 +313,7 @@ public boolean isBlocked(final TopicPartition topicPartition) { return !isAllowedMoreRecords(topicPartition); } + // todo move to partition state public boolean isPartitionRemovedOrNeverAssigned(ConsumerRecord rec) { TopicPartition topicPartition = toTopicPartition(rec); var partitionState = getPartitionState(topicPartition); @@ -333,10 +338,12 @@ public void onFailure(WorkContainer wc) { void maybeRegisterNewRecordAsWork(final EpochAndRecordsMap recordsMap) { log.debug("Incoming {} new records...", recordsMap.count()); for (var partition : recordsMap.partitions()) { - var recordsList = recordsMap.records(partition); + RecordsAndEpoch recordsList = recordsMap.records(partition); long epochOfInboundRecords = recordsList.getEpochOfPartitionAtPoll(); - for (var rec : recordsList.getRecords()) { - maybeRegisterNewRecordAsWork(epochOfInboundRecords, rec); + List> recordPollBatch = recordsList.getRecords(); + if (!recordPollBatch.isEmpty()) { + // should always not be empty, but... + maybeRegisterNewRecordAsWork(epochOfInboundRecords, recordsList); } } } @@ -344,26 +351,52 @@ void maybeRegisterNewRecordAsWork(final EpochAndRecordsMap recordsMap) { /** * @see #maybeRegisterNewRecordAsWork(EpochAndRecordsMap) */ - private void maybeRegisterNewRecordAsWork(@NonNull Long epochOfInboundRecords, @NonNull ConsumerRecord rec) { - // do epochs still match? do a proactive check, but the epoch will be checked again at work completion as well - var currentPartitionEpoch = getEpochOfPartitionForRecord(rec); - if (Objects.equals(epochOfInboundRecords, currentPartitionEpoch)) { + // todo move into PartitionState + // todo too deep + private void maybeRegisterNewRecordAsWork(@NonNull Long epochOfInboundRecords, @NonNull RecordsAndEpoch recordsList) { + List> recordPollBatch = recordsList.getRecords(); + + if (!recordPollBatch.isEmpty()) { + final Optional> recOpt = getFirst(recordPollBatch); + //noinspection OptionalGetWithoutIsPresent -- already checked not empty + ConsumerRecord sampleRecord = recOpt.get(); // NOSONAR + long batchStartOffset = sampleRecord.offset(); + + // do epochs still match? do a proactive check, but the epoch will be checked again at work completion as well + var currentPartitionEpoch = getEpochOfPartitionForRecord(sampleRecord); + if (Objects.equals(epochOfInboundRecords, currentPartitionEpoch)) { + + // todo move to partition state from here, as epoch apparently has to be tracked in PSM + if (isPartitionRemovedOrNeverAssigned(sampleRecord)) { + log.debug("Record in buffer for a partition no longer assigned. Dropping. TP: {} rec: {}", toTopicPartition(sampleRecord), sampleRecord); + } else { + //noinspection OptionalGetWithoutIsPresent -- already checked not empty + long batchEndOffset = getLast(recordPollBatch).get().offset(); // NOSONAR + + TopicPartition partition = new TopicPartition(sampleRecord.topic(), sampleRecord.partition()); + getPartitionState(partition).maybeTruncate(batchStartOffset, batchEndOffset); - if (isPartitionRemovedOrNeverAssigned(rec)) { - log.debug("Record in buffer for a partition no longer assigned. Dropping. TP: {} rec: {}", toTopicPartition(rec), rec); + maybeRegisterNewRecordAsWork(epochOfInboundRecords, recordPollBatch); + } + } else { + log.debug("Inbound record of work has epoch ({}) not matching currently assigned epoch for the applicable partition ({}), skipping", + epochOfInboundRecords, currentPartitionEpoch); } + } + } - if (isRecordPreviouslyCompleted(rec)) { - log.trace("Record previously completed, skipping. offset: {}", rec.offset()); + // todo move to partition state + private void maybeRegisterNewRecordAsWork(Long epochOfInboundRecords, List> recordPollBatch) { + for (var aRecord : recordPollBatch) { + if (isRecordPreviouslyCompleted(aRecord)) { + log.trace("Record previously completed, skipping. offset: {}", aRecord.offset()); } else { - var work = new WorkContainer<>(epochOfInboundRecords, rec, module); + //noinspection ObjectAllocationInLoop + var work = new WorkContainer<>(epochOfInboundRecords, aRecord, module); sm.addWorkContainer(work); addWorkContainer(work); } - } else { - log.debug("Inbound record of work has epoch ({}) not matching currently assigned epoch for the applicable partition ({}), skipping", - epochOfInboundRecords, currentPartitionEpoch); } } From 2ea35735eeaad32a59794b1c5b255a38d5d8bb59 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Sat, 1 Oct 2022 20:09:15 +0100 Subject: [PATCH 05/33] incomplete offsets docs --- .../offsets/BitSetEncoder.java | 4 +- .../offsets/OffsetEncoding.java | 6 ++ .../state/PartitionState.java | 68 +++++++++++-------- 3 files changed, 49 insertions(+), 29 deletions(-) diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/offsets/BitSetEncoder.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/offsets/BitSetEncoder.java index 04dcb05dc..47d25850c 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/offsets/BitSetEncoder.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/offsets/BitSetEncoder.java @@ -6,6 +6,7 @@ import io.confluent.csid.utils.StringUtils; import io.confluent.parallelconsumer.internal.InternalRuntimeError; +import io.confluent.parallelconsumer.state.PartitionState; import lombok.Getter; import lombok.extern.slf4j.Slf4j; @@ -27,12 +28,13 @@ * Sequential or not. Because as records are always in commit order, if we've seen a range of offsets, we know we've * seen all that exist (within said range). So if offset 8 is missing from the partition, we will encode it as having * been completed (when in fact it doesn't exist), because we only compare against known incompletes, and assume all - * others are complete. + * others are complete. See {@link PartitionState#incompleteOffsets} for more discussion on this. *

* So, when we deserialize, the INCOMPLETES collection is then restored, and that's what's used to compare to see if a * record should be skipped or not. So if record 8 is recorded as completed, it will be absent from the restored * INCOMPLETES list, and we are assured we will never see record 8. * + * @see PartitionState#incompleteOffsets * @see RunLengthEncoder * @see OffsetBitSet */ diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/offsets/OffsetEncoding.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/offsets/OffsetEncoding.java index 4e31b1b41..f788eea84 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/offsets/OffsetEncoding.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/offsets/OffsetEncoding.java @@ -3,6 +3,7 @@ /*- * Copyright (C) 2020-2022 Confluent, Inc. */ + import lombok.Getter; import lombok.RequiredArgsConstructor; import lombok.ToString; @@ -15,6 +16,11 @@ import static io.confluent.parallelconsumer.offsets.OffsetEncoding.Version.v1; import static io.confluent.parallelconsumer.offsets.OffsetEncoding.Version.v2; +/** + * Offset encoding MagicNumbers to {@link OffsetEncoder}. + * + * @author Antony Stubbs + */ @ToString @RequiredArgsConstructor public enum OffsetEncoding { diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java index 73af0a68c..390fa3516 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java @@ -26,6 +26,8 @@ import static lombok.AccessLevel.*; /** + * Our view of our state of the partitions that we've been assigned. + * * @see PartitionStateManager */ @ToString @@ -42,19 +44,42 @@ public class PartitionState { private final TopicPartition tp; /** - * Offset data beyond the highest committable offset, which haven't totally succeeded. + * Offsets beyond the highest committable offset (see {@link #getOffsetHighestSequentialSucceeded()}) which haven't + * totally succeeded. Based on decoded metadata and polled records (not offset ranges). + *

+ *

+ *

How does this handle gaps in the offsets in the source partitions?:

+ *

+ * We track per record acknowledgement, by only storing the offsets of records OF WHICH WE'VE RECEIVED + * through {@link KafkaConsumer#poll} calls. + *

+ * This is as explicitly opposed to looking at the lowest offset we've polled, and synthetically creating a list of + * EXPECTED offsets from the range from it to the highest polled. If we were to construct this offset range + * synthetically like this, then we would need to expect to process/receive records which might not exist, for + * whatever reason, usually due to compaction. + *

+ * Instead, the offsets tracked are only determined from the records we've given to process from the broker - we + * make no assumptions about which offsets exist. This way we don't have to worry about gaps in the offsets. Also, a + * nice outcome of this is that a gap in the offsets is effectively the same as, as far as we're concerned, an + * offset which has succeeded - because either way we have no action to take. *

* This is independent of the actual queued {@link WorkContainer}s. This is because to start with, data about * incomplete offsets come from the encoded metadata payload that gets committed along with the highest committable - * offset ({@link #getOffsetHighestSequentialSucceeded()}). They are not always in sync. + * offset ({@link #getOffsetHighestSequentialSucceeded()}) and so we don't yet have ConsumerRecord's for those + * offsets until we start polling for them. And so they are not always in sync. *

- * TreeSet so we can always get the lowest offset. + *

+ *

Concurrency:

*

* Needs to be concurrent because, the committer requesting the data to commit may be another thread - the broker * polling sub system - {@link BrokerPollSystem#maybeDoCommit}. The alternative to having this as a concurrent * collection, would be to have the control thread prepare possible commit data on every cycle, and park that data * so that the broker polling thread can grab it, if it wants to commit - i.e. the poller would not prepare/query - * the data for itself. See also #200 Refactor: Consider a shared nothing architecture. + * the data for itself. This requirement is removed in the upcoming PR #200 Refactor: Consider a shared nothing + * architecture. + * + * @see io.confluent.parallelconsumer.offsets.BitSetEncoder for disucssion on how this is impacts per record ack + * storage */ private ConcurrentSkipListSet incompleteOffsets; @@ -222,9 +247,7 @@ private void maybeTruncateBelow(long polledOffset) { long nextExpectedPolledOffset = this.getNextExpectedPolledOffset(); boolean bootstrapRecordAboveExpected = polledOffset > nextExpectedPolledOffset; if (bootstrapRecordAboveExpected) { - log.debug("Truncating state - offsets have been removed form the partition by the broker. Polled {} but expected {} - e.g. record retention expiring, with 'auto.offset.reset'", - polledOffset, - nextExpectedPolledOffset); + log.debug("Truncating state - offsets have been removed form the partition by the broker. Polled {} but expected {} - e.g. record retention expiring, with 'auto.offset.reset'", polledOffset, nextExpectedPolledOffset); NavigableSet truncatedIncompletes = incompleteOffsets.tailSet(polledOffset); ConcurrentSkipListSet wrapped = new ConcurrentSkipListSet<>(truncatedIncompletes); this.incompleteOffsets = wrapped; @@ -237,9 +260,7 @@ public void maybeTruncate(long batchStartOffset, long batchEndOffset) { long nextExpectedPolledOffset = this.getNextExpectedPolledOffset(); boolean bootstrapRecordAboveExpected = batchStartOffset > nextExpectedPolledOffset; if (bootstrapRecordAboveExpected) { - log.debug("Truncating state - offsets have been removed form the partition by the broker. Polled {} but expected {} - e.g. record retention expiring, with 'auto.offset.reset'", - batchStartOffset, - nextExpectedPolledOffset); + log.debug("Truncating state - offsets have been removed form the partition by the broker. Polled {} but expected {} - e.g. record retention expiring, with 'auto.offset.reset'", batchStartOffset, nextExpectedPolledOffset); NavigableSet truncatedIncompletes = incompleteOffsets.tailSet(batchStartOffset); ConcurrentSkipListSet wrapped = new ConcurrentSkipListSet<>(truncatedIncompletes); this.incompleteOffsets = wrapped; @@ -264,19 +285,15 @@ public boolean isRemoved() { } public Optional getCommitDataIfDirty() { - if (isDirty()) - return of(createOffsetAndMetadata()); - else - return empty(); + if (isDirty()) return of(createOffsetAndMetadata()); + else return empty(); } // visible for testing protected OffsetAndMetadata createOffsetAndMetadata() { Optional payloadOpt = tryToEncodeOffsets(); long nextOffset = getNextExpectedInitialPolledOffset(); - return payloadOpt - .map(s -> new OffsetAndMetadata(nextOffset, s)) - .orElseGet(() -> new OffsetAndMetadata(nextOffset)); + return payloadOpt.map(s -> new OffsetAndMetadata(nextOffset, s)).orElseGet(() -> new OffsetAndMetadata(nextOffset)); } /** @@ -294,8 +311,7 @@ protected long getNextExpectedInitialPolledOffset() { */ public Set getAllIncompleteOffsets() { //noinspection FuseStreamOperations - only in java 10 - return Collections.unmodifiableSet(incompleteOffsets.parallelStream() - .collect(Collectors.toSet())); + return Collections.unmodifiableSet(incompleteOffsets.parallelStream().collect(Collectors.toSet())); } /** @@ -306,11 +322,12 @@ public Set getIncompleteOffsetsBelowHighestSucceeded() { //noinspection FuseStreamOperations Collectors.toUnmodifiableSet since v10 return Collections.unmodifiableSet(incompleteOffsets.parallelStream() // todo less than or less than and equal? - .filter(x -> x < highestSucceeded) - .collect(Collectors.toSet())); + .filter(x -> x < highestSucceeded).collect(Collectors.toSet())); } /** + * The offset which is itself, and all before, all successfully completed (or skipped). + *

* Defined for our purpose (as only used in definition of what offset to poll for next), as the offset one below the * lowest incomplete offset. */ @@ -377,16 +394,11 @@ private boolean updateBlockFromEncodingResult(String offsetMapPayload) { // exceeded maximum API allowed, strip the payload mustStrip = true; setAllowedMoreRecords(false); - log.warn("Offset map data too large (size: {}) to fit in metadata payload hard limit of {} - cannot include in commit. " + - "Warning: messages might be replayed on rebalance. " + - "See kafka.coordinator.group.OffsetConfig#DefaultMaxMetadataSize = {} and issue #47.", - metaPayloadLength, DefaultMaxMetadataSize, DefaultMaxMetadataSize); + log.warn("Offset map data too large (size: {}) to fit in metadata payload hard limit of {} - cannot include in commit. " + "Warning: messages might be replayed on rebalance. " + "See kafka.coordinator.group.OffsetConfig#DefaultMaxMetadataSize = {} and issue #47.", metaPayloadLength, DefaultMaxMetadataSize, DefaultMaxMetadataSize); } else if (metaPayloadLength > getPressureThresholdValue()) { // and thus metaPayloadLength <= DefaultMaxMetadataSize // try to turn on back pressure before max size is reached setAllowedMoreRecords(false); - log.warn("Payload size {} higher than threshold {}, but still lower than max {}. Will write payload, but will " + - "not allow further messages, in order to allow the offset data to shrink (via succeeding messages).", - metaPayloadLength, getPressureThresholdValue(), DefaultMaxMetadataSize); + log.warn("Payload size {} higher than threshold {}, but still lower than max {}. Will write payload, but will " + "not allow further messages, in order to allow the offset data to shrink (via succeeding messages).", metaPayloadLength, getPressureThresholdValue(), DefaultMaxMetadataSize); } else { // and thus (metaPayloadLength <= pressureThresholdValue) setAllowedMoreRecords(true); From 835888a10f280c2788c3bd2e002dab4e71f08547 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Sat, 1 Oct 2022 20:23:32 +0100 Subject: [PATCH 06/33] docs and rename --- .../parallelconsumer/state/PartitionState.java | 14 +++++++++----- .../state/PartitionStateManager.java | 8 ++++---- .../state/RemovedPartitionState.java | 2 +- .../WorkManagerOffsetMapCodecManagerTest.java | 2 +- .../parallelconsumer/state/PartitionStateTest.java | 13 +++++++++++-- 5 files changed, 26 insertions(+), 13 deletions(-) diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java index 390fa3516..e84891753 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java @@ -132,15 +132,17 @@ public class PartitionState { private boolean allowedMoreRecords = true; /** - * Map of offsets to WorkUnits. + * Map of offsets to {@link WorkContainer}s. *

* Need to record globally consumed records, to ensure correct offset order committal. Cannot rely on incrementally - * advancing offsets, as this isn't a guarantee of kafka's. + * advancing offsets, as this isn't a guarantee of kafka's (see {@link #incompleteOffsets}). *

* Concurrent because either the broker poller thread or the control thread may be requesting offset to commit - * ({@link #getCommitDataIfDirty()}), or reading upon {@link #onPartitionsRemoved} + * ({@link #getCommitDataIfDirty()}), or reading upon {@link #onPartitionsRemoved}. This requirement is removed in + * the upcoming PR #200 Refactor: Consider a shared nothing * architecture. */ - // todo doesn't need to be concurrent any more? + // todo rename - it's not a queue of things to be committed - it's a collection of incomplete offsets and their WorkContainers + // todo delete? seems this can be replaced by #incompletes - the work container info isn't used @ToString.Exclude private final NavigableMap> commitQueue = new ConcurrentSkipListMap<>(); @@ -225,7 +227,7 @@ private void updateHighestSucceededOffsetSoFar(WorkContainer work) { } } - public void addWorkContainer(WorkContainer wc) { + public void addNewIncompleteWorkContainer(WorkContainer wc) { long newOffset = wc.offset(); // if (noWorkAddedYet) { @@ -236,6 +238,8 @@ public void addWorkContainer(WorkContainer wc) { maybeRaiseHighestSeenOffset(newOffset); commitQueue.put(newOffset, wc); + + // idempotently add the offset to our incompletes track - if it was already there from loading our metadata on startup, there is no affect incompleteOffsets.add(newOffset); } diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java index 488ca6b9c..03c6a25f5 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java @@ -25,7 +25,6 @@ import static io.confluent.csid.utils.JavaUtils.getFirst; import static io.confluent.csid.utils.JavaUtils.getLast; import static io.confluent.csid.utils.KafkaUtils.toTopicPartition; -import static io.confluent.csid.utils.StringUtils.msg; /** * In charge of managing {@link PartitionState}s. @@ -291,9 +290,10 @@ public long getHighestSeenOffset(final TopicPartition tp) { return getPartitionState(tp).getOffsetHighestSeen(); } - public void addWorkContainer(final WorkContainer wc) { + // todo move to partition state + public void addNewIncompleteWorkContainer(final WorkContainer wc) { var tp = wc.getTopicPartition(); - getPartitionState(tp).addWorkContainer(wc); + getPartitionState(tp).addNewIncompleteWorkContainer(wc); } /** @@ -395,7 +395,7 @@ private void maybeRegisterNewRecordAsWork(Long epochOfInboundRecords, List(epochOfInboundRecords, aRecord, module); sm.addWorkContainer(work); - addWorkContainer(work); + addNewIncompleteWorkContainer(work); } } } diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/RemovedPartitionState.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/RemovedPartitionState.java index aef4946f1..e3bc66065 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/RemovedPartitionState.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/RemovedPartitionState.java @@ -55,7 +55,7 @@ public TopicPartition getTp() { } @Override - public void addWorkContainer(final WorkContainer wc) { + public void addNewIncompleteWorkContainer(final WorkContainer wc) { // no-op log.warn("Dropping new work container for partition no longer assigned. WC: {}", wc); } diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/WorkManagerOffsetMapCodecManagerTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/WorkManagerOffsetMapCodecManagerTest.java index 5bdca1c90..5469c6335 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/WorkManagerOffsetMapCodecManagerTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/WorkManagerOffsetMapCodecManagerTest.java @@ -90,7 +90,7 @@ void setupMock() { private void injectSucceededWorkAtOffset(long offset) { WorkContainer workContainer = new WorkContainer<>(0, mockCr, mock(PCModuleTestEnv.class)); Mockito.doReturn(offset).when(mockCr).offset(); - state.addWorkContainer(workContainer); + state.addNewIncompleteWorkContainer(workContainer); state.onSuccess(workContainer); // in this case the highest seen is also the highest succeeded } diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java index 8f43fa215..499e81718 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java @@ -20,13 +20,14 @@ class PartitionStateTest { ModelUtils mu = new ModelUtils(new PCModuleTestEnv()); + TopicPartition tp = new TopicPartition("topic", 0); + /** * @see PartitionState#maybeTruncateBelow */ // parameter test with offsets closer together to check off by one @Test void bootstrapTruncation() { - TopicPartition tp = new TopicPartition("topic", 0); long unexpectedlyHighOffset = 20L; final long previouslyCommittedOffset = 11L; List incompletes = UniLists.of(previouslyCommittedOffset, 15L, unexpectedlyHighOffset, 60L, 80L); @@ -42,7 +43,7 @@ void bootstrapTruncation() { var w20 = mu.createWorkFor(unexpectedlyHighOffset); // bootstrap the first record, triggering truncation - it's offset #unexpectedlyHighOffset, but we were expecting #previouslyCommittedOffset - state.addWorkContainer(w20); + state.addNewIncompleteWorkContainer(w20); Truth.assertThat(state.getNextExpectedInitialPolledOffset()).isEqualTo(unexpectedlyHighOffset); @@ -51,4 +52,12 @@ void bootstrapTruncation() { } + /** + * test for offset gaps in partition data (i.e. compacted topics) + */ + void compactedTopic() { + Truth.assertThat(true).isFalse(); + } + + } \ No newline at end of file From f58c46721edbe50bf0be143cfe43f87ed5a6056d Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Sat, 1 Oct 2022 21:08:22 +0100 Subject: [PATCH 07/33] fixup! review --- .../io/confluent/parallelconsumer/state/PartitionStateTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java index 499e81718..d8165986e 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java @@ -25,7 +25,7 @@ class PartitionStateTest { /** * @see PartitionState#maybeTruncateBelow */ - // parameter test with offsets closer together to check off by one + // todo parameter test with offsets closer together to check off by one @Test void bootstrapTruncation() { long unexpectedlyHighOffset = 20L; From 000969c8ee770d7d5a54481b8003abaa04ba5f0f Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Sat, 1 Oct 2022 21:09:18 +0100 Subject: [PATCH 08/33] Revert "step - continuous checking and restructure" This reverts commit 23c289f4 --- .../io/confluent/csid/utils/JavaUtils.java | 10 +-- .../state/PartitionState.java | 3 +- .../state/PartitionStateManager.java | 63 +++++-------------- 3 files changed, 19 insertions(+), 57 deletions(-) diff --git a/parallel-consumer-core/src/main/java/io/confluent/csid/utils/JavaUtils.java b/parallel-consumer-core/src/main/java/io/confluent/csid/utils/JavaUtils.java index 37e56237d..bba786ed7 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/csid/utils/JavaUtils.java +++ b/parallel-consumer-core/src/main/java/io/confluent/csid/utils/JavaUtils.java @@ -20,13 +20,9 @@ @UtilityClass public class JavaUtils { - public static Optional getLast(final List someList) { - if (someList.isEmpty()) return Optional.empty(); - return Optional.of(someList.get(someList.size() - 1)); - } - - public static Optional getFirst(final List someList) { - return someList.isEmpty() ? Optional.empty() : Optional.of(someList.get(0)); + public static Optional getLast(final List commitHistory) { + if (commitHistory.isEmpty()) return Optional.empty(); + return Optional.of(commitHistory.get(commitHistory.size() - 1)); } public static Optional getOnlyOne(final Map stringMapMap) { diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java index e84891753..9588c6981 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java @@ -233,7 +233,7 @@ public void addNewIncompleteWorkContainer(WorkContainer wc) { // if (noWorkAddedYet) { // noWorkAddedYet = false; // long bootstrapOffset = wc.offset(); -// maybeTruncateBelow(newOffset); + maybeTruncateBelow(newOffset); // } maybeRaiseHighestSeenOffset(newOffset); @@ -429,6 +429,5 @@ public void onPartitionsRemoved(ShardManager sm) { public boolean isBlocked() { return !isAllowedMoreRecords(); } - } diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java index 03c6a25f5..f9f00670c 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java @@ -6,7 +6,6 @@ import io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder; import io.confluent.parallelconsumer.internal.*; -import io.confluent.parallelconsumer.internal.EpochAndRecordsMap.RecordsAndEpoch; import io.confluent.parallelconsumer.offsets.OffsetMapCodecManager; import lombok.Getter; import lombok.NonNull; @@ -22,8 +21,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; -import static io.confluent.csid.utils.JavaUtils.getFirst; -import static io.confluent.csid.utils.JavaUtils.getLast; import static io.confluent.csid.utils.KafkaUtils.toTopicPartition; /** @@ -246,7 +243,6 @@ boolean checkIfWorkIsStale(final WorkContainer workContainer) { return false; } - // todo move to partition state public boolean isRecordPreviouslyCompleted(ConsumerRecord rec) { var tp = toTopicPartition(rec); var partitionState = getPartitionState(tp); @@ -313,7 +309,6 @@ public boolean isBlocked(final TopicPartition topicPartition) { return !isAllowedMoreRecords(topicPartition); } - // todo move to partition state public boolean isPartitionRemovedOrNeverAssigned(ConsumerRecord rec) { TopicPartition topicPartition = toTopicPartition(rec); var partitionState = getPartitionState(topicPartition); @@ -338,12 +333,10 @@ public void onFailure(WorkContainer wc) { void maybeRegisterNewRecordAsWork(final EpochAndRecordsMap recordsMap) { log.debug("Incoming {} new records...", recordsMap.count()); for (var partition : recordsMap.partitions()) { - RecordsAndEpoch recordsList = recordsMap.records(partition); + var recordsList = recordsMap.records(partition); long epochOfInboundRecords = recordsList.getEpochOfPartitionAtPoll(); - List> recordPollBatch = recordsList.getRecords(); - if (!recordPollBatch.isEmpty()) { - // should always not be empty, but... - maybeRegisterNewRecordAsWork(epochOfInboundRecords, recordsList); + for (var rec : recordsList.getRecords()) { + maybeRegisterNewRecordAsWork(epochOfInboundRecords, rec); } } } @@ -351,52 +344,26 @@ void maybeRegisterNewRecordAsWork(final EpochAndRecordsMap recordsMap) { /** * @see #maybeRegisterNewRecordAsWork(EpochAndRecordsMap) */ - // todo move into PartitionState - // todo too deep - private void maybeRegisterNewRecordAsWork(@NonNull Long epochOfInboundRecords, @NonNull RecordsAndEpoch recordsList) { - List> recordPollBatch = recordsList.getRecords(); - - if (!recordPollBatch.isEmpty()) { - final Optional> recOpt = getFirst(recordPollBatch); - //noinspection OptionalGetWithoutIsPresent -- already checked not empty - ConsumerRecord sampleRecord = recOpt.get(); // NOSONAR - long batchStartOffset = sampleRecord.offset(); - - // do epochs still match? do a proactive check, but the epoch will be checked again at work completion as well - var currentPartitionEpoch = getEpochOfPartitionForRecord(sampleRecord); - if (Objects.equals(epochOfInboundRecords, currentPartitionEpoch)) { - - // todo move to partition state from here, as epoch apparently has to be tracked in PSM - if (isPartitionRemovedOrNeverAssigned(sampleRecord)) { - log.debug("Record in buffer for a partition no longer assigned. Dropping. TP: {} rec: {}", toTopicPartition(sampleRecord), sampleRecord); - } else { - //noinspection OptionalGetWithoutIsPresent -- already checked not empty - long batchEndOffset = getLast(recordPollBatch).get().offset(); // NOSONAR + private void maybeRegisterNewRecordAsWork(@NonNull Long epochOfInboundRecords, @NonNull ConsumerRecord rec) { + // do epochs still match? do a proactive check, but the epoch will be checked again at work completion as well + var currentPartitionEpoch = getEpochOfPartitionForRecord(rec); + if (Objects.equals(epochOfInboundRecords, currentPartitionEpoch)) { - TopicPartition partition = new TopicPartition(sampleRecord.topic(), sampleRecord.partition()); - getPartitionState(partition).maybeTruncate(batchStartOffset, batchEndOffset); - - maybeRegisterNewRecordAsWork(epochOfInboundRecords, recordPollBatch); - } - } else { - log.debug("Inbound record of work has epoch ({}) not matching currently assigned epoch for the applicable partition ({}), skipping", - epochOfInboundRecords, currentPartitionEpoch); + if (isPartitionRemovedOrNeverAssigned(rec)) { + log.debug("Record in buffer for a partition no longer assigned. Dropping. TP: {} rec: {}", toTopicPartition(rec), rec); } - } - } - // todo move to partition state - private void maybeRegisterNewRecordAsWork(Long epochOfInboundRecords, List> recordPollBatch) { - for (var aRecord : recordPollBatch) { - if (isRecordPreviouslyCompleted(aRecord)) { - log.trace("Record previously completed, skipping. offset: {}", aRecord.offset()); + if (isRecordPreviouslyCompleted(rec)) { + log.trace("Record previously completed, skipping. offset: {}", rec.offset()); } else { - //noinspection ObjectAllocationInLoop - var work = new WorkContainer<>(epochOfInboundRecords, aRecord, module); + var work = new WorkContainer<>(epochOfInboundRecords, rec, module); sm.addWorkContainer(work); addNewIncompleteWorkContainer(work); } + } else { + log.debug("Inbound record of work has epoch ({}) not matching currently assigned epoch for the applicable partition ({}), skipping", + epochOfInboundRecords, currentPartitionEpoch); } } From b4af22a6d641658e74a2a45d96e8b202b9dfdadb Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Sat, 1 Oct 2022 21:31:18 +0100 Subject: [PATCH 09/33] clean up tangents, clarify --- .../io/confluent/csid/utils/JavaUtils.java | 10 +++- .../state/PartitionState.java | 57 +++++++++++-------- .../state/PartitionStateTest.java | 3 +- 3 files changed, 40 insertions(+), 30 deletions(-) diff --git a/parallel-consumer-core/src/main/java/io/confluent/csid/utils/JavaUtils.java b/parallel-consumer-core/src/main/java/io/confluent/csid/utils/JavaUtils.java index bba786ed7..37e56237d 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/csid/utils/JavaUtils.java +++ b/parallel-consumer-core/src/main/java/io/confluent/csid/utils/JavaUtils.java @@ -20,9 +20,13 @@ @UtilityClass public class JavaUtils { - public static Optional getLast(final List commitHistory) { - if (commitHistory.isEmpty()) return Optional.empty(); - return Optional.of(commitHistory.get(commitHistory.size() - 1)); + public static Optional getLast(final List someList) { + if (someList.isEmpty()) return Optional.empty(); + return Optional.of(someList.get(someList.size() - 1)); + } + + public static Optional getFirst(final List someList) { + return someList.isEmpty() ? Optional.empty() : Optional.of(someList.get(0)); } public static Optional getOnlyOne(final Map stringMapMap) { diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java index 9588c6981..1d81a820c 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java @@ -81,10 +81,10 @@ public class PartitionState { * @see io.confluent.parallelconsumer.offsets.BitSetEncoder for disucssion on how this is impacts per record ack * storage */ - private ConcurrentSkipListSet incompleteOffsets; + private NavigableSet incompleteOffsets; /** - * Marker for the first record to be tracked. Used for some initial analysis. + * Marks whether any {@link WorkContainer}s have been added yet or not. Used for some initial analysis. */ private boolean noWorkAddedYet = true; @@ -140,11 +140,15 @@ public class PartitionState { * Concurrent because either the broker poller thread or the control thread may be requesting offset to commit * ({@link #getCommitDataIfDirty()}), or reading upon {@link #onPartitionsRemoved}. This requirement is removed in * the upcoming PR #200 Refactor: Consider a shared nothing * architecture. + * + * @deprecated the map structure isn't used anymore and can be replaced with the offsets tracked in + * {@link #incompleteOffsets} */ // todo rename - it's not a queue of things to be committed - it's a collection of incomplete offsets and their WorkContainers // todo delete? seems this can be replaced by #incompletes - the work container info isn't used @ToString.Exclude - private final NavigableMap> commitQueue = new ConcurrentSkipListMap<>(); + @Deprecated + private NavigableMap> commitQueue = new ConcurrentSkipListMap<>(); private NavigableMap> getCommitQueue() { return Collections.unmodifiableNavigableMap(commitQueue); @@ -230,11 +234,7 @@ private void updateHighestSucceededOffsetSoFar(WorkContainer work) { public void addNewIncompleteWorkContainer(WorkContainer wc) { long newOffset = wc.offset(); -// if (noWorkAddedYet) { -// noWorkAddedYet = false; -// long bootstrapOffset = wc.offset(); maybeTruncateBelow(newOffset); -// } maybeRaiseHighestSeenOffset(newOffset); commitQueue.put(newOffset, wc); @@ -246,31 +246,38 @@ public void addNewIncompleteWorkContainer(WorkContainer wc) { /** * If the offset is higher than expected, according to the previously committed / polled offset, truncate up to it. * Offsets between have disappeared and will never be polled again. + *

+ * Only runs if this is the first {@link WorkContainer} to be added since instantiation. */ private void maybeTruncateBelow(long polledOffset) { - long nextExpectedPolledOffset = this.getNextExpectedPolledOffset(); - boolean bootstrapRecordAboveExpected = polledOffset > nextExpectedPolledOffset; - if (bootstrapRecordAboveExpected) { - log.debug("Truncating state - offsets have been removed form the partition by the broker. Polled {} but expected {} - e.g. record retention expiring, with 'auto.offset.reset'", polledOffset, nextExpectedPolledOffset); - NavigableSet truncatedIncompletes = incompleteOffsets.tailSet(polledOffset); - ConcurrentSkipListSet wrapped = new ConcurrentSkipListSet<>(truncatedIncompletes); - this.incompleteOffsets = wrapped; + if (noWorkAddedYet) { + noWorkAddedYet = false; + log.trace("Not bootstrap polled records, so not checking for truncation"); + return; } - this.nextExpectedPolledOffset = polledOffset + 1; - } + long expectedBootstrapRecordOffset = this.getNextExpectedPolledOffset(); + + boolean bootstrapPolledRecordAboveExpected = polledOffset > expectedBootstrapRecordOffset; - public void maybeTruncate(long batchStartOffset, long batchEndOffset) { - long nextExpectedPolledOffset = this.getNextExpectedPolledOffset(); - boolean bootstrapRecordAboveExpected = batchStartOffset > nextExpectedPolledOffset; - if (bootstrapRecordAboveExpected) { - log.debug("Truncating state - offsets have been removed form the partition by the broker. Polled {} but expected {} - e.g. record retention expiring, with 'auto.offset.reset'", batchStartOffset, nextExpectedPolledOffset); - NavigableSet truncatedIncompletes = incompleteOffsets.tailSet(batchStartOffset); - ConcurrentSkipListSet wrapped = new ConcurrentSkipListSet<>(truncatedIncompletes); - this.incompleteOffsets = wrapped; + if (bootstrapPolledRecordAboveExpected) { + log.debug("Truncating state - removing records lower than {}. Offsets have been removed form the partition by the broker. Bootstrap polled {} but " + + "expected {} from loaded commit data- e.g. record retention expiring, with 'auto.offset.reset'", + polledOffset, + polledOffset, + expectedBootstrapRecordOffset); + + NavigableSet truncatedIncompletes = incompleteOffsets.tailSet(polledOffset, true); +// ConcurrentSkipListSet wrapped = new ConcurrentSkipListSet<>(truncatedIncompletes); +// this.incompleteOffsets = wrapped; + this.incompleteOffsets = truncatedIncompletes; + + + NavigableMap> truncatedQueue = commitQueue.tailMap(polledOffset, true); + this.commitQueue = truncatedQueue; } - this.nextExpectedPolledOffset = batchEndOffset + 1; + this.nextExpectedPolledOffset = polledOffset + 1; } private long nextExpectedPolledOffset = KAFKA_OFFSET_ABSENCE; diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java index d8165986e..44496726e 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java @@ -53,11 +53,10 @@ void bootstrapTruncation() { } /** - * test for offset gaps in partition data (i.e. compacted topics) + * Test for offset gaps in partition data (i.e. compacted topics) */ void compactedTopic() { Truth.assertThat(true).isFalse(); } - } \ No newline at end of file From 26ee4b8b0e3228a0c05ee2f590752aa447c14978 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Tue, 4 Oct 2022 11:57:26 -0500 Subject: [PATCH 10/33] pruneRemovedTrackedIncompleteOffsets and detect committed offset reset earlier --- .../internal/EpochAndRecordsMap.java | 3 +- .../state/PartitionState.java | 59 ++++++++++++++++--- .../state/PartitionStateManager.java | 15 ++++- 3 files changed, 66 insertions(+), 11 deletions(-) diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecordsMap.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecordsMap.java index 1edfa3ab9..1ad4f6aa0 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecordsMap.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecordsMap.java @@ -27,7 +27,7 @@ public EpochAndRecordsMap(ConsumerRecords poll, PartitionStateManager { var records = poll.records(partition); Long epochOfPartition = pm.getEpochOfPartition(partition); - RecordsAndEpoch entry = new RecordsAndEpoch(epochOfPartition, records); + RecordsAndEpoch entry = new RecordsAndEpoch(partition, epochOfPartition, records); recordMap.put(partition, entry); }); } @@ -63,6 +63,7 @@ public int count() { @Value public class RecordsAndEpoch { + @NonNull TopicPartition topicPartition; @NonNull Long epochOfPartitionAtPoll; @NonNull List> records; } diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java index 1d81a820c..a1c61e2a3 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java @@ -5,6 +5,7 @@ */ import io.confluent.parallelconsumer.internal.BrokerPollSystem; +import io.confluent.parallelconsumer.internal.EpochAndRecordsMap; import io.confluent.parallelconsumer.offsets.NoEncodingPossibleException; import io.confluent.parallelconsumer.offsets.OffsetMapCodecManager; import lombok.Getter; @@ -20,6 +21,8 @@ import java.util.concurrent.ConcurrentSkipListSet; import java.util.stream.Collectors; +import static io.confluent.csid.utils.JavaUtils.getFirst; +import static io.confluent.csid.utils.JavaUtils.getLast; import static io.confluent.parallelconsumer.offsets.OffsetMapCodecManager.DefaultMaxMetadataSize; import static java.util.Optional.empty; import static java.util.Optional.of; @@ -260,21 +263,28 @@ private void maybeTruncateBelow(long polledOffset) { boolean bootstrapPolledRecordAboveExpected = polledOffset > expectedBootstrapRecordOffset; + boolean bootstrapPolledRecordBelowExpected = polledOffset < expectedBootstrapRecordOffset; + if (bootstrapPolledRecordAboveExpected) { + // previously committed offset has been removed, or manual reset to higher offset detected log.debug("Truncating state - removing records lower than {}. Offsets have been removed form the partition by the broker. Bootstrap polled {} but " + "expected {} from loaded commit data- e.g. record retention expiring, with 'auto.offset.reset'", polledOffset, polledOffset, expectedBootstrapRecordOffset); - NavigableSet truncatedIncompletes = incompleteOffsets.tailSet(polledOffset, true); -// ConcurrentSkipListSet wrapped = new ConcurrentSkipListSet<>(truncatedIncompletes); -// this.incompleteOffsets = wrapped; - this.incompleteOffsets = truncatedIncompletes; - + this.incompleteOffsets = incompleteOffsets.tailSet(polledOffset, true); + this.commitQueue = commitQueue.tailMap(polledOffset, true); + } else if (bootstrapPolledRecordBelowExpected) { + // manual reset to lower offset detected + log.debug("CG offset has been reset to an earlier offset ({}) - truncating state - all records inclusively above will be replayed. Expecting {} but bootstrap poll was {}.", + polledOffset, + expectedBootstrapRecordOffset, + polledOffset + ); - NavigableMap> truncatedQueue = commitQueue.tailMap(polledOffset, true); - this.commitQueue = truncatedQueue; + this.incompleteOffsets = new ConcurrentSkipListSet<>(); + this.commitQueue = new ConcurrentSkipListMap<>(); } this.nextExpectedPolledOffset = polledOffset + 1; @@ -436,5 +446,40 @@ public void onPartitionsRemoved(ShardManager sm) { public boolean isBlocked() { return !isAllowedMoreRecords(); } + + /** + * Each time we poll a patch of records, check to see that as expected our tracked incomplete offsets exist in the + * set, otherwise they must have been removed from the underlying partition and should be removed from our tracking + * as we'll ever be given the record again to retry. + */ + @SuppressWarnings("OptionalGetWithoutIsPresent") // checked with isEmpty + protected void pruneRemovedTrackedIncompleteOffsets(EpochAndRecordsMap.RecordsAndEpoch polledRecordBatch) { + var records = polledRecordBatch.getRecords(); + + if (records.isEmpty()) { + log.warn("Polled an emtpy batch of records? {}", polledRecordBatch); + return; + } + + var offsetLookup = records.stream() + .map(ConsumerRecord::offset) + .collect(Collectors.toSet()); + + var low = getFirst(records).get().offset(); // NOSONAR see #isEmpty + var high = getLast(records).get().offset(); // NOSONAR see #isEmpty + + // for the incomplete offsets within this range of poll batch + var subsetFromBatchRange = incompleteOffsets.subSet(low, true, high, true); + + for (long offset : subsetFromBatchRange) { + boolean offsetMissingFromPolledRecords = !offsetLookup.contains(offset); + if (offsetMissingFromPolledRecords) { + // offset has been removed from partition, so remove from tracking as it will never be sent to be retried + boolean removed = incompleteOffsets.remove(offset); + assert removed; + } + } + } + } diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java index f9f00670c..e9e180600 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java @@ -22,6 +22,7 @@ import java.util.stream.Collectors; import static io.confluent.csid.utils.KafkaUtils.toTopicPartition; +import static io.confluent.csid.utils.StringUtils.msg; /** * In charge of managing {@link PartitionState}s. @@ -77,6 +78,10 @@ public PartitionState getPartitionState(TopicPartition tp) { return partitionStates.get(tp); } + private PartitionState getPartitionState(EpochAndRecordsMap.RecordsAndEpoch recordsList) { + return getPartitionState(recordsList.getTopicPartition()); + } + /** * Load offset map for assigned assignedPartitions */ @@ -333,9 +338,13 @@ public void onFailure(WorkContainer wc) { void maybeRegisterNewRecordAsWork(final EpochAndRecordsMap recordsMap) { log.debug("Incoming {} new records...", recordsMap.count()); for (var partition : recordsMap.partitions()) { - var recordsList = recordsMap.records(partition); - long epochOfInboundRecords = recordsList.getEpochOfPartitionAtPoll(); - for (var rec : recordsList.getRecords()) { + var polledRecordBatch = recordsMap.records(partition); + + var partitionState = getPartitionState(polledRecordBatch); + partitionState.pruneRemovedTrackedIncompleteOffsets(polledRecordBatch); + + long epochOfInboundRecords = polledRecordBatch.getEpochOfPartitionAtPoll(); + for (var rec : polledRecordBatch.getRecords()) { maybeRegisterNewRecordAsWork(epochOfInboundRecords, rec); } } From 87dd82e524b7b2a4116768c608d7b7820f9dfd0c Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Tue, 4 Oct 2022 21:58:19 -0500 Subject: [PATCH 11/33] test scenarios --- ...=> PartitionStateCommittedOffsetTest.java} | 22 ++++++++++++++++++- 1 file changed, 21 insertions(+), 1 deletion(-) rename parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/{PartitionStateTest.java => PartitionStateCommittedOffsetTest.java} (77%) diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java similarity index 77% rename from parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java rename to parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java index 44496726e..7ce67500d 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java @@ -16,7 +16,7 @@ import java.util.Optional; import java.util.stream.Collectors; -class PartitionStateTest { +class PartitionStateCommittedOffsetTest { ModelUtils mu = new ModelUtils(new PCModuleTestEnv()); @@ -59,4 +59,24 @@ void compactedTopic() { Truth.assertThat(true).isFalse(); } + /** + * CG offset has been changed to a lower offset (partition rewind / replay) (metdata lost?) + */ + void committedOffsetLower() { + } + + /** + * CG offset has been changed to something higher than expected (offset skip) (metdata lost?) + */ + void committedOffsetHigher() { + } + + /** + * CG offset has disappeared - committed offset hasn't been changed, but broker gives us a bootstrap poll result + * with a higher offset than expected. Could be caused by retention period, or compaction. + */ + void committedOffsetRemoved() { + } + + } \ No newline at end of file From 5c41b3feba9a3cdaaac62faf8f56f870fce8a956 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Wed, 5 Oct 2022 08:30:55 -0500 Subject: [PATCH 12/33] tests --- .../PartitionStateCommittedOffsetIT.java | 52 +++++++++++ .../parallelconsumer/state/ModelUtils.java | 1 + .../PartitionStateCommittedOffsetTest.java | 92 ++++++++++++++++--- .../state/PolledTestBatch.java | 58 ++++++++++++ 4 files changed, 191 insertions(+), 12 deletions(-) create mode 100644 parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java create mode 100644 parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PolledTestBatch.java diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java new file mode 100644 index 000000000..9871a4118 --- /dev/null +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java @@ -0,0 +1,52 @@ +package io.confluent.parallelconsumer.integrationTests.state; + +/*- + * Copyright (C) 2020-2022 Confluent, Inc. + */ + +import io.confluent.parallelconsumer.integrationTests.BrokerIntegrationTest; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.common.TopicPartition; + +class PartitionStateCommittedOffsetIT extends BrokerIntegrationTest { + + AdminClient ac; + + String groupId = getKcu().getConsumer().groupMetadata().groupId(); + + TopicPartition tp = new TopicPartition("topic", 0); + + /** + * Test for offset gaps in partition data (i.e. compacted topics) + */ + void compactedTopic() { + + } + + /** + * CG offset has been changed to a lower offset (partition rewind / replay) (metdata lost?) + */ + void committedOffsetLower() { +// ac.alterConsumerGroupOffsets(groupId, ) + } + + /** + * CG offset has been changed to something higher than expected (offset skip) (metdata lost?) + */ + void committedOffsetHigher() { + } + + /** + * CG offset has disappeared - committed offset hasn't been changed, but broker gives us a bootstrap poll result + * with a higher offset than expected. Could be caused by retention period, or compaction. + */ + void committedOffsetRemoved() { + } + + void cgOffsetsDeleted() { +// ac.deleteConsumerGroupOffsets() + + } + + +} \ No newline at end of file diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/ModelUtils.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/ModelUtils.java index a3ecd025f..36e56af8c 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/ModelUtils.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/ModelUtils.java @@ -24,6 +24,7 @@ @RequiredArgsConstructor public class ModelUtils { + @Getter private final PCModuleTestEnv module; public WorkContainer createWorkFor(long offset) { diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java index 7ce67500d..99c683ce6 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java @@ -5,8 +5,12 @@ */ import com.google.common.truth.Truth; +import io.confluent.csid.utils.Range; +import io.confluent.parallelconsumer.ManagedTruth; import io.confluent.parallelconsumer.internal.PCModuleTestEnv; import io.confluent.parallelconsumer.offsets.OffsetMapCodecManager.HighestOffsetAndIncompletes; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import org.junit.jupiter.api.Test; import pl.tlinkowski.unij.api.UniLists; @@ -18,28 +22,34 @@ class PartitionStateCommittedOffsetTest { + AdminClient ac; + ModelUtils mu = new ModelUtils(new PCModuleTestEnv()); TopicPartition tp = new TopicPartition("topic", 0); + long unexpectedlyHighOffset = 20L; + + final long previouslyCommittedOffset = 11L; + + List incompletes = UniLists.of(previouslyCommittedOffset, 15L, unexpectedlyHighOffset, 60L, 80L); + + List expectedTruncatedIncompletes = incompletes.stream() + .filter(offset -> offset >= unexpectedlyHighOffset) + .collect(Collectors.toList()); + + final long highestSeenOffset = 100L; + + HighestOffsetAndIncompletes offsetData = new HighestOffsetAndIncompletes(Optional.of(highestSeenOffset), new HashSet<>(incompletes)); + /** * @see PartitionState#maybeTruncateBelow */ // todo parameter test with offsets closer together to check off by one @Test void bootstrapTruncation() { - long unexpectedlyHighOffset = 20L; - final long previouslyCommittedOffset = 11L; - List incompletes = UniLists.of(previouslyCommittedOffset, 15L, unexpectedlyHighOffset, 60L, 80L); - List expectedTruncatedIncompletes = incompletes.stream() - .filter(offset -> offset >= unexpectedlyHighOffset) - .collect(Collectors.toList()); - - HighestOffsetAndIncompletes offsetData = new HighestOffsetAndIncompletes(Optional.of(100L), new HashSet<>(incompletes)); // todo fix set/list - PartitionState state = new PartitionState<>(tp, offsetData); - var w20 = mu.createWorkFor(unexpectedlyHighOffset); // bootstrap the first record, triggering truncation - it's offset #unexpectedlyHighOffset, but we were expecting #previouslyCommittedOffset @@ -55,27 +65,85 @@ void bootstrapTruncation() { /** * Test for offset gaps in partition data (i.e. compacted topics) */ + @Test void compactedTopic() { - Truth.assertThat(true).isFalse(); + } /** * CG offset has been changed to a lower offset (partition rewind / replay) (metdata lost?) */ + @Test void committedOffsetLower() { + PartitionState state = new PartitionState<>(tp, offsetData); + + PolledTestBatch polledTestBatch = new PolledTestBatch(mu, tp, highestSeenOffset); + + // +// var psm = new PartitionStateManager(mu.getModule(), mock(ShardManager.class)); +// psm.onass +// psm.maybeRegisterNewRecordAsWork(polledTestBatch.polledRecordBatch); + + // todo when PSM and PartitionState are refactored, these two calls in PS should be a single call + state.pruneRemovedTrackedIncompleteOffsets(polledTestBatch.polledRecordBatch.records(tp)); + for (var wc : polledTestBatch.polledBatchWCs) { + state.addNewIncompleteWorkContainer(wc); + } + + // + OffsetAndMetadata offsetAndMetadata = state.createOffsetAndMetadata(); + + ManagedTruth.assertThat(offsetAndMetadata).getOffset().isEqualTo(0L); + state.getAllIncompleteOffsets().containsAll(Range.range(highestSeenOffset).list()); } /** * CG offset has been changed to something higher than expected (offset skip) (metdata lost?) */ - void committedOffsetHigher() { + @Test + void bootstrapPollOffsetHigher() { + // committed state + PartitionState state = new PartitionState<>(tp, offsetData); + + // bootstrap poll + PolledTestBatch polledTestBatch = new PolledTestBatch(mu, tp, highestSeenOffset); + + // todo when PSM and PartitionState are refactored, these two calls in PS should be a single call + state.pruneRemovedTrackedIncompleteOffsets(polledTestBatch.polledRecordBatch.records(tp)); + for (var wc : polledTestBatch.polledBatchWCs) { + state.addNewIncompleteWorkContainer(wc); + } + + // + OffsetAndMetadata offsetAndMetadata = state.createOffsetAndMetadata(); + + ManagedTruth.assertThat(offsetAndMetadata).getOffset().isEqualTo(0L); + state.getAllIncompleteOffsets().containsAll(Range.range(highestSeenOffset).list()); } /** * CG offset has disappeared - committed offset hasn't been changed, but broker gives us a bootstrap poll result * with a higher offset than expected. Could be caused by retention period, or compaction. */ + @Test void committedOffsetRemoved() { + // committed state + PartitionState state = new PartitionState<>(tp, offsetData); + + // bootstrap poll + PolledTestBatch polledTestBatch = new PolledTestBatch(mu, tp, highestSeenOffset); + + // todo when PSM and PartitionState are refactored, these two calls in PS should be a single call + state.pruneRemovedTrackedIncompleteOffsets(polledTestBatch.polledRecordBatch.records(tp)); + for (var wc : polledTestBatch.polledBatchWCs) { + state.addNewIncompleteWorkContainer(wc); + } + + // + OffsetAndMetadata offsetAndMetadata = state.createOffsetAndMetadata(); + + ManagedTruth.assertThat(offsetAndMetadata).getOffset().isEqualTo(0L); + state.getAllIncompleteOffsets().containsAll(Range.range(highestSeenOffset).list()); } diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PolledTestBatch.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PolledTestBatch.java new file mode 100644 index 000000000..6b37920dd --- /dev/null +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PolledTestBatch.java @@ -0,0 +1,58 @@ +package io.confluent.parallelconsumer.state; + +import io.confluent.csid.utils.Range; +import io.confluent.parallelconsumer.internal.EpochAndRecordsMap; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.common.TopicPartition; +import org.mockito.Mockito; +import pl.tlinkowski.unij.api.UniMaps; + +import java.util.List; +import java.util.stream.Collectors; + +import static org.mockito.Mockito.mock; + +/** + * todo docs + * + * @author Antony Stubbs + */ +public class PolledTestBatch { + + final ModelUtils mu; + + private final long highestSeenOffset; + + private final TopicPartition tp; + + List> polledBatchWCs; + + List> polledBatch; + + EpochAndRecordsMap polledRecordBatch; + + public PolledTestBatch(ModelUtils mu, TopicPartition tp, long toOffset) { + this.mu = mu; + this.tp = tp; + this.highestSeenOffset = toOffset; + + create(tp, toOffset); + } + + void create(TopicPartition tp, long highestSeenOffset) { + this.polledBatchWCs = Range.range(highestSeenOffset).toStream().boxed() + .map(offset -> mu.createWorkFor(offset)) + .collect(Collectors.toList()); + this.polledBatch = polledBatchWCs.stream() + .map(WorkContainer::getCr) + .collect(Collectors.toList()); + + ConsumerRecords consumerRecords = new ConsumerRecords<>(UniMaps.of(tp, polledBatch)); + + PartitionStateManager mock = mock(PartitionStateManager.class); + Mockito.when(mock.getEpochOfPartition(tp)).thenReturn(0L); + this.polledRecordBatch = new EpochAndRecordsMap<>(consumerRecords, mock); + } + +} From 0a13ca692f30bdece9392139c9af780307448bfd Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Wed, 5 Oct 2022 22:47:10 -0500 Subject: [PATCH 13/33] issue #409 captured in test and fixed --- .../ParallelConsumerOptions.java | 2 + .../state/PartitionState.java | 40 ++++++---- .../PartitionStateCommittedOffsetTest.java | 76 +++++++++++-------- .../state/PolledTestBatch.java | 14 ++-- 4 files changed, 79 insertions(+), 53 deletions(-) diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java index 3105a291a..a26e4720e 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java @@ -12,6 +12,7 @@ import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.common.annotation.InterfaceStability; import java.time.Duration; import java.util.Objects; @@ -31,6 +32,7 @@ @Builder(toBuilder = true) @ToString @FieldNameConstants +@InterfaceStability.Stable public class ParallelConsumerOptions { /** diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java index a1c61e2a3..a2f68f0da 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java @@ -89,7 +89,8 @@ public class PartitionState { /** * Marks whether any {@link WorkContainer}s have been added yet or not. Used for some initial analysis. */ - private boolean noWorkAddedYet = true; + // todo rename + private boolean bootstrapPhase = true; /** * Cache view of the state of the partition. Is set dirty when the incomplete state of any offset changes. Is set @@ -113,6 +114,9 @@ public class PartitionState { *

* Note that this may in some conditions, there may be a gap between this and the next offset to poll - that being, * there may be some number of transaction marker records above it, and the next offset to poll. + *

+ * Note that as we only encode our offset map up to the highest succeeded offset (as encoding higher has no value), + * upon bootstrap, this will always start off as the same as the {@link #offsetHighestSeen}. */ @Getter(PUBLIC) private long offsetHighestSucceeded = KAFKA_OFFSET_ABSENCE; @@ -161,7 +165,7 @@ public PartitionState(TopicPartition tp, OffsetMapCodecManager.HighestOffsetAndI this.tp = tp; this.offsetHighestSeen = offsetData.getHighestSeenOffset().orElse(KAFKA_OFFSET_ABSENCE); this.incompleteOffsets = new ConcurrentSkipListSet<>(offsetData.getIncompleteOffsets()); - this.offsetHighestSucceeded = this.offsetHighestSeen; + this.offsetHighestSucceeded = this.offsetHighestSeen; // by definition, as we only encode up to the highest seen offset (inclusive) this.nextExpectedPolledOffset = getNextExpectedInitialPolledOffset(); } @@ -253,22 +257,25 @@ public void addNewIncompleteWorkContainer(WorkContainer wc) { * Only runs if this is the first {@link WorkContainer} to be added since instantiation. */ private void maybeTruncateBelow(long polledOffset) { - if (noWorkAddedYet) { - noWorkAddedYet = false; + if (!bootstrapPhase) { log.trace("Not bootstrap polled records, so not checking for truncation"); return; + } else { + bootstrapPhase = false; } - long expectedBootstrapRecordOffset = this.getNextExpectedPolledOffset(); +// long expectedBootstrapRecordOffset = this.getNextExpectedPolledOffset(); + long expectedBootstrapRecordOffset = getNextExpectedInitialPolledOffset(); + boolean bootstrapPolledRecordAboveExpected = polledOffset > expectedBootstrapRecordOffset; boolean bootstrapPolledRecordBelowExpected = polledOffset < expectedBootstrapRecordOffset; if (bootstrapPolledRecordAboveExpected) { - // previously committed offset has been removed, or manual reset to higher offset detected - log.debug("Truncating state - removing records lower than {}. Offsets have been removed form the partition by the broker. Bootstrap polled {} but " + - "expected {} from loaded commit data- e.g. record retention expiring, with 'auto.offset.reset'", + // previously committed offset record has been removed, or manual reset to higher offset detected + log.warn("Truncating state - removing records lower than {}. Offsets have been removed form the partition by the broker. Bootstrap polled {} but " + + "expected {} from loaded commit data. Could be caused by record retention or compaction.", polledOffset, polledOffset, expectedBootstrapRecordOffset); @@ -277,7 +284,7 @@ private void maybeTruncateBelow(long polledOffset) { this.commitQueue = commitQueue.tailMap(polledOffset, true); } else if (bootstrapPolledRecordBelowExpected) { // manual reset to lower offset detected - log.debug("CG offset has been reset to an earlier offset ({}) - truncating state - all records inclusively above will be replayed. Expecting {} but bootstrap poll was {}.", + log.warn("CG offset has been reset to an earlier offset ({}) - truncating state - all records inclusively above will be replayed. Expecting {} but bootstrap poll was {}.", polledOffset, expectedBootstrapRecordOffset, polledOffset @@ -323,7 +330,8 @@ protected OffsetAndMetadata createOffsetAndMetadata() { * Defines as the offset one below the highest sequentially succeeded offset. */ // visible for testing - protected long getNextExpectedInitialPolledOffset() { + // todo change back to protected? and enable protected level managed truth (seems to be limited to public) + public long getNextExpectedInitialPolledOffset() { return getOffsetHighestSequentialSucceeded() + 1; } @@ -338,6 +346,7 @@ public Set getAllIncompleteOffsets() { /** * @return incomplete offsets which are lower than the highest succeeded */ + // todo change from Set to List (order) public Set getIncompleteOffsetsBelowHighestSucceeded() { long highestSucceeded = getOffsetHighestSucceeded(); //noinspection FuseStreamOperations Collectors.toUnmodifiableSet since v10 @@ -474,9 +483,14 @@ protected void pruneRemovedTrackedIncompleteOffsets(EpochAndRecordsMap.Rec for (long offset : subsetFromBatchRange) { boolean offsetMissingFromPolledRecords = !offsetLookup.contains(offset); if (offsetMissingFromPolledRecords) { - // offset has been removed from partition, so remove from tracking as it will never be sent to be retried - boolean removed = incompleteOffsets.remove(offset); - assert removed; + log.warn("Offset {} has been removed from partition {} (as it's not been returned from a poll within a bounding batch), so it must be removed from tracking state, as it will never be sent again to be retried.", + offset, + getTp() + ); + boolean removed1 = incompleteOffsets.remove(offset); + assert removed1; + boolean removed2 = commitQueue.remove(offset) != null; + assert removed2; } } } diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java index 99c683ce6..369b777b8 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java @@ -6,7 +6,6 @@ import com.google.common.truth.Truth; import io.confluent.csid.utils.Range; -import io.confluent.parallelconsumer.ManagedTruth; import io.confluent.parallelconsumer.internal.PCModuleTestEnv; import io.confluent.parallelconsumer.offsets.OffsetMapCodecManager.HighestOffsetAndIncompletes; import org.apache.kafka.clients.admin.AdminClient; @@ -20,6 +19,8 @@ import java.util.Optional; import java.util.stream.Collectors; +import static io.confluent.parallelconsumer.ManagedTruth.assertThat; + class PartitionStateCommittedOffsetTest { AdminClient ac; @@ -30,16 +31,21 @@ class PartitionStateCommittedOffsetTest { long unexpectedlyHighOffset = 20L; - final long previouslyCommittedOffset = 11L; + long previouslyCommittedOffset = 11L; + + final long highestSeenOffset = 101L; - List incompletes = UniLists.of(previouslyCommittedOffset, 15L, unexpectedlyHighOffset, 60L, 80L); + /** + * @see PartitionState#offsetHighestSucceeded + */ + long highestSucceeded = highestSeenOffset; + + List incompletes = UniLists.of(previouslyCommittedOffset, 15L, unexpectedlyHighOffset, 60L, 80L, 95L, 96L, 97L, 98L, 100L); List expectedTruncatedIncompletes = incompletes.stream() .filter(offset -> offset >= unexpectedlyHighOffset) .collect(Collectors.toList()); - final long highestSeenOffset = 100L; - HighestOffsetAndIncompletes offsetData = new HighestOffsetAndIncompletes(Optional.of(highestSeenOffset), new HashSet<>(incompletes)); /** @@ -47,15 +53,13 @@ class PartitionStateCommittedOffsetTest { */ // todo parameter test with offsets closer together to check off by one @Test - void bootstrapTruncation() { + void bootstrapPollOffsetHigherDueToRentention() { PartitionState state = new PartitionState<>(tp, offsetData); - var w20 = mu.createWorkFor(unexpectedlyHighOffset); - // bootstrap the first record, triggering truncation - it's offset #unexpectedlyHighOffset, but we were expecting #previouslyCommittedOffset - state.addNewIncompleteWorkContainer(w20); - + addPollToState(state, new PolledTestBatch(mu, tp, unexpectedlyHighOffset, highestSeenOffset)); + // Truth.assertThat(state.getNextExpectedInitialPolledOffset()).isEqualTo(unexpectedlyHighOffset); Truth.assertThat(state.getIncompleteOffsetsBelowHighestSucceeded()).containsExactlyElementsIn(expectedTruncatedIncompletes); @@ -77,73 +81,79 @@ void compactedTopic() { void committedOffsetLower() { PartitionState state = new PartitionState<>(tp, offsetData); - PolledTestBatch polledTestBatch = new PolledTestBatch(mu, tp, highestSeenOffset); + long unexpectedLowerOffset = previouslyCommittedOffset - 5L; + + PolledTestBatch polledTestBatch = new PolledTestBatch(mu, tp, unexpectedLowerOffset, highestSeenOffset); // // var psm = new PartitionStateManager(mu.getModule(), mock(ShardManager.class)); // psm.onass // psm.maybeRegisterNewRecordAsWork(polledTestBatch.polledRecordBatch); - // todo when PSM and PartitionState are refactored, these two calls in PS should be a single call - state.pruneRemovedTrackedIncompleteOffsets(polledTestBatch.polledRecordBatch.records(tp)); - for (var wc : polledTestBatch.polledBatchWCs) { - state.addNewIncompleteWorkContainer(wc); - } + addPollToState(state, polledTestBatch); // OffsetAndMetadata offsetAndMetadata = state.createOffsetAndMetadata(); - ManagedTruth.assertThat(offsetAndMetadata).getOffset().isEqualTo(0L); + assertThat(offsetAndMetadata).getOffset().isEqualTo(0L); state.getAllIncompleteOffsets().containsAll(Range.range(highestSeenOffset).list()); } + private void addPollToState(PartitionState state, PolledTestBatch polledTestBatch) { + // todo when PSM and PartitionState are refactored, these two calls in PS should be a single call + state.pruneRemovedTrackedIncompleteOffsets(polledTestBatch.polledRecordBatch.records(tp)); + for (var wc : polledTestBatch.polledBatchWCs) { + // todo when PSM and PartitionState are refactored, this conditional should not be needed + if (!state.isRecordPreviouslyCompleted(wc.getCr())) { + state.addNewIncompleteWorkContainer(wc); + } + } + } + /** - * CG offset has been changed to something higher than expected (offset skip) (metdata lost?) + * CG offset has been changed to something higher than expected (offset skip) (metadata lost?) */ @Test - void bootstrapPollOffsetHigher() { + void bootstrapPollOffsetHigherViaManualCGRset() { // committed state PartitionState state = new PartitionState<>(tp, offsetData); // bootstrap poll - PolledTestBatch polledTestBatch = new PolledTestBatch(mu, tp, highestSeenOffset); + PolledTestBatch polledTestBatch = new PolledTestBatch(mu, tp, unexpectedlyHighOffset, highestSeenOffset); // todo when PSM and PartitionState are refactored, these two calls in PS should be a single call - state.pruneRemovedTrackedIncompleteOffsets(polledTestBatch.polledRecordBatch.records(tp)); - for (var wc : polledTestBatch.polledBatchWCs) { - state.addNewIncompleteWorkContainer(wc); - } + addPollToState(state, polledTestBatch); // OffsetAndMetadata offsetAndMetadata = state.createOffsetAndMetadata(); - ManagedTruth.assertThat(offsetAndMetadata).getOffset().isEqualTo(0L); + assertThat(offsetAndMetadata).getOffset().isEqualTo(0L); state.getAllIncompleteOffsets().containsAll(Range.range(highestSeenOffset).list()); } /** * CG offset has disappeared - committed offset hasn't been changed, but broker gives us a bootstrap poll result * with a higher offset than expected. Could be caused by retention period, or compaction. + * + * @implSpec issue #409: Committing old offset after OFFSET_OUT_OF_RANGE */ @Test - void committedOffsetRemoved() { + void bootstrapPollOffsetHigherDueToRetentionOrCompaction() { // committed state PartitionState state = new PartitionState<>(tp, offsetData); // bootstrap poll - PolledTestBatch polledTestBatch = new PolledTestBatch(mu, tp, highestSeenOffset); + PolledTestBatch polledTestBatch = new PolledTestBatch(mu, tp, unexpectedlyHighOffset, highestSeenOffset); // todo when PSM and PartitionState are refactored, these two calls in PS should be a single call - state.pruneRemovedTrackedIncompleteOffsets(polledTestBatch.polledRecordBatch.records(tp)); - for (var wc : polledTestBatch.polledBatchWCs) { - state.addNewIncompleteWorkContainer(wc); - } + addPollToState(state, polledTestBatch); // + assertThat(state).getNextExpectedInitialPolledOffset().isEqualTo(unexpectedlyHighOffset); OffsetAndMetadata offsetAndMetadata = state.createOffsetAndMetadata(); - ManagedTruth.assertThat(offsetAndMetadata).getOffset().isEqualTo(0L); - state.getAllIncompleteOffsets().containsAll(Range.range(highestSeenOffset).list()); + assertThat(offsetAndMetadata).getOffset().isEqualTo(unexpectedlyHighOffset); + assertThat(state).getAllIncompleteOffsets().containsExactlyElementsIn(expectedTruncatedIncompletes); } diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PolledTestBatch.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PolledTestBatch.java index 6b37920dd..44ddfd64f 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PolledTestBatch.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PolledTestBatch.java @@ -1,7 +1,7 @@ package io.confluent.parallelconsumer.state; -import io.confluent.csid.utils.Range; import io.confluent.parallelconsumer.internal.EpochAndRecordsMap; +import one.util.streamex.LongStreamEx; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.common.TopicPartition; @@ -32,18 +32,18 @@ public class PolledTestBatch { EpochAndRecordsMap polledRecordBatch; - public PolledTestBatch(ModelUtils mu, TopicPartition tp, long toOffset) { + public PolledTestBatch(ModelUtils mu, TopicPartition tp, long fromOffset, long toOffset) { this.mu = mu; this.tp = tp; this.highestSeenOffset = toOffset; - create(tp, toOffset); + create(tp, fromOffset, toOffset); } - void create(TopicPartition tp, long highestSeenOffset) { - this.polledBatchWCs = Range.range(highestSeenOffset).toStream().boxed() - .map(offset -> mu.createWorkFor(offset)) - .collect(Collectors.toList()); + void create(TopicPartition tp, long fromOffset, long highestSeenOffset) { + this.polledBatchWCs = LongStreamEx.range(fromOffset, highestSeenOffset + 1).boxed() + .map(mu::createWorkFor) + .toList(); this.polledBatch = polledBatchWCs.stream() .map(WorkContainer::getCr) .collect(Collectors.toList()); From d713b299f69b1a9b15c068f3fa615a52a830ccaf Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Wed, 5 Oct 2022 23:47:31 -0500 Subject: [PATCH 14/33] three scenarios captured and fixed --- .../state/PartitionState.java | 50 +++++--- .../PartitionStateCommittedOffsetTest.java | 112 ++++++++++-------- .../state/PolledTestBatch.java | 24 +++- 3 files changed, 118 insertions(+), 68 deletions(-) diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java index a2f68f0da..8231b76c5 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java @@ -15,6 +15,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; +import pl.tlinkowski.unij.api.UniSets; import java.util.*; import java.util.concurrent.ConcurrentSkipListMap; @@ -163,6 +164,11 @@ private NavigableMap> getCommitQueue() { public PartitionState(TopicPartition tp, OffsetMapCodecManager.HighestOffsetAndIncompletes offsetData) { this.tp = tp; + + initState(offsetData); + } + + private void initState(OffsetMapCodecManager.HighestOffsetAndIncompletes offsetData) { this.offsetHighestSeen = offsetData.getHighestSeenOffset().orElse(KAFKA_OFFSET_ABSENCE); this.incompleteOffsets = new ConcurrentSkipListSet<>(offsetData.getIncompleteOffsets()); this.offsetHighestSucceeded = this.offsetHighestSeen; // by definition, as we only encode up to the highest seen offset (inclusive) @@ -241,7 +247,7 @@ private void updateHighestSucceededOffsetSoFar(WorkContainer work) { public void addNewIncompleteWorkContainer(WorkContainer wc) { long newOffset = wc.offset(); - maybeTruncateBelow(newOffset); +// maybeTruncateBelowOrAbove(newOffset); maybeRaiseHighestSeenOffset(newOffset); commitQueue.put(newOffset, wc); @@ -256,12 +262,12 @@ public void addNewIncompleteWorkContainer(WorkContainer wc) { *

* Only runs if this is the first {@link WorkContainer} to be added since instantiation. */ - private void maybeTruncateBelow(long polledOffset) { - if (!bootstrapPhase) { - log.trace("Not bootstrap polled records, so not checking for truncation"); - return; - } else { + private void maybeTruncateBelowOrAbove(long polledOffset) { + if (bootstrapPhase) { bootstrapPhase = false; + } else { + // Not bootstrap phase anymore, so not checking for truncation + return; } // long expectedBootstrapRecordOffset = this.getNextExpectedPolledOffset(); @@ -284,19 +290,22 @@ private void maybeTruncateBelow(long polledOffset) { this.commitQueue = commitQueue.tailMap(polledOffset, true); } else if (bootstrapPolledRecordBelowExpected) { // manual reset to lower offset detected - log.warn("CG offset has been reset to an earlier offset ({}) - truncating state - all records inclusively above will be replayed. Expecting {} but bootstrap poll was {}.", + log.warn("Bootstrap polled offset has been reset to an earlier offset ({}) - truncating state - all records above including this will be replayed. Was expecting {} but bootstrap poll was {}.", polledOffset, expectedBootstrapRecordOffset, polledOffset ); - - this.incompleteOffsets = new ConcurrentSkipListSet<>(); - this.commitQueue = new ConcurrentSkipListMap<>(); + // reset state to the polled offset + // todo option to save incompletes data and only replay offsets between polled through to expected? (just replay the offsets of which we've moved backwards by) + initState(new OffsetMapCodecManager.HighestOffsetAndIncompletes(Optional.of(polledOffset - 1), UniSets.of())); +// this.incompleteOffsets = new ConcurrentSkipListSet<>(); +// this.commitQueue = new ConcurrentSkipListMap<>(); } this.nextExpectedPolledOffset = polledOffset + 1; } + // todo delete private long nextExpectedPolledOffset = KAFKA_OFFSET_ABSENCE; private long getNextExpectedPolledOffset() { @@ -470,27 +479,34 @@ protected void pruneRemovedTrackedIncompleteOffsets(EpochAndRecordsMap.Rec return; } - var offsetLookup = records.stream() + var low = getFirst(records).get().offset(); // NOSONAR see #isEmpty + + maybeTruncateBelowOrAbove(low); + + // build the hash set once, so we can do random access checks of our tracked incompletes + var polledOffsetLookup = records.stream() .map(ConsumerRecord::offset) .collect(Collectors.toSet()); - var low = getFirst(records).get().offset(); // NOSONAR see #isEmpty var high = getLast(records).get().offset(); // NOSONAR see #isEmpty // for the incomplete offsets within this range of poll batch var subsetFromBatchRange = incompleteOffsets.subSet(low, true, high, true); for (long offset : subsetFromBatchRange) { - boolean offsetMissingFromPolledRecords = !offsetLookup.contains(offset); + boolean offsetMissingFromPolledRecords = !polledOffsetLookup.contains(offset); if (offsetMissingFromPolledRecords) { - log.warn("Offset {} has been removed from partition {} (as it's not been returned from a poll within a bounding batch), so it must be removed from tracking state, as it will never be sent again to be retried.", + log.warn("Offset {} has been removed from partition {} (as it has not been returned within a polled batch which should have contained it - batch offset range is {} to {}), so it must be removed from tracking state, as it will never be sent again to be retried. This can be caused by PC rebalancing across a partition which has been compacted on offsets above the committed base offset, after initial load and before a rebalance.", offset, - getTp() + getTp(), + low, + high ); boolean removed1 = incompleteOffsets.remove(offset); assert removed1; - boolean removed2 = commitQueue.remove(offset) != null; - assert removed2; + // can't remove it from a collection wit would have never been added to +// boolean removed2 = commitQueue.remove(offset) != null; +// assert removed2; } } } diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java index 369b777b8..6d5045188 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java @@ -4,23 +4,29 @@ * Copyright (C) 2020-2022 Confluent, Inc. */ -import com.google.common.truth.Truth; -import io.confluent.csid.utils.Range; import io.confluent.parallelconsumer.internal.PCModuleTestEnv; import io.confluent.parallelconsumer.offsets.OffsetMapCodecManager.HighestOffsetAndIncompletes; +import one.util.streamex.LongStreamEx; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import org.junit.jupiter.api.Test; import pl.tlinkowski.unij.api.UniLists; +import pl.tlinkowski.unij.api.UniSets; import java.util.HashSet; import java.util.List; import java.util.Optional; +import java.util.Set; import java.util.stream.Collectors; import static io.confluent.parallelconsumer.ManagedTruth.assertThat; +/** + * @author Antony Stubbs + * @see PartitionState#maybeTruncateBelow + * @see PartitionState#pruneRemovedTrackedIncompleteOffsets + */ class PartitionStateCommittedOffsetTest { AdminClient ac; @@ -48,55 +54,59 @@ class PartitionStateCommittedOffsetTest { HighestOffsetAndIncompletes offsetData = new HighestOffsetAndIncompletes(Optional.of(highestSeenOffset), new HashSet<>(incompletes)); + PartitionState state = new PartitionState<>(tp, offsetData); + /** - * @see PartitionState#maybeTruncateBelow + * Test for offset gaps in partition data (i.e. compacted topics) */ - // todo parameter test with offsets closer together to check off by one @Test - void bootstrapPollOffsetHigherDueToRentention() { - PartitionState state = new PartitionState<>(tp, offsetData); + void compactedTopic() { + Set compacted = UniSets.of(80L, 95L, 97L); + long slightlyLowerRange = highestSeenOffset - 2L; // to check subsets don't mess with incompletes not represented in this polled batch + List polledOffsetsWithCompactedRemoved = LongStreamEx.range(previouslyCommittedOffset, slightlyLowerRange) + .filter(offset -> !compacted.contains(offset)) + .boxed().toList(); - // bootstrap the first record, triggering truncation - it's offset #unexpectedlyHighOffset, but we were expecting #previouslyCommittedOffset - addPollToState(state, new PolledTestBatch(mu, tp, unexpectedlyHighOffset, highestSeenOffset)); + // + PolledTestBatch polledTestBatch = new PolledTestBatch(mu, tp, polledOffsetsWithCompactedRemoved); // - Truth.assertThat(state.getNextExpectedInitialPolledOffset()).isEqualTo(unexpectedlyHighOffset); + addPollToState(state, polledTestBatch); - Truth.assertThat(state.getIncompleteOffsetsBelowHighestSucceeded()).containsExactlyElementsIn(expectedTruncatedIncompletes); + // + OffsetAndMetadata offsetAndMetadata = state.createOffsetAndMetadata(); - } + assertThat(offsetAndMetadata).getOffset().isEqualTo(previouslyCommittedOffset); - /** - * Test for offset gaps in partition data (i.e. compacted topics) - */ - @Test - void compactedTopic() { + var compactedIncompletes = incompletes.stream().filter(offset -> !compacted.contains(offset)).collect(Collectors.toList()); + assertThat(state).getAllIncompleteOffsets().containsExactlyElementsIn(compactedIncompletes); + // check still contains 100,101 } /** - * CG offset has been changed to a lower offset (partition rewind / replay) (metdata lost?) + * CG offset has been changed to a lower offset (partition rewind / replay). + *

+ * Metadata could be lost if it's a manual reset, otherwise it will still exist. If it's been lost, then we will + * bootstrap the partition as though it's the first time it's ever been seen, so nothing to do. + *

+ * If the offset and metadata is still there, then we have to handle the situation. */ @Test void committedOffsetLower() { - PartitionState state = new PartitionState<>(tp, offsetData); - - long unexpectedLowerOffset = previouslyCommittedOffset - 5L; + long randomlyChosenStepBackwards = 5L; + long unexpectedLowerOffset = previouslyCommittedOffset - randomlyChosenStepBackwards; PolledTestBatch polledTestBatch = new PolledTestBatch(mu, tp, unexpectedLowerOffset, highestSeenOffset); // -// var psm = new PartitionStateManager(mu.getModule(), mock(ShardManager.class)); -// psm.onass -// psm.maybeRegisterNewRecordAsWork(polledTestBatch.polledRecordBatch); - addPollToState(state, polledTestBatch); // OffsetAndMetadata offsetAndMetadata = state.createOffsetAndMetadata(); - assertThat(offsetAndMetadata).getOffset().isEqualTo(0L); - state.getAllIncompleteOffsets().containsAll(Range.range(highestSeenOffset).list()); + assertThat(offsetAndMetadata).getOffset().isEqualTo(unexpectedLowerOffset); + assertThat(state).getAllIncompleteOffsets().containsExactlyElementsIn(LongStreamEx.range(unexpectedLowerOffset, highestSeenOffset + 1).boxed().toList()); } private void addPollToState(PartitionState state, PolledTestBatch polledTestBatch) { @@ -104,38 +114,44 @@ private void addPollToState(PartitionState state, PolledTestBatc state.pruneRemovedTrackedIncompleteOffsets(polledTestBatch.polledRecordBatch.records(tp)); for (var wc : polledTestBatch.polledBatchWCs) { // todo when PSM and PartitionState are refactored, this conditional should not be needed - if (!state.isRecordPreviouslyCompleted(wc.getCr())) { + var offset = wc.offset(); + final boolean notPreviouslyCompleted = !state.isRecordPreviouslyCompleted(wc.getCr()); + if (notPreviouslyCompleted) { state.addNewIncompleteWorkContainer(wc); } } } - - /** - * CG offset has been changed to something higher than expected (offset skip) (metadata lost?) - */ - @Test - void bootstrapPollOffsetHigherViaManualCGRset() { - // committed state - PartitionState state = new PartitionState<>(tp, offsetData); - - // bootstrap poll - PolledTestBatch polledTestBatch = new PolledTestBatch(mu, tp, unexpectedlyHighOffset, highestSeenOffset); - - // todo when PSM and PartitionState are refactored, these two calls in PS should be a single call - addPollToState(state, polledTestBatch); - - // - OffsetAndMetadata offsetAndMetadata = state.createOffsetAndMetadata(); - - assertThat(offsetAndMetadata).getOffset().isEqualTo(0L); - state.getAllIncompleteOffsets().containsAll(Range.range(highestSeenOffset).list()); - } +// +// /** +// * +// */ +// @Test +// void bootstrapPollOffsetHigherViaManualCGRset() { +// // committed state +// PartitionState state = new PartitionState<>(tp, offsetData); +// +// // bootstrap poll +// PolledTestBatch polledTestBatch = new PolledTestBatch(mu, tp, unexpectedlyHighOffset, highestSeenOffset); +// +// // todo when PSM and PartitionState are refactored, these two calls in PS should be a single call +// addPollToState(state, polledTestBatch); +// +// // +// OffsetAndMetadata offsetAndMetadata = state.createOffsetAndMetadata(); +// +// assertThat(offsetAndMetadata).getOffset().isEqualTo(0L); +// state.getAllIncompleteOffsets().containsAll(Range.range(highestSeenOffset).list()); +// } /** * CG offset has disappeared - committed offset hasn't been changed, but broker gives us a bootstrap poll result * with a higher offset than expected. Could be caused by retention period, or compaction. + *

+ * If the CG offset has been changed to something higher than expected manually, then we will bootstrap the + * partition as though it's never been seen before, so nothing to do. * * @implSpec issue #409: Committing old offset after OFFSET_OUT_OF_RANGE + * @see PartitionState#maybeTruncateBelow */ @Test void bootstrapPollOffsetHigherDueToRetentionOrCompaction() { diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PolledTestBatch.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PolledTestBatch.java index 44ddfd64f..2749a4a1f 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PolledTestBatch.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PolledTestBatch.java @@ -1,7 +1,9 @@ package io.confluent.parallelconsumer.state; +import io.confluent.csid.utils.JavaUtils; import io.confluent.parallelconsumer.internal.EpochAndRecordsMap; import one.util.streamex.LongStreamEx; +import one.util.streamex.StreamEx; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.common.TopicPartition; @@ -37,11 +39,27 @@ public PolledTestBatch(ModelUtils mu, TopicPartition tp, long fromOffset, long t this.tp = tp; this.highestSeenOffset = toOffset; - create(tp, fromOffset, toOffset); + create(fromOffset, toOffset); } - void create(TopicPartition tp, long fromOffset, long highestSeenOffset) { - this.polledBatchWCs = LongStreamEx.range(fromOffset, highestSeenOffset + 1).boxed() + public PolledTestBatch(ModelUtils mu, TopicPartition tp, List polledOffsetsWithCompactedRemoved) { + this.mu = mu; + this.tp = tp; + //noinspection OptionalGetWithoutIsPresent + this.highestSeenOffset = JavaUtils.getLast(polledOffsetsWithCompactedRemoved).get(); + + create(polledOffsetsWithCompactedRemoved); + + } + + void create(long fromOffset, long highestSeenOffset) { + List offsets = LongStreamEx.range(fromOffset, highestSeenOffset + 1).boxed().toList(); + create(offsets); + } + + void create(List offsets) { + var offsetStream = StreamEx.of(offsets); + this.polledBatchWCs = offsetStream .map(mu::createWorkFor) .toList(); this.polledBatch = polledBatchWCs.stream() From 3336746fd4f6657368430a94b9705f7720bc55db Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Thu, 6 Oct 2022 13:06:03 +0100 Subject: [PATCH 15/33] SAVE: fixing tests --- .../offsets/OffsetEncodingTests.java | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingTests.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingTests.java index 56aa02e8e..6cbed585e 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingTests.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingTests.java @@ -31,6 +31,7 @@ import static io.confluent.parallelconsumer.ManagedTruth.assertTruth; import static io.confluent.parallelconsumer.offsets.OffsetEncoding.*; +import static io.confluent.parallelconsumer.state.PartitionState.KAFKA_OFFSET_ABSENCE; import static org.assertj.core.api.Assertions.assertThat; import static org.hamcrest.Matchers.in; import static org.hamcrest.Matchers.not; @@ -226,17 +227,19 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset { var newWm = new WorkManager<>(new PCModule<>(options)); newWm.onPartitionsAssigned(UniSets.of(tp)); - newWm.registerWork(new EpochAndRecordsMap(testRecords, newWm.getPm())); + newWm.registerWork(new EpochAndRecordsMap<>(testRecords, newWm.getPm())); var pm = newWm.getPm(); var partitionState = pm.getPartitionState(tp); if (assumeWorkingCodec(encoding, encodingsThatFail)) { - long offsetHighestSequentialSucceeded = partitionState.getOffsetHighestSequentialSucceeded(); - assertThat(offsetHighestSequentialSucceeded).isEqualTo(0); +// long offsetHighestSequentialSucceeded = partitionState.getOffsetHighestSequentialSucceeded(); +// assertThat(offsetHighestSequentialSucceeded).isEqualTo(0); + assertTruth(partitionState).getOffsetHighestSequentialSucceeded().isEqualTo(KAFKA_OFFSET_ABSENCE); - long offsetHighestSucceeded = partitionState.getOffsetHighestSucceeded(); - assertThat(offsetHighestSucceeded).isEqualTo(highest); +// long offsetHighestSucceeded = partitionState.getOffsetHighestSucceeded(); +// assertThat(offsetHighestSucceeded).isEqualTo(highest); + assertTruth(partitionState).getOffsetHighestSucceeded().isEqualTo(highest); long offsetHighestSeen = partitionState.getOffsetHighestSeen(); assertThat(offsetHighestSeen).isEqualTo(highest); From 66fe6f5e8ce8708d49e4116107a0c1011ec7fc6f Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Tue, 11 Oct 2022 12:32:56 +0100 Subject: [PATCH 16/33] review --- .../ParallelConsumerOptions.java | 2 +- .../state/PartitionState.java | 95 +++++++++---------- .../state/PartitionStateManager.java | 2 +- .../PartitionStateCommittedOffsetTest.java | 4 +- 4 files changed, 50 insertions(+), 53 deletions(-) diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java index a26e4720e..5d871da7f 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java @@ -32,7 +32,7 @@ @Builder(toBuilder = true) @ToString @FieldNameConstants -@InterfaceStability.Stable +@InterfaceStability.Evolving public class ParallelConsumerOptions { /** diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java index 8231b76c5..522cf955a 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java @@ -88,9 +88,8 @@ public class PartitionState { private NavigableSet incompleteOffsets; /** - * Marks whether any {@link WorkContainer}s have been added yet or not. Used for some initial analysis. + * Marks whether any {@link WorkContainer}s have been added yet or not. Used for some initial poll analysis. */ - // todo rename private boolean bootstrapPhase = true; /** @@ -150,10 +149,8 @@ public class PartitionState { * the upcoming PR #200 Refactor: Consider a shared nothing * architecture. * * @deprecated the map structure isn't used anymore and can be replaced with the offsets tracked in - * {@link #incompleteOffsets} + * {@link #incompleteOffsets} - refactored future PR */ - // todo rename - it's not a queue of things to be committed - it's a collection of incomplete offsets and their WorkContainers - // todo delete? seems this can be replaced by #incompletes - the work container info isn't used @ToString.Exclude @Deprecated private NavigableMap> commitQueue = new ConcurrentSkipListMap<>(); @@ -165,14 +162,13 @@ private NavigableMap> getCommitQueue() { public PartitionState(TopicPartition tp, OffsetMapCodecManager.HighestOffsetAndIncompletes offsetData) { this.tp = tp; - initState(offsetData); + initStateFromOffsetData(offsetData); } - private void initState(OffsetMapCodecManager.HighestOffsetAndIncompletes offsetData) { + private void initStateFromOffsetData(OffsetMapCodecManager.HighestOffsetAndIncompletes offsetData) { this.offsetHighestSeen = offsetData.getHighestSeenOffset().orElse(KAFKA_OFFSET_ABSENCE); this.incompleteOffsets = new ConcurrentSkipListSet<>(offsetData.getIncompleteOffsets()); this.offsetHighestSucceeded = this.offsetHighestSeen; // by definition, as we only encode up to the highest seen offset (inclusive) - this.nextExpectedPolledOffset = getNextExpectedInitialPolledOffset(); } private void maybeRaiseHighestSeenOffset(final long offset) { @@ -247,8 +243,6 @@ private void updateHighestSucceededOffsetSoFar(WorkContainer work) { public void addNewIncompleteWorkContainer(WorkContainer wc) { long newOffset = wc.offset(); -// maybeTruncateBelowOrAbove(newOffset); - maybeRaiseHighestSeenOffset(newOffset); commitQueue.put(newOffset, wc); @@ -270,15 +264,13 @@ private void maybeTruncateBelowOrAbove(long polledOffset) { return; } -// long expectedBootstrapRecordOffset = this.getNextExpectedPolledOffset(); long expectedBootstrapRecordOffset = getNextExpectedInitialPolledOffset(); + boolean pollAboveExpected = polledOffset > expectedBootstrapRecordOffset; - boolean bootstrapPolledRecordAboveExpected = polledOffset > expectedBootstrapRecordOffset; - - boolean bootstrapPolledRecordBelowExpected = polledOffset < expectedBootstrapRecordOffset; + boolean pollBelowExpected = polledOffset < expectedBootstrapRecordOffset; - if (bootstrapPolledRecordAboveExpected) { + if (pollAboveExpected) { // previously committed offset record has been removed, or manual reset to higher offset detected log.warn("Truncating state - removing records lower than {}. Offsets have been removed form the partition by the broker. Bootstrap polled {} but " + "expected {} from loaded commit data. Could be caused by record retention or compaction.", @@ -286,30 +278,24 @@ private void maybeTruncateBelowOrAbove(long polledOffset) { polledOffset, expectedBootstrapRecordOffset); + // truncate this.incompleteOffsets = incompleteOffsets.tailSet(polledOffset, true); this.commitQueue = commitQueue.tailMap(polledOffset, true); - } else if (bootstrapPolledRecordBelowExpected) { + } else if (pollBelowExpected) { // manual reset to lower offset detected - log.warn("Bootstrap polled offset has been reset to an earlier offset ({}) - truncating state - all records above including this will be replayed. Was expecting {} but bootstrap poll was {}.", + log.warn("Bootstrap polled offset has been reset to an earlier offset ({}) - truncating state - all records " + + "above (including this) will be replayed. Was expecting {} but bootstrap poll was {}.", polledOffset, expectedBootstrapRecordOffset, polledOffset ); - // reset state to the polled offset - // todo option to save incompletes data and only replay offsets between polled through to expected? (just replay the offsets of which we've moved backwards by) - initState(new OffsetMapCodecManager.HighestOffsetAndIncompletes(Optional.of(polledOffset - 1), UniSets.of())); -// this.incompleteOffsets = new ConcurrentSkipListSet<>(); -// this.commitQueue = new ConcurrentSkipListMap<>(); - } - - this.nextExpectedPolledOffset = polledOffset + 1; - } - // todo delete - private long nextExpectedPolledOffset = KAFKA_OFFSET_ABSENCE; - - private long getNextExpectedPolledOffset() { - return nextExpectedPolledOffset; + // reset + var resetHighestSeenOffset = Optional.empty(); + var resetIncompletesMap = UniSets.of(); + var offsetData = new OffsetMapCodecManager.HighestOffsetAndIncompletes(resetHighestSeenOffset, resetIncompletesMap); + initStateFromOffsetData(offsetData); + } } /** @@ -322,15 +308,18 @@ public boolean isRemoved() { } public Optional getCommitDataIfDirty() { - if (isDirty()) return of(createOffsetAndMetadata()); - else return empty(); + return isDirty() ? + of(createOffsetAndMetadata()) : + empty(); } // visible for testing protected OffsetAndMetadata createOffsetAndMetadata() { Optional payloadOpt = tryToEncodeOffsets(); long nextOffset = getNextExpectedInitialPolledOffset(); - return payloadOpt.map(s -> new OffsetAndMetadata(nextOffset, s)).orElseGet(() -> new OffsetAndMetadata(nextOffset)); + return payloadOpt + .map(encodedOffsets -> new OffsetAndMetadata(nextOffset, encodedOffsets)) + .orElseGet(() -> new OffsetAndMetadata(nextOffset)); } /** @@ -360,8 +349,8 @@ public Set getIncompleteOffsetsBelowHighestSucceeded() { long highestSucceeded = getOffsetHighestSucceeded(); //noinspection FuseStreamOperations Collectors.toUnmodifiableSet since v10 return Collections.unmodifiableSet(incompleteOffsets.parallelStream() - // todo less than or less than and equal? - .filter(x -> x < highestSucceeded).collect(Collectors.toSet())); + .filter(x -> x < highestSucceeded) + .collect(Collectors.toSet())); } /** @@ -433,11 +422,14 @@ private boolean updateBlockFromEncodingResult(String offsetMapPayload) { // exceeded maximum API allowed, strip the payload mustStrip = true; setAllowedMoreRecords(false); - log.warn("Offset map data too large (size: {}) to fit in metadata payload hard limit of {} - cannot include in commit. " + "Warning: messages might be replayed on rebalance. " + "See kafka.coordinator.group.OffsetConfig#DefaultMaxMetadataSize = {} and issue #47.", metaPayloadLength, DefaultMaxMetadataSize, DefaultMaxMetadataSize); + log.warn("Offset map data too large (size: {}) to fit in metadata payload hard limit of {} - cannot include in commit. " + + "Warning: messages might be replayed on rebalance. " + + "See kafka.coordinator.group.OffsetConfig#DefaultMaxMetadataSize = {} and issue #47.", metaPayloadLength, DefaultMaxMetadataSize, DefaultMaxMetadataSize); } else if (metaPayloadLength > getPressureThresholdValue()) { // and thus metaPayloadLength <= DefaultMaxMetadataSize // try to turn on back pressure before max size is reached setAllowedMoreRecords(false); - log.warn("Payload size {} higher than threshold {}, but still lower than max {}. Will write payload, but will " + "not allow further messages, in order to allow the offset data to shrink (via succeeding messages).", metaPayloadLength, getPressureThresholdValue(), DefaultMaxMetadataSize); + log.warn("Payload size {} higher than threshold {}, but still lower than max {}. Will write payload, but will " + + "not allow further messages, in order to allow the offset data to shrink (via succeeding messages).", metaPayloadLength, getPressureThresholdValue(), DefaultMaxMetadataSize); } else { // and thus (metaPayloadLength <= pressureThresholdValue) setAllowedMoreRecords(true); @@ -469,9 +461,12 @@ public boolean isBlocked() { * Each time we poll a patch of records, check to see that as expected our tracked incomplete offsets exist in the * set, otherwise they must have been removed from the underlying partition and should be removed from our tracking * as we'll ever be given the record again to retry. + *

+ *

+ * Also, does {@link #maybeTruncateBelowOrAbove}. */ @SuppressWarnings("OptionalGetWithoutIsPresent") // checked with isEmpty - protected void pruneRemovedTrackedIncompleteOffsets(EpochAndRecordsMap.RecordsAndEpoch polledRecordBatch) { + protected void maybeTruncateOrPruneTrackedOffsets(EpochAndRecordsMap.RecordsAndEpoch polledRecordBatch) { var records = polledRecordBatch.getRecords(); if (records.isEmpty()) { @@ -491,22 +486,24 @@ protected void pruneRemovedTrackedIncompleteOffsets(EpochAndRecordsMap.Rec var high = getLast(records).get().offset(); // NOSONAR see #isEmpty // for the incomplete offsets within this range of poll batch - var subsetFromBatchRange = incompleteOffsets.subSet(low, true, high, true); + var incompletesWithinPolledBatch = incompleteOffsets.subSet(low, true, high, true); - for (long offset : subsetFromBatchRange) { - boolean offsetMissingFromPolledRecords = !polledOffsetLookup.contains(offset); + for (long incompleteOffset : incompletesWithinPolledBatch) { + boolean offsetMissingFromPolledRecords = !polledOffsetLookup.contains(incompleteOffset); if (offsetMissingFromPolledRecords) { - log.warn("Offset {} has been removed from partition {} (as it has not been returned within a polled batch which should have contained it - batch offset range is {} to {}), so it must be removed from tracking state, as it will never be sent again to be retried. This can be caused by PC rebalancing across a partition which has been compacted on offsets above the committed base offset, after initial load and before a rebalance.", - offset, + log.warn("Offset {} has been removed from partition {} (as it has not been returned within a polled batch " + + "which should have contained it - batch offset range is {} to {}), so it must be removed " + + "from tracking state, as it will never be sent again to be retried. " + + "This can be caused by PC rebalancing across a partition which has been compacted on offsets above the committed " + + "base offset, after initial load and before a rebalance.", + incompleteOffset, getTp(), low, high ); - boolean removed1 = incompleteOffsets.remove(offset); - assert removed1; - // can't remove it from a collection wit would have never been added to -// boolean removed2 = commitQueue.remove(offset) != null; -// assert removed2; + boolean removedCheck = incompleteOffsets.remove(incompleteOffset); + assert removedCheck; + // don't need to remove it from the #commitQueue, as it would never have been added } } } diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java index e9e180600..8a654c1f9 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java @@ -341,7 +341,7 @@ void maybeRegisterNewRecordAsWork(final EpochAndRecordsMap recordsMap) { var polledRecordBatch = recordsMap.records(partition); var partitionState = getPartitionState(polledRecordBatch); - partitionState.pruneRemovedTrackedIncompleteOffsets(polledRecordBatch); + partitionState.maybeTruncateOrPruneTrackedOffsets(polledRecordBatch); long epochOfInboundRecords = polledRecordBatch.getEpochOfPartitionAtPoll(); for (var rec : polledRecordBatch.getRecords()) { diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java index 6d5045188..b049516f8 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java @@ -25,7 +25,7 @@ /** * @author Antony Stubbs * @see PartitionState#maybeTruncateBelow - * @see PartitionState#pruneRemovedTrackedIncompleteOffsets + * @see PartitionState#maybeTruncateOrPruneTrackedOffsets */ class PartitionStateCommittedOffsetTest { @@ -111,7 +111,7 @@ void committedOffsetLower() { private void addPollToState(PartitionState state, PolledTestBatch polledTestBatch) { // todo when PSM and PartitionState are refactored, these two calls in PS should be a single call - state.pruneRemovedTrackedIncompleteOffsets(polledTestBatch.polledRecordBatch.records(tp)); + state.maybeTruncateOrPruneTrackedOffsets(polledTestBatch.polledRecordBatch.records(tp)); for (var wc : polledTestBatch.polledBatchWCs) { // todo when PSM and PartitionState are refactored, this conditional should not be needed var offset = wc.offset(); From 05e5ddb46493d83e361dbde45a23aa40496167c0 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Tue, 11 Oct 2022 12:39:26 +0100 Subject: [PATCH 17/33] changelog --- CHANGELOG.adoc | 6 ++++++ README.adoc | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/CHANGELOG.adoc b/CHANGELOG.adoc index c3d166713..5243de648 100644 --- a/CHANGELOG.adoc +++ b/CHANGELOG.adoc @@ -14,6 +14,12 @@ ifndef::github_name[] toc::[] endif::[] +== 0.5.2.4 + +=== Fixes + +* fixes #409: Truncate state when bootstrap polled offset higher or lower than committed and prune missing records (#425) + == 0.5.2.3 === Improvements diff --git a/README.adoc b/README.adoc index e54bb742a..7548ccb35 100644 --- a/README.adoc +++ b/README.adoc @@ -1287,6 +1287,12 @@ ifndef::github_name[] toc::[] endif::[] +== 0.5.2.4 + +=== Fixes + +* fixes #409: Truncate state when bootstrap polled offset higher or lower than committed and prune missing records (#425) + == 0.5.2.3 === Improvements From 747b132dd9b2bf7bcd961b435f53112d5140093e Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Tue, 11 Oct 2022 13:02:22 +0100 Subject: [PATCH 18/33] review --- .../io/confluent/parallelconsumer/state/PolledTestBatch.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PolledTestBatch.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PolledTestBatch.java index 2749a4a1f..1b5d4bf39 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PolledTestBatch.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PolledTestBatch.java @@ -1,5 +1,9 @@ package io.confluent.parallelconsumer.state; +/*- + * Copyright (C) 2020-2022 Confluent, Inc. + */ + import io.confluent.csid.utils.JavaUtils; import io.confluent.parallelconsumer.internal.EpochAndRecordsMap; import one.util.streamex.LongStreamEx; From a8e28c30b70c70c2872d785486ceb83f6c98b4ec Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Tue, 11 Oct 2022 13:35:17 +0100 Subject: [PATCH 19/33] fix OffsetEncodingTests --- .../PartitionStateCommittedOffsetIT.java | 5 ++ .../offsets/OffsetEncodingTests.java | 61 +++++++++++++------ .../PartitionStateCommittedOffsetTest.java | 5 ++ .../src/test/resources/logback-test.xml | 5 +- 4 files changed, 55 insertions(+), 21 deletions(-) diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java index 9871a4118..cc044bf09 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java @@ -8,6 +8,11 @@ import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.common.TopicPartition; +/** + * todo docs + * + * @author Antony Stubbs + */ class PartitionStateCommittedOffsetIT extends BrokerIntegrationTest { AdminClient ac; diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingTests.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingTests.java index 6cbed585e..43295b819 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingTests.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingTests.java @@ -23,6 +23,7 @@ import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.ValueSource; import pl.tlinkowski.unij.api.UniLists; +import pl.tlinkowski.unij.api.UniMaps; import pl.tlinkowski.unij.api.UniSets; import java.nio.ByteBuffer; @@ -31,7 +32,6 @@ import static io.confluent.parallelconsumer.ManagedTruth.assertTruth; import static io.confluent.parallelconsumer.offsets.OffsetEncoding.*; -import static io.confluent.parallelconsumer.state.PartitionState.KAFKA_OFFSET_ABSENCE; import static org.assertj.core.api.Assertions.assertThat; import static org.hamcrest.Matchers.in; import static org.hamcrest.Matchers.not; @@ -116,6 +116,8 @@ void largeIncompleteOffsetValues(long nextExpectedOffset) { } /** + * Test for offset encoding when there is a very large range of offsets, and where the offsets aren't sequential. + *

* There's no guarantee that offsets are always sequential. The most obvious case is with a compacted topic - there * will always be offsets missing. * @@ -138,7 +140,8 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset OffsetSimultaneousEncoder.compressionForced = true; var records = new ArrayList>(); - records.add(new ConsumerRecord<>(INPUT_TOPIC, 0, 0, "akey", "avalue")); // will complete + final int FIRST_SUCCEEDED_OFFSET = 0; + records.add(new ConsumerRecord<>(INPUT_TOPIC, 0, FIRST_SUCCEEDED_OFFSET, "akey", "avalue")); // will complete records.add(new ConsumerRecord<>(INPUT_TOPIC, 0, 1, "akey", "avalue")); records.add(new ConsumerRecord<>(INPUT_TOPIC, 0, 4, "akey", "avalue")); records.add(new ConsumerRecord<>(INPUT_TOPIC, 0, 5, "akey", "avalue")); @@ -153,15 +156,15 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset records.add(new ConsumerRecord<>(INPUT_TOPIC, 0, 40_000, "akey", "avalue")); // higher than Short.MAX_VALUE int avoidOffByOne = 2; records.add(new ConsumerRecord<>(INPUT_TOPIC, 0, 40_000 + Short.MAX_VALUE + avoidOffByOne, "akey", "avalue")); // runlength higher than Short.MAX_VALUE - int highest = 40_000 + Short.MAX_VALUE + avoidOffByOne + 1; - records.add(new ConsumerRecord<>(INPUT_TOPIC, 0, highest, "akey", "avalue")); // will complete to force whole encoding + int highestSucceeded = 40_000 + Short.MAX_VALUE + avoidOffByOne + 1; + records.add(new ConsumerRecord<>(INPUT_TOPIC, 0, highestSucceeded, "akey", "avalue")); // will complete to force whole encoding var incompleteRecords = new ArrayList<>(records); - incompleteRecords.remove(incompleteRecords.stream().filter(x -> x.offset() == 0).findFirst().get()); + incompleteRecords.remove(incompleteRecords.stream().filter(x -> x.offset() == FIRST_SUCCEEDED_OFFSET).findFirst().get()); incompleteRecords.remove(incompleteRecords.stream().filter(x -> x.offset() == 69).findFirst().get()); incompleteRecords.remove(incompleteRecords.stream().filter(x -> x.offset() == 25_000).findFirst().get()); - incompleteRecords.remove(incompleteRecords.stream().filter(x -> x.offset() == highest).findFirst().get()); + incompleteRecords.remove(incompleteRecords.stream().filter(x -> x.offset() == highestSucceeded).findFirst().get()); List expected = incompleteRecords.stream().map(ConsumerRecord::offset) .sorted() @@ -179,6 +182,7 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset // write offsets final ParallelConsumerOptions newOptions = options.toBuilder().consumer(consumerSpy).build(); + final long FIRST_COMMITTED_OFFSET = 1L; { WorkManager wmm = new WorkManager<>(new PCModule<>(newOptions)); wmm.onPartitionsAssigned(UniSets.of(new TopicPartition(INPUT_TOPIC, 0))); @@ -187,18 +191,18 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset List> work = wmm.getWorkIfAvailable(); assertThat(work).hasSameSizeAs(records); - KafkaTestUtils.completeWork(wmm, work, 0); + KafkaTestUtils.completeWork(wmm, work, FIRST_SUCCEEDED_OFFSET); KafkaTestUtils.completeWork(wmm, work, 69); KafkaTestUtils.completeWork(wmm, work, 25_000); - KafkaTestUtils.completeWork(wmm, work, highest); + KafkaTestUtils.completeWork(wmm, work, highestSucceeded); // make the commit var completedEligibleOffsets = wmm.collectCommitDataForDirtyPartitions(); - assertThat(completedEligibleOffsets.get(tp).offset()).isEqualTo(1L); + assertThat(completedEligibleOffsets.get(tp).offset()).isEqualTo(FIRST_COMMITTED_OFFSET); consumerSpy.commitSync(completedEligibleOffsets); { @@ -206,7 +210,7 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset OffsetMapCodecManager om = new OffsetMapCodecManager<>(consumerSpy); OffsetMapCodecManager.forcedCodec = Optional.empty(); // turn off forced var state = wmm.getPm().getPartitionState(tp); - String bestPayload = om.makeOffsetMetadataPayload(1, state); + String bestPayload = om.makeOffsetMetadataPayload(FIRST_COMMITTED_OFFSET, state); assertThat(bestPayload).isNotEmpty(); } } @@ -214,7 +218,7 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset // check { var committed = consumerSpy.committed(UniSets.of(tp)).get(tp); - assertThat(committed.offset()).isEqualTo(1L); + assertThat(committed.offset()).isEqualTo(FIRST_COMMITTED_OFFSET); if (assumeWorkingCodec(encoding, encodingsThatFail)) { assertThat(committed.metadata()).isNotBlank(); @@ -227,22 +231,43 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset { var newWm = new WorkManager<>(new PCModule<>(options)); newWm.onPartitionsAssigned(UniSets.of(tp)); - newWm.registerWork(new EpochAndRecordsMap<>(testRecords, newWm.getPm())); + // var pm = newWm.getPm(); var partitionState = pm.getPartitionState(tp); if (assumeWorkingCodec(encoding, encodingsThatFail)) { + // check state reloaded ok from consumer + assertTruth(partitionState).getOffsetHighestSucceeded().isEqualTo(highestSucceeded); + } + + // + ConsumerRecords testRecordsWithBaseCommittedRecordRemoved = new ConsumerRecords<>(UniMaps.of(tp, + testRecords.records(tp) + .stream() + .filter(x -> + x.offset() >= FIRST_COMMITTED_OFFSET) + .collect(Collectors.toList()))); + EpochAndRecordsMap epochAndRecordsMap = new EpochAndRecordsMap<>(testRecordsWithBaseCommittedRecordRemoved, newWm.getPm()); + newWm.registerWork(epochAndRecordsMap); + + if (assumeWorkingCodec(encoding, encodingsThatFail)) { + // check state reloaded ok from consumer + assertTruth(partitionState).getOffsetHighestSucceeded().isEqualTo(highestSucceeded); + } + + // + if (assumeWorkingCodec(encoding, encodingsThatFail)) { // long offsetHighestSequentialSucceeded = partitionState.getOffsetHighestSequentialSucceeded(); // assertThat(offsetHighestSequentialSucceeded).isEqualTo(0); - assertTruth(partitionState).getOffsetHighestSequentialSucceeded().isEqualTo(KAFKA_OFFSET_ABSENCE); + assertTruth(partitionState).getOffsetHighestSequentialSucceeded().isEqualTo(FIRST_SUCCEEDED_OFFSET); // long offsetHighestSucceeded = partitionState.getOffsetHighestSucceeded(); -// assertThat(offsetHighestSucceeded).isEqualTo(highest); - assertTruth(partitionState).getOffsetHighestSucceeded().isEqualTo(highest); +// assertThat(offsetHighestSucceeded).isEqualTo(highestSucceeded); + assertTruth(partitionState).getOffsetHighestSucceeded().isEqualTo(highestSucceeded); long offsetHighestSeen = partitionState.getOffsetHighestSeen(); - assertThat(offsetHighestSeen).isEqualTo(highest); + assertThat(offsetHighestSeen).isEqualTo(highestSucceeded); var incompletes = partitionState.getIncompleteOffsetsBelowHighestSucceeded(); Truth.assertThat(incompletes).containsExactlyElementsIn(expected); @@ -259,10 +284,10 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset assertThat(offsetHighestSequentialSucceeded).isEqualTo(0); long offsetHighestSucceeded = partitionState.getOffsetHighestSucceeded(); - assertThat(offsetHighestSucceeded).isEqualTo(highest); + assertThat(offsetHighestSucceeded).isEqualTo(highestSucceeded); long offsetHighestSeen = partitionState.getOffsetHighestSeen(); - assertThat(offsetHighestSeen).isEqualTo(highest); + assertThat(offsetHighestSeen).isEqualTo(highestSucceeded); var incompletes = partitionState.getIncompleteOffsetsBelowHighestSucceeded(); Truth.assertThat(incompletes).containsExactlyElementsIn(expected); diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java index b049516f8..b71c2bd38 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java @@ -5,6 +5,7 @@ */ import io.confluent.parallelconsumer.internal.PCModuleTestEnv; +import io.confluent.parallelconsumer.offsets.OffsetEncodingTests; import io.confluent.parallelconsumer.offsets.OffsetMapCodecManager.HighestOffsetAndIncompletes; import one.util.streamex.LongStreamEx; import org.apache.kafka.clients.admin.AdminClient; @@ -23,7 +24,10 @@ import static io.confluent.parallelconsumer.ManagedTruth.assertThat; /** + * todo docs + * * @author Antony Stubbs + * @see OffsetEncodingTests#ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential * @see PartitionState#maybeTruncateBelow * @see PartitionState#maybeTruncateOrPruneTrackedOffsets */ @@ -152,6 +156,7 @@ private void addPollToState(PartitionState state, PolledTestBatc * * @implSpec issue #409: Committing old offset after OFFSET_OUT_OF_RANGE * @see PartitionState#maybeTruncateBelow + * @see OffsetEncodingTests#ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential */ @Test void bootstrapPollOffsetHigherDueToRetentionOrCompaction() { diff --git a/parallel-consumer-core/src/test/resources/logback-test.xml b/parallel-consumer-core/src/test/resources/logback-test.xml index 87c99a321..0a9472e95 100644 --- a/parallel-consumer-core/src/test/resources/logback-test.xml +++ b/parallel-consumer-core/src/test/resources/logback-test.xml @@ -27,8 +27,8 @@ - - + + @@ -79,7 +79,6 @@ - From 6b93b1f988aa01f74a864868ce598e6b57595f0c Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Tue, 11 Oct 2022 18:51:24 +0100 Subject: [PATCH 20/33] Draft PartitionStateCommittedOffsetIT, partially passing --- .../io/confluent/csid/utils/JavaUtils.java | 21 +- .../state/PartitionState.java | 12 +- .../BrokerIntegrationTest.java | 9 +- .../PartitionStateCommittedOffsetIT.java | 302 +++++++++++++++++- .../utils/KafkaClientUtils.java | 12 +- .../io/confluent/csid/utils/ThreadUtils.java | 4 + .../offsets/OffsetEncodingTests.java | 4 - .../PartitionStateCommittedOffsetTest.java | 22 +- .../state/PolledTestBatch.java | 8 +- 9 files changed, 345 insertions(+), 49 deletions(-) diff --git a/parallel-consumer-core/src/main/java/io/confluent/csid/utils/JavaUtils.java b/parallel-consumer-core/src/main/java/io/confluent/csid/utils/JavaUtils.java index 00507cba7..647175d8b 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/csid/utils/JavaUtils.java +++ b/parallel-consumer-core/src/main/java/io/confluent/csid/utils/JavaUtils.java @@ -8,12 +8,10 @@ import lombok.experimental.UtilityClass; import java.time.Duration; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.Optional; +import java.util.*; import java.util.function.Function; import java.util.stream.Collectors; +import java.util.stream.Stream; import static java.time.Duration.ofMillis; @@ -60,4 +58,19 @@ public static Map remap(Map map, )); } + public static List getRandom(List list, int quantity) { + if (list.size() < quantity) { + throw new IllegalArgumentException("List size is less than quantity"); + } + + return createRandomIntStream(list.size()) + .limit(quantity) + .map(list::get) + .collect(Collectors.toList()); + } + + private static Stream createRandomIntStream(int range) { + final Random random = new Random(); + return Stream.generate(() -> random.nextInt(range)); + } } diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java index 522cf955a..12ced89b1 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java @@ -30,7 +30,7 @@ import static lombok.AccessLevel.*; /** - * Our view of our state of the partitions that we've been assigned. + * Our view of the state of the partitions that we've been assigned. * * @see PartitionStateManager */ @@ -329,7 +329,7 @@ protected OffsetAndMetadata createOffsetAndMetadata() { */ // visible for testing // todo change back to protected? and enable protected level managed truth (seems to be limited to public) - public long getNextExpectedInitialPolledOffset() { + protected long getNextExpectedInitialPolledOffset() { return getOffsetHighestSequentialSucceeded() + 1; } @@ -423,13 +423,15 @@ private boolean updateBlockFromEncodingResult(String offsetMapPayload) { mustStrip = true; setAllowedMoreRecords(false); log.warn("Offset map data too large (size: {}) to fit in metadata payload hard limit of {} - cannot include in commit. " + - "Warning: messages might be replayed on rebalance. " + - "See kafka.coordinator.group.OffsetConfig#DefaultMaxMetadataSize = {} and issue #47.", metaPayloadLength, DefaultMaxMetadataSize, DefaultMaxMetadataSize); + "Warning: messages might be replayed on rebalance. " + + "See kafka.coordinator.group.OffsetConfig#DefaultMaxMetadataSize = {} and issue #47.", + metaPayloadLength, DefaultMaxMetadataSize, DefaultMaxMetadataSize); } else if (metaPayloadLength > getPressureThresholdValue()) { // and thus metaPayloadLength <= DefaultMaxMetadataSize // try to turn on back pressure before max size is reached setAllowedMoreRecords(false); log.warn("Payload size {} higher than threshold {}, but still lower than max {}. Will write payload, but will " + - "not allow further messages, in order to allow the offset data to shrink (via succeeding messages).", metaPayloadLength, getPressureThresholdValue(), DefaultMaxMetadataSize); + "not allow further messages, in order to allow the offset data to shrink (via succeeding messages).", + metaPayloadLength, getPressureThresholdValue(), DefaultMaxMetadataSize); } else { // and thus (metaPayloadLength <= pressureThresholdValue) setAllowedMoreRecords(true); diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/BrokerIntegrationTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/BrokerIntegrationTest.java index 7fea7ad30..d4402302f 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/BrokerIntegrationTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/BrokerIntegrationTest.java @@ -10,6 +10,7 @@ import io.confluent.csid.testcontainers.FilteredTestContainerSlf4jLogConsumer; import io.confluent.parallelconsumer.integrationTests.utils.KafkaClientUtils; import lombok.Getter; +import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.admin.CreateTopicsResult; import org.apache.kafka.clients.admin.NewTopic; @@ -21,6 +22,7 @@ import org.testcontainers.utility.DockerImageName; import pl.tlinkowski.unij.api.UniLists; +import java.util.List; import java.util.concurrent.ExecutionException; import static org.apache.commons.lang3.RandomUtils.nextInt; @@ -80,7 +82,7 @@ void close() { kcu.close(); } - void setupTopic() { + protected void setupTopic() { String name = LoadTest.class.getSimpleName(); setupTopic(name); } @@ -108,4 +110,9 @@ protected CreateTopicsResult ensureTopic(String topic, int numPartitions) { return topics; } + @SneakyThrows + protected List produceMessages(int quantity) { + return getKcu().produceMessages(getTopic(), quantity); + } + } diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java index cc044bf09..67272e227 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java @@ -4,53 +4,343 @@ * Copyright (C) 2020-2022 Confluent, Inc. */ +import com.google.common.truth.Truth; +import io.confluent.csid.utils.JavaUtils; +import io.confluent.csid.utils.ThreadUtils; +import io.confluent.parallelconsumer.ParallelEoSStreamProcessor; +import io.confluent.parallelconsumer.PollContext; import io.confluent.parallelconsumer.integrationTests.BrokerIntegrationTest; +import io.confluent.parallelconsumer.integrationTests.utils.KafkaClientUtils.GroupOption; +import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.AlterConfigOp; +import org.apache.kafka.clients.admin.ConfigEntry; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.config.ConfigResource; +import org.apache.kafka.common.config.TopicConfig; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.testcontainers.shaded.org.awaitility.Awaitility; +import pl.tlinkowski.unij.api.UniMaps; + +import java.util.*; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; + +import static io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder.PARTITION; +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.testcontainers.shaded.org.hamcrest.Matchers.equalTo; +import static pl.tlinkowski.unij.api.UniLists.of; /** - * todo docs + * Integration test versions of {@link io.confluent.parallelconsumer.state.PartitionStateCommittedOffsetTest}, where + * committed offset gets moved around or deleted, or random offsets are removed. * * @author Antony Stubbs + * @see io.confluent.parallelconsumer.state.PartitionStateCommittedOffsetTest */ +@Slf4j class PartitionStateCommittedOffsetIT extends BrokerIntegrationTest { AdminClient ac; - String groupId = getKcu().getConsumer().groupMetadata().groupId(); + String groupId; + + ParallelEoSStreamProcessor pc; + + TopicPartition tp; + + final int TO_PRODUCE = 200; - TopicPartition tp = new TopicPartition("topic", 0); + @BeforeEach + void setup() { + setupTopic(); + tp = new TopicPartition(getTopic(), 0); + groupId = getKcu().getConsumer().groupMetadata().groupId(); + this.ac = getKcu().getAdmin(); + } /** * Test for offset gaps in partition data (i.e. compacted topics) */ + @Test void compactedTopic() { + setupCompacted(); + + List keys = produceMessages(TO_PRODUCE); + + final int UNTIL_OFFSET = 50; + var processedOnFirstRun = runPcUntilOffset(UNTIL_OFFSET); + + closePC(); + + Set tombStonedKeys = new HashSet<>(sendRandomTombstones(keys, UNTIL_OFFSET, TO_PRODUCE)); + + var processedOnFirstRunWithTombstoneTargetsRemoved = processedOnFirstRun.stream() + .filter(context -> !tombStonedKeys.contains(context.key())) + .map(PollContext::key) + .collect(Collectors.toList()); + + var firstRunPartitioned = processedOnFirstRun.stream().collect(Collectors.partitioningBy(context -> tombStonedKeys.contains(context.key()))); + var saved = firstRunPartitioned.get(Boolean.FALSE); + var tombstoned = firstRunPartitioned.get(Boolean.TRUE); + + var tombstoneTargetOffsetsFromFirstRun = tombstoned.stream() + .filter(context -> tombStonedKeys.contains(context.key())) + .map(PollContext::offset) + .collect(Collectors.toList()); + + var tombStonedOffsetsFromKey = tombStonedKeys.stream() + .map(PartitionStateCommittedOffsetIT::getOffsetFromKey).collect(Collectors.toList()); + log.debug("First run produced, with tombstone targets removed: {}", processedOnFirstRunWithTombstoneTargetsRemoved); + + triggerTombStoneProcessing(); + + // The offsets of the tombstone targets should not be read in second run + final int expectedTotalNumberRecordsProduced = TO_PRODUCE + tombStonedOffsetsFromKey.size(); + final int expectedOffsetProcessedToSecondRun = TO_PRODUCE + tombStonedKeys.size(); + var processedOnSecondRun = runPcUntilOffset(expectedOffsetProcessedToSecondRun); + + final List offsetsFromSecondRunFromKey = processedOnSecondRun.stream() + .map(PollContext::key) + .collect(Collectors.toList()); + + final List offsetsFromSecond = processedOnSecondRun.stream() + .map(PollContext::offset) + .collect(Collectors.toList()); + + Truth.assertWithMessage("The offsets of the tombstone targets should not be read in second run") + .that(offsetsFromSecond) + .containsNoneIn(tombstoneTargetOffsetsFromFirstRun); + + final int expectedNumberOfRecordsProcessedSecondRun = TO_PRODUCE - tombStonedOffsetsFromKey.size(); + Truth.assertWithMessage("Processed on second run should be original sent, minus number of tombstones sent") + .that(processedOnSecondRun) + .hasSize(expectedNumberOfRecordsProcessedSecondRun); + } + + private static long getOffsetFromKey(String key) { + return Long.parseLong(key.substring(key.indexOf("-") + 1)); + } + + @SneakyThrows + private void setupCompacted() { + log.debug("Setting up aggressive compaction..."); + ConfigResource topicConfig = new ConfigResource(ConfigResource.Type.TOPIC, getTopic()); + + Collection alterConfigOps = new ArrayList<>(); + + alterConfigOps.add(new AlterConfigOp(new ConfigEntry(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT), AlterConfigOp.OpType.SET)); + alterConfigOps.add(new AlterConfigOp(new ConfigEntry(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG, "1"), AlterConfigOp.OpType.SET)); + alterConfigOps.add(new AlterConfigOp(new ConfigEntry(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, "0"), AlterConfigOp.OpType.SET)); + + var configs = UniMaps.of(topicConfig, alterConfigOps); + KafkaFuture all = ac.incrementalAlterConfigs(configs).all(); + all.get(5, SECONDS); + + log.debug("Compaction setup complete"); + } + + @SneakyThrows + private void triggerTombStoneProcessing() { + ThreadUtils.sleepSecondsLog(30); + } + + @SneakyThrows + private ArrayList sendRandomTombstones(List keys, int from, int to) { + var tombstoneKeys = new ArrayList(); + var howMany = (to - from) / 3; + // fix randomness + List> futures = JavaUtils.getRandom(keys, howMany).stream() + .map((String key) -> { + tombstoneKeys.add(key); + var tombstone = new ProducerRecord<>(getTopic(), key, "tombstone"); + return getKcu().getProducer() + .send(tombstone); + }) + .collect(Collectors.toList()); + List tombstoneOffsets = new ArrayList<>(); + for (Future future : futures) { + RecordMetadata recordMetadata = future.get(5, SECONDS); + tombstoneOffsets.add(recordMetadata.offset()); + } + log.debug("Generated tombstones: {}", tombstoneOffsets); + return tombstoneKeys; } /** * CG offset has been changed to a lower offset (partition rewind / replay) (metdata lost?) */ + @Test void committedOffsetLower() { -// ac.alterConsumerGroupOffsets(groupId, ) + produceMessages(TO_PRODUCE); + + runPcUntilOffset(50); + + closePC(); + + final int moveToOffset = 25; + + moveCommittedOffset(moveToOffset); + + runPcCheckStartIs(moveToOffset, TO_PRODUCE); + } + + /** + * Ensure that the PC starts at the correct offset + * + * @param targetStartOffset the offset to check that PC starts at + * @param checkUpTo the offset to run the PC until, while checking for the start offset + */ + private void runPcCheckStartIs(long targetStartOffset, long checkUpTo) { + this.pc = super.getKcu().buildPc(PARTITION); + pc.subscribe(of(getTopic())); + + AtomicLong lowest = new AtomicLong(Long.MAX_VALUE); + AtomicLong highest = new AtomicLong(); + + pc.poll(recordContexts -> { + long thisOffset = recordContexts.offset(); + if (thisOffset < lowest.get()) { + log.debug("Found lowest offset {}", thisOffset); + lowest.set(thisOffset); + } else { + highest.set(thisOffset); + } + }); + + Awaitility.await().untilAtomic(highest, equalTo(checkUpTo - 1)); + + pc.close(); + + Truth.assertWithMessage("Offset started as").that(lowest.get()).isEqualTo(targetStartOffset); + } + + @SneakyThrows + private void moveCommittedOffset(long offset) { + log.debug("Moving offset to {}", offset); + var data = UniMaps.of(tp, new OffsetAndMetadata(offset)); + var result = ac.alterConsumerGroupOffsets(groupId, data); + result.all().get(5, SECONDS); + log.debug("Moved offset to {}", offset); + } + + private void closePC() { + pc.close(); + } + + private ArrayList> runPcUntilOffset(long targetOffset) { + this.pc = super.getKcu().buildPc(PARTITION, GroupOption.NEW_GROUP); + + Queue> concurrentList = new ConcurrentLinkedQueue<>(); + AtomicLong current = new AtomicLong(); + pc.subscribe(of(getTopic())); + pc.poll(pollContext -> { + long thisOffset = pollContext.offset(); + if (thisOffset >= targetOffset) { + log.debug("Failing on {}", thisOffset); + } else { + log.debug("Processed {}", thisOffset); + current.set(thisOffset); + } + concurrentList.add(pollContext); + }); + + Awaitility.await().untilAtomic(current, equalTo(targetOffset - 1)); + log.debug("Consumed up to {}", targetOffset); + + var sorted = new ArrayList<>(concurrentList); + Collections.sort(sorted, Comparator.comparingLong(PollContext::offset)); + return sorted; } /** * CG offset has been changed to something higher than expected (offset skip) (metdata lost?) */ + @Test void committedOffsetHigher() { + final int quantity = 100; + produceMessages(quantity); + + runPcUntilOffset(50); + + closePC(); + + final int moveToOffset = 75; + + moveCommittedOffset(moveToOffset); + + runPcCheckStartIs(moveToOffset, quantity); } /** * CG offset has disappeared - committed offset hasn't been changed, but broker gives us a bootstrap poll result * with a higher offset than expected. Could be caused by retention period, or compaction. */ + @Test void committedOffsetRemoved() { + produceMessages(TO_PRODUCE); + + final int END_OFFSET = 50; + runPcUntilOffset(END_OFFSET); + + closePC(); + + causeCommittedOffsetToBeRemoved(END_OFFSET); + + produceMessages(TO_PRODUCE); + + final int TOTAL = TO_PRODUCE * 2; + runPcCheckStartIs(END_OFFSET + 1, TOTAL); + } + + private void causeCommittedOffsetToBeRemoved(long offset) { + throw new RuntimeException(); + } + + @Test + void cgOffsetsDeletedResetLatest() { + produceMessages(TO_PRODUCE); + + final int END_OFFSET = 50; + runPcUntilOffset(END_OFFSET); + + closePC(); + + causeCommittedConsumerGroupOffsetToBeDeleted(); + + produceMessages(TO_PRODUCE); + + final int TOTAL_PRODUCED = TO_PRODUCE * 2; + runPcCheckStartIs(TOTAL_PRODUCED, TOTAL_PRODUCED); } - void cgOffsetsDeleted() { -// ac.deleteConsumerGroupOffsets() + @Test + void cgOffsetsDeletedResetEarliest() { + produceMessages(TO_PRODUCE); + + final int END_OFFSET = 50; + runPcUntilOffset(END_OFFSET); + + closePC(); + + causeCommittedConsumerGroupOffsetToBeDeleted(); + + produceMessages(100); + + runPcCheckStartIs(0, TO_PRODUCE); + } + private void causeCommittedConsumerGroupOffsetToBeDeleted() { + throw new RuntimeException(); } diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java index cd4f8f99a..840f02b05 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java @@ -72,6 +72,7 @@ class PCVersion { @Getter private AdminClient admin; + private String groupId = GROUP_ID_PREFIX + nextInt(); /** @@ -290,9 +291,14 @@ public List produceMessages(String topicName, long numberToSend) throws } public ParallelEoSStreamProcessor buildPc(ProcessingOrder order, CommitMode commitMode, int maxPoll) { + return buildPc(order, commitMode, maxPoll, GroupOption.REUSE_GROUP); + } + + public ParallelEoSStreamProcessor buildPc(ProcessingOrder order, CommitMode commitMode, int maxPoll, GroupOption groupOption) { Properties consumerProps = new Properties(); consumerProps.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPoll); - KafkaConsumer newConsumer = createNewConsumer(false, consumerProps); + boolean newConsumerGroup = groupOption.equals(GroupOption.NEW_GROUP); + KafkaConsumer newConsumer = createNewConsumer(newConsumerGroup, consumerProps); lastConsumerConstructed = newConsumer; var pc = new ParallelEoSStreamProcessor<>(ParallelConsumerOptions.builder() @@ -308,6 +314,10 @@ public ParallelEoSStreamProcessor buildPc(ProcessingOrder order, return pc; } + public ParallelEoSStreamProcessor buildPc(ProcessingOrder key, GroupOption groupOption) { + return buildPc(key, PERIODIC_CONSUMER_ASYNCHRONOUS, 500, groupOption); + } + public ParallelEoSStreamProcessor buildPc(ProcessingOrder key) { return buildPc(key, PERIODIC_CONSUMER_ASYNCHRONOUS, 500); } diff --git a/parallel-consumer-core/src/test/java/io/confluent/csid/utils/ThreadUtils.java b/parallel-consumer-core/src/test/java/io/confluent/csid/utils/ThreadUtils.java index b6ac22d26..f176751b4 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/csid/utils/ThreadUtils.java +++ b/parallel-consumer-core/src/test/java/io/confluent/csid/utils/ThreadUtils.java @@ -29,4 +29,8 @@ public static void sleepLog(final int ms) { public static void sleepQuietly(long ms) { sleepQuietly((int) ms); } + + public static void sleepSecondsLog(int seconds) { + sleepLog(seconds * 1000); + } } diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingTests.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingTests.java index 43295b819..73f1555b9 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingTests.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingTests.java @@ -258,12 +258,8 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset // if (assumeWorkingCodec(encoding, encodingsThatFail)) { -// long offsetHighestSequentialSucceeded = partitionState.getOffsetHighestSequentialSucceeded(); -// assertThat(offsetHighestSequentialSucceeded).isEqualTo(0); assertTruth(partitionState).getOffsetHighestSequentialSucceeded().isEqualTo(FIRST_SUCCEEDED_OFFSET); -// long offsetHighestSucceeded = partitionState.getOffsetHighestSucceeded(); -// assertThat(offsetHighestSucceeded).isEqualTo(highestSucceeded); assertTruth(partitionState).getOffsetHighestSucceeded().isEqualTo(highestSucceeded); long offsetHighestSeen = partitionState.getOffsetHighestSeen(); diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java index b71c2bd38..ef62a857d 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java @@ -30,6 +30,7 @@ * @see OffsetEncodingTests#ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential * @see PartitionState#maybeTruncateBelow * @see PartitionState#maybeTruncateOrPruneTrackedOffsets + * @see io.confluent.parallelconsumer.integrationTests.state.PartitionStateCommittedOffsetIT */ class PartitionStateCommittedOffsetTest { @@ -125,27 +126,6 @@ private void addPollToState(PartitionState state, PolledTestBatc } } } -// -// /** -// * -// */ -// @Test -// void bootstrapPollOffsetHigherViaManualCGRset() { -// // committed state -// PartitionState state = new PartitionState<>(tp, offsetData); -// -// // bootstrap poll -// PolledTestBatch polledTestBatch = new PolledTestBatch(mu, tp, unexpectedlyHighOffset, highestSeenOffset); -// -// // todo when PSM and PartitionState are refactored, these two calls in PS should be a single call -// addPollToState(state, polledTestBatch); -// -// // -// OffsetAndMetadata offsetAndMetadata = state.createOffsetAndMetadata(); -// -// assertThat(offsetAndMetadata).getOffset().isEqualTo(0L); -// state.getAllIncompleteOffsets().containsAll(Range.range(highestSeenOffset).list()); -// } /** * CG offset has disappeared - committed offset hasn't been changed, but broker gives us a bootstrap poll result diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PolledTestBatch.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PolledTestBatch.java index 1b5d4bf39..31d89cdbc 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PolledTestBatch.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PolledTestBatch.java @@ -4,7 +4,6 @@ * Copyright (C) 2020-2022 Confluent, Inc. */ -import io.confluent.csid.utils.JavaUtils; import io.confluent.parallelconsumer.internal.EpochAndRecordsMap; import one.util.streamex.LongStreamEx; import one.util.streamex.StreamEx; @@ -20,7 +19,7 @@ import static org.mockito.Mockito.mock; /** - * todo docs + * Utility data class for creating test data around polled records from broker. * * @author Antony Stubbs */ @@ -28,8 +27,6 @@ public class PolledTestBatch { final ModelUtils mu; - private final long highestSeenOffset; - private final TopicPartition tp; List> polledBatchWCs; @@ -41,7 +38,6 @@ public class PolledTestBatch { public PolledTestBatch(ModelUtils mu, TopicPartition tp, long fromOffset, long toOffset) { this.mu = mu; this.tp = tp; - this.highestSeenOffset = toOffset; create(fromOffset, toOffset); } @@ -49,8 +45,6 @@ public PolledTestBatch(ModelUtils mu, TopicPartition tp, long fromOffset, long t public PolledTestBatch(ModelUtils mu, TopicPartition tp, List polledOffsetsWithCompactedRemoved) { this.mu = mu; this.tp = tp; - //noinspection OptionalGetWithoutIsPresent - this.highestSeenOffset = JavaUtils.getLast(polledOffsetsWithCompactedRemoved).get(); create(polledOffsetsWithCompactedRemoved); From e630134260eaa693324209c67d7e412e6cbf1291 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Tue, 11 Oct 2022 21:17:08 +0100 Subject: [PATCH 21/33] compaction test works --- .../BrokerIntegrationTest.java | 11 ++- .../PartitionStateCommittedOffsetIT.java | 98 +++++++++++++------ .../utils/KafkaClientUtils.java | 6 +- .../parallelconsumer/state/ModelUtils.java | 7 +- .../src/test/resources/logback-test.xml | 7 +- 5 files changed, 93 insertions(+), 36 deletions(-) diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/BrokerIntegrationTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/BrokerIntegrationTest.java index d4402302f..9cd3a103d 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/BrokerIntegrationTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/BrokerIntegrationTest.java @@ -53,6 +53,9 @@ public abstract class BrokerIntegrationTest { .withEnv("KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR", "1") //transaction.state.log.replication.factor .withEnv("KAFKA_TRANSACTION_STATE_LOG_MIN_ISR", "1") //transaction.state.log.min.isr .withEnv("KAFKA_TRANSACTION_STATE_LOG_NUM_PARTITIONS", "1") //transaction.state.log.num.partitions + //todo need to customise this for this test + // default produce batch size is - must be at least higher than it: 16KB + .withEnv("KAFKA_LOG_SEGMENT_BYTES", "40000") // try to speed up initial consumer group formation .withEnv("KAFKA_GROUP_INITIAL_REBALANCE_DELAY_MS", "500") // group.initial.rebalance.delay.ms default: 3000 .withReuse(true); @@ -110,9 +113,13 @@ protected CreateTopicsResult ensureTopic(String topic, int numPartitions) { return topics; } - @SneakyThrows protected List produceMessages(int quantity) { - return getKcu().produceMessages(getTopic(), quantity); + return produceMessages(quantity, ""); + } + + @SneakyThrows + protected List produceMessages(int quantity, String prefix) { + return getKcu().produceMessages(getTopic(), quantity, prefix); } } diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java index 67272e227..d9328a0b3 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java @@ -4,7 +4,6 @@ * Copyright (C) 2020-2022 Confluent, Inc. */ -import com.google.common.truth.Truth; import io.confluent.csid.utils.JavaUtils; import io.confluent.csid.utils.ThreadUtils; import io.confluent.parallelconsumer.ParallelEoSStreamProcessor; @@ -27,14 +26,18 @@ import org.junit.jupiter.api.Test; import org.testcontainers.shaded.org.awaitility.Awaitility; import pl.tlinkowski.unij.api.UniMaps; +import pl.tlinkowski.unij.api.UniSets; import java.util.*; -import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; +import static com.google.common.truth.Truth.assertThat; +import static com.google.common.truth.Truth.assertWithMessage; +import static io.confluent.csid.utils.JavaUtils.getLast; import static io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder.PARTITION; +import static io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder.UNORDERED; import static java.util.concurrent.TimeUnit.SECONDS; import static org.testcontainers.shaded.org.hamcrest.Matchers.equalTo; import static pl.tlinkowski.unij.api.UniLists.of; @@ -57,7 +60,7 @@ class PartitionStateCommittedOffsetIT extends BrokerIntegrationTest keys = produceMessages(TO_PRODUCE); - final int UNTIL_OFFSET = 50; - var processedOnFirstRun = runPcUntilOffset(UNTIL_OFFSET); + final int UNTIL_OFFSET = TO_PRODUCE / 2; + var processedOnFirstRun = runPcUntilOffset(UNTIL_OFFSET, TO_PRODUCE, UniSets.of(TO_PRODUCE - 3L)); + assertWithMessage("Last processed should be at least half of the total sent, so that there is incomplete data to track") + .that(getLast(processedOnFirstRun).get().offset()) + .isGreaterThan(TO_PRODUCE / 2); + // commit offset closePC(); - Set tombStonedKeys = new HashSet<>(sendRandomTombstones(keys, UNTIL_OFFSET, TO_PRODUCE)); + // + ArrayList tombStonedKeysRaw = sendRandomTombstones(keys, TO_PRODUCE); + Set tombStonedKeys = new HashSet<>(tombStonedKeysRaw); var processedOnFirstRunWithTombstoneTargetsRemoved = processedOnFirstRun.stream() .filter(context -> !tombStonedKeys.contains(context.key())) @@ -91,6 +100,11 @@ void compactedTopic() { var firstRunPartitioned = processedOnFirstRun.stream().collect(Collectors.partitioningBy(context -> tombStonedKeys.contains(context.key()))); var saved = firstRunPartitioned.get(Boolean.FALSE); var tombstoned = firstRunPartitioned.get(Boolean.TRUE); + log.debug("kept offsets: {}", saved.stream().mapToLong(PollContext::offset).boxed().collect(Collectors.toList())); + log.debug("kept keys: {}", saved.stream().map(PollContext::key).collect(Collectors.toList())); + log.debug("tombstoned offsets: {}", tombstoned.stream().map(PollContext::key).collect(Collectors.toList())); + log.debug("tombstoned keys: {}", tombstoned.stream().mapToLong(PollContext::offset).boxed().collect(Collectors.toList())); + var tombstoneTargetOffsetsFromFirstRun = tombstoned.stream() .filter(context -> tombStonedKeys.contains(context.key())) @@ -101,29 +115,37 @@ void compactedTopic() { .map(PartitionStateCommittedOffsetIT::getOffsetFromKey).collect(Collectors.toList()); log.debug("First run produced, with tombstone targets removed: {}", processedOnFirstRunWithTombstoneTargetsRemoved); + // triggerTombStoneProcessing(); // The offsets of the tombstone targets should not be read in second run final int expectedTotalNumberRecordsProduced = TO_PRODUCE + tombStonedOffsetsFromKey.size(); final int expectedOffsetProcessedToSecondRun = TO_PRODUCE + tombStonedKeys.size(); - var processedOnSecondRun = runPcUntilOffset(expectedOffsetProcessedToSecondRun); + var processedOnSecondRun = runPcUntilOffset(expectedOffsetProcessedToSecondRun).stream() + .filter(recordContexts -> !recordContexts.key().contains("compaction-trigger")) + .collect(Collectors.toList()); - final List offsetsFromSecondRunFromKey = processedOnSecondRun.stream() + // + List offsetsFromSecondRunFromKey = processedOnSecondRun.stream() .map(PollContext::key) .collect(Collectors.toList()); - final List offsetsFromSecond = processedOnSecondRun.stream() + assertWithMessage("All keys should still exist") + .that(offsetsFromSecondRunFromKey) + .containsAtLeastElementsIn(processedOnFirstRun.stream().map(PollContext::key).collect(Collectors.toList())); + + // + List offsetsFromSecond = processedOnSecondRun.stream() .map(PollContext::offset) .collect(Collectors.toList()); - Truth.assertWithMessage("The offsets of the tombstone targets should not be read in second run") + assertWithMessage("The offsets of the tombstone targets should not be read in second run") .that(offsetsFromSecond) .containsNoneIn(tombstoneTargetOffsetsFromFirstRun); + } - final int expectedNumberOfRecordsProcessedSecondRun = TO_PRODUCE - tombStonedOffsetsFromKey.size(); - Truth.assertWithMessage("Processed on second run should be original sent, minus number of tombstones sent") - .that(processedOnSecondRun) - .hasSize(expectedNumberOfRecordsProcessedSecondRun); + private ArrayList> runPcUntilOffset(int offset) { + return runPcUntilOffset(offset, offset, UniSets.of()); } private static long getOffsetFromKey(String key) { @@ -150,13 +172,15 @@ private void setupCompacted() { @SneakyThrows private void triggerTombStoneProcessing() { - ThreadUtils.sleepSecondsLog(30); + // send a lot of messages to fill up segments + List keys = produceMessages(TO_PRODUCE * 2, "log-compaction-trigger-"); + // or wait? + ThreadUtils.sleepSecondsLog(20); } @SneakyThrows - private ArrayList sendRandomTombstones(List keys, int from, int to) { + private ArrayList sendRandomTombstones(List keys, int howMany) { var tombstoneKeys = new ArrayList(); - var howMany = (to - from) / 3; // fix randomness List> futures = JavaUtils.getRandom(keys, howMany).stream() .map((String key) -> { @@ -172,7 +196,12 @@ private ArrayList sendRandomTombstones(List keys, int from, int tombstoneOffsets.add(recordMetadata.offset()); } - log.debug("Generated tombstones: {}", tombstoneOffsets); + tombstoneKeys.sort(Comparator.comparingLong(PartitionStateCommittedOffsetIT::getOffsetFromKey)); + + log.debug("Keys to tombstone: {}\n" + + "Offsets of the generated tombstone: {}", + tombstoneKeys, + tombstoneOffsets); return tombstoneKeys; } @@ -221,7 +250,7 @@ private void runPcCheckStartIs(long targetStartOffset, long checkUpTo) { pc.close(); - Truth.assertWithMessage("Offset started as").that(lowest.get()).isEqualTo(targetStartOffset); + assertWithMessage("Offset started as").that(lowest.get()).isEqualTo(targetStartOffset); } @SneakyThrows @@ -237,27 +266,36 @@ private void closePC() { pc.close(); } - private ArrayList> runPcUntilOffset(long targetOffset) { - this.pc = super.getKcu().buildPc(PARTITION, GroupOption.NEW_GROUP); + private ArrayList> runPcUntilOffset(long succeedUpToOffset, long expectedProcessToOffset, Set exceptionsToSucceed) { + log.debug("Running PC until offset {}", succeedUpToOffset); + this.pc = super.getKcu().buildPc(UNORDERED, GroupOption.NEW_GROUP); - Queue> concurrentList = new ConcurrentLinkedQueue<>(); - AtomicLong current = new AtomicLong(); + SortedSet> seenOffsets = Collections.synchronizedSortedSet(new TreeSet<>(Comparator.comparingLong(PollContext::offset))); + AtomicLong succeededUpTo = new AtomicLong(); pc.subscribe(of(getTopic())); pc.poll(pollContext -> { + seenOffsets.add(pollContext); long thisOffset = pollContext.offset(); - if (thisOffset >= targetOffset) { + if (exceptionsToSucceed.contains(thisOffset)) { + log.debug("Exceptional offset {} succeeded", thisOffset); + } else if (thisOffset >= succeedUpToOffset) { log.debug("Failing on {}", thisOffset); + throw new RuntimeException("Failing on " + thisOffset); } else { - log.debug("Processed {}", thisOffset); - current.set(thisOffset); + succeededUpTo.set(thisOffset); + log.debug("Succeeded {}", thisOffset); } - concurrentList.add(pollContext); }); - Awaitility.await().untilAtomic(current, equalTo(targetOffset - 1)); - log.debug("Consumed up to {}", targetOffset); + Awaitility.await().untilAsserted(() -> { + assertThat(seenOffsets).isNotEmpty(); + assertThat(seenOffsets.last().offset()).isGreaterThan(expectedProcessToOffset - 2); + }); + log.debug("Consumed up to {}", succeedUpToOffset); + + pc.close(); - var sorted = new ArrayList<>(concurrentList); + var sorted = new ArrayList<>(seenOffsets); Collections.sort(sorted, Comparator.comparingLong(PollContext::offset)); return sorted; } diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java index 840f02b05..da68bb293 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java @@ -260,13 +260,17 @@ public List createTopics(int numTopics) { } public List produceMessages(String topicName, long numberToSend) throws InterruptedException, ExecutionException { + return produceMessages(topicName, numberToSend, ""); + } + + public List produceMessages(String topicName, long numberToSend, String prefix) throws InterruptedException, ExecutionException { log.info("Producing {} messages to {}", numberToSend, topicName); final List expectedKeys = new ArrayList<>(); List> sends = new ArrayList<>(); try (Producer kafkaProducer = createNewProducer(false)) { var mu = new ModelUtils(new PCModuleTestEnv()); - List> recs = mu.createProducerRecords(topicName, numberToSend); + List> recs = mu.createProducerRecords(topicName, numberToSend, prefix); for (var record : recs) { Future send = kafkaProducer.send(record, (meta, exception) -> { diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/ModelUtils.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/ModelUtils.java index 1197e1628..20c45e35c 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/ModelUtils.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/ModelUtils.java @@ -80,11 +80,14 @@ public ConsumerGroupMetadata consumerGroupMeta() { return new ConsumerGroupMetadata(groupId); } - public List> createProducerRecords(String topicName, long numberToSend) { + return createProducerRecords(topicName, numberToSend, ""); + } + + public List> createProducerRecords(String topicName, long numberToSend, String prefix) { List> recs = new ArrayList<>(); for (int i = 0; i < numberToSend; i++) { - String key = "key-" + i; + String key = prefix + "key-" + i; ProducerRecord record = new ProducerRecord<>(topicName, key, "value-" + i); recs.add(record); } diff --git a/parallel-consumer-core/src/test/resources/logback-test.xml b/parallel-consumer-core/src/test/resources/logback-test.xml index 0a9472e95..257e279ca 100644 --- a/parallel-consumer-core/src/test/resources/logback-test.xml +++ b/parallel-consumer-core/src/test/resources/logback-test.xml @@ -26,9 +26,14 @@ + + + + + - + From 1b73d18b2b643188b87f1b0cf7f8c17abde820a8 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Tue, 11 Oct 2022 21:27:13 +0100 Subject: [PATCH 22/33] run compaction on seperate broker --- .../BrokerIntegrationTest.java | 30 ++++++++++++------- .../PartitionStateCommittedOffsetIT.java | 21 +++++++++++-- 2 files changed, 39 insertions(+), 12 deletions(-) diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/BrokerIntegrationTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/BrokerIntegrationTest.java index 9cd3a103d..8a93a6fed 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/BrokerIntegrationTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/BrokerIntegrationTest.java @@ -12,6 +12,7 @@ import lombok.Getter; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.StringUtils; import org.apache.kafka.clients.admin.CreateTopicsResult; import org.apache.kafka.clients.admin.NewTopic; import org.junit.jupiter.api.AfterEach; @@ -49,16 +50,25 @@ public abstract class BrokerIntegrationTest { * https://www.testcontainers.org/test_framework_integration/manual_lifecycle_control/#singleton-containers * https://github.com/testcontainers/testcontainers-java/pull/1781 */ - public static KafkaContainer kafkaContainer = new KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:7.2.2")) - .withEnv("KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR", "1") //transaction.state.log.replication.factor - .withEnv("KAFKA_TRANSACTION_STATE_LOG_MIN_ISR", "1") //transaction.state.log.min.isr - .withEnv("KAFKA_TRANSACTION_STATE_LOG_NUM_PARTITIONS", "1") //transaction.state.log.num.partitions - //todo need to customise this for this test - // default produce batch size is - must be at least higher than it: 16KB - .withEnv("KAFKA_LOG_SEGMENT_BYTES", "40000") - // try to speed up initial consumer group formation - .withEnv("KAFKA_GROUP_INITIAL_REBALANCE_DELAY_MS", "500") // group.initial.rebalance.delay.ms default: 3000 - .withReuse(true); + public static KafkaContainer kafkaContainer = createKafkaContainer(null); + + public static KafkaContainer createKafkaContainer(String logSgmentSize) { + KafkaContainer base = new KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:7.2.2")) + .withEnv("KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR", "1") //transaction.state.log.replication.factor + .withEnv("KAFKA_TRANSACTION_STATE_LOG_MIN_ISR", "1") //transaction.state.log.min.isr + .withEnv("KAFKA_TRANSACTION_STATE_LOG_NUM_PARTITIONS", "1") //transaction.state.log.num.partitions + //todo need to customise this for this test + // default produce batch size is - must be at least higher than it: 16KB + // try to speed up initial consumer group formation + .withEnv("KAFKA_GROUP_INITIAL_REBALANCE_DELAY_MS", "500") // group.initial.rebalance.delay.ms default: 3000 + .withReuse(true); + + if (StringUtils.isNotBlank(logSgmentSize)) { + base = base.withEnv("KAFKA_LOG_SEGMENT_BYTES", logSgmentSize); + } + + return base; + } static { kafkaContainer.start(); diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java index d9328a0b3..84621fe76 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java @@ -9,6 +9,7 @@ import io.confluent.parallelconsumer.ParallelEoSStreamProcessor; import io.confluent.parallelconsumer.PollContext; import io.confluent.parallelconsumer.integrationTests.BrokerIntegrationTest; +import io.confluent.parallelconsumer.integrationTests.utils.KafkaClientUtils; import io.confluent.parallelconsumer.integrationTests.utils.KafkaClientUtils.GroupOption; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; @@ -24,6 +25,7 @@ import org.apache.kafka.common.config.TopicConfig; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.testcontainers.containers.KafkaContainer; import org.testcontainers.shaded.org.awaitility.Awaitility; import pl.tlinkowski.unij.api.UniMaps; import pl.tlinkowski.unij.api.UniSets; @@ -60,7 +62,7 @@ class PartitionStateCommittedOffsetIT extends BrokerIntegrationTest keys = produceMessages(TO_PRODUCE); final int UNTIL_OFFSET = TO_PRODUCE / 2; @@ -142,6 +157,8 @@ void compactedTopic() { assertWithMessage("The offsets of the tombstone targets should not be read in second run") .that(offsetsFromSecond) .containsNoneIn(tombstoneTargetOffsetsFromFirstRun); + + compactingBroker.close(); } private ArrayList> runPcUntilOffset(int offset) { @@ -175,7 +192,7 @@ private void triggerTombStoneProcessing() { // send a lot of messages to fill up segments List keys = produceMessages(TO_PRODUCE * 2, "log-compaction-trigger-"); // or wait? - ThreadUtils.sleepSecondsLog(20); + ThreadUtils.sleepSecondsLog(10); } @SneakyThrows From bd52b2cb6e531141a8064299d2fb3202fda95f2a Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Tue, 11 Oct 2022 22:35:40 +0100 Subject: [PATCH 23/33] compaction test review, fix higher / lower tests --- CHANGELOG.adoc | 3 +- .../AbstractParallelEoSStreamProcessor.java | 4 +- .../state/PartitionState.java | 35 ++++--- .../PartitionStateCommittedOffsetIT.java | 93 ++++++++++++------- .../utils/KafkaClientUtils.java | 1 + 5 files changed, 88 insertions(+), 48 deletions(-) diff --git a/CHANGELOG.adoc b/CHANGELOG.adoc index 5243de648..5043a8334 100644 --- a/CHANGELOG.adoc +++ b/CHANGELOG.adoc @@ -18,7 +18,8 @@ endif::[] === Fixes -* fixes #409: Truncate state when bootstrap polled offset higher or lower than committed and prune missing records (#425) +* fixes #409: Fixes support for compacted topics and when commit offset is reset +** Truncate state when bootstrap polled offset higher or lower than committed and prune missing records (#425) == 0.5.2.3 diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/AbstractParallelEoSStreamProcessor.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/AbstractParallelEoSStreamProcessor.java index 8ad18b983..893e8a457 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/AbstractParallelEoSStreamProcessor.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/AbstractParallelEoSStreamProcessor.java @@ -246,7 +246,9 @@ protected AbstractParallelEoSStreamProcessor(ParallelConsumerOptions newOp module.setParallelEoSStreamProcessor(this); - log.info("Confluent Parallel Consumer initialise... Options: {}", newOptions); + log.info("Confluent Parallel Consumer initialise... groupId: {}, Options: {}", + newOptions.getConsumer().groupMetadata().groupId(), + newOptions); options = newOptions; options.validate(); diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java index 12ced89b1..f114023ae 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java @@ -272,7 +272,7 @@ private void maybeTruncateBelowOrAbove(long polledOffset) { if (pollAboveExpected) { // previously committed offset record has been removed, or manual reset to higher offset detected - log.warn("Truncating state - removing records lower than {}. Offsets have been removed form the partition by the broker. Bootstrap polled {} but " + + log.warn("Truncating state - removing records lower than {}. Offsets have been removed from the partition by the broker. Bootstrap polled {} but " + "expected {} from loaded commit data. Could be caused by record retention or compaction.", polledOffset, polledOffset, @@ -472,7 +472,7 @@ protected void maybeTruncateOrPruneTrackedOffsets(EpochAndRecordsMap.Recor var records = polledRecordBatch.getRecords(); if (records.isEmpty()) { - log.warn("Polled an emtpy batch of records? {}", polledRecordBatch); + log.warn("Polled an empty batch of records? {}", polledRecordBatch); return; } @@ -489,25 +489,30 @@ protected void maybeTruncateOrPruneTrackedOffsets(EpochAndRecordsMap.Recor // for the incomplete offsets within this range of poll batch var incompletesWithinPolledBatch = incompleteOffsets.subSet(low, true, high, true); - + var offsetsToRemoveFromTracking = new ArrayList(); for (long incompleteOffset : incompletesWithinPolledBatch) { boolean offsetMissingFromPolledRecords = !polledOffsetLookup.contains(incompleteOffset); + if (offsetMissingFromPolledRecords) { - log.warn("Offset {} has been removed from partition {} (as it has not been returned within a polled batch " + - "which should have contained it - batch offset range is {} to {}), so it must be removed " + - "from tracking state, as it will never be sent again to be retried. " + - "This can be caused by PC rebalancing across a partition which has been compacted on offsets above the committed " + - "base offset, after initial load and before a rebalance.", - incompleteOffset, - getTp(), - low, - high - ); - boolean removedCheck = incompleteOffsets.remove(incompleteOffset); - assert removedCheck; + offsetsToRemoveFromTracking.add(incompleteOffset); // don't need to remove it from the #commitQueue, as it would never have been added } } + if (!offsetsToRemoveFromTracking.isEmpty()) { + log.warn("Offsets {} have been removed from partition {} (as they were not been returned within a polled batch " + + "which should have contained them - batch offset range is {} to {}), so they be removed " + + "from tracking state, as they will never be sent again to be retried. " + + "This can be caused by PC rebalancing across a partition which has been compacted on offsets above the committed " + + "base offset, after initial load and before a rebalance.", + offsetsToRemoveFromTracking, + getTp(), + low, + high + ); + boolean removedCheck = incompleteOffsets.removeAll(offsetsToRemoveFromTracking); + assert removedCheck; + } + } } diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java index 84621fe76..546207c76 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java @@ -80,7 +80,7 @@ void compactedTopic() { // setup our extra special compacting broker KafkaContainer compactingBroker = null; { - KafkaContainer compactingBroker = BrokerIntegrationTest.createKafkaContainer("40000"); + compactingBroker = BrokerIntegrationTest.createKafkaContainer("40000"); compactingBroker.start(); kcu = new KafkaClientUtils(compactingBroker); kcu.open(); @@ -104,39 +104,39 @@ void compactedTopic() { closePC(); // - ArrayList tombStonedKeysRaw = sendRandomTombstones(keys, TO_PRODUCE); - Set tombStonedKeys = new HashSet<>(tombStonedKeysRaw); + ArrayList compactionKeysRaw = sendRandomCompactionRecords(keys, TO_PRODUCE); + Set compactedKeys = new HashSet<>(compactionKeysRaw); var processedOnFirstRunWithTombstoneTargetsRemoved = processedOnFirstRun.stream() - .filter(context -> !tombStonedKeys.contains(context.key())) + .filter(context -> !compactedKeys.contains(context.key())) .map(PollContext::key) .collect(Collectors.toList()); - var firstRunPartitioned = processedOnFirstRun.stream().collect(Collectors.partitioningBy(context -> tombStonedKeys.contains(context.key()))); + var firstRunPartitioned = processedOnFirstRun.stream().collect(Collectors.partitioningBy(context -> compactedKeys.contains(context.key()))); var saved = firstRunPartitioned.get(Boolean.FALSE); - var tombstoned = firstRunPartitioned.get(Boolean.TRUE); + var compacted = firstRunPartitioned.get(Boolean.TRUE); log.debug("kept offsets: {}", saved.stream().mapToLong(PollContext::offset).boxed().collect(Collectors.toList())); log.debug("kept keys: {}", saved.stream().map(PollContext::key).collect(Collectors.toList())); - log.debug("tombstoned offsets: {}", tombstoned.stream().map(PollContext::key).collect(Collectors.toList())); - log.debug("tombstoned keys: {}", tombstoned.stream().mapToLong(PollContext::offset).boxed().collect(Collectors.toList())); + log.debug("compacted offsets: {}", compacted.stream().map(PollContext::key).collect(Collectors.toList())); + log.debug("compacted keys: {}", compacted.stream().mapToLong(PollContext::offset).boxed().collect(Collectors.toList())); - var tombstoneTargetOffsetsFromFirstRun = tombstoned.stream() - .filter(context -> tombStonedKeys.contains(context.key())) + var tombstoneTargetOffsetsFromFirstRun = compacted.stream() + .filter(context -> compactedKeys.contains(context.key())) .map(PollContext::offset) .collect(Collectors.toList()); - var tombStonedOffsetsFromKey = tombStonedKeys.stream() + var tombStonedOffsetsFromKey = compactedKeys.stream() .map(PartitionStateCommittedOffsetIT::getOffsetFromKey).collect(Collectors.toList()); - log.debug("First run produced, with tombstone targets removed: {}", processedOnFirstRunWithTombstoneTargetsRemoved); + log.debug("First run produced, with compaction targets removed: {}", processedOnFirstRunWithTombstoneTargetsRemoved); // triggerTombStoneProcessing(); // The offsets of the tombstone targets should not be read in second run final int expectedTotalNumberRecordsProduced = TO_PRODUCE + tombStonedOffsetsFromKey.size(); - final int expectedOffsetProcessedToSecondRun = TO_PRODUCE + tombStonedKeys.size(); - var processedOnSecondRun = runPcUntilOffset(expectedOffsetProcessedToSecondRun).stream() + final int expectedOffsetProcessedToSecondRun = TO_PRODUCE + compactedKeys.size(); + var processedOnSecondRun = runPcUntilOffset(expectedOffsetProcessedToSecondRun, GroupOption.REUSE_GROUP).stream() .filter(recordContexts -> !recordContexts.key().contains("compaction-trigger")) .collect(Collectors.toList()); @@ -145,16 +145,25 @@ void compactedTopic() { .map(PollContext::key) .collect(Collectors.toList()); - assertWithMessage("All keys should still exist") - .that(offsetsFromSecondRunFromKey) - .containsAtLeastElementsIn(processedOnFirstRun.stream().map(PollContext::key).collect(Collectors.toList())); +// assertWithMessage("All keys should still exist") +// .that(offsetsFromSecondRunFromKey) +// .containsAtLeastElementsIn(processedOnFirstRun.stream().map(PollContext::key).collect(Collectors.toList())); // List offsetsFromSecond = processedOnSecondRun.stream() .map(PollContext::offset) .collect(Collectors.toList()); - assertWithMessage("The offsets of the tombstone targets should not be read in second run") +// assertWithMessage("The offsets of the tombstone targets should not be read in second run") +// .that(offsetsFromSecond) +// .containsNoneIn(tombstoneTargetOffsetsFromFirstRun); + + assertWithMessage("Finish reading rest of records from %s to %s", + UNTIL_OFFSET, + TO_PRODUCE) + .that(processedOnSecondRun.size()).isGreaterThan(TO_PRODUCE - UNTIL_OFFSET); + + assertWithMessage("Off the offsets read on the second run, offsets that were compacted (below the initial produce target) should now be removed, as they were replaced with newer ones.") .that(offsetsFromSecond) .containsNoneIn(tombstoneTargetOffsetsFromFirstRun); @@ -162,7 +171,11 @@ void compactedTopic() { } private ArrayList> runPcUntilOffset(int offset) { - return runPcUntilOffset(offset, offset, UniSets.of()); + return runPcUntilOffset(offset, offset, UniSets.of(), GroupOption.NEW_GROUP); + } + + private ArrayList> runPcUntilOffset(int offset, GroupOption reuseGroup) { + return runPcUntilOffset(Long.MAX_VALUE, offset, UniSets.of(), reuseGroup); } private static long getOffsetFromKey(String key) { @@ -192,17 +205,19 @@ private void triggerTombStoneProcessing() { // send a lot of messages to fill up segments List keys = produceMessages(TO_PRODUCE * 2, "log-compaction-trigger-"); // or wait? - ThreadUtils.sleepSecondsLog(10); + final int pauseSeconds = 20; + log.info("Pausing for {} seconds to allow for compaction", pauseSeconds); + ThreadUtils.sleepSecondsLog(pauseSeconds); } @SneakyThrows - private ArrayList sendRandomTombstones(List keys, int howMany) { + private ArrayList sendRandomCompactionRecords(List keys, int howMany) { var tombstoneKeys = new ArrayList(); // fix randomness List> futures = JavaUtils.getRandom(keys, howMany).stream() .map((String key) -> { tombstoneKeys.add(key); - var tombstone = new ProducerRecord<>(getTopic(), key, "tombstone"); + var tombstone = new ProducerRecord<>(getTopic(), key, "compactor"); return getKcu().getProducer() .send(tombstone); }) @@ -235,7 +250,7 @@ void committedOffsetLower() { final int moveToOffset = 25; - moveCommittedOffset(moveToOffset); + moveCommittedOffset(kcu.getGroupId(), moveToOffset); runPcCheckStartIs(moveToOffset, TO_PRODUCE); } @@ -246,8 +261,8 @@ void committedOffsetLower() { * @param targetStartOffset the offset to check that PC starts at * @param checkUpTo the offset to run the PC until, while checking for the start offset */ - private void runPcCheckStartIs(long targetStartOffset, long checkUpTo) { - this.pc = super.getKcu().buildPc(PARTITION); + private void runPcCheckStartIs(long targetStartOffset, long checkUpTo, GroupOption groupOption) { + this.pc = super.getKcu().buildPc(PARTITION, groupOption); pc.subscribe(of(getTopic())); AtomicLong lowest = new AtomicLong(Long.MAX_VALUE); @@ -270,9 +285,13 @@ private void runPcCheckStartIs(long targetStartOffset, long checkUpTo) { assertWithMessage("Offset started as").that(lowest.get()).isEqualTo(targetStartOffset); } + private void moveCommittedOffset(int moveToOffset) { + moveCommittedOffset(groupId, moveToOffset); + } + @SneakyThrows - private void moveCommittedOffset(long offset) { - log.debug("Moving offset to {}", offset); + private void moveCommittedOffset(String groupId, long offset) { + log.debug("Moving offset of {} to {}", groupId, offset); var data = UniMaps.of(tp, new OffsetAndMetadata(offset)); var result = ac.alterConsumerGroupOffsets(groupId, data); result.all().get(5, SECONDS); @@ -283,9 +302,14 @@ private void closePC() { pc.close(); } + private ArrayList> runPcUntilOffset(long succeedUpToOffset, long expectedProcessToOffset, Set exceptionsToSucceed) { + return runPcUntilOffset(succeedUpToOffset, expectedProcessToOffset, exceptionsToSucceed, GroupOption.NEW_GROUP); + } + + private ArrayList> runPcUntilOffset(long succeedUpToOffset, long expectedProcessToOffset, Set exceptionsToSucceed, GroupOption newGroup) { log.debug("Running PC until offset {}", succeedUpToOffset); - this.pc = super.getKcu().buildPc(UNORDERED, GroupOption.NEW_GROUP); + this.pc = super.getKcu().buildPc(UNORDERED, newGroup); SortedSet> seenOffsets = Collections.synchronizedSortedSet(new TreeSet<>(Comparator.comparingLong(PollContext::offset))); AtomicLong succeededUpTo = new AtomicLong(); @@ -300,7 +324,7 @@ private ArrayList> runPcUntilOffset(long succeedUpTo throw new RuntimeException("Failing on " + thisOffset); } else { succeededUpTo.set(thisOffset); - log.debug("Succeeded {}", thisOffset); + log.debug("Succeeded {}: {}", thisOffset, pollContext.getSingleRecord()); } }); @@ -308,7 +332,7 @@ private ArrayList> runPcUntilOffset(long succeedUpTo assertThat(seenOffsets).isNotEmpty(); assertThat(seenOffsets.last().offset()).isGreaterThan(expectedProcessToOffset - 2); }); - log.debug("Consumed up to {}", succeedUpToOffset); + log.debug("Consumed up to {}", seenOffsets.last().offset()); pc.close(); @@ -331,9 +355,16 @@ void committedOffsetHigher() { final int moveToOffset = 75; - moveCommittedOffset(moveToOffset); + // reslolve groupId mess + moveCommittedOffset(kcu.getGroupId(), moveToOffset); runPcCheckStartIs(moveToOffset, quantity); + var gkcu5 = kcu.getConsumer().groupMetadata().groupId(); + + } + + private void runPcCheckStartIs(int targetStartOffset, int checkUpTo) { + runPcCheckStartIs(targetStartOffset, checkUpTo, GroupOption.REUSE_GROUP); } /** diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java index da68bb293..01467e9fe 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java @@ -73,6 +73,7 @@ class PCVersion { @Getter private AdminClient admin; + @Getter private String groupId = GROUP_ID_PREFIX + nextInt(); /** From 97f3c438f2b7054f05b55f5bcbf0d91add724323 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Wed, 12 Oct 2022 00:18:47 +0100 Subject: [PATCH 24/33] offset removed test works for RESET == EARLIEST --- .../state/PartitionState.java | 2 +- .../BrokerIntegrationTest.java | 9 +- .../PartitionStateCommittedOffsetIT.java | 121 ++++++++++++++---- .../utils/KafkaClientUtils.java | 8 +- .../src/test/resources/logback-test.xml | 2 +- 5 files changed, 109 insertions(+), 33 deletions(-) diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java index f114023ae..01b661ddc 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java @@ -272,7 +272,7 @@ private void maybeTruncateBelowOrAbove(long polledOffset) { if (pollAboveExpected) { // previously committed offset record has been removed, or manual reset to higher offset detected - log.warn("Truncating state - removing records lower than {}. Offsets have been removed from the partition by the broker. Bootstrap polled {} but " + + log.warn("Truncating state - removing records lower than {}. Offsets have been removed from the partition by the broker or committed offset has been raised. Bootstrap polled {} but " + "expected {} from loaded commit data. Could be caused by record retention or compaction.", polledOffset, polledOffset, diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/BrokerIntegrationTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/BrokerIntegrationTest.java index 8a93a6fed..50f5a5b6c 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/BrokerIntegrationTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/BrokerIntegrationTest.java @@ -25,6 +25,7 @@ import java.util.List; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; import static org.apache.commons.lang3.RandomUtils.nextInt; import static org.assertj.core.api.Assertions.assertThat; @@ -52,7 +53,7 @@ public abstract class BrokerIntegrationTest { */ public static KafkaContainer kafkaContainer = createKafkaContainer(null); - public static KafkaContainer createKafkaContainer(String logSgmentSize) { + public static KafkaContainer createKafkaContainer(String logSegmentSize) { KafkaContainer base = new KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:7.2.2")) .withEnv("KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR", "1") //transaction.state.log.replication.factor .withEnv("KAFKA_TRANSACTION_STATE_LOG_MIN_ISR", "1") //transaction.state.log.min.isr @@ -63,8 +64,8 @@ public static KafkaContainer createKafkaContainer(String logSgmentSize) { .withEnv("KAFKA_GROUP_INITIAL_REBALANCE_DELAY_MS", "500") // group.initial.rebalance.delay.ms default: 3000 .withReuse(true); - if (StringUtils.isNotBlank(logSgmentSize)) { - base = base.withEnv("KAFKA_LOG_SEGMENT_BYTES", logSgmentSize); + if (StringUtils.isNotBlank(logSegmentSize)) { + base = base.withEnv("KAFKA_LOG_SEGMENT_BYTES", logSegmentSize); } return base; @@ -114,7 +115,7 @@ protected CreateTopicsResult ensureTopic(String topic, int numPartitions) { NewTopic e1 = new NewTopic(topic, numPartitions, (short) 1); CreateTopicsResult topics = kcu.getAdmin().createTopics(UniLists.of(e1)); try { - Void all = topics.all().get(); + Void all = topics.all().get(1, TimeUnit.SECONDS); } catch (ExecutionException e) { // fine } catch (Exception e) { diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java index 546207c76..b5da46b05 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java @@ -6,16 +6,21 @@ import io.confluent.csid.utils.JavaUtils; import io.confluent.csid.utils.ThreadUtils; +import io.confluent.parallelconsumer.ManagedTruth; import io.confluent.parallelconsumer.ParallelEoSStreamProcessor; import io.confluent.parallelconsumer.PollContext; import io.confluent.parallelconsumer.integrationTests.BrokerIntegrationTest; import io.confluent.parallelconsumer.integrationTests.utils.KafkaClientUtils; import io.confluent.parallelconsumer.integrationTests.utils.KafkaClientUtils.GroupOption; +import lombok.NonNull; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.AlterConfigOp; import org.apache.kafka.clients.admin.ConfigEntry; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; @@ -30,8 +35,11 @@ import pl.tlinkowski.unij.api.UniMaps; import pl.tlinkowski.unij.api.UniSets; +import java.time.Duration; import java.util.*; +import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; @@ -41,7 +49,7 @@ import static io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder.PARTITION; import static io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder.UNORDERED; import static java.util.concurrent.TimeUnit.SECONDS; -import static org.testcontainers.shaded.org.hamcrest.Matchers.equalTo; +import static org.testcontainers.shaded.org.hamcrest.Matchers.greaterThanOrEqualTo; import static pl.tlinkowski.unij.api.UniLists.of; /** @@ -77,18 +85,7 @@ void setup() { */ @Test void compactedTopic() { - // setup our extra special compacting broker - KafkaContainer compactingBroker = null; - { - compactingBroker = BrokerIntegrationTest.createKafkaContainer("40000"); - compactingBroker.start(); - kcu = new KafkaClientUtils(compactingBroker); - kcu.open(); - - setup(); - } - - setupCompacted(); + KafkaContainer compactingBroker = setupCompactingKafkaBroker(); var TO_PRODUCE = this.TO_PRODUCE / 10; @@ -170,6 +167,26 @@ void compactedTopic() { compactingBroker.close(); } + /** + * Setup our extra special compacting broker + */ + @NonNull + private KafkaContainer setupCompactingKafkaBroker() { + KafkaContainer compactingBroker = null; + { + compactingBroker = BrokerIntegrationTest.createKafkaContainer("40000"); + compactingBroker.start(); + kcu = new KafkaClientUtils(compactingBroker); + kcu.open(); + + setup(); + } + + setupCompacted(); + + return compactingBroker; + } + private ArrayList> runPcUntilOffset(int offset) { return runPcUntilOffset(offset, offset, UniSets.of(), GroupOption.NEW_GROUP); } @@ -278,7 +295,7 @@ private void runPcCheckStartIs(long targetStartOffset, long checkUpTo, GroupOpti } }); - Awaitility.await().untilAtomic(highest, equalTo(checkUpTo - 1)); + Awaitility.await().untilAtomic(highest, greaterThanOrEqualTo(checkUpTo - 1)); pc.close(); @@ -312,7 +329,7 @@ private ArrayList> runPcUntilOffset(long succeedUpTo this.pc = super.getKcu().buildPc(UNORDERED, newGroup); SortedSet> seenOffsets = Collections.synchronizedSortedSet(new TreeSet<>(Comparator.comparingLong(PollContext::offset))); - AtomicLong succeededUpTo = new AtomicLong(); + SortedSet> succeededOffsets = Collections.synchronizedSortedSet(new TreeSet<>(Comparator.comparingLong(PollContext::offset))); pc.subscribe(of(getTopic())); pc.poll(pollContext -> { seenOffsets.add(pollContext); @@ -323,8 +340,8 @@ private ArrayList> runPcUntilOffset(long succeedUpTo log.debug("Failing on {}", thisOffset); throw new RuntimeException("Failing on " + thisOffset); } else { - succeededUpTo.set(thisOffset); log.debug("Succeeded {}: {}", thisOffset, pollContext.getSingleRecord()); + succeededOffsets.add(pollContext); } }); @@ -332,7 +349,7 @@ private ArrayList> runPcUntilOffset(long succeedUpTo assertThat(seenOffsets).isNotEmpty(); assertThat(seenOffsets.last().offset()).isGreaterThan(expectedProcessToOffset - 2); }); - log.debug("Consumed up to {}", seenOffsets.last().offset()); + log.debug("Succeeded up to: {} Consumed up to {}", succeededOffsets.last().offset(), seenOffsets.last().offset()); pc.close(); @@ -373,23 +390,75 @@ private void runPcCheckStartIs(int targetStartOffset, int checkUpTo) { */ @Test void committedOffsetRemoved() { - produceMessages(TO_PRODUCE); + try (KafkaContainer compactingKafkaBroker = setupCompactingKafkaBroker()) { +// KafkaContainer compactingKafkaBroker = setupCompactingKafkaBroker(); + log.debug("Compacting broker started {}", compactingKafkaBroker); - final int END_OFFSET = 50; - runPcUntilOffset(END_OFFSET); + var producedCount = produceMessages(TO_PRODUCE).size(); - closePC(); + final int END_OFFSET = 50; + groupId = getKcu().getGroupId(); + runPcUntilOffset(END_OFFSET); - causeCommittedOffsetToBeRemoved(END_OFFSET); + closePC(); - produceMessages(TO_PRODUCE); + final String compactedKey = "key-50"; - final int TOTAL = TO_PRODUCE * 2; - runPcCheckStartIs(END_OFFSET + 1, TOTAL); + // before compaction + checkHowManyRecordsWithKeyPresent(compactedKey, 1, TO_PRODUCE); + + causeCommittedOffsetToBeRemoved(END_OFFSET); + + // after compaction + checkHowManyRecordsWithKeyPresent(compactedKey, 1, TO_PRODUCE + 2); + + producedCount = producedCount + produceMessages(TO_PRODUCE).size(); + + final int TOTAL = TO_PRODUCE * 2; + + getKcu().setGroupId(groupId); + final int EXPECTED_RESET_OFFSET = 0; + runPcCheckStartIs(EXPECTED_RESET_OFFSET, producedCount); + } } + private void checkHowManyRecordsWithKeyPresent(String keyToSearchFor, int expectedQuantityToFind, long upToOffset) { + log.debug("Looking for {} records with key {} up to offset {}", expectedQuantityToFind, keyToSearchFor, upToOffset); + + KafkaConsumer newConsumer = kcu.createNewConsumer(GroupOption.NEW_GROUP); + newConsumer.subscribe(of(getTopic())); + newConsumer.seekToBeginning(UniSets.of()); + final List> records = new ArrayList<>(); + long highest = -1; + while (highest < upToOffset - 1) { + ConsumerRecords poll = newConsumer.poll(Duration.ofSeconds(1)); + records.addAll(poll.records(tp)); + var lastOpt = getLast(records); + if (lastOpt.isPresent()) { + highest = lastOpt.get().offset(); + } + } + + var collect = records.stream().filter(value -> value.key().equals(keyToSearchFor)).collect(Collectors.toList()); + ManagedTruth.assertThat(collect).hasSize(expectedQuantityToFind); + } + + @SneakyThrows private void causeCommittedOffsetToBeRemoved(long offset) { - throw new RuntimeException(); + sendCompactionKeyForOffset(offset); + sendCompactionKeyForOffset(offset + 1); + + checkHowManyRecordsWithKeyPresent("key-" + offset, 2, TO_PRODUCE + 2); + + triggerTombStoneProcessing(); + } + + private void sendCompactionKeyForOffset(long offset) throws InterruptedException, ExecutionException, TimeoutException { + String key = "key-" + offset; + ProducerRecord compactingRecord = new ProducerRecord<>(getTopic(), 0, key, "compactor"); + getKcu().getProducer() + .send(compactingRecord) + .get(1, SECONDS); } @Test diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java index 01467e9fe..329f669f9 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java @@ -11,6 +11,7 @@ import io.confluent.parallelconsumer.internal.PCModuleTestEnv; import io.confluent.parallelconsumer.state.ModelUtils; import lombok.Getter; +import lombok.Setter; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import one.util.streamex.IntStreamEx; @@ -74,6 +75,7 @@ class PCVersion { private AdminClient admin; @Getter + @Setter private String groupId = GROUP_ID_PREFIX + nextInt(); /** @@ -110,12 +112,16 @@ private Properties setupConsumerProps(String groupIdToUse) { // consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, groupIdToUse); - consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, OffsetResetStrategy.EARLIEST.name().toLowerCase()); consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false); consumerProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); consumerProps.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.toString().toLowerCase(Locale.ROOT)); + // Reset + consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, OffsetResetStrategy.EARLIEST.name().toLowerCase()); +// consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, OffsetResetStrategy.LATEST.name().toLowerCase()); +// consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, OffsetResetStrategy.NONE.name().toLowerCase()); + // // consumerProps.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 10); // consumerProps.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 100); diff --git a/parallel-consumer-core/src/test/resources/logback-test.xml b/parallel-consumer-core/src/test/resources/logback-test.xml index 257e279ca..8fce65ad1 100644 --- a/parallel-consumer-core/src/test/resources/logback-test.xml +++ b/parallel-consumer-core/src/test/resources/logback-test.xml @@ -33,7 +33,7 @@ - + From 9bcb14e588a9eb3fbc42e4bfe2ae8b6927cea59c Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Wed, 12 Oct 2022 01:48:49 +0100 Subject: [PATCH 25/33] PartitionStateCommittedOffsetIT all green --- .../PartitionStateCommittedOffsetIT.java | 187 ++++++++++-------- .../utils/KafkaClientUtils.java | 7 +- 2 files changed, 113 insertions(+), 81 deletions(-) diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java index b5da46b05..0814e6a2d 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java @@ -4,6 +4,7 @@ * Copyright (C) 2020-2022 Confluent, Inc. */ +import com.google.common.truth.Truth; import io.confluent.csid.utils.JavaUtils; import io.confluent.csid.utils.ThreadUtils; import io.confluent.parallelconsumer.ManagedTruth; @@ -15,13 +16,11 @@ import lombok.NonNull; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.AlterConfigOp; import org.apache.kafka.clients.admin.ConfigEntry; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.*; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.KafkaFuture; @@ -30,8 +29,11 @@ import org.apache.kafka.common.config.TopicConfig; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.shaded.org.awaitility.Awaitility; +import pl.tlinkowski.unij.api.UniLists; import pl.tlinkowski.unij.api.UniMaps; import pl.tlinkowski.unij.api.UniSets; @@ -49,7 +51,6 @@ import static io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder.PARTITION; import static io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder.UNORDERED; import static java.util.concurrent.TimeUnit.SECONDS; -import static org.testcontainers.shaded.org.hamcrest.Matchers.greaterThanOrEqualTo; import static pl.tlinkowski.unij.api.UniLists.of; /** @@ -62,6 +63,7 @@ @Slf4j class PartitionStateCommittedOffsetIT extends BrokerIntegrationTest { + public static final OffsetResetStrategy DEFAULT_OFFSET_RESET_POLICY = OffsetResetStrategy.EARLIEST; AdminClient ac; String groupId; @@ -72,6 +74,8 @@ class PartitionStateCommittedOffsetIT extends BrokerIntegrationTest> runPcUntilOffset(int offset) { - return runPcUntilOffset(offset, offset, UniSets.of(), GroupOption.NEW_GROUP); + private List> runPcUntilOffset(int offset) { + return runPcUntilOffset(DEFAULT_OFFSET_RESET_POLICY, offset); + } + + private List> runPcUntilOffset(OffsetResetStrategy offsetResetPolicy, int offset) { + return runPcUntilOffset(offsetResetPolicy, offset, offset, UniSets.of(), GroupOption.NEW_GROUP); } - private ArrayList> runPcUntilOffset(int offset, GroupOption reuseGroup) { - return runPcUntilOffset(Long.MAX_VALUE, offset, UniSets.of(), reuseGroup); + private List> runPcUntilOffset(int offset, GroupOption reuseGroup) { + return runPcUntilOffset(DEFAULT_OFFSET_RESET_POLICY, Long.MAX_VALUE, offset, UniSets.of(), reuseGroup); } private static long getOffsetFromKey(String key) { @@ -218,13 +226,15 @@ private void setupCompacted() { } @SneakyThrows - private void triggerTombStoneProcessing() { + private List triggerTombStoneProcessing() { // send a lot of messages to fill up segments List keys = produceMessages(TO_PRODUCE * 2, "log-compaction-trigger-"); // or wait? final int pauseSeconds = 20; log.info("Pausing for {} seconds to allow for compaction", pauseSeconds); ThreadUtils.sleepSecondsLog(pauseSeconds); + + return keys; } @SneakyThrows @@ -278,6 +288,7 @@ void committedOffsetLower() { * @param targetStartOffset the offset to check that PC starts at * @param checkUpTo the offset to run the PC until, while checking for the start offset */ + @SneakyThrows private void runPcCheckStartIs(long targetStartOffset, long checkUpTo, GroupOption groupOption) { this.pc = super.getKcu().buildPc(PARTITION, groupOption); pc.subscribe(of(getTopic())); @@ -290,16 +301,31 @@ private void runPcCheckStartIs(long targetStartOffset, long checkUpTo, GroupOpti if (thisOffset < lowest.get()) { log.debug("Found lowest offset {}", thisOffset); lowest.set(thisOffset); - } else { + } else if (thisOffset > highest.get()) { highest.set(thisOffset); } }); - Awaitility.await().untilAtomic(highest, greaterThanOrEqualTo(checkUpTo - 1)); + // + AtomicLong bumpersSent = new AtomicLong(); + Awaitility.await().untilAsserted(() -> { + // in case we're at the end of the topic, add some messages to make sure we get a poll response + getKcu().produceMessages(getTopic(), 1, "poll-bumper"); + bumpersSent.incrementAndGet(); + + assertWithMessage("Highest seen offset") + .that(highest.get()) + .isAtLeast(checkUpTo - 1); + }); pc.close(); - assertWithMessage("Offset started as").that(lowest.get()).isEqualTo(targetStartOffset); + var adjustExpected = switch (offsetResetStrategy) { + case EARLIEST -> targetStartOffset; + case LATEST -> targetStartOffset + 1; + case NONE -> throw new IllegalStateException("NONE not supported"); + }; + assertWithMessage("Offset started as").that(lowest.get()).isEqualTo(adjustExpected); } private void moveCommittedOffset(int moveToOffset) { @@ -320,17 +346,20 @@ private void closePC() { } - private ArrayList> runPcUntilOffset(long succeedUpToOffset, long expectedProcessToOffset, Set exceptionsToSucceed) { - return runPcUntilOffset(succeedUpToOffset, expectedProcessToOffset, exceptionsToSucceed, GroupOption.NEW_GROUP); + private List> runPcUntilOffset(long succeedUpToOffset, long expectedProcessToOffset, Set exceptionsToSucceed) { + return runPcUntilOffset(DEFAULT_OFFSET_RESET_POLICY, succeedUpToOffset, expectedProcessToOffset, exceptionsToSucceed, GroupOption.NEW_GROUP); } - private ArrayList> runPcUntilOffset(long succeedUpToOffset, long expectedProcessToOffset, Set exceptionsToSucceed, GroupOption newGroup) { - log.debug("Running PC until offset {}", succeedUpToOffset); + @SneakyThrows + private List> runPcUntilOffset(OffsetResetStrategy offsetResetPolicy, long succeedUpToOffset, long expectedProcessToOffset, Set exceptionsToSucceed, GroupOption newGroup) { + log.debug("Running PC until at least offset {}", succeedUpToOffset); this.pc = super.getKcu().buildPc(UNORDERED, newGroup); SortedSet> seenOffsets = Collections.synchronizedSortedSet(new TreeSet<>(Comparator.comparingLong(PollContext::offset))); SortedSet> succeededOffsets = Collections.synchronizedSortedSet(new TreeSet<>(Comparator.comparingLong(PollContext::offset))); + pc.subscribe(of(getTopic())); + pc.poll(pollContext -> { seenOffsets.add(pollContext); long thisOffset = pollContext.offset(); @@ -345,17 +374,39 @@ private ArrayList> runPcUntilOffset(long succeedUpTo } }); - Awaitility.await().untilAsserted(() -> { - assertThat(seenOffsets).isNotEmpty(); - assertThat(seenOffsets.last().offset()).isGreaterThan(expectedProcessToOffset - 2); - }); - log.debug("Succeeded up to: {} Consumed up to {}", succeededOffsets.last().offset(), seenOffsets.last().offset()); + // give first poll a chance to run + ThreadUtils.sleepSecondsLog(1); + + getKcu().produceMessages(getTopic(), 1, "poll-bumper"); + + if (offsetResetPolicy.equals(OffsetResetStrategy.NONE)) { + Awaitility.await().untilAsserted(() -> { + assertWithMessage("PC crashed / failed fast").that(pc.isClosedOrFailed()).isTrue(); + assertThat(pc.getFailureCause()).hasCauseThat().hasMessageThat().contains("Error in BrokerPollSystem system"); + var stackTrace = ExceptionUtils.getStackTrace(pc.getFailureCause()); + Truth.assertThat(stackTrace).contains("Undefined offset with no reset policy for partitions"); + }); + return UniLists.of(); + } else { + + Awaitility.await() + .failFast(pc::isClosedOrFailed) + .untilAsserted(() -> { + assertThat(seenOffsets).isNotEmpty(); + assertThat(seenOffsets.last().offset()).isGreaterThan(expectedProcessToOffset - 2); + }); + + if (!succeededOffsets.isEmpty()) { + log.debug("Succeeded up to: {}", succeededOffsets.last().offset()); + } + log.debug("Consumed up to {}", seenOffsets.last().offset()); - pc.close(); + pc.close(); - var sorted = new ArrayList<>(seenOffsets); - Collections.sort(sorted, Comparator.comparingLong(PollContext::offset)); - return sorted; + var sorted = new ArrayList<>(seenOffsets); + Collections.sort(sorted, Comparator.comparingLong(PollContext::offset)); + return sorted; + } } /** @@ -388,18 +439,30 @@ private void runPcCheckStartIs(int targetStartOffset, int checkUpTo) { * CG offset has disappeared - committed offset hasn't been changed, but broker gives us a bootstrap poll result * with a higher offset than expected. Could be caused by retention period, or compaction. */ - @Test - void committedOffsetRemoved() { + @EnumSource(value = OffsetResetStrategy.class) + @ParameterizedTest + void committedOffsetRemoved(OffsetResetStrategy offsetResetPolicy) { + this.offsetResetStrategy = offsetResetPolicy; try (KafkaContainer compactingKafkaBroker = setupCompactingKafkaBroker()) { -// KafkaContainer compactingKafkaBroker = setupCompactingKafkaBroker(); - log.debug("Compacting broker started {}", compactingKafkaBroker); + log.debug("Compacting broker started {}", compactingKafkaBroker.getBootstrapServers()); + + kcu = new KafkaClientUtils(compactingKafkaBroker); + kcu.setOffsetResetPolicy(offsetResetPolicy); + kcu.open(); var producedCount = produceMessages(TO_PRODUCE).size(); final int END_OFFSET = 50; groupId = getKcu().getGroupId(); - runPcUntilOffset(END_OFFSET); + runPcUntilOffset(offsetResetPolicy, END_OFFSET); + + // + if (offsetResetPolicy.equals(OffsetResetStrategy.NONE)) { + // test finished + return; + } + // closePC(); final String compactedKey = "key-50"; @@ -407,17 +470,19 @@ void committedOffsetRemoved() { // before compaction checkHowManyRecordsWithKeyPresent(compactedKey, 1, TO_PRODUCE); - causeCommittedOffsetToBeRemoved(END_OFFSET); + final int triggerRecordsCount = causeCommittedOffsetToBeRemoved(END_OFFSET); // after compaction checkHowManyRecordsWithKeyPresent(compactedKey, 1, TO_PRODUCE + 2); - producedCount = producedCount + produceMessages(TO_PRODUCE).size(); - - final int TOTAL = TO_PRODUCE * 2; + producedCount = producedCount + triggerRecordsCount; + final int EXPECTED_RESET_OFFSET = switch (offsetResetPolicy) { + case EARLIEST -> 0; + case LATEST -> producedCount + 4; + case NONE -> -1; // will crash / fail fast + }; getKcu().setGroupId(groupId); - final int EXPECTED_RESET_OFFSET = 0; runPcCheckStartIs(EXPECTED_RESET_OFFSET, producedCount); } } @@ -426,8 +491,10 @@ private void checkHowManyRecordsWithKeyPresent(String keyToSearchFor, int expect log.debug("Looking for {} records with key {} up to offset {}", expectedQuantityToFind, keyToSearchFor, upToOffset); KafkaConsumer newConsumer = kcu.createNewConsumer(GroupOption.NEW_GROUP); - newConsumer.subscribe(of(getTopic())); - newConsumer.seekToBeginning(UniSets.of()); + newConsumer.assign(of(tp)); + newConsumer.seekToBeginning(UniSets.of(tp)); + long positionAfter = newConsumer.position(tp); // trigger eager seek + assertThat(positionAfter).isEqualTo(0); final List> records = new ArrayList<>(); long highest = -1; while (highest < upToOffset - 1) { @@ -444,13 +511,15 @@ private void checkHowManyRecordsWithKeyPresent(String keyToSearchFor, int expect } @SneakyThrows - private void causeCommittedOffsetToBeRemoved(long offset) { + private int causeCommittedOffsetToBeRemoved(long offset) { sendCompactionKeyForOffset(offset); sendCompactionKeyForOffset(offset + 1); checkHowManyRecordsWithKeyPresent("key-" + offset, 2, TO_PRODUCE + 2); - triggerTombStoneProcessing(); + List strings = triggerTombStoneProcessing(); + + return 2 + strings.size(); } private void sendCompactionKeyForOffset(long offset) throws InterruptedException, ExecutionException, TimeoutException { @@ -461,42 +530,4 @@ private void sendCompactionKeyForOffset(long offset) throws InterruptedException .get(1, SECONDS); } - @Test - void cgOffsetsDeletedResetLatest() { - produceMessages(TO_PRODUCE); - - final int END_OFFSET = 50; - runPcUntilOffset(END_OFFSET); - - closePC(); - - causeCommittedConsumerGroupOffsetToBeDeleted(); - - produceMessages(TO_PRODUCE); - - final int TOTAL_PRODUCED = TO_PRODUCE * 2; - runPcCheckStartIs(TOTAL_PRODUCED, TOTAL_PRODUCED); - } - - @Test - void cgOffsetsDeletedResetEarliest() { - produceMessages(TO_PRODUCE); - - final int END_OFFSET = 50; - runPcUntilOffset(END_OFFSET); - - closePC(); - - causeCommittedConsumerGroupOffsetToBeDeleted(); - - produceMessages(100); - - runPcCheckStartIs(0, TO_PRODUCE); - } - - private void causeCommittedConsumerGroupOffsetToBeDeleted() { - throw new RuntimeException(); - } - - } \ No newline at end of file diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java index 329f669f9..890c788a3 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java @@ -68,6 +68,9 @@ class PCVersion { @Getter private KafkaConsumer consumer; + @Setter + private OffsetResetStrategy offsetResetPolicy = OffsetResetStrategy.EARLIEST; + @Getter private KafkaProducer producer; @@ -118,9 +121,7 @@ private Properties setupConsumerProps(String groupIdToUse) { consumerProps.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.toString().toLowerCase(Locale.ROOT)); // Reset - consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, OffsetResetStrategy.EARLIEST.name().toLowerCase()); -// consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, OffsetResetStrategy.LATEST.name().toLowerCase()); -// consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, OffsetResetStrategy.NONE.name().toLowerCase()); + consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, offsetResetPolicy.name().toLowerCase()); // // consumerProps.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 10); From a1cffd09ce230fba9da58c80d0971d5027747b3e Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Wed, 12 Oct 2022 09:55:30 +0100 Subject: [PATCH 26/33] temp make method public for TG --- .../io/confluent/parallelconsumer/state/PartitionState.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java index 01b661ddc..ec74f4522 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java @@ -329,7 +329,7 @@ protected OffsetAndMetadata createOffsetAndMetadata() { */ // visible for testing // todo change back to protected? and enable protected level managed truth (seems to be limited to public) - protected long getNextExpectedInitialPolledOffset() { + public long getNextExpectedInitialPolledOffset() { return getOffsetHighestSequentialSucceeded() + 1; } From f5b5ce25e4fee85ec8070909fef486b10794cebc Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Wed, 12 Oct 2022 10:03:05 +0100 Subject: [PATCH 27/33] fix scm tag to HEAD --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index a40d87c64..ac3efae77 100644 --- a/pom.xml +++ b/pom.xml @@ -55,7 +55,7 @@ scm:git:git://github.com:confluentinc/parallel-consumer.git scm:git:git@github.com:confluentinc/parallel-consumer.git https://github.com/confluentinc/parallel-consumer.git - 0.5.2.2 + HEAD From 80ab1924b327fa2d88f9f51d755cc6072d9c3cb5 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Wed, 12 Oct 2022 10:42:32 +0100 Subject: [PATCH 28/33] validate first --- .../AbstractParallelEoSStreamProcessor.java | 20 ++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/AbstractParallelEoSStreamProcessor.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/AbstractParallelEoSStreamProcessor.java index 893e8a457..3de2bac88 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/AbstractParallelEoSStreamProcessor.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/AbstractParallelEoSStreamProcessor.java @@ -244,21 +244,19 @@ protected AbstractParallelEoSStreamProcessor(ParallelConsumerOptions newOp protected AbstractParallelEoSStreamProcessor(ParallelConsumerOptions newOptions, PCModule module) { Objects.requireNonNull(newOptions, "Options must be supplied"); + options = newOptions; + this.consumer = options.getConsumer(); + + validateConfiguration(); + module.setParallelEoSStreamProcessor(this); log.info("Confluent Parallel Consumer initialise... groupId: {}, Options: {}", newOptions.getConsumer().groupMetadata().groupId(), newOptions); - options = newOptions; - options.validate(); this.dynamicExtraLoadFactor = module.dynamicExtraLoadFactor(); - this.consumer = options.getConsumer(); - - checkGroupIdConfigured(consumer); - checkNotSubscribed(consumer); - checkAutoCommitIsDisabled(consumer); workerThreadPool = setupWorkerPool(newOptions.getMaxConcurrency()); @@ -278,6 +276,14 @@ protected AbstractParallelEoSStreamProcessor(ParallelConsumerOptions newOp } } + private void validateConfiguration() { + options.validate(); + + checkGroupIdConfigured(consumer); + checkNotSubscribed(consumer); + checkAutoCommitIsDisabled(consumer); + } + private void checkGroupIdConfigured(final org.apache.kafka.clients.consumer.Consumer consumer) { try { consumer.groupMetadata(); From 17d13ebfe0a695848570723aab50a9d0d2ed8aab Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Wed, 12 Oct 2022 11:12:29 +0100 Subject: [PATCH 29/33] review --- CHANGELOG.adoc | 5 +++-- .../state/PartitionStateCommittedOffsetIT.java | 11 +---------- .../state/PartitionStateCommittedOffsetTest.java | 8 ++------ .../src/test/resources/logback-test.xml | 2 +- 4 files changed, 7 insertions(+), 19 deletions(-) diff --git a/CHANGELOG.adoc b/CHANGELOG.adoc index 5043a8334..7dec603fe 100644 --- a/CHANGELOG.adoc +++ b/CHANGELOG.adoc @@ -18,8 +18,9 @@ endif::[] === Fixes -* fixes #409: Fixes support for compacted topics and when commit offset is reset -** Truncate state when bootstrap polled offset higher or lower than committed and prune missing records (#425) +* fixes #409: Adds support for compacted topics and commit offset resetting (#425) +** Truncate the offset state when bootstrap polled offset higher or lower than committed +** Prune missing records from the tracked incomplete offset state, when they're missing from polled batches == 0.5.2.3 diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java index 0814e6a2d..3f5d91643 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java @@ -146,22 +146,13 @@ void compactedTopic() { .map(PollContext::key) .collect(Collectors.toList()); -// assertWithMessage("All keys should still exist") -// .that(offsetsFromSecondRunFromKey) -// .containsAtLeastElementsIn(processedOnFirstRun.stream().map(PollContext::key).collect(Collectors.toList())); - // List offsetsFromSecond = processedOnSecondRun.stream() .map(PollContext::offset) .collect(Collectors.toList()); -// assertWithMessage("The offsets of the tombstone targets should not be read in second run") -// .that(offsetsFromSecond) -// .containsNoneIn(tombstoneTargetOffsetsFromFirstRun); - assertWithMessage("Finish reading rest of records from %s to %s", - UNTIL_OFFSET, - TO_PRODUCE) + UNTIL_OFFSET, TO_PRODUCE) .that(processedOnSecondRun.size()).isGreaterThan(TO_PRODUCE - UNTIL_OFFSET); assertWithMessage("Off the offsets read on the second run, offsets that were compacted (below the initial produce target) should now be removed, as they were replaced with newer ones.") diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java index ef62a857d..ecfe38c8f 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java @@ -24,7 +24,8 @@ import static io.confluent.parallelconsumer.ManagedTruth.assertThat; /** - * todo docs + * Unit test for PartitionState behaviour when committed offsets are changed and random records are removed (compaction) + * which already are tracked in the offset map. * * @author Antony Stubbs * @see OffsetEncodingTests#ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential @@ -46,11 +47,6 @@ class PartitionStateCommittedOffsetTest { final long highestSeenOffset = 101L; - /** - * @see PartitionState#offsetHighestSucceeded - */ - long highestSucceeded = highestSeenOffset; - List incompletes = UniLists.of(previouslyCommittedOffset, 15L, unexpectedlyHighOffset, 60L, 80L, 95L, 96L, 97L, 98L, 100L); List expectedTruncatedIncompletes = incompletes.stream() diff --git a/parallel-consumer-core/src/test/resources/logback-test.xml b/parallel-consumer-core/src/test/resources/logback-test.xml index 8fce65ad1..acead0240 100644 --- a/parallel-consumer-core/src/test/resources/logback-test.xml +++ b/parallel-consumer-core/src/test/resources/logback-test.xml @@ -28,7 +28,7 @@ - + From a5ebf6ab5bd7659edbe7d7b565252bb3eefd6885 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Wed, 12 Oct 2022 12:40:04 +0100 Subject: [PATCH 30/33] rollback public -> protected. Add support for protected in TG later --- .../io/confluent/parallelconsumer/state/PartitionState.java | 3 +-- .../state/PartitionStateCommittedOffsetTest.java | 3 ++- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java index ec74f4522..402561a79 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionState.java @@ -328,8 +328,7 @@ protected OffsetAndMetadata createOffsetAndMetadata() { * Defines as the offset one below the highest sequentially succeeded offset. */ // visible for testing - // todo change back to protected? and enable protected level managed truth (seems to be limited to public) - public long getNextExpectedInitialPolledOffset() { + protected long getNextExpectedInitialPolledOffset() { return getOffsetHighestSequentialSucceeded() + 1; } diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java index ecfe38c8f..f96d0c6f4 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/PartitionStateCommittedOffsetTest.java @@ -4,6 +4,7 @@ * Copyright (C) 2020-2022 Confluent, Inc. */ +import com.google.common.truth.Truth; import io.confluent.parallelconsumer.internal.PCModuleTestEnv; import io.confluent.parallelconsumer.offsets.OffsetEncodingTests; import io.confluent.parallelconsumer.offsets.OffsetMapCodecManager.HighestOffsetAndIncompletes; @@ -146,7 +147,7 @@ void bootstrapPollOffsetHigherDueToRetentionOrCompaction() { addPollToState(state, polledTestBatch); // - assertThat(state).getNextExpectedInitialPolledOffset().isEqualTo(unexpectedlyHighOffset); + Truth.assertThat(state.getNextExpectedInitialPolledOffset()).isEqualTo(unexpectedlyHighOffset); OffsetAndMetadata offsetAndMetadata = state.createOffsetAndMetadata(); assertThat(offsetAndMetadata).getOffset().isEqualTo(unexpectedlyHighOffset); From b6a39cbfc10b5994b110ca1916f4ec05d5c77fb7 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Wed, 12 Oct 2022 13:41:16 +0100 Subject: [PATCH 31/33] fix dangling clients - cannot reach broker errors --- .../BrokerIntegrationTest.java | 5 +- .../CloseAndOpenOffsetTest.java | 30 +- .../integrationTests/KafkaSanityTests.java | 4 +- .../integrationTests/LoadTest.java | 10 +- .../MultiInstanceRebalanceTest.java | 6 +- .../integrationTests/MultiTopicTest.java | 2 +- .../OffsetCommittingSanityTest.java | 8 +- .../integrationTests/RebalanceTest.java | 4 +- .../TransactionAndCommitModeTest.java | 6 +- .../TransactionTimeoutsTest.java | 8 +- .../VeryLargeMessageVolumeTest.java | 6 +- .../PartitionStateCommittedOffsetIT.java | 358 +++++++++--------- .../utils/KafkaClientUtils.java | 2 +- 13 files changed, 215 insertions(+), 234 deletions(-) diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/BrokerIntegrationTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/BrokerIntegrationTest.java index 50f5a5b6c..2131d79b8 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/BrokerIntegrationTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/BrokerIntegrationTest.java @@ -9,6 +9,7 @@ import io.confluent.csid.testcontainers.FilteredTestContainerSlf4jLogConsumer; import io.confluent.parallelconsumer.integrationTests.utils.KafkaClientUtils; +import lombok.AccessLevel; import lombok.Getter; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; @@ -75,8 +76,8 @@ public static KafkaContainer createKafkaContainer(String logSegmentSize) { kafkaContainer.start(); } - @Getter - protected KafkaClientUtils kcu = new KafkaClientUtils(kafkaContainer); + @Getter(AccessLevel.PROTECTED) + private final KafkaClientUtils kcu = new KafkaClientUtils(kafkaContainer); @BeforeAll static void followKafkaLogs() { diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/CloseAndOpenOffsetTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/CloseAndOpenOffsetTest.java index 7f8364e48..87c355cfc 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/CloseAndOpenOffsetTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/CloseAndOpenOffsetTest.java @@ -105,8 +105,8 @@ void offsetsOpenClose(OffsetEncoding encoding) { } // - KafkaConsumer newConsumerOne = kcu.createNewConsumer(); - KafkaProducer producerOne = kcu.createNewProducer(true); + KafkaConsumer newConsumerOne = getKcu().createNewConsumer(); + KafkaProducer producerOne = getKcu().createNewProducer(true); var options = ParallelConsumerOptions.builder() .ordering(UNORDERED) .commitMode(PERIODIC_TRANSACTIONAL_PRODUCER) @@ -183,8 +183,8 @@ void offsetsOpenClose(OffsetEncoding encoding) { // second client { // - KafkaConsumer newConsumerThree = kcu.createNewConsumer(customClientId("THREE-my-client")); - KafkaProducer producerThree = kcu.createNewProducer(true); + KafkaConsumer newConsumerThree = getKcu().createNewConsumer(customClientId("THREE-my-client")); + KafkaProducer producerThree = getKcu().createNewProducer(true); var optionsThree = options.toBuilder().consumer(newConsumerThree).producer(producerThree).build(); try (var asyncThree = new ParallelEoSStreamProcessor(optionsThree)) { asyncThree.subscribe(UniLists.of(rebalanceTopic)); @@ -216,7 +216,7 @@ private Properties customClientId(final String id) { } private void send(String topic, int partition, Integer value) throws InterruptedException, ExecutionException { - RecordMetadata recordMetadata = kcu.getProducer().send(new ProducerRecord<>(topic, partition, value.toString(), value.toString())).get(); + RecordMetadata recordMetadata = getKcu().getProducer().send(new ProducerRecord<>(topic, partition, value.toString(), value.toString())).get(); } private void send(int quantity, String topic, int partition) throws InterruptedException, ExecutionException { @@ -224,7 +224,7 @@ private void send(int quantity, String topic, int partition) throws InterruptedE var futures = new ArrayList>(); // async for (Integer index : Range.range(quantity)) { - Future send = kcu.getProducer().send(new ProducerRecord<>(topic, partition, index.toString(), index.toString())); + Future send = getKcu().getProducer().send(new ProducerRecord<>(topic, partition, index.toString(), index.toString())); futures.add(send); } // block until finished @@ -245,10 +245,10 @@ void correctOffsetVerySimple() { // send a single message String expectedPayload = "0"; - kcu.getProducer().send(new ProducerRecord<>(topic, expectedPayload, expectedPayload)); + getKcu().getProducer().send(new ProducerRecord<>(topic, expectedPayload, expectedPayload)); - KafkaConsumer consumer = kcu.createNewConsumer(); - KafkaProducer producerOne = kcu.createNewProducer(true); + KafkaConsumer consumer = getKcu().createNewConsumer(); + KafkaProducer producerOne = getKcu().createNewProducer(true); var options = ParallelConsumerOptions.builder() .ordering(UNORDERED) .consumer(consumer) @@ -277,8 +277,8 @@ void correctOffsetVerySimple() { // log.debug("Starting up new client"); - KafkaConsumer newConsumerThree = kcu.createNewConsumer(customClientId("THREE-my-client")); - KafkaProducer producerThree = kcu.createNewProducer(true); + KafkaConsumer newConsumerThree = getKcu().createNewConsumer(customClientId("THREE-my-client")); + KafkaProducer producerThree = getKcu().createNewProducer(true); ParallelConsumerOptions optionsThree = options.toBuilder() .consumer(newConsumerThree) .producer(producerThree) @@ -330,8 +330,8 @@ void largeNumberOfMessagesSmallOffsetBitmap() { // step 1 { - KafkaConsumer consumer = kcu.createNewConsumer(); - KafkaProducer producerOne = kcu.createNewProducer(true); + KafkaConsumer consumer = getKcu().createNewConsumer(); + KafkaProducer producerOne = getKcu().createNewProducer(true); var options = baseOptions.toBuilder() .consumer(consumer) .producer(producerOne) @@ -364,8 +364,8 @@ void largeNumberOfMessagesSmallOffsetBitmap() { // step 2 { // - KafkaConsumer newConsumerThree = kcu.createNewConsumer(customClientId("THREE-my-client")); - KafkaProducer producerThree = kcu.createNewProducer(true); + KafkaConsumer newConsumerThree = getKcu().createNewConsumer(customClientId("THREE-my-client")); + KafkaProducer producerThree = getKcu().createNewProducer(true); var optionsThree = baseOptions.toBuilder() .consumer(newConsumerThree) .producer(producerThree) diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/KafkaSanityTests.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/KafkaSanityTests.java index b73cd65ba..d31023709 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/KafkaSanityTests.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/KafkaSanityTests.java @@ -38,7 +38,7 @@ public class KafkaSanityTests extends BrokerIntegrationTest { public void pausedConsumerStillLongPollsForNothing() { log.info("Setup topic"); setupTopic(); - KafkaConsumer consumer = kcu.getConsumer(); + KafkaConsumer consumer = getKcu().getConsumer(); log.info("Subscribe to topic"); consumer.subscribe(UniLists.of(topic)); Set assignment = consumer.assignment(); @@ -78,7 +78,7 @@ void offsetMetadataSpaceAvailable() { .as("approximate sanity - ensure start state settings (shared static state :`( )") .isGreaterThan(3000); - KafkaConsumer consumer = kcu.getConsumer(); + KafkaConsumer consumer = getKcu().getConsumer(); TopicPartition tpOne = new TopicPartition(topic, 0); TopicPartition tpTwo = new TopicPartition(topic, 1); HashMap map = new HashMap<>(); diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/LoadTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/LoadTest.java index 1bcb355d5..4334e8acc 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/LoadTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/LoadTest.java @@ -60,7 +60,7 @@ public void setupTestData() { void timedNormalKafkaConsumerTest() { setupTestData(); // subscribe in advance, it can be a few seconds - kcu.getConsumer().subscribe(UniLists.of(topic)); + getKcu().getConsumer().subscribe(UniLists.of(topic)); readRecordsPlainConsumer(total, topic); } @@ -70,13 +70,13 @@ void timedNormalKafkaConsumerTest() { void asyncConsumeAndProcess() { setupTestData(); - KafkaConsumer newConsumer = kcu.createNewConsumer(); + KafkaConsumer newConsumer = getKcu().createNewConsumer(); // boolean tx = true; ParallelConsumerOptions options = ParallelConsumerOptions.builder() .ordering(ParallelConsumerOptions.ProcessingOrder.KEY) .commitMode(PERIODIC_TRANSACTIONAL_PRODUCER) - .producer(kcu.createNewProducer(tx)) + .producer(getKcu().createNewProducer(tx)) .consumer(newConsumer) .maxConcurrency(3) .build(); @@ -127,7 +127,7 @@ private void readRecordsPlainConsumer(int total, String topic) { Executors.newCachedThreadPool().submit(() -> { while (allRecords.size() < total) { - ConsumerRecords poll = kcu.getConsumer().poll(ofMillis(500)); + ConsumerRecords poll = getKcu().getConsumer().poll(ofMillis(500)); log.info("Polled batch of {} messages", poll.count()); //save @@ -176,7 +176,7 @@ private void publishMessages(int keyRange, int total, String topic) { String value = RandomStringUtils.randomAlphabetic(messageSizeInBytes); var producerRecord = new ProducerRecord<>(topic, key, value); try { - var meta = kcu.getProducer().send(producerRecord); + var meta = getKcu().getProducer().send(producerRecord); futureMetadataResultsFromPublishing.add(meta); } catch (Exception e) { throw new RuntimeException(e); diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/MultiInstanceRebalanceTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/MultiInstanceRebalanceTest.java index 63a66a82f..80f2c90fa 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/MultiInstanceRebalanceTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/MultiInstanceRebalanceTest.java @@ -121,7 +121,7 @@ private void runTest(int maxPoll, CommitMode commitMode, ProcessingOrder order, log.info("Producing {} messages before starting test", expectedMessageCount); List> sends = new ArrayList<>(); int preProduceCount = (int) (expectedMessageCount * fractionOfMessagesToPreProduce); - try (Producer kafkaProducer = kcu.createNewProducer(false)) { + try (Producer kafkaProducer = getKcu().createNewProducer(false)) { for (int i = 0; i < preProduceCount; i++) { String key = "key-" + i; Future send = kafkaProducer.send(new ProducerRecord<>(inputName, key, "value-" + i), (meta, exception) -> { @@ -160,7 +160,7 @@ private void runTest(int maxPoll, CommitMode commitMode, ProcessingOrder order, public void run() { // pre-produce messages to input-topic log.info("Producing {} messages before starting test", expectedMessageCount); - try (Producer kafkaProducer = kcu.createNewProducer(false)) { + try (Producer kafkaProducer = getKcu().createNewProducer(false)) { for (int i = preProduceCount; i < expectedMessageCount; i++) { // slow things down just a tad // Thread.sleep(1); @@ -363,7 +363,7 @@ public void run() { Properties consumerProps = new Properties(); consumerProps.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPoll); - KafkaConsumer newConsumer = kcu.createNewConsumer(false, consumerProps); + KafkaConsumer newConsumer = getKcu().createNewConsumer(false, consumerProps); this.parallelConsumer = new ParallelEoSStreamProcessor<>(ParallelConsumerOptions.builder() .ordering(order) diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/MultiTopicTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/MultiTopicTest.java index 16258587a..9988eff7f 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/MultiTopicTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/MultiTopicTest.java @@ -63,7 +63,7 @@ void multiTopic(ProcessingOrder order) { pc.close(); // - Consumer assertingConsumer = kcu.createNewConsumer(false); + Consumer assertingConsumer = getKcu().createNewConsumer(false); await().atMost(Duration.ofSeconds(10)) .untilAsserted(() -> { assertSeparateConsumerCommit(assertingConsumer, new HashSet<>(multiTopics), recordsPerTopic); diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/OffsetCommittingSanityTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/OffsetCommittingSanityTest.java index 5b4bc242c..24ae5d306 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/OffsetCommittingSanityTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/OffsetCommittingSanityTest.java @@ -47,7 +47,7 @@ void shouldNotSkipAnyMessagesOnRestartRoot() throws Exception { List producedOffsets = new ArrayList<>(); List consumedOffsets = new ArrayList<>(); - KafkaProducer kafkaProducer = kcu.createNewProducer(false); + KafkaProducer kafkaProducer = getKcu().createNewProducer(false); // offset 0 sendCheckClose(topicNameForTest, producedOffsets, consumedOffsets, kafkaProducer, "key-0", "value-0", true); @@ -70,7 +70,7 @@ void shouldNotSkipAnyMessagesOnRestartAsDescribed() throws Exception { List producedOffsets = new ArrayList<>(); List consumedOffsets = new ArrayList<>(); - KafkaProducer kafkaProducer = kcu.createNewProducer(NOT_TRANSACTIONAL); + KafkaProducer kafkaProducer = getKcu().createNewProducer(NOT_TRANSACTIONAL); // offset 0 sendCheckClose(topicNameForTest, producedOffsets, consumedOffsets, kafkaProducer, "key-0", "value-0", CheckMode.CHECK_CONSUMED); @@ -118,7 +118,7 @@ var record = new ProducerRecord<>(topic, key, val); producedOffsets.add(offset); // - var newConsumer = kcu.createNewConsumer(false); + var newConsumer = getKcu().createNewConsumer(false); var pc = createParallelConsumer(topic, newConsumer); // @@ -146,7 +146,7 @@ public enum CheckMode { */ private void assertCommittedOffset(String topicNameForTest, long expectedOffset) { // assert committed offset - var newConsumer = kcu.createNewConsumer(false); + var newConsumer = getKcu().createNewConsumer(false); newConsumer.subscribe(UniSets.of(topicNameForTest)); // increased poll timeout to allow for delay under load during parallel test execution newConsumer.poll(ofSeconds(5)); diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/RebalanceTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/RebalanceTest.java index b05c07d69..ec75af5c8 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/RebalanceTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/RebalanceTest.java @@ -69,7 +69,7 @@ void commitUponRevoke() { var count = new AtomicLong(); // - kcu.produceMessages(topic, numberOfRecordsToProduce); + getKcu().produceMessages(topic, numberOfRecordsToProduce); // effectively disable commit pc.setTimeBetweenCommits(INFINITE); @@ -85,7 +85,7 @@ void commitUponRevoke() { // cause rebalance final Duration newPollTimeout = Duration.ofSeconds(5); log.debug("Creating new consumer in same group and subscribing to same topic set with a no record timeout of {}, expect this phase to take entire timeout...", newPollTimeout); - var newConsumer = kcu.createNewConsumer(REUSE_GROUP); + var newConsumer = getKcu().createNewConsumer(REUSE_GROUP); newConsumer.subscribe(UniLists.of(topic)); log.debug("Polling with new group member for records with timeout {}...", newPollTimeout); ConsumerRecords newConsumersPollResult = newConsumer.poll(newPollTimeout); diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java index 1204b50a7..c4abf7f34 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java @@ -129,7 +129,7 @@ private void runTest(int maxPoll, CommitMode commitMode, ProcessingOrder order, List expectedKeys = new ArrayList<>(); log.info("Producing {} messages before starting test", expectedMessageCount); List> sends = new ArrayList<>(); - try (Producer kafkaProducer = kcu.createNewProducer(false)) { + try (Producer kafkaProducer = getKcu().createNewProducer(false)) { for (int i = 0; i < expectedMessageCount; i++) { String key = "key-" + i; Future send = kafkaProducer.send(new ProducerRecord<>(inputName, key, "value-" + i), (meta, exception) -> { @@ -151,11 +151,11 @@ private void runTest(int maxPoll, CommitMode commitMode, ProcessingOrder order, // run parallel-consumer log.debug("Starting test"); - KafkaProducer newProducer = kcu.createNewProducer(commitMode); + KafkaProducer newProducer = getKcu().createNewProducer(commitMode); Properties consumerProps = new Properties(); consumerProps.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPoll); - KafkaConsumer newConsumer = kcu.createNewConsumer(true, consumerProps); + KafkaConsumer newConsumer = getKcu().createNewConsumer(true, consumerProps); // increased PC concurrency - improves test stability and performance. int numThreads = 64; diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionTimeoutsTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionTimeoutsTest.java index 28c6a7af6..81ef0443d 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionTimeoutsTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionTimeoutsTest.java @@ -68,7 +68,7 @@ void setup(PCModule module) { pc = new ParallelEoSStreamProcessor<>(module.options(), module); - kcu.produceMessages(getTopic(), NUMBER_TO_SEND); + getKcu().produceMessages(getTopic(), NUMBER_TO_SEND); pc.subscribe(of(getTopic())); @@ -79,8 +79,8 @@ void setup(PCModule module) { private ParallelConsumerOptions.ParallelConsumerOptionsBuilder createOptions() { return ParallelConsumerOptions.builder() - .consumer(kcu.createNewConsumer()) - .producer(kcu.createNewProducer(CommitMode.PERIODIC_TRANSACTIONAL_PRODUCER)) + .consumer(getKcu().createNewConsumer()) + .producer(getKcu().createNewProducer(CommitMode.PERIODIC_TRANSACTIONAL_PRODUCER)) .commitMode(CommitMode.PERIODIC_TRANSACTIONAL_PRODUCER) .commitLockAcquisitionTimeout(ofSeconds(1)) .defaultMessageRetryDelay(ofMillis(100)) @@ -162,7 +162,7 @@ void commitTimeout(int multiple) { // check what was committed at shutdown to the input topic, re-using same group id as PC, to access what was committed at shutdown commit attempt // 2nd commit attempt during shutdown will have succeeded - var newConsumer = kcu.createNewConsumer(originalGroupId); + var newConsumer = getKcu().createNewConsumer(originalGroupId); var assertCommittedToPartition = assertThat(newConsumer).hasCommittedToPartition(getTopic(), partitionNumber); assertCommittedToPartition.offset(offsetToGoVerySlow); diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/VeryLargeMessageVolumeTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/VeryLargeMessageVolumeTest.java index a4a7c292f..6c32c8c09 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/VeryLargeMessageVolumeTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/VeryLargeMessageVolumeTest.java @@ -88,7 +88,7 @@ private void runTest(int maxPoll, CommitMode commitMode, ProcessingOrder order) long expectedMessageCount = 1_000_000; log.info("Producing {} messages before starting test", expectedMessageCount); List> sends = new ArrayList<>(); - try (Producer kafkaProducer = kcu.createNewProducer(false)) { + try (Producer kafkaProducer = getKcu().createNewProducer(false)) { for (int i = 0; i < expectedMessageCount; i++) { String key = "key-" + i; Future send = kafkaProducer.send(new ProducerRecord<>(inputName, key, "value-" + i), (meta, exception) -> { @@ -110,11 +110,11 @@ private void runTest(int maxPoll, CommitMode commitMode, ProcessingOrder order) // run parallel-consumer log.debug("Starting test"); - KafkaProducer newProducer = kcu.createNewProducer(commitMode.equals(PERIODIC_TRANSACTIONAL_PRODUCER)); + KafkaProducer newProducer = getKcu().createNewProducer(commitMode.equals(PERIODIC_TRANSACTIONAL_PRODUCER)); Properties consumerProps = new Properties(); consumerProps.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPoll); - KafkaConsumer newConsumer = kcu.createNewConsumer(true, consumerProps); + KafkaConsumer newConsumer = getKcu().createNewConsumer(true, consumerProps); var pc = new ParallelEoSStreamProcessor<>(ParallelConsumerOptions.builder() .ordering(order) diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java index 3f5d91643..ab3649024 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/state/PartitionStateCommittedOffsetIT.java @@ -8,7 +8,6 @@ import io.confluent.csid.utils.JavaUtils; import io.confluent.csid.utils.ThreadUtils; import io.confluent.parallelconsumer.ManagedTruth; -import io.confluent.parallelconsumer.ParallelEoSStreamProcessor; import io.confluent.parallelconsumer.PollContext; import io.confluent.parallelconsumer.integrationTests.BrokerIntegrationTest; import io.confluent.parallelconsumer.integrationTests.utils.KafkaClientUtils; @@ -17,7 +16,6 @@ import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.exception.ExceptionUtils; -import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.AlterConfigOp; import org.apache.kafka.clients.admin.ConfigEntry; import org.apache.kafka.clients.consumer.*; @@ -64,11 +62,6 @@ class PartitionStateCommittedOffsetIT extends BrokerIntegrationTest { public static final OffsetResetStrategy DEFAULT_OFFSET_RESET_POLICY = OffsetResetStrategy.EARLIEST; - AdminClient ac; - - String groupId; - - ParallelEoSStreamProcessor pc; TopicPartition tp; @@ -80,8 +73,6 @@ class PartitionStateCommittedOffsetIT extends BrokerIntegrationTest keys = produceMessages(TO_PRODUCE); + var TO_PRODUCE = this.TO_PRODUCE / 10; // local override, produce less - final int UNTIL_OFFSET = TO_PRODUCE / 2; - var processedOnFirstRun = runPcUntilOffset(UNTIL_OFFSET, TO_PRODUCE, UniSets.of(TO_PRODUCE - 3L)); - assertWithMessage("Last processed should be at least half of the total sent, so that there is incomplete data to track") - .that(getLast(processedOnFirstRun).get().offset()) - .isGreaterThan(TO_PRODUCE / 2); + List keys = produceMessages(TO_PRODUCE); - // commit offset - closePC(); + final int UNTIL_OFFSET = TO_PRODUCE / 2; + var processedOnFirstRun = runPcUntilOffset(UNTIL_OFFSET, TO_PRODUCE, UniSets.of(TO_PRODUCE - 3L)); + assertWithMessage("Last processed should be at least half of the total sent, so that there is incomplete data to track") + .that(getLast(processedOnFirstRun).get().offset()) + .isGreaterThan(TO_PRODUCE / 2); - // - ArrayList compactionKeysRaw = sendRandomCompactionRecords(keys, TO_PRODUCE); - Set compactedKeys = new HashSet<>(compactionKeysRaw); + // + ArrayList compactionKeysRaw = sendRandomCompactionRecords(keys, TO_PRODUCE); + Set compactedKeys = new HashSet<>(compactionKeysRaw); - var processedOnFirstRunWithTombstoneTargetsRemoved = processedOnFirstRun.stream() - .filter(context -> !compactedKeys.contains(context.key())) - .map(PollContext::key) - .collect(Collectors.toList()); + var processedOnFirstRunWithTombstoneTargetsRemoved = processedOnFirstRun.stream() + .filter(context -> !compactedKeys.contains(context.key())) + .map(PollContext::key) + .collect(Collectors.toList()); - var firstRunPartitioned = processedOnFirstRun.stream().collect(Collectors.partitioningBy(context -> compactedKeys.contains(context.key()))); - var saved = firstRunPartitioned.get(Boolean.FALSE); - var compacted = firstRunPartitioned.get(Boolean.TRUE); - log.debug("kept offsets: {}", saved.stream().mapToLong(PollContext::offset).boxed().collect(Collectors.toList())); - log.debug("kept keys: {}", saved.stream().map(PollContext::key).collect(Collectors.toList())); - log.debug("compacted offsets: {}", compacted.stream().map(PollContext::key).collect(Collectors.toList())); - log.debug("compacted keys: {}", compacted.stream().mapToLong(PollContext::offset).boxed().collect(Collectors.toList())); + var firstRunPartitioned = processedOnFirstRun.stream().collect(Collectors.partitioningBy(context -> compactedKeys.contains(context.key()))); + var saved = firstRunPartitioned.get(Boolean.FALSE); + var compacted = firstRunPartitioned.get(Boolean.TRUE); + log.debug("kept offsets: {}", saved.stream().mapToLong(PollContext::offset).boxed().collect(Collectors.toList())); + log.debug("kept keys: {}", saved.stream().map(PollContext::key).collect(Collectors.toList())); + log.debug("compacted offsets: {}", compacted.stream().map(PollContext::key).collect(Collectors.toList())); + log.debug("compacted keys: {}", compacted.stream().mapToLong(PollContext::offset).boxed().collect(Collectors.toList())); - var tombstoneTargetOffsetsFromFirstRun = compacted.stream() - .filter(context -> compactedKeys.contains(context.key())) - .map(PollContext::offset) - .collect(Collectors.toList()); + var tombstoneTargetOffsetsFromFirstRun = compacted.stream() + .filter(context -> compactedKeys.contains(context.key())) + .map(PollContext::offset) + .collect(Collectors.toList()); - var tombStonedOffsetsFromKey = compactedKeys.stream() - .map(PartitionStateCommittedOffsetIT::getOffsetFromKey).collect(Collectors.toList()); - log.debug("First run produced, with compaction targets removed: {}", processedOnFirstRunWithTombstoneTargetsRemoved); + var tombStonedOffsetsFromKey = compactedKeys.stream() + .map(PartitionStateCommittedOffsetIT::getOffsetFromKey).collect(Collectors.toList()); + log.debug("First run produced, with compaction targets removed: {}", processedOnFirstRunWithTombstoneTargetsRemoved); - // - triggerTombStoneProcessing(); + // + triggerTombStoneProcessing(); - // The offsets of the tombstone targets should not be read in second run - final int expectedTotalNumberRecordsProduced = TO_PRODUCE + tombStonedOffsetsFromKey.size(); - final int expectedOffsetProcessedToSecondRun = TO_PRODUCE + compactedKeys.size(); - var processedOnSecondRun = runPcUntilOffset(expectedOffsetProcessedToSecondRun, GroupOption.REUSE_GROUP).stream() - .filter(recordContexts -> !recordContexts.key().contains("compaction-trigger")) - .collect(Collectors.toList()); + // The offsets of the tombstone targets should not be read in second run + final int expectedOffsetProcessedToSecondRun = TO_PRODUCE + compactedKeys.size(); + var processedOnSecondRun = runPcUntilOffset(expectedOffsetProcessedToSecondRun, GroupOption.REUSE_GROUP).stream() + .filter(recordContexts -> !recordContexts.key().contains("compaction-trigger")) + .collect(Collectors.toList()); - // - List offsetsFromSecondRunFromKey = processedOnSecondRun.stream() - .map(PollContext::key) - .collect(Collectors.toList()); - - // - List offsetsFromSecond = processedOnSecondRun.stream() - .map(PollContext::offset) - .collect(Collectors.toList()); + // + List offsetsFromSecond = processedOnSecondRun.stream() + .map(PollContext::offset) + .collect(Collectors.toList()); - assertWithMessage("Finish reading rest of records from %s to %s", - UNTIL_OFFSET, TO_PRODUCE) - .that(processedOnSecondRun.size()).isGreaterThan(TO_PRODUCE - UNTIL_OFFSET); + assertWithMessage("Finish reading rest of records from %s to %s", + UNTIL_OFFSET, TO_PRODUCE) + .that(processedOnSecondRun.size()).isGreaterThan(TO_PRODUCE - UNTIL_OFFSET); - assertWithMessage("Off the offsets read on the second run, offsets that were compacted (below the initial produce target) should now be removed, as they were replaced with newer ones.") - .that(offsetsFromSecond) - .containsNoneIn(tombstoneTargetOffsetsFromFirstRun); + assertWithMessage("Off the offsets read on the second run, offsets that were compacted (below the initial produce target) should now be removed, as they were replaced with newer ones.") + .that(offsetsFromSecond) + .containsNoneIn(tombstoneTargetOffsetsFromFirstRun); - compactingBroker.close(); + } } /** - * Setup our extra special compacting broker + * Set up our extra special compacting broker */ @NonNull private KafkaContainer setupCompactingKafkaBroker() { KafkaContainer compactingBroker = null; { + // set up new broker compactingBroker = BrokerIntegrationTest.createKafkaContainer("40000"); compactingBroker.start(); - kcu = new KafkaClientUtils(compactingBroker); - kcu.open(); setup(); } @@ -210,7 +191,7 @@ private void setupCompacted() { alterConfigOps.add(new AlterConfigOp(new ConfigEntry(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, "0"), AlterConfigOp.OpType.SET)); var configs = UniMaps.of(topicConfig, alterConfigOps); - KafkaFuture all = ac.incrementalAlterConfigs(configs).all(); + KafkaFuture all = getKcu().getAdmin().incrementalAlterConfigs(configs).all(); all.get(5, SECONDS); log.debug("Compaction setup complete"); @@ -264,11 +245,9 @@ void committedOffsetLower() { runPcUntilOffset(50); - closePC(); - final int moveToOffset = 25; - moveCommittedOffset(kcu.getGroupId(), moveToOffset); + moveCommittedOffset(getKcu().getGroupId(), moveToOffset); runPcCheckStartIs(moveToOffset, TO_PRODUCE); } @@ -281,122 +260,125 @@ void committedOffsetLower() { */ @SneakyThrows private void runPcCheckStartIs(long targetStartOffset, long checkUpTo, GroupOption groupOption) { - this.pc = super.getKcu().buildPc(PARTITION, groupOption); - pc.subscribe(of(getTopic())); - - AtomicLong lowest = new AtomicLong(Long.MAX_VALUE); - AtomicLong highest = new AtomicLong(); - - pc.poll(recordContexts -> { - long thisOffset = recordContexts.offset(); - if (thisOffset < lowest.get()) { - log.debug("Found lowest offset {}", thisOffset); - lowest.set(thisOffset); - } else if (thisOffset > highest.get()) { - highest.set(thisOffset); - } - }); - - // - AtomicLong bumpersSent = new AtomicLong(); - Awaitility.await().untilAsserted(() -> { - // in case we're at the end of the topic, add some messages to make sure we get a poll response - getKcu().produceMessages(getTopic(), 1, "poll-bumper"); - bumpersSent.incrementAndGet(); - - assertWithMessage("Highest seen offset") - .that(highest.get()) - .isAtLeast(checkUpTo - 1); - }); + try (var tempPc = super.getKcu().buildPc(PARTITION, groupOption);) { + tempPc.subscribe(of(getTopic())); + + AtomicLong lowest = new AtomicLong(Long.MAX_VALUE); + AtomicLong highest = new AtomicLong(); + + tempPc.poll(recordContexts -> { + long thisOffset = recordContexts.offset(); + if (thisOffset < lowest.get()) { + log.debug("Found lowest offset {}", thisOffset); + lowest.set(thisOffset); + } else if (thisOffset > highest.get()) { + highest.set(thisOffset); + } + }); - pc.close(); + // + AtomicLong bumpersSent = new AtomicLong(); + Awaitility.await().untilAsserted(() -> { + // in case we're at the end of the topic, add some messages to make sure we get a poll response + getKcu().produceMessages(getTopic(), 1, "poll-bumper"); + bumpersSent.incrementAndGet(); - var adjustExpected = switch (offsetResetStrategy) { - case EARLIEST -> targetStartOffset; - case LATEST -> targetStartOffset + 1; - case NONE -> throw new IllegalStateException("NONE not supported"); - }; - assertWithMessage("Offset started as").that(lowest.get()).isEqualTo(adjustExpected); - } + assertWithMessage("Highest seen offset") + .that(highest.get()) + .isAtLeast(checkUpTo - 1); + }); - private void moveCommittedOffset(int moveToOffset) { - moveCommittedOffset(groupId, moveToOffset); + var adjustExpected = switch (offsetResetStrategy) { + case EARLIEST -> targetStartOffset; + case LATEST -> targetStartOffset + 1; + case NONE -> throw new IllegalStateException("NONE not supported"); + }; + assertWithMessage("Offset started as").that(lowest.get()).isEqualTo(adjustExpected); + } } @SneakyThrows private void moveCommittedOffset(String groupId, long offset) { log.debug("Moving offset of {} to {}", groupId, offset); var data = UniMaps.of(tp, new OffsetAndMetadata(offset)); - var result = ac.alterConsumerGroupOffsets(groupId, data); + var result = getKcu().getAdmin().alterConsumerGroupOffsets(groupId, data); result.all().get(5, SECONDS); log.debug("Moved offset to {}", offset); } - private void closePC() { - pc.close(); - } - - private List> runPcUntilOffset(long succeedUpToOffset, long expectedProcessToOffset, Set exceptionsToSucceed) { return runPcUntilOffset(DEFAULT_OFFSET_RESET_POLICY, succeedUpToOffset, expectedProcessToOffset, exceptionsToSucceed, GroupOption.NEW_GROUP); } @SneakyThrows - private List> runPcUntilOffset(OffsetResetStrategy offsetResetPolicy, long succeedUpToOffset, long expectedProcessToOffset, Set exceptionsToSucceed, GroupOption newGroup) { + private List> runPcUntilOffset(OffsetResetStrategy offsetResetPolicy, + long succeedUpToOffset, + long expectedProcessToOffset, + Set exceptionsToSucceed, + GroupOption newGroup) { log.debug("Running PC until at least offset {}", succeedUpToOffset); - this.pc = super.getKcu().buildPc(UNORDERED, newGroup); + super.getKcu().setOffsetResetPolicy(offsetResetPolicy); + var tempPc = super.getKcu().buildPc(UNORDERED, newGroup); + try { // can't use auto closeable because close is complicated as it's expected to crash and close rethrows error + + SortedSet> seenOffsets = Collections.synchronizedSortedSet(new TreeSet<>(Comparator.comparingLong(PollContext::offset))); + SortedSet> succeededOffsets = Collections.synchronizedSortedSet(new TreeSet<>(Comparator.comparingLong(PollContext::offset))); + + tempPc.subscribe(of(getTopic())); + + tempPc.poll(pollContext -> { + seenOffsets.add(pollContext); + long thisOffset = pollContext.offset(); + if (exceptionsToSucceed.contains(thisOffset)) { + log.debug("Exceptional offset {} succeeded", thisOffset); + } else if (thisOffset >= succeedUpToOffset) { + log.debug("Failing on {}", thisOffset); + throw new RuntimeException("Failing on " + thisOffset); + } else { + log.debug("Succeeded {}: {}", thisOffset, pollContext.getSingleRecord()); + succeededOffsets.add(pollContext); + } + }); - SortedSet> seenOffsets = Collections.synchronizedSortedSet(new TreeSet<>(Comparator.comparingLong(PollContext::offset))); - SortedSet> succeededOffsets = Collections.synchronizedSortedSet(new TreeSet<>(Comparator.comparingLong(PollContext::offset))); + // give first poll a chance to run + ThreadUtils.sleepSecondsLog(1); - pc.subscribe(of(getTopic())); + getKcu().produceMessages(getTopic(), 1, "poll-bumper"); - pc.poll(pollContext -> { - seenOffsets.add(pollContext); - long thisOffset = pollContext.offset(); - if (exceptionsToSucceed.contains(thisOffset)) { - log.debug("Exceptional offset {} succeeded", thisOffset); - } else if (thisOffset >= succeedUpToOffset) { - log.debug("Failing on {}", thisOffset); - throw new RuntimeException("Failing on " + thisOffset); + if (offsetResetPolicy.equals(OffsetResetStrategy.NONE)) { + Awaitility.await().untilAsserted(() -> { + assertWithMessage("PC crashed / failed fast").that(tempPc.isClosedOrFailed()).isTrue(); + assertThat(tempPc.getFailureCause()).hasCauseThat().hasMessageThat().contains("Error in BrokerPollSystem system"); + var stackTrace = ExceptionUtils.getStackTrace(tempPc.getFailureCause()); + Truth.assertThat(stackTrace).contains("Undefined offset with no reset policy for partitions"); + }); + return UniLists.of(); } else { - log.debug("Succeeded {}: {}", thisOffset, pollContext.getSingleRecord()); - succeededOffsets.add(pollContext); - } - }); - // give first poll a chance to run - ThreadUtils.sleepSecondsLog(1); + Awaitility.await() + .failFast(tempPc::isClosedOrFailed) + .untilAsserted(() -> { + assertThat(seenOffsets).isNotEmpty(); + assertThat(seenOffsets.last().offset()).isGreaterThan(expectedProcessToOffset - 2); + }); - getKcu().produceMessages(getTopic(), 1, "poll-bumper"); + if (!succeededOffsets.isEmpty()) { + log.debug("Succeeded up to: {}", succeededOffsets.last().offset()); + } + log.debug("Consumed up to {}", seenOffsets.last().offset()); - if (offsetResetPolicy.equals(OffsetResetStrategy.NONE)) { - Awaitility.await().untilAsserted(() -> { - assertWithMessage("PC crashed / failed fast").that(pc.isClosedOrFailed()).isTrue(); - assertThat(pc.getFailureCause()).hasCauseThat().hasMessageThat().contains("Error in BrokerPollSystem system"); - var stackTrace = ExceptionUtils.getStackTrace(pc.getFailureCause()); - Truth.assertThat(stackTrace).contains("Undefined offset with no reset policy for partitions"); - }); - return UniLists.of(); - } else { - - Awaitility.await() - .failFast(pc::isClosedOrFailed) - .untilAsserted(() -> { - assertThat(seenOffsets).isNotEmpty(); - assertThat(seenOffsets.last().offset()).isGreaterThan(expectedProcessToOffset - 2); - }); - - if (!succeededOffsets.isEmpty()) { - log.debug("Succeeded up to: {}", succeededOffsets.last().offset()); - } - log.debug("Consumed up to {}", seenOffsets.last().offset()); + var sorted = new ArrayList<>(seenOffsets); + Collections.sort(sorted, Comparator.comparingLong(PollContext::offset)); - pc.close(); - var sorted = new ArrayList<>(seenOffsets); - Collections.sort(sorted, Comparator.comparingLong(PollContext::offset)); - return sorted; + return sorted; + } + } finally { + try { + tempPc.close(); // close manually in this branch only, as in other branch it crashes + } catch (Exception e) { + log.debug("Cause will get rethrown close on the NONE parameter branch", e); + } } } @@ -410,15 +392,13 @@ void committedOffsetHigher() { runPcUntilOffset(50); - closePC(); - final int moveToOffset = 75; // reslolve groupId mess - moveCommittedOffset(kcu.getGroupId(), moveToOffset); + moveCommittedOffset(getKcu().getGroupId(), moveToOffset); runPcCheckStartIs(moveToOffset, quantity); - var gkcu5 = kcu.getConsumer().groupMetadata().groupId(); + var gkcu5 = getKcu().getConsumer().groupMetadata().groupId(); } @@ -434,17 +414,19 @@ private void runPcCheckStartIs(int targetStartOffset, int checkUpTo) { @ParameterizedTest void committedOffsetRemoved(OffsetResetStrategy offsetResetPolicy) { this.offsetResetStrategy = offsetResetPolicy; - try (KafkaContainer compactingKafkaBroker = setupCompactingKafkaBroker()) { + try ( + KafkaContainer compactingKafkaBroker = setupCompactingKafkaBroker(); + KafkaClientUtils clientUtils = new KafkaClientUtils(compactingKafkaBroker); + ) { log.debug("Compacting broker started {}", compactingKafkaBroker.getBootstrapServers()); - kcu = new KafkaClientUtils(compactingKafkaBroker); - kcu.setOffsetResetPolicy(offsetResetPolicy); - kcu.open(); + clientUtils.setOffsetResetPolicy(offsetResetPolicy); + clientUtils.open(); var producedCount = produceMessages(TO_PRODUCE).size(); final int END_OFFSET = 50; - groupId = getKcu().getGroupId(); + var groupId = clientUtils.getGroupId(); runPcUntilOffset(offsetResetPolicy, END_OFFSET); // @@ -453,9 +435,6 @@ void committedOffsetRemoved(OffsetResetStrategy offsetResetPolicy) { return; } - // - closePC(); - final String compactedKey = "key-50"; // before compaction @@ -473,7 +452,7 @@ void committedOffsetRemoved(OffsetResetStrategy offsetResetPolicy) { case LATEST -> producedCount + 4; case NONE -> -1; // will crash / fail fast }; - getKcu().setGroupId(groupId); + clientUtils.setGroupId(groupId); runPcCheckStartIs(EXPECTED_RESET_OFFSET, producedCount); } } @@ -481,24 +460,25 @@ void committedOffsetRemoved(OffsetResetStrategy offsetResetPolicy) { private void checkHowManyRecordsWithKeyPresent(String keyToSearchFor, int expectedQuantityToFind, long upToOffset) { log.debug("Looking for {} records with key {} up to offset {}", expectedQuantityToFind, keyToSearchFor, upToOffset); - KafkaConsumer newConsumer = kcu.createNewConsumer(GroupOption.NEW_GROUP); - newConsumer.assign(of(tp)); - newConsumer.seekToBeginning(UniSets.of(tp)); - long positionAfter = newConsumer.position(tp); // trigger eager seek - assertThat(positionAfter).isEqualTo(0); - final List> records = new ArrayList<>(); - long highest = -1; - while (highest < upToOffset - 1) { - ConsumerRecords poll = newConsumer.poll(Duration.ofSeconds(1)); - records.addAll(poll.records(tp)); - var lastOpt = getLast(records); - if (lastOpt.isPresent()) { - highest = lastOpt.get().offset(); + try (KafkaConsumer newConsumer = getKcu().createNewConsumer(GroupOption.NEW_GROUP);) { + newConsumer.assign(of(tp)); + newConsumer.seekToBeginning(UniSets.of(tp)); + long positionAfter = newConsumer.position(tp); // trigger eager seek + assertThat(positionAfter).isEqualTo(0); + final List> records = new ArrayList<>(); + long highest = -1; + while (highest < upToOffset - 1) { + ConsumerRecords poll = newConsumer.poll(Duration.ofSeconds(1)); + records.addAll(poll.records(tp)); + var lastOpt = getLast(records); + if (lastOpt.isPresent()) { + highest = lastOpt.get().offset(); + } } - } - var collect = records.stream().filter(value -> value.key().equals(keyToSearchFor)).collect(Collectors.toList()); - ManagedTruth.assertThat(collect).hasSize(expectedQuantityToFind); + var collect = records.stream().filter(value -> value.key().equals(keyToSearchFor)).collect(Collectors.toList()); + ManagedTruth.assertThat(collect).hasSize(expectedQuantityToFind); + } } @SneakyThrows diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java index 890c788a3..b2ad82a63 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java @@ -53,7 +53,7 @@ * @author Antony Stubbs */ @Slf4j -public class KafkaClientUtils { +public class KafkaClientUtils implements AutoCloseable { public static final int MAX_POLL_RECORDS = 10_000; public static final String GROUP_ID_PREFIX = "group-1-"; From 2e6deb0897dcf5978dd9815439bdb5d96880afda Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Wed, 12 Oct 2022 13:44:59 +0100 Subject: [PATCH 32/33] docs --- README.adoc | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/README.adoc b/README.adoc index 7548ccb35..0ae6c91cc 100644 --- a/README.adoc +++ b/README.adoc @@ -1291,7 +1291,9 @@ endif::[] === Fixes -* fixes #409: Truncate state when bootstrap polled offset higher or lower than committed and prune missing records (#425) +* fixes #409: Adds support for compacted topics and commit offset resetting (#425) +** Truncate the offset state when bootstrap polled offset higher or lower than committed +** Prune missing records from the tracked incomplete offset state, when they're missing from polled batches == 0.5.2.3 From 1d2902f0cdc4d90f8046244755eb1802fb897f91 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Wed, 12 Oct 2022 14:02:43 +0100 Subject: [PATCH 33/33] fixup! fix dangling clients - cannot reach broker errors --- parallel-consumer-core/src/test/resources/logback-test.xml | 3 ++- .../parallelconsumer/examples/streams/StreamsAppTest.java | 6 +++--- .../vertx/integrationTests/VertxConcurrencyIT.java | 6 +++--- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/parallel-consumer-core/src/test/resources/logback-test.xml b/parallel-consumer-core/src/test/resources/logback-test.xml index acead0240..b812c8a7e 100644 --- a/parallel-consumer-core/src/test/resources/logback-test.xml +++ b/parallel-consumer-core/src/test/resources/logback-test.xml @@ -28,7 +28,8 @@ - + diff --git a/parallel-consumer-examples/parallel-consumer-example-streams/src/test/java/io/confluent/parallelconsumer/examples/streams/StreamsAppTest.java b/parallel-consumer-examples/parallel-consumer-example-streams/src/test/java/io/confluent/parallelconsumer/examples/streams/StreamsAppTest.java index 6356afa42..0d7dfc575 100644 --- a/parallel-consumer-examples/parallel-consumer-example-streams/src/test/java/io/confluent/parallelconsumer/examples/streams/StreamsAppTest.java +++ b/parallel-consumer-examples/parallel-consumer-example-streams/src/test/java/io/confluent/parallelconsumer/examples/streams/StreamsAppTest.java @@ -27,7 +27,7 @@ public void test() { coreApp.run(); - try (Producer kafkaProducer = kcu.createNewProducer(false)) { + try (Producer kafkaProducer = getKcu().createNewProducer(false)) { kafkaProducer.send(new ProducerRecord<>(StreamsApp.inputTopic, "a key 1", "a value")); kafkaProducer.send(new ProducerRecord<>(StreamsApp.inputTopic, "a key 2", "a value")); @@ -46,12 +46,12 @@ class StreamsAppUnderTest extends StreamsApp { @Override Consumer getKafkaConsumer() { - return kcu.getConsumer(); + return getKcu().getConsumer(); } @Override Producer getKafkaProducer() { - return kcu.createNewProducer(false); + return getKcu().createNewProducer(false); } @Override diff --git a/parallel-consumer-vertx/src/test-integration/java/io/confluent/parallelconsumer/vertx/integrationTests/VertxConcurrencyIT.java b/parallel-consumer-vertx/src/test-integration/java/io/confluent/parallelconsumer/vertx/integrationTests/VertxConcurrencyIT.java index 60bd6eb81..31838f9b8 100644 --- a/parallel-consumer-vertx/src/test-integration/java/io/confluent/parallelconsumer/vertx/integrationTests/VertxConcurrencyIT.java +++ b/parallel-consumer-vertx/src/test-integration/java/io/confluent/parallelconsumer/vertx/integrationTests/VertxConcurrencyIT.java @@ -147,7 +147,7 @@ void testVertxConcurrency() { log.info("Producing {} messages before starting test", expectedMessageCount); List> sends = new ArrayList<>(); - try (Producer kafkaProducer = kcu.createNewProducer(false)) { + try (Producer kafkaProducer = getKcu().createNewProducer(false)) { for (int i = 0; i < expectedMessageCount; i++) { String key = "key-" + i; Future send = kafkaProducer.send(new ProducerRecord<>(inputName, key, "value-" + i), (meta, exception) -> { @@ -169,10 +169,10 @@ void testVertxConcurrency() { // run parallel-consumer log.debug("Starting test"); - KafkaProducer newProducer = kcu.createNewProducer(commitMode.equals(PERIODIC_TRANSACTIONAL_PRODUCER)); + KafkaProducer newProducer = getKcu().createNewProducer(commitMode.equals(PERIODIC_TRANSACTIONAL_PRODUCER)); Properties consumerProps = new Properties(); - KafkaConsumer newConsumer = kcu.createNewConsumer(true, consumerProps); + KafkaConsumer newConsumer = getKcu().createNewConsumer(true, consumerProps); var pc = new VertxParallelEoSStreamProcessor(ParallelConsumerOptions.builder() .ordering(order)