From 21f08f6cdcd0516a57903382a6f009ed215f73f1 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Tue, 8 Mar 2022 22:28:58 +0000 Subject: [PATCH 01/54] START: the basics of a single queue --- .../AbstractParallelEoSStreamProcessor.java | 51 ++++++++++++------ .../internal/BrokerPollSystem.java | 52 +++++++++++++------ 2 files changed, 71 insertions(+), 32 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 ff9747d6d..03a19c478 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 @@ -10,15 +10,9 @@ import io.confluent.parallelconsumer.PollContextInternal; import io.confluent.parallelconsumer.state.WorkContainer; import io.confluent.parallelconsumer.state.WorkManager; -import lombok.AccessLevel; -import lombok.Getter; -import lombok.Setter; -import lombok.SneakyThrows; +import lombok.*; import lombok.extern.slf4j.Slf4j; -import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.consumer.MockConsumer; +import org.apache.kafka.clients.consumer.*; import org.apache.kafka.clients.consumer.internals.ConsumerCoordinator; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.Time; @@ -99,7 +93,21 @@ public abstract class AbstractParallelEoSStreamProcessor implements Parall * Collection of work waiting to be */ @Getter(PROTECTED) - private final BlockingQueue> workMailBox = new LinkedBlockingQueue<>(); // Thread safe, highly performant, non blocking + private final BlockingQueue workMailBox = new LinkedBlockingQueue<>(); // Thread safe, highly performant, non blocking + + @Value + private class ActionItem { + WorkContainer workContainer; + ConsumerRecords consumerRecords; + +// public static ActionItem ofRecords(ConsumerRecords polledRecords) { +// return new ActionItem(null, polledRecords); +// } +// +// public static ActionItem ofWork(WorkContainer work) { +// new ActionItem(work, null); +// } + } private final BrokerPollSystem brokerPollSubsystem; @@ -880,7 +888,7 @@ private void transitionToClosing() { */ private void processWorkCompleteMailBox() { log.trace("Processing mailbox (might block waiting for results)..."); - Set> results = new HashSet<>(); + Set results = new HashSet<>(); final Duration timeToBlockFor = getTimeToBlockFor(); @@ -893,7 +901,7 @@ private void processWorkCompleteMailBox() { // wait for work, with a timeToBlockFor for sanity log.trace("Blocking poll {}", timeToBlockFor); try { - WorkContainer firstBlockingPoll = workMailBox.poll(timeToBlockFor.toMillis(), MILLISECONDS); + ActionItem firstBlockingPoll = workMailBox.poll(timeToBlockFor.toMillis(), MILLISECONDS); if (firstBlockingPoll == null) { log.debug("Mailbox results returned null, indicating timeToBlockFor (which was set as {})", timeToBlockFor); } else { @@ -915,10 +923,16 @@ private void processWorkCompleteMailBox() { workMailBox.drainTo(results, size); log.trace("Processing drained work {}...", results.size()); - for (var work : results) { - MDC.put("offset", work.toString()); - wm.handleFutureResult(work); - MDC.clear(); + for (var action : results) { + WorkContainer work = action.getWorkContainer(); + if (work == null) { + ConsumerRecords consumerRecords = action.getConsumerRecords(); + wm.registerWork(consumerRecords); + } else { + MDC.put("offset", work.toString()); + wm.handleFutureResult(work); + MDC.clear(); + } } } @@ -1118,7 +1132,12 @@ protected void onUserFunctionSuccess(WorkContainer wc, List resultsFrom protected void addToMailbox(WorkContainer wc) { String state = wc.isUserFunctionSucceeded() ? "succeeded" : "FAILED"; log.debug("Adding {} {} to mailbox...", state, wc); - workMailBox.add(wc); + workMailBox.add(new ActionItem(wc, null)); + } + + public void registerWork(ConsumerRecords polledRecords) { + log.debug("Adding {} to mailbox...", polledRecords); + workMailBox.add(new ActionItem(null, polledRecords)); } /** diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java index ea7d3826c..75cea9844 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java @@ -107,22 +107,7 @@ private boolean controlLoop() { committer.ifPresent(x -> x.claim()); try { while (state != closed) { - log.trace("Loop: Broker poller: ({})", state); - if (state == running || state == draining) { // if draining - subs will be paused, so use this to just sleep - ConsumerRecords polledRecords = pollBrokerForRecords(); - log.debug("Got {} records in poll result", polledRecords.count()); - - if (!polledRecords.isEmpty()) { - log.trace("Loop: Register work"); - wm.registerWork(polledRecords); - - // notify control work has been registered, in case it's sleeping waiting for work that will never come - if (wm.isStarvedForNewWork()) { - log.trace("Apparently no work is being done, make sure Control is awake to receive messages"); - pc.notifySomethingToDo(); - } - } - } + handlePoll(); maybeDoCommit(); @@ -143,6 +128,41 @@ private boolean controlLoop() { } } + private void handlePoll() { + log.trace("Loop: Broker poller: ({})", state); + if (state == running || state == draining) { // if draining - subs will be paused, so use this to just sleep + ConsumerRecords polledRecords = pollBrokerForRecords(); + log.debug("Got {} records in poll result", polledRecords.count()); + + if (!polledRecords.isEmpty()) { + log.trace("Loop: Register work"); + pc.registerWork(polledRecords); +// wm.registerWork(polledRecords); + + // notify control work has been registered, in case it's sleeping waiting for work that will never come + if (wm.isStarvedForNewWork()) { + log.trace("Apparently no work is being done, make sure Control is awake to receive messages"); + pc.notifySomethingToDo(); + } + } + } + } + + private void transitionToCloseMaybe() { + // make sure everything is committed + if (isResponsibleForCommits() && !wm.isRecordsAwaitingToBeCommitted()) { + // transition to closing + state = State.closing; + } else { + log.trace("Draining, but work still needs to be committed. Yielding thread to avoid busy wait."); + try { + Thread.sleep(1); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + } + private void doClose() { log.debug("Doing close..."); doPause(); From 3f1f3ff01cfbd23d6a2ae4e0e0d33111db5d6081 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Tue, 8 Mar 2022 22:50:25 +0000 Subject: [PATCH 02/54] step: remove work mailbox manager --- .../AbstractParallelEoSStreamProcessor.java | 43 ++- .../internal/BrokerPollSystem.java | 15 +- .../state/PartitionMonitor.java | 15 +- .../state/WorkMailBoxManager.java | 262 +++++++++--------- .../parallelconsumer/state/WorkManager.java | 183 +++++++----- 5 files changed, 272 insertions(+), 246 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 03a19c478..34e7af5a9 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 @@ -627,15 +627,15 @@ private void controlLoop(Function, List> userFu // int newWork = handleWork(userFunction, callback); - if (state == running) { - if (!wm.isSufficientlyLoaded() & brokerPollSubsystem.isPaused()) { - // can occur - log.debug("Found Poller paused with not enough front loaded messages, ensuring poller is awake (mail: {} vs target: {})", - wm.getAmountOfWorkQueuedWaitingIngestion(), - options.getTargetAmountOfRecordsInFlight()); - brokerPollSubsystem.wakeupIfPaused(); - } - } +// if (state == running) { +// if (!wm.isSufficientlyLoaded() & brokerPollSubsystem.isPaused()) { + // can occur +// log.debug("Found Poller paused with not enough front loaded messages, ensuring poller is awake (mail: {} vs target: {})", +//// wm.getAmountOfWorkQueuedWaitingIngestion(), +// options.getTargetAmountOfRecordsInFlight()); +// brokerPollSubsystem.wakeupIfPaused(); +// } +// } log.trace("Loop: Process mailbox"); processWorkCompleteMailBox(); @@ -820,20 +820,16 @@ protected int getQueueTargetLoaded() { * Checks the system has enough pressure in the pipeline of work, if not attempts to step up the load factor. */ protected void checkPipelinePressure() { - boolean moreWorkInQueuesAvailableThatHaveNotBeenPulled = wm.getAmountOfWorkQueuedWaitingIngestion() > options.getTargetAmountOfRecordsInFlight(); if (log.isTraceEnabled()) log.trace("Queue pressure check: (current size: {}, loaded target: {}, factor: {}) " + - "if (isPoolQueueLow() {} && moreWorkInQueuesAvailableThatHaveNotBeenPulled {} && lastWorkRequestWasFulfilled {}))", + "if (isPoolQueueLow() {} && lastWorkRequestWasFulfilled {}))", getNumberOfUserFunctionsQueued(), getQueueTargetLoaded(), dynamicExtraLoadFactor.getCurrentFactor(), isPoolQueueLow(), - moreWorkInQueuesAvailableThatHaveNotBeenPulled, lastWorkRequestWasFulfilled); - if (isPoolQueueLow() - && moreWorkInQueuesAvailableThatHaveNotBeenPulled - && lastWorkRequestWasFulfilled) { + if (isPoolQueueLow() && lastWorkRequestWasFulfilled) { boolean steppedUp = dynamicExtraLoadFactor.maybeStepUp(); if (steppedUp) { log.debug("isPoolQueueLow(): Executor pool queue is not loaded with enough work (queue: {} vs target: {}), stepped up loading factor to {}", @@ -855,10 +851,10 @@ private boolean isPoolQueueLow() { int queueSize = getNumberOfUserFunctionsQueued(); int queueTarget = getPoolLoadTarget(); boolean workAmountBelowTarget = queueSize <= queueTarget; - boolean hasWorkInMailboxes = wm.hasWorkAwaitingIngestionToShards(); - log.debug("isPoolQueueLow()? workAmountBelowTarget {} {} vs {} && wm.hasWorkInMailboxes() {};", - workAmountBelowTarget, queueSize, queueTarget, hasWorkInMailboxes); - return workAmountBelowTarget && hasWorkInMailboxes; +// boolean hasWorkInMailboxes = wm.hasWorkAwaitingIngestionToShards(); + log.debug("isPoolQueueLow()? workAmountBelowTarget {} {} vs {};", + workAmountBelowTarget, queueSize, queueTarget); + return workAmountBelowTarget; } private void drain() { @@ -939,18 +935,11 @@ private void processWorkCompleteMailBox() { /** * The amount of time to block poll in this cycle * - * @return either the duration until next commit, or next work retry - * @see WorkManager#isStarvedForNewWork() + * @return either the duration until next commit, or next work retry // * @see WorkManager#isStarvedForNewWork() * @see WorkManager#getTotalWorkAwaitingIngestion() * @see ParallelConsumerOptions#getTargetAmountOfRecordsInFlight() */ private Duration getTimeToBlockFor() { - // should not block as not enough work is being done, and there's more work to ingest - boolean ingestionWorkAndStarved = wm.hasWorkAwaitingIngestionToShards() && wm.isStarvedForNewWork(); - if (ingestionWorkAndStarved) { - log.debug("Work waiting to be ingested, and not enough work in flight - will not block"); - return Duration.ofMillis(0); - } // if less than target work already in flight, don't sleep longer than the next retry time for failed work, if it exists - so that we can wake up and maybe retry the failed work else if (!wm.isWorkInFlightMeetingTarget()) { // though check if we have work awaiting retry diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java index 75cea9844..b45347861 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java @@ -139,11 +139,11 @@ private void handlePoll() { pc.registerWork(polledRecords); // wm.registerWork(polledRecords); - // notify control work has been registered, in case it's sleeping waiting for work that will never come - if (wm.isStarvedForNewWork()) { - log.trace("Apparently no work is being done, make sure Control is awake to receive messages"); - pc.notifySomethingToDo(); - } +// // notify control work has been registered, in case it's sleeping waiting for work that will never come +// if (wm.isStarvedForNewWork()) { +// log.trace("Apparently no work is being done, make sure Control is awake to receive messages"); +// pc.notifySomethingToDo(); +// } } } } @@ -225,10 +225,9 @@ private void doPauseMaybe() { }); } else { if (log.isDebugEnabled()) { - log.debug("Should pause but pause rate limit exceeded {} vs {}. Queued: {}", + log.debug("Should pause but pause rate limit exceeded {} vs {}.", pauseLimiter.getElapsedDuration(), - pauseLimiter.getRate(), - wm.getAmountOfWorkQueuedWaitingIngestion()); + pauseLimiter.getRate()); } } } diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java index 65c19b2dd..00bb9635a 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java @@ -14,10 +14,7 @@ import lombok.RequiredArgsConstructor; import lombok.Setter; import lombok.extern.slf4j.Slf4j; -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.*; import org.apache.kafka.common.TopicPartition; import java.time.Clock; @@ -360,6 +357,16 @@ boolean maybeRegisterNewRecordAsWork(final ConsumerRecord rec) { } } + + /** + * @see #maybeRegisterNewRecordAsWork(ConsumerRecord) + */ + public void maybeRegisterNewRecordAsWork(ConsumerRecords records) { + for (ConsumerRecord consumerRec : records) { + maybeRegisterNewRecordAsWork(consumerRec); + } + } + public Map collectDirtyCommitData() { var dirties = new HashMap(); for (var state : getAssignedPartitions().values()) { diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkMailBoxManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkMailBoxManager.java index 8e9e04a9c..4ef00bb9a 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkMailBoxManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkMailBoxManager.java @@ -3,134 +3,134 @@ /*- * Copyright (C) 2020-2022 Confluent, Inc. */ -import io.confluent.parallelconsumer.internal.BrokerPollSystem; -import io.confluent.parallelconsumer.internal.CountingCRLinkedList; -import lombok.extern.slf4j.Slf4j; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.common.TopicPartition; - -import java.util.Collection; -import java.util.LinkedList; -import java.util.Queue; -import java.util.concurrent.LinkedBlockingQueue; - -import static io.confluent.csid.utils.KafkaUtils.toTopicPartition; - -/** - * Handles the incoming mail for {@link WorkManager}. - */ -@Slf4j -public class WorkMailBoxManager { - - /** - * The number of nested {@link ConsumerRecord} entries in the shared blocking mail box. Cached for performance. - */ - private int sharedBoxNestedRecordCount; - - /** - * The shared mailbox. Doesn't need to be thread safe as we already need synchronize on it. - */ - private final LinkedBlockingQueue> workInbox = new LinkedBlockingQueue<>(); - - /** - * Mailbox where mail is transferred to immediately. - */ - private final CountingCRLinkedList internalBatchMailQueue = new CountingCRLinkedList<>(); - - /** - * Queue of records flattened from the {@link #internalBatchMailQueue}. - *

- * This is needed because {@link java.util.concurrent.BlockingQueue#drainTo(Collection)} must drain to a collection - * of the same type. We could have {@link BrokerPollSystem} do the flattening, but that would require many calls to - * the Concurrent queue, where this only needs one. Also as we don't expect there to be that many elements in these - * collections (as they contain large batches of records), the overhead will be small. - */ - private final Queue> internalFlattenedMailQueue = new LinkedList<>(); - - /** - * @return amount of work queued in the mail box, awaiting processing into shards, not exact - */ - Integer getAmountOfWorkQueuedWaitingIngestion() { - return sharedBoxNestedRecordCount + - internalBatchMailQueue.getNestedCount() + - internalFlattenedMailQueue.size(); - } - - /** - * Work must be registered in offset order - *

- * Thread safe for use by control and broker poller thread. - * - * @see WorkManager#onSuccessResult - * @see WorkManager#raisePartitionHighWaterMark - */ - public void registerWork(final ConsumerRecords records) { - synchronized (workInbox) { - sharedBoxNestedRecordCount += records.count(); - workInbox.add(records); - } - } - - - /** - * Must synchronise to keep sharedBoxNestedRecordCount in lock step with the inbox. Register is easy, but drain you - * need to run through an intermediary collection and then count the nested elements, to know how many to subtract - * from the Atomic nested count. - *

- * Plus registering work is relatively infrequent, so shouldn't worry about a little synchronized here - makes it - * much simpler. - */ - private void drainSharedMailbox() { - synchronized (workInbox) { - workInbox.drainTo(internalBatchMailQueue); - sharedBoxNestedRecordCount = 0; - } - } - - /** - * Take our inbound messages from the {@link BrokerPollSystem} and add them to our registry. - */ - private synchronized void flattenBatchQueue() { - drainSharedMailbox(); - - // flatten - while (!internalBatchMailQueue.isEmpty()) { - ConsumerRecords consumerRecords = internalBatchMailQueue.poll(); - log.debug("Flattening {} records", consumerRecords.count()); - for (final ConsumerRecord consumerRecord : consumerRecords) { - internalFlattenedMailQueue.add(consumerRecord); - } - } - } - - /** - * Remove revoked work from the mailbox - */ - public synchronized void onPartitionsRemoved(final Collection removedPartitions) { - log.debug("Removing stale work from inbox queues"); - flattenBatchQueue(); - internalFlattenedMailQueue.removeIf(rec -> - removedPartitions.contains(toTopicPartition(rec)) - ); - } - - public synchronized boolean internalFlattenedMailQueueIsEmpty() { - return internalFlattenedMailQueue.isEmpty(); - } - - /** - * @return the next element in our outbound queue, or null if empty - */ - public synchronized ConsumerRecord internalFlattenedMailQueuePoll() { - if (internalBatchMailQueue.isEmpty()) { - // flatten the batch queue in batches when needed - flattenBatchQueue(); - } - return internalFlattenedMailQueue.poll(); - } - - public int internalFlattenedMailQueueSize() { - return internalFlattenedMailQueue.size(); - } -} +//import io.confluent.parallelconsumer.internal.BrokerPollSystem; +//import io.confluent.parallelconsumer.internal.CountingCRLinkedList; +//import lombok.extern.slf4j.Slf4j; +//import org.apache.kafka.clients.consumer.ConsumerRecord; +//import org.apache.kafka.clients.consumer.ConsumerRecords; +//import org.apache.kafka.common.TopicPartition; +// +//import java.util.Collection; +//import java.util.LinkedList; +//import java.util.Queue; +//import java.util.concurrent.LinkedBlockingQueue; +// +//import static io.confluent.csid.utils.KafkaUtils.toTopicPartition; +// +///** +// * Handles the incoming mail for {@link WorkManager}. +// */ +//@Slf4j +//public class WorkMailBoxManager { +// +// /** +// * The number of nested {@link ConsumerRecord} entries in the shared blocking mail box. Cached for performance. +// */ +// private int sharedBoxNestedRecordCount; +// +// /** +// * The shared mailbox. Doesn't need to be thread safe as we already need synchronize on it. +// */ +// private final LinkedBlockingQueue> workInbox = new LinkedBlockingQueue<>(); +// +// /** +// * Mailbox where mail is transferred to immediately. +// */ +// private final CountingCRLinkedList internalBatchMailQueue = new CountingCRLinkedList<>(); +// +// /** +// * Queue of records flattened from the {@link #internalBatchMailQueue}. +// *

+// * This is needed because {@link java.util.concurrent.BlockingQueue#drainTo(Collection)} must drain to a collection +// * of the same type. We could have {@link BrokerPollSystem} do the flattening, but that would require many calls to +// * the Concurrent queue, where this only needs one. Also as we don't expect there to be that many elements in these +// * collections (as they contain large batches of records), the overhead will be small. +// */ +// private final Queue> internalFlattenedMailQueue = new LinkedList<>(); +// +// /** +// * @return amount of work queued in the mail box, awaiting processing into shards, not exact +// */ +// Integer getAmountOfWorkQueuedWaitingIngestion() { +// return sharedBoxNestedRecordCount + +// internalBatchMailQueue.getNestedCount() + +// internalFlattenedMailQueue.size(); +// } +// +// /** +// * Work must be registered in offset order +// *

+// * Thread safe for use by control and broker poller thread. +// * +// * @see WorkManager#onSuccessResult +// * @see WorkManager#raisePartitionHighWaterMark +// */ +// public void registerWork(final ConsumerRecords records) { +// synchronized (workInbox) { +// sharedBoxNestedRecordCount += records.count(); +// workInbox.add(records); +// } +// } +// +// +// /** +// * Must synchronise to keep sharedBoxNestedRecordCount in lock step with the inbox. Register is easy, but drain you +// * need to run through an intermediary collection and then count the nested elements, to know how many to subtract +// * from the Atomic nested count. +// *

+// * Plus registering work is relatively infrequent, so shouldn't worry about a little synchronized here - makes it +// * much simpler. +// */ +// private void drainSharedMailbox() { +// synchronized (workInbox) { +// workInbox.drainTo(internalBatchMailQueue); +// sharedBoxNestedRecordCount = 0; +// } +// } +// +// /** +// * Take our inbound messages from the {@link BrokerPollSystem} and add them to our registry. +// */ +// private synchronized void flattenBatchQueue() { +// drainSharedMailbox(); +// +// // flatten +// while (!internalBatchMailQueue.isEmpty()) { +// ConsumerRecords consumerRecords = internalBatchMailQueue.poll(); +// log.debug("Flattening {} records", consumerRecords.count()); +// for (final ConsumerRecord consumerRecord : consumerRecords) { +// internalFlattenedMailQueue.add(consumerRecord); +// } +// } +// } +// +// /** +// * Remove revoked work from the mailbox +// */ +// public synchronized void onPartitionsRemoved(final Collection removedPartitions) { +// log.debug("Removing stale work from inbox queues"); +// flattenBatchQueue(); +// internalFlattenedMailQueue.removeIf(rec -> +// removedPartitions.contains(toTopicPartition(rec)) +// ); +// } +// +// public synchronized boolean internalFlattenedMailQueueIsEmpty() { +// return internalFlattenedMailQueue.isEmpty(); +// } +// +// /** +// * @return the next element in our outbound queue, or null if empty +// */ +// public synchronized ConsumerRecord internalFlattenedMailQueuePoll() { +// if (internalBatchMailQueue.isEmpty()) { +// // flatten the batch queue in batches when needed +// flattenBatchQueue(); +// } +// return internalFlattenedMailQueue.poll(); +// } +// +// public int internalFlattenedMailQueueSize() { +// return internalFlattenedMailQueue.size(); +// } +//} diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java index 93d3aecd1..97bc81a1f 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java @@ -64,7 +64,7 @@ public class WorkManager implements ConsumerRebalanceListener { */ private final DynamicLoadFactor dynamicLoadFactor; - private final WorkMailBoxManager wmbm; +// private final WorkMailBoxManager wmbm; @Getter private int numberRecordsOutForProcessing = 0; @@ -90,7 +90,7 @@ public WorkManager(final ParallelConsumerOptions newOptions, final org.apa final DynamicLoadFactor dynamicExtraLoadFactor, Clock clock) { this.options = newOptions; this.dynamicLoadFactor = dynamicExtraLoadFactor; - this.wmbm = new WorkMailBoxManager<>(); +// this.wmbm = new WorkMailBoxManager<>(); this.sm = new ShardManager<>(options, this, clock); this.pm = new PartitionMonitor<>(consumer, sm, options, clock); } @@ -126,41 +126,65 @@ public void onPartitionsLost(Collection partitions) { } void onPartitionsRemoved(final Collection partitions) { - wmbm.onPartitionsRemoved(partitions); +// wmbm.onPartitionsRemoved(partitions); } public void registerWork(ConsumerRecords records) { - wmbm.registerWork(records); - } - - /** - * Moves the requested amount of work from initial queues into work queues, if available. - * - * @param requestedMaxWorkToRetrieve try to move at least this many messages into the inbound queues - * @return the number of extra records ingested due to request - */ - private int ingestPolledRecordsIntoQueues(long requestedMaxWorkToRetrieve) { - log.debug("Will attempt to register the requested {} - {} available in internal mailbox", - requestedMaxWorkToRetrieve, wmbm.internalFlattenedMailQueueSize()); - - // - var takenWorkCount = 0; - boolean continueIngesting; - do { - ConsumerRecord polledRecord = wmbm.internalFlattenedMailQueuePoll(); - boolean recordAddedAsWork = pm.maybeRegisterNewRecordAsWork(polledRecord); - if (recordAddedAsWork) { - takenWorkCount++; - } - boolean polledQueueNotExhausted = polledRecord != null; - boolean ingestTargetNotSatisfied = takenWorkCount < requestedMaxWorkToRetrieve; - continueIngesting = ingestTargetNotSatisfied && polledQueueNotExhausted; - } while (continueIngesting); - - log.debug("{} new records were registered.", takenWorkCount); - - return takenWorkCount; - } +// wmbm.registerWork(records); + pm.maybeRegisterNewRecordAsWork(records); + } + +// /** +// * Moves the requested amount of work from initial queues into work queues, if available. +// * +// * @param requestedMaxWorkToRetrieve try to move at least this many messages into the inbound queues +// * @return the number of extra records ingested due to request +// */ +// private int ingestPolledRecordsIntoQueues(long requestedMaxWorkToRetrieve) { +// log.debug("Will attempt to register the requested {} - {} available in internal mailbox", +// requestedMaxWorkToRetrieve, wmbm.internalFlattenedMailQueueSize()); +// +// // +// var takenWorkCount = 0; +// boolean continueIngesting; +// do { +// ConsumerRecord polledRecord = wmbm.internalFlattenedMailQueuePoll(); +// boolean recordAddedAsWork = pm.maybeRegisterNewRecordAsWork(polledRecord); +// if (recordAddedAsWork) { +// takenWorkCount++; +// } +// boolean polledQueueNotExhausted = polledRecord != null; +// boolean ingestTargetNotSatisfied = takenWorkCount < requestedMaxWorkToRetrieve; +// continueIngesting = ingestTargetNotSatisfied && polledQueueNotExhausted; +// } while (continueIngesting); +// +// log.debug("{} new records were registered.", takenWorkCount); +// +// return takenWorkCount; +// } + +// private int ingestPolledRecordsIntoQueues(long ) { +// log.debug("Will attempt to register the requested {} - {} available in internal mailbox", +// requestedMaxWorkToRetrieve, wmbm.internalFlattenedMailQueueSize()); +// +// // +// var takenWorkCount = 0; +// boolean continueIngesting; +// do { +// ConsumerRecord polledRecord = wmbm.internalFlattenedMailQueuePoll(); +// boolean recordAddedAsWork = pm.maybeRegisterNewRecordAsWork(polledRecord); +// if (recordAddedAsWork) { +// takenWorkCount++; +// } +// boolean polledQueueNotExhausted = polledRecord != null; +// boolean ingestTargetNotSatisfied = takenWorkCount < requestedMaxWorkToRetrieve; +// continueIngesting = ingestTargetNotSatisfied && polledQueueNotExhausted; +// } while (continueIngesting); +// +// log.debug("{} new records were registered.", takenWorkCount); +// +// return takenWorkCount; +// } /** * Get work with no limit on quantity, useful for testing. @@ -178,7 +202,7 @@ public List> getWorkIfAvailable(final int requestedMaxWorkTo return UniLists.of(); } - int ingested = tryToEnsureQuantityOfWorkQueuedAvailable(requestedMaxWorkToRetrieve); +// int ingested = tryToEnsureQuantityOfWorkQueuedAvailable(requestedMaxWorkToRetrieve); // var work = sm.getWorkIfAvailable(requestedMaxWorkToRetrieve); @@ -190,27 +214,27 @@ public List> getWorkIfAvailable(final int requestedMaxWorkTo return work; } - /** - * Tries to ensure there are at least this many records available in the queues - * - * @return the number of extra records ingested due to request - */ - // todo rename - shunt messages from internal buffer into queues - // visible for testing - public int tryToEnsureQuantityOfWorkQueuedAvailable(final int requestedMaxWorkToRetrieve) { - // todo this counts all partitions as a whole - this may cause some partitions to starve. need to round robin it? - long available = sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); - long extraNeededFromInboxToSatisfy = requestedMaxWorkToRetrieve - available; - log.debug("Requested: {}, available in shards: {}, will try to process from mailbox the delta of: {}", - requestedMaxWorkToRetrieve, available, extraNeededFromInboxToSatisfy); - - int ingested = ingestPolledRecordsIntoQueues(extraNeededFromInboxToSatisfy); - log.debug("Ingested an extra {} records", ingested); - - long ingestionOffBy = extraNeededFromInboxToSatisfy - ingested; - - return ingested; - } +// /** +// * Tries to ensure there are at least this many records available in the queues +// * +// * @return the number of extra records ingested due to request +// */ +// // todo rename - shunt messages from internal buffer into queues +// // visible for testing +// public int tryToEnsureQuantityOfWorkQueuedAvailable(final int requestedMaxWorkToRetrieve) { +// // todo this counts all partitions as a whole - this may cause some partitions to starve. need to round robin it? +// long available = sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); +// long extraNeededFromInboxToSatisfy = requestedMaxWorkToRetrieve - available; +// log.debug("Requested: {}, available in shards: {}, will try to process from mailbox the delta of: {}", +// requestedMaxWorkToRetrieve, available, extraNeededFromInboxToSatisfy); +// +// int ingested = ingestPolledRecordsIntoQueues(extraNeededFromInboxToSatisfy); +// log.debug("Ingested an extra {} records", ingested); +// +// long ingestionOffBy = extraNeededFromInboxToSatisfy - ingested; +// +// return ingested; +// } public void onSuccessResult(WorkContainer wc) { log.trace("Work success ({}), removing from processing shard queue", wc); @@ -248,9 +272,9 @@ public long getNumberOfEntriesInPartitionQueues() { return pm.getNumberOfEntriesInPartitionQueues(); } - public Integer getAmountOfWorkQueuedWaitingIngestion() { - return wmbm.getAmountOfWorkQueuedWaitingIngestion(); - } +// public Integer getAmountOfWorkQueuedWaitingIngestion() { +// return wmbm.getAmountOfWorkQueuedWaitingIngestion(); +// } public Map collectCommitDataForDirtyPartitions() { return pm.collectDirtyCommitData(); @@ -287,7 +311,7 @@ public boolean shouldThrottle() { * should be downloaded (or pipelined in the Consumer) */ public boolean isSufficientlyLoaded() { - return getAmountOfWorkQueuedWaitingIngestion() > options.getTargetAmountOfRecordsInFlight() * getLoadingFactor(); + return getTotalWorkAwaitingIngestion() > options.getTargetAmountOfRecordsInFlight() * getLoadingFactor(); } private int getLoadingFactor() { @@ -310,23 +334,30 @@ public boolean isWorkInFlightMeetingTarget() { * @return Work count in mailbox plus work added to the processing shards */ public long getTotalWorkAwaitingIngestion() { - long workQueuedInShardsCount = sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); - Integer workQueuedInMailboxCount = getAmountOfWorkQueuedWaitingIngestion(); - return workQueuedInShardsCount + workQueuedInMailboxCount; +// return sm.getNumberOfEntriesInPartitionQueues + return sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); +// long workQueuedInShardsCount = sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); +// Integer workQueuedInMailboxCount = getAmountOfWorkQueuedWaitingIngestion(); +// return workQueuedInShardsCount + workQueuedInMailboxCount; } - public boolean hasWorkAwaitingIngestionToShards() { - return getAmountOfWorkQueuedWaitingIngestion() > 0; + public long getNumberOfWorkQueuedInShardsAwaitingSelection() { + return sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); } +// public boolean hasWorkAwaitingIngestionToShards() { +// return getAmountOfWorkQueuedWaitingIngestion() > 0; +// } + public boolean hasWorkInCommitQueues() { return pm.hasWorkInCommitQueues(); } public boolean isRecordsAwaitingProcessing() { - long partitionWorkRemainingCount = sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); - boolean internalQueuesNotEmpty = hasWorkAwaitingIngestionToShards(); - return partitionWorkRemainingCount > 0 || internalQueuesNotEmpty; + return sm.getNumberOfWorkQueuedInShardsAwaitingSelection() > 0; +// long partitionWorkRemainingCount = sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); +// boolean internalQueuesNotEmpty = hasWorkAwaitingIngestionToShards(); +// return partitionWorkRemainingCount > 0 || internalQueuesNotEmpty; } public boolean isRecordsAwaitingToBeCommitted() { @@ -361,12 +392,12 @@ public Optional getLowestRetryTime() { return sm.getLowestRetryTime(); } - /** - * @return true if more records are needed to be sent out for processing (not enough in queues to satisfy - * concurrency target) - */ - public boolean isStarvedForNewWork() { - long queued = getTotalWorkAwaitingIngestion(); - return queued < options.getTargetAmountOfRecordsInFlight(); - } +// /** +// * @return true if more records are needed to be sent out for processing (not enough in queues to satisfy +// * concurrency target) +// */ +// public boolean isStarvedForNewWork() { +// long queued = getTotalWorkAwaitingIngestion(); +// return queued < options.getTargetAmountOfRecordsInFlight(); +// } } From 92b50cbedf8322efd4898ff7f9bc7b0bc960c1bd Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Tue, 8 Mar 2022 23:54:35 +0000 Subject: [PATCH 03/54] step: BROKEN: assign epoch to record immediately --- .../AbstractParallelEoSStreamProcessor.java | 13 ++++-- .../internal/BrokerPollSystem.java | 45 ++++++++++++++++--- .../state/PartitionMonitor.java | 10 +++-- .../parallelconsumer/state/WorkManager.java | 3 +- .../offsets/OffsetEncodingTests.java | 6 ++- 5 files changed, 59 insertions(+), 18 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 34e7af5a9..bd7dda342 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 @@ -12,7 +12,10 @@ import io.confluent.parallelconsumer.state.WorkManager; import lombok.*; import lombok.extern.slf4j.Slf4j; -import org.apache.kafka.clients.consumer.*; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.MockConsumer; import org.apache.kafka.clients.consumer.internals.ConsumerCoordinator; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.Time; @@ -98,7 +101,7 @@ public abstract class AbstractParallelEoSStreamProcessor implements Parall @Value private class ActionItem { WorkContainer workContainer; - ConsumerRecords consumerRecords; + BrokerPollSystem.EpochAndRecords consumerRecords; // public static ActionItem ofRecords(ConsumerRecords polledRecords) { // return new ActionItem(null, polledRecords); @@ -332,6 +335,7 @@ public void onPartitionsRevoked(Collection partitions) { public void onPartitionsAssigned(Collection partitions) { numberOfAssignedPartitions = numberOfAssignedPartitions + partitions.size(); log.info("Assigned {} total ({} new) partition(s) {}", numberOfAssignedPartitions, partitions.size(), partitions); + brokerPollSubsystem.onPartitionsAssigned(partitions); wm.onPartitionsAssigned(partitions); usersConsumerRebalanceListener.ifPresent(x -> x.onPartitionsAssigned(partitions)); notifySomethingToDo(); @@ -345,6 +349,7 @@ public void onPartitionsAssigned(Collection partitions) { @Override public void onPartitionsLost(Collection partitions) { numberOfAssignedPartitions = numberOfAssignedPartitions - partitions.size(); + brokerPollSubsystem.onPartitionsLost(partitions); wm.onPartitionsLost(partitions); usersConsumerRebalanceListener.ifPresent(x -> x.onPartitionsLost(partitions)); } @@ -922,7 +927,7 @@ private void processWorkCompleteMailBox() { for (var action : results) { WorkContainer work = action.getWorkContainer(); if (work == null) { - ConsumerRecords consumerRecords = action.getConsumerRecords(); + BrokerPollSystem.EpochAndRecords consumerRecords = action.getConsumerRecords(); wm.registerWork(consumerRecords); } else { MDC.put("offset", work.toString()); @@ -1124,7 +1129,7 @@ protected void addToMailbox(WorkContainer wc) { workMailBox.add(new ActionItem(wc, null)); } - public void registerWork(ConsumerRecords polledRecords) { + public void registerWork(BrokerPollSystem.EpochAndRecords polledRecords) { log.debug("Adding {} to mailbox...", polledRecords); workMailBox.add(new ActionItem(null, polledRecords)); } diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java index b45347861..c1fe46436 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java @@ -10,7 +10,9 @@ import lombok.Getter; import lombok.Setter; import lombok.SneakyThrows; +import lombok.Value; import lombok.extern.slf4j.Slf4j; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.common.TopicPartition; import org.slf4j.MDC; @@ -18,6 +20,7 @@ import javax.naming.InitialContext; import javax.naming.NamingException; import java.time.Duration; +import java.util.Collection; import java.util.Optional; import java.util.Set; import java.util.concurrent.*; @@ -34,7 +37,7 @@ * @param */ @Slf4j -public class BrokerPollSystem implements OffsetCommitter { +public class BrokerPollSystem implements OffsetCommitter, ConsumerRebalanceListener { private final ConsumerManager consumerManager; @@ -59,11 +62,14 @@ public class BrokerPollSystem implements OffsetCommitter { private final WorkManager wm; + private long epoch = 0L; + public BrokerPollSystem(ConsumerManager consumerMgr, WorkManager wm, AbstractParallelEoSStreamProcessor pc, final ParallelConsumerOptions options) { this.wm = wm; this.pc = pc; this.consumerManager = consumerMgr; + switch (options.getCommitMode()) { case PERIODIC_CONSUMER_SYNC, PERIODIC_CONSUMER_ASYNCHRONOUS -> { ConsumerOffsetCommitter consumerCommitter = new ConsumerOffsetCommitter<>(consumerMgr, wm, options); @@ -131,10 +137,10 @@ private boolean controlLoop() { private void handlePoll() { log.trace("Loop: Broker poller: ({})", state); if (state == running || state == draining) { // if draining - subs will be paused, so use this to just sleep - ConsumerRecords polledRecords = pollBrokerForRecords(); - log.debug("Got {} records in poll result", polledRecords.count()); + EpochAndRecords polledRecords = pollBrokerForRecords(); + log.debug("Got {} records in poll result", polledRecords.getPoll().count()); - if (!polledRecords.isEmpty()) { + if (!polledRecords.getPoll().isEmpty()) { log.trace("Loop: Register work"); pc.registerWork(polledRecords); // wm.registerWork(polledRecords); @@ -186,7 +192,33 @@ private boolean isResponsibleForCommits() { return committer.isPresent(); } - private ConsumerRecords pollBrokerForRecords() { + @Override + public void onPartitionsRevoked(Collection partitions) { + epoch++; + } + + @Override + public void onPartitionsAssigned(Collection partitions) { + epoch++; + } + + @Value + public + class EpochAndRecords { + ConsumerRecords poll; + long myEpoch; + + public EpochAndRecords(ConsumerRecords poll) { + this.poll = poll; + this.myEpoch = getEpoch(); + } + } + + private long getEpoch() { + return epoch; + } + + private EpochAndRecords pollBrokerForRecords() { managePauseOfSubscription(); log.debug("Subscriptions are paused: {}", paused); @@ -196,8 +228,9 @@ private ConsumerRecords pollBrokerForRecords() { log.debug("Long polling broker with timeout {}, might appear to sleep here if subs are paused, or no data available on broker. Run state: {}", thisLongPollTimeout, state); ConsumerRecords poll = consumerManager.poll(thisLongPollTimeout); + log.debug("Poll completed"); - return poll; + return new EpochAndRecords(poll); } /** diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java index 00bb9635a..4843d12e5 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java @@ -332,7 +332,7 @@ public void onFailure(WorkContainer wc) { * * @return true if the record was taken, false if it was skipped (previously successful) */ - boolean maybeRegisterNewRecordAsWork(final ConsumerRecord rec) { + boolean maybeRegisterNewRecordAsWork(final ConsumerRecord rec, final long epoch) { if (rec == null) return false; synchronized (partitionStates) { @@ -361,9 +361,11 @@ boolean maybeRegisterNewRecordAsWork(final ConsumerRecord rec) { /** * @see #maybeRegisterNewRecordAsWork(ConsumerRecord) */ - public void maybeRegisterNewRecordAsWork(ConsumerRecords records) { - for (ConsumerRecord consumerRec : records) { - maybeRegisterNewRecordAsWork(consumerRec); + public void maybeRegisterNewRecordAsWork(BrokerPollSystem.EpochAndRecords records) { + // todo unchecked + ConsumerRecords poll = records.getPoll(); + for (ConsumerRecord consumerRec : poll) { + maybeRegisterNewRecordAsWork(consumerRec, records.getMyEpoch()); } } diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java index 97bc81a1f..414c8bb4b 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java @@ -13,7 +13,6 @@ import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import pl.tlinkowski.unij.api.UniLists; @@ -129,7 +128,7 @@ void onPartitionsRemoved(final Collection partitions) { // wmbm.onPartitionsRemoved(partitions); } - public void registerWork(ConsumerRecords records) { + public void registerWork(BrokerPollSystem.EpochAndRecords records) { // wmbm.registerWork(records); pm.maybeRegisterNewRecordAsWork(records); } 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 0507a78ec..3d2a5c034 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 @@ -8,6 +8,8 @@ import io.confluent.csid.utils.KafkaTestUtils; import io.confluent.parallelconsumer.ParallelConsumerOptions; import io.confluent.parallelconsumer.ParallelEoSStreamProcessorTestBase; +import io.confluent.parallelconsumer.internal.BrokerPollSystem; +import io.confluent.parallelconsumer.internal.EpochAndRecords; import io.confluent.parallelconsumer.state.WorkContainer; import io.confluent.parallelconsumer.state.WorkManager; import lombok.SneakyThrows; @@ -177,7 +179,7 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset { WorkManager wmm = new WorkManager<>(options, consumerSpy); wmm.onPartitionsAssigned(UniSets.of(new TopicPartition(INPUT_TOPIC, 0))); - wmm.registerWork(testRecords); + wmm.registerWork(new BrokerPollSystem.EpochAndRecords(testRecords)); List> work = wmm.getWorkIfAvailable(); assertThat(work).hasSameSizeAs(records); @@ -222,7 +224,7 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset { var newWm = new WorkManager<>(options, consumerSpy); newWm.onPartitionsAssigned(UniSets.of(tp)); - newWm.registerWork(testRecords); + newWm.registerWork(new EpochAndRecords(testRecords, 0)); var pm = newWm.getPm(); var partitionState = pm.getPartitionState(tp); From 1653bc21ae50b770bf53e8b06e0f4c7b508db080 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Thu, 17 Mar 2022 11:48:07 +0000 Subject: [PATCH 04/54] step - trying to test perf --- .../AbstractParallelEoSStreamProcessor.java | 8 +-- .../internal/BrokerPollSystem.java | 15 +--- .../internal/EpochAndRecords.java | 15 ++++ .../state/PartitionMonitor.java | 3 +- .../parallelconsumer/state/WorkManager.java | 12 +++- .../offsets/OffsetEncodingTests.java | 4 +- .../state/WorkManagerTest.java | 68 +++++++++++++++++-- 7 files changed, 98 insertions(+), 27 deletions(-) create mode 100644 parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecords.java 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 bd7dda342..d992f3d2a 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 @@ -101,7 +101,7 @@ public abstract class AbstractParallelEoSStreamProcessor implements Parall @Value private class ActionItem { WorkContainer workContainer; - BrokerPollSystem.EpochAndRecords consumerRecords; + EpochAndRecords consumerRecords; // public static ActionItem ofRecords(ConsumerRecords polledRecords) { // return new ActionItem(null, polledRecords); @@ -927,7 +927,7 @@ private void processWorkCompleteMailBox() { for (var action : results) { WorkContainer work = action.getWorkContainer(); if (work == null) { - BrokerPollSystem.EpochAndRecords consumerRecords = action.getConsumerRecords(); + EpochAndRecords consumerRecords = action.getConsumerRecords(); wm.registerWork(consumerRecords); } else { MDC.put("offset", work.toString()); @@ -946,7 +946,7 @@ private void processWorkCompleteMailBox() { */ private Duration getTimeToBlockFor() { // if less than target work already in flight, don't sleep longer than the next retry time for failed work, if it exists - so that we can wake up and maybe retry the failed work - else if (!wm.isWorkInFlightMeetingTarget()) { + if (!wm.isWorkInFlightMeetingTarget()) { // though check if we have work awaiting retry var lowestScheduledOpt = wm.getLowestRetryTime(); if (lowestScheduledOpt.isPresent()) { @@ -1129,7 +1129,7 @@ protected void addToMailbox(WorkContainer wc) { workMailBox.add(new ActionItem(wc, null)); } - public void registerWork(BrokerPollSystem.EpochAndRecords polledRecords) { + public void registerWork(EpochAndRecords polledRecords) { log.debug("Adding {} to mailbox...", polledRecords); workMailBox.add(new ActionItem(null, polledRecords)); } diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java index c1fe46436..8c045adb8 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java @@ -10,7 +10,6 @@ import lombok.Getter; import lombok.Setter; import lombok.SneakyThrows; -import lombok.Value; import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -202,18 +201,6 @@ public void onPartitionsAssigned(Collection partitions) { epoch++; } - @Value - public - class EpochAndRecords { - ConsumerRecords poll; - long myEpoch; - - public EpochAndRecords(ConsumerRecords poll) { - this.poll = poll; - this.myEpoch = getEpoch(); - } - } - private long getEpoch() { return epoch; } @@ -230,7 +217,7 @@ private EpochAndRecords pollBrokerForRecords() { ConsumerRecords poll = consumerManager.poll(thisLongPollTimeout); log.debug("Poll completed"); - return new EpochAndRecords(poll); + return new EpochAndRecords(poll, getEpoch()); } /** diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecords.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecords.java new file mode 100644 index 000000000..a9c91ecb1 --- /dev/null +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecords.java @@ -0,0 +1,15 @@ +package io.confluent.parallelconsumer.internal; + +import lombok.Value; +import org.apache.kafka.clients.consumer.ConsumerRecords; + +@Value +public class EpochAndRecords { + ConsumerRecords poll; + long myEpoch; + + public EpochAndRecords(ConsumerRecords poll, long epoch) { + this.poll = poll; + this.myEpoch = epoch; + } +} diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java index 4843d12e5..601343a50 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java @@ -8,6 +8,7 @@ import io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder; import io.confluent.parallelconsumer.internal.AbstractParallelEoSStreamProcessor; import io.confluent.parallelconsumer.internal.BrokerPollSystem; +import io.confluent.parallelconsumer.internal.EpochAndRecords; import io.confluent.parallelconsumer.internal.InternalRuntimeError; import io.confluent.parallelconsumer.offsets.OffsetMapCodecManager; import lombok.Getter; @@ -361,7 +362,7 @@ boolean maybeRegisterNewRecordAsWork(final ConsumerRecord rec, final long /** * @see #maybeRegisterNewRecordAsWork(ConsumerRecord) */ - public void maybeRegisterNewRecordAsWork(BrokerPollSystem.EpochAndRecords records) { + public void maybeRegisterNewRecordAsWork(EpochAndRecords records) { // todo unchecked ConsumerRecords poll = records.getPoll(); for (ConsumerRecord consumerRec : poll) { diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java index 414c8bb4b..a62ae0fe5 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java @@ -6,6 +6,8 @@ import io.confluent.csid.utils.TimeUtils; import io.confluent.parallelconsumer.ParallelConsumerOptions; +import io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder; +import io.confluent.parallelconsumer.internal.*; import io.confluent.parallelconsumer.internal.AbstractParallelEoSStreamProcessor; import io.confluent.parallelconsumer.internal.BrokerPollSystem; import io.confluent.parallelconsumer.internal.DynamicLoadFactor; @@ -13,6 +15,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import pl.tlinkowski.unij.api.UniLists; @@ -128,7 +131,14 @@ void onPartitionsRemoved(final Collection partitions) { // wmbm.onPartitionsRemoved(partitions); } - public void registerWork(BrokerPollSystem.EpochAndRecords records) { + /** + * for testing only + */ + public void registerWork(ConsumerRecords records) { + registerWork(new EpochAndRecords(records, 0)); + } + + public void registerWork(EpochAndRecords records) { // wmbm.registerWork(records); pm.maybeRegisterNewRecordAsWork(records); } 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 3d2a5c034..be6329695 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 @@ -8,7 +8,7 @@ import io.confluent.csid.utils.KafkaTestUtils; import io.confluent.parallelconsumer.ParallelConsumerOptions; import io.confluent.parallelconsumer.ParallelEoSStreamProcessorTestBase; -import io.confluent.parallelconsumer.internal.BrokerPollSystem; +import io.confluent.parallelconsumer.internal.EpochAndRecords; import io.confluent.parallelconsumer.internal.EpochAndRecords; import io.confluent.parallelconsumer.state.WorkContainer; import io.confluent.parallelconsumer.state.WorkManager; @@ -179,7 +179,7 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset { WorkManager wmm = new WorkManager<>(options, consumerSpy); wmm.onPartitionsAssigned(UniSets.of(new TopicPartition(INPUT_TOPIC, 0))); - wmm.registerWork(new BrokerPollSystem.EpochAndRecords(testRecords)); + wmm.registerWork(new EpochAndRecords(testRecords)); List> work = wmm.getWorkIfAvailable(); assertThat(work).hasSameSizeAs(records); diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java index e284bcd0c..916a94cc9 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java @@ -10,6 +10,7 @@ import io.confluent.csid.utils.TimeUtils; import io.confluent.parallelconsumer.FakeRuntimeError; import io.confluent.parallelconsumer.ParallelConsumerOptions; +import io.confluent.parallelconsumer.internal.EpochAndRecords; import io.confluent.parallelconsumer.truth.CommitHistorySubject; import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -32,7 +33,9 @@ import java.time.Duration; import java.util.*; +import java.util.stream.Collectors; +import static com.google.common.truth.Truth.assertWithMessage; import static io.confluent.csid.utils.Range.range; import static io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder.*; import static java.time.Duration.ofSeconds; @@ -70,8 +73,7 @@ private void setupWorkManager(ParallelConsumerOptions build) { log.debug("Heard some successful work: {}", work); successfulWork.add(work); }); - int partition = 0; - assignPartition(partition); + } private void assignPartition(final int partition) { @@ -83,19 +85,25 @@ private TopicPartition getTopicPartition(int partition) { return new TopicPartition(INPUT_TOPIC, partition); } + private void registerSomeWork() { + registerSomeWork(0); + } + /** * Adds 3 units of work */ - private void registerSomeWork() { + private void registerSomeWork(int partition) { + assignPartition(partition); + String key = "key-0"; - int partition = 0; + var rec0 = makeRec("0", key, partition); var rec1 = makeRec("1", key, partition); var rec2 = makeRec("2", key, partition); Map>> m = new HashMap<>(); m.put(getTopicPartition(partition), of(rec0, rec1, rec2)); var recs = new ConsumerRecords<>(m); - wm.registerWork(recs); + wm.registerWork(new EpochAndRecords(recs, 0)); } private ConsumerRecord makeRec(String value, String key, int partition) { @@ -658,4 +666,54 @@ void resumesFromNextShard(ParallelConsumerOptions.ProcessingOrder order) { } } + + /** + * Checks that when using shards are not starved when there's enough work queued to satisfy poll request from the + * initial request (without needing to iterate to other shards) + */ + @Test + void starvation() { + setupWorkManager(ParallelConsumerOptions.builder() + .ordering(PARTITION) + .build()); + + registerSomeWork(0); + registerSomeWork(1); + registerSomeWork(2); + + var allWork = new ArrayList>(); + + { + var work = wm.maybeGetWorkIfAvailable(2); + allWork.addAll(work); + + assertWithMessage("Should be able to get 2 records of work, one from each partition shard") + .that(work).hasSize(2); + + // + var tpOne = work.get(0).getTopicPartition(); + var tpTwo = work.get(1).getTopicPartition(); + assertWithMessage("The partitions should be different") + .that(tpOne).isNotEqualTo(tpTwo); + + } + + { + var work = wm.maybeGetWorkIfAvailable(2); + assertWithMessage("Should be able to get only 1 more, from the third shard") + .that(work).hasSize(1); + allWork.addAll(work); + + // + var tpOne = work.get(0).getTopicPartition(); + } + + assertWithMessage("TPs all unique") + .that(allWork.stream() + .map(WorkContainer::getTopicPartition) + .collect(Collectors.toList())) + .containsNoDuplicates(); + + } + } From 103c677cdc379f64cb85d1e392211fe366f73435 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Thu, 17 Mar 2022 11:51:23 +0000 Subject: [PATCH 05/54] update --- .../confluent/parallelconsumer/state/WorkManagerTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java index 916a94cc9..df4a6a2a9 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java @@ -640,8 +640,8 @@ void resumesFromNextShard(ParallelConsumerOptions.ProcessingOrder order) { var recs = new ConsumerRecords<>(m); wm.registerWork(recs); - // force ingestion of records - see refactor: Queue unification #219 - wm.tryToEnsureQuantityOfWorkQueuedAvailable(100); +// // force ingestion of records - see refactor: Queue unification #219 +// wm.tryToEnsureQuantityOfWorkQueuedAvailable(100); var workContainersOne = wm.getWorkIfAvailable(1); var workContainersTwo = wm.getWorkIfAvailable(1); @@ -684,7 +684,7 @@ void starvation() { var allWork = new ArrayList>(); { - var work = wm.maybeGetWorkIfAvailable(2); + var work = wm.getWorkIfAvailable(2); allWork.addAll(work); assertWithMessage("Should be able to get 2 records of work, one from each partition shard") @@ -699,7 +699,7 @@ void starvation() { } { - var work = wm.maybeGetWorkIfAvailable(2); + var work = wm.getWorkIfAvailable(2); assertWithMessage("Should be able to get only 1 more, from the third shard") .that(work).hasSize(1); allWork.addAll(work); From eed21906f9f55eb1794b36c2177ade70a3473dc7 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Thu, 17 Mar 2022 11:54:44 +0000 Subject: [PATCH 06/54] logs --- parallel-consumer-core/src/test/resources/logback-test.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/parallel-consumer-core/src/test/resources/logback-test.xml b/parallel-consumer-core/src/test/resources/logback-test.xml index 906410547..22f7628f0 100644 --- a/parallel-consumer-core/src/test/resources/logback-test.xml +++ b/parallel-consumer-core/src/test/resources/logback-test.xml @@ -31,6 +31,8 @@ + + From 02ee3cd412a69d9a0e629cdca535b9ebd58dafc9 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Thu, 17 Mar 2022 15:21:10 +0000 Subject: [PATCH 07/54] fix: Debug output for sorted encoding pairs PriorityQueue only provides a sorted `poll`, whereas TreeSet iterates in sorted order. --- .../io/confluent/parallelconsumer/offsets/BitSetEncoder.java | 2 +- .../parallelconsumer/offsets/EncodedOffsetPair.java | 4 +++- .../parallelconsumer/offsets/OffsetSimultaneousEncoder.java | 5 ++--- 3 files changed, 6 insertions(+), 5 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 1ecdc556e..47f02dae5 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 @@ -108,7 +108,7 @@ private ByteBuffer initV2(int bitsetEntriesRequired) throws BitSetEncodingNotSup private ByteBuffer initV1(int bitsetEntriesRequired) throws BitSetEncodingNotSupportedException { if (bitsetEntriesRequired > Short.MAX_VALUE) { // need to upgrade to using Integer for the bitset length, but can't change serialisation format in-place - throw new BitSetEncodingNotSupportedException("BitSet V1 too long to encode, bitset length overflows Short.MAX_VALUE: " + bitsetEntriesRequired + ". (max: " + Short.MAX_VALUE + ")"); + throw new BitSetEncodingNotSupportedException("input too long to encode for BitSet V1, length overflows Short.MAX_VALUE: " + bitsetEntriesRequired + ". (max: " + Short.MAX_VALUE + ")"); } // prep bit set buffer diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/offsets/EncodedOffsetPair.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/offsets/EncodedOffsetPair.java index 7a9697969..65dd6318e 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/offsets/EncodedOffsetPair.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/offsets/EncodedOffsetPair.java @@ -11,6 +11,7 @@ import lombok.extern.slf4j.Slf4j; import java.nio.ByteBuffer; +import java.util.Comparator; import static io.confluent.parallelconsumer.offsets.OffsetBitSet.deserialiseBitSetWrap; import static io.confluent.parallelconsumer.offsets.OffsetBitSet.deserialiseBitSetWrapToIncompletes; @@ -27,6 +28,7 @@ @Slf4j public final class EncodedOffsetPair implements Comparable { + public static final Comparator SIZE_COMPARATOR = Comparator.comparingInt(x -> x.data.capacity()); @Getter OffsetEncoding encoding; @Getter @@ -42,7 +44,7 @@ public final class EncodedOffsetPair implements Comparable { @Override public int compareTo(EncodedOffsetPair o) { - return Integer.compare(data.capacity(), o.getData().capacity()); + return SIZE_COMPARATOR.compare(this, o); } /** diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/offsets/OffsetSimultaneousEncoder.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/offsets/OffsetSimultaneousEncoder.java index 237bccdad..55471af66 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/offsets/OffsetSimultaneousEncoder.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/offsets/OffsetSimultaneousEncoder.java @@ -61,8 +61,7 @@ public class OffsetSimultaneousEncoder { * @see #packSmallest() */ @Getter - PriorityQueue sortedEncodings = new PriorityQueue<>(); - + SortedSet sortedEncodings = new TreeSet<>(); /** * Force the encoder to also add the compressed versions. Useful for testing. @@ -226,7 +225,7 @@ public byte[] packSmallest() throws NoEncodingPossibleException { if (sortedEncodings.isEmpty()) { throw new NoEncodingPossibleException("No encodings could be used"); } - final EncodedOffsetPair best = this.sortedEncodings.poll(); + final EncodedOffsetPair best = this.sortedEncodings.first(); log.debug("Compression chosen is: {}", best.encoding.name()); return packEncoding(best); } From 83fda734d4db0603d2601a2ee83eb17c06b3f7b9 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Thu, 17 Mar 2022 14:58:00 +0000 Subject: [PATCH 08/54] save --- .../internal/AbstractParallelEoSStreamProcessor.java | 9 ++++++++- .../offsets/OffsetMapCodecManager.java | 10 ++++++++-- .../confluent/parallelconsumer/state/WorkManager.java | 9 ++++++--- .../integrationTests/LargeVolumeInMemoryTests.java | 3 +++ .../integrationTests/MultiInstanceHighVolumeTest.java | 2 +- .../src/test/resources/logback-test.xml | 10 +++++----- 6 files changed, 31 insertions(+), 12 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 d992f3d2a..0b40d1146 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 @@ -945,6 +945,13 @@ private void processWorkCompleteMailBox() { * @see ParallelConsumerOptions#getTargetAmountOfRecordsInFlight() */ private Duration getTimeToBlockFor() { +// // should not block as not enough work is being done, and there's more work to ingest +// boolean ingestionWorkAndStarved = wm.hasWorkAwaitingIngestionToShards() && wm.isStarvedForNewWork(); +// if (ingestionWorkAndStarved) { +// log.debug("Work waiting to be ingested, and not enough work in flight - will not block"); +// return Duration.ofMillis(0); +// } + // if less than target work already in flight, don't sleep longer than the next retry time for failed work, if it exists - so that we can wake up and maybe retry the failed work if (!wm.isWorkInFlightMeetingTarget()) { // though check if we have work awaiting retry @@ -1125,7 +1132,7 @@ protected void onUserFunctionSuccess(WorkContainer wc, List resultsFrom protected void addToMailbox(WorkContainer wc) { String state = wc.isUserFunctionSucceeded() ? "succeeded" : "FAILED"; - log.debug("Adding {} {} to mailbox...", state, wc); + log.trace("Adding {} {} to mailbox...", state, wc); workMailBox.add(new ActionItem(wc, null)); } 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 9054b411b..1978510ee 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 @@ -15,6 +15,7 @@ import java.nio.ByteBuffer; import java.nio.charset.Charset; import java.util.*; +import java.util.stream.Collectors; import static io.confluent.csid.utils.StringUtils.msg; import static java.nio.charset.StandardCharsets.UTF_8; @@ -189,8 +190,13 @@ String serialiseIncompleteOffsetMapToBase64(long finalOffsetForPartition, Partit byte[] encodeOffsetsCompressed(long finalOffsetForPartition, PartitionState partitionState) throws NoEncodingPossibleException { var incompleteOffsets = partitionState.getIncompleteOffsetsBelowHighestSucceeded(); long highestSucceeded = partitionState.getOffsetHighestSucceeded(); - log.debug("Encoding partition {}: highest suceeded {}, incomplete offsets {}, ", partitionState.getTp(), highestSucceeded, incompleteOffsets); - OffsetSimultaneousEncoder simultaneousEncoder = new OffsetSimultaneousEncoder(finalOffsetForPartition, highestSucceeded, incompleteOffsets).invoke(); + if (log.isDebugEnabled()) { + log.debug("Encoding partition {}, highest succeeded {}, incomplete offsets to encode {}", + partitionState.getTp(), + highestSucceeded, + incompleteOffsets.stream().filter(x -> x < offsetHighestSucceeded).collect(Collectors.toList())); + } + OffsetSimultaneousEncoder simultaneousEncoder = new OffsetSimultaneousEncoder(finalOffsetForPartition, offsetHighestSucceeded, incompleteOffsets).invoke(); // if (forcedCodec.isPresent()) { diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java index a62ae0fe5..67b7b1cff 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java @@ -6,11 +6,10 @@ import io.confluent.csid.utils.TimeUtils; import io.confluent.parallelconsumer.ParallelConsumerOptions; -import io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder; -import io.confluent.parallelconsumer.internal.*; import io.confluent.parallelconsumer.internal.AbstractParallelEoSStreamProcessor; import io.confluent.parallelconsumer.internal.BrokerPollSystem; import io.confluent.parallelconsumer.internal.DynamicLoadFactor; +import io.confluent.parallelconsumer.internal.EpochAndRecords; import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; @@ -217,7 +216,11 @@ public List> getWorkIfAvailable(final int requestedMaxWorkTo var work = sm.getWorkIfAvailable(requestedMaxWorkToRetrieve); // - log.debug("Got {} records of work. In-flight: {}, Awaiting in commit (partition) queues: {}", work.size(), getNumberRecordsOutForProcessing(), getNumberOfEntriesInPartitionQueues()); + log.debug("Got {} of {} requested records of work. In-flight: {}, Awaiting in commit (partition) queues: {}", + work.size(), + requestedMaxWorkToRetrieve, + getNumberRecordsOutForProcessing(), + getNumberOfEntriesInPartitionQueues()); numberRecordsOutForProcessing += work.size(); return work; diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/LargeVolumeInMemoryTests.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/LargeVolumeInMemoryTests.java index f12d7f7b3..f0fb03e8e 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/LargeVolumeInMemoryTests.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/LargeVolumeInMemoryTests.java @@ -74,7 +74,10 @@ void load(CommitMode commitMode) { // allMessagesConsumedLatch.await(defaultTimeoutSeconds, SECONDS); // waitAtMost(defaultTimeout).until(() -> producerSpy.consumerGroupOffsetsHistory().size() > 0); + + // todo can remove? parallelConsumer.waitForProcessedNotCommitted(defaultTimeout.multipliedBy(10)); + parallelConsumer.close(); // assert quantity of produced messages diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/MultiInstanceHighVolumeTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/MultiInstanceHighVolumeTest.java index a0876227a..21572334a 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/MultiInstanceHighVolumeTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/MultiInstanceHighVolumeTest.java @@ -43,7 +43,7 @@ class MultiInstanceHighVolumeTest extends BrokerIntegrationTest public AtomicInteger processedCount = new AtomicInteger(0); public AtomicInteger producedCount = new AtomicInteger(0); - int maxPoll = 500; + int maxPoll = 5000000; ParallelConsumerOptions.CommitMode commitMode = ParallelConsumerOptions.CommitMode.PERIODIC_CONSUMER_SYNC; ParallelConsumerOptions.ProcessingOrder order = ParallelConsumerOptions.ProcessingOrder.KEY; diff --git a/parallel-consumer-core/src/test/resources/logback-test.xml b/parallel-consumer-core/src/test/resources/logback-test.xml index 22f7628f0..8bb0a5519 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 @@ - + @@ -38,7 +38,7 @@ - + @@ -49,10 +49,10 @@ - - + + - + From 2a37d46977bc51495e952307e9a2895b704e3826 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Fri, 25 Mar 2022 18:30:55 +0000 Subject: [PATCH 09/54] rebase update --- .../offsets/OffsetMapCodecManager.java | 4 ++-- .../parallelconsumer/offsets/OffsetEncodingTests.java | 10 ++++++---- 2 files changed, 8 insertions(+), 6 deletions(-) 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 1978510ee..a234f7ebb 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 @@ -194,9 +194,9 @@ byte[] encodeOffsetsCompressed(long finalOffsetForPartition, PartitionState x < offsetHighestSucceeded).collect(Collectors.toList())); + incompleteOffsets.stream().filter(x -> x < highestSucceeded).collect(Collectors.toList())); } - OffsetSimultaneousEncoder simultaneousEncoder = new OffsetSimultaneousEncoder(finalOffsetForPartition, offsetHighestSucceeded, incompleteOffsets).invoke(); + OffsetSimultaneousEncoder simultaneousEncoder = new OffsetSimultaneousEncoder(finalOffsetForPartition, highestSucceeded, incompleteOffsets).invoke(); // if (forcedCodec.isPresent()) { 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 be6329695..ea2267c36 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 @@ -9,7 +9,6 @@ import io.confluent.parallelconsumer.ParallelConsumerOptions; import io.confluent.parallelconsumer.ParallelEoSStreamProcessorTestBase; import io.confluent.parallelconsumer.internal.EpochAndRecords; -import io.confluent.parallelconsumer.internal.EpochAndRecords; import io.confluent.parallelconsumer.state.WorkContainer; import io.confluent.parallelconsumer.state.WorkManager; import lombok.SneakyThrows; @@ -179,7 +178,7 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset { WorkManager wmm = new WorkManager<>(options, consumerSpy); wmm.onPartitionsAssigned(UniSets.of(new TopicPartition(INPUT_TOPIC, 0))); - wmm.registerWork(new EpochAndRecords(testRecords)); + wmm.registerWork(new EpochAndRecords<>(testRecords, 0)); List> work = wmm.getWorkIfAvailable(); assertThat(work).hasSameSizeAs(records); @@ -249,7 +248,9 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset // force ingestion early, and check results { - int ingested = newWm.tryToEnsureQuantityOfWorkQueuedAvailable(Integer.MAX_VALUE); + // todo ingestion no longer a thing - what to do here? + Truth.assertThat(true).isFalse(); +// int ingested = newWm.tryToEnsureQuantityOfWorkQueuedAvailable(Integer.MAX_VALUE); if (!encodingsThatFail.contains(encoding)) { long offsetHighestSequentialSucceeded = partitionState.getOffsetHighestSequentialSucceeded(); @@ -264,7 +265,8 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset var incompletes = partitionState.getIncompleteOffsetsBelowHighestSucceeded(); Truth.assertThat(incompletes).containsExactlyElementsIn(expected); - assertThat(ingested).isEqualTo(testRecords.count() - 4); // 4 were succeeded + //todo and here +// assertThat(ingested).isEqualTo(testRecords.count() - 4); // 4 were succeeded Truth.assertThat(pm.isRecordPreviouslyCompleted(anIncompleteRecord)).isFalse(); } } From b17c838e89cd347874465bbe324ef53e16345d35 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Mon, 4 Apr 2022 18:35:37 +0100 Subject: [PATCH 10/54] step --- .../confluent/parallelconsumer/internal/EpochAndRecords.java | 4 ++++ .../io/confluent/parallelconsumer/offsets/BitSetEncoder.java | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecords.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecords.java index a9c91ecb1..e28690402 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecords.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecords.java @@ -1,5 +1,9 @@ package io.confluent.parallelconsumer.internal; +/*- + * Copyright (C) 2020-2022 Confluent, Inc. + */ + import lombok.Value; import org.apache.kafka.clients.consumer.ConsumerRecords; 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 47f02dae5..93cb0e085 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 @@ -1,7 +1,7 @@ package io.confluent.parallelconsumer.offsets; /*- - * Copyright (C) 2020-2021 Confluent, Inc. + * Copyright (C) 2020-2022 Confluent, Inc. */ import io.confluent.csid.utils.StringUtils; From e1141e4b3949d761d71445d2e647cea42c024cc1 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Tue, 5 Apr 2022 13:33:38 +0100 Subject: [PATCH 11/54] save --- parallel-consumer-core/pom.xml | 6 ++ .../state/WorkManagerTest.java | 57 +++++++++++-------- 2 files changed, 38 insertions(+), 25 deletions(-) diff --git a/parallel-consumer-core/pom.xml b/parallel-consumer-core/pom.xml index f725eee0c..d92f7836a 100644 --- a/parallel-consumer-core/pom.xml +++ b/parallel-consumer-core/pom.xml @@ -112,6 +112,12 @@ 7.2.8.RELEASE test + + io.stubbs.truth + truth-generator-api + 0.1-SNAPSHOT + test + diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java index df4a6a2a9..c63b56063 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java @@ -9,6 +9,7 @@ import io.confluent.csid.utils.LongPollingMockConsumer; import io.confluent.csid.utils.TimeUtils; import io.confluent.parallelconsumer.FakeRuntimeError; +import io.confluent.parallelconsumer.ManagedTruth; import io.confluent.parallelconsumer.ParallelConsumerOptions; import io.confluent.parallelconsumer.internal.EpochAndRecords; import io.confluent.parallelconsumer.truth.CommitHistorySubject; @@ -30,6 +31,7 @@ import org.junit.jupiter.params.provider.ValueSource; import org.threeten.extra.MutableClock; import pl.tlinkowski.unij.api.UniLists; +import pl.tlinkowski.unij.api.UniMaps; import java.time.Duration; import java.util.*; @@ -77,11 +79,11 @@ private void setupWorkManager(ParallelConsumerOptions build) { } private void assignPartition(final int partition) { - wm.onPartitionsAssigned(UniLists.of(getTopicPartition(partition))); + wm.onPartitionsAssigned(UniLists.of(topicPartitionOf(partition))); } @NotNull - private TopicPartition getTopicPartition(int partition) { + private TopicPartition topicPartitionOf(int partition) { return new TopicPartition(INPUT_TOPIC, partition); } @@ -101,7 +103,7 @@ private void registerSomeWork(int partition) { var rec1 = makeRec("1", key, partition); var rec2 = makeRec("2", key, partition); Map>> m = new HashMap<>(); - m.put(getTopicPartition(partition), of(rec0, rec1, rec2)); + m.put(topicPartitionOf(partition), of(rec0, rec1, rec2)); var recs = new ConsumerRecords<>(m); wm.registerWork(new EpochAndRecords(recs, 0)); } @@ -112,15 +114,23 @@ private ConsumerRecord makeRec(String value, String key, int par return stringStringConsumerRecord; } - @Test - void testRemovedUnordered() { - setupWorkManager(ParallelConsumerOptions.builder().ordering(UNORDERED).build()); + @ParameterizedTest + @EnumSource + void basic(ParallelConsumerOptions.ProcessingOrder order) { + setupWorkManager(ParallelConsumerOptions.builder() + .ordering(order) + .build()); registerSomeWork(); int max = 1; var gottenWork = wm.getWorkIfAvailable(max); - assertThat(gottenWork).hasSize(1); - assertOffsets(gottenWork, of(0)); + + if (order == UNORDERED) { + assertThat(gottenWork).hasSize(3); + } else { + assertThat(gottenWork).hasSize(1); + assertOffsets(gottenWork, of(0)); + } wm.onSuccessResult(gottenWork.get(0)); @@ -344,7 +354,7 @@ void insertWrongOrderPreservesOffsetOrdering() { var rec2 = new ConsumerRecord<>(INPUT_TOPIC, partition, 6, key, "value"); var rec3 = new ConsumerRecord<>(INPUT_TOPIC, partition, 8, key, "value"); Map>> m = new HashMap<>(); - m.put(getTopicPartition(partition), of(rec2, rec3, rec)); + m.put(topicPartitionOf(partition), of(rec2, rec3, rec)); var recs = new ConsumerRecords<>(m); // @@ -453,7 +463,7 @@ void orderedByPartitionsParallel() { var rec2 = new ConsumerRecord<>(INPUT_TOPIC, partition, 6, "66", "value"); var rec3 = new ConsumerRecord<>(INPUT_TOPIC, partition, 8, "66", "value"); Map>> m = new HashMap<>(); - m.put(getTopicPartition(partition), of(rec2, rec3, rec)); + m.put(topicPartitionOf(partition), of(rec2, rec3, rec)); var recs = new ConsumerRecords<>(m); // @@ -499,7 +509,7 @@ void orderedByKeyParallel() { var rec5 = new ConsumerRecord<>(INPUT_TOPIC, partition, 15, "key-a", "value"); var rec6 = new ConsumerRecord<>(INPUT_TOPIC, partition, 20, "key-c", "value"); Map>> m = new HashMap<>(); - m.put(getTopicPartition(partition), of(rec2, rec3, rec0, rec4, rec5, rec6)); + m.put(topicPartitionOf(partition), of(rec2, rec3, rec0, rec4, rec5, rec6)); var recs = new ConsumerRecords<>(m); // @@ -528,18 +538,14 @@ void orderedByKeyParallel() { assertOffsets(works, of()); } - @Test - @Disabled - public void unorderedPartitionsGreedy() { - } - - // @Test @ParameterizedTest @ValueSource(ints = {1, 2, 5, 10, 20, 30, 50, 1000}) void highVolumeKeyOrder(int quantity) { int uniqueKeys = 100; - var build = ParallelConsumerOptions.builder().ordering(KEY).build(); + var build = ParallelConsumerOptions.builder() + .ordering(KEY) + .build(); setupWorkManager(build); KafkaTestUtils ktu = new KafkaTestUtils(INPUT_TOPIC, null, new LongPollingMockConsumer<>(OffsetResetStrategy.EARLIEST)); @@ -550,18 +556,19 @@ void highVolumeKeyOrder(int quantity) { var flattened = ktu.flatten(records.values()); flattened.sort(comparingLong(ConsumerRecord::offset)); - Map>> m = new HashMap<>(); - m.put(getTopicPartition(0), flattened); - var recs = new ConsumerRecords<>(m); + var recs = new ConsumerRecords<>(UniMaps.of(topicPartitionOf(0), flattened)); // wm.registerWork(recs); + // + + // List> work = wm.getWorkIfAvailable(); // - assertThat(work).hasSameSizeAs(records.keySet()); + ManagedTruth.assertTruth(work).hasSameSizeAs(records); } @Test @@ -611,7 +618,7 @@ void workQueuesEmptyWhenAllWorkComplete() { var sync = completedFutureOffsets.values().stream().findFirst().get(); Truth.assertThat(sync.offset()).isEqualTo(3); Truth.assertThat(sync.metadata()).isEmpty(); - PartitionState state = wm.getPm().getPartitionState(getTopicPartition(0)); + PartitionState state = wm.getPm().getPartitionState(topicPartitionOf(0)); Truth.assertThat(state.getAllIncompleteOffsets()).isEmpty(); } @@ -634,9 +641,9 @@ void resumesFromNextShard(ParallelConsumerOptions.ProcessingOrder order) { assignPartition(2); Map>> m = new HashMap<>(); var rec = new ConsumerRecord<>(INPUT_TOPIC, 1, 11, "11", "value"); - m.put(getTopicPartition(1), of(rec)); + m.put(topicPartitionOf(1), of(rec)); var rec2 = new ConsumerRecord<>(INPUT_TOPIC, 2, 21, "21", "value"); - m.put(getTopicPartition(2), of(rec2)); + m.put(topicPartitionOf(2), of(rec2)); var recs = new ConsumerRecords<>(m); wm.registerWork(recs); From 5a3bb55892ea9733604fde80d733439533027a6e Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Tue, 5 Apr 2022 14:26:58 +0100 Subject: [PATCH 12/54] save --- parallel-consumer-core/pom.xml | 6 ---- .../state/PartitionMonitor.java | 3 +- .../parallelconsumer/state/WorkManager.java | 4 +-- .../confluent/csid/utils/KafkaTestUtils.java | 1 + .../OffsetEncodingBackPressureTest.java | 30 +++++++++++-------- .../state/WorkManagerTest.java | 30 ++++++++++++++----- .../src/test/resources/logback-test.xml | 5 ++-- 7 files changed, 48 insertions(+), 31 deletions(-) diff --git a/parallel-consumer-core/pom.xml b/parallel-consumer-core/pom.xml index d92f7836a..f725eee0c 100644 --- a/parallel-consumer-core/pom.xml +++ b/parallel-consumer-core/pom.xml @@ -112,12 +112,6 @@ 7.2.8.RELEASE test - - io.stubbs.truth - truth-generator-api - 0.1-SNAPSHOT - test - diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java index 601343a50..75af74ec1 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java @@ -362,8 +362,7 @@ boolean maybeRegisterNewRecordAsWork(final ConsumerRecord rec, final long /** * @see #maybeRegisterNewRecordAsWork(ConsumerRecord) */ - public void maybeRegisterNewRecordAsWork(EpochAndRecords records) { - // todo unchecked + public void maybeRegisterNewRecordAsWork(EpochAndRecords records) { ConsumerRecords poll = records.getPoll(); for (ConsumerRecord consumerRec : poll) { maybeRegisterNewRecordAsWork(consumerRec, records.getMyEpoch()); diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java index 67b7b1cff..0b4e51007 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java @@ -13,7 +13,6 @@ import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; @@ -127,11 +126,12 @@ public void onPartitionsLost(Collection partitions) { } void onPartitionsRemoved(final Collection partitions) { + // no-op - nothing to do // wmbm.onPartitionsRemoved(partitions); } /** - * for testing only + * Hard codes epoch as genesis - for testing only */ public void registerWork(ConsumerRecords records) { registerWork(new EpochAndRecords(records, 0)); diff --git a/parallel-consumer-core/src/test/java/io/confluent/csid/utils/KafkaTestUtils.java b/parallel-consumer-core/src/test/java/io/confluent/csid/utils/KafkaTestUtils.java index 98c8873ed..7fe977c56 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/csid/utils/KafkaTestUtils.java +++ b/parallel-consumer-core/src/test/java/io/confluent/csid/utils/KafkaTestUtils.java @@ -246,6 +246,7 @@ public void send(MockConsumer consumerSpy, HashMap consumerSpy, List> records) { + log.debug("Sending {} more messages to the consumer stub", records.size()); // send records in `correct` offset order as declared by the input data, regardless of the order of the input list List> sorted = new ArrayList(records); Collections.sort(sorted, Comparator.comparingLong(ConsumerRecord::offset)); diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java index d8b8c5b03..6c9ebc31d 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java @@ -92,17 +92,19 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O if (rec.offset() == offsetToBlock) { int attemptNumber = attempts.incrementAndGet(); if (attemptNumber == 1) { - log.debug("force first message to 'never' complete, causing a large offset encoding (lots of messages completing above the low water mark"); - awaitLatch(msgLock, 120); - log.debug("very slow message awoken, throwing exception"); + log.debug("Force first message to 'never' complete, causing a large offset encoding (lots of messages completing above the low water mark. Waiting for msgLock countdown."); + int sleepFor = 120; + awaitLatch(msgLock, sleepFor); + log.debug("Very slow message awoken, throwing exception"); throw new FakeRuntimeError("Fake error"); } else { - log.debug("Second attempt, sleeping"); + log.debug("Second attempt, waiting for msgLockTwo countdown"); awaitLatch(msgLockTwo, 60); log.debug("Second attempt, unlocked, succeeding"); } } else if (rec.offset() == 2l) { awaitLatch(msgLockThree); + log.debug("// msg 2L unblocked"); } else { sleepQuietly(1); } @@ -156,7 +158,7 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O assertThat(partitionBlocked).isFalse(); } - // feed more messages in order to threshold block - as Bitset requires linearly as much space as we are feeding messages into it, it's gauranteed to block + log.debug("// feed more messages in order to threshold block - as Bitset requires linearly as much space as we are feeding messages into it, it's guaranteed to block"); int extraRecordsToBlockWithThresholdBlocks = numberOfRecords / 2; { assertThat(wm.getPm().isAllowedMoreRecords(topicPartition)).isTrue(); // should initially be not blocked @@ -196,25 +198,29 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O ); } - // test max payload exceeded, payload dropped + log.debug("// test max payload exceeded, payload dropped"); int processedBeforePartitionBlock = userFuncFinishedCount.get(); int extraMessages = numberOfRecords + extraRecordsToBlockWithThresholdBlocks / 2; + log.debug("// messages already sent {}, sending {} more", processedBeforePartitionBlock, extraMessages); { - // force system to allow more records (i.e. the actual system attempts to never allow the payload to grow this big) + log.debug("// force system to allow more records (i.e. the actual system attempts to never allow the payload to grow this big)"); PartitionMonitor.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(2); // - msgLockThree.countDown(); // unlock to make state dirty to get a commit + log.debug("// unlock to make state dirty to get a commit"); + msgLockThree.countDown(); + log.debug("// send {} more messages", extraMessages); ktu.send(consumerSpy, ktu.generateRecords(extraMessages)); awaitForOneLoopCycle(); parallelConsumer.requestCommitAsap(); - // + log.debug("// wait for the new message to be processed"); await().atMost(defaultTimeout).untilAsserted(() -> - assertThat(userFuncFinishedCount.get()).isEqualTo(processedBeforePartitionBlock + extraMessages + 1) // some new message processed + assertThat(userFuncFinishedCount.get()).isEqualTo(processedBeforePartitionBlock + extraMessages + 1) ); - // assert payload missing from commit now + + log.debug("// assert payload missing from commit now"); await().untilAsserted(() -> { OffsetAndMetadata partitionCommit = getLastCommit(); assertThat(partitionCommit.offset()).isZero(); @@ -222,7 +228,7 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O }); } - // test failed messages can retry + log.debug("// test failed messages can retry"); { Duration aggressiveDelay = ofMillis(100); WorkContainer.setDefaultRetryDelay(aggressiveDelay); // more aggressive retry diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java index c63b56063..2f54d6306 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java @@ -122,21 +122,33 @@ void basic(ParallelConsumerOptions.ProcessingOrder order) { .build()); registerSomeWork(); - int max = 1; - var gottenWork = wm.getWorkIfAvailable(max); + // + var gottenWork = wm.getWorkIfAvailable(); if (order == UNORDERED) { assertThat(gottenWork).hasSize(3); + assertOffsets(gottenWork, of(0, 1, 2)); } else { assertThat(gottenWork).hasSize(1); assertOffsets(gottenWork, of(0)); } + // wm.onSuccessResult(gottenWork.get(0)); - gottenWork = wm.getWorkIfAvailable(max); - assertThat(gottenWork).hasSize(1); - assertOffsets(gottenWork, of(1)); + // + gottenWork = wm.getWorkIfAvailable(); + + if (order == UNORDERED) { + assertThat(gottenWork).isEmpty(); + } else { + assertThat(gottenWork).hasSize(1); + assertOffsets(gottenWork, of(1)); + } + + // + gottenWork = wm.getWorkIfAvailable(); + assertThat(gottenWork).isEmpty(); } @Test @@ -556,13 +568,17 @@ void highVolumeKeyOrder(int quantity) { var flattened = ktu.flatten(records.values()); flattened.sort(comparingLong(ConsumerRecord::offset)); - var recs = new ConsumerRecords<>(UniMaps.of(topicPartitionOf(0), flattened)); + int partition = 0; + var recs = new ConsumerRecords<>(UniMaps.of(topicPartitionOf(partition), flattened)); + + assignPartition(partition); // wm.registerWork(recs); // - + long awaiting = wm.getSm().getNumberOfWorkQueuedInShardsAwaitingSelection(); + assertThat(awaiting).isEqualTo(quantity); // List> work = wm.getWorkIfAvailable(); diff --git a/parallel-consumer-core/src/test/resources/logback-test.xml b/parallel-consumer-core/src/test/resources/logback-test.xml index 8bb0a5519..f8bfd5f45 100644 --- a/parallel-consumer-core/src/test/resources/logback-test.xml +++ b/parallel-consumer-core/src/test/resources/logback-test.xml @@ -28,9 +28,10 @@ - + - + + From 6a1464c9d665a673acedacdc85353b3af8caf9ab Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Tue, 5 Apr 2022 18:09:21 +0100 Subject: [PATCH 13/54] save: unit test version of offset encoding backpressure test --- .../internal/BrokerPollSystem.java | 4 +- .../internal/EpochAndRecords.java | 4 +- .../state/PartitionMonitor.java | 5 +- .../state/PartitionState.java | 4 + .../confluent/csid/utils/KafkaTestUtils.java | 17 +- .../OffsetEncodingBackPressureTest.java | 19 +- .../OffsetEncodingBackPressureUnitTest.java | 257 ++++++++++++++++++ .../offsets/OffsetEncodingTests.java | 6 +- .../state/WorkManagerTest.java | 2 - 9 files changed, 296 insertions(+), 22 deletions(-) create mode 100644 parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java index 8c045adb8..2a6e73257 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java @@ -137,9 +137,9 @@ private void handlePoll() { log.trace("Loop: Broker poller: ({})", state); if (state == running || state == draining) { // if draining - subs will be paused, so use this to just sleep EpochAndRecords polledRecords = pollBrokerForRecords(); - log.debug("Got {} records in poll result", polledRecords.getPoll().count()); + log.debug("Got {} records in poll result", polledRecords.getConsumerRecs().count()); - if (!polledRecords.getPoll().isEmpty()) { + if (!polledRecords.getConsumerRecs().isEmpty()) { log.trace("Loop: Register work"); pc.registerWork(polledRecords); // wm.registerWork(polledRecords); diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecords.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecords.java index e28690402..3e29ae22d 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecords.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecords.java @@ -9,11 +9,11 @@ @Value public class EpochAndRecords { - ConsumerRecords poll; + ConsumerRecords consumerRecs; long myEpoch; public EpochAndRecords(ConsumerRecords poll, long epoch) { - this.poll = poll; + this.consumerRecs = poll; this.myEpoch = epoch; } } diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java index 75af74ec1..b6a189cd4 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java @@ -46,6 +46,7 @@ public class PartitionMonitor implements ConsumerRebalanceListener { */ @Getter @Setter + // todo remove static private static double USED_PAYLOAD_THRESHOLD_MULTIPLIER = 0.75; private final Consumer consumer; @@ -363,8 +364,8 @@ boolean maybeRegisterNewRecordAsWork(final ConsumerRecord rec, final long * @see #maybeRegisterNewRecordAsWork(ConsumerRecord) */ public void maybeRegisterNewRecordAsWork(EpochAndRecords records) { - ConsumerRecords poll = records.getPoll(); - for (ConsumerRecord consumerRec : poll) { + ConsumerRecords recordCollection = records.getConsumerRecs(); + for (ConsumerRecord consumerRec : recordCollection) { maybeRegisterNewRecordAsWork(consumerRec, records.getMyEpoch()); } } 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 e2ccf4094..cd634689f 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 @@ -9,6 +9,7 @@ import io.confluent.parallelconsumer.offsets.OffsetMapCodecManager; import lombok.Getter; import lombok.Setter; +import lombok.ToString; import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.OffsetAndMetadata; @@ -27,6 +28,7 @@ import static java.util.Optional.of; import static lombok.AccessLevel.*; +@ToString @Slf4j public class PartitionState { @@ -100,6 +102,7 @@ public class PartitionState { * ({@link #getCommitDataIfDirty()}), or reading upon {@link #onPartitionsRemoved} */ // todo doesn't need to be concurrent any more? + @ToString.Exclude private final NavigableMap> commitQueue = new ConcurrentSkipListMap<>(); private NavigableMap> getCommitQueue() { @@ -313,3 +316,4 @@ public void onPartitionsRemoved(ShardManager sm) { sm.removeAnyShardsReferencedBy(getCommitQueue()); } } + diff --git a/parallel-consumer-core/src/test/java/io/confluent/csid/utils/KafkaTestUtils.java b/parallel-consumer-core/src/test/java/io/confluent/csid/utils/KafkaTestUtils.java index 7fe977c56..7ebeda8f1 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/csid/utils/KafkaTestUtils.java +++ b/parallel-consumer-core/src/test/java/io/confluent/csid/utils/KafkaTestUtils.java @@ -170,7 +170,8 @@ public static void assertLastCommitIs(final LongPollingMockConsumer> generateRecords(int quantity) { HashMap>> integerListHashMap = generateRecords(defaultKeys, quantity); - return flatten(integerListHashMap.values()); + Collection>> values = integerListHashMap.values(); + return flatten(values); } /** @@ -210,12 +211,12 @@ public ArrayList> generateRecordsForKey(Integer k return records; } - public List flatten(Collection> listlist) { - List all = new ArrayList<>(); - for (Collection value : listlist) { + public List> flatten(Collection>> listlist) { + SortedSet> all = new TreeSet<>(Comparator.comparing(ConsumerRecord::offset)); + for (Collection> value : listlist) { all.addAll(value); } - return all; + return new ArrayList<>(all); } @Setter @@ -248,8 +249,8 @@ public void send(MockConsumer consumerSpy, HashMap consumerSpy, List> records) { log.debug("Sending {} more messages to the consumer stub", records.size()); // send records in `correct` offset order as declared by the input data, regardless of the order of the input list - List> sorted = new ArrayList(records); - Collections.sort(sorted, Comparator.comparingLong(ConsumerRecord::offset)); + List> sorted = new ArrayList<>(records); + sorted.sort(Comparator.comparingLong(ConsumerRecord::offset)); for (ConsumerRecord record : sorted) { consumerSpy.addRecord(record); } @@ -276,7 +277,7 @@ public static void completeWork(final WorkManager wmm, final Wor } public List> sendRecords(final int i) { - List> consumerRecords = generateRecords(i); + var consumerRecords = generateRecords(i); send(consumerSpy, consumerRecords); return consumerRecords; } diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java index 6c9ebc31d..834c31b27 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java @@ -15,6 +15,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; +import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.parallel.Isolated; import org.junit.jupiter.api.parallel.ResourceAccessMode; @@ -48,11 +49,20 @@ * indirectly on the behaviour of the metadata size, even if not so explicitly. *

* See {@link OffsetMapCodecManager#METADATA_DATA_SIZE_RESOURCE_LOCK} + * + * @see OffsetMapCodecManager#METADATA_DATA_SIZE_RESOURCE_LOCK + * @see OffsetEncodingBackPressureUnitTest */ @Isolated // messes with static state - breaks other tests running in parallel @Slf4j +// todo this test is way too complicated and needs to be rewritten - OffsetEncodingBackPressureUnitTest class OffsetEncodingBackPressureTest extends ParallelEoSStreamProcessorTestBase { + @AfterAll + static void cleanup() { + PartitionMonitor.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(0.75); + } + /** * Tests that when required space for encoding offset becomes too large, back pressure is put into the system so * that no further messages for the given partitions can be taken for processing, until more messages complete. @@ -80,10 +90,10 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O CountDownLatch msgLock = new CountDownLatch(1); CountDownLatch msgLockTwo = new CountDownLatch(1); CountDownLatch msgLockThree = new CountDownLatch(1); - final int numberOfBlockedMessages = 2; AtomicInteger attempts = new AtomicInteger(0); long offsetToBlock = 0; List blockedOffsets = UniLists.of(0L, 2L); + final int numberOfBlockedMessages = blockedOffsets.size(); parallelConsumer.poll((rec) -> { @@ -166,7 +176,7 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O ktu.send(consumerSpy, ktu.generateRecords(extraRecordsToBlockWithThresholdBlocks)); awaitForOneLoopCycle(); - // assert partition now blocked from threshold + log.debug("// assert partition now blocked from threshold"); waitAtMost(ofSeconds(30)) .untilAsserted( () -> assertThat(wm.getPm().isBlocked(topicPartition)) @@ -180,7 +190,7 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O parallelConsumer.requestCommitAsap(); awaitForOneLoopCycle(); - // assert blocked, but can still write payload + log.debug("// assert blocked, but can still write payload"); // assert the committed offset metadata contains a payload waitAtMost(defaultTimeout).untilAsserted(() -> { @@ -198,6 +208,7 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O ); } + // recreates the situation where the payload size is too large and must be dropped log.debug("// test max payload exceeded, payload dropped"); int processedBeforePartitionBlock = userFuncFinishedCount.get(); int extraMessages = numberOfRecords + extraRecordsToBlockWithThresholdBlocks / 2; @@ -205,6 +216,8 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O { log.debug("// force system to allow more records (i.e. the actual system attempts to never allow the payload to grow this big)"); PartitionMonitor.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(2); + parallelConsumer.requestCommitAsap(); + awaitForOneLoopCycle(); // log.debug("// unlock to make state dirty to get a commit"); diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java new file mode 100644 index 000000000..803dc9e48 --- /dev/null +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java @@ -0,0 +1,257 @@ +package io.confluent.parallelconsumer.offsets; + +/*- + * Copyright (C) 2020-2022 Confluent, Inc. + */ + +import io.confluent.parallelconsumer.ParallelEoSStreamProcessorTestBase; +import io.confluent.parallelconsumer.state.PartitionMonitor; +import io.confluent.parallelconsumer.state.PartitionState; +import io.confluent.parallelconsumer.state.WorkContainer; +import io.confluent.parallelconsumer.state.WorkManager; +import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; +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.clients.consumer.OffsetAndMetadata; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.ResourceAccessMode; +import org.junit.jupiter.api.parallel.ResourceLock; +import pl.tlinkowski.unij.api.UniLists; +import pl.tlinkowski.unij.api.UniMaps; + +import java.time.Duration; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +import static io.confluent.parallelconsumer.ManagedTruth.assertTruth; +import static java.time.Duration.ofMillis; + +/** + * UnitTest version of {@link OffsetEncodingBackPressureTest}. + * + * @see OffsetEncodingBackPressureTest + */ +@Slf4j +class OffsetEncodingBackPressureUnitTest extends ParallelEoSStreamProcessorTestBase { + + @SneakyThrows + @Test + @ResourceLock(value = OffsetMapCodecManager.METADATA_DATA_SIZE_RESOURCE_LOCK, mode = ResourceAccessMode.READ_WRITE) + void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws OffsetDecodingError { + final int numberOfRecords = 1_00; + + // todo - very smelly - store for restoring + var realMax = OffsetMapCodecManager.DefaultMaxMetadataSize; + + // todo don't use static public accessors to change things - makes parallel testing harder and is smelly + OffsetMapCodecManager.DefaultMaxMetadataSize = 40; // reduce available to make testing easier + OffsetMapCodecManager.forcedCodec = Optional.of(OffsetEncoding.BitSetV2); // force one that takes a predictable large amount of space + + + var wm = parallelConsumer.getWm(); + var pm = wm.getPm(); + PartitionState partitionState = pm.getPartitionState(topicPartition); + + sendRecordsToWM(numberOfRecords, wm); + + final int numberOfBlockedMessages = 2; + var samplingOfShouldBeCompleteOffsets = UniLists.of(1L, 50L, 99L, (long) numberOfRecords - numberOfBlockedMessages); + var blockedOffsets = UniLists.of(0L, 2L); + + var completes = LongStreamEx.of(numberOfRecords).filter(x -> !blockedOffsets.contains(x)).boxed().toList(); + + List> workIfAvailable = wm.getWorkIfAvailable(); + assertTruth(workIfAvailable).hasSize(numberOfRecords); + + List> toSucceed = workIfAvailable.stream().filter(x -> !blockedOffsets.contains(x.offset())).collect(Collectors.toList()); + toSucceed.forEach(wm::onSuccessResult); + + try { + + // # assert commit ok - nothing blocked + { + Optional commitDataIfDirty = partitionState.getCommitDataIfDirty(); + assertTruth(partitionState).isAllowedMoreRecords(); + + int expectedHighestSeenOffset = numberOfRecords - 1; + // check("getOffsetHighestSucceeded()").that(actual.getOffsetHighestSucceeded()).isEqualTo(expected); + assertTruth(partitionState).getOffsetHighestSeen().isEqualTo(expectedHighestSeenOffset); + assertTruth(partitionState).getCommitDataIfDirty().hasOffsetEqualTo(0); + } + + + log.debug("// feed more messages in order to threshold block - as Bitset requires linearly as much space as we are feeding messages into it, it's guaranteed to block"); + int extraRecordsToBlockWithThresholdBlocks = numberOfRecords / 2; + { + sendRecordsToWM(extraRecordsToBlockWithThresholdBlocks, wm); + succeedExcept(wm, blockedOffsets); + + // triggers recompute of blockage + Optional commitDataIfDirty = partitionState.getCommitDataIfDirty(); + + log.debug("// assert partition now blocked from threshold"); + assertTruth(partitionState).isNotAllowedMoreRecords(); + + log.debug("// assert blocked, but can still write payload"); + assertTruth(partitionState).getCommitDataIfDirty().hasOffsetEqualTo(0L); + + // "The only incomplete record now is offset zero, which we are blocked on" + assertTruth(partitionState).getOffsetHighestSeen().isEqualTo(numberOfRecords + extraRecordsToBlockWithThresholdBlocks - 1); + assertTruth(partitionState).getCommitDataIfDirty().getMetadata().isNotEmpty(); + // todo assert with message "The only incomplete record now is offset zero, which we are blocked on" + assertTruth(partitionState).getAllIncompleteOffsets().containsNoneIn(samplingOfShouldBeCompleteOffsets); + assertTruth(partitionState).getAllIncompleteOffsets().containsExactlyElementsIn(blockedOffsets); + } + + + // recreates the situation where the payload size is too large and must be dropped + log.debug("// test max payload exceeded, payload dropped"); + int processedBeforePartitionBlock = extraRecordsToBlockWithThresholdBlocks + numberOfRecords - blockedOffsets.size(); + int extraMessages = numberOfRecords + extraRecordsToBlockWithThresholdBlocks / 2; + log.debug("// messages already sent {}, sending {} more", processedBeforePartitionBlock, extraMessages); + { + log.debug("// force system to allow more records (i.e. the actual system attempts to never allow the payload to grow this big)"); + PartitionMonitor.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(2); + + // + // unlock 2L as well + unblock(wm, workIfAvailable, 2L); + log.debug("// unlock to make state dirty to get a commit"); + Optional commitDataIfDirty = partitionState.getCommitDataIfDirty(); +// msgLockThree.countDown(); + + // + log.debug("// send {} more messages", extraMessages); + sendRecordsToWM(extraMessages, wm); + succeedExcept(wm, UniLists.of(0L)); + +// ktu.send(consumerSpy, ktu.generateRecords(extraMessages)); + +// awaitForOneLoopCycle(); +// parallelConsumer.requestCommitAsap(); + +// log.debug("// wait for the new message to be processed"); +// await().atMost(defaultTimeout).untilAsserted(() -> +// assertThat(userFuncFinishedCount.get()).isEqualTo(processedBeforePartitionBlock + extraMessages + 1) +// ); + + log.debug("// assert payload missing from commit now"); +// await().untilAsserted(() -> { +// Optional commitDataIfDirty = partitionState.getCommitDataIfDirty(); +// assertThat(partitionCommit.offset()).isZero(); +// Truth8.assertThat(commitDataIfDirty).isPresent(); +// assertTruth(commitDataIfDirty).get.getOffset().isEqualTo(0); + assertTruth(partitionState).getCommitDataIfDirty().hasOffsetEqualTo(0); +// assertThat(partitionCommit.metadata()).isBlank(); // missing offset encoding as too large + assertTruth(partitionState).getCommitDataIfDirty().getMetadata().isEmpty(); +// }); + } + + log.debug("// test failed messages can retry"); + { + { + // check it's not returned + List workIfAvailable1 = StreamEx.of(wm.getWorkIfAvailable()).map(WorkContainer::offset).toList(); + assertTruth(workIfAvailable1).doesNotContain(0L); + } + + // more aggressive retry + Duration aggressiveDelay = ofMillis(100); + WorkContainer.setDefaultRetryDelay(aggressiveDelay); + + // release message that was blocking partition progression + // fail the message +// msgLock.countDown(); + + wm.onFailureResult(findWC(workIfAvailable, 0L)); + + { + List workIfAvailable1 = StreamEx.of(wm.getWorkIfAvailable()).map(WorkContainer::offset).toList(); + assertTruth(workIfAvailable1).contains(0L); + } + +// +// +// // wait for the retry +// awaitForOneLoopCycle(); +// sleepQuietly(aggressiveDelay.toMillis()); +// await().until(() -> attempts.get() >= 2); +// +// // assert partition still blocked +// awaitForOneLoopCycle(); +// await().untilAsserted(() -> assertThat(wm.getPm().isAllowedMoreRecords(topicPartition)).isFalse()); + + // release the message for the second time, allowing it to succeed +// msgLockTwo.countDown(); + unblock(wm, workIfAvailable, 0L); + } + + // assert partition is now not blocked + { +// awaitForOneLoopCycle(); +// await().untilAsserted(() -> assertThat(wm.getPm().isAllowedMoreRecords(topicPartition)).isTrue()); + Optional commitDataIfDirty = partitionState.getCommitDataIfDirty(); + assertTruth(partitionState).isAllowedMoreRecords(); + + } + + // assert all committed, nothing blocked- next expected offset is now 1+ the offset of the final message we sent + { +// await().untilAsserted(() -> { +// List offsets = extractAllPartitionsOffsetsSequentially(false); +// assertThat(offsets).contains(userFuncFinishedCount.get()); +// }); + assertTruth(partitionState).getCommitDataIfDirty().getOffset().isEqualTo(processedBeforePartitionBlock + extraMessages + numberOfBlockedMessages); + +// await().untilAsserted(() -> assertThat(wm.getPm().isAllowedMoreRecords(topicPartition)).isTrue()); + assertTruth(partitionState).isAllowedMoreRecords(); + + } + } finally { +// // make sure to unlock threads - speeds up failed tests, instead of waiting for latch or close timeouts +// msgLock.countDown(); +// msgLockTwo.countDown(); + + // todo restore static defaults - lazy way to override settings at runtime but causes bugs by allowing them to be statically changeable + OffsetMapCodecManager.DefaultMaxMetadataSize = realMax; // todo wow this is smelly, but convenient + OffsetMapCodecManager.forcedCodec = Optional.empty(); + } + + + } + + private void succeedExcept(WorkManager wm, List incomplete) { + var workIfAvailable = wm.getWorkIfAvailable(); + var toSucceed = workIfAvailable.stream() + .filter(x -> !incomplete.contains(x.offset())) + .collect(Collectors.toList()); + toSucceed.forEach(wm::onSuccessResult); + } + + private void unblock(WorkManager wm, List> from, long offsetToUnblock) { + var unblock = findWC(from, offsetToUnblock); + wm.onSuccessResult(unblock); + } + + private WorkContainer findWC(List> from, long offsetToUnblock) { + return from.stream().filter(x -> x.offset() == offsetToUnblock).findFirst().get(); + } + + private void sendRecordsToWM(int numberOfRecords, WorkManager wm) { + log.debug("~Sending {} more records", numberOfRecords); + List> records = ktu.generateRecords(numberOfRecords); + wm.registerWork(new ConsumerRecords<>(UniMaps.of(topicPartition, records))); + } +// +// private OffsetAndMetadata getLastCommit() { +// List>> commitHistory = getCommitHistory(); +// Map> lastCommit = getLast(commitHistory).get(); +// Map allPartitionCommits = getOnlyOne(lastCommit).get(); +// return allPartitionCommits.get(topicPartition); +// } + +} 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 ea2267c36..be88df707 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 @@ -160,9 +160,9 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset incompleteRecords.remove(incompleteRecords.stream().filter(x -> x.offset() == 25_000).findFirst().get()); incompleteRecords.remove(incompleteRecords.stream().filter(x -> x.offset() == highest).findFirst().get()); - List expected = incompleteRecords.stream().map(ConsumerRecord::offset) + var expected = incompleteRecords.stream().map(ConsumerRecord::offset) .sorted() - .collect(Collectors.toList()); + .collect(Collectors.toSet()); // ktu.send(consumerSpy, records); @@ -171,7 +171,7 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset ParallelConsumerOptions options = parallelConsumer.getWm().getOptions(); HashMap>> recordsMap = new HashMap<>(); TopicPartition tp = new TopicPartition(INPUT_TOPIC, 0); - recordsMap.put(tp, records); + recordsMap.put(tp, new ArrayList<>(records)); ConsumerRecords testRecords = new ConsumerRecords<>(recordsMap); // write offsets diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java index 2f54d6306..a6aad1113 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java @@ -41,7 +41,6 @@ import static io.confluent.csid.utils.Range.range; import static io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder.*; import static java.time.Duration.ofSeconds; -import static java.util.Comparator.comparingLong; import static org.assertj.core.api.Assertions.assertThat; import static pl.tlinkowski.unij.api.UniLists.of; @@ -566,7 +565,6 @@ void highVolumeKeyOrder(int quantity) { var records = ktu.generateRecords(keys, quantity); var flattened = ktu.flatten(records.values()); - flattened.sort(comparingLong(ConsumerRecord::offset)); int partition = 0; var recs = new ConsumerRecords<>(UniMaps.of(topicPartitionOf(partition), flattened)); From 0604934e8492b5de3a887ba90400038c07db6056 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Tue, 5 Apr 2022 18:25:23 +0100 Subject: [PATCH 14/54] save --- .../confluent/csid/utils/LatchTestUtils.java | 2 +- ...actParallelEoSStreamProcessorTestBase.java | 2 +- .../OffsetEncodingBackPressureTest.java | 5 +- .../OffsetEncodingBackPressureUnitTest.java | 56 ++----------------- .../src/test/resources/logback-test.xml | 2 + 5 files changed, 12 insertions(+), 55 deletions(-) diff --git a/parallel-consumer-core/src/test/java/io/confluent/csid/utils/LatchTestUtils.java b/parallel-consumer-core/src/test/java/io/confluent/csid/utils/LatchTestUtils.java index 408c89ea6..888e8f644 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/csid/utils/LatchTestUtils.java +++ b/parallel-consumer-core/src/test/java/io/confluent/csid/utils/LatchTestUtils.java @@ -51,7 +51,7 @@ public static void awaitLatch(final CountDownLatch latch, final int seconds) { seconds, seconds - toSeconds(between(start, now()))); } if (latchReachedZero) { - log.trace("Latch released"); + log.trace("Latch was released (#countdown)"); } else { throw new TimeoutException("Latch await timeout (" + seconds + " seconds) - " + latch.getCount() + " count remaining"); } diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java index 2988f5540..955ece43c 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java @@ -143,7 +143,7 @@ public void close() { if (parentParallelConsumer.getFailureCause() != null) { log.error("PC has error - test failed"); } - log.debug("Test finished, closing pc..."); + log.debug("Test ended (maybe a failure), closing pc..."); parentParallelConsumer.close(); } else { log.debug("Test finished, pc already closed."); diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java index 834c31b27..71deb4315 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java @@ -55,7 +55,6 @@ */ @Isolated // messes with static state - breaks other tests running in parallel @Slf4j -// todo this test is way too complicated and needs to be rewritten - OffsetEncodingBackPressureUnitTest class OffsetEncodingBackPressureTest extends ParallelEoSStreamProcessorTestBase { @AfterAll @@ -279,8 +278,8 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O } } finally { // make sure to unlock threads - speeds up failed tests, instead of waiting for latch or close timeouts - msgLock.countDown(); - msgLockTwo.countDown(); +// msgLock.countDown(); +// msgLockTwo.countDown(); // todo restore static defaults - lazy way to override settings at runtime but causes bugs by allowing them to be statically changeable OffsetMapCodecManager.DefaultMaxMetadataSize = realMax; // todo wow this is smelly, but convenient diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java index 803dc9e48..c03ee2408 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java @@ -122,33 +122,16 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O unblock(wm, workIfAvailable, 2L); log.debug("// unlock to make state dirty to get a commit"); Optional commitDataIfDirty = partitionState.getCommitDataIfDirty(); -// msgLockThree.countDown(); // log.debug("// send {} more messages", extraMessages); sendRecordsToWM(extraMessages, wm); succeedExcept(wm, UniLists.of(0L)); -// ktu.send(consumerSpy, ktu.generateRecords(extraMessages)); - -// awaitForOneLoopCycle(); -// parallelConsumer.requestCommitAsap(); - -// log.debug("// wait for the new message to be processed"); -// await().atMost(defaultTimeout).untilAsserted(() -> -// assertThat(userFuncFinishedCount.get()).isEqualTo(processedBeforePartitionBlock + extraMessages + 1) -// ); - log.debug("// assert payload missing from commit now"); -// await().untilAsserted(() -> { -// Optional commitDataIfDirty = partitionState.getCommitDataIfDirty(); -// assertThat(partitionCommit.offset()).isZero(); -// Truth8.assertThat(commitDataIfDirty).isPresent(); -// assertTruth(commitDataIfDirty).get.getOffset().isEqualTo(0); + assertTruth(partitionState).getCommitDataIfDirty().hasOffsetEqualTo(0); -// assertThat(partitionCommit.metadata()).isBlank(); // missing offset encoding as too large assertTruth(partitionState).getCommitDataIfDirty().getMetadata().isEmpty(); -// }); } log.debug("// test failed messages can retry"); @@ -164,8 +147,6 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O WorkContainer.setDefaultRetryDelay(aggressiveDelay); // release message that was blocking partition progression - // fail the message -// msgLock.countDown(); wm.onFailureResult(findWC(workIfAvailable, 0L)); @@ -174,26 +155,13 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O assertTruth(workIfAvailable1).contains(0L); } -// -// -// // wait for the retry -// awaitForOneLoopCycle(); -// sleepQuietly(aggressiveDelay.toMillis()); -// await().until(() -> attempts.get() >= 2); -// -// // assert partition still blocked -// awaitForOneLoopCycle(); -// await().untilAsserted(() -> assertThat(wm.getPm().isAllowedMoreRecords(topicPartition)).isFalse()); - - // release the message for the second time, allowing it to succeed -// msgLockTwo.countDown(); + unblock(wm, workIfAvailable, 0L); } // assert partition is now not blocked { -// awaitForOneLoopCycle(); -// await().untilAsserted(() -> assertThat(wm.getPm().isAllowedMoreRecords(topicPartition)).isTrue()); + Optional commitDataIfDirty = partitionState.getCommitDataIfDirty(); assertTruth(partitionState).isAllowedMoreRecords(); @@ -201,20 +169,14 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O // assert all committed, nothing blocked- next expected offset is now 1+ the offset of the final message we sent { -// await().untilAsserted(() -> { -// List offsets = extractAllPartitionsOffsetsSequentially(false); -// assertThat(offsets).contains(userFuncFinishedCount.get()); -// }); + assertTruth(partitionState).getCommitDataIfDirty().getOffset().isEqualTo(processedBeforePartitionBlock + extraMessages + numberOfBlockedMessages); -// await().untilAsserted(() -> assertThat(wm.getPm().isAllowedMoreRecords(topicPartition)).isTrue()); assertTruth(partitionState).isAllowedMoreRecords(); } } finally { -// // make sure to unlock threads - speeds up failed tests, instead of waiting for latch or close timeouts -// msgLock.countDown(); -// msgLockTwo.countDown(); + // todo restore static defaults - lazy way to override settings at runtime but causes bugs by allowing them to be statically changeable OffsetMapCodecManager.DefaultMaxMetadataSize = realMax; // todo wow this is smelly, but convenient @@ -246,12 +208,6 @@ private void sendRecordsToWM(int numberOfRecords, WorkManager wm List> records = ktu.generateRecords(numberOfRecords); wm.registerWork(new ConsumerRecords<>(UniMaps.of(topicPartition, records))); } -// -// private OffsetAndMetadata getLastCommit() { -// List>> commitHistory = getCommitHistory(); -// Map> lastCommit = getLast(commitHistory).get(); -// Map allPartitionCommits = getOnlyOne(lastCommit).get(); -// return allPartitionCommits.get(topicPartition); -// } + } diff --git a/parallel-consumer-core/src/test/resources/logback-test.xml b/parallel-consumer-core/src/test/resources/logback-test.xml index f8bfd5f45..a4b7ad5c1 100644 --- a/parallel-consumer-core/src/test/resources/logback-test.xml +++ b/parallel-consumer-core/src/test/resources/logback-test.xml @@ -32,6 +32,8 @@ + + From 4eeb0086b9942825e0a388d41603a804757c9dfb Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Tue, 5 Apr 2022 21:20:55 +0100 Subject: [PATCH 15/54] omg - hashsets vs queues, wow --- .../AbstractParallelEoSStreamProcessor.java | 5 +- .../state/PartitionState.java | 6 ++ .../state/ProcessingShard.java | 15 +++- .../parallelconsumer/state/ShardManager.java | 1 + ...actParallelEoSStreamProcessorTestBase.java | 4 +- .../OffsetEncodingBackPressureTest.java | 76 ++++++++++++++----- .../OffsetEncodingBackPressureUnitTest.java | 2 + 7 files changed, 82 insertions(+), 27 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 0b40d1146..2d4e00cc3 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 @@ -889,7 +889,7 @@ private void transitionToClosing() { */ private void processWorkCompleteMailBox() { log.trace("Processing mailbox (might block waiting for results)..."); - Set results = new HashSet<>(); + Queue results = new ArrayDeque<>(); final Duration timeToBlockFor = getTimeToBlockFor(); @@ -927,7 +927,7 @@ private void processWorkCompleteMailBox() { for (var action : results) { WorkContainer work = action.getWorkContainer(); if (work == null) { - EpochAndRecords consumerRecords = action.getConsumerRecords(); + EpochAndRecords consumerRecords = action.getConsumerRecords(); wm.registerWork(consumerRecords); } else { MDC.put("offset", work.toString()); @@ -1138,6 +1138,7 @@ protected void addToMailbox(WorkContainer wc) { public void registerWork(EpochAndRecords polledRecords) { log.debug("Adding {} to mailbox...", polledRecords); + workMailBox.add(new ActionItem(null, polledRecords)); } 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 cd634689f..d19ab6ff1 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 @@ -186,9 +186,15 @@ private void updateHighestSucceededOffsetSoFar(WorkContainer work) { } public void addWorkContainer(WorkContainer wc) { + long offsetHighestSeen = getOffsetHighestSeen(); + if (wc.offset() != offsetHighestSeen + 1) { + log.error(""); + } + maybeRaiseHighestSeenOffset(wc.offset()); commitQueue.put(wc.offset(), wc); incompleteOffsets.add(wc.offset()); + } /** diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ProcessingShard.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ProcessingShard.java index 22ff310fd..ef63571b2 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ProcessingShard.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ProcessingShard.java @@ -35,6 +35,7 @@ public class ProcessingShard { * Uses a ConcurrentSkipListMap instead of a TreeMap as under high pressure there appears to be some concurrency * errors (missing WorkContainers). */ + @Getter private final NavigableMap> entries = new ConcurrentSkipListMap<>(); @Getter(PRIVATE) @@ -74,9 +75,19 @@ public Optional> getWorkForOffset(long offset) { } public long getCountOfWorkAwaitingSelection() { - return entries.values().parallelStream() + return entries.values().stream() // todo missing pm.isBlocked(topicPartition) ? - .filter(kvWorkContainer -> kvWorkContainer.isAvailableToTakeAsWork()) + .filter(WorkContainer::isAvailableToTakeAsWork) + .count(); + } + + public long getCountOfWorkTracked() { + return entries.size(); + } + + public long getCountWorkInFlight() { + return entries.values().stream() + .filter(WorkContainer::isInFlight) .count(); } diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ShardManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ShardManager.java index e8da53748..0c6900489 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ShardManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ShardManager.java @@ -58,6 +58,7 @@ public class ShardManager { * @see WorkManager#getWorkIfAvailable() */ // todo performance: disable/remove if using partition order + @Getter private final Map> processingShards = new ConcurrentHashMap<>(); private final NavigableSet> retryQueue = new TreeSet<>(Comparator.comparing(wc -> wc.getDelayUntilRetryDue())); diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java index 955ece43c..6d38659bb 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java @@ -79,7 +79,7 @@ public abstract class AbstractParallelEoSStreamProcessorTestBase { protected AbstractParallelEoSStreamProcessor parentParallelConsumer; - public static int defaultTimeoutSeconds = 30; + public static int defaultTimeoutSeconds = 3000; public static Duration defaultTimeout = ofSeconds(defaultTimeoutSeconds); protected static long defaultTimeoutMs = defaultTimeout.toMillis(); @@ -287,7 +287,7 @@ private void blockingLoopLatchTrigger(int waitForCount) { loopLatchV = new CountDownLatch(waitForCount); try { boolean timeout = !loopLatchV.await(defaultTimeoutSeconds, SECONDS); - if (timeout) + if (timeout || parentParallelConsumer.isClosedOrFailed()) throw new TimeoutException(msg("Timeout of {}, waiting for {} counts, on latch with {} left", defaultTimeout, waitForCount, loopLatchV.getCount())); } catch (InterruptedException e) { log.error("Interrupted while waiting for loop latch - timeout was {}", defaultTimeout); diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java index 71deb4315..7ba425b87 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java @@ -9,10 +9,9 @@ import io.confluent.parallelconsumer.FakeRuntimeError; import io.confluent.parallelconsumer.ParallelEoSStreamProcessorTestBase; import io.confluent.parallelconsumer.offsets.OffsetMapCodecManager.HighestOffsetAndIncompletes; -import io.confluent.parallelconsumer.state.PartitionMonitor; -import io.confluent.parallelconsumer.state.WorkContainer; -import io.confluent.parallelconsumer.state.WorkManager; +import io.confluent.parallelconsumer.state.*; import lombok.extern.slf4j.Slf4j; +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import org.junit.jupiter.api.AfterAll; @@ -23,12 +22,11 @@ import pl.tlinkowski.unij.api.UniLists; import java.time.Duration; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; +import java.util.*; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; import static io.confluent.csid.utils.JavaUtils.getLast; import static io.confluent.csid.utils.JavaUtils.getOnlyOne; @@ -83,9 +81,12 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O OffsetMapCodecManager.DefaultMaxMetadataSize = 40; // reduce available to make testing easier OffsetMapCodecManager.forcedCodec = Optional.of(OffsetEncoding.BitSetV2); // force one that takes a predictable large amount of space - ktu.send(consumerSpy, ktu.generateRecords(numberOfRecords)); + List> records = ktu.generateRecords(numberOfRecords); + ktu.send(consumerSpy, records); + + AtomicInteger userFuncFinishedCount = new AtomicInteger(); + AtomicInteger userFuncStartCount = new AtomicInteger(); - AtomicInteger userFuncFinishedCount = new AtomicInteger(0); CountDownLatch msgLock = new CountDownLatch(1); CountDownLatch msgLockTwo = new CountDownLatch(1); CountDownLatch msgLockThree = new CountDownLatch(1); @@ -94,16 +95,22 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O List blockedOffsets = UniLists.of(0L, 2L); final int numberOfBlockedMessages = blockedOffsets.size(); - parallelConsumer.poll((rec) -> { + WorkManager wm = parallelConsumer.getWm(); + final PartitionState partitionState = wm.getPm().getPartitionState(topicPartition); + + ConcurrentLinkedQueue seen = new ConcurrentLinkedQueue<>(); + parallelConsumer.poll((rec) -> { + seen.add(rec.offset()); + userFuncStartCount.incrementAndGet(); // block the partition to create bigger and bigger offset encoding blocks // don't let offset 0 finish if (rec.offset() == offsetToBlock) { int attemptNumber = attempts.incrementAndGet(); if (attemptNumber == 1) { log.debug("Force first message to 'never' complete, causing a large offset encoding (lots of messages completing above the low water mark. Waiting for msgLock countdown."); - int sleepFor = 120; - awaitLatch(msgLock, sleepFor); + int timeout = 120; + awaitLatch(msgLock, timeout); log.debug("Very slow message awoken, throwing exception"); throw new FakeRuntimeError("Fake error"); } else { @@ -111,15 +118,17 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O awaitLatch(msgLockTwo, 60); log.debug("Second attempt, unlocked, succeeding"); } - } else if (rec.offset() == 2l) { + } else if (rec.offset() == 2L) { awaitLatch(msgLockThree); log.debug("// msg 2L unblocked"); } else { sleepQuietly(1); } - userFuncFinishedCount.getAndIncrement(); + userFuncFinishedCount.incrementAndGet(); }); + ShardManager sm = wm.getSm(); + try { // wait for all pre-produced messages to be processed and produced @@ -129,7 +138,17 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O //, () -> parallelConsumer.getFailureCause()) // requires https://github.com/awaitility/awaitility/issues/178#issuecomment-734769761 .pollInterval(1, SECONDS) .untilAsserted(() -> { + ProcessingShard stringStringProcessingShard = sm.getProcessingShards().get(topicPartition); + if (stringStringProcessingShard != null) { + long countOfWorkAwaitingSelection = stringStringProcessingShard.getCountOfWorkAwaitingSelection(); + NavigableMap> entries = stringStringProcessingShard.getEntries(); + boolean b = sm.workIsWaitingToBeProcessed(); + long countWorkInFlight = stringStringProcessingShard.getCountWorkInFlight(); + long countOfWorkTracked = stringStringProcessingShard.getCountOfWorkTracked(); + long numberOfWorkQueuedInShardsAwaitingSelection = sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); + } assertThat(userFuncFinishedCount.get()).isEqualTo(numberOfRecords - numberOfBlockedMessages); +// Truth.assertThat(numberOfWorkQueuedInShardsAwaitingSelection).isEqualTo(-4); }); // # assert commit ok - nothing blocked @@ -159,7 +178,6 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O assertThat(highestSeenOffset).as("offset 99 is encoded as having been seen").isEqualTo(expectedHighestSeenOffset); } - WorkManager wm = parallelConsumer.getWm(); // partition not blocked { @@ -202,25 +220,30 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O .deserialiseIncompleteOffsetMapFromBase64(0L, meta); Truth.assertWithMessage("The only incomplete record now is offset zero, which we are blocked on") .that(incompletes.getIncompleteOffsets()).containsExactlyElementsIn(blockedOffsets); - Truth8.assertThat(incompletes.getHighestSeenOffset()).hasValue(numberOfRecords + extraRecordsToBlockWithThresholdBlocks - 1); + int expectedHighestSeen = numberOfRecords + extraRecordsToBlockWithThresholdBlocks - 1; + Truth8.assertThat(incompletes.getHighestSeenOffset()).hasValue(expectedHighestSeen); } ); } // recreates the situation where the payload size is too large and must be dropped log.debug("// test max payload exceeded, payload dropped"); - int processedBeforePartitionBlock = userFuncFinishedCount.get(); - int extraMessages = numberOfRecords + extraRecordsToBlockWithThresholdBlocks / 2; - log.debug("// messages already sent {}, sending {} more", processedBeforePartitionBlock, extraMessages); + +// log.debug("// messages already sent {}, sending {} more", processedBeforePartitionBlock, extraMessages); { + long numberOfWorkQueuedInShardsAwaitingSelection = sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); log.debug("// force system to allow more records (i.e. the actual system attempts to never allow the payload to grow this big)"); - PartitionMonitor.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(2); + PartitionMonitor.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(30); parallelConsumer.requestCommitAsap(); awaitForOneLoopCycle(); + // log.debug("// unlock to make state dirty to get a commit"); + msgLockThree.countDown(); + int processedBeforePartitionBlock = userFuncFinishedCount.get(); + int extraMessages = numberOfRecords + extraRecordsToBlockWithThresholdBlocks / 2; log.debug("// send {} more messages", extraMessages); ktu.send(consumerSpy, ktu.generateRecords(extraMessages)); @@ -229,7 +252,18 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O log.debug("// wait for the new message to be processed"); await().atMost(defaultTimeout).untilAsserted(() -> - assertThat(userFuncFinishedCount.get()).isEqualTo(processedBeforePartitionBlock + extraMessages + 1) + { + long numberOfWorkQueuedInShardsAwaitingSelection1 = sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); + ShardManager sm1 = sm; + List seen1 = seen.stream().sorted().collect(Collectors.toList()); + long offsetHighestSucceeded = partitionState.getOffsetHighestSucceeded(); + long offsetHighestSeen = partitionState.getOffsetHighestSeen(); + long offsetHighestSequentialSucceeded = partitionState.getOffsetHighestSequentialSucceeded(); + int i = userFuncStartCount.get(); + int i1 = userFuncFinishedCount.get(); + int expectedUserFunctionFinishedCount = processedBeforePartitionBlock + extraMessages + 1; + assertThat(userFuncFinishedCount.get()).isEqualTo(expectedUserFunctionFinishedCount); + } ); log.debug("// assert payload missing from commit now"); diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java index c03ee2408..798477f40 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java @@ -4,6 +4,7 @@ * Copyright (C) 2020-2022 Confluent, Inc. */ +import com.google.common.truth.Truth; import io.confluent.parallelconsumer.ParallelEoSStreamProcessorTestBase; import io.confluent.parallelconsumer.state.PartitionMonitor; import io.confluent.parallelconsumer.state.PartitionState; @@ -207,6 +208,7 @@ private void sendRecordsToWM(int numberOfRecords, WorkManager wm log.debug("~Sending {} more records", numberOfRecords); List> records = ktu.generateRecords(numberOfRecords); wm.registerWork(new ConsumerRecords<>(UniMaps.of(topicPartition, records))); + Truth.assertThat(wm.getTotalWorkAwaitingIngestion()).isEqualTo(numberOfRecords); } From bcfc9c1f8bf21a814bf1105e6451b03ccf563033 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Wed, 6 Apr 2022 10:49:53 +0100 Subject: [PATCH 16/54] review --- .../AbstractParallelEoSStreamProcessor.java | 70 +++++++++++-------- .../internal/BrokerPollSystem.java | 23 +++--- .../internal/CountingCRLinkedList.java | 1 + .../internal/EpochAndRecords.java | 5 ++ .../offsets/BitSetEncoder.java | 2 +- .../offsets/OffsetMapCodecManager.java | 3 +- .../state/PartitionMonitor.java | 50 +++++-------- .../state/PartitionState.java | 9 +-- .../parallelconsumer/state/ShardManager.java | 2 +- .../parallelconsumer/state/WorkManager.java | 39 +++++------ ...actParallelEoSStreamProcessorTestBase.java | 2 +- .../ParallelEoSStreamProcessorTest.java | 2 +- .../OffsetEncodingBackPressureTest.java | 40 ++++++----- .../OffsetEncodingBackPressureUnitTest.java | 6 +- .../state/WorkManagerTest.java | 34 ++++----- 15 files changed, 144 insertions(+), 144 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 2d4e00cc3..c6c8493c3 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 @@ -43,6 +43,7 @@ import static java.time.Duration.ofMillis; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.SECONDS; +import static lombok.AccessLevel.PRIVATE; import static lombok.AccessLevel.PROTECTED; /** @@ -52,6 +53,7 @@ public abstract class AbstractParallelEoSStreamProcessor implements ParallelConsumer, ConsumerRebalanceListener, Closeable { public static final String MDC_INSTANCE_ID = "pcId"; + public static final String MDC_OFFSET_MARKER = "offset"; @Getter(PROTECTED) protected final ParallelConsumerOptions options; @@ -98,18 +100,30 @@ public abstract class AbstractParallelEoSStreamProcessor implements Parall @Getter(PROTECTED) private final BlockingQueue workMailBox = new LinkedBlockingQueue<>(); // Thread safe, highly performant, non blocking + /** + * Either or + */ @Value - private class ActionItem { + @RequiredArgsConstructor(access = PRIVATE) + private static class ActionItem { WorkContainer workContainer; - EpochAndRecords consumerRecords; + EpochAndRecords consumerRecords; -// public static ActionItem ofRecords(ConsumerRecords polledRecords) { -// return new ActionItem(null, polledRecords); -// } -// -// public static ActionItem ofWork(WorkContainer work) { -// new ActionItem(work, null); -// } + private boolean isWorkResult() { + return workContainer != null; + } + + private boolean isNewConsumerRecords() { + return !isWorkResult(); + } + + private static ActionItem of(EpochAndRecords polledRecords) { + return new ActionItem<>(null, polledRecords); + } + + public static ActionItem of(WorkContainer work) { + return new ActionItem(work, null); + } } private final BrokerPollSystem brokerPollSubsystem; @@ -632,15 +646,14 @@ private void controlLoop(Function, List> userFu // int newWork = handleWork(userFunction, callback); -// if (state == running) { -// if (!wm.isSufficientlyLoaded() & brokerPollSubsystem.isPaused()) { - // can occur -// log.debug("Found Poller paused with not enough front loaded messages, ensuring poller is awake (mail: {} vs target: {})", -//// wm.getAmountOfWorkQueuedWaitingIngestion(), -// options.getTargetAmountOfRecordsInFlight()); -// brokerPollSubsystem.wakeupIfPaused(); -// } -// } + if (state == running) { + if (!wm.isSufficientlyLoaded() & brokerPollSubsystem.isPaused()) { + log.debug("Found Poller paused with not enough front loaded messages, ensuring poller is awake (mail: {} vs target: {})", + wm.getNumberOfWorkQueuedInShardsAwaitingSelection(), + options.getTargetAmountOfRecordsInFlight()); + brokerPollSubsystem.wakeupIfPaused(); + } + } log.trace("Loop: Process mailbox"); processWorkCompleteMailBox(); @@ -678,7 +691,7 @@ private void controlLoop(Function, List> userFu // end of loop log.trace("End of control loop, waiting processing {}, remaining in partition queues: {}, out for processing: {}. In state: {}", - wm.getTotalWorkAwaitingIngestion(), wm.getNumberOfEntriesInPartitionQueues(), wm.getNumberRecordsOutForProcessing(), state); + wm.getNumberOfWorkQueuedInShardsAwaitingSelection(), wm.getNumberOfEntriesInPartitionQueues(), wm.getNumberRecordsOutForProcessing(), state); } private int handleWork(final Function, List> userFunction, final Consumer callback) { @@ -925,14 +938,13 @@ private void processWorkCompleteMailBox() { log.trace("Processing drained work {}...", results.size()); for (var action : results) { - WorkContainer work = action.getWorkContainer(); - if (work == null) { - EpochAndRecords consumerRecords = action.getConsumerRecords(); - wm.registerWork(consumerRecords); + if (action.isNewConsumerRecords()) { + wm.registerWork(action.getConsumerRecords()); } else { - MDC.put("offset", work.toString()); + WorkContainer work = action.getWorkContainer(); + MDC.put(MDC_OFFSET_MARKER, work.toString()); wm.handleFutureResult(work); - MDC.clear(); + MDC.remove(MDC_OFFSET_MARKER); } } } @@ -941,7 +953,6 @@ private void processWorkCompleteMailBox() { * The amount of time to block poll in this cycle * * @return either the duration until next commit, or next work retry // * @see WorkManager#isStarvedForNewWork() - * @see WorkManager#getTotalWorkAwaitingIngestion() * @see ParallelConsumerOptions#getTargetAmountOfRecordsInFlight() */ private Duration getTimeToBlockFor() { @@ -1133,13 +1144,12 @@ protected void onUserFunctionSuccess(WorkContainer wc, List resultsFrom protected void addToMailbox(WorkContainer wc) { String state = wc.isUserFunctionSucceeded() ? "succeeded" : "FAILED"; log.trace("Adding {} {} to mailbox...", state, wc); - workMailBox.add(new ActionItem(wc, null)); + workMailBox.add(ActionItem.of(wc)); } - public void registerWork(EpochAndRecords polledRecords) { + public void registerWork(EpochAndRecords polledRecords) { log.debug("Adding {} to mailbox...", polledRecords); - - workMailBox.add(new ActionItem(null, polledRecords)); + workMailBox.add(ActionItem.of(polledRecords)); } /** diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java index 2a6e73257..f160e517c 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java @@ -28,6 +28,7 @@ import static io.confluent.parallelconsumer.internal.AbstractParallelEoSStreamProcessor.MDC_INSTANCE_ID; import static io.confluent.parallelconsumer.internal.State.*; import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static lombok.AccessLevel.PRIVATE; /** * Subsystem for polling the broker for messages. @@ -61,9 +62,14 @@ public class BrokerPollSystem implements OffsetCommitter, ConsumerRebalanc private final WorkManager wm; - private long epoch = 0L; + /** + * Gets incremented every time there's a new assignment event. This epoch is forever associated with a record, and + * is used to more easily determine stale records. + */ + @Getter(PRIVATE) + private long partitionAssignmentEpoch = 0L; - public BrokerPollSystem(ConsumerManager consumerMgr, WorkManager wm, AbstractParallelEoSStreamProcessor pc, final ParallelConsumerOptions options) { + public BrokerPollSystem(ConsumerManager consumerMgr, WorkManager wm, AbstractParallelEoSStreamProcessor pc, final ParallelConsumerOptions options) { this.wm = wm; this.pc = pc; @@ -153,6 +159,7 @@ private void handlePoll() { } } + // todo ? private void transitionToCloseMaybe() { // make sure everything is committed if (isResponsibleForCommits() && !wm.isRecordsAwaitingToBeCommitted()) { @@ -193,19 +200,15 @@ private boolean isResponsibleForCommits() { @Override public void onPartitionsRevoked(Collection partitions) { - epoch++; + partitionAssignmentEpoch++; } @Override public void onPartitionsAssigned(Collection partitions) { - epoch++; - } - - private long getEpoch() { - return epoch; + partitionAssignmentEpoch++; } - private EpochAndRecords pollBrokerForRecords() { + private EpochAndRecords pollBrokerForRecords() { managePauseOfSubscription(); log.debug("Subscriptions are paused: {}", paused); @@ -217,7 +220,7 @@ private EpochAndRecords pollBrokerForRecords() { ConsumerRecords poll = consumerManager.poll(thisLongPollTimeout); log.debug("Poll completed"); - return new EpochAndRecords(poll, getEpoch()); + return new EpochAndRecords<>(poll, getPartitionAssignmentEpoch()); } /** diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/CountingCRLinkedList.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/CountingCRLinkedList.java index 28cbcb201..20a54610c 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/CountingCRLinkedList.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/CountingCRLinkedList.java @@ -18,6 +18,7 @@ * Also protects against concurrent modification exceptions, as we don't need to traverse the list to count the nested * elements. The count won't then be always exact, but it doesn't need to be. */ +// todo used? @EqualsAndHashCode(callSuper = true) public class CountingCRLinkedList extends LinkedList> implements Queue> { diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecords.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecords.java index 3e29ae22d..5893425fe 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecords.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecords.java @@ -7,6 +7,11 @@ import lombok.Value; import org.apache.kafka.clients.consumer.ConsumerRecords; +/** + * For tagging polled records with our epoch + * + * @see BrokerPollSystem#partitionAssignmentEpoch + */ @Value public class EpochAndRecords { ConsumerRecords consumerRecs; 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 93cb0e085..04dcb05dc 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 @@ -108,7 +108,7 @@ private ByteBuffer initV2(int bitsetEntriesRequired) throws BitSetEncodingNotSup private ByteBuffer initV1(int bitsetEntriesRequired) throws BitSetEncodingNotSupportedException { if (bitsetEntriesRequired > Short.MAX_VALUE) { // need to upgrade to using Integer for the bitset length, but can't change serialisation format in-place - throw new BitSetEncodingNotSupportedException("input too long to encode for BitSet V1, length overflows Short.MAX_VALUE: " + bitsetEntriesRequired + ". (max: " + Short.MAX_VALUE + ")"); + throw new BitSetEncodingNotSupportedException("Input too long to encode for BitSet V1, length overflows Short.MAX_VALUE: " + bitsetEntriesRequired + ". (max: " + Short.MAX_VALUE + ")"); } // prep bit set buffer 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 a234f7ebb..2149b0231 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 @@ -15,7 +15,6 @@ import java.nio.ByteBuffer; import java.nio.charset.Charset; import java.util.*; -import java.util.stream.Collectors; import static io.confluent.csid.utils.StringUtils.msg; import static java.nio.charset.StandardCharsets.UTF_8; @@ -194,7 +193,7 @@ byte[] encodeOffsetsCompressed(long finalOffsetForPartition, PartitionState x < highestSucceeded).collect(Collectors.toList())); + partitionState.getIncompleteOffsetsBelowHighestSucceeded()); } OffsetSimultaneousEncoder simultaneousEncoder = new OffsetSimultaneousEncoder(finalOffsetForPartition, highestSucceeded, incompleteOffsets).invoke(); diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java index b6a189cd4..37024e801 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java @@ -15,7 +15,10 @@ import lombok.RequiredArgsConstructor; import lombok.Setter; import lombok.extern.slf4j.Slf4j; -import org.apache.kafka.clients.consumer.*; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import java.time.Clock; @@ -331,45 +334,28 @@ public void onFailure(WorkContainer wc) { * work with the {@link TopicPartition}'s {@link PartitionState} and the {@link ShardManager}. Keeping the two * different views in sync. Of course now, having a shared nothing architecture would mean all access to the state * is by a single thread, and so this could never occur (see ). - * - * @return true if the record was taken, false if it was skipped (previously successful) */ - boolean maybeRegisterNewRecordAsWork(final ConsumerRecord rec, final long epoch) { - if (rec == null) return false; - + void maybeRegisterNewRecordAsWork(final EpochAndRecords records) { +// if (records == null) return false; synchronized (partitionStates) { - if (isPartitionRemovedOrNeverAssigned(rec)) { - log.debug("Record in buffer for a partition no longer assigned. Dropping. TP: {} rec: {}", toTopicPartition(rec), rec); - return false; - } - - if (isRecordPreviouslyCompleted(rec)) { - log.trace("Record previously completed, skipping. offset: {}", rec.offset()); - return false; - } else { - int currentPartitionEpoch = getEpoch(rec); - var wc = new WorkContainer<>(currentPartitionEpoch, rec, options.getRetryDelayProvider(), clock); - - sm.addWorkContainer(wc); + for (var rec : records.getConsumerRecs()) { + if (isPartitionRemovedOrNeverAssigned(rec)) { + log.debug("Record in buffer for a partition no longer assigned. Dropping. TP: {} rec: {}", toTopicPartition(rec), rec); + } - addWorkContainer(wc); + if (isRecordPreviouslyCompleted(rec)) { + log.trace("Record previously completed, skipping. offset: {}", rec.offset()); + } else { + int currentPartitionEpoch = getEpoch(rec); + var work = new WorkContainer<>(currentPartitionEpoch, rec, options.getRetryDelayProvider(), clock); - return true; + sm.addWorkContainer(work); + addWorkContainer(work); + } } } } - - /** - * @see #maybeRegisterNewRecordAsWork(ConsumerRecord) - */ - public void maybeRegisterNewRecordAsWork(EpochAndRecords records) { - ConsumerRecords recordCollection = records.getConsumerRecs(); - for (ConsumerRecord consumerRec : recordCollection) { - maybeRegisterNewRecordAsWork(consumerRec, records.getMyEpoch()); - } - } - public Map collectDirtyCommitData() { var dirties = new HashMap(); for (var state : getAssignedPartitions().values()) { 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 d19ab6ff1..2269e4b3d 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 @@ -186,10 +186,10 @@ private void updateHighestSucceededOffsetSoFar(WorkContainer work) { } public void addWorkContainer(WorkContainer wc) { - long offsetHighestSeen = getOffsetHighestSeen(); - if (wc.offset() != offsetHighestSeen + 1) { - log.error(""); - } +// long offsetHighestSeen = getOffsetHighestSeen(); +// if (wc.offset() != offsetHighestSeen + 1) { +// log.error(""); +// } maybeRaiseHighestSeenOffset(wc.offset()); commitQueue.put(wc.offset(), wc); @@ -239,6 +239,7 @@ public Set getAllIncompleteOffsets() { */ 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) diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ShardManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ShardManager.java index 0c6900489..f8fc4d038 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ShardManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ShardManager.java @@ -58,7 +58,7 @@ public class ShardManager { * @see WorkManager#getWorkIfAvailable() */ // todo performance: disable/remove if using partition order - @Getter +// @Getter private final Map> processingShards = new ConcurrentHashMap<>(); private final NavigableSet> retryQueue = new TreeSet<>(Comparator.comparing(wc -> wc.getDelayUntilRetryDue())); diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java index 0b4e51007..d2ad4700a 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java @@ -13,7 +13,6 @@ import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; -import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import pl.tlinkowski.unij.api.UniLists; @@ -130,14 +129,14 @@ void onPartitionsRemoved(final Collection partitions) { // wmbm.onPartitionsRemoved(partitions); } - /** - * Hard codes epoch as genesis - for testing only - */ - public void registerWork(ConsumerRecords records) { - registerWork(new EpochAndRecords(records, 0)); - } +// /** +// * Hard codes epoch as genesis - for testing only +// */ +// public void registerWork(ConsumerRecords records) { +// registerWork(new EpochAndRecords(records, 0)); +// } - public void registerWork(EpochAndRecords records) { + public void registerWork(EpochAndRecords records) { // wmbm.registerWork(records); pm.maybeRegisterNewRecordAsWork(records); } @@ -323,7 +322,7 @@ public boolean shouldThrottle() { * should be downloaded (or pipelined in the Consumer) */ public boolean isSufficientlyLoaded() { - return getTotalWorkAwaitingIngestion() > options.getTargetAmountOfRecordsInFlight() * getLoadingFactor(); + return getNumberOfWorkQueuedInShardsAwaitingSelection() > (long) options.getTargetAmountOfRecordsInFlight() * getLoadingFactor(); } private int getLoadingFactor() { @@ -341,17 +340,17 @@ public boolean hasWorkInFlight() { public boolean isWorkInFlightMeetingTarget() { return getNumberRecordsOutForProcessing() >= options.getTargetAmountOfRecordsInFlight(); } - - /** - * @return Work count in mailbox plus work added to the processing shards - */ - public long getTotalWorkAwaitingIngestion() { -// return sm.getNumberOfEntriesInPartitionQueues - return sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); -// long workQueuedInShardsCount = sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); -// Integer workQueuedInMailboxCount = getAmountOfWorkQueuedWaitingIngestion(); -// return workQueuedInShardsCount + workQueuedInMailboxCount; - } +// +// /** +// * @return Work count in mailbox plus work added to the processing shards +// */ +// public long getTotalWorkAwaitingIngestion() { +//// return sm.getNumberOfEntriesInPartitionQueues +// return sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); +//// long workQueuedInShardsCount = sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); +//// Integer workQueuedInMailboxCount = getAmountOfWorkQueuedWaitingIngestion(); +//// return workQueuedInShardsCount + workQueuedInMailboxCount; +// } public long getNumberOfWorkQueuedInShardsAwaitingSelection() { return sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java index 6d38659bb..5bd8de06b 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java @@ -79,7 +79,7 @@ public abstract class AbstractParallelEoSStreamProcessorTestBase { protected AbstractParallelEoSStreamProcessor parentParallelConsumer; - public static int defaultTimeoutSeconds = 3000; + public static int defaultTimeoutSeconds = 30; public static Duration defaultTimeout = ofSeconds(defaultTimeoutSeconds); protected static long defaultTimeoutMs = defaultTimeout.toMillis(); diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorTest.java index 1214a9c78..3a8bc97ed 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorTest.java @@ -119,7 +119,7 @@ void offsetsAreNeverCommittedForMessagesStillInFlightSimplest(CommitMode commitM awaitLatch(startBarrierLatch); // zero records waiting, 2 out for processing - assertThat(parallelConsumer.getWm().getTotalWorkAwaitingIngestion()).isZero(); + assertThat(parallelConsumer.getWm().getNumberOfWorkQueuedInShardsAwaitingSelection()).isZero(); assertThat(parallelConsumer.getWm().getNumberRecordsOutForProcessing()).isEqualTo(2); // finish processing 1 diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java index 7ba425b87..680cce779 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java @@ -22,11 +22,13 @@ import pl.tlinkowski.unij.api.UniLists; import java.time.Duration; -import java.util.*; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; import static io.confluent.csid.utils.JavaUtils.getLast; import static io.confluent.csid.utils.JavaUtils.getOnlyOne; @@ -138,15 +140,15 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O //, () -> parallelConsumer.getFailureCause()) // requires https://github.com/awaitility/awaitility/issues/178#issuecomment-734769761 .pollInterval(1, SECONDS) .untilAsserted(() -> { - ProcessingShard stringStringProcessingShard = sm.getProcessingShards().get(topicPartition); - if (stringStringProcessingShard != null) { - long countOfWorkAwaitingSelection = stringStringProcessingShard.getCountOfWorkAwaitingSelection(); - NavigableMap> entries = stringStringProcessingShard.getEntries(); - boolean b = sm.workIsWaitingToBeProcessed(); - long countWorkInFlight = stringStringProcessingShard.getCountWorkInFlight(); - long countOfWorkTracked = stringStringProcessingShard.getCountOfWorkTracked(); - long numberOfWorkQueuedInShardsAwaitingSelection = sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); - } +// ProcessingShard stringStringProcessingShard = sm.getProcessingShards().get(topicPartition); +// if (stringStringProcessingShard != null) { +// long countOfWorkAwaitingSelection = stringStringProcessingShard.getCountOfWorkAwaitingSelection(); +// NavigableMap> entries = stringStringProcessingShard.getEntries(); +// boolean b = sm.workIsWaitingToBeProcessed(); +// long countWorkInFlight = stringStringProcessingShard.getCountWorkInFlight(); +// long countOfWorkTracked = stringStringProcessingShard.getCountOfWorkTracked(); +// long numberOfWorkQueuedInShardsAwaitingSelection = sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); +// } assertThat(userFuncFinishedCount.get()).isEqualTo(numberOfRecords - numberOfBlockedMessages); // Truth.assertThat(numberOfWorkQueuedInShardsAwaitingSelection).isEqualTo(-4); }); @@ -253,14 +255,14 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O log.debug("// wait for the new message to be processed"); await().atMost(defaultTimeout).untilAsserted(() -> { - long numberOfWorkQueuedInShardsAwaitingSelection1 = sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); - ShardManager sm1 = sm; - List seen1 = seen.stream().sorted().collect(Collectors.toList()); - long offsetHighestSucceeded = partitionState.getOffsetHighestSucceeded(); - long offsetHighestSeen = partitionState.getOffsetHighestSeen(); - long offsetHighestSequentialSucceeded = partitionState.getOffsetHighestSequentialSucceeded(); - int i = userFuncStartCount.get(); - int i1 = userFuncFinishedCount.get(); +// long numberOfWorkQueuedInShardsAwaitingSelection1 = sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); +// ShardManager sm1 = sm; +// List seen1 = seen.stream().sorted().collect(Collectors.toList()); +// long offsetHighestSucceeded = partitionState.getOffsetHighestSucceeded(); +// long offsetHighestSeen = partitionState.getOffsetHighestSeen(); +// long offsetHighestSequentialSucceeded = partitionState.getOffsetHighestSequentialSucceeded(); +// int i = userFuncStartCount.get(); +// int i1 = userFuncFinishedCount.get(); int expectedUserFunctionFinishedCount = processedBeforePartitionBlock + extraMessages + 1; assertThat(userFuncFinishedCount.get()).isEqualTo(expectedUserFunctionFinishedCount); } diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java index 798477f40..d96bd123d 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java @@ -6,6 +6,7 @@ import com.google.common.truth.Truth; import io.confluent.parallelconsumer.ParallelEoSStreamProcessorTestBase; +import io.confluent.parallelconsumer.internal.EpochAndRecords; import io.confluent.parallelconsumer.state.PartitionMonitor; import io.confluent.parallelconsumer.state.PartitionState; import io.confluent.parallelconsumer.state.WorkContainer; @@ -207,8 +208,9 @@ private WorkContainer findWC(List> private void sendRecordsToWM(int numberOfRecords, WorkManager wm) { log.debug("~Sending {} more records", numberOfRecords); List> records = ktu.generateRecords(numberOfRecords); - wm.registerWork(new ConsumerRecords<>(UniMaps.of(topicPartition, records))); - Truth.assertThat(wm.getTotalWorkAwaitingIngestion()).isEqualTo(numberOfRecords); + int hardCodedEpochForTesting = 0; + wm.registerWork(new EpochAndRecords<>(new ConsumerRecords<>(UniMaps.of(topicPartition, records)), hardCodedEpochForTesting)); + Truth.assertThat(wm.getNumberOfWorkQueuedInShardsAwaitingSelection()).isEqualTo(numberOfRecords); } diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java index a6aad1113..9e43c5251 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java @@ -369,7 +369,7 @@ void insertWrongOrderPreservesOffsetOrdering() { var recs = new ConsumerRecords<>(m); // - wm.registerWork(recs); + registerWork(recs); int max = 10; @@ -392,6 +392,11 @@ void insertWrongOrderPreservesOffsetOrdering() { assertOffsets(works, of(1, 6)); } + private void registerWork(ConsumerRecords recs) { + wm.registerWork(new EpochAndRecords<>(recs, 0)); + } + + private void fail(WorkContainer wc) { wc.onUserFunctionFailure(null); wm.onFailureResult(wc); @@ -443,22 +448,6 @@ public int size() { } } - @Test - @Disabled - public void multipleFailures() { - } - - - @Test - @Disabled - public void delayedOrdered() { - } - - @Test - @Disabled - public void delayedUnordered() { - } - @Test void orderedByPartitionsParallel() { ParallelConsumerOptions build = ParallelConsumerOptions.builder() @@ -478,7 +467,7 @@ void orderedByPartitionsParallel() { var recs = new ConsumerRecords<>(m); // - wm.registerWork(recs); + registerWork(recs); // var works = wm.getWorkIfAvailable(); @@ -524,7 +513,7 @@ void orderedByKeyParallel() { var recs = new ConsumerRecords<>(m); // - wm.registerWork(recs); + registerWork(recs); // var works = wm.getWorkIfAvailable(); @@ -572,7 +561,7 @@ void highVolumeKeyOrder(int quantity) { assignPartition(partition); // - wm.registerWork(recs); + registerWork(recs); // long awaiting = wm.getSm().getNumberOfWorkQueuedInShardsAwaitingSelection(); @@ -659,7 +648,7 @@ void resumesFromNextShard(ParallelConsumerOptions.ProcessingOrder order) { var rec2 = new ConsumerRecord<>(INPUT_TOPIC, 2, 21, "21", "value"); m.put(topicPartitionOf(2), of(rec2)); var recs = new ConsumerRecords<>(m); - wm.registerWork(recs); + registerWork(recs); // // force ingestion of records - see refactor: Queue unification #219 // wm.tryToEnsureQuantityOfWorkQueuedAvailable(100); @@ -691,6 +680,9 @@ void resumesFromNextShard(ParallelConsumerOptions.ProcessingOrder order) { /** * Checks that when using shards are not starved when there's enough work queued to satisfy poll request from the * initial request (without needing to iterate to other shards) + * + * @see #236 Under some conditions, a + * shard (by partition or key), can get starved for attention */ @Test void starvation() { From 6054ac5ec9578d104803f79717a6167be231d186 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Wed, 6 Apr 2022 10:53:06 +0100 Subject: [PATCH 17/54] review --- .../internal/CountingCRLinkedList.java | 98 +++++++++---------- 1 file changed, 49 insertions(+), 49 deletions(-) diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/CountingCRLinkedList.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/CountingCRLinkedList.java index 20a54610c..6fd12fa1c 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/CountingCRLinkedList.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/CountingCRLinkedList.java @@ -1,54 +1,54 @@ package io.confluent.parallelconsumer.internal; /*- - * Copyright (C) 2020-2021 Confluent, Inc. + * Copyright (C) 2020-2022 Confluent, Inc. */ -import lombok.EqualsAndHashCode; -import lombok.Getter; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; - -import java.util.LinkedList; -import java.util.Queue; - -/** - * Cached view of the nested number of records in this queue. - *

- * Also protects against concurrent modification exceptions, as we don't need to traverse the list to count the nested - * elements. The count won't then be always exact, but it doesn't need to be. - */ -// todo used? -@EqualsAndHashCode(callSuper = true) -public class CountingCRLinkedList extends LinkedList> implements Queue> { - - /** - * The number of nested {@link ConsumerRecord} in this collection. As this is a non blocking collection, this won't - * be exact. - */ - @Getter - private int nestedCount = 0; - - @Override - public void add(final int index, final ConsumerRecords element) { - nestedCount = nestedCount + element.count(); - super.add(index, element); - } - - @Override - public boolean add(final ConsumerRecords element) { - nestedCount = nestedCount + element.count(); - return super.add(element); - } - - @Override - public ConsumerRecords poll() { - ConsumerRecords poll = super.poll(); - if (poll != null) { - int numberOfNestedMessages = poll.count(); - nestedCount = nestedCount - numberOfNestedMessages; - } - return poll; - } - -} +//import lombok.EqualsAndHashCode; +//import lombok.Getter; +//import org.apache.kafka.clients.consumer.ConsumerRecord; +//import org.apache.kafka.clients.consumer.ConsumerRecords; +// +//import java.util.LinkedList; +//import java.util.Queue; +// +///** +// * Cached view of the nested number of records in this queue. +// *

+// * Also protects against concurrent modification exceptions, as we don't need to traverse the list to count the nested +// * elements. The count won't then be always exact, but it doesn't need to be. +// */ +//// todo used? +//@EqualsAndHashCode(callSuper = true) +//public class CountingCRLinkedList extends LinkedList> implements Queue> { +// +// /** +// * The number of nested {@link ConsumerRecord} in this collection. As this is a non blocking collection, this won't +// * be exact. +// */ +// @Getter +// private int nestedCount = 0; +// +// @Override +// public void add(final int index, final ConsumerRecords element) { +// nestedCount = nestedCount + element.count(); +// super.add(index, element); +// } +// +// @Override +// public boolean add(final ConsumerRecords element) { +// nestedCount = nestedCount + element.count(); +// return super.add(element); +// } +// +// @Override +// public ConsumerRecords poll() { +// ConsumerRecords poll = super.poll(); +// if (poll != null) { +// int numberOfNestedMessages = poll.count(); +// nestedCount = nestedCount - numberOfNestedMessages; +// } +// return poll; +// } +// +//} From c96862997acce18e40af23d5fb8c93ffdd9d29aa Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Wed, 6 Apr 2022 15:41:25 +0100 Subject: [PATCH 18/54] review --- .../OffsetEncodingBackPressureTest.java | 11 ++++++++--- .../offsets/OffsetEncodingTests.java | 18 ++++++++++-------- .../truth/TruthGeneratorTests.java | 2 +- .../src/test/resources/logback-test.xml | 12 ++++++------ 4 files changed, 25 insertions(+), 18 deletions(-) diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java index 680cce779..409e476be 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java @@ -34,6 +34,8 @@ import static io.confluent.csid.utils.JavaUtils.getOnlyOne; import static io.confluent.csid.utils.LatchTestUtils.awaitLatch; import static io.confluent.csid.utils.ThreadUtils.sleepQuietly; +import static io.confluent.parallelconsumer.ManagedTruth.assertTruth; +import static io.confluent.parallelconsumer.ManagedTruth.assertWithMessage; import static java.time.Duration.ofMillis; import static java.time.Duration.ofSeconds; import static java.util.concurrent.TimeUnit.SECONDS; @@ -198,9 +200,12 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O log.debug("// assert partition now blocked from threshold"); waitAtMost(ofSeconds(30)) .untilAsserted( - () -> assertThat(wm.getPm().isBlocked(topicPartition)) - .as("Partition SHOULD be blocked due to back pressure") - .isTrue()); // blocked + () -> { + assertWithMessage("Partition SHOULD be blocked due to back pressure") + .that(partitionState) + .isAllowedMoreRecords(); + assertTruth(partitionState).isNotAllowedMoreRecords(); + }); // blocked Long partitionOffsetHighWaterMarks = wm.getPm().getHighestSeenOffset(topicPartition); assertThat(partitionOffsetHighWaterMarks) 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 be88df707..39b7e303b 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 @@ -28,6 +28,7 @@ import java.util.*; import java.util.stream.Collectors; +import static io.confluent.parallelconsumer.ManagedTruth.assertTruth; import static io.confluent.parallelconsumer.offsets.OffsetEncoding.*; import static org.assertj.core.api.Assertions.assertThat; import static org.hamcrest.Matchers.in; @@ -160,9 +161,9 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset incompleteRecords.remove(incompleteRecords.stream().filter(x -> x.offset() == 25_000).findFirst().get()); incompleteRecords.remove(incompleteRecords.stream().filter(x -> x.offset() == highest).findFirst().get()); - var expected = incompleteRecords.stream().map(ConsumerRecord::offset) + List expected = incompleteRecords.stream().map(ConsumerRecord::offset) .sorted() - .collect(Collectors.toSet()); + .collect(Collectors.toList()); // ktu.send(consumerSpy, records); @@ -246,10 +247,10 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset var anIncompleteRecord = records.get(3); Truth.assertThat(pm.isRecordPreviouslyCompleted(anIncompleteRecord)).isFalse(); - // force ingestion early, and check results + // check state { // todo ingestion no longer a thing - what to do here? - Truth.assertThat(true).isFalse(); +// Truth.assertThat(true).isFalse(); // int ingested = newWm.tryToEnsureQuantityOfWorkQueuedAvailable(Integer.MAX_VALUE); if (!encodingsThatFail.contains(encoding)) { @@ -274,7 +275,7 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset var workRetrieved = newWm.getWorkIfAvailable(); var workRetrievedOffsets = workRetrieved.stream().map(WorkContainer::offset).collect(Collectors.toList()); - Truth.assertThat(workRetrieved).isNotEmpty(); + assertTruth(workRetrieved).isNotEmpty(); switch (encoding) { case BitSet, BitSetCompressed, // BitSetV1 both get a short overflow due to the length being too long @@ -285,9 +286,10 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset assertThat(workRetrievedOffsets).doesNotContainSequence(expected); } default -> { - assertThat(workRetrievedOffsets) - .as("Contains only incomplete records") - .containsExactlyElementsOf(expected); + Truth.assertWithMessage("Contains only incomplete records") + .that(workRetrievedOffsets) + .containsExactlyElementsIn(expected) + .inOrder(); } } } diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/truth/TruthGeneratorTests.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/truth/TruthGeneratorTests.java index 15824200b..233d65199 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/truth/TruthGeneratorTests.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/truth/TruthGeneratorTests.java @@ -25,7 +25,7 @@ void generate() { // todo check legacy's also contribute to subject graph assertTruth(new ConsumerRecords<>(UniMaps.of())).getPartitions().isEmpty(); - assertTruth(PodamUtils.createInstance(OffsetAndMetadata.class)).hasOffsetEqualTo(1); + assertTruth(new OffsetAndMetadata(0)).hasOffsetEqualTo(0); assertTruth(PodamUtils.createInstance(TopicPartition.class)).hasTopic().isNotEmpty(); diff --git a/parallel-consumer-core/src/test/resources/logback-test.xml b/parallel-consumer-core/src/test/resources/logback-test.xml index a4b7ad5c1..a9345a2ed 100644 --- a/parallel-consumer-core/src/test/resources/logback-test.xml +++ b/parallel-consumer-core/src/test/resources/logback-test.xml @@ -28,13 +28,13 @@ - + - + - + - + @@ -52,8 +52,8 @@ - - + + From 0f993ddacd29f13b1e80bdd51c1ba728910f48bd Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Wed, 6 Apr 2022 15:45:19 +0100 Subject: [PATCH 19/54] review --- .../offsets/OffsetEncodingBackPressureUnitTest.java | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java index d96bd123d..054b1471d 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java @@ -23,6 +23,7 @@ import org.junit.jupiter.api.parallel.ResourceLock; import pl.tlinkowski.unij.api.UniLists; import pl.tlinkowski.unij.api.UniMaps; +import pl.tlinkowski.unij.api.UniSets; import java.time.Duration; import java.util.List; @@ -30,6 +31,7 @@ import java.util.stream.Collectors; import static io.confluent.parallelconsumer.ManagedTruth.assertTruth; +import static io.confluent.parallelconsumer.ManagedTruth.assertWithMessage; import static java.time.Duration.ofMillis; /** @@ -104,9 +106,13 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O // "The only incomplete record now is offset zero, which we are blocked on" assertTruth(partitionState).getOffsetHighestSeen().isEqualTo(numberOfRecords + extraRecordsToBlockWithThresholdBlocks - 1); assertTruth(partitionState).getCommitDataIfDirty().getMetadata().isNotEmpty(); - // todo assert with message "The only incomplete record now is offset zero, which we are blocked on" - assertTruth(partitionState).getAllIncompleteOffsets().containsNoneIn(samplingOfShouldBeCompleteOffsets); - assertTruth(partitionState).getAllIncompleteOffsets().containsExactlyElementsIn(blockedOffsets); + assertTruth(partitionState) + .getAllIncompleteOffsets() + .containsNoneIn(samplingOfShouldBeCompleteOffsets); + assertWithMessage("The only incomplete record now is offset zero, which we are blocked on") + .that(partitionState).getAllIncompleteOffsets().containsExactlyElementsIn(blockedOffsets); + assertWithMessage("The only incomplete record now is offset zero, which we are blocked on") + .that(partitionState).getAllIncompleteOffsets().containsExactlyElementsIn(UniSets.of(0L)); } From 189dc59ff95875d29f0be7d2b64567ec8d08ab3e Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Thu, 21 Apr 2022 11:40:49 +0100 Subject: [PATCH 20/54] step --- .../generate_test_sources__e.xml | 28 +++++++++++++++++++ .../io/confluent/csid/utils/MathUtils.java | 22 --------------- .../OffsetEncodingBackPressureUnitTest.java | 3 -- 3 files changed, 28 insertions(+), 25 deletions(-) create mode 100644 .idea/runConfigurations/generate_test_sources__e.xml delete mode 100644 parallel-consumer-core/src/main/java/io/confluent/csid/utils/MathUtils.java diff --git a/.idea/runConfigurations/generate_test_sources__e.xml b/.idea/runConfigurations/generate_test_sources__e.xml new file mode 100644 index 000000000..2918f8a3f --- /dev/null +++ b/.idea/runConfigurations/generate_test_sources__e.xml @@ -0,0 +1,28 @@ + + + + + + + + diff --git a/parallel-consumer-core/src/main/java/io/confluent/csid/utils/MathUtils.java b/parallel-consumer-core/src/main/java/io/confluent/csid/utils/MathUtils.java deleted file mode 100644 index 3bcfd0012..000000000 --- a/parallel-consumer-core/src/main/java/io/confluent/csid/utils/MathUtils.java +++ /dev/null @@ -1,22 +0,0 @@ -package io.confluent.csid.utils; - -/*- - * Copyright (C) 2020-2022 Confluent, Inc. - */ - -public class MathUtils { - - /** - * @deprecated old, was used with AssertJ - not needed with Truth (has fuzzy matching built in) - */ - @Deprecated - public static boolean isLessWithin(final int needle, final int target, final int percent) { - int i = target * (1 - percent / 100); - - int diff = target - needle; - int norm = Math.abs(diff); - int off = norm / target * 100; - return off < percent; - } - -} diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java index 054b1471d..ce62b020b 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java @@ -23,7 +23,6 @@ import org.junit.jupiter.api.parallel.ResourceLock; import pl.tlinkowski.unij.api.UniLists; import pl.tlinkowski.unij.api.UniMaps; -import pl.tlinkowski.unij.api.UniSets; import java.time.Duration; import java.util.List; @@ -111,8 +110,6 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O .containsNoneIn(samplingOfShouldBeCompleteOffsets); assertWithMessage("The only incomplete record now is offset zero, which we are blocked on") .that(partitionState).getAllIncompleteOffsets().containsExactlyElementsIn(blockedOffsets); - assertWithMessage("The only incomplete record now is offset zero, which we are blocked on") - .that(partitionState).getAllIncompleteOffsets().containsExactlyElementsIn(UniSets.of(0L)); } From 908d8ed2fe6f92ade8cfaac00281dfd4420fc4f6 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Thu, 21 Apr 2022 13:09:34 +0100 Subject: [PATCH 21/54] step --- .../parallelconsumer/offsets/OffsetEncodingBackPressureTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java index 409e476be..39cb89651 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java @@ -189,6 +189,7 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O assertThat(partitionBlocked).isFalse(); } + // log.debug("// feed more messages in order to threshold block - as Bitset requires linearly as much space as we are feeding messages into it, it's guaranteed to block"); int extraRecordsToBlockWithThresholdBlocks = numberOfRecords / 2; { From 7547ec6d803816efc1a29faf7484de393e576e3c Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Thu, 21 Apr 2022 13:56:36 +0100 Subject: [PATCH 22/54] fix test --- .../offsets/OffsetEncodingBackPressureTest.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java index 39cb89651..099b90505 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java @@ -34,7 +34,6 @@ import static io.confluent.csid.utils.JavaUtils.getOnlyOne; import static io.confluent.csid.utils.LatchTestUtils.awaitLatch; import static io.confluent.csid.utils.ThreadUtils.sleepQuietly; -import static io.confluent.parallelconsumer.ManagedTruth.assertTruth; import static io.confluent.parallelconsumer.ManagedTruth.assertWithMessage; import static java.time.Duration.ofMillis; import static java.time.Duration.ofSeconds; @@ -204,8 +203,7 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O () -> { assertWithMessage("Partition SHOULD be blocked due to back pressure") .that(partitionState) - .isAllowedMoreRecords(); - assertTruth(partitionState).isNotAllowedMoreRecords(); + .isNotAllowedMoreRecords(); }); // blocked Long partitionOffsetHighWaterMarks = wm.getPm().getHighestSeenOffset(topicPartition); From 939a15e597ad3aeb4536a8faa120d3bda4a83d84 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Thu, 21 Apr 2022 15:28:52 +0100 Subject: [PATCH 23/54] step - test fix? --- .../parallelconsumer/state/PartitionState.java | 7 +++++++ .../offsets/OffsetEncodingBackPressureTest.java | 12 ++++++++++++ 2 files changed, 19 insertions(+) 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 2269e4b3d..2567f8b58 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 @@ -322,5 +322,12 @@ private double getPressureThresholdValue() { public void onPartitionsRemoved(ShardManager sm) { sm.removeAnyShardsReferencedBy(getCommitQueue()); } + + /** + * @see #isAllowedMoreRecords() + */ + public boolean isBlocked() { + return !isAllowedMoreRecords(); + } } diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java index 099b90505..1e524b39b 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java @@ -201,9 +201,21 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O waitAtMost(ofSeconds(30)) .untilAsserted( () -> { + // old + assertThat(wm.getPm().isBlocked(topicPartition)) + .as("Partition SHOULD be blocked due to back pressure") + .isTrue(); + + // new + assertWithMessage("Partition SHOULD be blocked due to back pressure") + .that(partitionState) + .isBlocked(); + assertWithMessage("Partition SHOULD be blocked due to back pressure") .that(partitionState) .isNotAllowedMoreRecords(); + + // }); // blocked Long partitionOffsetHighWaterMarks = wm.getPm().getHighestSeenOffset(topicPartition); From 3fa6ae3902c3c6cd4641c8ef7cb6c045be451aea Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Thu, 21 Apr 2022 15:44:56 +0100 Subject: [PATCH 24/54] step - test fix? --- .../io/confluent/parallelconsumer/state/PartitionState.java | 3 +++ 1 file changed, 3 insertions(+) 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 2567f8b58..67c6617b4 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 @@ -324,6 +324,9 @@ public void onPartitionsRemoved(ShardManager sm) { } /** + * Convenience method for readability + * + * @return true if {@link #isAllowedMoreRecords()} is false * @see #isAllowedMoreRecords() */ public boolean isBlocked() { From c44f50a3a7f5317b1d74537ab8b35ae07cf0ba5c Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Thu, 21 Apr 2022 17:01:49 +0100 Subject: [PATCH 25/54] step - test fix? - make sure unit test also cleans up --- .../parallelconsumer/state/PartitionMonitor.java | 3 ++- .../AbstractParallelEoSStreamProcessorTestBase.java | 1 + .../offsets/OffsetEncodingBackPressureTest.java | 12 +++++++----- .../offsets/OffsetEncodingBackPressureUnitTest.java | 13 ++++++++----- 4 files changed, 18 insertions(+), 11 deletions(-) diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java index 37024e801..614768c7f 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java @@ -42,6 +42,7 @@ // todo rename to partition manager public class PartitionMonitor implements ConsumerRebalanceListener { + public static final double USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT = 0.75; /** * Best efforts attempt to prevent usage of offset payload beyond X% - as encoding size test is currently only done * per batch, we need to leave some buffer for the required space to overrun before hitting the hard limit where we @@ -50,7 +51,7 @@ public class PartitionMonitor implements ConsumerRebalanceListener { @Getter @Setter // todo remove static - private static double USED_PAYLOAD_THRESHOLD_MULTIPLIER = 0.75; + private static double USED_PAYLOAD_THRESHOLD_MULTIPLIER = USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT; private final Consumer consumer; diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java index 5bd8de06b..6733bbcc1 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java @@ -46,6 +46,7 @@ import static org.mockito.Mockito.*; import static pl.tlinkowski.unij.api.UniLists.of; +// todo migrate commit assertion methods in to a Truth Subject @Slf4j public abstract class AbstractParallelEoSStreamProcessorTestBase { diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java index 1e524b39b..001ae24c0 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java @@ -35,6 +35,7 @@ import static io.confluent.csid.utils.LatchTestUtils.awaitLatch; import static io.confluent.csid.utils.ThreadUtils.sleepQuietly; import static io.confluent.parallelconsumer.ManagedTruth.assertWithMessage; +import static io.confluent.parallelconsumer.state.PartitionMonitor.USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT; import static java.time.Duration.ofMillis; import static java.time.Duration.ofSeconds; import static java.util.concurrent.TimeUnit.SECONDS; @@ -60,7 +61,7 @@ class OffsetEncodingBackPressureTest extends ParallelEoSStreamProcessorTestBase @AfterAll static void cleanup() { - PartitionMonitor.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(0.75); + PartitionMonitor.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT); } /** @@ -84,6 +85,7 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O OffsetMapCodecManager.DefaultMaxMetadataSize = 40; // reduce available to make testing easier OffsetMapCodecManager.forcedCodec = Optional.of(OffsetEncoding.BitSetV2); // force one that takes a predictable large amount of space + // List> records = ktu.generateRecords(numberOfRecords); ktu.send(consumerSpy, records); @@ -198,13 +200,13 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O awaitForOneLoopCycle(); log.debug("// assert partition now blocked from threshold"); - waitAtMost(ofSeconds(30)) + waitAtMost(ofSeconds(10)) .untilAsserted( () -> { // old - assertThat(wm.getPm().isBlocked(topicPartition)) - .as("Partition SHOULD be blocked due to back pressure") - .isTrue(); +// assertThat(wm.getPm().isBlocked(topicPartition)) +// .as("Partition SHOULD be blocked due to back pressure") +// .isTrue(); // new assertWithMessage("Partition SHOULD be blocked due to back pressure") diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java index ce62b020b..7d3535152 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.parallel.ResourceAccessMode; import org.junit.jupiter.api.parallel.ResourceLock; @@ -31,6 +32,7 @@ import static io.confluent.parallelconsumer.ManagedTruth.assertTruth; import static io.confluent.parallelconsumer.ManagedTruth.assertWithMessage; +import static io.confluent.parallelconsumer.state.PartitionMonitor.USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT; import static java.time.Duration.ofMillis; /** @@ -41,6 +43,11 @@ @Slf4j class OffsetEncodingBackPressureUnitTest extends ParallelEoSStreamProcessorTestBase { + @AfterAll + static void cleanup() { + PartitionMonitor.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT); + } + @SneakyThrows @Test @ResourceLock(value = OffsetMapCodecManager.METADATA_DATA_SIZE_RESOURCE_LOCK, mode = ResourceAccessMode.READ_WRITE) @@ -54,7 +61,7 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O OffsetMapCodecManager.DefaultMaxMetadataSize = 40; // reduce available to make testing easier OffsetMapCodecManager.forcedCodec = Optional.of(OffsetEncoding.BitSetV2); // force one that takes a predictable large amount of space - + // var wm = parallelConsumer.getWm(); var pm = wm.getPm(); PartitionState partitionState = pm.getPartitionState(topicPartition); @@ -181,14 +188,10 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O } } finally { - - // todo restore static defaults - lazy way to override settings at runtime but causes bugs by allowing them to be statically changeable OffsetMapCodecManager.DefaultMaxMetadataSize = realMax; // todo wow this is smelly, but convenient OffsetMapCodecManager.forcedCodec = Optional.empty(); } - - } private void succeedExcept(WorkManager wm, List incomplete) { From eff0b13fd1796aefbeae8a974e47040f4dd6daec Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Thu, 21 Apr 2022 17:05:30 +0100 Subject: [PATCH 26/54] step - test fix? - make sure unit test also cleans up --- .idea/runConfigurations/generate_test_sources__e.xml | 4 +++- pom.xml | 1 + 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/.idea/runConfigurations/generate_test_sources__e.xml b/.idea/runConfigurations/generate_test_sources__e.xml index 2918f8a3f..2456e1d82 100644 --- a/.idea/runConfigurations/generate_test_sources__e.xml +++ b/.idea/runConfigurations/generate_test_sources__e.xml @@ -12,6 +12,8 @@

- * This must live outside of {@link PartitionState}, as it must be tracked across partition lifecycles. + * NOTE: This must live outside of {@link PartitionState}, as it must be tracked across partition lifecycles. *

* Starts at zero. + *

+ * NOTE: Must be concurrent because it can be set by one thread, but read by another. */ - private final Map partitionsAssignmentEpochs = new ConcurrentHashMap<>(); + private final Map partitionsAssignmentEpochs = new ConcurrentHashMap<>(); private final Clock clock; public PartitionState getPartitionState(TopicPartition tp) { - // may cause the system to wait for a rebalance to finish // by locking on partitionState, may cause the system to wait for a rebalance to finish - synchronized (partitionStates) { - return partitionStates.get(tp); - } +// synchronized (partitionStates) { + return partitionStates.get(tp); +// } } /** @@ -89,34 +87,34 @@ public PartitionState getPartitionState(TopicPartition tp) { @Override public void onPartitionsAssigned(Collection assignedPartitions) { log.debug("Partitions assigned: {}", assignedPartitions); - synchronized (this.partitionStates) { - - for (final TopicPartition partitionAssignment : assignedPartitions) { - boolean isAlreadyAssigned = this.partitionStates.containsKey(partitionAssignment); - if (isAlreadyAssigned) { - PartitionState previouslyAssignedState = partitionStates.get(partitionAssignment); - if (previouslyAssignedState.isRemoved()) { - log.trace("Reassignment of previously revoked partition {} - state: {}", partitionAssignment, previouslyAssignedState); - } else { - log.warn("New assignment of partition which already exists and isn't recorded as removed in " + - "partition state. Could be a state bug - was the partition revocation somehow missed, " + - "or is this a race? Please file a GH issue. Partition: {}, state: {}", partitionAssignment, previouslyAssignedState); - } +// synchronized (this.partitionStates) { + + for (final TopicPartition partitionAssignment : assignedPartitions) { + boolean isAlreadyAssigned = this.partitionStates.containsKey(partitionAssignment); + if (isAlreadyAssigned) { + PartitionState previouslyAssignedState = partitionStates.get(partitionAssignment); + if (previouslyAssignedState.isRemoved()) { + log.trace("Reassignment of previously revoked partition {} - state: {}", partitionAssignment, previouslyAssignedState); + } else { + log.warn("New assignment of partition which already exists and isn't recorded as removed in " + + "partition state. Could be a state bug - was the partition revocation somehow missed, " + + "or is this a race? Please file a GH issue. Partition: {}, state: {}", partitionAssignment, previouslyAssignedState); } } + } - incrementPartitionAssignmentEpoch(assignedPartitions); - - try { - OffsetMapCodecManager om = new OffsetMapCodecManager<>(this.consumer); // todo remove throw away instance creation - #233 - var partitionStates = om.loadPartitionStateForAssignment(assignedPartitions); - this.partitionStates.putAll(partitionStates); - } catch (Exception e) { - log.error("Error in onPartitionsAssigned", e); - throw e; - } + incrementPartitionAssignmentEpoch(assignedPartitions); + try { + OffsetMapCodecManager om = new OffsetMapCodecManager<>(this.consumer); // todo remove throw away instance creation - #233 + var partitionStates = om.loadPartitionStateForAssignment(assignedPartitions); + this.partitionStates.putAll(partitionStates); + } catch (Exception e) { + log.error("Error in onPartitionsAssigned", e); + throw e; } + +// } } /** @@ -139,10 +137,10 @@ public void onPartitionsRevoked(Collection partitions) { } void onPartitionsRemoved(final Collection partitions) { - synchronized (this.partitionStates) { - incrementPartitionAssignmentEpoch(partitions); - resetOffsetMapAndRemoveWork(partitions); - } +// synchronized (this.partitionStates) { + incrementPartitionAssignmentEpoch(partitions); + resetOffsetMapAndRemoveWork(partitions); +// } } /** @@ -201,19 +199,28 @@ private void resetOffsetMapAndRemoveWork(Collection allRemovedPa } } - public int getEpoch(final ConsumerRecord rec) { + /** + * @return the current epoch of the partition this record belongs to + */ + public Long getEpochOfPartitionForRecord(final ConsumerRecord rec) { var tp = toTopicPartition(rec); - Integer epoch = partitionsAssignmentEpochs.get(tp); - rec.topic(); + Long epoch = partitionsAssignmentEpochs.get(tp); if (epoch == null) { throw new InternalRuntimeError(msg("Received message for a partition which is not assigned: {}", rec)); } return epoch; } + /** + * @return the current epoch of the partition + */ + public Long getEpochOfPartition(TopicPartition partition) { + return partitionsAssignmentEpochs.get(partition); + } + private void incrementPartitionAssignmentEpoch(final Collection partitions) { for (final TopicPartition partition : partitions) { - int epoch = partitionsAssignmentEpochs.getOrDefault(partition, -1); + Long epoch = partitionsAssignmentEpochs.getOrDefault(partition, -1L); epoch++; partitionsAssignmentEpochs.put(partition, epoch); } @@ -230,8 +237,8 @@ private void incrementPartitionAssignmentEpoch(final Collection boolean checkIfWorkIsStale(final WorkContainer workContainer) { var topicPartitionKey = workContainer.getTopicPartition(); - Integer currentPartitionEpoch = partitionsAssignmentEpochs.get(topicPartitionKey); - int workEpoch = workContainer.getEpoch(); + Long currentPartitionEpoch = partitionsAssignmentEpochs.get(topicPartitionKey); + long workEpoch = workContainer.getEpoch(); boolean partitionNotAssigned = isPartitionRemovedOrNeverAssigned(workContainer.getCr()); @@ -330,31 +337,46 @@ public void onFailure(WorkContainer wc) { /** * Takes a record as work and puts it into internal queues, unless it's been previously recorded as completed as per * loaded records. - *

- * Locking on partition state here, means that the check for assignment is in the same sync block as registering the - * work with the {@link TopicPartition}'s {@link PartitionState} and the {@link ShardManager}. Keeping the two - * different views in sync. Of course now, having a shared nothing architecture would mean all access to the state - * is by a single thread, and so this could never occur (see ). */ - void maybeRegisterNewRecordAsWork(final EpochAndRecords records) { + void maybeRegisterNewRecordAsWork(final EpochAndRecords recordsCollection) { // if (records == null) return false; - synchronized (partitionStates) { - for (var rec : records.getConsumerRecs()) { - if (isPartitionRemovedOrNeverAssigned(rec)) { - log.debug("Record in buffer for a partition no longer assigned. Dropping. TP: {} rec: {}", toTopicPartition(rec), rec); - } +// /** +// * Locking on partition state here, means that the check for partition assignment (that the source partition of the +// * work is still assigned) is in the same sync block as registering the work with the {@link TopicPartition}'s +// * {@link PartitionState} and the {@link ShardManager}. Keeping the two different views in sync. Of course now, +// * having a shared nothing architecture would mean all access to the state is by a single thread, and so this could +// * never occur. +// */ +// synchronized (partitionStates) { + + for (var partition : recordsCollection.partitions()) { + var records = recordsCollection.records(partition); + var epochOfInboundRecords = records.getEpochOfPartitionAtPoll(); + for (var rec : records.getRecords()) { + + // 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)) { + + if (isPartitionRemovedOrNeverAssigned(rec)) { + log.debug("Record in buffer for a partition no longer assigned. Dropping. TP: {} rec: {}", toTopicPartition(rec), rec); + } - if (isRecordPreviouslyCompleted(rec)) { - log.trace("Record previously completed, skipping. offset: {}", rec.offset()); - } else { - int currentPartitionEpoch = getEpoch(rec); - var work = new WorkContainer<>(currentPartitionEpoch, rec, options.getRetryDelayProvider(), clock); + if (isRecordPreviouslyCompleted(rec)) { + log.trace("Record previously completed, skipping. offset: {}", rec.offset()); + } else { + var work = new WorkContainer<>(epochOfInboundRecords, rec, options.getRetryDelayProvider(), clock); - sm.addWorkContainer(work); - addWorkContainer(work); + 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); } } } +// } } public Map collectDirtyCommitData() { 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 67c6617b4..026097658 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 @@ -186,6 +186,7 @@ private void updateHighestSucceededOffsetSoFar(WorkContainer work) { } public void addWorkContainer(WorkContainer wc) { + // for test debugging // long offsetHighestSeen = getOffsetHighestSeen(); // if (wc.offset() != offsetHighestSeen + 1) { // log.error(""); @@ -194,7 +195,6 @@ public void addWorkContainer(WorkContainer wc) { maybeRaiseHighestSeenOffset(wc.offset()); commitQueue.put(wc.offset(), wc); incompleteOffsets.add(wc.offset()); - } /** diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkContainer.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkContainer.java index 584759ac7..a14ed5285 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkContainer.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkContainer.java @@ -38,7 +38,7 @@ public class WorkContainer implements Comparable> { * in the system of in flight. */ @Getter - private final int epoch; + private final long epoch; /** * Simple way to differentiate treatment based on type @@ -85,7 +85,7 @@ public class WorkContainer implements Comparable> { // static instance so can't access generics - but don't need them as Options class ensures type is correct private static Function retryDelayProvider; - public WorkContainer(int epoch, ConsumerRecord cr, Function, Duration> retryDelayProvider, String workType, Clock clock) { + public WorkContainer(long epoch, ConsumerRecord cr, Function, Duration> retryDelayProvider, String workType, Clock clock) { Objects.requireNonNull(workType); this.epoch = epoch; @@ -99,7 +99,7 @@ public WorkContainer(int epoch, ConsumerRecord cr, Function cr, Function, Duration> retryDelayProvider, Clock clock) { + public WorkContainer(long epoch, ConsumerRecord cr, Function, Duration> retryDelayProvider, Clock clock) { this(epoch, cr, retryDelayProvider, DEFAULT_TYPE, clock); } diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/MultiInstanceHighVolumeTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/MultiInstanceHighVolumeTest.java index a0876227a..21572334a 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/MultiInstanceHighVolumeTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/MultiInstanceHighVolumeTest.java @@ -43,7 +43,7 @@ class MultiInstanceHighVolumeTest extends BrokerIntegrationTest public AtomicInteger processedCount = new AtomicInteger(0); public AtomicInteger producedCount = new AtomicInteger(0); - int maxPoll = 500; + int maxPoll = 5000000; ParallelConsumerOptions.CommitMode commitMode = ParallelConsumerOptions.CommitMode.PERIODIC_CONSUMER_SYNC; ParallelConsumerOptions.ProcessingOrder order = ParallelConsumerOptions.ProcessingOrder.KEY; diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java index 7d3535152..47675a31f 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java @@ -214,10 +214,8 @@ private WorkContainer findWC(List> private void sendRecordsToWM(int numberOfRecords, WorkManager wm) { log.debug("~Sending {} more records", numberOfRecords); List> records = ktu.generateRecords(numberOfRecords); - int hardCodedEpochForTesting = 0; - wm.registerWork(new EpochAndRecords<>(new ConsumerRecords<>(UniMaps.of(topicPartition, records)), hardCodedEpochForTesting)); + wm.registerWork(new EpochAndRecords<>(new ConsumerRecords<>(UniMaps.of(topicPartition, records)), wm.getPm())); Truth.assertThat(wm.getNumberOfWorkQueuedInShardsAwaitingSelection()).isEqualTo(numberOfRecords); } - } 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 39b7e303b..5c2358d54 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 @@ -179,7 +179,7 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset { WorkManager wmm = new WorkManager<>(options, consumerSpy); wmm.onPartitionsAssigned(UniSets.of(new TopicPartition(INPUT_TOPIC, 0))); - wmm.registerWork(new EpochAndRecords<>(testRecords, 0)); + wmm.registerWork(new EpochAndRecords<>(testRecords, wmm.getPm())); List> work = wmm.getWorkIfAvailable(); assertThat(work).hasSameSizeAs(records); @@ -224,7 +224,7 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset { var newWm = new WorkManager<>(options, consumerSpy); newWm.onPartitionsAssigned(UniSets.of(tp)); - newWm.registerWork(new EpochAndRecords(testRecords, 0)); + newWm.registerWork(new EpochAndRecords(testRecords, newWm.getPm())); var pm = newWm.getPm(); var partitionState = pm.getPartitionState(tp); diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java index 9e43c5251..4a6926def 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java @@ -104,7 +104,7 @@ private void registerSomeWork(int partition) { Map>> m = new HashMap<>(); m.put(topicPartitionOf(partition), of(rec0, rec1, rec2)); var recs = new ConsumerRecords<>(m); - wm.registerWork(new EpochAndRecords(recs, 0)); + wm.registerWork(new EpochAndRecords(recs, wm.getPm())); } private ConsumerRecord makeRec(String value, String key, int partition) { @@ -393,7 +393,7 @@ void insertWrongOrderPreservesOffsetOrdering() { } private void registerWork(ConsumerRecords recs) { - wm.registerWork(new EpochAndRecords<>(recs, 0)); + wm.registerWork(new EpochAndRecords<>(recs, wm.getPm())); } From 94ebc5c1aec9ea5bc5766baafd9a9baa2dcb95c2 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Thu, 21 Apr 2022 20:32:24 +0100 Subject: [PATCH 31/54] turn max poll back to default (500) --- .../integrationTests/MultiInstanceHighVolumeTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/MultiInstanceHighVolumeTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/MultiInstanceHighVolumeTest.java index 21572334a..035541042 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/MultiInstanceHighVolumeTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/MultiInstanceHighVolumeTest.java @@ -43,7 +43,7 @@ class MultiInstanceHighVolumeTest extends BrokerIntegrationTest public AtomicInteger processedCount = new AtomicInteger(0); public AtomicInteger producedCount = new AtomicInteger(0); - int maxPoll = 5000000; + int maxPoll = 500; // 500 is the kafka default ParallelConsumerOptions.CommitMode commitMode = ParallelConsumerOptions.CommitMode.PERIODIC_CONSUMER_SYNC; ParallelConsumerOptions.ProcessingOrder order = ParallelConsumerOptions.ProcessingOrder.KEY; From 854c0fab0281c1032f5d8de46c76ce8e64c99f15 Mon Sep 17 00:00:00 2001 From: Niels Date: Thu, 17 Feb 2022 13:47:33 +0100 Subject: [PATCH 32/54] Draft for pause/resume functionality (confluentinc/parallel-consumer#193) --- .../parallelconsumer/ParallelConsumer.java | 29 ++++++++++ .../AbstractParallelEoSStreamProcessor.java | 26 ++++++++- .../internal/BrokerPollSystem.java | 53 +++++++++++++++++-- .../parallelconsumer/internal/State.java | 10 +++- 4 files changed, 111 insertions(+), 7 deletions(-) diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumer.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumer.java index b3e8bc152..921ba2e97 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumer.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumer.java @@ -49,6 +49,35 @@ public interface ParallelConsumer extends DrainingCloseable { */ void subscribe(Pattern pattern, ConsumerRebalanceListener callback); + /** + * Pause this consumer (i.e. transition to state {@link io.confluent.parallelconsumer.internal.State#paused paused} for the + * controller and the broker poll system). + *

+ * This operation only has an effect if the consumer is currently in state + * {@link io.confluent.parallelconsumer.internal.State#running running}. In all other + * {@link io.confluent.parallelconsumer.internal.State State}s calling this method will be a no-op. + *

+ * If the consumer is paused, the system will stop polling for new records from the Kafka Broker and also stop submitting + * work that has already been polled to the processing pool. + * Already submitted in flight work however will be finished (i.e. active workers are not interrupted). + *

+ * Note: This does not actively pause the subscription on the underlying Kafka Broker. Also pending offset commits will + * still be performed. + *

+ */ + void pauseIfRunning(); + + /** + * Resume this consumer (i.e. transition to state {@link io.confluent.parallelconsumer.internal.State#running running} for the + * controller and the broker poll system). + *

+ * This operation only has an effect if the consumer is currently in state + * {@link io.confluent.parallelconsumer.internal.State#paused paused}. In all other + * {@link io.confluent.parallelconsumer.internal.State State}s calling this method will be a no-op. + *

+ */ + void resumeIfPaused(); + /** * A simple tuple structure. * 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 5ec0343f0..625a540b3 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 @@ -658,7 +658,7 @@ private void controlLoop(Function, List> userFu log.trace("Loop: Process mailbox"); processWorkCompleteMailBox(); - if (state == running) { + if (state == running || state == paused) { // offsets will be committed when the consumer has its partitions revoked log.trace("Loop: Maybe commit"); commitOffsetsMaybe(); @@ -1047,7 +1047,7 @@ private boolean lingeringOnCommitWouldBeBeneficial() { private Duration getTimeToNextCommitCheck() { // draining is a normal running mode for the controller - if (state == running || state == draining) { + if (state == running || state == draining || state == paused) { Duration timeSinceLastCommit = getTimeSinceLastCheck(); Duration timeBetweenCommits = getTimeBetweenCommits(); @SuppressWarnings("UnnecessaryLocalVariable") @@ -1214,4 +1214,26 @@ private void clearCommitCommand() { } } + @Override + public void pauseIfRunning() { + if (this.state == State.running) { + log.debug("Transitioning to state paused."); + this.state = State.paused; + brokerPollSubsystem.pausePollingAndWorkRegistrationIfRunning(); + } else { + log.debug("Skipping transition to state paused. Current state is {}.", this.state); + } + } + + @Override + public void resumeIfPaused() { + if (this.state == State.paused) { + log.debug("Transitioning to state running."); + brokerPollSubsystem.resumePollingAndWorkRegistrationIfPaused(); + this.state = State.running; + } else { + log.debug("Skipping transition to state running. Current state is {}.", this.state); + } + } + } diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java index 908408e7f..9f21e652c 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java @@ -1,9 +1,5 @@ package io.confluent.parallelconsumer.internal; -/*- - * Copyright (C) 2020-2022 Confluent, Inc. - */ - import io.confluent.parallelconsumer.ParallelConsumerOptions; import io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode; import io.confluent.parallelconsumer.state.WorkManager; @@ -44,6 +40,11 @@ public class BrokerPollSystem implements OffsetCommitter private Optional> pollControlThreadFuture; + /** + * While {@link io.confluent.parallelconsumer.internal.State#paused paused} is an externally controlled state that + * temporarily stops polling and work registration, the {@code paused} flag is used internally to pause + * subscriptions if polling needs to be throttled. + */ @Getter private volatile boolean paused = false; @@ -129,6 +130,18 @@ private boolean controlLoop() { doClose(); } } + + if (state == State.paused) { + // as long as the state is paused, this loop doesn't do any I/O work + // therefor we sleep for 100 ms to reduce CPU load + try { + Thread.sleep(100); + } catch (InterruptedException e) { + log.debug("Loop: Sleep in state paused was interrupted."); + // swallow the exception and just go back to business, keeping the interrupt state set + Thread.currentThread().interrupt(); + } + } } log.debug("Broker poller thread finished normally, returning OK (true) to future..."); return true; @@ -367,4 +380,36 @@ public void wakeupIfPaused() { if (paused) consumerManager.wakeup(); } + + /** + * Pause polling from the underlying Kafka Broker. + *

+ * Note: If the poll system is currently not in state {@link io.confluent.parallelconsumer.internal.State#running + * running}, calling this method will be a no-op. + *

+ */ + public void pausePollingAndWorkRegistrationIfRunning() { + if (this.state == State.running) { + log.debug("Transitioning broker poll system to state paused."); + this.state = State.paused; + } else { + log.debug("Skipping transition of broker poll system to state paused. Current state is {}.", this.state); + } + } + + /** + * Resume polling from the underlying Kafka Broker. + *

+ * Note: If the poll system is currently not in state {@link io.confluent.parallelconsumer.internal.State#paused + * paused}, calling this method will be a no-op. + *

+ */ + public void resumePollingAndWorkRegistrationIfPaused() { + if (this.state == State.paused) { + log.debug("Transitioning broker poll system to state running."); + this.state = State.running; + } else { + log.debug("Skipping transition of broker poll system to state running. Current state is {}.", this.state); + } + } } diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/State.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/State.java index 40670ddd7..ea137bf12 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/State.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/State.java @@ -1,7 +1,7 @@ package io.confluent.parallelconsumer.internal; /*- - * Copyright (C) 2020-2021 Confluent, Inc. + * Copyright (C) 2020-2022 Confluent, Inc. */ /** @@ -10,6 +10,14 @@ public enum State { unused, running, + /** + * When paused, the system will stop polling for new records from the broker and also stop submitting work that has already + * been polled to the processing pool. + * Committing offsets however + * Already submitted in flight work however will be finished (i.e. ). + * Parallel stream processor handleWork + */ + paused, /** * When draining, the system will stop polling for more records, but will attempt to process all already downloaded * records. Note that if you choose to close without draining, records already processed will still be committed From 1401a28b3bed4b4bc3de79904da59ac7489fb55c Mon Sep 17 00:00:00 2001 From: Niels Date: Fri, 18 Feb 2022 14:05:03 +0100 Subject: [PATCH 33/54] Improve logging + add unit tests (confluentinc/parallel-consumer#193). --- .../AbstractParallelEoSStreamProcessor.java | 8 +- .../internal/BrokerPollSystem.java | 14 +- ...llelEoSStreamProcessorPauseResumeTest.java | 242 ++++++++++++++++++ 3 files changed, 253 insertions(+), 11 deletions(-) create mode 100644 parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParrallelEoSStreamProcessorPauseResumeTest.java 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 625a540b3..dc4a68a56 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 @@ -1217,22 +1217,22 @@ private void clearCommitCommand() { @Override public void pauseIfRunning() { if (this.state == State.running) { - log.debug("Transitioning to state paused."); + log.info("Transitioning parallel consumer to state paused."); this.state = State.paused; brokerPollSubsystem.pausePollingAndWorkRegistrationIfRunning(); } else { - log.debug("Skipping transition to state paused. Current state is {}.", this.state); + log.info("Skipping transition of parallel consumer to state paused. Current state is {}.", this.state); } } @Override public void resumeIfPaused() { if (this.state == State.paused) { - log.debug("Transitioning to state running."); + log.info("Transitioning paarallel consumer to state running."); brokerPollSubsystem.resumePollingAndWorkRegistrationIfPaused(); this.state = State.running; } else { - log.debug("Skipping transition to state running. Current state is {}.", this.state); + log.info("Skipping transition of parallel consumer to state running. Current state is {}.", this.state); } } diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java index 9f21e652c..7dec222e4 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java @@ -133,11 +133,11 @@ private boolean controlLoop() { if (state == State.paused) { // as long as the state is paused, this loop doesn't do any I/O work - // therefor we sleep for 100 ms to reduce CPU load + // therefore we sleep for 50 ms to reduce CPU load try { - Thread.sleep(100); + Thread.sleep(50L); } catch (InterruptedException e) { - log.debug("Loop: Sleep in state paused was interrupted."); + log.debug("Loop: Sleep in state paused was interrupted.", e); // swallow the exception and just go back to business, keeping the interrupt state set Thread.currentThread().interrupt(); } @@ -390,10 +390,10 @@ public void wakeupIfPaused() { */ public void pausePollingAndWorkRegistrationIfRunning() { if (this.state == State.running) { - log.debug("Transitioning broker poll system to state paused."); + log.info("Transitioning broker poll system to state paused."); this.state = State.paused; } else { - log.debug("Skipping transition of broker poll system to state paused. Current state is {}.", this.state); + log.info("Skipping transition of broker poll system to state paused. Current state is {}.", this.state); } } @@ -406,10 +406,10 @@ public void pausePollingAndWorkRegistrationIfRunning() { */ public void resumePollingAndWorkRegistrationIfPaused() { if (this.state == State.paused) { - log.debug("Transitioning broker poll system to state running."); + log.info("Transitioning broker poll system to state running."); this.state = State.running; } else { - log.debug("Skipping transition of broker poll system to state running. Current state is {}.", this.state); + log.info("Skipping transition of broker poll system to state running. Current state is {}.", this.state); } } } diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParrallelEoSStreamProcessorPauseResumeTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParrallelEoSStreamProcessorPauseResumeTest.java new file mode 100644 index 000000000..3915c5f87 --- /dev/null +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParrallelEoSStreamProcessorPauseResumeTest.java @@ -0,0 +1,242 @@ +package io.confluent.parallelconsumer; + +import static java.util.concurrent.TimeUnit.SECONDS; + +import java.util.Optional; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import java.util.function.Supplier; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.assertj.core.api.BDDAssertions; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; +import org.slf4j.LoggerFactory; + +import ch.qos.logback.classic.Level; +import ch.qos.logback.classic.Logger; +import ch.qos.logback.classic.LoggerContext; +import ch.qos.logback.classic.filter.LevelFilter; +import ch.qos.logback.classic.spi.ILoggingEvent; +import ch.qos.logback.core.AppenderBase; +import ch.qos.logback.core.spi.FilterReply; +import io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode; +import io.confluent.parallelconsumer.internal.AbstractParallelEoSStreamProcessor; +import io.confluent.parallelconsumer.internal.BrokerPollSystem; +import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; + +/** + * Test for pause/resume feature of the parallel consumer (see {@code GH#193}). + * + * @author niels.oertel + */ +@Timeout(value = 10, unit = SECONDS) +@Slf4j +class ParrallelEoSStreamProcessorPauseResumeTest extends ParallelEoSStreamProcessorTestBase { + + private static final AtomicLong MY_ID_GENERATOR = new AtomicLong(); + + private static final AtomicLong RECORD_BATCH_KEY_GENERATOR = new AtomicLong(); + + private ControlLoopTracker controlLoopTracker; + + private static class TestUserFunction implements Consumer> { + + AtomicLong numProcessedRecords = new AtomicLong(); + + @Override + public void accept(ConsumerRecord t) { + numProcessedRecords.incrementAndGet(); + } + + public void reset() { + numProcessedRecords.set(0L); + } + } + + private ParallelConsumerOptions getBaseOptions(final CommitMode commitMode) { + return ParallelConsumerOptions.builder() + .commitMode(commitMode) + .consumer(consumerSpy) + .maxConcurrency(3) + .build(); + } + + private void addRecords(final int numRecords) { + long recordBatchKey = RECORD_BATCH_KEY_GENERATOR.incrementAndGet(); + log.debug("Producing {} records with batch key {}.", numRecords, recordBatchKey); + for (int i = 0; i < numRecords; ++i) { + consumerSpy.addRecord(ktu.makeRecord("key-" + recordBatchKey + i, "v0-test-" + i)); + } + } + + /** + * Log message appender that monitors the entry messages of the two control + * loops and counts how often they have been executed to build reliable tests + * that don't need to use {@link Thread#sleep(long)} to wait for changes to take + * effect. + *

+ * Note: {@link ParallelEoSStreamProcessorTestBase#waitForSomeLoopCycles(int)} + * provides similar functionality but not for the {@link BroerPollSystem}. + *

+ * + * @author niels.oertel + */ + public static class ControlLoopTracker extends AppenderBase { + + private static final AtomicLong ID_GENERATOR = new AtomicLong(); + + private static final String PSP_CONTROL_LOOP_MESSAGE = "Loop: Process mailbox"; + + private static final String BPS_CONTROL_LOOP_MESSAGE = "Loop: Broker poller: ({})"; + + private final String myInstanceId; + + private final AtomicLong pspControlLoopCounter = new AtomicLong(); + + private final AtomicLong bpsControlLoopCounter = new AtomicLong(); + + public ControlLoopTracker(final String myInstanceId) { + this.myInstanceId = myInstanceId; + this.setName("ControlLoopMonitor-" + ID_GENERATOR.incrementAndGet()); + } + + @Override + protected void append(final ILoggingEvent e) { + String myInstanceId = e.getMDCPropertyMap().get(AbstractParallelEoSStreamProcessor.MDC_INSTANCE_ID); + if (!this.myInstanceId.equals(myInstanceId)) { + // log message doesn't belong to the parallel consumer that is tracked + } else if (AbstractParallelEoSStreamProcessor.class.getName().equals(e.getLoggerName()) + && PSP_CONTROL_LOOP_MESSAGE.equals(e.getMessage())) { + // this is a control loop message from the parallel stream processor + pspControlLoopCounter.incrementAndGet(); + } else if (BrokerPollSystem.class.getName().equals(e.getLoggerName()) + && BPS_CONTROL_LOOP_MESSAGE.equals(e.getMessage())) { + // this is a control loop message from the broker poll system + bpsControlLoopCounter.incrementAndGet(); + } else { + // this is a message from the parallel consumer that is tracked but we're not + // interested in it + } + } + + public void reset() { + this.pspControlLoopCounter.set(0L); + this.bpsControlLoopCounter.set(0L); + } + + public void waitForSomeControlLoopCycles(int numCycles, long timeout, TimeUnit unit) { + long currentPspControlLoopCounter = pspControlLoopCounter.get(); + long currentBspControlLoopCounter = bpsControlLoopCounter.get(); + waitForSomeLoopCycles(currentPspControlLoopCounter, pspControlLoopCounter::get, numCycles, timeout, unit); + waitForSomeLoopCycles(currentBspControlLoopCounter, bpsControlLoopCounter::get, numCycles, timeout, unit); + } + + public void waitForSomeParallelStreamProcessorControlLoopCycles(int numCycles, long timeout, TimeUnit unit) { + waitForSomeLoopCycles(pspControlLoopCounter.get(), pspControlLoopCounter::get, numCycles, timeout, unit); + } + + public void waitForSomeBrokerPollSystemControlLoopCycles(int numCycles, long timeout, TimeUnit unit) { + waitForSomeLoopCycles(bpsControlLoopCounter.get(), bpsControlLoopCounter::get, numCycles, timeout, unit); + } + + private static void waitForSomeLoopCycles(long lastCount, Supplier currentCounter, int numCycles, + long timeout, TimeUnit unit) { + Awaitility.waitAtMost(timeout, unit).alias("Hello world: " + lastCount).until(currentCounter::get, + currentCount -> currentCount >= lastCount + numCycles); + } + } + + @AfterEach + void cleanup() { + if (null != controlLoopTracker) { + LoggerContext lc = (LoggerContext) LoggerFactory.getILoggerFactory(); + lc.getLoggerList().forEach(logger -> logger.detachAppender(controlLoopTracker.getName())); + controlLoopTracker.reset(); + controlLoopTracker.stop(); + controlLoopTracker = null; + } + + if (null != parallelConsumer) { + parallelConsumer.close(); + } + } + + private void setupParallelConsumerInstanceAndLogCapture(final CommitMode commitMode) { + setupParallelConsumerInstance(getBaseOptions(commitMode)); + + // register unique ID on the parallel consumer + String myId = "p/r-test-" + MY_ID_GENERATOR.incrementAndGet(); + parallelConsumer.setMyId(Optional.of(myId)); + + // setup the log capture to be able to follow the two loops + LoggerContext lc = (LoggerContext) LoggerFactory.getILoggerFactory(); + + // add filter to the STDOUT appender to ensure it stays on level INFO + LevelFilter levelInfoFilter = new LevelFilter(); + levelInfoFilter.setContext(lc); + levelInfoFilter.setLevel(Level.INFO); + levelInfoFilter.setOnMismatch(FilterReply.DENY); + levelInfoFilter.start(); + lc.getLogger(Logger.ROOT_LOGGER_NAME).getAppender("STDOUT").addFilter(levelInfoFilter); + + // create control loop tracker and register it as log appender + controlLoopTracker = new ControlLoopTracker(myId); + controlLoopTracker.setContext(lc); + Logger parallelStreamProcessorLogger = (Logger) LoggerFactory.getLogger(AbstractParallelEoSStreamProcessor.class); + parallelStreamProcessorLogger.setLevel(Level.TRACE); + parallelStreamProcessorLogger.addAppender(controlLoopTracker); + Logger brokerPollSystemLogger = (Logger) LoggerFactory.getLogger(BrokerPollSystem.class); + brokerPollSystemLogger.setLevel(Level.TRACE); + brokerPollSystemLogger.addAppender(controlLoopTracker); + controlLoopTracker.start(); + } + + @ParameterizedTest() + @EnumSource(CommitMode.class) + @SneakyThrows + void pausingAndResumingProcessingShouldWork(final CommitMode commitMode) { + int numTestRecordsPerBatch = 1_000; + + // setup parallel consumer with custom processing function + setupParallelConsumerInstanceAndLogCapture(commitMode); + TestUserFunction testUserFunction = new TestUserFunction(); + parallelConsumer.poll(testUserFunction); + + // produce some messages + addRecords(numTestRecordsPerBatch); + + // wait for processing to finish + Awaitility + .waitAtMost(5L, TimeUnit.SECONDS) + .pollDelay(50L, TimeUnit.MILLISECONDS) + .until(testUserFunction.numProcessedRecords::get, numRecords -> numTestRecordsPerBatch == numRecords); + testUserFunction.reset(); + + // pause parallel consumer and wait for control loops to catch up + parallelConsumer.pauseIfRunning(); + controlLoopTracker.waitForSomeParallelStreamProcessorControlLoopCycles(1, 5L, TimeUnit.SECONDS); + controlLoopTracker.waitForSomeBrokerPollSystemControlLoopCycles(1, 5L, TimeUnit.SECONDS); + + // produce more messages -> nothing should be processed + addRecords(numTestRecordsPerBatch); + controlLoopTracker.waitForSomeControlLoopCycles(5, 5L, TimeUnit.SECONDS); + + // shouldn't have produced any records + BDDAssertions.assertThat(testUserFunction.numProcessedRecords).hasValue(0L); + + // resume parallel consumer -> messages should be processed now + parallelConsumer.resumeIfPaused(); + Awaitility + .waitAtMost(5, TimeUnit.SECONDS) + .until(testUserFunction.numProcessedRecords::get, numRecords -> numTestRecordsPerBatch == numRecords); + testUserFunction.reset(); + } + + // TODO: Add also test for pausing under load +} From cada440add1445542297ac40e2dcee2aa82d9c39 Mon Sep 17 00:00:00 2001 From: Niels Date: Sat, 19 Feb 2022 20:25:28 +0100 Subject: [PATCH 34/54] Fix copyrights. --- .../ParrallelEoSStreamProcessorPauseResumeTest.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParrallelEoSStreamProcessorPauseResumeTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParrallelEoSStreamProcessorPauseResumeTest.java index 3915c5f87..6291d7a0b 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParrallelEoSStreamProcessorPauseResumeTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParrallelEoSStreamProcessorPauseResumeTest.java @@ -1,5 +1,9 @@ package io.confluent.parallelconsumer; +/*- + * Copyright (C) 2020-2022 Confluent, Inc. + */ + import static java.util.concurrent.TimeUnit.SECONDS; import java.util.Optional; From 276163118e505d865f9ebebca0b1361478164c7e Mon Sep 17 00:00:00 2001 From: Niels Date: Mon, 28 Feb 2022 11:41:34 +0100 Subject: [PATCH 35/54] Don't pause the BrokerPollSystem when pausing the consumer. Improve javadoc + test coverage (confluentinc#193). --- .../parallelconsumer/ParallelConsumer.java | 37 ++-- .../AbstractParallelEoSStreamProcessor.java | 6 +- .../internal/BrokerPollSystem.java | 12 -- .../parallelconsumer/internal/State.java | 9 +- ...lelEoSStreamProcessorPauseResumeTest.java} | 191 ++++++++++++++++-- 5 files changed, 201 insertions(+), 54 deletions(-) rename parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/{ParrallelEoSStreamProcessorPauseResumeTest.java => ParallelEoSStreamProcessorPauseResumeTest.java} (54%) diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumer.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumer.java index 921ba2e97..bfcac16cc 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumer.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumer.java @@ -50,30 +50,33 @@ public interface ParallelConsumer extends DrainingCloseable { void subscribe(Pattern pattern, ConsumerRebalanceListener callback); /** - * Pause this consumer (i.e. transition to state {@link io.confluent.parallelconsumer.internal.State#paused paused} for the - * controller and the broker poll system). + * Pause this consumer (i.e. stop processing of messages). *

- * This operation only has an effect if the consumer is currently in state - * {@link io.confluent.parallelconsumer.internal.State#running running}. In all other - * {@link io.confluent.parallelconsumer.internal.State State}s calling this method will be a no-op. - *

- * If the consumer is paused, the system will stop polling for new records from the Kafka Broker and also stop submitting - * work that has already been polled to the processing pool. - * Already submitted in flight work however will be finished (i.e. active workers are not interrupted). - *

- * Note: This does not actively pause the subscription on the underlying Kafka Broker. Also pending offset commits will - * still be performed. + * This operation only has an effect if the consumer is currently running. In all other cases calling this method + * will be silent a no-op. + *

+ *

+ * Once the consumer is paused, the system will stop submitting work to the processing pool. Already submitted in + * flight work however will be finished. + * This includes work that is currently being processed inside a user function as well as work that has already + * been submitted to the processing pool but has not been picked up by a free worker yet. + *

+ *

+ * General remarks: + *

    + *
  • A paused consumer may still keep polling for new work until internal buffers are filled.
  • + *
  • This operation does not actively pause the subscription on the underlying Kafka Broker (compared to + * {@link org.apache.kafka.clients.consumer.KafkaConsumer#pause KafkaConsumer#pause}).
  • + *
  • Pending offset commits will still be performed when the consumer is paused.
  • *

    */ void pauseIfRunning(); /** - * Resume this consumer (i.e. transition to state {@link io.confluent.parallelconsumer.internal.State#running running} for the - * controller and the broker poll system). + * Resume this consumer (i.e. continue processing of messages). *

    - * This operation only has an effect if the consumer is currently in state - * {@link io.confluent.parallelconsumer.internal.State#paused paused}. In all other - * {@link io.confluent.parallelconsumer.internal.State State}s calling this method will be a no-op. + * This operation only has an effect if the consumer is currently paused. In all other cases calling this method + * will be a silent no-op. *

    */ void resumeIfPaused(); 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 dc4a68a56..560238bf1 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 @@ -1219,9 +1219,8 @@ public void pauseIfRunning() { if (this.state == State.running) { log.info("Transitioning parallel consumer to state paused."); this.state = State.paused; - brokerPollSubsystem.pausePollingAndWorkRegistrationIfRunning(); } else { - log.info("Skipping transition of parallel consumer to state paused. Current state is {}.", this.state); + log.debug("Skipping transition of parallel consumer to state paused. Current state is {}.", this.state); } } @@ -1229,10 +1228,9 @@ public void pauseIfRunning() { public void resumeIfPaused() { if (this.state == State.paused) { log.info("Transitioning paarallel consumer to state running."); - brokerPollSubsystem.resumePollingAndWorkRegistrationIfPaused(); this.state = State.running; } else { - log.info("Skipping transition of parallel consumer to state running. Current state is {}.", this.state); + log.debug("Skipping transition of parallel consumer to state running. Current state is {}.", this.state); } } diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java index 7dec222e4..254ab6618 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java @@ -130,18 +130,6 @@ private boolean controlLoop() { doClose(); } } - - if (state == State.paused) { - // as long as the state is paused, this loop doesn't do any I/O work - // therefore we sleep for 50 ms to reduce CPU load - try { - Thread.sleep(50L); - } catch (InterruptedException e) { - log.debug("Loop: Sleep in state paused was interrupted.", e); - // swallow the exception and just go back to business, keeping the interrupt state set - Thread.currentThread().interrupt(); - } - } } log.debug("Broker poller thread finished normally, returning OK (true) to future..."); return true; diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/State.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/State.java index ea137bf12..1ea8dfed1 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/State.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/State.java @@ -11,11 +11,10 @@ public enum State { unused, running, /** - * When paused, the system will stop polling for new records from the broker and also stop submitting work that has already - * been polled to the processing pool. - * Committing offsets however - * Already submitted in flight work however will be finished (i.e. ). - * Parallel stream processor handleWork + * When paused, the system will stop submitting work to the processing pool. Polling for new work however may + * continue until internal buffers have been filled sufficiently and the auto-throttling takes effect. + * In flight work will not be affected by transitioning to this state (i.e. processing will finish without any + * interrupts being sent). */ paused, /** diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParrallelEoSStreamProcessorPauseResumeTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorPauseResumeTest.java similarity index 54% rename from parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParrallelEoSStreamProcessorPauseResumeTest.java rename to parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorPauseResumeTest.java index 6291d7a0b..171c7926e 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParrallelEoSStreamProcessorPauseResumeTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorPauseResumeTest.java @@ -6,14 +6,23 @@ import static java.util.concurrent.TimeUnit.SECONDS; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; import java.util.function.Supplier; +import java.util.stream.Collectors; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.assertj.core.api.BDDAssertions; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; import org.awaitility.Awaitility; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Timeout; @@ -21,6 +30,8 @@ import org.junit.jupiter.params.provider.EnumSource; import org.slf4j.LoggerFactory; +import com.google.common.truth.Truth; + import ch.qos.logback.classic.Level; import ch.qos.logback.classic.Logger; import ch.qos.logback.classic.LoggerContext; @@ -41,7 +52,7 @@ */ @Timeout(value = 10, unit = SECONDS) @Slf4j -class ParrallelEoSStreamProcessorPauseResumeTest extends ParallelEoSStreamProcessorTestBase { +class ParallelEoSStreamProcessorPauseResumeTest extends ParallelEoSStreamProcessorTestBase { private static final AtomicLong MY_ID_GENERATOR = new AtomicLong(); @@ -51,23 +62,48 @@ class ParrallelEoSStreamProcessorPauseResumeTest extends ParallelEoSStreamProces private static class TestUserFunction implements Consumer> { - AtomicLong numProcessedRecords = new AtomicLong(); + private final AtomicLong numProcessedRecords = new AtomicLong(); + + /** + * The number of in flight records. Note that this may not exactly match the + * real number of in flight records as parallel consumer has a wrapper around + * the user function so incrementing/decrementing the counter is a little bit + * delayed. + */ + private final AtomicInteger numInFlightRecords = new AtomicInteger(); + + private final ReentrantLock mutex = new ReentrantLock(); + + public void lockProcessing() { + mutex.lock(); + } + + public void unlockProcessing() { + mutex.unlock(); + } @Override public void accept(ConsumerRecord t) { - numProcessedRecords.incrementAndGet(); + numInFlightRecords.incrementAndGet(); + try { + lockProcessing(); + numProcessedRecords.incrementAndGet(); + } finally { + unlockProcessing(); + numInFlightRecords.decrementAndGet(); + } } - + public void reset() { numProcessedRecords.set(0L); } } - private ParallelConsumerOptions getBaseOptions(final CommitMode commitMode) { + private ParallelConsumerOptions getBaseOptions(final CommitMode commitMode, int maxConcurrency) { return ParallelConsumerOptions.builder() .commitMode(commitMode) .consumer(consumerSpy) - .maxConcurrency(3) + .maxConcurrency(maxConcurrency) .build(); } @@ -171,8 +207,8 @@ void cleanup() { } } - private void setupParallelConsumerInstanceAndLogCapture(final CommitMode commitMode) { - setupParallelConsumerInstance(getBaseOptions(commitMode)); + private void setupParallelConsumerInstanceAndLogCapture(final CommitMode commitMode, final int maxConcurrency) { + setupParallelConsumerInstance(getBaseOptions(commitMode, maxConcurrency)); // register unique ID on the parallel consumer String myId = "p/r-test-" + MY_ID_GENERATOR.incrementAndGet(); @@ -201,16 +237,55 @@ private void setupParallelConsumerInstanceAndLogCapture(final CommitMode commitM controlLoopTracker.start(); } + private long getOverallCommittedOffset() { + return getCommittedOffsetsByPartitions().values().stream().collect(Collectors.summingLong(Long::longValue)); + } + + private Map getCommittedOffsetsByPartitions() { + List>> commitHistory = getCommitHistory(); + if (commitHistory.isEmpty()) { + return Collections.emptyMap(); + } + + Set consumerGroups = commitHistory.stream().flatMap(c -> c.keySet().stream()) + .collect(Collectors.toSet()); + // verify that test setup is correct (this method only supports a single consumer group for now) + Truth.assertThat(consumerGroups).hasSize(1); + String consumerGroupName = consumerGroups.iterator().next(); + + // get the last committed offse for each partitions + Map result = new HashMap<>(); + for (Map> commit : getCommitHistory()) { + commit.getOrDefault(consumerGroupName, Collections.emptyMap()) + .forEach((partition, offsetAndMetadata) -> result.put(partition, offsetAndMetadata.offset())); + } + return result; + } + + private TestUserFunction createTestSetup(final CommitMode commitMode, final int maxConcurrency) { + // setup parallel consumer with custom processing function + setupParallelConsumerInstanceAndLogCapture(commitMode, maxConcurrency); + TestUserFunction testUserFunction = new TestUserFunction(); + parallelConsumer.poll(testUserFunction); + + // ensure that commit offset start at 0 -> otherwise there is a bug in the test setup + Truth.assertThat(getOverallCommittedOffset()).isEqualTo(0L); + + return testUserFunction; + } + + /** + * This test verifies that no new records are submitted to the workers once the consumer is paused. + * + * @param commitMode The commit mode to be configured for the parallel consumer. + */ @ParameterizedTest() @EnumSource(CommitMode.class) @SneakyThrows void pausingAndResumingProcessingShouldWork(final CommitMode commitMode) { int numTestRecordsPerBatch = 1_000; - // setup parallel consumer with custom processing function - setupParallelConsumerInstanceAndLogCapture(commitMode); - TestUserFunction testUserFunction = new TestUserFunction(); - parallelConsumer.poll(testUserFunction); + TestUserFunction testUserFunction = createTestSetup(commitMode, 3); // produce some messages addRecords(numTestRecordsPerBatch); @@ -219,7 +294,13 @@ void pausingAndResumingProcessingShouldWork(final CommitMode commitMode) { Awaitility .waitAtMost(5L, TimeUnit.SECONDS) .pollDelay(50L, TimeUnit.MILLISECONDS) + .alias(numTestRecordsPerBatch + " records should be processed") .until(testUserFunction.numProcessedRecords::get, numRecords -> numTestRecordsPerBatch == numRecords); + // overall committed offset should reach the same value + Awaitility + .waitAtMost(5L, TimeUnit.SECONDS) + .alias("sum of consumer offsets should reach " + numTestRecordsPerBatch) + .until(this::getOverallCommittedOffset, numRecords -> numTestRecordsPerBatch == numRecords); testUserFunction.reset(); // pause parallel consumer and wait for control loops to catch up @@ -232,15 +313,93 @@ void pausingAndResumingProcessingShouldWork(final CommitMode commitMode) { controlLoopTracker.waitForSomeControlLoopCycles(5, 5L, TimeUnit.SECONDS); // shouldn't have produced any records - BDDAssertions.assertThat(testUserFunction.numProcessedRecords).hasValue(0L); + Truth.assertThat(testUserFunction.numProcessedRecords.get()).isEqualTo(0L); + + // overall committed offset should stay at old value + Truth.assertThat(getOverallCommittedOffset()).isEqualTo(numTestRecordsPerBatch); // resume parallel consumer -> messages should be processed now parallelConsumer.resumeIfPaused(); Awaitility - .waitAtMost(5, TimeUnit.SECONDS) + .waitAtMost(5L, TimeUnit.SECONDS) + .alias(numTestRecordsPerBatch + " records should be processed") + .until(testUserFunction.numProcessedRecords::get, numRecords -> numTestRecordsPerBatch == numRecords); + // overall committed offset should reach the total of two batches that were processed + Awaitility + .waitAtMost(5L, TimeUnit.SECONDS) + .alias("sum of consumer offsets should reach " + 2L * numTestRecordsPerBatch) + .until(this::getOverallCommittedOffset, numRecords -> 2L * numTestRecordsPerBatch == numRecords); + testUserFunction.reset(); + } + + /** + * This test verifies that in flight work is finished successfully when the consumer is paused. In flight work is + * work that's currently being processed inside a user function has already been submitted to be processed based + * on the dynamic load factor. + * The test also verifies that new offsets are committed once the in-flight work finishes even if the consumer is + * still paused. + * + * @param commitMode The commit mode to be configured for the parallel consumer. + */ + @ParameterizedTest() + @EnumSource(CommitMode.class) + @SneakyThrows + void testThatInFlightWorkIsFinishedSuccessfullyAndOffsetsAreCommitted(final CommitMode commitMode) { + int degreeOfParallelism = 3; + int numTestRecordsPerBatch = 1_000; + + TestUserFunction testUserFunction = createTestSetup(commitMode, degreeOfParallelism); + // block processing in the user function to ensure we have in flight work once we pause the consumer + testUserFunction.lockProcessing(); + + // produce some messages + addRecords(numTestRecordsPerBatch); + + // wait until we have enough records in flight + Awaitility + .waitAtMost(5L, TimeUnit.SECONDS) + .pollDelay(50L, TimeUnit.MILLISECONDS) + .alias(degreeOfParallelism + " records should be in flight processed") + .until(testUserFunction.numInFlightRecords::get, numInFlightRecords -> degreeOfParallelism == numInFlightRecords); + + // overall committed consumer offset should still be at 0 + Truth.assertThat(getOverallCommittedOffset()).isEqualTo(0L); + + // pause parallel consumer and wait for control loops to catch up + parallelConsumer.pauseIfRunning(); + controlLoopTracker.waitForSomeParallelStreamProcessorControlLoopCycles(1, 5L, TimeUnit.SECONDS); + controlLoopTracker.waitForSomeBrokerPollSystemControlLoopCycles(1, 5L, TimeUnit.SECONDS); + + // unlock the user function + testUserFunction.unlockProcessing(); + + // in flight messages + buffered messages should get processed now (exact number is based on dynamic load factor) + Awaitility + .waitAtMost(5L, TimeUnit.SECONDS) + .pollDelay(50L, TimeUnit.MILLISECONDS) + .alias("at least " + degreeOfParallelism + " records should be processed") + .until(testUserFunction.numProcessedRecords::get, numRecords -> degreeOfParallelism <= numRecords); + // overall committed offset should reach the same value + Awaitility + .waitAtMost(5L, TimeUnit.SECONDS) + .alias("sum of consumer offsets should reach number of processed records") + .until(this::getOverallCommittedOffset, numRecords -> testUserFunction.numProcessedRecords.get() == numRecords); + // shouldn't have any more in flight records now + Truth.assertThat(testUserFunction.numInFlightRecords.get()).isEqualTo(0); + + // resume parallel consumer -> other pending messages should be processed now + parallelConsumer.resumeIfPaused(); + Awaitility + .waitAtMost(5L, TimeUnit.SECONDS) + .alias(numTestRecordsPerBatch + " records should be processed") .until(testUserFunction.numProcessedRecords::get, numRecords -> numTestRecordsPerBatch == numRecords); + // overall committed offset should reach the total number of processed records + Awaitility + .waitAtMost(5L, TimeUnit.SECONDS) + .alias("sum of consumer offsets should reach number of processed records") + .until(this::getOverallCommittedOffset, numRecords -> testUserFunction.numProcessedRecords.get() == numRecords); testUserFunction.reset(); } - + // TODO: Add also test for pausing under load } From dcc01bc1c3c0b64fcbe5d988f7c03af46830f948 Mon Sep 17 00:00:00 2001 From: Niels Date: Mon, 28 Feb 2022 12:02:22 +0100 Subject: [PATCH 36/54] Polish: Make MDC key for work container descriptor a constant. --- .../AbstractParallelEoSStreamProcessor.java | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 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 560238bf1..7ba3a3f1d 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 @@ -53,7 +53,12 @@ public abstract class AbstractParallelEoSStreamProcessor implements ParallelConsumer, ConsumerRebalanceListener, Closeable { public static final String MDC_INSTANCE_ID = "pcId"; - public static final String MDC_OFFSET_MARKER = "offset"; + + /** + * Key for the work container descriptor that will be added to the {@link MDC diagnostic context} while inside a + * user function. + */ + public static final String MDC_WORK_CONTAINER_DESCRIPTOR = "offset"; @Getter(PROTECTED) protected final ParallelConsumerOptions options; @@ -942,9 +947,9 @@ private void processWorkCompleteMailBox() { wm.registerWork(action.getConsumerRecords()); } else { WorkContainer work = action.getWorkContainer(); - MDC.put(MDC_OFFSET_MARKER, work.toString()); + MDC.put(MDC_WORK_CONTAINER_DESCRIPTOR, work.toString()); wm.handleFutureResult(work); - MDC.remove(MDC_OFFSET_MARKER); + MDC.remove(MDC_WORK_CONTAINER_DESCRIPTOR); } } } @@ -1088,7 +1093,7 @@ protected List, R>> runUserFunct try { if (log.isDebugEnabled()) { // first offset of the batch - MDC.put("offset", workContainerBatch.get(0).offset() + ""); + MDC.put(MDC_WORK_CONTAINER_DESCRIPTOR, workContainerBatch.get(0).offset() + ""); } log.trace("Pool received: {}", workContainerBatch); From 0d9a4fcf6ab70af0cdc527ddd4366b7a7f952b4e Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Wed, 2 Mar 2022 13:26:23 +0000 Subject: [PATCH 37/54] step: tweeks --- .../AbstractParallelEoSStreamProcessor.java | 11 +- ...llelEoSStreamProcessorPauseResumeTest.java | 294 ++++++------------ .../src/test/resources/logback-test.xml | 11 +- .../src/test/resources/logback-test.xml | 2 +- 4 files changed, 104 insertions(+), 214 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 7ba3a3f1d..2089158ad 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 @@ -663,7 +663,7 @@ private void controlLoop(Function, List> userFu log.trace("Loop: Process mailbox"); processWorkCompleteMailBox(); - if (state == running || state == paused) { + if (isIdlingOrRunning()) { // offsets will be committed when the consumer has its partitions revoked log.trace("Loop: Maybe commit"); commitOffsetsMaybe(); @@ -991,6 +991,11 @@ private Duration getTimeToBlockFor() { return effectiveCommitAttemptDelay; } + private boolean isIdlingOrRunning() { + return state == running || state == draining || state == paused; + } + + /** * Conditionally commit offsets to broker */ @@ -1052,7 +1057,7 @@ private boolean lingeringOnCommitWouldBeBeneficial() { private Duration getTimeToNextCommitCheck() { // draining is a normal running mode for the controller - if (state == running || state == draining || state == paused) { + if (isIdlingOrRunning()) { Duration timeSinceLastCommit = getTimeSinceLastCheck(); Duration timeBetweenCommits = getTimeBetweenCommits(); @SuppressWarnings("UnnecessaryLocalVariable") @@ -1072,6 +1077,7 @@ private Duration getTimeSinceLastCheck() { private void commitOffsetsThatAreReady() { log.debug("Committing offsets that are ready..."); synchronized (commitCommand) { + log.debug("Committing offsets that are ready..."); committer.retrieveOffsetsAndCommit(); clearCommitCommand(); this.lastCommitTime = Instant.now(); @@ -1234,6 +1240,7 @@ public void resumeIfPaused() { if (this.state == State.paused) { log.info("Transitioning paarallel consumer to state running."); this.state = State.running; + notifySomethingToDo(); } else { log.debug("Skipping transition of parallel consumer to state running. Current state is {}.", this.state); } diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorPauseResumeTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorPauseResumeTest.java index 171c7926e..5b07dcb7b 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorPauseResumeTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorPauseResumeTest.java @@ -4,53 +4,31 @@ * Copyright (C) 2020-2022 Confluent, Inc. */ -import static java.util.concurrent.TimeUnit.SECONDS; - -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.locks.ReentrantLock; -import java.util.function.Consumer; -import java.util.function.Supplier; -import java.util.stream.Collectors; - +import io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode; +import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import org.awaitility.Awaitility; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; -import org.slf4j.LoggerFactory; -import com.google.common.truth.Truth; +import java.util.*; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.ReentrantLock; +import java.util.function.Consumer; +import java.util.stream.Collectors; -import ch.qos.logback.classic.Level; -import ch.qos.logback.classic.Logger; -import ch.qos.logback.classic.LoggerContext; -import ch.qos.logback.classic.filter.LevelFilter; -import ch.qos.logback.classic.spi.ILoggingEvent; -import ch.qos.logback.core.AppenderBase; -import ch.qos.logback.core.spi.FilterReply; -import io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode; -import io.confluent.parallelconsumer.internal.AbstractParallelEoSStreamProcessor; -import io.confluent.parallelconsumer.internal.BrokerPollSystem; -import lombok.SneakyThrows; -import lombok.extern.slf4j.Slf4j; +import static com.google.common.truth.Truth.assertThat; /** * Test for pause/resume feature of the parallel consumer (see {@code GH#193}). * * @author niels.oertel */ -@Timeout(value = 10, unit = SECONDS) @Slf4j class ParallelEoSStreamProcessorPauseResumeTest extends ParallelEoSStreamProcessorTestBase { @@ -58,17 +36,14 @@ class ParallelEoSStreamProcessorPauseResumeTest extends ParallelEoSStreamProcess private static final AtomicLong RECORD_BATCH_KEY_GENERATOR = new AtomicLong(); - private ControlLoopTracker controlLoopTracker; - private static class TestUserFunction implements Consumer> { private final AtomicLong numProcessedRecords = new AtomicLong(); /** - * The number of in flight records. Note that this may not exactly match the - * real number of in flight records as parallel consumer has a wrapper around - * the user function so incrementing/decrementing the counter is a little bit - * delayed. + * The number of in flight records. Note that this may not exactly match the real number of in flight records as + * parallel consumer has a wrapper around the user function so incrementing/decrementing the counter is a little + * bit delayed. */ private final AtomicInteger numInFlightRecords = new AtomicInteger(); @@ -84,6 +59,7 @@ public void unlockProcessing() { @Override public void accept(ConsumerRecord t) { + log.debug("Received: {}", t); numInFlightRecords.incrementAndGet(); try { lockProcessing(); @@ -101,10 +77,10 @@ public void reset() { private ParallelConsumerOptions getBaseOptions(final CommitMode commitMode, int maxConcurrency) { return ParallelConsumerOptions.builder() - .commitMode(commitMode) - .consumer(consumerSpy) - .maxConcurrency(maxConcurrency) - .build(); + .commitMode(commitMode) + .consumer(consumerSpy) + .maxConcurrency(maxConcurrency) + .build(); } private void addRecords(final int numRecords) { @@ -113,98 +89,7 @@ private void addRecords(final int numRecords) { for (int i = 0; i < numRecords; ++i) { consumerSpy.addRecord(ktu.makeRecord("key-" + recordBatchKey + i, "v0-test-" + i)); } - } - - /** - * Log message appender that monitors the entry messages of the two control - * loops and counts how often they have been executed to build reliable tests - * that don't need to use {@link Thread#sleep(long)} to wait for changes to take - * effect. - *

    - * Note: {@link ParallelEoSStreamProcessorTestBase#waitForSomeLoopCycles(int)} - * provides similar functionality but not for the {@link BroerPollSystem}. - *

    - * - * @author niels.oertel - */ - public static class ControlLoopTracker extends AppenderBase { - - private static final AtomicLong ID_GENERATOR = new AtomicLong(); - - private static final String PSP_CONTROL_LOOP_MESSAGE = "Loop: Process mailbox"; - - private static final String BPS_CONTROL_LOOP_MESSAGE = "Loop: Broker poller: ({})"; - - private final String myInstanceId; - - private final AtomicLong pspControlLoopCounter = new AtomicLong(); - - private final AtomicLong bpsControlLoopCounter = new AtomicLong(); - - public ControlLoopTracker(final String myInstanceId) { - this.myInstanceId = myInstanceId; - this.setName("ControlLoopMonitor-" + ID_GENERATOR.incrementAndGet()); - } - - @Override - protected void append(final ILoggingEvent e) { - String myInstanceId = e.getMDCPropertyMap().get(AbstractParallelEoSStreamProcessor.MDC_INSTANCE_ID); - if (!this.myInstanceId.equals(myInstanceId)) { - // log message doesn't belong to the parallel consumer that is tracked - } else if (AbstractParallelEoSStreamProcessor.class.getName().equals(e.getLoggerName()) - && PSP_CONTROL_LOOP_MESSAGE.equals(e.getMessage())) { - // this is a control loop message from the parallel stream processor - pspControlLoopCounter.incrementAndGet(); - } else if (BrokerPollSystem.class.getName().equals(e.getLoggerName()) - && BPS_CONTROL_LOOP_MESSAGE.equals(e.getMessage())) { - // this is a control loop message from the broker poll system - bpsControlLoopCounter.incrementAndGet(); - } else { - // this is a message from the parallel consumer that is tracked but we're not - // interested in it - } - } - - public void reset() { - this.pspControlLoopCounter.set(0L); - this.bpsControlLoopCounter.set(0L); - } - - public void waitForSomeControlLoopCycles(int numCycles, long timeout, TimeUnit unit) { - long currentPspControlLoopCounter = pspControlLoopCounter.get(); - long currentBspControlLoopCounter = bpsControlLoopCounter.get(); - waitForSomeLoopCycles(currentPspControlLoopCounter, pspControlLoopCounter::get, numCycles, timeout, unit); - waitForSomeLoopCycles(currentBspControlLoopCounter, bpsControlLoopCounter::get, numCycles, timeout, unit); - } - - public void waitForSomeParallelStreamProcessorControlLoopCycles(int numCycles, long timeout, TimeUnit unit) { - waitForSomeLoopCycles(pspControlLoopCounter.get(), pspControlLoopCounter::get, numCycles, timeout, unit); - } - - public void waitForSomeBrokerPollSystemControlLoopCycles(int numCycles, long timeout, TimeUnit unit) { - waitForSomeLoopCycles(bpsControlLoopCounter.get(), bpsControlLoopCounter::get, numCycles, timeout, unit); - } - - private static void waitForSomeLoopCycles(long lastCount, Supplier currentCounter, int numCycles, - long timeout, TimeUnit unit) { - Awaitility.waitAtMost(timeout, unit).alias("Hello world: " + lastCount).until(currentCounter::get, - currentCount -> currentCount >= lastCount + numCycles); - } - } - - @AfterEach - void cleanup() { - if (null != controlLoopTracker) { - LoggerContext lc = (LoggerContext) LoggerFactory.getILoggerFactory(); - lc.getLoggerList().forEach(logger -> logger.detachAppender(controlLoopTracker.getName())); - controlLoopTracker.reset(); - controlLoopTracker.stop(); - controlLoopTracker = null; - } - - if (null != parallelConsumer) { - parallelConsumer.close(); - } + log.debug("Finished producing {} records with batch key {}.", numRecords, recordBatchKey); } private void setupParallelConsumerInstanceAndLogCapture(final CommitMode commitMode, final int maxConcurrency) { @@ -213,34 +98,14 @@ private void setupParallelConsumerInstanceAndLogCapture(final CommitMode commitM // register unique ID on the parallel consumer String myId = "p/r-test-" + MY_ID_GENERATOR.incrementAndGet(); parallelConsumer.setMyId(Optional.of(myId)); - - // setup the log capture to be able to follow the two loops - LoggerContext lc = (LoggerContext) LoggerFactory.getILoggerFactory(); - - // add filter to the STDOUT appender to ensure it stays on level INFO - LevelFilter levelInfoFilter = new LevelFilter(); - levelInfoFilter.setContext(lc); - levelInfoFilter.setLevel(Level.INFO); - levelInfoFilter.setOnMismatch(FilterReply.DENY); - levelInfoFilter.start(); - lc.getLogger(Logger.ROOT_LOGGER_NAME).getAppender("STDOUT").addFilter(levelInfoFilter); - - // create control loop tracker and register it as log appender - controlLoopTracker = new ControlLoopTracker(myId); - controlLoopTracker.setContext(lc); - Logger parallelStreamProcessorLogger = (Logger) LoggerFactory.getLogger(AbstractParallelEoSStreamProcessor.class); - parallelStreamProcessorLogger.setLevel(Level.TRACE); - parallelStreamProcessorLogger.addAppender(controlLoopTracker); - Logger brokerPollSystemLogger = (Logger) LoggerFactory.getLogger(BrokerPollSystem.class); - brokerPollSystemLogger.setLevel(Level.TRACE); - brokerPollSystemLogger.addAppender(controlLoopTracker); - controlLoopTracker.start(); } + // todo delete private long getOverallCommittedOffset() { return getCommittedOffsetsByPartitions().values().stream().collect(Collectors.summingLong(Long::longValue)); } + // todo delete private Map getCommittedOffsetsByPartitions() { List>> commitHistory = getCommitHistory(); if (commitHistory.isEmpty()) { @@ -250,7 +115,7 @@ private Map getCommittedOffsetsByPartitions() { Set consumerGroups = commitHistory.stream().flatMap(c -> c.keySet().stream()) .collect(Collectors.toSet()); // verify that test setup is correct (this method only supports a single consumer group for now) - Truth.assertThat(consumerGroups).hasSize(1); + assertThat(consumerGroups).hasSize(1); String consumerGroupName = consumerGroups.iterator().next(); // get the last committed offse for each partitions @@ -269,7 +134,7 @@ private TestUserFunction createTestSetup(final CommitMode commitMode, final int parallelConsumer.poll(testUserFunction); // ensure that commit offset start at 0 -> otherwise there is a bug in the test setup - Truth.assertThat(getOverallCommittedOffset()).isEqualTo(0L); + awaitForCommit(0); return testUserFunction; } @@ -284,60 +149,70 @@ private TestUserFunction createTestSetup(final CommitMode commitMode, final int @SneakyThrows void pausingAndResumingProcessingShouldWork(final CommitMode commitMode) { int numTestRecordsPerBatch = 1_000; + long totalRecordsExpected = 2L * numTestRecordsPerBatch; TestUserFunction testUserFunction = createTestSetup(commitMode, 3); // produce some messages addRecords(numTestRecordsPerBatch); +// ktu.sendRecords(numTestRecordsPerBatch); // wait for processing to finish Awaitility - .waitAtMost(5L, TimeUnit.SECONDS) - .pollDelay(50L, TimeUnit.MILLISECONDS) - .alias(numTestRecordsPerBatch + " records should be processed") - .until(testUserFunction.numProcessedRecords::get, numRecords -> numTestRecordsPerBatch == numRecords); + .waitAtMost(defaultTimeout) + .pollDelay(50L, TimeUnit.MILLISECONDS) + .alias(numTestRecordsPerBatch + " records should be processed") + .untilAsserted(() -> assertThat(testUserFunction.numProcessedRecords.get()).isEqualTo(numTestRecordsPerBatch)); +// .until(testUserFunction.numProcessedRecords::get, numRecords -> numRecords == numTestRecordsPerBatch); // overall committed offset should reach the same value Awaitility - .waitAtMost(5L, TimeUnit.SECONDS) - .alias("sum of consumer offsets should reach " + numTestRecordsPerBatch) - .until(this::getOverallCommittedOffset, numRecords -> numTestRecordsPerBatch == numRecords); + .waitAtMost(defaultTimeout) + .alias("sum of consumer offsets should reach " + numTestRecordsPerBatch) + .untilAsserted(() -> assertThat(getOverallCommittedOffset()).isEqualTo(numTestRecordsPerBatch)); + // .until(this::getOverallCommittedOffset, numRecords -> numTestRecordsPerBatch == numRecords); testUserFunction.reset(); // pause parallel consumer and wait for control loops to catch up parallelConsumer.pauseIfRunning(); - controlLoopTracker.waitForSomeParallelStreamProcessorControlLoopCycles(1, 5L, TimeUnit.SECONDS); - controlLoopTracker.waitForSomeBrokerPollSystemControlLoopCycles(1, 5L, TimeUnit.SECONDS); +// +// controlLoopTracker.waitForSomeParallelStreamProcessorControlLoopCycles(1, defaultTimeout); +// controlLoopTracker.waitForSomeBrokerPollSystemControlLoopCycles(1, defaultTimeout); + + awaitForOneLoopCycle(); // produce more messages -> nothing should be processed addRecords(numTestRecordsPerBatch); - controlLoopTracker.waitForSomeControlLoopCycles(5, 5L, TimeUnit.SECONDS); +// controlLoopTracker.waitForSomeControlLoopCycles(5, defaultTimeout); + awaitForSomeLoopCycles(2); // shouldn't have produced any records - Truth.assertThat(testUserFunction.numProcessedRecords.get()).isEqualTo(0L); + assertThat(testUserFunction.numProcessedRecords.get()).isEqualTo(0L); // overall committed offset should stay at old value - Truth.assertThat(getOverallCommittedOffset()).isEqualTo(numTestRecordsPerBatch); + assertThat(getOverallCommittedOffset()).isEqualTo(numTestRecordsPerBatch); - // resume parallel consumer -> messages should be processed now + // resume parallel consumer -> parallelConsumer.resumeIfPaused(); + + // messages should be processed now Awaitility - .waitAtMost(5L, TimeUnit.SECONDS) - .alias(numTestRecordsPerBatch + " records should be processed") - .until(testUserFunction.numProcessedRecords::get, numRecords -> numTestRecordsPerBatch == numRecords); + .waitAtMost(defaultTimeout) + .alias(numTestRecordsPerBatch + " records should be processed") + .untilAsserted(() -> assertThat(testUserFunction.numProcessedRecords.get()).isEqualTo(numTestRecordsPerBatch)); +// .until(testUserFunction.numProcessedRecords::get, numRecords -> numTestRecordsPerBatch == numRecords); // overall committed offset should reach the total of two batches that were processed Awaitility - .waitAtMost(5L, TimeUnit.SECONDS) - .alias("sum of consumer offsets should reach " + 2L * numTestRecordsPerBatch) - .until(this::getOverallCommittedOffset, numRecords -> 2L * numTestRecordsPerBatch == numRecords); - testUserFunction.reset(); + .waitAtMost(defaultTimeout) + .alias("sum of consumer offsets should reach " + totalRecordsExpected) +// .until(this::getOverallCommittedOffset, numRecords -> totalRecordsExpected == numRecords); + .untilAsserted(() -> assertThat(getOverallCommittedOffset()).isEqualTo(totalRecordsExpected)); } /** * This test verifies that in flight work is finished successfully when the consumer is paused. In flight work is - * work that's currently being processed inside a user function has already been submitted to be processed based - * on the dynamic load factor. - * The test also verifies that new offsets are committed once the in-flight work finishes even if the consumer is - * still paused. + * work that's currently being processed inside a user function has already been submitted to be processed based on + * the dynamic load factor. The test also verifies that new offsets are committed once the in-flight work finishes + * even if the consumer is still paused. * * @param commitMode The commit mode to be configured for the parallel consumer. */ @@ -357,49 +232,54 @@ void testThatInFlightWorkIsFinishedSuccessfullyAndOffsetsAreCommitted(final Comm // wait until we have enough records in flight Awaitility - .waitAtMost(5L, TimeUnit.SECONDS) - .pollDelay(50L, TimeUnit.MILLISECONDS) - .alias(degreeOfParallelism + " records should be in flight processed") - .until(testUserFunction.numInFlightRecords::get, numInFlightRecords -> degreeOfParallelism == numInFlightRecords); + .waitAtMost(defaultTimeout) + .pollDelay(50L, TimeUnit.MILLISECONDS) + .alias(degreeOfParallelism + " records should be in flight processed") + .until(testUserFunction.numInFlightRecords::get, numInFlightRecords -> degreeOfParallelism == numInFlightRecords); // overall committed consumer offset should still be at 0 - Truth.assertThat(getOverallCommittedOffset()).isEqualTo(0L); + assertThat(getOverallCommittedOffset()).isEqualTo(0L); // pause parallel consumer and wait for control loops to catch up parallelConsumer.pauseIfRunning(); - controlLoopTracker.waitForSomeParallelStreamProcessorControlLoopCycles(1, 5L, TimeUnit.SECONDS); - controlLoopTracker.waitForSomeBrokerPollSystemControlLoopCycles(1, 5L, TimeUnit.SECONDS); + awaitForOneLoopCycle(); +// controlLoopTracker.waitForSomeParallelStreamProcessorControlLoopCycles(1, defaultTimeout); +// controlLoopTracker.waitForSomeBrokerPollSystemControlLoopCycles(1, defaultTimeout); // unlock the user function testUserFunction.unlockProcessing(); // in flight messages + buffered messages should get processed now (exact number is based on dynamic load factor) Awaitility - .waitAtMost(5L, TimeUnit.SECONDS) - .pollDelay(50L, TimeUnit.MILLISECONDS) - .alias("at least " + degreeOfParallelism + " records should be processed") - .until(testUserFunction.numProcessedRecords::get, numRecords -> degreeOfParallelism <= numRecords); + .waitAtMost(defaultTimeout) + .pollDelay(50L, TimeUnit.MILLISECONDS) + .alias("at least " + degreeOfParallelism + " records should be processed") + .until(testUserFunction.numProcessedRecords::get, numRecords -> degreeOfParallelism <= numRecords); + // overall committed offset should reach the same value Awaitility - .waitAtMost(5L, TimeUnit.SECONDS) - .alias("sum of consumer offsets should reach number of processed records") - .until(this::getOverallCommittedOffset, numRecords -> testUserFunction.numProcessedRecords.get() == numRecords); - // shouldn't have any more in flight records now - Truth.assertThat(testUserFunction.numInFlightRecords.get()).isEqualTo(0); + .waitAtMost(defaultTimeout) + .alias("sum of consumer offsets should reach number of processed records") + .until(this::getOverallCommittedOffset, numRecords -> testUserFunction.numProcessedRecords.get() == numRecords); + + // shouldn't have anymore in flight records now + assertThat(testUserFunction.numInFlightRecords.get()).isEqualTo(0); + assertThat(parallelConsumer.getWm().getNumberRecordsOutForProcessing()).isEqualTo(0); - // resume parallel consumer -> other pending messages should be processed now + // resume parallel consumer -> parallelConsumer.resumeIfPaused(); + + // other pending messages should be processed now Awaitility - .waitAtMost(5L, TimeUnit.SECONDS) - .alias(numTestRecordsPerBatch + " records should be processed") - .until(testUserFunction.numProcessedRecords::get, numRecords -> numTestRecordsPerBatch == numRecords); + .waitAtMost(defaultTimeout) + .alias(numTestRecordsPerBatch + " records should be processed") + .until(testUserFunction.numProcessedRecords::get, numRecords -> numTestRecordsPerBatch == numRecords); // overall committed offset should reach the total number of processed records Awaitility - .waitAtMost(5L, TimeUnit.SECONDS) - .alias("sum of consumer offsets should reach number of processed records") - .until(this::getOverallCommittedOffset, numRecords -> testUserFunction.numProcessedRecords.get() == numRecords); + .waitAtMost(defaultTimeout) + .alias("sum of consumer offsets should reach number of processed records") + .until(this::getOverallCommittedOffset, numRecords -> testUserFunction.numProcessedRecords.get() == numRecords); testUserFunction.reset(); } - // TODO: Add also test for pausing under load } diff --git a/parallel-consumer-core/src/test/resources/logback-test.xml b/parallel-consumer-core/src/test/resources/logback-test.xml index a9345a2ed..c255b1b7d 100644 --- a/parallel-consumer-core/src/test/resources/logback-test.xml +++ b/parallel-consumer-core/src/test/resources/logback-test.xml @@ -21,15 +21,14 @@ - + - - - + + @@ -61,6 +60,10 @@ + + + + diff --git a/parallel-consumer-vertx/src/test/resources/logback-test.xml b/parallel-consumer-vertx/src/test/resources/logback-test.xml index e3e153db2..d8d564c02 100644 --- a/parallel-consumer-vertx/src/test/resources/logback-test.xml +++ b/parallel-consumer-vertx/src/test/resources/logback-test.xml @@ -3,7 +3,7 @@ Copyright (C) 2020-2022 Confluent, Inc. --> - + - - + + From 439ad74260419a7563a26b5eecb4d66f2e697905 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Mon, 14 Mar 2022 17:58:09 +0000 Subject: [PATCH 39/54] step: review --- .../parallelconsumer/ParallelConsumer.java | 21 ++++++++----------- .../src/test/resources/logback-test.xml | 4 ++-- 2 files changed, 11 insertions(+), 14 deletions(-) diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumer.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumer.java index bfcac16cc..218f77fe8 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumer.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumer.java @@ -8,9 +8,9 @@ import io.confluent.parallelconsumer.internal.DrainingCloseable; import lombok.Data; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.KafkaConsumer; import java.util.Collection; -import java.util.function.Consumer; import java.util.regex.Pattern; // tag::javadoc[] @@ -24,28 +24,27 @@ * @param key consume / produce key type * @param value consume / produce value type * @see AbstractParallelEoSStreamProcessor - * @see #poll(Consumer) */ // end::javadoc[] public interface ParallelConsumer extends DrainingCloseable { /** - * @see org.apache.kafka.clients.consumer.KafkaConsumer#subscribe(Collection) + * @see KafkaConsumer#subscribe(Collection) */ void subscribe(Collection topics); /** - * @see org.apache.kafka.clients.consumer.KafkaConsumer#subscribe(Pattern) + * @see KafkaConsumer#subscribe(Pattern) */ void subscribe(Pattern pattern); /** - * @see org.apache.kafka.clients.consumer.KafkaConsumer#subscribe(Collection, ConsumerRebalanceListener) + * @see KafkaConsumer#subscribe(Collection, ConsumerRebalanceListener) */ void subscribe(Collection topics, ConsumerRebalanceListener callback); /** - * @see org.apache.kafka.clients.consumer.KafkaConsumer#subscribe(Pattern, ConsumerRebalanceListener) + * @see KafkaConsumer#subscribe(Pattern, ConsumerRebalanceListener) */ void subscribe(Pattern pattern, ConsumerRebalanceListener callback); @@ -54,19 +53,17 @@ public interface ParallelConsumer extends DrainingCloseable { *

    * This operation only has an effect if the consumer is currently running. In all other cases calling this method * will be silent a no-op. - *

    *

    * Once the consumer is paused, the system will stop submitting work to the processing pool. Already submitted in - * flight work however will be finished. - * This includes work that is currently being processed inside a user function as well as work that has already - * been submitted to the processing pool but has not been picked up by a free worker yet. - *

    + * flight work however will be finished. This includes work that is currently being processed inside a user function + * as well as work that has already been submitted to the processing pool but has not been picked up by a free + * worker yet. *

    * General remarks: *

      *
    • A paused consumer may still keep polling for new work until internal buffers are filled.
    • *
    • This operation does not actively pause the subscription on the underlying Kafka Broker (compared to - * {@link org.apache.kafka.clients.consumer.KafkaConsumer#pause KafkaConsumer#pause}).
    • + * {@link KafkaConsumer#pause KafkaConsumer#pause}). *
    • Pending offset commits will still be performed when the consumer is paused.
    • *

      */ diff --git a/parallel-consumer-core/src/test/resources/logback-test.xml b/parallel-consumer-core/src/test/resources/logback-test.xml index c78aa105f..0e52a4aa3 100644 --- a/parallel-consumer-core/src/test/resources/logback-test.xml +++ b/parallel-consumer-core/src/test/resources/logback-test.xml @@ -60,8 +60,8 @@ - - + + From 753ea81496ed116526f026d71512415d3bf1bd57 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Mon, 14 Mar 2022 20:06:42 +0000 Subject: [PATCH 40/54] step: turn back up default timeout to 30 - 10 is too low for some things --- .../AbstractParallelEoSStreamProcessorTestBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java index 0c3a3117f..75115e9c4 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java @@ -82,7 +82,7 @@ public abstract class AbstractParallelEoSStreamProcessorTestBase { protected AbstractParallelEoSStreamProcessor parentParallelConsumer; - public static int defaultTimeoutSeconds = 10; + public static int defaultTimeoutSeconds = 30; public static Duration defaultTimeout = ofSeconds(defaultTimeoutSeconds); protected static long defaultTimeoutMs = defaultTimeout.toMillis(); From d7f997823709aa5c1524fa602d301550826b5685 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Wed, 23 Mar 2022 14:12:10 +0000 Subject: [PATCH 41/54] update --- .../ParallelEoSStreamProcessorPauseResumeTest.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorPauseResumeTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorPauseResumeTest.java index 478c8e7c5..4d3b0d157 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorPauseResumeTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorPauseResumeTest.java @@ -7,7 +7,6 @@ import io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; -import org.apache.kafka.clients.consumer.ConsumerRecord; import org.awaitility.Awaitility; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; @@ -31,7 +30,7 @@ class ParallelEoSStreamProcessorPauseResumeTest extends ParallelEoSStreamProcess private static final AtomicInteger RECORD_SET_KEY_GENERATOR = new AtomicInteger(); - private static class TestUserFunction implements Consumer> { + private static class TestUserFunction implements Consumer> { private final AtomicInteger numProcessedRecords = new AtomicInteger(); @@ -53,7 +52,7 @@ public void unlockProcessing() { } @Override - public void accept(ConsumerRecord t) { + public void accept(PollContext t) { log.debug("Received: {}", t); numInFlightRecords.incrementAndGet(); try { From 74eac432809e04add7d1d62e76a444ca582193d4 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Wed, 23 Mar 2022 14:19:02 +0000 Subject: [PATCH 42/54] review --- README.adoc | 2 +- .../internal/AbstractParallelEoSStreamProcessor.java | 2 +- parallel-consumer-core/src/test/resources/logback-test.xml | 2 +- src/docs/README.adoc | 1 + 4 files changed, 4 insertions(+), 3 deletions(-) diff --git a/README.adoc b/README.adoc index 80f4c58d9..b2779f31f 100644 --- a/README.adoc +++ b/README.adoc @@ -246,6 +246,7 @@ without operational burden or harming the cluster's performance * Java 8 compatibility * Throttle control and broker liveliness management * Clean draining shutdown cycle +* Manual Pause / Resume (pausing of partitions is also automatic, whenever back pressure has built up) //image:https://codecov.io/gh/astubbs/parallel-consumer/branch/master/graph/badge.svg["Coverage",https://codecov.io/gh/astubbs/parallel-consumer] //image:https://travis-ci.com/astubbs/parallel-consumer.svg?branch=master["Build Status", link="https://travis-ci.com/astubbs/parallel-consumer"] @@ -472,7 +473,6 @@ See {issues_link}/12[issue #12], and the `ParallelConsumer` JavaDoc: * @param key consume / produce key type * @param value consume / produce value type * @see AbstractParallelEoSStreamProcessor - * @see #poll(Consumer) */ ---- 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 2089158ad..78c642225 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 @@ -58,7 +58,7 @@ public abstract class AbstractParallelEoSStreamProcessor implements Parall * Key for the work container descriptor that will be added to the {@link MDC diagnostic context} while inside a * user function. */ - public static final String MDC_WORK_CONTAINER_DESCRIPTOR = "offset"; + private static final String MDC_WORK_CONTAINER_DESCRIPTOR = "offset"; @Getter(PROTECTED) protected final ParallelConsumerOptions options; diff --git a/parallel-consumer-core/src/test/resources/logback-test.xml b/parallel-consumer-core/src/test/resources/logback-test.xml index 0e52a4aa3..2c18baf75 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 @@ - + diff --git a/src/docs/README.adoc b/src/docs/README.adoc index 46de72b1c..acb88a12a 100644 --- a/src/docs/README.adoc +++ b/src/docs/README.adoc @@ -244,6 +244,7 @@ without operational burden or harming the cluster's performance * Java 8 compatibility * Throttle control and broker liveliness management * Clean draining shutdown cycle +* Manual Pause / Resume (pausing of partitions is also automatic, whenever back pressure has built up) //image:https://codecov.io/gh/astubbs/parallel-consumer/branch/master/graph/badge.svg["Coverage",https://codecov.io/gh/astubbs/parallel-consumer] //image:https://travis-ci.com/astubbs/parallel-consumer.svg?branch=master["Build Status", link="https://travis-ci.com/astubbs/parallel-consumer"] From 76c6fefa00925bc99203fb227f989b8fea3bfa28 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Fri, 25 Mar 2022 18:17:35 +0000 Subject: [PATCH 43/54] rebase compile update --- ...bstractParallelEoSStreamProcessorTestBase.java | 15 +++++++-------- .../src/test/resources/logback-test.xml | 2 +- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java index 75115e9c4..ec309bc50 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java @@ -33,7 +33,6 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; -import java.util.stream.Stream; import static io.confluent.csid.utils.LatchTestUtils.awaitLatch; import static io.confluent.csid.utils.StringUtils.msg; @@ -346,7 +345,7 @@ public List getCommitHistoryFlattened() { public List getCommitHistoryFlattenedMeta() { return (isUsingTransactionalProducer()) ? ktu.getProducerCommitsMeta(producerSpy) - : extractAllPartitionsOffsetsSequentiallyMeta(); + : extractAllPartitionsOffsetsSequentiallyMeta(true); } public void assertCommits(List offsets, String description) { @@ -382,7 +381,7 @@ public void assertCommits(List offsets, Optional description) { * Flattens the offsets of all partitions into a single sequential list */ protected List extractAllPartitionsOffsetsSequentially(boolean trimGenesis) { - return extractAllPartitionsOffsetsSequentiallyMeta().stream(). + return extractAllPartitionsOffsetsSequentiallyMeta(trimGenesis).stream(). map(x -> (int) x.offset()) // int cast a luxury in test context - no big offsets .collect(Collectors.toList()); } @@ -390,18 +389,18 @@ protected List extractAllPartitionsOffsetsSequentially(boolean trimGene /** * Flattens the offsets of all partitions into a single sequential list */ - protected List extractAllPartitionsOffsetsSequentiallyMeta() { + protected List extractAllPartitionsOffsetsSequentiallyMeta(boolean trimGenesis) { // copy the list for safe concurrent access List> history = new ArrayList<>(consumerSpy.getCommitHistoryInt()); return history.stream() .flatMap(commits -> { - Collection values = new ArrayList<>(commits.values()); // 4 debugging - Stream rawOffsets = values.stream().map(meta -> (int) meta.offset()); + var rawValues = new ArrayList<>(commits.values()).stream(); // 4 debugging +// Stream rawOffsets = values.stream().map(meta -> (int) meta.offset()); if (trimGenesis) - return rawOffsets.filter(x -> x != 0); + return rawValues.filter(x -> x.offset() != 0); else - return rawOffsets; // int cast a luxury in test context - no big offsets + return rawValues; // int cast a luxury in test context - no big offsets } ).collect(Collectors.toList()); } diff --git a/parallel-consumer-core/src/test/resources/logback-test.xml b/parallel-consumer-core/src/test/resources/logback-test.xml index 2c18baf75..70070d308 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 b85fd2daddf1f296321967c4a6f0579dc8670961 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Fri, 22 Apr 2022 10:13:47 +0100 Subject: [PATCH 44/54] review --- .../AbstractParallelEoSStreamProcessor.java | 17 +--- .../internal/BrokerPollSystem.java | 48 +---------- .../internal/CountingCRLinkedList.java | 54 ------------- ...ndRecords.java => EpochAndRecordsMap.java} | 4 +- .../state/PartitionMonitor.java | 81 ++++++++----------- .../parallelconsumer/state/WorkManager.java | 4 +- .../OffsetEncodingBackPressureTest.java | 45 +---------- .../OffsetEncodingBackPressureUnitTest.java | 4 +- .../offsets/OffsetEncodingTests.java | 6 +- .../state/WorkManagerTest.java | 6 +- 10 files changed, 56 insertions(+), 213 deletions(-) delete mode 100644 parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/CountingCRLinkedList.java rename parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/{EpochAndRecords.java => EpochAndRecordsMap.java} (93%) 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 5ec0343f0..816945ae3 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 @@ -107,7 +107,7 @@ public abstract class AbstractParallelEoSStreamProcessor implements Parall @RequiredArgsConstructor(access = PRIVATE) private static class ActionItem { WorkContainer workContainer; - EpochAndRecords consumerRecords; + EpochAndRecordsMap consumerRecords; private boolean isWorkResult() { return workContainer != null; @@ -117,7 +117,7 @@ private boolean isNewConsumerRecords() { return !isWorkResult(); } - private static ActionItem of(EpochAndRecords polledRecords) { + private static ActionItem of(EpochAndRecordsMap polledRecords) { return new ActionItem<>(null, polledRecords); } @@ -363,7 +363,6 @@ public void onPartitionsAssigned(Collection partitions) { @Override public void onPartitionsLost(Collection partitions) { numberOfAssignedPartitions = numberOfAssignedPartitions - partitions.size(); -// brokerPollSubsystem.onPartitionsLost(partitions); wm.onPartitionsLost(partitions); usersConsumerRebalanceListener.ifPresent(x -> x.onPartitionsLost(partitions)); } @@ -869,7 +868,6 @@ private boolean isPoolQueueLow() { int queueSize = getNumberOfUserFunctionsQueued(); int queueTarget = getPoolLoadTarget(); boolean workAmountBelowTarget = queueSize <= queueTarget; -// boolean hasWorkInMailboxes = wm.hasWorkAwaitingIngestionToShards(); log.debug("isPoolQueueLow()? workAmountBelowTarget {} {} vs {};", workAmountBelowTarget, queueSize, queueTarget); return workAmountBelowTarget; @@ -956,13 +954,6 @@ private void processWorkCompleteMailBox() { * @see ParallelConsumerOptions#getTargetAmountOfRecordsInFlight() */ private Duration getTimeToBlockFor() { -// // should not block as not enough work is being done, and there's more work to ingest -// boolean ingestionWorkAndStarved = wm.hasWorkAwaitingIngestionToShards() && wm.isStarvedForNewWork(); -// if (ingestionWorkAndStarved) { -// log.debug("Work waiting to be ingested, and not enough work in flight - will not block"); -// return Duration.ofMillis(0); -// } - // if less than target work already in flight, don't sleep longer than the next retry time for failed work, if it exists - so that we can wake up and maybe retry the failed work if (!wm.isWorkInFlightMeetingTarget()) { // though check if we have work awaiting retry @@ -1041,7 +1032,7 @@ private boolean lingeringOnCommitWouldBeBeneficial() { workIsWaitingToBeCompletedSuccessfully, workInFlight, workWaitingInMailbox, !workWaitingToCommit); boolean result = workIsWaitingToBeCompletedSuccessfully || workInFlight || workWaitingInMailbox || !workWaitingToCommit; - // disable - commit frequency takes care of lingering? is this outdated? + // todo disable - commit frequency takes care of lingering? is this outdated? return false; } @@ -1147,7 +1138,7 @@ protected void addToMailbox(WorkContainer wc) { workMailBox.add(ActionItem.of(wc)); } - public void registerWork(EpochAndRecords polledRecords) { + public void registerWork(EpochAndRecordsMap polledRecords) { log.debug("Adding {} to mailbox...", polledRecords); workMailBox.add(ActionItem.of(polledRecords)); } diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java index 908408e7f..3fc470eb7 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/BrokerPollSystem.java @@ -34,9 +34,7 @@ * @param */ @Slf4j -public class BrokerPollSystem implements OffsetCommitter -// , ConsumerRebalanceListener -{ +public class BrokerPollSystem implements OffsetCommitter { private final ConsumerManager consumerManager; @@ -61,13 +59,6 @@ public class BrokerPollSystem implements OffsetCommitter private final WorkManager wm; -// /** -// * Gets incremented every time there's a new assignment event. This epoch is forever associated with a record, and -// * is used to more easily determine stale records. -// */ -// @Getter(PRIVATE) -// private long partitionAssignmentEpoch = 0L; - public BrokerPollSystem(ConsumerManager consumerMgr, WorkManager wm, AbstractParallelEoSStreamProcessor pc, final ParallelConsumerOptions options) { this.wm = wm; this.pc = pc; @@ -148,29 +139,6 @@ private void handlePoll() { if (count > 0) { log.trace("Loop: Register work"); pc.registerWork(polledRecords); -// wm.registerWork(polledRecords); - -// // notify control work has been registered, in case it's sleeping waiting for work that will never come -// if (wm.isStarvedForNewWork()) { -// log.trace("Apparently no work is being done, make sure Control is awake to receive messages"); -// pc.notifySomethingToDo(); -// } - } - } - } - - // todo ? - private void transitionToCloseMaybe() { - // make sure everything is committed - if (isResponsibleForCommits() && !wm.isRecordsAwaitingToBeCommitted()) { - // transition to closing - state = State.closing; - } else { - log.trace("Draining, but work still needs to be committed. Yielding thread to avoid busy wait."); - try { - Thread.sleep(1); - } catch (InterruptedException e) { - e.printStackTrace(); } } } @@ -198,17 +166,7 @@ private boolean isResponsibleForCommits() { return committer.isPresent(); } -// @Override -// public void onPartitionsRevoked(Collection partitions) { -// partitionAssignmentEpoch++; -// } -// -// @Override -// public void onPartitionsAssigned(Collection partitions) { -// partitionAssignmentEpoch++; -// } - - private EpochAndRecords pollBrokerForRecords() { + private EpochAndRecordsMap pollBrokerForRecords() { managePauseOfSubscription(); log.debug("Subscriptions are paused: {}", paused); @@ -222,7 +180,7 @@ private EpochAndRecords pollBrokerForRecords() { log.debug("Poll completed"); // build records map - return new EpochAndRecords<>(poll, wm.getPm()); + return new EpochAndRecordsMap<>(poll, wm.getPm()); } /** diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/CountingCRLinkedList.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/CountingCRLinkedList.java deleted file mode 100644 index 6fd12fa1c..000000000 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/CountingCRLinkedList.java +++ /dev/null @@ -1,54 +0,0 @@ -package io.confluent.parallelconsumer.internal; - -/*- - * Copyright (C) 2020-2022 Confluent, Inc. - */ - -//import lombok.EqualsAndHashCode; -//import lombok.Getter; -//import org.apache.kafka.clients.consumer.ConsumerRecord; -//import org.apache.kafka.clients.consumer.ConsumerRecords; -// -//import java.util.LinkedList; -//import java.util.Queue; -// -///** -// * Cached view of the nested number of records in this queue. -// *

      -// * Also protects against concurrent modification exceptions, as we don't need to traverse the list to count the nested -// * elements. The count won't then be always exact, but it doesn't need to be. -// */ -//// todo used? -//@EqualsAndHashCode(callSuper = true) -//public class CountingCRLinkedList extends LinkedList> implements Queue> { -// -// /** -// * The number of nested {@link ConsumerRecord} in this collection. As this is a non blocking collection, this won't -// * be exact. -// */ -// @Getter -// private int nestedCount = 0; -// -// @Override -// public void add(final int index, final ConsumerRecords element) { -// nestedCount = nestedCount + element.count(); -// super.add(index, element); -// } -// -// @Override -// public boolean add(final ConsumerRecords element) { -// nestedCount = nestedCount + element.count(); -// return super.add(element); -// } -// -// @Override -// public ConsumerRecords poll() { -// ConsumerRecords poll = super.poll(); -// if (poll != null) { -// int numberOfNestedMessages = poll.count(); -// nestedCount = nestedCount - numberOfNestedMessages; -// } -// return poll; -// } -// -//} diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecords.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecordsMap.java similarity index 93% rename from parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecords.java rename to parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecordsMap.java index 7575ab3a5..95d6e3f7e 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecords.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/internal/EpochAndRecordsMap.java @@ -18,11 +18,11 @@ * @see BrokerPollSystem#partitionAssignmentEpoch */ @Value -public class EpochAndRecords { +public class EpochAndRecordsMap { Map recordMap = new HashMap<>(); - public EpochAndRecords(ConsumerRecords poll, PartitionMonitor pm) { + public EpochAndRecordsMap(ConsumerRecords poll, PartitionMonitor pm) { poll.partitions().forEach(partition -> { var records = poll.records(partition); Long epochOfPartition = pm.getEpochOfPartition(partition); diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java index d920a0682..b3468903a 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java @@ -8,7 +8,7 @@ import io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder; import io.confluent.parallelconsumer.internal.AbstractParallelEoSStreamProcessor; import io.confluent.parallelconsumer.internal.BrokerPollSystem; -import io.confluent.parallelconsumer.internal.EpochAndRecords; +import io.confluent.parallelconsumer.internal.EpochAndRecordsMap; import io.confluent.parallelconsumer.internal.InternalRuntimeError; import io.confluent.parallelconsumer.offsets.OffsetMapCodecManager; import lombok.Getter; @@ -36,7 +36,7 @@ */ @Slf4j @RequiredArgsConstructor -// todo rename to partition manager +// todo rename to partition state manager public class PartitionMonitor implements ConsumerRebalanceListener { public static final double USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT = 0.75; @@ -75,10 +75,7 @@ public class PartitionMonitor implements ConsumerRebalanceListener { private final Clock clock; public PartitionState getPartitionState(TopicPartition tp) { - // by locking on partitionState, may cause the system to wait for a rebalance to finish -// synchronized (partitionStates) { return partitionStates.get(tp); -// } } /** @@ -87,7 +84,6 @@ public PartitionState getPartitionState(TopicPartition tp) { @Override public void onPartitionsAssigned(Collection assignedPartitions) { log.debug("Partitions assigned: {}", assignedPartitions); -// synchronized (this.partitionStates) { for (final TopicPartition partitionAssignment : assignedPartitions) { boolean isAlreadyAssigned = this.partitionStates.containsKey(partitionAssignment); @@ -113,8 +109,6 @@ public void onPartitionsAssigned(Collection assignedPartitions) log.error("Error in onPartitionsAssigned", e); throw e; } - -// } } /** @@ -137,10 +131,8 @@ public void onPartitionsRevoked(Collection partitions) { } void onPartitionsRemoved(final Collection partitions) { -// synchronized (this.partitionStates) { incrementPartitionAssignmentEpoch(partitions); resetOffsetMapAndRemoveWork(partitions); -// } } /** @@ -338,45 +330,40 @@ public void onFailure(WorkContainer wc) { * Takes a record as work and puts it into internal queues, unless it's been previously recorded as completed as per * loaded records. */ - void maybeRegisterNewRecordAsWork(final EpochAndRecords recordsCollection) { -// if (records == null) return false; -// /** -// * Locking on partition state here, means that the check for partition assignment (that the source partition of the -// * work is still assigned) is in the same sync block as registering the work with the {@link TopicPartition}'s -// * {@link PartitionState} and the {@link ShardManager}. Keeping the two different views in sync. Of course now, -// * having a shared nothing architecture would mean all access to the state is by a single thread, and so this could -// * never occur. -// */ -// synchronized (partitionStates) { - - for (var partition : recordsCollection.partitions()) { - var records = recordsCollection.records(partition); - var epochOfInboundRecords = records.getEpochOfPartitionAtPoll(); - for (var rec : records.getRecords()) { - - // 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)) { - - if (isPartitionRemovedOrNeverAssigned(rec)) { - log.debug("Record in buffer for a partition no longer assigned. Dropping. TP: {} rec: {}", toTopicPartition(rec), rec); - } - - if (isRecordPreviouslyCompleted(rec)) { - log.trace("Record previously completed, skipping. offset: {}", rec.offset()); - } else { - var work = new WorkContainer<>(epochOfInboundRecords, rec, options.getRetryDelayProvider(), clock); - - 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); - } + void maybeRegisterNewRecordAsWork(final EpochAndRecordsMap recordsMap) { + for (var partition : recordsMap.partitions()) { + var recordsList = recordsMap.records(partition); + var epochOfInboundRecords = recordsList.getEpochOfPartitionAtPoll(); + for (var rec : recordsList.getRecords()) { + maybeRegisterNewRecordAsWork(epochOfInboundRecords, rec); + } + } + } + + /** + * @see #maybeRegisterNewRecordAsWork(EpochAndRecordsMap) + */ + private void maybeRegisterNewRecordAsWork(Long epochOfInboundRecords, 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)) { + + if (isPartitionRemovedOrNeverAssigned(rec)) { + log.debug("Record in buffer for a partition no longer assigned. Dropping. TP: {} rec: {}", toTopicPartition(rec), rec); } + + if (isRecordPreviouslyCompleted(rec)) { + log.trace("Record previously completed, skipping. offset: {}", rec.offset()); + } else { + var work = new WorkContainer<>(epochOfInboundRecords, rec, options.getRetryDelayProvider(), clock); + + 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); } -// } } public Map collectDirtyCommitData() { diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java index d2ad4700a..3cfba685a 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java @@ -9,7 +9,7 @@ import io.confluent.parallelconsumer.internal.AbstractParallelEoSStreamProcessor; import io.confluent.parallelconsumer.internal.BrokerPollSystem; import io.confluent.parallelconsumer.internal.DynamicLoadFactor; -import io.confluent.parallelconsumer.internal.EpochAndRecords; +import io.confluent.parallelconsumer.internal.EpochAndRecordsMap; import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; @@ -136,7 +136,7 @@ void onPartitionsRemoved(final Collection partitions) { // registerWork(new EpochAndRecords(records, 0)); // } - public void registerWork(EpochAndRecords records) { + public void registerWork(EpochAndRecordsMap records) { // wmbm.registerWork(records); pm.maybeRegisterNewRecordAsWork(records); } diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java index 001ae24c0..57ac406ec 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java @@ -143,17 +143,7 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O //, () -> parallelConsumer.getFailureCause()) // requires https://github.com/awaitility/awaitility/issues/178#issuecomment-734769761 .pollInterval(1, SECONDS) .untilAsserted(() -> { -// ProcessingShard stringStringProcessingShard = sm.getProcessingShards().get(topicPartition); -// if (stringStringProcessingShard != null) { -// long countOfWorkAwaitingSelection = stringStringProcessingShard.getCountOfWorkAwaitingSelection(); -// NavigableMap> entries = stringStringProcessingShard.getEntries(); -// boolean b = sm.workIsWaitingToBeProcessed(); -// long countWorkInFlight = stringStringProcessingShard.getCountWorkInFlight(); -// long countOfWorkTracked = stringStringProcessingShard.getCountOfWorkTracked(); -// long numberOfWorkQueuedInShardsAwaitingSelection = sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); -// } assertThat(userFuncFinishedCount.get()).isEqualTo(numberOfRecords - numberOfBlockedMessages); -// Truth.assertThat(numberOfWorkQueuedInShardsAwaitingSelection).isEqualTo(-4); }); // # assert commit ok - nothing blocked @@ -202,23 +192,9 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O log.debug("// assert partition now blocked from threshold"); waitAtMost(ofSeconds(10)) .untilAsserted( - () -> { - // old -// assertThat(wm.getPm().isBlocked(topicPartition)) -// .as("Partition SHOULD be blocked due to back pressure") -// .isTrue(); - - // new - assertWithMessage("Partition SHOULD be blocked due to back pressure") - .that(partitionState) - .isBlocked(); - - assertWithMessage("Partition SHOULD be blocked due to back pressure") - .that(partitionState) - .isNotAllowedMoreRecords(); - - // - }); // blocked + () -> assertWithMessage("Partition SHOULD be blocked due to back pressure") + .that(partitionState) + .isBlocked()); // blocked Long partitionOffsetHighWaterMarks = wm.getPm().getHighestSeenOffset(topicPartition); assertThat(partitionOffsetHighWaterMarks) @@ -248,10 +224,7 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O // recreates the situation where the payload size is too large and must be dropped log.debug("// test max payload exceeded, payload dropped"); - -// log.debug("// messages already sent {}, sending {} more", processedBeforePartitionBlock, extraMessages); { - long numberOfWorkQueuedInShardsAwaitingSelection = sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); log.debug("// force system to allow more records (i.e. the actual system attempts to never allow the payload to grow this big)"); PartitionMonitor.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(30); parallelConsumer.requestCommitAsap(); @@ -273,14 +246,6 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O log.debug("// wait for the new message to be processed"); await().atMost(defaultTimeout).untilAsserted(() -> { -// long numberOfWorkQueuedInShardsAwaitingSelection1 = sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); -// ShardManager sm1 = sm; -// List seen1 = seen.stream().sorted().collect(Collectors.toList()); -// long offsetHighestSucceeded = partitionState.getOffsetHighestSucceeded(); -// long offsetHighestSeen = partitionState.getOffsetHighestSeen(); -// long offsetHighestSequentialSucceeded = partitionState.getOffsetHighestSequentialSucceeded(); -// int i = userFuncStartCount.get(); -// int i1 = userFuncFinishedCount.get(); int expectedUserFunctionFinishedCount = processedBeforePartitionBlock + extraMessages + 1; assertThat(userFuncFinishedCount.get()).isEqualTo(expectedUserFunctionFinishedCount); } @@ -331,10 +296,6 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O await().untilAsserted(() -> assertThat(wm.getPm().isAllowedMoreRecords(topicPartition)).isTrue()); } } finally { - // make sure to unlock threads - speeds up failed tests, instead of waiting for latch or close timeouts -// msgLock.countDown(); -// msgLockTwo.countDown(); - // todo restore static defaults - lazy way to override settings at runtime but causes bugs by allowing them to be statically changeable OffsetMapCodecManager.DefaultMaxMetadataSize = realMax; // todo wow this is smelly, but convenient OffsetMapCodecManager.forcedCodec = Optional.empty(); diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java index 47675a31f..014f9eaf8 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java @@ -6,7 +6,7 @@ import com.google.common.truth.Truth; import io.confluent.parallelconsumer.ParallelEoSStreamProcessorTestBase; -import io.confluent.parallelconsumer.internal.EpochAndRecords; +import io.confluent.parallelconsumer.internal.EpochAndRecordsMap; import io.confluent.parallelconsumer.state.PartitionMonitor; import io.confluent.parallelconsumer.state.PartitionState; import io.confluent.parallelconsumer.state.WorkContainer; @@ -214,7 +214,7 @@ private WorkContainer findWC(List> private void sendRecordsToWM(int numberOfRecords, WorkManager wm) { log.debug("~Sending {} more records", numberOfRecords); List> records = ktu.generateRecords(numberOfRecords); - wm.registerWork(new EpochAndRecords<>(new ConsumerRecords<>(UniMaps.of(topicPartition, records)), wm.getPm())); + wm.registerWork(new EpochAndRecordsMap<>(new ConsumerRecords<>(UniMaps.of(topicPartition, records)), wm.getPm())); Truth.assertThat(wm.getNumberOfWorkQueuedInShardsAwaitingSelection()).isEqualTo(numberOfRecords); } 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 5c2358d54..2ae448d58 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 @@ -8,7 +8,7 @@ import io.confluent.csid.utils.KafkaTestUtils; import io.confluent.parallelconsumer.ParallelConsumerOptions; import io.confluent.parallelconsumer.ParallelEoSStreamProcessorTestBase; -import io.confluent.parallelconsumer.internal.EpochAndRecords; +import io.confluent.parallelconsumer.internal.EpochAndRecordsMap; import io.confluent.parallelconsumer.state.WorkContainer; import io.confluent.parallelconsumer.state.WorkManager; import lombok.SneakyThrows; @@ -179,7 +179,7 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset { WorkManager wmm = new WorkManager<>(options, consumerSpy); wmm.onPartitionsAssigned(UniSets.of(new TopicPartition(INPUT_TOPIC, 0))); - wmm.registerWork(new EpochAndRecords<>(testRecords, wmm.getPm())); + wmm.registerWork(new EpochAndRecordsMap<>(testRecords, wmm.getPm())); List> work = wmm.getWorkIfAvailable(); assertThat(work).hasSameSizeAs(records); @@ -224,7 +224,7 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset { var newWm = new WorkManager<>(options, consumerSpy); newWm.onPartitionsAssigned(UniSets.of(tp)); - newWm.registerWork(new EpochAndRecords(testRecords, newWm.getPm())); + newWm.registerWork(new EpochAndRecordsMap(testRecords, newWm.getPm())); var pm = newWm.getPm(); var partitionState = pm.getPartitionState(tp); diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java index 4a6926def..afb6ff16c 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/state/WorkManagerTest.java @@ -11,7 +11,7 @@ import io.confluent.parallelconsumer.FakeRuntimeError; import io.confluent.parallelconsumer.ManagedTruth; import io.confluent.parallelconsumer.ParallelConsumerOptions; -import io.confluent.parallelconsumer.internal.EpochAndRecords; +import io.confluent.parallelconsumer.internal.EpochAndRecordsMap; import io.confluent.parallelconsumer.truth.CommitHistorySubject; import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -104,7 +104,7 @@ private void registerSomeWork(int partition) { Map>> m = new HashMap<>(); m.put(topicPartitionOf(partition), of(rec0, rec1, rec2)); var recs = new ConsumerRecords<>(m); - wm.registerWork(new EpochAndRecords(recs, wm.getPm())); + wm.registerWork(new EpochAndRecordsMap(recs, wm.getPm())); } private ConsumerRecord makeRec(String value, String key, int partition) { @@ -393,7 +393,7 @@ void insertWrongOrderPreservesOffsetOrdering() { } private void registerWork(ConsumerRecords recs) { - wm.registerWork(new EpochAndRecords<>(recs, wm.getPm())); + wm.registerWork(new EpochAndRecordsMap<>(recs, wm.getPm())); } From c6056fe8970b789179ccc0b92d5b364cbd4896f4 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Fri, 22 Apr 2022 10:39:21 +0100 Subject: [PATCH 45/54] review --- .../AbstractParallelEoSStreamProcessor.java | 5 + .../state/PartitionMonitor.java | 2 + .../state/PartitionState.java | 16 +-- .../parallelconsumer/state/ShardManager.java | 3 +- .../state/WorkMailBoxManager.java | 136 ------------------ .../parallelconsumer/state/WorkManager.java | 120 +--------------- .../LargeVolumeInMemoryTests.java | 4 +- .../offsets/OffsetEncodingTests.java | 19 +-- 8 files changed, 28 insertions(+), 277 deletions(-) delete mode 100644 parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkMailBoxManager.java 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 816945ae3..e65694324 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 @@ -522,7 +522,12 @@ private boolean isResponsibleForCommits() { /** * Block the calling thread until no more messages are being processed. + *

      + * Used for testing. + * + * @deprecated no longer used, will be removed in next version */ + @Deprecated(forRemoval = true) @SneakyThrows public void waitForProcessedNotCommitted(Duration timeout) { log.debug("Waiting processed but not committed..."); diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java index b3468903a..a5af59759 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java @@ -33,6 +33,8 @@ * In charge of managing {@link PartitionState}s. *

      * This state is shared between the {@link BrokerPollSystem} thread and the {@link AbstractParallelEoSStreamProcessor}. + * + * @see PartitionState */ @Slf4j @RequiredArgsConstructor 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 026097658..59a2df993 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 @@ -32,6 +32,12 @@ @Slf4j public class PartitionState { + /** + * Symbolic value for a parameter which is initialised as having an offset absent (instead of using Optional or + * null) + */ + public static final long KAFKA_OFFSET_ABSENCE = -1L; + @Getter private final TopicPartition tp; @@ -73,7 +79,7 @@ public class PartitionState { * Highest offset which has completed successfully ("succeeded"). */ @Getter(PUBLIC) - private long offsetHighestSucceeded = -1L; + private long offsetHighestSucceeded = KAFKA_OFFSET_ABSENCE; /** * If true, more messages are allowed to process for this partition. @@ -111,7 +117,7 @@ private NavigableMap> getCommitQueue() { public PartitionState(TopicPartition tp, OffsetMapCodecManager.HighestOffsetAndIncompletes offsetData) { this.tp = tp; - this.offsetHighestSeen = offsetData.getHighestSeenOffset().orElse(-1L); + this.offsetHighestSeen = offsetData.getHighestSeenOffset().orElse(KAFKA_OFFSET_ABSENCE); this.incompleteOffsets = new ConcurrentSkipListSet<>(offsetData.getIncompleteOffsets()); this.offsetHighestSucceeded = this.offsetHighestSeen; } @@ -186,12 +192,6 @@ private void updateHighestSucceededOffsetSoFar(WorkContainer work) { } public void addWorkContainer(WorkContainer wc) { - // for test debugging -// long offsetHighestSeen = getOffsetHighestSeen(); -// if (wc.offset() != offsetHighestSeen + 1) { -// log.error(""); -// } - maybeRaiseHighestSeenOffset(wc.offset()); commitQueue.put(wc.offset(), wc); incompleteOffsets.add(wc.offset()); diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ShardManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ShardManager.java index f8fc4d038..215494c6e 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ShardManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ShardManager.java @@ -57,8 +57,7 @@ public class ShardManager { * @see K * @see WorkManager#getWorkIfAvailable() */ - // todo performance: disable/remove if using partition order -// @Getter + // performance: could disable/remove if using partition order - but probably not worth the added complexity in the code to handle an extra special case private final Map> processingShards = new ConcurrentHashMap<>(); private final NavigableSet> retryQueue = new TreeSet<>(Comparator.comparing(wc -> wc.getDelayUntilRetryDue())); diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkMailBoxManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkMailBoxManager.java deleted file mode 100644 index 4ef00bb9a..000000000 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkMailBoxManager.java +++ /dev/null @@ -1,136 +0,0 @@ -package io.confluent.parallelconsumer.state; - -/*- - * Copyright (C) 2020-2022 Confluent, Inc. - */ -//import io.confluent.parallelconsumer.internal.BrokerPollSystem; -//import io.confluent.parallelconsumer.internal.CountingCRLinkedList; -//import lombok.extern.slf4j.Slf4j; -//import org.apache.kafka.clients.consumer.ConsumerRecord; -//import org.apache.kafka.clients.consumer.ConsumerRecords; -//import org.apache.kafka.common.TopicPartition; -// -//import java.util.Collection; -//import java.util.LinkedList; -//import java.util.Queue; -//import java.util.concurrent.LinkedBlockingQueue; -// -//import static io.confluent.csid.utils.KafkaUtils.toTopicPartition; -// -///** -// * Handles the incoming mail for {@link WorkManager}. -// */ -//@Slf4j -//public class WorkMailBoxManager { -// -// /** -// * The number of nested {@link ConsumerRecord} entries in the shared blocking mail box. Cached for performance. -// */ -// private int sharedBoxNestedRecordCount; -// -// /** -// * The shared mailbox. Doesn't need to be thread safe as we already need synchronize on it. -// */ -// private final LinkedBlockingQueue> workInbox = new LinkedBlockingQueue<>(); -// -// /** -// * Mailbox where mail is transferred to immediately. -// */ -// private final CountingCRLinkedList internalBatchMailQueue = new CountingCRLinkedList<>(); -// -// /** -// * Queue of records flattened from the {@link #internalBatchMailQueue}. -// *

      -// * This is needed because {@link java.util.concurrent.BlockingQueue#drainTo(Collection)} must drain to a collection -// * of the same type. We could have {@link BrokerPollSystem} do the flattening, but that would require many calls to -// * the Concurrent queue, where this only needs one. Also as we don't expect there to be that many elements in these -// * collections (as they contain large batches of records), the overhead will be small. -// */ -// private final Queue> internalFlattenedMailQueue = new LinkedList<>(); -// -// /** -// * @return amount of work queued in the mail box, awaiting processing into shards, not exact -// */ -// Integer getAmountOfWorkQueuedWaitingIngestion() { -// return sharedBoxNestedRecordCount + -// internalBatchMailQueue.getNestedCount() + -// internalFlattenedMailQueue.size(); -// } -// -// /** -// * Work must be registered in offset order -// *

      -// * Thread safe for use by control and broker poller thread. -// * -// * @see WorkManager#onSuccessResult -// * @see WorkManager#raisePartitionHighWaterMark -// */ -// public void registerWork(final ConsumerRecords records) { -// synchronized (workInbox) { -// sharedBoxNestedRecordCount += records.count(); -// workInbox.add(records); -// } -// } -// -// -// /** -// * Must synchronise to keep sharedBoxNestedRecordCount in lock step with the inbox. Register is easy, but drain you -// * need to run through an intermediary collection and then count the nested elements, to know how many to subtract -// * from the Atomic nested count. -// *

      -// * Plus registering work is relatively infrequent, so shouldn't worry about a little synchronized here - makes it -// * much simpler. -// */ -// private void drainSharedMailbox() { -// synchronized (workInbox) { -// workInbox.drainTo(internalBatchMailQueue); -// sharedBoxNestedRecordCount = 0; -// } -// } -// -// /** -// * Take our inbound messages from the {@link BrokerPollSystem} and add them to our registry. -// */ -// private synchronized void flattenBatchQueue() { -// drainSharedMailbox(); -// -// // flatten -// while (!internalBatchMailQueue.isEmpty()) { -// ConsumerRecords consumerRecords = internalBatchMailQueue.poll(); -// log.debug("Flattening {} records", consumerRecords.count()); -// for (final ConsumerRecord consumerRecord : consumerRecords) { -// internalFlattenedMailQueue.add(consumerRecord); -// } -// } -// } -// -// /** -// * Remove revoked work from the mailbox -// */ -// public synchronized void onPartitionsRemoved(final Collection removedPartitions) { -// log.debug("Removing stale work from inbox queues"); -// flattenBatchQueue(); -// internalFlattenedMailQueue.removeIf(rec -> -// removedPartitions.contains(toTopicPartition(rec)) -// ); -// } -// -// public synchronized boolean internalFlattenedMailQueueIsEmpty() { -// return internalFlattenedMailQueue.isEmpty(); -// } -// -// /** -// * @return the next element in our outbound queue, or null if empty -// */ -// public synchronized ConsumerRecord internalFlattenedMailQueuePoll() { -// if (internalBatchMailQueue.isEmpty()) { -// // flatten the batch queue in batches when needed -// flattenBatchQueue(); -// } -// return internalFlattenedMailQueue.poll(); -// } -// -// public int internalFlattenedMailQueueSize() { -// return internalFlattenedMailQueue.size(); -// } -//} diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java index 3cfba685a..278fd1cce 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java @@ -44,7 +44,7 @@ public class WorkManager implements ConsumerRebalanceListener { @Getter - private final ParallelConsumerOptions options; + private final ParallelConsumerOptions options; // todo rename PSM, PartitionStateManager // todo make private @@ -63,8 +63,6 @@ public class WorkManager implements ConsumerRebalanceListener { */ private final DynamicLoadFactor dynamicLoadFactor; -// private final WorkMailBoxManager wmbm; - @Getter private int numberRecordsOutForProcessing = 0; @@ -89,7 +87,6 @@ public WorkManager(final ParallelConsumerOptions newOptions, final org.apa final DynamicLoadFactor dynamicExtraLoadFactor, Clock clock) { this.options = newOptions; this.dynamicLoadFactor = dynamicExtraLoadFactor; -// this.wmbm = new WorkMailBoxManager<>(); this.sm = new ShardManager<>(options, this, clock); this.pm = new PartitionMonitor<>(consumer, sm, options, clock); } @@ -126,73 +123,12 @@ public void onPartitionsLost(Collection partitions) { void onPartitionsRemoved(final Collection partitions) { // no-op - nothing to do -// wmbm.onPartitionsRemoved(partitions); } -// /** -// * Hard codes epoch as genesis - for testing only -// */ -// public void registerWork(ConsumerRecords records) { -// registerWork(new EpochAndRecords(records, 0)); -// } - public void registerWork(EpochAndRecordsMap records) { -// wmbm.registerWork(records); pm.maybeRegisterNewRecordAsWork(records); } -// /** -// * Moves the requested amount of work from initial queues into work queues, if available. -// * -// * @param requestedMaxWorkToRetrieve try to move at least this many messages into the inbound queues -// * @return the number of extra records ingested due to request -// */ -// private int ingestPolledRecordsIntoQueues(long requestedMaxWorkToRetrieve) { -// log.debug("Will attempt to register the requested {} - {} available in internal mailbox", -// requestedMaxWorkToRetrieve, wmbm.internalFlattenedMailQueueSize()); -// -// // -// var takenWorkCount = 0; -// boolean continueIngesting; -// do { -// ConsumerRecord polledRecord = wmbm.internalFlattenedMailQueuePoll(); -// boolean recordAddedAsWork = pm.maybeRegisterNewRecordAsWork(polledRecord); -// if (recordAddedAsWork) { -// takenWorkCount++; -// } -// boolean polledQueueNotExhausted = polledRecord != null; -// boolean ingestTargetNotSatisfied = takenWorkCount < requestedMaxWorkToRetrieve; -// continueIngesting = ingestTargetNotSatisfied && polledQueueNotExhausted; -// } while (continueIngesting); -// -// log.debug("{} new records were registered.", takenWorkCount); -// -// return takenWorkCount; -// } - -// private int ingestPolledRecordsIntoQueues(long ) { -// log.debug("Will attempt to register the requested {} - {} available in internal mailbox", -// requestedMaxWorkToRetrieve, wmbm.internalFlattenedMailQueueSize()); -// -// // -// var takenWorkCount = 0; -// boolean continueIngesting; -// do { -// ConsumerRecord polledRecord = wmbm.internalFlattenedMailQueuePoll(); -// boolean recordAddedAsWork = pm.maybeRegisterNewRecordAsWork(polledRecord); -// if (recordAddedAsWork) { -// takenWorkCount++; -// } -// boolean polledQueueNotExhausted = polledRecord != null; -// boolean ingestTargetNotSatisfied = takenWorkCount < requestedMaxWorkToRetrieve; -// continueIngesting = ingestTargetNotSatisfied && polledQueueNotExhausted; -// } while (continueIngesting); -// -// log.debug("{} new records were registered.", takenWorkCount); -// -// return takenWorkCount; -// } - /** * Get work with no limit on quantity, useful for testing. */ @@ -209,8 +145,6 @@ public List> getWorkIfAvailable(final int requestedMaxWorkTo return UniLists.of(); } -// int ingested = tryToEnsureQuantityOfWorkQueuedAvailable(requestedMaxWorkToRetrieve); - // var work = sm.getWorkIfAvailable(requestedMaxWorkToRetrieve); @@ -225,28 +159,6 @@ public List> getWorkIfAvailable(final int requestedMaxWorkTo return work; } -// /** -// * Tries to ensure there are at least this many records available in the queues -// * -// * @return the number of extra records ingested due to request -// */ -// // todo rename - shunt messages from internal buffer into queues -// // visible for testing -// public int tryToEnsureQuantityOfWorkQueuedAvailable(final int requestedMaxWorkToRetrieve) { -// // todo this counts all partitions as a whole - this may cause some partitions to starve. need to round robin it? -// long available = sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); -// long extraNeededFromInboxToSatisfy = requestedMaxWorkToRetrieve - available; -// log.debug("Requested: {}, available in shards: {}, will try to process from mailbox the delta of: {}", -// requestedMaxWorkToRetrieve, available, extraNeededFromInboxToSatisfy); -// -// int ingested = ingestPolledRecordsIntoQueues(extraNeededFromInboxToSatisfy); -// log.debug("Ingested an extra {} records", ingested); -// -// long ingestionOffBy = extraNeededFromInboxToSatisfy - ingested; -// -// return ingested; -// } - public void onSuccessResult(WorkContainer wc) { log.trace("Work success ({}), removing from processing shard queue", wc); @@ -283,10 +195,6 @@ public long getNumberOfEntriesInPartitionQueues() { return pm.getNumberOfEntriesInPartitionQueues(); } -// public Integer getAmountOfWorkQueuedWaitingIngestion() { -// return wmbm.getAmountOfWorkQueuedWaitingIngestion(); -// } - public Map collectCommitDataForDirtyPartitions() { return pm.collectDirtyCommitData(); } @@ -340,35 +248,17 @@ public boolean hasWorkInFlight() { public boolean isWorkInFlightMeetingTarget() { return getNumberRecordsOutForProcessing() >= options.getTargetAmountOfRecordsInFlight(); } -// -// /** -// * @return Work count in mailbox plus work added to the processing shards -// */ -// public long getTotalWorkAwaitingIngestion() { -//// return sm.getNumberOfEntriesInPartitionQueues -// return sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); -//// long workQueuedInShardsCount = sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); -//// Integer workQueuedInMailboxCount = getAmountOfWorkQueuedWaitingIngestion(); -//// return workQueuedInShardsCount + workQueuedInMailboxCount; -// } public long getNumberOfWorkQueuedInShardsAwaitingSelection() { return sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); } -// public boolean hasWorkAwaitingIngestionToShards() { -// return getAmountOfWorkQueuedWaitingIngestion() > 0; -// } - public boolean hasWorkInCommitQueues() { return pm.hasWorkInCommitQueues(); } public boolean isRecordsAwaitingProcessing() { return sm.getNumberOfWorkQueuedInShardsAwaitingSelection() > 0; -// long partitionWorkRemainingCount = sm.getNumberOfWorkQueuedInShardsAwaitingSelection(); -// boolean internalQueuesNotEmpty = hasWorkAwaitingIngestionToShards(); -// return partitionWorkRemainingCount > 0 || internalQueuesNotEmpty; } public boolean isRecordsAwaitingToBeCommitted() { @@ -403,12 +293,4 @@ public Optional getLowestRetryTime() { return sm.getLowestRetryTime(); } -// /** -// * @return true if more records are needed to be sent out for processing (not enough in queues to satisfy -// * concurrency target) -// */ -// public boolean isStarvedForNewWork() { -// long queued = getTotalWorkAwaitingIngestion(); -// return queued < options.getTargetAmountOfRecordsInFlight(); -// } } diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/LargeVolumeInMemoryTests.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/LargeVolumeInMemoryTests.java index f0fb03e8e..402c51536 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/LargeVolumeInMemoryTests.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/LargeVolumeInMemoryTests.java @@ -75,8 +75,6 @@ void load(CommitMode commitMode) { allMessagesConsumedLatch.await(defaultTimeoutSeconds, SECONDS); // waitAtMost(defaultTimeout).until(() -> producerSpy.consumerGroupOffsetsHistory().size() > 0); - // todo can remove? - parallelConsumer.waitForProcessedNotCommitted(defaultTimeout.multipliedBy(10)); parallelConsumer.close(); @@ -290,7 +288,7 @@ private void testTiming(int numberOfKeys, int quantityOfMessagesToProduce) { throw new RuntimeException("bad step, expected message(s) is missing: " + missing); } - assertThat(producerSpy.history().size()).as("Finally, all messages expected messages were produced").isEqualTo(quantityOfMessagesToProduce); + assertThat(producerSpy.history()).as("Finally, all messages expected messages were produced").hasSize(quantityOfMessagesToProduce); if (isUsingTransactionalProducer()) { List>> groupOffsetsHistory = producerSpy.consumerGroupOffsetsHistory(); // tx assertThat(groupOffsetsHistory).as("No offsets committed").hasSizeGreaterThan(0); // tx 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 2ae448d58..cd2a0d866 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 @@ -213,7 +213,7 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset var committed = consumerSpy.committed(UniSets.of(tp)).get(tp); assertThat(committed.offset()).isEqualTo(1L); - if (!encodingsThatFail.contains(encoding)) { + if (assumeWorkingCodec(encoding, encodingsThatFail)) { assertThat(committed.metadata()).isNotBlank(); } } @@ -229,7 +229,7 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset var pm = newWm.getPm(); var partitionState = pm.getPartitionState(tp); - if (!encodingsThatFail.contains(encoding)) { + if (assumeWorkingCodec(encoding, encodingsThatFail)) { long offsetHighestSequentialSucceeded = partitionState.getOffsetHighestSequentialSucceeded(); assertThat(offsetHighestSequentialSucceeded).isEqualTo(0); @@ -249,11 +249,7 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset // check state { - // todo ingestion no longer a thing - what to do here? -// Truth.assertThat(true).isFalse(); -// int ingested = newWm.tryToEnsureQuantityOfWorkQueuedAvailable(Integer.MAX_VALUE); - - if (!encodingsThatFail.contains(encoding)) { + if (assumeWorkingCodec(encoding, encodingsThatFail)) { long offsetHighestSequentialSucceeded = partitionState.getOffsetHighestSequentialSucceeded(); assertThat(offsetHighestSequentialSucceeded).isEqualTo(0); @@ -266,8 +262,6 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset var incompletes = partitionState.getIncompleteOffsetsBelowHighestSucceeded(); Truth.assertThat(incompletes).containsExactlyElementsIn(expected); - //todo and here -// assertThat(ingested).isEqualTo(testRecords.count() - 4); // 4 were succeeded Truth.assertThat(pm.isRecordPreviouslyCompleted(anIncompleteRecord)).isFalse(); } } @@ -297,6 +291,13 @@ void ensureEncodingGracefullyWorksWhenOffsetsAreVeryLargeAndNotSequential(Offset OffsetSimultaneousEncoder.compressionForced = false; } + /** + * A {@link OffsetEncoding} that works in this test scenario + */ + private boolean assumeWorkingCodec(OffsetEncoding encoding, List encodingsThatFail) { + return !encodingsThatFail.contains(encoding); + } + /** * This version of non sequential test just test the encoder directly, and is only half the story, as at the * encoding stage they don't know which offsets have never been seen, and assume simply working with continuous From 333ccac851b1627045dd74ca2dba04f42c213631 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Fri, 22 Apr 2022 10:43:37 +0100 Subject: [PATCH 46/54] review --- .../AbstractParallelEoSStreamProcessor.java | 26 ++++++++++--------- 1 file changed, 14 insertions(+), 12 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 e65694324..6d4be4eee 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 @@ -98,14 +98,16 @@ public abstract class AbstractParallelEoSStreamProcessor implements Parall * Collection of work waiting to be */ @Getter(PROTECTED) - private final BlockingQueue> workMailBox = new LinkedBlockingQueue<>(); // Thread safe, highly performant, non blocking + private final BlockingQueue> workMailBox = new LinkedBlockingQueue<>(); // Thread safe, highly performant, non blocking /** - * Either or + * An inbound message to the controller. + *

      + * Currently, an Either type class, representing either newly polled records to ingest, or a work result. */ @Value @RequiredArgsConstructor(access = PRIVATE) - private static class ActionItem { + private static class ControllerEventMessage { WorkContainer workContainer; EpochAndRecordsMap consumerRecords; @@ -117,12 +119,12 @@ private boolean isNewConsumerRecords() { return !isWorkResult(); } - private static ActionItem of(EpochAndRecordsMap polledRecords) { - return new ActionItem<>(null, polledRecords); + private static ControllerEventMessage of(EpochAndRecordsMap polledRecords) { + return new ControllerEventMessage<>(null, polledRecords); } - public static ActionItem of(WorkContainer work) { - return new ActionItem(work, null); + public static ControllerEventMessage of(WorkContainer work) { + return new ControllerEventMessage(work, null); } } @@ -349,7 +351,6 @@ public void onPartitionsRevoked(Collection partitions) { public void onPartitionsAssigned(Collection partitions) { numberOfAssignedPartitions = numberOfAssignedPartitions + partitions.size(); log.info("Assigned {} total ({} new) partition(s) {}", numberOfAssignedPartitions, partitions.size(), partitions); -// brokerPollSubsystem.onPartitionsAssigned(partitions); wm.onPartitionsAssigned(partitions); usersConsumerRebalanceListener.ifPresent(x -> x.onPartitionsAssigned(partitions)); notifySomethingToDo(); @@ -527,6 +528,7 @@ private boolean isResponsibleForCommits() { * * @deprecated no longer used, will be removed in next version */ + // TODO delete @Deprecated(forRemoval = true) @SneakyThrows public void waitForProcessedNotCommitted(Duration timeout) { @@ -905,7 +907,7 @@ private void transitionToClosing() { */ private void processWorkCompleteMailBox() { log.trace("Processing mailbox (might block waiting for results)..."); - Queue> results = new ArrayDeque<>(); + Queue> results = new ArrayDeque<>(); final Duration timeToBlockFor = getTimeToBlockFor(); @@ -955,7 +957,7 @@ private void processWorkCompleteMailBox() { /** * The amount of time to block poll in this cycle * - * @return either the duration until next commit, or next work retry // * @see WorkManager#isStarvedForNewWork() + * @return either the duration until next commit, or next work retry * @see ParallelConsumerOptions#getTargetAmountOfRecordsInFlight() */ private Duration getTimeToBlockFor() { @@ -1140,12 +1142,12 @@ protected void onUserFunctionSuccess(WorkContainer wc, List resultsFrom protected void addToMailbox(WorkContainer wc) { String state = wc.isUserFunctionSucceeded() ? "succeeded" : "FAILED"; log.trace("Adding {} {} to mailbox...", state, wc); - workMailBox.add(ActionItem.of(wc)); + workMailBox.add(ControllerEventMessage.of(wc)); } public void registerWork(EpochAndRecordsMap polledRecords) { log.debug("Adding {} to mailbox...", polledRecords); - workMailBox.add(ActionItem.of(polledRecords)); + workMailBox.add(ControllerEventMessage.of(polledRecords)); } /** From d751aa4a763a22cff08511abe6c8d70310f7c7cb Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Fri, 22 Apr 2022 10:50:51 +0100 Subject: [PATCH 47/54] review --- .../io/confluent/parallelconsumer/state/PartitionMonitor.java | 2 +- .../parallelconsumer/state/RemovedPartitionState.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java index a5af59759..8ec227ddf 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java @@ -214,7 +214,7 @@ public Long getEpochOfPartition(TopicPartition partition) { private void incrementPartitionAssignmentEpoch(final Collection partitions) { for (final TopicPartition partition : partitions) { - Long epoch = partitionsAssignmentEpochs.getOrDefault(partition, -1L); + Long epoch = partitionsAssignmentEpochs.getOrDefault(partition, PartitionState.KAFKA_OFFSET_ABSENCE); epoch++; partitionsAssignmentEpochs.put(partition, epoch); } 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 50b2f9749..0167442f8 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 @@ -82,13 +82,13 @@ public Set getIncompleteOffsetsBelowHighestSucceeded() { @Override public long getOffsetHighestSeen() { log.debug(NO_OP); - return -1L; + return PartitionState.KAFKA_OFFSET_ABSENCE; } @Override public long getOffsetHighestSucceeded() { log.debug(NO_OP); - return -1L; + return PartitionState.KAFKA_OFFSET_ABSENCE; } @Override From aa5c0e19b6ca026dc64387d1d1c2e5b542ca6a2c Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Fri, 22 Apr 2022 10:52:49 +0100 Subject: [PATCH 48/54] fix --- .../internal/AbstractParallelEoSStreamProcessor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 6d4be4eee..a6f95eb97 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 @@ -529,7 +529,7 @@ private boolean isResponsibleForCommits() { * @deprecated no longer used, will be removed in next version */ // TODO delete - @Deprecated(forRemoval = true) + @Deprecated @SneakyThrows public void waitForProcessedNotCommitted(Duration timeout) { log.debug("Waiting processed but not committed..."); From 3b51ffe8f6a49ae64f0b789ce830988b2b70f4da Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Fri, 22 Apr 2022 10:15:07 +0100 Subject: [PATCH 49/54] START: Rename PartitionMonitor to PartitionStateManager --- .../internal/EpochAndRecordsMap.java | 4 ++-- .../parallelconsumer/state/PartitionState.java | 2 +- ...tionMonitor.java => PartitionStateManager.java} | 3 +-- .../parallelconsumer/state/ProcessingShard.java | 2 +- .../state/RemovedPartitionState.java | 14 +++++++------- .../parallelconsumer/state/WorkManager.java | 7 +++---- .../offsets/OffsetEncodingBackPressureTest.java | 6 +++--- .../OffsetEncodingBackPressureUnitTest.java | 8 ++++---- .../src/test/resources/logback-test.xml | 2 +- 9 files changed, 23 insertions(+), 25 deletions(-) rename parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/{PartitionMonitor.java => PartitionStateManager.java} (99%) 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 95d6e3f7e..ffd7c0874 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 @@ -4,7 +4,7 @@ * Copyright (C) 2020-2022 Confluent, Inc. */ -import io.confluent.parallelconsumer.state.PartitionMonitor; +import io.confluent.parallelconsumer.state.PartitionStateManager; import lombok.Value; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -22,7 +22,7 @@ public class EpochAndRecordsMap { Map recordMap = new HashMap<>(); - public EpochAndRecordsMap(ConsumerRecords poll, PartitionMonitor pm) { + public EpochAndRecordsMap(ConsumerRecords poll, PartitionStateManager pm) { poll.partitions().forEach(partition -> { var records = poll.records(partition); Long epochOfPartition = pm.getEpochOfPartition(partition); 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 59a2df993..ebf33fe81 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 @@ -316,7 +316,7 @@ private boolean updateBlockFromEncodingResult(String offsetMapPayload) { } private double getPressureThresholdValue() { - return DefaultMaxMetadataSize * PartitionMonitor.getUSED_PAYLOAD_THRESHOLD_MULTIPLIER(); + return DefaultMaxMetadataSize * PartitionStateManager.getUSED_PAYLOAD_THRESHOLD_MULTIPLIER(); } public void onPartitionsRemoved(ShardManager sm) { diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java similarity index 99% rename from parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java rename to parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java index 8ec227ddf..1ec416567 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java @@ -38,8 +38,7 @@ */ @Slf4j @RequiredArgsConstructor -// todo rename to partition state manager -public class PartitionMonitor implements ConsumerRebalanceListener { +public class PartitionStateManager implements ConsumerRebalanceListener { public static final double USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT = 0.75; /** diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ProcessingShard.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ProcessingShard.java index ef63571b2..9727fcb73 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ProcessingShard.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ProcessingShard.java @@ -43,7 +43,7 @@ public class ProcessingShard { private final ParallelConsumerOptions options; - private final PartitionMonitor pm; + private final PartitionStateManager pm; private final RateLimiter slowWarningRateLimit = new RateLimiter(5); 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 0167442f8..aef4946f1 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 @@ -18,13 +18,13 @@ * references or {@link Optional}s. By replacing with a no op implementation, we protect for stale messages still in * queues which reference it, among other things. *

      - * The alternative to this implementation, is having {@link PartitionMonitor#getPartitionState(TopicPartition)} return - * {@link Optional}, which forces the implicit null check everywhere partition state is retrieved. This was drafted to a - * degree, but found to be extremely invasive, where this solution with decent separation of concerns and encapsulation, - * is sufficient and potentially more useful as is non-destructive. Potential issue is that of memory leak as the - * collection will forever expand. However, even massive partition counts to a single consumer would be in the hundreds - * of thousands, this would only result in hundreds of thousands of {@link TopicPartition} object keys all pointing to - * the same instance of {@link RemovedPartitionState}. + * The alternative to this implementation, is having {@link PartitionStateManager#getPartitionState(TopicPartition)} + * return {@link Optional}, which forces the implicit null check everywhere partition state is retrieved. This was + * drafted to a degree, but found to be extremely invasive, where this solution with decent separation of concerns and + * encapsulation, is sufficient and potentially more useful as is non-destructive. Potential issue is that of memory + * leak as the collection will forever expand. However, even massive partition counts to a single consumer would be in + * the hundreds of thousands, this would only result in hundreds of thousands of {@link TopicPartition} object keys all + * pointing to the same instance of {@link RemovedPartitionState}. */ @Slf4j public class RemovedPartitionState extends PartitionState { diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java index 278fd1cce..d92317405 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java @@ -46,10 +46,9 @@ public class WorkManager implements ConsumerRebalanceListener { @Getter private final ParallelConsumerOptions options; - // todo rename PSM, PartitionStateManager // todo make private @Getter(PUBLIC) - final PartitionMonitor pm; + final PartitionStateManager pm; // todo make private @Getter(PUBLIC) @@ -88,7 +87,7 @@ public WorkManager(final ParallelConsumerOptions newOptions, final org.apa this.options = newOptions; this.dynamicLoadFactor = dynamicExtraLoadFactor; this.sm = new ShardManager<>(options, this, clock); - this.pm = new PartitionMonitor<>(consumer, sm, options, clock); + this.pm = new PartitionStateManager<>(consumer, sm, options, clock); } /** @@ -177,7 +176,7 @@ public void onSuccessResult(WorkContainer wc) { /** * Can run from controller or poller thread, depending on which is responsible for committing * - * @see PartitionMonitor#onOffsetCommitSuccess(Map) + * @see PartitionStateManager#onOffsetCommitSuccess(Map) */ public void onOffsetCommitSuccess(Map committed) { pm.onOffsetCommitSuccess(committed); diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java index 57ac406ec..1bfdd4703 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java @@ -35,7 +35,7 @@ import static io.confluent.csid.utils.LatchTestUtils.awaitLatch; import static io.confluent.csid.utils.ThreadUtils.sleepQuietly; import static io.confluent.parallelconsumer.ManagedTruth.assertWithMessage; -import static io.confluent.parallelconsumer.state.PartitionMonitor.USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT; +import static io.confluent.parallelconsumer.state.PartitionStateManager.USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT; import static java.time.Duration.ofMillis; import static java.time.Duration.ofSeconds; import static java.util.concurrent.TimeUnit.SECONDS; @@ -61,7 +61,7 @@ class OffsetEncodingBackPressureTest extends ParallelEoSStreamProcessorTestBase @AfterAll static void cleanup() { - PartitionMonitor.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT); + PartitionStateManager.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT); } /** @@ -226,7 +226,7 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O log.debug("// test max payload exceeded, payload dropped"); { log.debug("// force system to allow more records (i.e. the actual system attempts to never allow the payload to grow this big)"); - PartitionMonitor.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(30); + PartitionStateManager.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(30); parallelConsumer.requestCommitAsap(); awaitForOneLoopCycle(); diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java index 014f9eaf8..7d420d5a2 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java @@ -7,8 +7,8 @@ import com.google.common.truth.Truth; import io.confluent.parallelconsumer.ParallelEoSStreamProcessorTestBase; import io.confluent.parallelconsumer.internal.EpochAndRecordsMap; -import io.confluent.parallelconsumer.state.PartitionMonitor; import io.confluent.parallelconsumer.state.PartitionState; +import io.confluent.parallelconsumer.state.PartitionStateManager; import io.confluent.parallelconsumer.state.WorkContainer; import io.confluent.parallelconsumer.state.WorkManager; import lombok.SneakyThrows; @@ -32,7 +32,7 @@ import static io.confluent.parallelconsumer.ManagedTruth.assertTruth; import static io.confluent.parallelconsumer.ManagedTruth.assertWithMessage; -import static io.confluent.parallelconsumer.state.PartitionMonitor.USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT; +import static io.confluent.parallelconsumer.state.PartitionStateManager.USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT; import static java.time.Duration.ofMillis; /** @@ -45,7 +45,7 @@ class OffsetEncodingBackPressureUnitTest extends ParallelEoSStreamProcessorTestB @AfterAll static void cleanup() { - PartitionMonitor.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT); + PartitionStateManager.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT); } @SneakyThrows @@ -127,7 +127,7 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O log.debug("// messages already sent {}, sending {} more", processedBeforePartitionBlock, extraMessages); { log.debug("// force system to allow more records (i.e. the actual system attempts to never allow the payload to grow this big)"); - PartitionMonitor.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(2); + PartitionStateManager.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(2); // // unlock 2L as well diff --git a/parallel-consumer-core/src/test/resources/logback-test.xml b/parallel-consumer-core/src/test/resources/logback-test.xml index a9345a2ed..e7d4e3b4f 100644 --- a/parallel-consumer-core/src/test/resources/logback-test.xml +++ b/parallel-consumer-core/src/test/resources/logback-test.xml @@ -43,7 +43,7 @@ - + From cc5debfbf4b6e941a0523b12f78bb0341c151a07 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Fri, 22 Apr 2022 11:33:48 +0100 Subject: [PATCH 50/54] START: Rename PartitionMonitor to PartitionStateManager --- .../internal/EpochAndRecordsMap.java | 4 ++-- .../parallelconsumer/state/PartitionState.java | 5 ++++- ...tionMonitor.java => PartitionStateManager.java} | 3 +-- .../parallelconsumer/state/ProcessingShard.java | 2 +- .../state/RemovedPartitionState.java | 14 +++++++------- .../parallelconsumer/state/WorkManager.java | 7 +++---- .../offsets/OffsetEncodingBackPressureTest.java | 6 +++--- .../OffsetEncodingBackPressureUnitTest.java | 8 ++++---- .../src/test/resources/logback-test.xml | 2 +- 9 files changed, 26 insertions(+), 25 deletions(-) rename parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/{PartitionMonitor.java => PartitionStateManager.java} (99%) 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 95d6e3f7e..ffd7c0874 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 @@ -4,7 +4,7 @@ * Copyright (C) 2020-2022 Confluent, Inc. */ -import io.confluent.parallelconsumer.state.PartitionMonitor; +import io.confluent.parallelconsumer.state.PartitionStateManager; import lombok.Value; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -22,7 +22,7 @@ public class EpochAndRecordsMap { Map recordMap = new HashMap<>(); - public EpochAndRecordsMap(ConsumerRecords poll, PartitionMonitor pm) { + public EpochAndRecordsMap(ConsumerRecords poll, PartitionStateManager pm) { poll.partitions().forEach(partition -> { var records = poll.records(partition); Long epochOfPartition = pm.getEpochOfPartition(partition); 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 59a2df993..96fc04eb8 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 @@ -28,6 +28,9 @@ import static java.util.Optional.of; import static lombok.AccessLevel.*; +/** + * @see PartitionStateManager + */ @ToString @Slf4j public class PartitionState { @@ -316,7 +319,7 @@ private boolean updateBlockFromEncodingResult(String offsetMapPayload) { } private double getPressureThresholdValue() { - return DefaultMaxMetadataSize * PartitionMonitor.getUSED_PAYLOAD_THRESHOLD_MULTIPLIER(); + return DefaultMaxMetadataSize * PartitionStateManager.getUSED_PAYLOAD_THRESHOLD_MULTIPLIER(); } public void onPartitionsRemoved(ShardManager sm) { diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java similarity index 99% rename from parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java rename to parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java index 8ec227ddf..1ec416567 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionMonitor.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/PartitionStateManager.java @@ -38,8 +38,7 @@ */ @Slf4j @RequiredArgsConstructor -// todo rename to partition state manager -public class PartitionMonitor implements ConsumerRebalanceListener { +public class PartitionStateManager implements ConsumerRebalanceListener { public static final double USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT = 0.75; /** diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ProcessingShard.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ProcessingShard.java index ef63571b2..9727fcb73 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ProcessingShard.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/ProcessingShard.java @@ -43,7 +43,7 @@ public class ProcessingShard { private final ParallelConsumerOptions options; - private final PartitionMonitor pm; + private final PartitionStateManager pm; private final RateLimiter slowWarningRateLimit = new RateLimiter(5); 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 0167442f8..aef4946f1 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 @@ -18,13 +18,13 @@ * references or {@link Optional}s. By replacing with a no op implementation, we protect for stale messages still in * queues which reference it, among other things. *

      - * The alternative to this implementation, is having {@link PartitionMonitor#getPartitionState(TopicPartition)} return - * {@link Optional}, which forces the implicit null check everywhere partition state is retrieved. This was drafted to a - * degree, but found to be extremely invasive, where this solution with decent separation of concerns and encapsulation, - * is sufficient and potentially more useful as is non-destructive. Potential issue is that of memory leak as the - * collection will forever expand. However, even massive partition counts to a single consumer would be in the hundreds - * of thousands, this would only result in hundreds of thousands of {@link TopicPartition} object keys all pointing to - * the same instance of {@link RemovedPartitionState}. + * The alternative to this implementation, is having {@link PartitionStateManager#getPartitionState(TopicPartition)} + * return {@link Optional}, which forces the implicit null check everywhere partition state is retrieved. This was + * drafted to a degree, but found to be extremely invasive, where this solution with decent separation of concerns and + * encapsulation, is sufficient and potentially more useful as is non-destructive. Potential issue is that of memory + * leak as the collection will forever expand. However, even massive partition counts to a single consumer would be in + * the hundreds of thousands, this would only result in hundreds of thousands of {@link TopicPartition} object keys all + * pointing to the same instance of {@link RemovedPartitionState}. */ @Slf4j public class RemovedPartitionState extends PartitionState { diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java index 278fd1cce..d92317405 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/state/WorkManager.java @@ -46,10 +46,9 @@ public class WorkManager implements ConsumerRebalanceListener { @Getter private final ParallelConsumerOptions options; - // todo rename PSM, PartitionStateManager // todo make private @Getter(PUBLIC) - final PartitionMonitor pm; + final PartitionStateManager pm; // todo make private @Getter(PUBLIC) @@ -88,7 +87,7 @@ public WorkManager(final ParallelConsumerOptions newOptions, final org.apa this.options = newOptions; this.dynamicLoadFactor = dynamicExtraLoadFactor; this.sm = new ShardManager<>(options, this, clock); - this.pm = new PartitionMonitor<>(consumer, sm, options, clock); + this.pm = new PartitionStateManager<>(consumer, sm, options, clock); } /** @@ -177,7 +176,7 @@ public void onSuccessResult(WorkContainer wc) { /** * Can run from controller or poller thread, depending on which is responsible for committing * - * @see PartitionMonitor#onOffsetCommitSuccess(Map) + * @see PartitionStateManager#onOffsetCommitSuccess(Map) */ public void onOffsetCommitSuccess(Map committed) { pm.onOffsetCommitSuccess(committed); diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java index 57ac406ec..1bfdd4703 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureTest.java @@ -35,7 +35,7 @@ import static io.confluent.csid.utils.LatchTestUtils.awaitLatch; import static io.confluent.csid.utils.ThreadUtils.sleepQuietly; import static io.confluent.parallelconsumer.ManagedTruth.assertWithMessage; -import static io.confluent.parallelconsumer.state.PartitionMonitor.USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT; +import static io.confluent.parallelconsumer.state.PartitionStateManager.USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT; import static java.time.Duration.ofMillis; import static java.time.Duration.ofSeconds; import static java.util.concurrent.TimeUnit.SECONDS; @@ -61,7 +61,7 @@ class OffsetEncodingBackPressureTest extends ParallelEoSStreamProcessorTestBase @AfterAll static void cleanup() { - PartitionMonitor.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT); + PartitionStateManager.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT); } /** @@ -226,7 +226,7 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O log.debug("// test max payload exceeded, payload dropped"); { log.debug("// force system to allow more records (i.e. the actual system attempts to never allow the payload to grow this big)"); - PartitionMonitor.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(30); + PartitionStateManager.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(30); parallelConsumer.requestCommitAsap(); awaitForOneLoopCycle(); diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java index 014f9eaf8..7d420d5a2 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/offsets/OffsetEncodingBackPressureUnitTest.java @@ -7,8 +7,8 @@ import com.google.common.truth.Truth; import io.confluent.parallelconsumer.ParallelEoSStreamProcessorTestBase; import io.confluent.parallelconsumer.internal.EpochAndRecordsMap; -import io.confluent.parallelconsumer.state.PartitionMonitor; import io.confluent.parallelconsumer.state.PartitionState; +import io.confluent.parallelconsumer.state.PartitionStateManager; import io.confluent.parallelconsumer.state.WorkContainer; import io.confluent.parallelconsumer.state.WorkManager; import lombok.SneakyThrows; @@ -32,7 +32,7 @@ import static io.confluent.parallelconsumer.ManagedTruth.assertTruth; import static io.confluent.parallelconsumer.ManagedTruth.assertWithMessage; -import static io.confluent.parallelconsumer.state.PartitionMonitor.USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT; +import static io.confluent.parallelconsumer.state.PartitionStateManager.USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT; import static java.time.Duration.ofMillis; /** @@ -45,7 +45,7 @@ class OffsetEncodingBackPressureUnitTest extends ParallelEoSStreamProcessorTestB @AfterAll static void cleanup() { - PartitionMonitor.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT); + PartitionStateManager.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(USED_PAYLOAD_THRESHOLD_MULTIPLIER_DEFAULT); } @SneakyThrows @@ -127,7 +127,7 @@ void backPressureShouldPreventTooManyMessagesBeingQueuedForProcessing() throws O log.debug("// messages already sent {}, sending {} more", processedBeforePartitionBlock, extraMessages); { log.debug("// force system to allow more records (i.e. the actual system attempts to never allow the payload to grow this big)"); - PartitionMonitor.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(2); + PartitionStateManager.setUSED_PAYLOAD_THRESHOLD_MULTIPLIER(2); // // unlock 2L as well diff --git a/parallel-consumer-core/src/test/resources/logback-test.xml b/parallel-consumer-core/src/test/resources/logback-test.xml index a9345a2ed..e7d4e3b4f 100644 --- a/parallel-consumer-core/src/test/resources/logback-test.xml +++ b/parallel-consumer-core/src/test/resources/logback-test.xml @@ -43,7 +43,7 @@ - + From b6ad2ecf592734f05cb4024d3ac6b0a85cf9b748 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Mon, 16 May 2022 15:54:11 +0100 Subject: [PATCH 51/54] docs --- CHANGELOG.adoc | 8 +++++++- README.adoc | 11 +++++++++-- .../parallelconsumer/ParallelEoSStreamProcessor.java | 3 +-- src/docs/README_TEMPLATE.adoc | 3 ++- 4 files changed, 19 insertions(+), 6 deletions(-) diff --git a/CHANGELOG.adoc b/CHANGELOG.adoc index 40ebe93fa..8184b56a9 100644 --- a/CHANGELOG.adoc +++ b/CHANGELOG.adoc @@ -12,11 +12,17 @@ endif::[] == Next Version +== v0.5.1.0 + +=== Features + +* #193: Pause / Resume PC (circuit breaker) without unsubscribing from topics + == v0.5.0.1 === Fixes and Improvements -- fixes: #225 Build support for Java 17, 18 (#289) +* fixes: #225 Build support for Java 17, 18 (#289) == v0.5.0.0 diff --git a/README.adoc b/README.adoc index 0d606144d..9aae114f4 100644 --- a/README.adoc +++ b/README.adoc @@ -250,7 +250,8 @@ without operational burden or harming the cluster's performance * Java 8 compatibility * Throttle control and broker liveliness management * Clean draining shutdown cycle -* Manual Pause / Resume (pausing of partitions is also automatic, whenever back pressure has built up) +* Manual Pause / Resume of entire PC without unsubscribing from topics (useful for implementing a simplistic https://en.wikipedia.org/wiki/Circuit_breaker_design_pattern[circuit breaker]) +** Note: Pausing of a partition is also automatic, whenever back pressure has built up on a given partition //image:https://codecov.io/gh/astubbs/parallel-consumer/branch/master/graph/badge.svg["Coverage",https://codecov.io/gh/astubbs/parallel-consumer] //image:https://travis-ci.com/astubbs/parallel-consumer.svg?branch=master["Build Status", link="https://travis-ci.com/astubbs/parallel-consumer"] @@ -1162,11 +1163,17 @@ endif::[] == Next Version +== v0.5.1.0 + +=== Features + +* #193: Pause / Resume PC (circuit breaker) without unsubscribing from topics + == v0.5.0.1 === Fixes and Improvements -- fixes: #225 Build support for Java 17, 18 (#289) +* fixes: #225 Build support for Java 17, 18 (#289) == v0.5.0.0 diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessor.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessor.java index e9486aab7..f29b74c92 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessor.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessor.java @@ -19,8 +19,7 @@ import static io.confluent.parallelconsumer.internal.UserFunctions.carefullyRun; @Slf4j -public class ParallelEoSStreamProcessor extends AbstractParallelEoSStreamProcessor - implements ParallelStreamProcessor { +public class ParallelEoSStreamProcessor extends AbstractParallelEoSStreamProcessor implements ParallelStreamProcessor { /** * Construct the AsyncConsumer by wrapping this passed in conusmer and producer, which can be configured any which diff --git a/src/docs/README_TEMPLATE.adoc b/src/docs/README_TEMPLATE.adoc index 99076f473..b7a116890 100644 --- a/src/docs/README_TEMPLATE.adoc +++ b/src/docs/README_TEMPLATE.adoc @@ -248,7 +248,8 @@ without operational burden or harming the cluster's performance * Java 8 compatibility * Throttle control and broker liveliness management * Clean draining shutdown cycle -* Manual Pause / Resume (pausing of partitions is also automatic, whenever back pressure has built up) +* Manual Pause / Resume of entire PC without unsubscribing from topics (useful for implementing a simplistic https://en.wikipedia.org/wiki/Circuit_breaker_design_pattern[circuit breaker]) +** Note: Pausing of a partition is also automatic, whenever back pressure has built up on a given partition //image:https://codecov.io/gh/astubbs/parallel-consumer/branch/master/graph/badge.svg["Coverage",https://codecov.io/gh/astubbs/parallel-consumer] //image:https://travis-ci.com/astubbs/parallel-consumer.svg?branch=master["Build Status", link="https://travis-ci.com/astubbs/parallel-consumer"] From 9164c44663018e39947476302dd86c487f3e80fa Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Mon, 16 May 2022 16:43:36 +0100 Subject: [PATCH 52/54] review --- .../parallelconsumer/ParallelEoSStreamProcessor.java | 5 +++-- .../AbstractParallelEoSStreamProcessorTestBase.java | 5 ++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessor.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessor.java index f29b74c92..ff46569b3 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessor.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessor.java @@ -19,10 +19,11 @@ import static io.confluent.parallelconsumer.internal.UserFunctions.carefullyRun; @Slf4j -public class ParallelEoSStreamProcessor extends AbstractParallelEoSStreamProcessor implements ParallelStreamProcessor { +public class ParallelEoSStreamProcessor extends AbstractParallelEoSStreamProcessor + implements ParallelStreamProcessor { /** - * Construct the AsyncConsumer by wrapping this passed in conusmer and producer, which can be configured any which + * Construct the AsyncConsumer by wrapping this passed in consumer and producer, which can be configured any which * way as per normal. * * @see ParallelConsumerOptions diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java index ec309bc50..246a72450 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java @@ -336,13 +336,13 @@ public void assertCommitsContains(List offsets) { assertThat(commits).containsAll(offsets); } - public List getCommitHistoryFlattened() { + private List getCommitHistoryFlattened() { return (isUsingTransactionalProducer()) ? ktu.getProducerCommitsFlattened(producerSpy) : extractAllPartitionsOffsetsSequentially(false); } - public List getCommitHistoryFlattenedMeta() { + private List getCommitHistoryFlattenedMeta() { return (isUsingTransactionalProducer()) ? ktu.getProducerCommitsMeta(producerSpy) : extractAllPartitionsOffsetsSequentiallyMeta(true); @@ -396,7 +396,6 @@ protected List extractAllPartitionsOffsetsSequentiallyMeta(bo .flatMap(commits -> { var rawValues = new ArrayList<>(commits.values()).stream(); // 4 debugging -// Stream rawOffsets = values.stream().map(meta -> (int) meta.offset()); if (trimGenesis) return rawValues.filter(x -> x.offset() != 0); else From 968704b6644d953426a90d3e02690891441c2ab6 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Mon, 16 May 2022 17:42:26 +0100 Subject: [PATCH 53/54] review review --- .../parallelconsumer/PollContextInternal.java | 2 ++ .../parallelconsumer/RecordContextInternal.java | 2 ++ ...bstractParallelEoSStreamProcessorTestBase.java | 2 +- ...ParallelEoSStreamProcessorPauseResumeTest.java | 15 +++++++++------ src/docs/README_TEMPLATE.adoc | 3 ++- 5 files changed, 16 insertions(+), 8 deletions(-) diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/PollContextInternal.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/PollContextInternal.java index acf12eb0c..108627f42 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/PollContextInternal.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/PollContextInternal.java @@ -6,6 +6,7 @@ import io.confluent.parallelconsumer.state.WorkContainer; import lombok.Getter; +import lombok.ToString; import lombok.experimental.Delegate; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -16,6 +17,7 @@ /** * Internal only view on the {@link PollContext}. */ +@ToString public class PollContextInternal { @Delegate diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/RecordContextInternal.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/RecordContextInternal.java index 44c1a04ed..1385116cb 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/RecordContextInternal.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/RecordContextInternal.java @@ -6,10 +6,12 @@ import io.confluent.parallelconsumer.state.WorkContainer; import lombok.Getter; +import lombok.ToString; /** * Internal only view of the {@link RecordContext} class. */ +@ToString public class RecordContextInternal { @Getter diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java index 246a72450..dc5d8e968 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/AbstractParallelEoSStreamProcessorTestBase.java @@ -336,7 +336,7 @@ public void assertCommitsContains(List offsets) { assertThat(commits).containsAll(offsets); } - private List getCommitHistoryFlattened() { + protected List getCommitHistoryFlattened() { return (isUsingTransactionalProducer()) ? ktu.getProducerCommitsFlattened(producerSpy) : extractAllPartitionsOffsetsSequentially(false); diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorPauseResumeTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorPauseResumeTest.java index 4d3b0d157..f1ff2a889 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorPauseResumeTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorPauseResumeTest.java @@ -17,6 +17,7 @@ import java.util.function.Consumer; import static com.google.common.truth.Truth.assertThat; +import static io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder.UNORDERED; /** * Test for pause/resume feature of the parallel consumer (see {@code GH#193}). @@ -48,6 +49,7 @@ public void lockProcessing() { } public void unlockProcessing() { + log.info("Unlocking processing"); mutex.unlock(); } @@ -57,7 +59,8 @@ public void accept(PollContext t) { numInFlightRecords.incrementAndGet(); try { lockProcessing(); - numProcessedRecords.incrementAndGet(); + int numProcessed = numProcessedRecords.incrementAndGet(); + log.info("Processed complete, incremented to {}", numProcessed); } finally { unlockProcessing(); numInFlightRecords.decrementAndGet(); @@ -173,14 +176,14 @@ void pausingAndResumingProcessingShouldWork(final CommitMode commitMode) { @SneakyThrows void testThatInFlightWorkIsFinishedSuccessfullyAndOffsetsAreCommitted(final CommitMode commitMode) { int degreeOfParallelism = 3; - int numTestRecordsPerBatch = 1_000; + int numTestRecordsPerSet = 1_000; TestUserFunction testUserFunction = createTestSetup(commitMode, degreeOfParallelism); // block processing in the user function to ensure we have in flight work once we pause the consumer testUserFunction.lockProcessing(); // produce some messages - addRecordsWithSetKey(numTestRecordsPerBatch); + addRecordsWithSetKey(numTestRecordsPerSet); // wait until we have enough records in flight Awaitility @@ -217,11 +220,11 @@ void testThatInFlightWorkIsFinishedSuccessfullyAndOffsetsAreCommitted(final Comm // other pending messages should be processed now Awaitility .waitAtMost(defaultTimeout) - .alias(numTestRecordsPerBatch + " records should be processed") - .untilAsserted(() -> assertThat(testUserFunction.numProcessedRecords.get()).isEqualTo(numTestRecordsPerBatch)); + .alias(numTestRecordsPerSet + " records should be processed") + .untilAsserted(() -> assertThat(testUserFunction.numProcessedRecords.get()).isEqualTo(numTestRecordsPerSet)); // overall committed offset should reach the total number of processed records - awaitForCommit(testUserFunction.numProcessedRecords.get()); + awaitForCommit(numTestRecordsPerSet); } } diff --git a/src/docs/README_TEMPLATE.adoc b/src/docs/README_TEMPLATE.adoc index b7a116890..30c5ad2d8 100644 --- a/src/docs/README_TEMPLATE.adoc +++ b/src/docs/README_TEMPLATE.adoc @@ -248,7 +248,8 @@ without operational burden or harming the cluster's performance * Java 8 compatibility * Throttle control and broker liveliness management * Clean draining shutdown cycle -* Manual Pause / Resume of entire PC without unsubscribing from topics (useful for implementing a simplistic https://en.wikipedia.org/wiki/Circuit_breaker_design_pattern[circuit breaker]) +* Manual global pause / resume of all partitions, without unsubscribing from topics (useful for implementing a simplistic https://en.wikipedia.org/wiki/Circuit_breaker_design_pattern[circuit breaker]) +** Circuit breaker patterns for individual paritions or keys can be done through throwing failure exceptions in the processing function (see https://github.com/confluentinc/parallel-consumer/pull/291[PR #291 Explicit terminal and retriable exceptions] for further refinement) ** Note: Pausing of a partition is also automatic, whenever back pressure has built up on a given partition //image:https://codecov.io/gh/astubbs/parallel-consumer/branch/master/graph/badge.svg["Coverage",https://codecov.io/gh/astubbs/parallel-consumer] From 2a9e5c864bc66b24dabf5d3029671b77941c526e Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Mon, 16 May 2022 18:59:54 +0100 Subject: [PATCH 54/54] fix test - processing order will affect the actual offsets that are committed --- .../ParallelEoSStreamProcessorPauseResumeTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorPauseResumeTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorPauseResumeTest.java index f1ff2a889..5bf8f834e 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorPauseResumeTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorPauseResumeTest.java @@ -76,6 +76,8 @@ private ParallelConsumerOptions getBaseOptions(final CommitMode return ParallelConsumerOptions.builder() .commitMode(commitMode) .consumer(consumerSpy) + // UNORDERED so that we get nice linear offsets in our processing order (PARTITION has no concurrency, KEY depends on your keys + .ordering(UNORDERED) .maxConcurrency(maxConcurrency) .build(); }