From dff070fe79886d4395a84aa9214813063c4ef30c Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Wed, 18 Nov 2020 15:08:41 +0000 Subject: [PATCH 01/11] Adds unit test method logging - something is wrong with the AutoService method (todo) --- .../confluent/csid/utils/MyRunListener.java | 54 +++++++++++++++++++ .../io/confluent/csid/utils/TestLogger.java | 38 ------------- ...it.platform.launcher.TestExecutionListener | 1 + .../src/test/resources/logback-test.xml | 6 +++ 4 files changed, 61 insertions(+), 38 deletions(-) create mode 100644 parallel-consumer-core/src/test/java/io/confluent/csid/utils/MyRunListener.java delete mode 100644 parallel-consumer-core/src/test/java/io/confluent/csid/utils/TestLogger.java create mode 100644 parallel-consumer-core/src/test/resources/META-INF/services/org.junit.platform.launcher.TestExecutionListener diff --git a/parallel-consumer-core/src/test/java/io/confluent/csid/utils/MyRunListener.java b/parallel-consumer-core/src/test/java/io/confluent/csid/utils/MyRunListener.java new file mode 100644 index 000000000..52a9ba670 --- /dev/null +++ b/parallel-consumer-core/src/test/java/io/confluent/csid/utils/MyRunListener.java @@ -0,0 +1,54 @@ +package io.confluent.csid.utils; + +import com.google.auto.service.AutoService; +import lombok.extern.slf4j.Slf4j; +import org.junit.platform.engine.TestExecutionResult; +import org.junit.platform.engine.reporting.ReportEntry; +import org.junit.platform.launcher.TestExecutionListener; +import org.junit.platform.launcher.TestIdentifier; +import org.junit.platform.launcher.TestPlan; + +import static io.confluent.csid.utils.StringUtils.msg; + +/** + * Print out test names in Maven output, not just class names. Useful for added context for failures in CI runs. + *

+ * {@link AutoService} should work and load this automaticly, but something isn't working. So this is loaded manually + * through the META-INF/serices/org.junit.platform.launcher.TestExecutionListener + *

+ *  https://stackoverflow.com/questions/49937451/junit-5-is-it-possible-to-set-a-testexecutionlistener-in-maven-surefire-plugin
+ *  https://junit.org/junit5/docs/current/user-guide/#launcher-api-listeners-custom
+ *  https://github.com/google/auto/tree/master/service
+ *  https://stackoverflow.com/a/50058828/105741
+ * 
+ */ +@AutoService(TestExecutionListener.class) +public class MyRunListener implements TestExecutionListener { + + private final String template = """ + + ========= + JUNIT {}: {} ({}) + ========= + """; + + @Override + public void testPlanExecutionStarted(final TestPlan testPlan) { + log(msg(template, "Test plan execution started", testPlan, "")); + } + + private void log(final String msg) { + System.out.println(msg); + } + + @Override + public void executionSkipped(final TestIdentifier testIdentifier, final String reason) { + log(msg(template, "skipped", testIdentifier.getDisplayName(), reason)); + } + + @Override + public void executionStarted(final TestIdentifier testIdentifier) { + log(msg(template, "started", testIdentifier.getDisplayName(), testIdentifier.getLegacyReportingName())); + } + +} diff --git a/parallel-consumer-core/src/test/java/io/confluent/csid/utils/TestLogger.java b/parallel-consumer-core/src/test/java/io/confluent/csid/utils/TestLogger.java deleted file mode 100644 index 972e07f20..000000000 --- a/parallel-consumer-core/src/test/java/io/confluent/csid/utils/TestLogger.java +++ /dev/null @@ -1,38 +0,0 @@ -package io.confluent.csid.utils; - -/*- - * Copyright (C) 2020 Confluent, Inc. - */ - -import com.google.auto.service.AutoService; -import lombok.extern.slf4j.Slf4j; -import org.junit.jupiter.api.extension.BeforeEachCallback; -import org.junit.jupiter.api.extension.ExtensionContext; -import org.junit.platform.launcher.TestExecutionListener; -import org.junit.platform.launcher.TestIdentifier; - -/** - * Print out test names in Maven output, not just class names. Useful for added context for failures in CI runs. - *

- * Happy times... =D - *

- * https://stackoverflow.com/questions/49937451/junit-5-is-it-possible-to-set-a-testexecutionlistener-in-maven-surefire-plugin - * https://junit.org/junit5/docs/current/user-guide/#launcher-api-listeners-custom - * https://github.com/google/auto/tree/master/service - * https://stackoverflow.com/a/50058828/105741 - */ -@AutoService(TestExecutionListener.class) -@Slf4j -public class TestLogger implements TestExecutionListener { - - @Override - public void executionStarted(TestIdentifier testIdentifier) { -// log.info("-------------------------------------------------------"); -// log.info(testIdentifier.getDisplayName()); -// log.info("-------------------------------------------------------"); - System.out.println("-------------------------------------------------------"); - System.out.println(testIdentifier.getDisplayName()); - System.out.println("-------------------------------------------------------"); - } - -} diff --git a/parallel-consumer-core/src/test/resources/META-INF/services/org.junit.platform.launcher.TestExecutionListener b/parallel-consumer-core/src/test/resources/META-INF/services/org.junit.platform.launcher.TestExecutionListener new file mode 100644 index 000000000..3b6460b54 --- /dev/null +++ b/parallel-consumer-core/src/test/resources/META-INF/services/org.junit.platform.launcher.TestExecutionListener @@ -0,0 +1 @@ +io.confluent.csid.utils.MyRunListener \ No newline at end of file diff --git a/parallel-consumer-core/src/test/resources/logback-test.xml b/parallel-consumer-core/src/test/resources/logback-test.xml index 0c8e39099..aa004bb17 100644 --- a/parallel-consumer-core/src/test/resources/logback-test.xml +++ b/parallel-consumer-core/src/test/resources/logback-test.xml @@ -25,6 +25,12 @@ + + + + + + From ff41d135fb4890f5f10d3462320b9dbc5d466c54 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Mon, 23 Nov 2020 15:09:40 +0000 Subject: [PATCH 02/11] feature: Choose between Consumer commit or Producer transactional commits - Choose either Consumer sync or async commits - Fixes #25 https://github.com/confluentinc/parallel-consumer/issues/25: -- Sometimes a a transaction error occurs - Cannot call send in state COMMITTING_TRANSACTION #25 - ReentrantReadWrite lock protects non-thread safe transactional producer from incorrect multithreaded use - Wider lock to prevent transaction's containing produced messages that they shouldn't - Implement non transactional synchronous commit sync properly - Select tests adapted to non transactional as well - Must start tx in MockProducer as well - Adds supervision to poller - Fixes a performance issue with the async committer not being woken up - Enhances tests to run under multiple commit modes - Fixes example app tests - incorrectly testing wrong thing and MockProducer not configured to auto complete - Make committer thread revoke partitions and commit - Have onPartitionsRevoked be responsible for committing on close, instead of an explicit call to commit by controller - Make sure Broker Poller now drains properly, committing any waiting work - Add missing revoke flow to MockConsumer wrapper - Add missing latch timeout check --- .gitignore | 5 +- .idea/runConfigurations/All_examples.xml | 2 +- README.adoc | 59 ++-- .../io/confluent/csid/utils/StringUtils.java | 8 +- .../AbstractOffsetCommitter.java | 59 ++++ .../parallelconsumer/BrokerPollSystem.java | 192 +++++++---- .../parallelconsumer/ConsumerManager.java | 114 +++++++ .../ConsumerOffsetCommitter.java | 203 ++++++++++++ .../ErrorInUserFunctionException.java | 10 + .../parallelconsumer/InternalError.java | 20 ++ .../parallelconsumer/OffsetCommitter.java | 5 + .../ParallelConsumerOptions.java | 82 ++++- .../ParallelEoSStreamProcessor.java | 310 ++++++++--------- .../parallelconsumer/ProducerManager.java | 311 ++++++++++++++++++ .../parallelconsumer/UserFunctions.java | 38 +++ .../parallelconsumer/WorkManager.java | 53 +-- .../CloseAndOpenOffsetTest.java | 85 ++++- .../integrationTests/KafkaTest.java | 36 +- .../integrationTests/LoadTest.java | 6 +- .../TransactionAndCommitModeTest.java | 143 ++++++++ .../integrationTests/VolumeTests.java | 116 ++++--- .../utils/KafkaClientUtils.java | 7 +- .../confluent/csid/utils/KafkaTestUtils.java | 44 ++- .../csid/utils/LongPollingMockConsumer.java | 92 +++++- .../ParallelEoSStreamProcessorTest.java | 176 ++++++---- .../ParallelEoSStreamProcessorTestBase.java | 142 ++++++-- .../parallelconsumer/WorkManagerTest.java | 6 +- .../test/resources/junit-platform.properties | 2 +- .../src/test/resources/logback-test.xml | 23 +- .../examples/core/CoreApp.java | 50 +-- .../examples/core/CoreAppTest.java | 31 +- ...logback-test.xml => logback-temp-test.xml} | 2 +- .../examples/streams/StreamsApp.java | 11 +- .../examples/streams/StreamsAppTest.java | 2 +- .../parallel-consumer-example-vertx/pom.xml | 13 + .../examples/vertx/VertxApp.java | 12 +- .../examples/vertx/VertxAppTest.java | 29 +- ...StreamVertxParallelEoSStreamProcessor.java | 10 +- .../VertxParallelEoSStreamProcessor.java | 16 +- .../parallelconsumer/vertx/VertxTest.java | 12 +- src/docs/README.adoc | 29 +- 41 files changed, 2033 insertions(+), 533 deletions(-) create mode 100644 parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/AbstractOffsetCommitter.java create mode 100644 parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ConsumerManager.java create mode 100644 parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ConsumerOffsetCommitter.java create mode 100644 parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ErrorInUserFunctionException.java create mode 100644 parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/InternalError.java create mode 100644 parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/OffsetCommitter.java create mode 100644 parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ProducerManager.java create mode 100644 parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/UserFunctions.java create mode 100644 parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java rename parallel-consumer-examples/parallel-consumer-example-core/src/test/resources/{logback-test.xml => logback-temp-test.xml} (94%) diff --git a/.gitignore b/.gitignore index f5833c742..05e3ba15b 100644 --- a/.gitignore +++ b/.gitignore @@ -29,4 +29,7 @@ hs_err_pid* *.iml target/ .DS_Store -*.versionsBackup \ No newline at end of file +*.versionsBackup + +# JENV +.java-version \ No newline at end of file diff --git a/.idea/runConfigurations/All_examples.xml b/.idea/runConfigurations/All_examples.xml index ae2384feb..8c82d61b5 100644 --- a/.idea/runConfigurations/All_examples.xml +++ b/.idea/runConfigurations/All_examples.xml @@ -7,7 +7,7 @@

+ * Otherwise we can interrupt other operations like {@link KafkaConsumer#commitSync()}. + */ + public void wakeup() { + // boolean reduces the chances of a mis-timed call to wakeup, but doesn't prevent all spurious wake up calls to other methods like #commit + // if the call to wakeup happens /after/ the check for a wake up state inside #poll, then the next call will through the wake up exception (i.e. #commit) + if (pollingBroker.get()) { + log.debug("Waking up consumer"); + consumer.wakeup(); + } + } + + public void commitSync(final Map offsetsToSend) { + // we dont' want to be woken up during a commit, only polls + boolean inProgress = true; + noWakeups++; + while (inProgress) { + try { + consumer.commitSync(offsetsToSend); + inProgress = false; + } catch (WakeupException w) { + log.debug("Got woken up, retry. errors: " + erroneousWakups + " none: " + noWakeups + " correct:" + correctPollWakeups, w); + erroneousWakups++; + } + } + } + + public void commitAsync(Map offsets, OffsetCommitCallback callback) { + // we dont' want to be woken up during a commit, only polls + boolean inProgress = true; + noWakeups++; + while (inProgress) { + try { + consumer.commitAsync(offsets, callback); + inProgress = false; + } catch (WakeupException w) { + log.debug("Got woken up, retry. errors: " + erroneousWakups + " none: " + noWakeups + " correct:" + correctPollWakeups, w); + erroneousWakups++; + } + } + } + + public ConsumerGroupMetadata groupMetadata() { + return consumer.groupMetadata(); + } + + public void close(final Duration defaultTimeout) { + consumer.close(defaultTimeout); + } + + public Set assignment() { + return consumer.assignment(); + } + + public void pause(final Set assignment) { + consumer.pause(assignment); + } + + public Set paused() { + return consumer.paused(); + } + + public void resume(final Set pausedTopics) { + consumer.resume(pausedTopics); + } + +} diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ConsumerOffsetCommitter.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ConsumerOffsetCommitter.java new file mode 100644 index 000000000..bbe525851 --- /dev/null +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ConsumerOffsetCommitter.java @@ -0,0 +1,203 @@ +package io.confluent.parallelconsumer; + +import io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode; +import lombok.extern.slf4j.Slf4j; +import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; + +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.ReentrantLock; + +import static io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode.CONSUMER_SYNC; +import static io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode.TRANSACTIONAL_PRODUCER; + +/** + * Committer that uses the Kafka Consumer to commit either synchronously or asynchronously + * + * @see CommitMode + */ +@Slf4j +public class ConsumerOffsetCommitter extends AbstractOffsetCommitter implements OffsetCommitter { + + private final CommitMode commitMode; + + /** + * Used to synchronise threads on changing {@link #commitCount}, {@link #commitPerformed} and constructing the + * {@link Condition} to wait on, that's all. + */ + private final ReentrantLock commitLock = new ReentrantLock(true); + + /** + * Used to signal to waiting threads, that their commit has been performed when requested. + * + * @see #commitPerformed + * @see #commitAndWaitForCondition() + */ + private Condition commitPerformed = commitLock.newCondition(); + + /** + * The number of commits made. Use as a logical clock to sanity check expectations and synchronisation when commits + * are requested versus performed. + */ + private final AtomicLong commitCount = new AtomicLong(0); + + /** + * Set to true when a thread requests a commit to be performed, by the controling thread. + */ + private final AtomicBoolean commitRequested = new AtomicBoolean(false); + + private Optional owningThread = Optional.empty(); + + public ConsumerOffsetCommitter(final ConsumerManager newConsumer, final WorkManager newWorkManager, final ParallelConsumerOptions options) { + super(newConsumer, newWorkManager); + commitMode = options.getCommitMode(); + if (commitMode.equals(TRANSACTIONAL_PRODUCER)) { + throw new IllegalArgumentException("Cannot use " + commitMode + " when using " + this.getClass().getSimpleName()); + } + } + + // todo abstraction leak - find another way + private boolean direct = false; + + /** + * Might block if using {@link CommitMode#CONSUMER_SYNC} + * + * @see CommitMode + */ + void commit() { + if (isOwner()) { + // if owning thread is asking, then perform the commit directly (this is the thread that controls the consumer) + // this can happen when the system is closing, using Consumer commit mode, and partitions are revoked and we want to commit + direct = true; + retrieveOffsetsAndCommit(); + } else if (isSync()) { + log.debug("Sync commit"); + commitAndWaitForCondition(); + log.debug("Finished waiting"); + } else { + // async + // we just request the commit and hope + log.debug("Async commit to be requested"); + requestCommitInternal(); + } + } + + @Override + protected void commitOffsets(final Map offsetsToSend, final ConsumerGroupMetadata groupMetadata) { + if (offsetsToSend.isEmpty()) { + log.trace("Nothing to commit"); + return; + } + switch (commitMode) { + case CONSUMER_SYNC -> { + log.debug("Committing offsets Sync"); + consumerMgr.commitSync(offsetsToSend); + } + case CONSUMER_ASYNCHRONOUS -> { + // + log.debug("Committing offsets Async"); + consumerMgr.commitAsync(offsetsToSend, (offsets, exception) -> { + if (exception != null) { + log.error("Error committing offsets", exception); + // todo keep work in limbo until async response is received? + } + }); + } + default -> { + throw new IllegalArgumentException("Cannot use " + commitMode + " when using " + this.getClass().getSimpleName()); + } + } + } + + /** + * @see #commit() + */ + @Override + protected void postCommit() { + // only signal if we are in sync mode, and current thread isn't the owner (if we are owner, then we are committing directly) + if (!direct && commitMode.equals(CONSUMER_SYNC)) + signalCommitPerformed(); + } + + private boolean isOwner() { + return Thread.currentThread().equals(owningThread.orElse(null)); + } + + private void signalCommitPerformed() { + log.debug("Starting Signaling commit finished"); + if (!commitLock.isHeldByCurrentThread()) + throw new IllegalStateException("Lock already held"); + commitLock.lock(); + try { + commitCount.incrementAndGet(); + log.debug("Signaling commit finished"); + commitPerformed.signalAll(); + log.debug("Finished Signaling commit finished"); + } finally { + commitLock.unlock(); + } + } + + private void commitAndWaitForCondition() { + commitLock.lock(); + + try { + this.commitPerformed = commitLock.newCondition(); + long currentCount = commitCount.get(); + requestCommitInternal(); + consumerMgr.wakeup(); + while (currentCount == commitCount.get()) { + if (currentCount == commitCount.get()) { + log.debug("Requesting commit again"); + requestCommitInternal(); + } else { + commitRequested.set(false); + } + try { + log.debug("Waiting on commit"); + commitPerformed.await(); + } catch (InterruptedException e) { + log.debug("Interrupted waiting for commit condition", e); + } + } + log.debug("Signaled"); + } finally { + commitLock.unlock(); + } + } + + private void requestCommitInternal() { + commitLock.lock(); + try { + commitRequested.set(true); + consumerMgr.wakeup(); + } finally { + commitLock.unlock(); + } + + } + + void maybeDoCommit() { + commitLock.lock(); + try { + if (commitRequested.get()) { + retrieveOffsetsAndCommit(); + } + } finally { + commitLock.unlock(); + } + } + + public boolean isSync() { + return commitMode.equals(CONSUMER_SYNC); + } + + public void claim() { + owningThread = Optional.of(Thread.currentThread()); + } +} diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ErrorInUserFunctionException.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ErrorInUserFunctionException.java new file mode 100644 index 000000000..d41b51594 --- /dev/null +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ErrorInUserFunctionException.java @@ -0,0 +1,10 @@ +package io.confluent.parallelconsumer; + +/** + * This exception is only used when there is an exception thrown from code provided by the user. + */ +public class ErrorInUserFunctionException extends RuntimeException { + public ErrorInUserFunctionException(final String message, final Throwable cause) { + super(message, cause); + } +} diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/InternalError.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/InternalError.java new file mode 100644 index 000000000..71c15870d --- /dev/null +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/InternalError.java @@ -0,0 +1,20 @@ +package io.confluent.parallelconsumer; + +public class InternalError extends RuntimeException { + + public InternalError(final String message) { + super(message); + } + + public InternalError(final String message, final Throwable cause) { + super(message, cause); + } + + public InternalError(final Throwable cause) { + super(cause); + } + + public InternalError(final String message, final Throwable cause, final boolean enableSuppression, final boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } +} diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/OffsetCommitter.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/OffsetCommitter.java new file mode 100644 index 000000000..19d40baa0 --- /dev/null +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/OffsetCommitter.java @@ -0,0 +1,5 @@ +package io.confluent.parallelconsumer; + +public interface OffsetCommitter { + void retrieveOffsetsAndCommit(); +} diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java index a1ea68039..2654020b6 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java @@ -4,16 +4,27 @@ * Copyright (C) 2020 Confluent, Inc. */ +import io.confluent.csid.utils.StringUtils; import lombok.Builder; import lombok.Getter; +import lombok.ToString; + +import java.util.Properties; + +import static io.confluent.csid.utils.StringUtils.msg; +import static io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode.TRANSACTIONAL_PRODUCER; /** * The options for the {@link ParallelEoSStreamProcessor} system. */ @Getter -@Builder +@Builder(toBuilder = true) +@ToString public class ParallelConsumerOptions { + /** + * The ordering guarantee to use. + */ public enum ProcessingOrder { /** * No ordering is guaranteed, not even partition order. Fastest. Concurrency is at most the max number of @@ -21,8 +32,8 @@ public enum ProcessingOrder { */ UNORDERED, /** - * Process messages within a partition in order, but process multiple partitions in parallel. Similar to - * running more consumer for a topic. Concurrency is at most the number of partitions. + * Process messages within a partition in order, but process multiple partitions in parallel. Similar to running + * more consumer for a topic. Concurrency is at most the number of partitions. */ PARTITION, /** @@ -32,27 +43,88 @@ public enum ProcessingOrder { KEY } + /** + * The type of commit to be made, with either a transactions configured Producer where messages produced are + * committed back to the Broker along with the offsets they originated from, or with the faster simpler Consumer + * offset system either synchronously or asynchronously + */ + public enum CommitMode { + /** + * Commits through the Producer using transactions. Slowest fot he options, but no duplicates in Kafka + * guaranteed (message replay may cause duplicates in external systems which is unavoidable with Kafka). + */ + TRANSACTIONAL_PRODUCER, + /** + * Synchronous commits with the Consumer. Much faster than {@link #TRANSACTIONAL_PRODUCER}. Slower but + * potentially less duplicates than {@link #CONSUMER_ASYNCHRONOUS} upon replay. + */ + CONSUMER_SYNC, + /** + * Fastest option, under normal conditions will have few of no duplicates. Under failure revocery may have more + * duplicates than {@link #CONSUMER_SYNC}. + */ + CONSUMER_ASYNCHRONOUS + } + /** * The order type to use */ @Builder.Default private final ProcessingOrder ordering = ProcessingOrder.UNORDERED; + @Builder.Default + private final CommitMode commitMode = CommitMode.CONSUMER_ASYNCHRONOUS; + + /** + * When using a produce flow, when producing the message, either + *

+ * This is separate from using an IDEMPOTENT Producer, which can be used, along with {@link + * CommitMode#CONSUMER_SYNC} or {@link CommitMode#CONSUMER_ASYNCHRONOUS}. + *

+ * Must be set to true if being used with a transactional producer. Producers state. However, forcing it to be + * specified makes the choice more verbose? + *

+ * TODO we could just auto detect this from the + *

+ * TODO delete in favor of CommitMode + */ + @Builder.Default + private final boolean usingTransactionalProducer = false; + /** * Don't have more than this many uncommitted messages in process - * TODO change this to per topic? global? + *

+ * TODO docs - needs renaming. Messages aren't "uncommitted", they're just in the comitted offset map instead of the + * committed base offset */ @Builder.Default - private final int maxUncommittedMessagesToHandlePerPartition = 1000; + private final int maxUncommittedMessagesToHandle = 1000; /** * Don't process any more than this many messages concurrently + *

+ * TODO docs differentiate from thread count, vertx etc. remove to vertx module? */ @Builder.Default private final int maxConcurrency = 100; + /** + * TODO docs. rename to max concurrency. differentiate between this and vertx threads + */ @Builder.Default private final int numberOfThreads = 16; + // TODO remove - or include when checking passed property arguments instead of instances + // using reflection instead +// @Builder.Default +// private final Properties producerConfig = new Properties(); + public void validate() { + boolean commitModeIsTx = commitMode.equals(TRANSACTIONAL_PRODUCER); + if (this.isUsingTransactionalProducer() ^ commitModeIsTx) { + throw new IllegalArgumentException(msg("Using transaction producer mode ({}) without matching commit mode ({})", + this.isUsingTransactionalProducer(), + commitMode)); + } + } } 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 98ffa39e6..ca750b6bd 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 @@ -10,7 +10,6 @@ import org.apache.kafka.clients.consumer.*; import org.apache.kafka.clients.consumer.internals.ConsumerCoordinator; import org.apache.kafka.clients.producer.*; -import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.Time; import org.slf4j.MDC; @@ -32,6 +31,7 @@ import static io.confluent.csid.utils.BackportUtils.toSeconds; import static io.confluent.csid.utils.Range.range; import static io.confluent.csid.utils.StringUtils.msg; +import static io.confluent.parallelconsumer.UserFunctions.carefullyRun; import static java.time.Duration.ofSeconds; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.SECONDS; @@ -42,6 +42,8 @@ @Slf4j public class ParallelEoSStreamProcessor implements ParallelStreamProcessor, ConsumerRebalanceListener, Closeable { + private final ParallelConsumerOptions options; + /** * Injectable clock for testing */ @@ -54,9 +56,8 @@ public class ParallelEoSStreamProcessor implements ParallelStreamProcessor private Instant lastCommit = Instant.now(); - private boolean inTransaction = false; + final ProducerManager producerManager; - private final org.apache.kafka.clients.producer.Producer producer; private final org.apache.kafka.clients.consumer.Consumer consumer; /** @@ -95,6 +96,8 @@ public class ParallelEoSStreamProcessor implements ParallelStreamProcessor */ private final AtomicBoolean currentlyPollingWorkCompleteMailBox = new AtomicBoolean(); + private final OffsetCommitter committer; + /** * The run state of the controller. * @@ -122,47 +125,47 @@ enum State { * * @see ParallelConsumerOptions */ - public ParallelEoSStreamProcessor(org.apache.kafka.clients.consumer.Consumer consumer, - org.apache.kafka.clients.producer.Producer producer, - ParallelConsumerOptions options) { - log.debug("Confluent async consumer initialise"); + public ParallelEoSStreamProcessor(org.apache.kafka.clients.consumer.Consumer newConsumer, + org.apache.kafka.clients.producer.Producer newProducer, + ParallelConsumerOptions newOptions) { + log.info("Confluent Parallel Consumer initialise... Options: {}", newOptions); - Objects.requireNonNull(consumer); - Objects.requireNonNull(producer); - Objects.requireNonNull(options); + Objects.requireNonNull(newConsumer); + Objects.requireNonNull(newProducer); + Objects.requireNonNull(newOptions); - checkNotSubscribed(consumer); - checkAutoCommitIsDisabled(consumer); + options = newOptions; + options.validate(); - // - this.producer = producer; - this.consumer = consumer; + checkNotSubscribed(newConsumer); + checkAutoCommitIsDisabled(newConsumer); - workerPool = Executors.newFixedThreadPool(options.getNumberOfThreads()); + this.consumer = newConsumer; - // - this.wm = new WorkManager<>(options, consumer); + this.workerPool = Executors.newFixedThreadPool(newOptions.getNumberOfThreads()); - // - this.brokerPollSubsystem = new BrokerPollSystem<>(consumer, wm, this); + this.wm = new WorkManager<>(newOptions, newConsumer); - // - try { - log.debug("Initialising producer transaction session..."); - producer.initTransactions(); - } catch (KafkaException e) { - log.error("Make sure your producer is setup for transactions - specifically make sure it's {} is set.", ProducerConfig.TRANSACTIONAL_ID_CONFIG, e); - throw e; + ConsumerManager consumerMgr = new ConsumerManager<>(newConsumer); + this.producerManager = new ProducerManager<>(newProducer, consumerMgr, this.wm, options); + + this.brokerPollSubsystem = new BrokerPollSystem<>(consumerMgr, wm, this, newOptions); + + if (options.isUsingTransactionalProducer()) { + this.committer = this.producerManager; + } else { + this.committer = this.brokerPollSubsystem; } + } - private void checkNotSubscribed(org.apache.kafka.clients.consumer.Consumer consumer) { - if (consumer instanceof MockConsumer) + private void checkNotSubscribed(org.apache.kafka.clients.consumer.Consumer consumerToCheck) { + if (consumerToCheck instanceof MockConsumer) // disabled for unit tests which don't test rebalancing return; - Set subscription = consumer.subscription(); - Set assignment = consumer.assignment(); - if (subscription.size() != 0 || assignment.size() != 0) { + Set subscription = consumerToCheck.subscription(); + Set assignment = consumerToCheck.assignment(); + if (!subscription.isEmpty() || !assignment.isEmpty()) { throw new IllegalStateException("Consumer subscription must be managed by this class. Use " + this.getClass().getName() + "#subcribe methods instead."); } } @@ -195,12 +198,19 @@ public void subscribe(Pattern pattern, ConsumerRebalanceListener callback) { /** * Commit our offsets + *

+ * Make sure the calling thread is the thread which performs commit - i.e. is the {@link OffsetCommitter}. */ @Override public void onPartitionsRevoked(Collection partitions) { - commitOffsetsThatAreReady(); - wm.onPartitionsRevoked(partitions); - usersConsumerRebalanceListener.ifPresent(x -> x.onPartitionsRevoked(partitions)); + try { + log.debug("Partitions revoked (onPartitionsRevoked), state: {}", state); + commitOffsetsThatAreReady(); + wm.onPartitionsRevoked(partitions); + usersConsumerRebalanceListener.ifPresent(x -> x.onPartitionsRevoked(partitions)); + } catch (Exception e) { + throw new InternalError("onPartitionsRevoked event error", e); + } } /** @@ -255,7 +265,10 @@ private void checkAutoCommitIsDisabled(org.apache.kafka.clients.consumer.Consume public void poll(Consumer> usersVoidConsumptionFunction) { Function, List> wrappedUserFunc = (record) -> { log.trace("asyncPoll - Consumed a record ({}), executing void function...", record.offset()); - usersVoidConsumptionFunction.accept(record); + + carefullyRun(usersVoidConsumptionFunction, record); + + log.trace("asyncPoll - user function finished ok."); return UniLists.of(); // user function returns no produce records, so we satisfy our api }; Consumer voidCallBack = (ignore) -> log.trace("Void callback applied."); @@ -268,15 +281,17 @@ public void pollAndProduce(Function, List> callback) { // wrap user func to add produce function Function, List>> wrappedUserFunc = (consumedRecord) -> { - List> recordListToProduce = userFunction.apply(consumedRecord); + + List> recordListToProduce = carefullyRun(userFunction, consumedRecord); + if (recordListToProduce.isEmpty()) { log.warn("No result returned from function to send."); } - log.trace("asyncPoll and Stream - Consumed and a record ({}), and returning a derivative result record to be produced: {}", consumedRecord, recordListToProduce); + log.trace("asyncPoll and Stream - Consumed a record ({}), and returning a derivative result record to be produced: {}", consumedRecord, recordListToProduce); List> results = new ArrayList<>(); for (ProducerRecord toProduce : recordListToProduce) { - RecordMetadata produceResultMeta = produceMessage(toProduce); + RecordMetadata produceResultMeta = producerManager.produceMessage(toProduce); var result = new ConsumeProduceResult<>(consumedRecord, toProduce, produceResultMeta); results.add(result); } @@ -286,30 +301,6 @@ public void pollAndProduce(Function, List - * Implementation uses the blocking API, performance upgrade in later versions, is not an issue for the common use - * case ({@link #poll(Consumer)}). - * - * @see #pollAndProduce(Function, Consumer) - */ - RecordMetadata produceMessage(ProducerRecord outMsg) { - // only needed if not using tx - Callback callback = (RecordMetadata metadata, Exception exception) -> { - if (exception != null) { - log.error("Error producing result message", exception); - throw new RuntimeException("Error producing result message", exception); - } - }; - Future send = producer.send(outMsg, callback); - try { - return send.get(); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - @Override public void close() { // use a longer timeout, to cover for evey other step using the default @@ -325,20 +316,22 @@ public void close(Duration timeout, DrainingMode drainMode) { } else { log.info("Signaling to close..."); - switch (drainMode){ - case DRAIN: + switch (drainMode) { + case DRAIN -> { log.info("Will wait for all in flight to complete before"); transitionToDraining(); - case DONT_DRAIN: + } + case DONT_DRAIN -> { log.info("Not waiting for in flight to complete, will transition directly to closing"); transitionToClosing(); + } } waitForClose(timeout); } if (controlThreadFuture.isPresent()) { - log.trace("Checking for control thread exception..."); + log.debug("Checking for control thread exception..."); Future future = controlThreadFuture.get(); future.get(toSeconds(timeout), SECONDS); // throws exception if supervisor saw one } @@ -349,9 +342,10 @@ public void close(Duration timeout, DrainingMode drainMode) { private void waitForClose(Duration timeout) throws TimeoutException, ExecutionException { log.info("Waiting on closed state..."); while (!state.equals(closed)) { - log.trace("Still waiting for system to close..."); try { - boolean signaled = this.controlThreadFuture.get().get(toSeconds(timeout), SECONDS); + Future booleanFuture = this.controlThreadFuture.get(); + log.debug("Blocking on control future"); + boolean signaled = booleanFuture.get(toSeconds(timeout), SECONDS); if (!signaled) throw new TimeoutException("Timeout waiting for system to close (" + timeout + ")"); } catch (InterruptedException e) { @@ -361,6 +355,7 @@ private void waitForClose(Duration timeout) throws TimeoutException, ExecutionEx log.error("Execution or timeout exception", e); throw e; } + log.trace("Still waiting for system to close..."); } } @@ -371,12 +366,9 @@ private void doClose(Duration timeout) throws TimeoutException, ExecutionExcepti log.debug("Closing and waiting for broker poll system..."); brokerPollSubsystem.closeAndWait(); - log.debug("Closing producer, assuming no more in flight..."); - if (this.inTransaction) { - // close started after tx began, but before work was done, otherwise a tx wouldn't have been started - producer.abortTransaction(); - } - producer.close(timeout); + maybeCloseConsumer(); + + producerManager.close(timeout); log.debug("Shutting down execution pool..."); List unfinished = workerPool.shutdownNow(); @@ -387,7 +379,7 @@ private void doClose(Duration timeout) throws TimeoutException, ExecutionExcepti log.trace("Awaiting worker pool termination..."); boolean interrupted = true; while (interrupted) { - log.warn("Still interrupted"); + log.debug("Still interrupted"); try { boolean terminationFinishedWithoutTimeout = workerPool.awaitTermination(toSeconds(DrainingCloseable.DEFAULT_TIMEOUT), SECONDS); interrupted = false; @@ -406,26 +398,43 @@ private void doClose(Duration timeout) throws TimeoutException, ExecutionExcepti this.state = closed; } + /** + * To keep things simple, make sure the correct thread which can make a commit, is the one to close the consumer. + * This way, if partitions are revoked, the commit can be made inline. + */ + private void maybeCloseConsumer() { + if (isResponsibleForCommits()) { + consumer.close(); + } + } + + private boolean isResponsibleForCommits() { + return (committer instanceof ProducerManager); + } + /** * Block the calling thread until no more messages are being processed. */ @SneakyThrows - public void waitForNoInFlight(Duration timeout) { - log.debug("Waiting for no in flight..."); + public void waitForProcessedNotCommitted(Duration timeout) { + log.debug("Waiting processed but not commmitted..."); var timer = Time.SYSTEM.timer(timeout); - while (!noInFlight()) { - log.trace("Waiting for no in flight..."); + while (wm.isRecordsAwaitingToBeCommitted()) { + log.trace("Waiting for no in processing..."); Thread.sleep(100); timer.update(); if (timer.isExpired()) { - throw new TimeoutException("Waiting for no more records in-flight"); + throw new TimeoutException("Waiting for no more records in processing"); } } log.debug("No longer anything in flight."); } - private boolean noInFlight() { - return !wm.isWorkRemaining() || areMyThreadsDone(); + private boolean isRecordsAwaitingProcessing() { + boolean isRecordsAwaitingProcessing = wm.isRecordsAwaitingProcessing(); + boolean threadsDone = areMyThreadsDone(); + log.trace("isRecordsAwaitingProcessing {} || threadsDone {}", isRecordsAwaitingProcessing, threadsDone); + return isRecordsAwaitingProcessing || threadsDone; } private void transitionToDraining() { @@ -462,6 +471,8 @@ private boolean areMyThreadsDone() { */ protected void supervisorLoop(Function, List> userFunction, Consumer callback) { + log.info("Control loop starting up..."); + if (state != State.unused) { throw new IllegalStateException(msg("Invalid state - the consumer cannot be used more than once (current " + "state is {})", state)); @@ -469,26 +480,22 @@ protected void supervisorLoop(Function, List> userFu state = running; } - // - producer.beginTransaction(); - this.inTransaction = true; - // run main pool loop in thread Callable controlTask = () -> { - log.trace("Control task scheduled"); Thread controlThread = Thread.currentThread(); controlThread.setName("control"); + log.trace("Control task scheduled"); this.blockableControlThread = controlThread; while (state != closed) { try { controlLoop(userFunction, callback); } catch (Exception e) { - log.error("Error from poll control thread ({}), will attempt controlled shutdown, then rethrow...", e.getMessage(), e); + log.error("Error from poll control thread, will attempt controlled shutdown, then rethrow. Error: " + e.getMessage(), e); doClose(DrainingCloseable.DEFAULT_TIMEOUT); // attempt to close throw new RuntimeException("Error from poll control thread: " + e.getMessage(), e); } } - log.trace("Poll task ended (state:{}).", state); + log.info("Control loop ending clean (state:{})...", state); return true; }; @@ -503,8 +510,8 @@ protected void supervisorLoop(Function, List> userFu * Main control loop */ private void controlLoop(Function, List> userFunction, - Consumer callback) throws TimeoutException, ExecutionException { - if (state == running) { + Consumer callback) throws TimeoutException, ExecutionException, InterruptedException { + if (state == running || state == draining) { log.trace("Loop: Get work"); var records = wm.maybeGetWork(); @@ -512,11 +519,21 @@ private void controlLoop(Function, List> userFunctio submitWorkToPool(userFunction, callback, records); } + if (state == running) { + if (!wm.isSufficientlyLoaded()) { + log.debug("Found not enough messages queued up, ensuring poller is awake"); + brokerPollSubsystem.wakeup(); + } + } + log.trace("Loop: Process mailbox"); processWorkCompleteMailBox(); - log.trace("Loop: Maybe commit"); - commitOffsetsMaybe(); + if (state == running) { + // offsets will be committed when the consumer has its partitions revoked + log.trace("Loop: Maybe commit"); + commitOffsetsMaybe(); + } // run call back log.trace("Loop: Running {} loop end plugin(s)", controlLoopHooks.size()); @@ -524,10 +541,17 @@ private void controlLoop(Function, List> userFunctio log.debug("Current state: {}", state); switch (state) { - case draining -> drain(); - case closing -> doClose(DrainingCloseable.DEFAULT_TIMEOUT); + case draining -> { + drain(); + } + case closing -> { + doClose(DrainingCloseable.DEFAULT_TIMEOUT); + } } + // sanity - supervise the poller + brokerPollSubsystem.supervise(); + // end of loop log.trace("End of control loop, {} remaining in work manager. In state: {}", wm.getPartitionWorkRemainingCount(), state); } @@ -535,7 +559,7 @@ private void controlLoop(Function, List> userFunctio private void drain() { log.debug("Signaling to drain..."); brokerPollSubsystem.drain(); - if (noInFlight()) { + if (!isRecordsAwaitingProcessing()) { transitionToClosing(); } } @@ -563,7 +587,7 @@ private void processWorkCompleteMailBox() { // blocking get the head of the queue WorkContainer firstBlockingPoll = null; try { - log.debug("Blocking until next scheduled offset commit attempt for {}", timeout); + log.debug("Blocking poll on work until next scheduled offset commit attempt for {}", timeout); currentlyPollingWorkCompleteMailBox.getAndSet(true); // wait for work, with a timeout for sanity firstBlockingPoll = workMailBox.poll(timeout.toMillis(), MILLISECONDS); @@ -587,7 +611,7 @@ private void processWorkCompleteMailBox() { try { secondPollNonBlocking = workMailBox.poll(0, SECONDS); } catch (InterruptedException e) { - log.warn("Interrupted waiting on work results", e); + log.debug("Interrupted waiting on work results", e); } if (secondPollNonBlocking != null) { results.add(secondPollNonBlocking); @@ -610,7 +634,7 @@ private void commitOffsetsMaybe() { boolean commitFrequencyOK = toSeconds(elapsedSinceLast) >= toSeconds(timeBetweenCommits); if (commitFrequencyOK || lingeringOnCommitWouldBeBeneficial()) { if (!commitFrequencyOK) { - log.trace("Commit too frequent, but no benefit in lingering"); + log.debug("Commit too frequent, but no benefit in lingering"); } commitOffsetsThatAreReady(); lastCommit = Instant.now(); @@ -643,7 +667,7 @@ private Duration getTimeToNextCommit() { if (state == running) { return getTimeBetweenCommits().minus(getTimeSinceLastCommit()); } else { - log.debug("System not {}, so don't wait to commit", running); + log.debug("System not {} (state: {}), so don't wait to commit, only a small thread yield time", running, state); return Duration.ZERO; } } @@ -653,61 +677,8 @@ private Duration getTimeSinceLastCommit() { return Duration.between(lastCommit, now); } - /** - * Get offsets from {@link WorkManager} that are ready to commit - */ private void commitOffsetsThatAreReady() { - log.trace("Loop: Find completed work to commit offsets"); - // todo shouldn't be removed until commit succeeds (there's no harm in committing the same offset twice) - Map offsetsToSend = wm.findCompletedEligibleOffsetsAndRemove(); - if (offsetsToSend.isEmpty()) { - log.trace("No offsets ready"); - } else { - log.debug("Committing offsets for {} partition(s): {}", offsetsToSend.size(), offsetsToSend); - ConsumerGroupMetadata groupMetadata = consumer.groupMetadata(); - - producer.sendOffsetsToTransaction(offsetsToSend, groupMetadata); - // see {@link KafkaProducer#commit} this can be interrupted and is safe to retry - boolean notCommitted = true; - int retryCount = 0; - int arbitrarilyChosenLimitForArbitraryErrorSituation = 200; - Exception lastErrorSavedForRethrow = null; - while (notCommitted) { - if (retryCount > arbitrarilyChosenLimitForArbitraryErrorSituation) { - String msg = msg("Retired too many times ({} > limit of {}), giving up. See error above.", retryCount, arbitrarilyChosenLimitForArbitraryErrorSituation); - log.error(msg, lastErrorSavedForRethrow); - throw new RuntimeException(msg, lastErrorSavedForRethrow); - } - try { - if (producer instanceof MockProducer) { - // see bug https://issues.apache.org/jira/browse/KAFKA-10382 - // KAFKA-10382 - MockProducer is not ThreadSafe, ideally it should be as the implementation it mocks is - synchronized (producer) { - producer.commitTransaction(); - } - } else { - producer.commitTransaction(); - } - - this.inTransaction = false; - - wm.onOffsetCommitSuccess(offsetsToSend); - - notCommitted = false; - if (retryCount > 0) { - log.warn("Commit success, but took {} tries.", retryCount); - } - } catch (Exception e) { - log.warn("Commit exception, will retry, have tried {} times (see KafkaProducer#commit)", retryCount, e); - lastErrorSavedForRethrow = e; - retryCount++; - } - } - - // begin tx for next cycle - producer.beginTransaction(); - this.inTransaction = true; - } + committer.retrieveOffsetsAndCommit(); } protected void handleFutureResult(WorkContainer wc) { @@ -737,14 +708,16 @@ protected void onSuccess(WorkContainer wc) { private void submitWorkToPool(Function, List> usersFunction, Consumer callback, List> workToProcess) { - - for (var work : workToProcess) { - // for each record, construct dispatch to the executor and capture a Future - log.trace("Sending work ({}) to pool", work); - Future outputRecordFuture = workerPool.submit(() -> { - return userFunctionRunner(usersFunction, callback, work); - }); - work.setFuture(outputRecordFuture); + if (!workToProcess.isEmpty()) { + log.debug("New work incoming: {}, Pool stats: {}", workToProcess.size(), workerPool); + for (var work : workToProcess) { + // for each record, construct dispatch to the executor and capture a Future + log.trace("Sending work ({}) to pool", work); + Future outputRecordFuture = workerPool.submit(() -> { + return userFunctionRunner(usersFunction, callback, work); + }); + work.setFuture(outputRecordFuture); + } } } @@ -777,7 +750,7 @@ protected List, R>> userFunctionRunner(Function wc, List resultsFrom protected void addToMailbox(WorkContainer wc) { log.trace("Adding {} to mailbox...", wc); workMailBox.add(wc); + log.trace("Finished adding. {}", wc); } /** @@ -809,8 +783,12 @@ protected void addToMailbox(WorkContainer wc) { */ void notifyNewWorkRegistered() { if (currentlyPollingWorkCompleteMailBox.get()) { - log.trace("Interrupting control thread: Knock knock, wake up! You've got mail (tm)!"); - this.blockableControlThread.interrupt(); + if (!producerManager.isTransactionInProgress()) { + log.trace("Interrupting control thread: Knock knock, wake up! You've got mail (tm)!"); + this.blockableControlThread.interrupt(); + } else { + log.trace("Would have interrupted control thread, but TX in progress"); + } } else { log.trace("Work box not being polled currently, so thread not blocked, will come around to the bail box in the next looop."); } diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ProducerManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ProducerManager.java new file mode 100644 index 000000000..746c40674 --- /dev/null +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ProducerManager.java @@ -0,0 +1,311 @@ +package io.confluent.parallelconsumer; + +import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; +import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.producer.*; +import org.apache.kafka.clients.producer.internals.TransactionManager; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; + +import java.lang.reflect.Field; +import java.lang.reflect.Method; +import java.time.Duration; +import java.util.ConcurrentModificationException; +import java.util.Map; +import java.util.concurrent.Future; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import static io.confluent.csid.utils.StringUtils.msg; +import static io.confluent.parallelconsumer.ParallelEoSStreamProcessor.State.closing; + +@Slf4j +public class ProducerManager extends AbstractOffsetCommitter implements OffsetCommitter { + + protected final Producer producer; + + private final ParallelConsumerOptions options; + + private final boolean producerIsConfiguredForTransactions; + + /** + * The {@link KafkaProducer) isn't actually completely thread safe, at least when using it transactionally. We must + * be careful not to send messages to the producer, while we are committing a transaction - "Cannot call send in + * state COMMITTING_TRANSACTION". + */ + private ReentrantReadWriteLock producerTransactionLock; + + // nasty reflection + private Field txManagerField; + private Method txManagerMethodIsCompleting; + private Method txManagerMethodIsReady; + + public ProducerManager(final Producer newProducer, final ConsumerManager newConsumer, final WorkManager wm, ParallelConsumerOptions options) { + super(newConsumer, wm); + this.producer = newProducer; + this.options = options; + + producerIsConfiguredForTransactions = setupReflection(); + + initProducer(newProducer); + } + + private void initProducer(final Producer newProducer) { + producerTransactionLock = new ReentrantReadWriteLock(true); + + // String transactionIdProp = options.getProducerConfig().getProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG); + // boolean txIdSupplied = isBlank(transactionIdProp); + if (options.isUsingTransactionalProducer()) { + if (!producerIsConfiguredForTransactions) { + throw new IllegalArgumentException("Using transactional option, yet Producer doesn't have a transaction ID - Producer needs a transaction id"); + } + try { + log.debug("Initialising producer transaction session..."); + producer.initTransactions(); + producer.beginTransaction(); + } catch (KafkaException e) { + log.error("Make sure your producer is setup for transactions - specifically make sure it's {} is set.", ProducerConfig.TRANSACTIONAL_ID_CONFIG, e); + throw e; + } + } else { + if (producerIsConfiguredForTransactions) { + throw new IllegalArgumentException("Not using non-transactional option, but Producer has a transaction ID - Producer must not have a transaction ID for this option"); + } + } + } + + /** + * Nasty reflection but better than relying on user supplying their config + * + * @see ParallelEoSStreamProcessor#checkAutoCommitIsDisabled + */ + @SneakyThrows + private boolean getProducerIsTransactional() { + if (producer instanceof MockProducer) { + // can act as both, delegate to user selection + return options.isUsingTransactionalProducer(); + } else { + TransactionManager transactionManager = getTransactionManager(); + if (transactionManager == null) { + return false; + } else { + return transactionManager.isTransactional(); + } + } + } + + @SneakyThrows + private TransactionManager getTransactionManager() { + if (txManagerField == null) return null; + TransactionManager transactionManager = (TransactionManager) txManagerField.get(producer); + return transactionManager; + } + + /** + * Produce a message back to the broker. + *

+ * Implementation uses the blocking API, performance upgrade in later versions, is not an issue for the more common + * use case where messages aren't produced. + * + * @see ParallelConsumer#poll + * @see ParallelStreamProcessor#pollAndProduce + */ + RecordMetadata produceMessage(ProducerRecord outMsg) { + // only needed if not using tx + Callback callback = (RecordMetadata metadata, Exception exception) -> { + if (exception != null) { + log.error("Error producing result message", exception); + throw new RuntimeException("Error producing result message", exception); + } + }; + + ReentrantReadWriteLock.ReadLock readLock = producerTransactionLock.readLock(); + readLock.lock(); + Future send; + try { + send = producer.send(outMsg, callback); + } finally { + readLock.unlock(); + } + + // wait on the send results + try { + return send.get(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + protected void preAcquireWork() { + acquireCommitLock(); + } + + @Override + protected void postCommit() { + // only release lock when commit successful + if (producerTransactionLock.getWriteHoldCount() > 1) // sanity + throw new ConcurrentModificationException("Lock held too many times, won't be released problem and will cause deadlock"); + + releaseCommitLock(); + } + + @Override + protected void commitOffsets(final Map offsetsToSend, final ConsumerGroupMetadata groupMetadata) { + log.debug("Transactional offset commit starting"); + if (!options.isUsingTransactionalProducer()) { + throw new IllegalStateException("Bug: cannot use if not using transactional producer"); + } + + producer.sendOffsetsToTransaction(offsetsToSend, groupMetadata); + // see {@link KafkaProducer#commit} this can be interrupted and is safe to retry + boolean committed = false; + int retryCount = 0; + int arbitrarilyChosenLimitForArbitraryErrorSituation = 200; + Exception lastErrorSavedForRethrow = null; + while (!committed) { + if (retryCount > arbitrarilyChosenLimitForArbitraryErrorSituation) { + String msg = msg("Retired too many times ({} > limit of {}), giving up. See error above.", retryCount, arbitrarilyChosenLimitForArbitraryErrorSituation); + log.error(msg, lastErrorSavedForRethrow); + throw new RuntimeException(msg, lastErrorSavedForRethrow); + } + try { + if (producer instanceof MockProducer) { + // see bug https://issues.apache.org/jira/browse/KAFKA-10382 + // KAFKA-10382 - MockProducer is not ThreadSafe, ideally it should be as the implementation it mocks is + synchronized (producer) { + producer.commitTransaction(); + producer.beginTransaction(); + } + } else { + + // producer commit lock should already be acquired at this point, before work was retrieved to commit, + // so that more messages don't sneak into this tx block - the consumer records of which won't yet be + // in this offset collection + ensureLockHeld(); + + boolean retrying = retryCount > 0; + if (retrying) { + if (isTransactionCompleting()) { + // try wait again + producer.commitTransaction(); + } + if (isTransactionReady()) { + // tx has completed since we last tried, start a new one + producer.beginTransaction(); + } + boolean ready = (lastErrorSavedForRethrow != null) ? !lastErrorSavedForRethrow.getMessage().contains("Invalid transition attempted from state READY to state COMMITTING_TRANSACTION") : true; + if (ready) { + // try again + log.error("Was already ready - tx completed between interrupt and retry"); + } + } else { + // happy path + producer.commitTransaction(); + producer.beginTransaction(); + } + } + + committed = true; + if (retryCount > 0) { + log.warn("Commit success, but took {} tries.", retryCount); + } + } catch (Exception e) { + log.warn("Commit exception, will retry, have tried {} times (see KafkaProducer#commit)", retryCount, e); + lastErrorSavedForRethrow = e; + retryCount++; + } + } + } + + /** + * @return boolean which shows if we are setup for transactions or now + */ + @SneakyThrows + private boolean setupReflection() { + if (producer instanceof KafkaProducer) { + txManagerField = producer.getClass().getDeclaredField("transactionManager"); + txManagerField.setAccessible(true); + + boolean producerIsConfiguredForTransactions = getProducerIsTransactional(); + if (producerIsConfiguredForTransactions) { + TransactionManager transactionManager = getTransactionManager(); + txManagerMethodIsCompleting = transactionManager.getClass().getDeclaredMethod("isCompleting"); + txManagerMethodIsCompleting.setAccessible(true); + + txManagerMethodIsReady = transactionManager.getClass().getDeclaredMethod("isReady"); + txManagerMethodIsReady.setAccessible(true); + } + return producerIsConfiguredForTransactions; + } else if (producer instanceof MockProducer) { + // can act as both, delegate to user selection + return options.isUsingTransactionalProducer(); + } else { + // unknown + return false; + } + } + + /** + * TODO talk about alternatives to this brute force approach for retrying committing transactions + */ + @SneakyThrows + private boolean isTransactionCompleting() { + if (producer instanceof MockProducer) return false; + return (boolean) txManagerMethodIsCompleting.invoke(getTransactionManager()); + } + + /** + * TODO talk about alternatives to this brute force approach for retrying committing transactions + */ + @SneakyThrows + private boolean isTransactionReady() { + if (producer instanceof MockProducer) return true; + return (boolean) txManagerMethodIsReady.invoke(getTransactionManager()); + } + + /** + * Assumes the system is drained at this point, or draining is not desired. + */ + public void close(final Duration timeout) { + log.debug("Closing producer, assuming no more in flight..."); + if (options.isUsingTransactionalProducer() && !isTransactionReady()) { + acquireCommitLock(); + try { + // close started after tx began, but before work was done, otherwise a tx wouldn't have been started + producer.abortTransaction(); + } finally { + releaseCommitLock(); + } + } + producer.close(timeout); + } + + private void acquireCommitLock() { + if (producerTransactionLock.getWriteHoldCount() > 0) + throw new ConcurrentModificationException("Lock already held"); + ReentrantReadWriteLock.WriteLock writeLock = producerTransactionLock.writeLock(); + if (producerTransactionLock.isWriteLocked() && !producerTransactionLock.isWriteLockedByCurrentThread()) { + throw new ConcurrentModificationException(this.getClass().getSimpleName() + " is not safe for multi-threaded access"); + } + writeLock.lock(); + } + + private void releaseCommitLock() { + log.trace("Release commit lock"); + ReentrantReadWriteLock.WriteLock writeLock = producerTransactionLock.writeLock(); + if (!producerTransactionLock.isWriteLockedByCurrentThread()) + throw new IllegalStateException("Not held be me"); + writeLock.unlock(); + } + + private void ensureLockHeld() { + if (!producerTransactionLock.isWriteLockedByCurrentThread()) + throw new IllegalStateException("Expected commit lock to be held"); + } + + public boolean isTransactionInProgress() { + return producerTransactionLock.isWriteLocked(); + } +} diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/UserFunctions.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/UserFunctions.java new file mode 100644 index 000000000..45c26cb8b --- /dev/null +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/UserFunctions.java @@ -0,0 +1,38 @@ +package io.confluent.parallelconsumer; + +import lombok.experimental.UtilityClass; + +import java.util.function.BiFunction; +import java.util.function.Consumer; +import java.util.function.Function; + +@UtilityClass +public class UserFunctions { + + public static final String MSG = "Error occurred in code supplied by user"; + + public static R carefullyRun(BiFunction wrappedFunction, T t, U u) { + try { + return wrappedFunction.apply(t, u); + } catch (Exception e) { + throw new ErrorInUserFunctionException(MSG, e); + } + } + + public static B carefullyRun(Function wrappedFunction, A a) { + try { + return wrappedFunction.apply(a); + } catch (Exception e) { + throw new ErrorInUserFunctionException(MSG, e); + } + } + + public static void carefullyRun(Consumer wrappedFunction, A a) { + try { + wrappedFunction.accept(a); + } catch (Exception e) { + throw new ErrorInUserFunctionException(MSG, e); + } + } + +} diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/WorkManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/WorkManager.java index 0f360ddae..fe54543b1 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/WorkManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/WorkManager.java @@ -19,9 +19,11 @@ import pl.tlinkowski.unij.api.UniLists; import pl.tlinkowski.unij.api.UniSets; +import java.time.Duration; import java.util.*; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Consumer; import static io.confluent.csid.utils.KafkaUtils.toTP; @@ -73,10 +75,13 @@ public class WorkManager implements ConsumerRebalanceListener { private int inFlightCount = 0; /** - * The multiple of {@link ParallelConsumerOptions#getMaxConcurrency()} that should be pre-loaded awaiting processing. - * Consumer already pipelines, so we shouldn't need to pipeline ourselves too much. + * The multiple of {@link ParallelConsumerOptions#getMaxConcurrency()} that should be pre-loaded awaiting + * processing. Consumer already pipelines, so we shouldn't need to pipeline ourselves too much. + *

+ * Note how this relates to {@link BrokerPollSystem#getLongPollTimeout()} - if longPollTimeout is high and loading + * factor is low, there may not be enough messages queued up to satisfy demand. */ - private final int loadingFactor = 2; + private final int loadingFactor = 3; /** * Useful for testing @@ -262,7 +267,7 @@ public List> maybeGetWork() { * @param requestedMaxWorkToRetrieve ignored unless less than {@link ParallelConsumerOptions#maxConcurrency} */ public List> maybeGetWork(int requestedMaxWorkToRetrieve) { - int minWorkToGetSetting = min(min(requestedMaxWorkToRetrieve, options.getMaxConcurrency()), options.getMaxUncommittedMessagesToHandlePerPartition()); + int minWorkToGetSetting = min(min(requestedMaxWorkToRetrieve, options.getMaxConcurrency()), options.getMaxUncommittedMessagesToHandle()); int workToGetDelta = minWorkToGetSetting - getInFlightCount(); // optimise early @@ -321,8 +326,9 @@ public List> maybeGetWork(int requestedMaxWorkToRetrieve) { work.addAll(shardWork); } - log.debug("Returning {} records of work", work.size()); + log.debug("Got {} records of work", work.size()); inFlightCount += work.size(); + return work; } @@ -367,8 +373,14 @@ public int getShardWorkRemainingCount() { return count; } - boolean isWorkRemaining() { - return getPartitionWorkRemainingCount() > 0; + boolean isRecordsAwaitingProcessing() { + int partitionWorkRemainingCount = getShardWorkRemainingCount(); + return partitionWorkRemainingCount > 0; + } + + boolean isRecordsAwaitingToBeCommitted() { + int partitionWorkRemainingCount = getPartitionWorkRemainingCount(); + return partitionWorkRemainingCount > 0; } public WorkContainer getWorkContainerForRecord(ConsumerRecord rec) { @@ -388,7 +400,8 @@ boolean hasComittableOffsets() { } /** - * TODO: This entire loop could be possibly redundant, if we instead track low water mark, and incomplete offsets as work is submitted and returned. + * TODO: This entire loop could be possibly redundant, if we instead track low water mark, and incomplete offsets as + * work is submitted and returned. */ @SneakyThrows Map findCompletedEligibleOffsetsAndRemove(boolean remove) { @@ -480,7 +493,8 @@ Map findCompletedEligibleOffsetsAndRemove /** * Once all the offset maps have been calculated, check if they're too big, and if so, remove all of them. *

- * Implication of this is that if the system has to recover from this offset, then it will have to replay all the messages that were otherwise complete. + * Implication of this is that if the system has to recover from this offset, then it will have to replay all the + * messages that were otherwise complete. * * @see OffsetMapCodecManager#DefaultMaxMetadataSize */ @@ -491,7 +505,7 @@ private void maybeStripOffsetPayload(Map offs // retaining the offset map feature, at the cost of potential performance by hitting a soft maximum in our uncommitted concurrent processing. if (totalOffsetMetaSize > OffsetMapCodecManager.DefaultMaxMetadataSize) { log.warn("Offset map data too large (size: {}) to fit in metadata payload - stripping offset map out. " + - "See kafka.coordinator.group.OffsetConfig#DefaultMaxMetadataSize = 4096", + "See kafka.coordinator.group.OffsetConfig#DefaultMaxMetadataSize = 4096", totalOffsetMetaSize); // strip payload for (var entry : offsetsToSend.entrySet()) { @@ -507,7 +521,8 @@ private void maybeStripOffsetPayload(Map offs * Truncate our tracked offsets as a commit was successful, so the low water mark rises, and we dont' need to track * as much anymore. *

- * When commits are made to broker, we can throw away all the individually tracked offsets before the committed offset. + * When commits are made to broker, we can throw away all the individually tracked offsets before the committed + * offset. */ public void onOffsetCommitSuccess(Map offsetsToSend) { // partitionOffsetHighWaterMarks this will get overwritten in due course @@ -522,18 +537,18 @@ public void onOffsetCommitSuccess(Map offsets } public boolean shouldThrottle() { - return isOverMax(); + return isSufficientlyLoaded(); } - private boolean isOverMax() { + boolean isSufficientlyLoaded() { int remaining = getPartitionWorkRemainingCount(); - boolean loadedEnough = remaining > options.getMaxConcurrency() * loadingFactor; - boolean overMaxInFlight = remaining > options.getMaxUncommittedMessagesToHandlePerPartition(); - boolean isOverMax = loadedEnough || overMaxInFlight; - if (isOverMax) { - log.debug("loadedEnough {} || overMaxInFlight {}", loadedEnough, overMaxInFlight); + boolean loadedEnoughInPipeline = remaining > options.getMaxConcurrency() * loadingFactor; + boolean overMaxUncommitted = remaining > options.getMaxUncommittedMessagesToHandle(); + boolean remainingIsSufficient = loadedEnoughInPipeline || overMaxUncommitted; + if (remainingIsSufficient) { + log.debug("loadedEnoughInPipeline {} || overMaxUncommitted {}", loadedEnoughInPipeline, overMaxUncommitted); } - return isOverMax; + return remainingIsSufficient; } public int getInFlightCount() { diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/CloseAndOpenOffsetTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/CloseAndOpenOffsetTest.java index bb19351af..47c26403a 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/CloseAndOpenOffsetTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/CloseAndOpenOffsetTest.java @@ -4,6 +4,8 @@ */ package io.confluent.parallelconsumer.integrationTests; +import io.confluent.csid.utils.KafkaTestUtils; +import io.confluent.parallelconsumer.ParallelConsumer; import io.confluent.parallelconsumer.ParallelEoSStreamProcessor; import io.confluent.parallelconsumer.ParallelConsumerOptions; import io.confluent.parallelconsumer.OffsetMapCodecManager; @@ -11,12 +13,11 @@ import io.confluent.csid.utils.Range; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.*; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.TopicPartition; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -25,12 +26,16 @@ import pl.tlinkowski.unij.api.UniSets; import java.time.Duration; -import java.util.ArrayList; -import java.util.Set; +import java.util.*; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import static io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode.TRANSACTIONAL_PRODUCER; +import static io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder.UNORDERED; import static java.time.Duration.ofSeconds; import static org.assertj.core.api.Assertions.assertThat; import static org.awaitility.Awaitility.await; @@ -43,7 +48,7 @@ public class CloseAndOpenOffsetTest extends KafkaTest { Duration normalTimeout = ofSeconds(5); - Duration debugTimeout = Duration.ofMinutes(10); + Duration debugTimeout = Duration.ofMinutes(1); // use debug timeout while debugging // Duration timeoutToUse = debugTimeout; @@ -56,6 +61,14 @@ void setup() { rebalanceTopic = "close-and-open-" + RandomUtils.nextInt(); } + /** + * publish some messages some fail shutdown startup again consume again - check we only consume the failed messages + *

+ * Sometimes fails as 5 is not comitted in the first run and comes out in the 2nd + *

+ * NB: messages 4 and 2 are made to fail + */ + @Timeout(value = 60) @SneakyThrows @Test void offsetsOpenClose() { @@ -67,7 +80,11 @@ void offsetsOpenClose() { } // 1 client - ParallelConsumerOptions options = ParallelConsumerOptions.builder().ordering(ParallelConsumerOptions.ProcessingOrder.UNORDERED).build(); + ParallelConsumerOptions options = ParallelConsumerOptions.builder() + .ordering(UNORDERED) + .usingTransactionalProducer(true) + .commitMode(TRANSACTIONAL_PRODUCER) + .build(); kcu.props.put(ConsumerConfig.CLIENT_ID_CONFIG, "ONE-my-client"); KafkaConsumer newConsumerOne = kcu.createNewConsumer(); @@ -79,7 +96,7 @@ void offsetsOpenClose() { asyncOne.subscribe(UniLists.of(rebalanceTopic)); // read some messages - var readByOne = new ArrayList>(); + var readByOne = new ConcurrentLinkedQueue>(); asyncOne.poll(x -> { log.info("Read by consumer ONE: {}", x); if (x.value().equals("4")) { @@ -93,6 +110,9 @@ void offsetsOpenClose() { readByOne.add(x); }); + // wait for initial 0 commit + Thread.sleep(500); + // send(rebalanceTopic, 0, 0); send(rebalanceTopic, 0, 1); @@ -102,12 +122,33 @@ void offsetsOpenClose() { send(rebalanceTopic, 0, 5); // all are processed except msg 2 and 4, which holds up the queue - await().atMost(debugTimeout).untilAsserted(() -> assertThat(readByOne).hasSize(4)); + await().alias("check all except 2 and 4 are processed").atMost(normalTimeout).untilAsserted(() -> { + ArrayList> copy = new ArrayList<>(readByOne); + assertThat(copy.stream() + .map(x -> x.value()).collect(Collectors.toList())) + .containsOnly("0", "1", "3", "5"); + } + ); + + // wait until all expected records have been processed and committed + // need to wait for final message processing's offset data to be committed + // TODO test for event/trigger instead - could consume offsets topic but have to decode the binary + // could listen to a produce topic, but currently it doesn't use the produce flow + // could add a commit listener to the api, but that's heavy just for this? + // could use Consumer#committed to check and decode, but it's not thread safe + // sleep is lazy but much much simpler + Thread.sleep(500); // commit what we've done so far, don't wait for failing messages to be retried (message 4) log.info("Closing consumer, committing offset map"); asyncOne.closeDontDrainFirst(); + await().alias("check all except 2 and 4 are processed").atMost(normalTimeout).untilAsserted(() -> + assertThat(readByOne.stream() + .map(x -> x.value()).collect(Collectors.toList())) + .containsOnly("0", "1", "3", "5")); + + // kcu.props.put(ConsumerConfig.CLIENT_ID_CONFIG, "THREE-my-client"); KafkaConsumer newConsumerThree = kcu.createNewConsumer(); @@ -116,14 +157,16 @@ void offsetsOpenClose() { asyncThree.subscribe(UniLists.of(rebalanceTopic)); // read what we're given - var readByThree = new ArrayList>(); + var readByThree = new ConcurrentLinkedQueue>(); asyncThree.poll(x -> { log.info("Read by consumer THREE: {}", x.value()); readByThree.add(x); }); // only 2 and 4 should be delivered again, as everything else was processed successfully - await().atMost(timeoutToUse).untilAsserted(() -> assertThat(readByThree).extracting(ConsumerRecord::value).containsExactlyInAnyOrder("2", "4")); + await().atMost(timeoutToUse).untilAsserted(() -> + assertThat(readByThree).extracting(ConsumerRecord::value) + .containsExactlyInAnyOrder("2", "4")); } } @@ -160,23 +203,33 @@ void correctOffsetVerySimple() { KafkaConsumer consumer = kcu.createNewConsumer(); KafkaProducer producerOne = kcu.createNewProducer(true); - ParallelConsumerOptions options = ParallelConsumerOptions.builder().ordering(ParallelConsumerOptions.ProcessingOrder.UNORDERED).build(); + ParallelConsumerOptions options = ParallelConsumerOptions.builder() + .ordering(UNORDERED) + .usingTransactionalProducer(true) + .commitMode(TRANSACTIONAL_PRODUCER) + .build(); try (var asyncOne = new ParallelEoSStreamProcessor<>(consumer, producerOne, options)) { asyncOne.subscribe(UniLists.of(topic)); var readByOne = new ArrayList>(); - asyncOne.poll(readByOne::add); + asyncOne.poll(msg -> { + log.debug("Reading {}", msg); + readByOne.add(msg); + }); // the single message is processed await().untilAsserted(() -> assertThat(readByOne) .extracting(ConsumerRecord::value) .containsExactly("0")); + } finally { + log.debug("asyncOne closed"); } // + log.debug("Starting up new client"); kcu.props.put(ConsumerConfig.CLIENT_ID_CONFIG, "THREE-my-client"); KafkaConsumer newConsumerThree = kcu.createNewConsumer(); KafkaProducer producerThree = kcu.createNewProducer(true); @@ -220,7 +273,11 @@ void largeNumberOfMessagesSmallOffsetBitmap() { KafkaConsumer consumer = kcu.createNewConsumer(); KafkaProducer producerOne = kcu.createNewProducer(true); - ParallelConsumerOptions options = ParallelConsumerOptions.builder().ordering(ParallelConsumerOptions.ProcessingOrder.UNORDERED).build(); + ParallelConsumerOptions options = ParallelConsumerOptions.builder() + .ordering(UNORDERED) + .usingTransactionalProducer(true) + .commitMode(TRANSACTIONAL_PRODUCER) + .build(); var asyncOne = new ParallelEoSStreamProcessor<>(consumer, producerOne, options); asyncOne.subscribe(UniLists.of(topic)); diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/KafkaTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/KafkaTest.java index 43e7afb2e..14501362b 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/KafkaTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/KafkaTest.java @@ -6,7 +6,6 @@ import io.confluent.parallelconsumer.integrationTests.utils.KafkaClientUtils; import io.confluent.csid.testcontainers.FilteredTestContainerSlf4jLogConsumer; -import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.admin.CreateTopicsResult; import org.apache.kafka.clients.admin.NewTopic; @@ -18,11 +17,14 @@ import org.testcontainers.utility.DockerImageName; import pl.tlinkowski.unij.api.UniLists; +import java.util.concurrent.ExecutionException; + import static org.apache.commons.lang3.RandomUtils.nextInt; import static org.assertj.core.api.Assertions.assertThat; @Testcontainers @Slf4j +// TODO rename to broker integration test public abstract class KafkaTest { int numPartitions = 2; // as default consumer settings are max request 50 max per partition 1 - 50/1=50 @@ -37,6 +39,8 @@ public abstract class KafkaTest { .withEnv("KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR", "1") //transaction.state.log.replication.factor .withEnv("KAFKA_TRANSACTION_STATE_LOG_MIN_ISR", "1") //transaction.state.log.min.isr .withEnv("KAFKA_TRANSACTION_STATE_LOG_NUM_PARTITIONS", "1") //transaction.state.log.num.partitions + // try to speed up initial consumer group formation + .withEnv("KAFKA_GROUP_INITIAL_REBALANCE_DELAY_MS", "500") // group.initial.rebalance.delay.ms default: 3000 .withReuse(true); static { @@ -45,13 +49,13 @@ public abstract class KafkaTest { protected KafkaClientUtils kcu = new KafkaClientUtils(kafkaContainer); - @BeforeAll - static void followKafkaLogs(){ - if(log.isDebugEnabled()) { - FilteredTestContainerSlf4jLogConsumer logConsumer = new FilteredTestContainerSlf4jLogConsumer(log); - kafkaContainer.followOutput(logConsumer); + @BeforeAll + static void followKafkaLogs() { + if (log.isDebugEnabled()) { + FilteredTestContainerSlf4jLogConsumer logConsumer = new FilteredTestContainerSlf4jLogConsumer(log); + kafkaContainer.followOutput(logConsumer); + } } - } @BeforeEach void open() { @@ -64,18 +68,30 @@ void close() { } void setupTopic() { + String name = LoadTest.class.getSimpleName(); + setupTopic(name); + } + + protected String setupTopic(String name) { assertThat(kafkaContainer.isRunning()).isTrue(); // sanity - topic = LoadTest.class.getSimpleName() + "-" + nextInt(); + topic = name + "-" + nextInt(); ensureTopic(topic, numPartitions); + + return name; } - @SneakyThrows protected void ensureTopic(String topic, int numPartitions) { NewTopic e1 = new NewTopic(topic, numPartitions, (short) 1); CreateTopicsResult topics = kcu.admin.createTopics(UniLists.of(e1)); - Void all = topics.all().get(); + try { + Void all = topics.all().get(); + } catch (ExecutionException e) { + // fine + } catch (Exception e) { + throw new RuntimeException(e); + } } } diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/LoadTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/LoadTest.java index 0e724c1f9..0bebc18bf 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/LoadTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/LoadTest.java @@ -30,6 +30,7 @@ import static io.confluent.csid.utils.GeneralTestUtils.time; import static io.confluent.csid.utils.Range.range; +import static io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode.TRANSACTIONAL_PRODUCER; import static java.time.Duration.*; import static me.tongfei.progressbar.ProgressBar.wrap; import static org.apache.commons.lang3.RandomUtils.nextInt; @@ -72,10 +73,11 @@ void asyncConsumeAndProcess() { kcu.consumer.subscribe(UniLists.of(topic)); // - ParallelConsumerOptions options = ParallelConsumerOptions.builder().ordering(ParallelConsumerOptions.ProcessingOrder.KEY).build(); + boolean tx = true; + ParallelConsumerOptions options = ParallelConsumerOptions.builder().ordering(ParallelConsumerOptions.ProcessingOrder.KEY).usingTransactionalProducer(tx).commitMode(TRANSACTIONAL_PRODUCER).build(); KafkaConsumer newConsumer = kcu.createNewConsumer(); newConsumer.subscribe(Pattern.compile(topic)); - var async = new ParallelEoSStreamProcessor<>(newConsumer, kcu.createNewProducer(true), options); + var async = new ParallelEoSStreamProcessor<>(newConsumer, kcu.createNewProducer(tx), options); AtomicInteger msgCount = new AtomicInteger(0); ProgressBar pb = new ProgressBarBuilder() diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java new file mode 100644 index 000000000..d61d1af1d --- /dev/null +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java @@ -0,0 +1,143 @@ +package io.confluent.parallelconsumer.integrationTests; + +/*- + * Copyright (C) 2020 Confluent, Inc. + */ + +import io.confluent.parallelconsumer.ParallelConsumerOptions; +import io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode; +import io.confluent.parallelconsumer.ParallelEoSStreamProcessor; +import io.confluent.parallelconsumer.integrationTests.KafkaTest; +import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.awaitility.core.ConditionTimeoutException; +import org.junit.jupiter.api.RepeatedTest; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; +import pl.tlinkowski.unij.api.UniLists; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; + +import static io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode.TRANSACTIONAL_PRODUCER; +import static java.time.Duration.ofSeconds; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; +import static org.awaitility.Awaitility.waitAtMost; + +/** + * Originally created to reproduce bug #25 https://github.com/confluentinc/parallel-consumer/issues/25 which was a known + * issue with multi threaded use of the {@link org.apache.kafka.clients.producer.KafkaProducer}. + *

+ * After fixing multi threading issue, using Producer transactions was made optional, and this test grew to uncover + * several issues with the new implementation of committing offsets through the {@link + * org.apache.kafka.clients.consumer.KafkaConsumer}. + * + * @see io.confluent.parallelconsumer.OffsetCommitter + * @see io.confluent.parallelconsumer.ConsumerOffsetCommitter + * @see io.confluent.parallelconsumer.ProducerManager + */ +@Slf4j +public class TransactionAndCommitModeTest extends KafkaTest { + + int LOW_MAX_POLL_RECORDS_CONFIG = 1; + int DEFAULT_MAX_POLL_RECORDS_CONFIG = 500; + int HIGH_MAX_POLL_RECORDS_CONFIG = 10_000; + + public List processedAndProducedKeys = Collections.synchronizedList(new ArrayList<>()); + public AtomicInteger processedCount = new AtomicInteger(0); + public AtomicInteger producedCount = new AtomicInteger(0); + + // default + @ParameterizedTest() + @EnumSource(CommitMode.class) + public void testDefaultMaxPoll(CommitMode commitMode) { + runTest(DEFAULT_MAX_POLL_RECORDS_CONFIG, commitMode); + } + + @RepeatedTest(5) + public void testTransactionalDefaultMaxPoll() { + runTest(DEFAULT_MAX_POLL_RECORDS_CONFIG, TRANSACTIONAL_PRODUCER); + } + + // low + @ParameterizedTest() + @EnumSource(CommitMode.class) + public void testLowMaxPoll(CommitMode commitMode) { + runTest(LOW_MAX_POLL_RECORDS_CONFIG, commitMode); + } + + // high counts + @ParameterizedTest() + @EnumSource(CommitMode.class) + public void testHighMaxPollEnum(CommitMode commitMode) { + runTest(HIGH_MAX_POLL_RECORDS_CONFIG, commitMode); + } + + private void runTest(int maxPoll, CommitMode commitMode) { + boolean tx = commitMode.equals(TRANSACTIONAL_PRODUCER); + runTest(tx, maxPoll, commitMode); + } + + @SneakyThrows + private void runTest(boolean tx, int maxPoll, CommitMode commitMode) { + String inputName = setupTopic(this.getClass().getSimpleName() + "-input"); + String outputName = setupTopic(this.getClass().getSimpleName() + "-input"); + + // pre-produce messages to input-topic + List expectedKeys = new ArrayList<>(); + int expectedMessageCount = 1000; + log.info("Producing {} messages before starting application", expectedMessageCount); + try (Producer kafkaProducer = kcu.createNewProducer(false)) { + for (int i = 0; i < expectedMessageCount; i++) { + String key = "key-" + i; + kafkaProducer.send(new ProducerRecord<>(inputName, key, "value-" + i)); + expectedKeys.add(key); + } + } + + // run parallel-consumer + log.info("Starting application..."); + ParallelEoSStreamProcessor pc = new ParallelEoSStreamProcessor<>(kcu.consumer, kcu.producer, ParallelConsumerOptions.builder().build()); + pc.subscribe(UniLists.of(inputName)); // <5> + + pc.pollAndProduce(record -> { + processedCount.incrementAndGet(); + return UniLists.of(new ProducerRecord<>(outputName, record.key(), "data")); + }, consumeProduceResult -> { + producedCount.incrementAndGet(); + processedAndProducedKeys.add(consumeProduceResult.getIn().key()); + } + ); + + // wait for all pre-produced messages to be processed and produced + String failureMessage = "All keys sent to input-topic should be processed and produced"; + try { + waitAtMost(ofSeconds(10)).alias(failureMessage).untilAsserted(() -> { + log.debug("Processed-count: " + processedCount.get()); + log.debug("Produced-count: " + producedCount.get()); + List processedAndProducedKeysCopy = new ArrayList<>(processedAndProducedKeys); // avoid concurrent-modification in assert + assertThat(processedAndProducedKeysCopy).contains(expectedKeys.toArray(new String[0])); + }); + } catch (ConditionTimeoutException e) { + log.debug("Expected keys=" + expectedKeys + ""); + log.debug("Processed and produced keys=" + processedAndProducedKeys + ""); + expectedKeys.removeAll(processedAndProducedKeys); + log.debug("Missing keys=" + expectedKeys); + fail(failureMessage); + } + + pc.closeDrainFirst(); + + assertThat(processedCount.get()) + .as("messages processed and produced by parallel-consumer should be equal") + .isEqualTo(producedCount.get()); + + pc.close(); + } + +} diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/VolumeTests.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/VolumeTests.java index 7c6a07612..df0f3c5c1 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/VolumeTests.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/VolumeTests.java @@ -4,6 +4,8 @@ */ package io.confluent.parallelconsumer.integrationTests; +import io.confluent.parallelconsumer.ParallelConsumerOptions; +import io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode; import io.confluent.parallelconsumer.ParallelEoSStreamProcessorTestBase; import io.confluent.csid.utils.KafkaTestUtils; import lombok.SneakyThrows; @@ -16,8 +18,8 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.TopicPartition; import org.assertj.core.util.Lists; -import org.awaitility.Awaitility; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; import org.testcontainers.shaded.com.google.common.collect.Maps; import pl.tlinkowski.unij.api.UniLists; @@ -25,13 +27,17 @@ import java.util.*; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; +import java.util.stream.Collectors; +import static io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode.CONSUMER_SYNC; +import static io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode.TRANSACTIONAL_PRODUCER; import static io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder.*; import static io.confluent.csid.utils.GeneralTestUtils.time; import static io.confluent.csid.utils.Range.range; import static java.util.Comparator.comparing; import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.waitAtMost; import static org.mockito.Mockito.mock; /** @@ -43,10 +49,16 @@ public class VolumeTests extends ParallelEoSStreamProcessorTestBase { KafkaTestUtils ku = new KafkaTestUtils(consumerSpy); @SneakyThrows - @Test - public void load() { + @ParameterizedTest() + @EnumSource(CommitMode.class) + public void load(CommitMode commitMode) { setupClients(); - setupAsyncConsumerInstance(UNORDERED); + boolean useTxProducer = commitMode.equals(TRANSACTIONAL_PRODUCER); + setupParallelConsumerInstance(ParallelConsumerOptions.builder() + .ordering(UNORDERED) + .usingTransactionalProducer(useTxProducer) + .commitMode(commitMode) + .build()); // int quantityOfMessagesToProduce = 1_000_000; int quantityOfMessagesToProduce = 500; @@ -54,42 +66,44 @@ public void load() { List> records = ku.generateRecords(quantityOfMessagesToProduce); ku.send(consumerSpy, records); - CountDownLatch latch = new CountDownLatch(quantityOfMessagesToProduce); + CountDownLatch allMessagesConsumedLatch = new CountDownLatch(quantityOfMessagesToProduce); parallelConsumer.pollAndProduce((rec) -> { ProducerRecord mock = mock(ProducerRecord.class); return UniLists.of(mock); }, (x) -> { // log.debug(x.toString()); - latch.countDown(); + allMessagesConsumedLatch.countDown(); }); // + allMessagesConsumedLatch.await(defaultTimeoutSeconds, SECONDS); // waitAtMost(defaultTimeout).until(() -> producerSpy.consumerGroupOffsetsHistory().size() > 0); - latch.await(defaultTimeoutSeconds, SECONDS); - parallelConsumer.waitForNoInFlight(defaultTimeout.multipliedBy(10)); + parallelConsumer.waitForProcessedNotCommitted(defaultTimeout.multipliedBy(10)); parallelConsumer.close(); - // assert quantity of produced messages List> history = producerSpy.history(); assertThat(history).hasSize(quantityOfMessagesToProduce); - // assert order of produced messages - - - // assert order of commits -// int keySize = ku.getKeys().size(); - List>> commitHistory = producerSpy.consumerGroupOffsetsHistory(); - Map> mostRecent = commitHistory.get(commitHistory.size() - 1); - Map topicPartitionOffsetAndMetadataMap = mostRecent.get(CONSUMER_GROUP_ID); - OffsetAndMetadata mostRecentTPCommit = topicPartitionOffsetAndMetadataMap.get(new TopicPartition(INPUT_TOPIC, 0)); - - assertCommitsAlwaysIncrease(); - - long mostRecentCommitOffset = findMostRecentCommitOffset(producerSpy); - assertThat(mostRecentCommitOffset).isEqualTo(quantityOfMessagesToProduce); + if (useTxProducer) { + // assert order of commits + assertCommitsAlwaysIncrease(); + + // producer (tx) commits + List>> producerCommits = producerSpy.consumerGroupOffsetsHistory(); + assertThat(producerCommits).isNotEmpty(); + long mostRecentProducerCommitOffset = findMostRecentCommitOffset(producerSpy); // tx + assertThat(mostRecentProducerCommitOffset).isEqualTo(quantityOfMessagesToProduce); + } else { + // assert commit messages + List> consumerCommitHistory = consumerSpy.getCommitHistoryInt(); + + assertThat(consumerCommitHistory).isNotEmpty(); + long mostRecentConsumerCommitOffset = consumerCommitHistory.get(consumerCommitHistory.size() - 1).values().stream().collect(Collectors.toList()).get(0).offset(); // non-tx + assertThat(mostRecentConsumerCommitOffset).isEqualTo(quantityOfMessagesToProduce); + } // TODO: Assert process ordering } @@ -114,6 +128,7 @@ private void assertCommitsAlwaysIncrease() { private long findMostRecentCommitOffset(MockProducer producerSpy) { List>> commitHistory = producerSpy.consumerGroupOffsetsHistory(); + assertThat(commitHistory).as("No offsets committed").hasSizeGreaterThan(0); Map> mostRecent = commitHistory.get(commitHistory.size() - 1); Map topicPartitionOffsetAndMetadataMap = mostRecent.get(CONSUMER_GROUP_ID); OffsetAndMetadata mostRecentTPCommit = topicPartitionOffsetAndMetadataMap.get(new TopicPartition(INPUT_TOPIC, 0)); @@ -126,41 +141,63 @@ private long findMostRecentCommitOffset(MockProducer producerSpy) { *

* Doesn't currently compare different key sizes, only partition order vs key order vs unordered */ - @Test - public void timingOfDifferentOrderingTypes() { + @ParameterizedTest() + @EnumSource(CommitMode.class) + public void timingOfDifferentOrderingTypes(CommitMode commitMode) { var quantityOfMessagesToProduce = 10_00; var defaultNumKeys = 20; + boolean useTxProducer = commitMode.equals(TRANSACTIONAL_PRODUCER); + + ParallelConsumerOptions baseOptions = ParallelConsumerOptions.builder() + .ordering(UNORDERED) + .usingTransactionalProducer(useTxProducer) + .commitMode(commitMode) + .build(); + + setupParallelConsumerInstance(baseOptions); + Duration unorderedDuration = null; for (var round : range(2)) { // warm up round first - setupAsyncConsumerInstance(UNORDERED); + setupParallelConsumerInstance(baseOptions.toBuilder().ordering(UNORDERED).build()); log.debug("No order"); unorderedDuration = time(() -> testTiming(defaultNumKeys, quantityOfMessagesToProduce)); log.info("Duration for Unordered processing in round {} with {} keys was {}", round, defaultNumKeys, unorderedDuration); } var keyResults = Maps.newTreeMap(); - setupAsyncConsumerInstance(KEY); for (var keySize : UniLists.of(1, 2, 5, 10, 20, 50, 100, 1_000)) { - setupAsyncConsumerInstance(KEY); + setupParallelConsumerInstance(baseOptions.toBuilder().ordering(KEY).build()); log.debug("By key, {} keys", keySize); Duration keyOrderDuration = time(() -> testTiming(keySize, quantityOfMessagesToProduce)); log.info("Duration for Key order processing {} keys was {}", keySize, keyOrderDuration); keyResults.put(keySize, keyOrderDuration); } - setupAsyncConsumerInstance(PARTITION); + setupParallelConsumerInstance(baseOptions.toBuilder().ordering(PARTITION).build()); log.debug("By partition"); Duration partitionOrderDuration = time(() -> testTiming(defaultNumKeys, quantityOfMessagesToProduce)); log.info("Duration for Partition order processing {} keys was {}", defaultNumKeys, partitionOrderDuration); log.info("Key duration results:\n{}", keyResults); + log.info("Unordered duration: {}", unorderedDuration); + int numOfKeysToCompare = 5; // needs to be small enough that there's a significant difference between unordered and key of x Duration keyOrderHalfDefaultKeySize = keyResults.get(numOfKeysToCompare); - assertThat(unorderedDuration).as("UNORDERED is faster than PARTITION order").isLessThan(partitionOrderDuration); - assertThat(unorderedDuration).as("UNORDERED is faster than KEY order, keySize of: " + numOfKeysToCompare).isLessThan(keyOrderHalfDefaultKeySize); - assertThat(keyOrderHalfDefaultKeySize).as("KEY order is faster than PARTITION order").isLessThan(partitionOrderDuration); + assertThat(unorderedDuration).as("UNORDERED should be faster than PARTITION order") + .isLessThan(partitionOrderDuration); + + if (commitMode.equals(CONSUMER_SYNC)) { + assertThat(unorderedDuration).as("Committing synchronously from the controller causes a large overhead, making UNORDERED very close in speed to KEY order, keySize of: " + numOfKeysToCompare) + .isCloseTo(keyOrderHalfDefaultKeySize, keyOrderHalfDefaultKeySize.plus(keyOrderHalfDefaultKeySize.dividedBy(5))); // within 20% + } else { + assertThat(unorderedDuration).as("UNORDERED should be faster than KEY order, keySize of: " + numOfKeysToCompare) + .isLessThan(keyOrderHalfDefaultKeySize); + } + + assertThat(keyOrderHalfDefaultKeySize).as("KEY order is faster than PARTITION order") + .isLessThan(partitionOrderDuration); } /** @@ -188,7 +225,6 @@ private void testTiming(int numberOfKeys, int quantityOfMessagesToProduce) { long sleepTime = (long) (Math.random() * rangeOfTimeSimulatedProcessingTakesMs); sleep(sleepTime); ProducerRecord stub = new ProducerRecord<>(OUTPUT_TOPIC, "sk:" + rec.key(), "Processing took: " + sleepTime + ". SourceV:" + rec.value()); -// ProducerRecord stub = new ProducerRecord<>(OUTPUT_TOPIC, "sk:" + rec.key(), rec.value()); bar.stepTo(producerSpy.history().size()); return UniLists.of(stub); }, (x) -> { @@ -196,7 +232,7 @@ private void testTiming(int numberOfKeys, int quantityOfMessagesToProduce) { // log.debug(x.toString()); }); - Awaitility.waitAtMost(defaultTimeout.multipliedBy(10)).untilAsserted(() -> { + waitAtMost(defaultTimeout.multipliedBy(10)).untilAsserted(() -> { assertThat(super.successfulWork.size()).as("All messages expected messages were processed and successful").isEqualTo(quantityOfMessagesToProduce); assertThat(producerSpy.history().size()).as("All messages expected messages were processed and results produced").isEqualTo(quantityOfMessagesToProduce); }); @@ -205,8 +241,6 @@ private void testTiming(int numberOfKeys, int quantityOfMessagesToProduce) { log.info("Closing async client"); parallelConsumer.close(); - List>> groupOffsetsHistory = producerSpy.consumerGroupOffsetsHistory(); - assertCommitsAlwaysIncrease(); // @@ -250,7 +284,13 @@ private void testTiming(int numberOfKeys, int quantityOfMessagesToProduce) { } assertThat(producerSpy.history().size()).as("Finally, all messages expected messages were produced").isEqualTo(quantityOfMessagesToProduce); - assertThat(groupOffsetsHistory).as("No offsets committed").hasSizeGreaterThan(0); + if (isUsingTransactionalProducer()) { + List>> groupOffsetsHistory = producerSpy.consumerGroupOffsetsHistory(); // tx + assertThat(groupOffsetsHistory).as("No offsets committed").hasSizeGreaterThan(0); // tx + } else { + List> commitHistory = consumerSpy.getCommitHistoryInt(); + assertThat(commitHistory).as("No offsets committed").hasSizeGreaterThan(0); // non-tx + } // clear messages super.successfulWork.clear(); diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java index ff9808c39..17ba965c3 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java @@ -105,7 +105,12 @@ public KafkaProducer createNewProducer(boolean tx) { var txProps = new Properties(); txProps.putAll(props); if (tx) { - txProps.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, this.getClass().getSimpleName()); // required for tx + // random number so we get a unique producer tx session each time. Normally wouldn't do this in production, + // but sometimes running in the test suite our producers step on each other between test runs and this causes + // Producer Fenced exceptions: + // Error looks like: Producer attempted an operation with an old epoch. Either there is a newer producer with + // the same transactionalId, or the producer's transaction has been expired by the broker. + txProps.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, this.getClass().getSimpleName() + ":" + RandomUtils.nextInt()); // required for tx } KafkaProducer kvKafkaProducer = new KafkaProducer<>(txProps); log.debug("New producer {}", kvKafkaProducer); 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 b95a2ea6f..3dd09da7b 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 @@ -9,12 +9,12 @@ import lombok.RequiredArgsConstructor; import lombok.Setter; import lombok.extern.slf4j.Slf4j; -import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.MockConsumer; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.producer.MockProducer; import org.apache.kafka.common.TopicPartition; +import org.assertj.core.api.Assertions; import org.assertj.core.util.Lists; import pl.tlinkowski.unij.api.UniLists; @@ -22,12 +22,11 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; +import static io.confluent.csid.utils.Range.range; import static io.confluent.parallelconsumer.ParallelEoSStreamProcessorTestBase.CONSUMER_GROUP_ID; import static io.confluent.parallelconsumer.ParallelEoSStreamProcessorTestBase.INPUT_TOPIC; -import static io.confluent.csid.utils.Range.range; import static java.lang.Math.random; import static org.assertj.core.api.Assertions.assertThat; -import static org.mockito.Mockito.when; @Slf4j @RequiredArgsConstructor @@ -37,11 +36,7 @@ public class KafkaTestUtils { int offset = 0; - public static final ConsumerGroupMetadata DEFAULT_GROUP_METADATA = new ConsumerGroupMetadata(CONSUMER_GROUP_ID); - - public static void setupConsumer(MockConsumer mc) { - when(mc.groupMetadata()).thenReturn(DEFAULT_GROUP_METADATA); - + public static void assignConsumerToTopic(final MockConsumer mc) { TopicPartition tp1 = new TopicPartition(INPUT_TOPIC, 1); TopicPartition tp0 = new TopicPartition(INPUT_TOPIC, 0); mc.assign(Lists.list(tp0, tp1)); @@ -91,29 +86,35 @@ public static void assertCommits(MockProducer mp, List expectedOffsets, .containsExactlyElementsOf(expectedOffsets); } + public static void assertCommitLists(MockProducer mp, List> expectedPartitionOffsets, Optional description) { + assertCommitLists(mp.consumerGroupOffsetsHistory(), expectedPartitionOffsets, description); + } + /** - * Collects into a set - ignore repeated commits ({@link OffsetMapCodecManager}) + * Collects into a set - ignore repeated commits ({@link OffsetMapCodecManager}). + * + * Ignores duplicates. * * @see OffsetMapCodecManager */ - public static void assertCommitLists(MockProducer mp, List> expectedPartitionOffsets, Optional description) { + public static void assertCommitLists(List>> history, List> expectedPartitionOffsets, Optional description) { log.info("Asserting commits of {}", expectedPartitionOffsets); - List>> history = mp.consumerGroupOffsetsHistory(); AtomicReference topicName = new AtomicReference<>(""); - var results = new HashMap>(); // set - ignore repeated commits ({@link OffsetMap}) + var partitionToCommittedOffsets = new HashMap>(); // set - ignore repeated commits ({@link OffsetMap}) history.stream().forEachOrdered(histories -> { // get all partition offsets and flatten - var group = histories.get(CONSUMER_GROUP_ID); - for (var actualPartitionOffsets : group.entrySet()) { - TopicPartition key = actualPartitionOffsets.getKey(); + var partitionCommits = histories.get(CONSUMER_GROUP_ID); + for (var singlePartitionCommit : partitionCommits.entrySet()) { + TopicPartition key = singlePartitionCommit.getKey(); topicName.set(key.topic()); - OffsetAndMetadata commit = actualPartitionOffsets.getValue(); + OffsetAndMetadata commit = singlePartitionCommit.getValue(); int offset = (int) commit.offset(); - results.computeIfAbsent(key, x -> new HashSet<>()).add(offset); + partitionToCommittedOffsets.computeIfAbsent(key, x -> new HashSet<>()).add(offset); } }); + // compute the matching expected map var expectedMap = new HashMap>(); for (int i = 0; i < expectedPartitionOffsets.size(); i++) { List offsets = expectedPartitionOffsets.get(i); @@ -121,10 +122,17 @@ public static void assertCommitLists(MockProducer mp, List> expect expectedMap.put(tp, new HashSet<>(offsets)); } - assertThat(results).describedAs(description.orElse("Which offsets are committed and in the expected order")) + assertThat(partitionToCommittedOffsets).describedAs(description.orElse("Which offsets are committed and in the expected order")) .containsExactlyEntriesOf(expectedMap); } + public static void assertLastCommitIs(final LongPollingMockConsumer mockConsumer, final int expected) { + List> commits = mockConsumer.getCommitHistoryInt(); + Assertions.assertThat(commits).isNotEmpty(); + long offset = (int) commits.get(commits.size() - 1).values().iterator().next().offset(); + Assertions.assertThat(offset).isEqualTo(expected); + } + public List> generateRecords(int quantity) { HashMap>> integerListHashMap = generateRecords(defaultKeys, quantity); return flatten(integerListHashMap.values()); diff --git a/parallel-consumer-core/src/test/java/io/confluent/csid/utils/LongPollingMockConsumer.java b/parallel-consumer-core/src/test/java/io/confluent/csid/utils/LongPollingMockConsumer.java index daebfe0b9..18629f957 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/csid/utils/LongPollingMockConsumer.java +++ b/parallel-consumer-core/src/test/java/io/confluent/csid/utils/LongPollingMockConsumer.java @@ -4,12 +4,19 @@ * Copyright (C) 2020 Confluent, Inc. */ +import lombok.Getter; +import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.MockConsumer; -import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.clients.consumer.*; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.clients.producer.MockProducer; +import org.apache.kafka.common.TopicPartition; +import java.lang.reflect.Field; import java.time.Duration; +import java.util.*; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; /** * Used in tests to stub out the behaviour of the real Broker and Client's long polling system (the mock Kafka Consumer @@ -21,6 +28,9 @@ @Slf4j public class LongPollingMockConsumer extends MockConsumer { + @Getter + private final List> commitHistoryInt = new ArrayList<>(); + public LongPollingMockConsumer(OffsetResetStrategy offsetResetStrategy) { super(offsetResetStrategy); } @@ -53,5 +63,79 @@ public synchronized void wakeup() { } } + @Override + public synchronized void commitAsync(final Map offsets, final OffsetCommitCallback callback) { + commitHistoryInt.add(offsets); + super.commitAsync(offsets, callback); + } + + /** + * Makes the commit history look like the {@link MockProducer}s one so we can use the same assert method. + * + * @see KafkaTestUtils#assertCommitLists(List, List, Optional) + */ + private List>> injectConsumerGroupId(final List> commitHistory) { + String groupId = this.groupMetadata().groupId(); + return commitHistory.stream() + .map(x -> { + HashMap> stringMapHashMap = new HashMap<>(); + stringMapHashMap.put(groupId, x); + return stringMapHashMap; + }) + .collect(Collectors.toList()); + } + + /** + * Makes the commit history look like the {@link MockProducer}s one so we can use the same assert method. + * + * @see KafkaTestUtils#assertCommitLists(List, List, Optional) + */ + public List>> getCommitHistoryWithGropuId() { + return injectConsumerGroupId(getCommitHistoryInt()); + } + + @Override + @SneakyThrows + public synchronized void close(final long timeout, final TimeUnit unit) { + revokeAssignment(); + super.close(timeout, unit); + } + + /** + * {@link MockConsumer} fails to implement any {@link ConsumerRebalanceListener} system, so we manually revoke + * assignments, use reflection to access the registered rebalance listener, call the listener, and only then close + * the consumer. + * + * @see io.confluent.parallelconsumer.ParallelEoSStreamProcessor#onPartitionsRevoked + */ + private void revokeAssignment() throws NoSuchFieldException, IllegalAccessException { + // access listener + Field subscriptionsField = MockConsumer.class.getDeclaredField("subscriptions"); //NoSuchFieldException + subscriptionsField.setAccessible(true); + SubscriptionState subscriptionState = (SubscriptionState) subscriptionsField.get(this); //IllegalAccessException + ConsumerRebalanceListener consumerRebalanceListener = subscriptionState.rebalanceListener(); + + // execute + if (consumerRebalanceListener == null) { + log.debug("No rebalance listener assigned - on revoke can't fire"); + } else { + Set assignment = super.assignment(); + consumerRebalanceListener.onPartitionsRevoked(assignment); + } + } + + public void subscribeWithRebalanceAndAssignment(final List topics, int partitions) { + List topicPartitions = topics.stream() + .flatMap(y -> Range.rangeStream(partitions).boxed() + .map(x -> new TopicPartition(y, x))) + .collect(Collectors.toList()); + super.rebalance(topicPartitions); + // + HashMap beginningOffsets = new HashMap<>(); + for (var tp : topicPartitions) { + beginningOffsets.put(tp, 0L); + } + super.updateBeginningOffsets(beginningOffsets); -} + } +} \ No newline at end of file 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 923ba0a4c..fbef02bdc 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 @@ -4,20 +4,20 @@ * Copyright (C) 2020 Confluent, Inc. */ -import io.confluent.csid.utils.WallClock; +import io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; 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.clients.producer.ProducerRecord; -import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.TopicPartition; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; import org.mockito.ArgumentMatchers; import org.mockito.Mockito; @@ -26,12 +26,13 @@ import java.util.concurrent.CountDownLatch; import java.util.function.Function; -import static io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder.KEY; import static io.confluent.csid.utils.GeneralTestUtils.time; import static io.confluent.csid.utils.KafkaUtils.toTP; import static io.confluent.csid.utils.Range.range; +import static io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode.TRANSACTIONAL_PRODUCER; +import static io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder.KEY; import static java.time.Duration.ofMillis; - +import static java.time.Duration.ofSeconds; import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.*; import static org.awaitility.Awaitility.await; @@ -58,9 +59,12 @@ public void setupData() { primeFirstRecord(); } - @Test + @ParameterizedTest() + @EnumSource(CommitMode.class) @SneakyThrows - public void failingActionNothingCommitted() { + public void failingActionNothingCommitted(CommitMode commitMode) { + setupParallelConsumerInstance(commitMode); + parallelConsumer.poll((ignore) -> { throw new RuntimeException("My user's function error"); }); @@ -71,7 +75,8 @@ public void failingActionNothingCommitted() { // assertCommits(of(0), "All erroring, nothing committed except initial"); - List>> maps = producerSpy.consumerGroupOffsetsHistory(); + List>> maps = getCommitHistory(); + assertThat(maps).isNotEmpty(); List metas = new ArrayList<>(); for (final Map> map : maps) { for (final Map value : map.values()) { @@ -88,9 +93,12 @@ public void failingActionNothingCommitted() { } } - @Test + @ParameterizedTest() + @EnumSource(CommitMode.class) @SneakyThrows - public void offsetsAreNeverCommittedForMessagesStillInFlightSimplest() { + public void offsetsAreNeverCommittedForMessagesStillInFlightSimplest(CommitMode commitMode) { + setupParallelConsumerInstance(commitMode); + sendSecondRecord(consumerSpy); var locks = constructLatches(2); @@ -117,7 +125,7 @@ public void offsetsAreNeverCommittedForMessagesStillInFlightSimplest() { // make sure no offsets are committed assertCommits(of(0), "Partition is blocked"); - // So it's data setup can be used in other tests, finish 0 + // So it's data is setup can be used in other tests, finish 0 releaseAndWait(locks, 0); log.debug("Closing..."); @@ -126,26 +134,36 @@ public void offsetsAreNeverCommittedForMessagesStillInFlightSimplest() { assertThat(processedStates.values()).as("sanity - all expected messages are processed").containsExactly(true, true); } - @Test - public void offsetsAreNeverCommittedForMessagesStillInFlightShort() { - offsetsAreNeverCommittedForMessagesStillInFlightSimplest(); + private void setupParallelConsumerInstance(final CommitMode commitMode) { + setupParallelConsumerInstance(getBaseOptions(commitMode)); + // created a new client above, so have to send the prime record again + primeFirstRecord(); + } - log.info("Test start"); + private ParallelConsumerOptions getBaseOptions(final CommitMode commitMode) { + return ParallelConsumerOptions.builder() + .commitMode(commitMode) + .usingTransactionalProducer(commitMode.equals(TRANSACTIONAL_PRODUCER)) + .build(); + } - // make sure offset 1, not 0 is committed - // check only 1 is now committed, not committing 0 as well is a performance thing - verify(producerSpy, - after(verificationWaitDelay) - .atLeast(1) - .description("wait for at least one commit call")) - .commitTransaction(); + @ParameterizedTest() + @EnumSource(CommitMode.class) + @SneakyThrows + public void offsetsAreNeverCommittedForMessagesStillInFlightShort(CommitMode commitMode) { + offsetsAreNeverCommittedForMessagesStillInFlightSimplest(commitMode); + log.info("Test start"); + // next expected offset is now 2 assertCommits(of(0, 2), "Only one of the two offsets committed, as they were coalesced for efficiency"); } @Disabled - @Test - public void offsetsAreNeverCommittedForMessagesStillInFlightLong() { + @ParameterizedTest() + @EnumSource(CommitMode.class) + public void offsetsAreNeverCommittedForMessagesStillInFlightLong(CommitMode commitMode) { + setupParallelConsumerInstance(commitMode); + sendSecondRecord(consumerSpy); // send three messages - 0, 1, 2 @@ -224,8 +242,14 @@ public void offsetsAreNeverCommittedForMessagesStillInFlightLong() { parallelConsumer.close(); } - @Test - public void offsetCommitsAreIsolatedPerPartition() { + @ParameterizedTest() + @EnumSource(CommitMode.class) + @SneakyThrows + public void offsetCommitsAreIsolatedPerPartition(CommitMode commitMode) { + setupParallelConsumerInstance(commitMode); + +// @Test +// public void offsetCommitsAreIsolatedPerPartition() { sendSecondRecord(consumerSpy); // send three messages - 0,1, to one partition and 3,4 to another partition petitions @@ -252,6 +276,8 @@ public void offsetCommitsAreIsolatedPerPartition() { // finish processing 1 releaseAndWait(locks, 1); + waitForSomeLoopCycles(5); + // make sure only base offsets are committed assertCommits(of(0, 2)); assertCommitLists(of(of(0), of(2))); @@ -267,12 +293,20 @@ public void offsetCommitsAreIsolatedPerPartition() { // finish 0 releaseAndWait(locks, 0); + // async consumer is slower to execute the commit. We could just wait, or we could add an event to the async consumer commit cycle + if (isUsingAsyncCommits()) + waitForOneLoopCycle(); + // make sure offset 0 and 1 is committed assertCommitLists(of(of(0, 2), of(2, 3))); // finish 3 releaseAndWait(locks, 3); + // async consumer is slower to execute the commit. We could just wait, or we could add an event to the async consumer commit cycle + if (isUsingAsyncCommits()) + waitForOneLoopCycle(); + // assertCommitLists(of(of(0, 2), of(2, 3, 4))); } @@ -288,28 +322,39 @@ public void avro() { assertThat(false).isTrue(); } - @Test - @SneakyThrows - public void controlFlowException() { - // - WallClock mock = mock(WallClock.class); - when(mock.getNow()).thenThrow(new RuntimeException("My fake control loop error")); - parallelConsumer.setClock(mock); + @ParameterizedTest() + @EnumSource(CommitMode.class) + public void controlFlowException(CommitMode commitMode) { + // setup again manually to use subscribe instead of assign (for revoke testing) + instantiateConsumerProducer(); + parallelConsumer = initAsyncConsumer(getBaseOptions(commitMode)); + subscribeParallelConsumerAndMockConsumerTo(INPUT_TOPIC); + setupData(); + + + // cause a control loop error + parallelConsumer.addLoopEndCallBack(() -> { + throw new RuntimeException("My fake control loop error"); + }); + // parallelConsumer.poll((ignore) -> { - // ignore + log.info("Ignoring {}", ignore); }); // close and retrieve exception in control loop assertThatThrownBy(() -> { - parallelConsumer.closeDontDrainFirst(); + parallelConsumer.closeDrainFirst(ofSeconds(10)); }).hasMessageContainingAll("Error", "poll", "thread", "fake control"); } - @Test + @ParameterizedTest() + @EnumSource(CommitMode.class) @SneakyThrows - public void testVoid() { + public void testVoid(CommitMode commitMode) { + setupParallelConsumerInstance(commitMode); + int expected = 1; var msgCompleteBarrier = new CountDownLatch(expected); parallelConsumer.poll((record) -> { @@ -322,11 +367,18 @@ public void testVoid() { waitForSomeLoopCycles(1); + parallelConsumer.close(); + assertCommits(of(0, 1)); verify(myRecordProcessingAction, times(expected)).apply(any()); - verify(producerSpy, atLeastOnce()).commitTransaction(); - verify(producerSpy, atLeastOnce()).sendOffsetsToTransaction(anyMap(), ArgumentMatchers.any()); + + // assert internal methods - shouldn't really need this as we already check the commit history above through the + // spy, so can leave in for the old producer style + if (commitMode.equals(TRANSACTIONAL_PRODUCER)) { + verify(producerSpy, atLeastOnce()).commitTransaction(); + verify(producerSpy, atLeastOnce()).sendOffsetsToTransaction(anyMap(), ArgumentMatchers.any()); + } } /** @@ -339,16 +391,6 @@ private void waitForInitialBootstrapCommit() { .untilAsserted(() -> assertCommits(of(0))); } - @Test - public void testProducerStep() { - ProducerRecord outMsg = new ProducerRecord(OUTPUT_TOPIC, ""); - RecordMetadata prodResult = parallelConsumer.produceMessage(outMsg); - assertThat(prodResult).isNotNull(); - - List> history = producerSpy.history(); - assertThat(history).hasSize(1); - } - @Test @Disabled public void userSucceedsButProduceToBrokerFails() { @@ -381,18 +423,22 @@ public void failingMessagesThatAreRetriedDontBreakProcessingOrders() { public void ifTooManyMessagesAreInFlightDontPollBrokerForMore() { } + @ParameterizedTest() + @EnumSource(CommitMode.class) @SneakyThrows - @Test @Disabled - public void processInKeyOrder() { - ParallelConsumerOptions options = ParallelConsumerOptions.builder().ordering(KEY).build(); - setupAsyncConsumerInstance(options); + public void processInKeyOrder(CommitMode commitMode) { + setupParallelConsumerInstance(ParallelConsumerOptions.builder() + .commitMode(commitMode) + .ordering(KEY) + .usingTransactionalProducer(commitMode.equals(TRANSACTIONAL_PRODUCER)) + .build()); + // created a new client above, so have to send the prime record again + primeFirstRecord(); // sanity check assertThat(parallelConsumer.wm.getOptions().getOrdering()).isEqualTo(KEY); - primeFirstRecord(); - sendSecondRecord(consumerSpy); // 0,1 previously sent to partition 0 @@ -534,7 +580,7 @@ public void processInKeyOrder() { @Test public void processInKeyOrderWorkNotReturnedDoesntBreakCommits() { ParallelConsumerOptions options = ParallelConsumerOptions.builder().ordering(KEY).build(); - setupAsyncConsumerInstance(options); + setupParallelConsumerInstance(options); primeFirstRecord(); sendSecondRecord(consumerSpy); @@ -612,8 +658,11 @@ public void processInKeyOrderWorkNotReturnedDoesntBreakCommits() { } } - @Test - public void closeAfterSingleMessageShouldBeEventBasedFast() { + @ParameterizedTest() + @EnumSource(CommitMode.class) + public void closeAfterSingleMessageShouldBeEventBasedFast(CommitMode commitMode) { + setupParallelConsumerInstance(commitMode); + var msgCompleteBarrier = new CountDownLatch(1); parallelConsumer.poll((ignore) -> { @@ -627,6 +676,10 @@ public void closeAfterSingleMessageShouldBeEventBasedFast() { // allow for offset to be committed waitForOneLoopCycle(); + if (isUsingAsyncCommits()) { + waitForOneLoopCycle(); + } + assertCommits(of(0, 1)); // close @@ -639,8 +692,11 @@ public void closeAfterSingleMessageShouldBeEventBasedFast() { .isLessThan(ofMillis(500)); } - @Test - public void closeWithoutRunningShouldBeEventBasedFast() { + @ParameterizedTest() + @EnumSource(CommitMode.class) + public void closeWithoutRunningShouldBeEventBasedFast(CommitMode commitMode) { + setupParallelConsumerInstance(getBaseOptions(commitMode)); + parallelConsumer.closeDontDrainFirst(); } diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorTestBase.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorTestBase.java index c20cbdbbe..f02b79fb2 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorTestBase.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorTestBase.java @@ -6,28 +6,33 @@ import io.confluent.csid.utils.KafkaTestUtils; import io.confluent.csid.utils.LongPollingMockConsumer; +import io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.*; import org.apache.kafka.clients.producer.MockProducer; import org.apache.kafka.common.TopicPartition; +import org.assertj.core.util.Lists; import org.junit.jupiter.api.BeforeEach; +import pl.tlinkowski.unij.api.UniLists; import pl.tlinkowski.unij.api.UniMaps; import java.time.Duration; import java.util.*; -import java.util.concurrent.Callable; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; +import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; import static io.confluent.csid.utils.Range.range; +import static io.confluent.csid.utils.StringUtils.msg; +import static io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode.*; import static java.time.Duration.ofMillis; import static java.time.Duration.ofSeconds; import static java.util.concurrent.TimeUnit.SECONDS; +import static org.assertj.core.api.Assertions.assertThat; import static org.awaitility.Awaitility.waitAtMost; import static org.mockito.Mockito.*; +import static pl.tlinkowski.unij.api.UniLists.of; @Slf4j public class ParallelEoSStreamProcessorTestBase { @@ -36,24 +41,27 @@ public class ParallelEoSStreamProcessorTestBase { public static final String OUTPUT_TOPIC = "output"; public static final String CONSUMER_GROUP_ID = "my-group"; + public static final ConsumerGroupMetadata DEFAULT_GROUP_METADATA = new ConsumerGroupMetadata(CONSUMER_GROUP_ID); + /** * The frequency with which we pretend to poll the broker for records - actually the pretend long poll timeout. A * lower value shouldn't affect test speed much unless many different batches of messages are "published". + * * @see LongPollingMockConsumer#poll(Duration) */ - public static final int DEFAULT_BROKER_POLL_FREQUENCY_MS = 100; + public static final int DEFAULT_BROKER_POLL_FREQUENCY_MS = 1000; /** - * The commit interval for the main {@link ParallelEoSStreamProcessor} control thread. Actually the timeout that we poll the - * {@link LinkedBlockingQueue} for. A lower value will increase the frequency of control loop cycles, making our - * test waiting go faster. + * The commit interval for the main {@link ParallelEoSStreamProcessor} control thread. Actually the timeout that we + * poll the {@link LinkedBlockingQueue} for. A lower value will increase the frequency of control loop cycles, + * making our test waiting go faster. * * @see ParallelEoSStreamProcessor#workMailBox * @see ParallelEoSStreamProcessor#processWorkCompleteMailBox */ - public static final int DEFAULT_COMMIT_INTERVAL_MAX_MS = 100; + public static final int DEFAULT_COMMIT_INTERVAL_MAX_MS = 1000; - protected MockConsumer consumerSpy; + protected LongPollingMockConsumer consumerSpy; protected MockProducer producerSpy; protected ParallelEoSStreamProcessor parallelConsumer; @@ -62,7 +70,7 @@ public class ParallelEoSStreamProcessorTestBase { protected static Duration defaultTimeout = ofSeconds(defaultTimeoutSeconds); protected static long defaultTimeoutMs = defaultTimeout.toMillis(); - protected static Duration infiniteTimeout = Duration.ofMinutes(20); + protected static Duration effectivelyInfiniteTimeout = Duration.ofMinutes(20); ParallelEoSStreamProcessorTest.MyAction myRecordProcessingAction; @@ -84,13 +92,13 @@ public class ParallelEoSStreamProcessorTestBase { @BeforeEach public void setupAsyncConsumerTestBase() { - setupAsyncConsumerInstance(ParallelConsumerOptions.builder().build()); + setupParallelConsumerInstance(ParallelConsumerOptions.builder().commitMode(CONSUMER_SYNC).build()); } protected List> successfulWork = Collections.synchronizedList(new ArrayList<>()); private void setupWorkManager(WorkManager wm) { - wm.getSuccessfulWorkListeners().add((work)->{ + wm.getSuccessfulWorkListeners().add((work) -> { log.debug("Test work listener heard some successful work: {}", work); successfulWork.add(work); }); @@ -102,25 +110,41 @@ protected void primeFirstRecord() { } protected MockConsumer setupClients() { - MockConsumer consumer = new LongPollingMockConsumer<>(OffsetResetStrategy.EARLIEST); + instantiateConsumerProducer(); + + ktu = new KafkaTestUtils(consumerSpy); + + KafkaTestUtils.assignConsumerToTopic(this.consumerSpy); + + return consumerSpy; + } + + protected void instantiateConsumerProducer() { + LongPollingMockConsumer consumer = new LongPollingMockConsumer<>(OffsetResetStrategy.EARLIEST); MockProducer producer = new MockProducer<>(true, null, null); // TODO do async testing this.producerSpy = spy(producer); this.consumerSpy = spy(consumer); myRecordProcessingAction = mock(ParallelEoSStreamProcessorTest.MyAction.class); - ktu = new KafkaTestUtils(consumerSpy); - - KafkaTestUtils.setupConsumer(this.consumerSpy); + when(consumerSpy.groupMetadata()).thenReturn(DEFAULT_GROUP_METADATA); + } - return consumerSpy; + /** + * Need to make sure we only use {@link ParallelEoSStreamProcessor#subscribe} methods, and not do manual assignment, + * otherwise rebalance listeneres don't fire (because there are never rebalances). + */ + protected void subscribeParallelConsumerAndMockConsumerTo(String topic) { + List of = of(topic); + parallelConsumer.subscribe(of); + consumerSpy.subscribeWithRebalanceAndAssignment(of, 2); } - protected void setupAsyncConsumerInstance(ParallelConsumerOptions.ProcessingOrder order) { - setupAsyncConsumerInstance(ParallelConsumerOptions.builder().ordering(order).build()); + protected void setupParallelConsumerInstance(ProcessingOrder order) { + setupParallelConsumerInstance(ParallelConsumerOptions.builder().ordering(order).build()); } - protected void setupAsyncConsumerInstance(ParallelConsumerOptions parallelConsumerOptions) { + protected void setupParallelConsumerInstance(ParallelConsumerOptions parallelConsumerOptions) { setupClients(); parallelConsumer = initAsyncConsumer(parallelConsumerOptions); @@ -203,7 +227,9 @@ protected void waitUntilTrue(Callable booleanCallable) { private void blockingLoopLatchTrigger(int waitForCount) { log.debug("Waiting on {} cycles on loop latch...", waitForCount); loopLatchV = new CountDownLatch(waitForCount); - loopLatchV.await(defaultTimeoutSeconds, TimeUnit.SECONDS); + boolean await = loopLatchV.await(defaultTimeoutSeconds, SECONDS); + if (!await) + throw new TimeoutException(msg("Timeout {} waiting for latch", defaultTimeoutSeconds)); } @SneakyThrows @@ -222,16 +248,76 @@ protected void waitForCommitExact(int partition, int offset) { any(ConsumerGroupMetadata.class)); } - public void assertCommits(List integers, String description) { - KafkaTestUtils.assertCommits(producerSpy, integers, Optional.of(description)); + public void assertCommits(List offsets, String description) { + assertCommits(offsets, Optional.of(description)); + } + + /** + * Flattens the offsets of all partitions into a single sequential list + */ + public void assertCommits(List offsets, Optional description) { + if (isUsingTransactionalProducer()) { + KafkaTestUtils.assertCommits(producerSpy, offsets, description); + assertThat(extractAllPartitionsOffsetsSequentially()).isEmpty(); + } else { + List collect = extractAllPartitionsOffsetsSequentially(); + // duplicates are ok + // is there a nicer optional way? + // {@link Optional#ifPresentOrElse} only @since 9 + if (description.isPresent()) { + assertThat(collect).as(description.get()).hasSameElementsAs(offsets); + } else { + assertThat(collect).hasSameElementsAs(offsets); + } + + KafkaTestUtils.assertCommits(producerSpy, UniLists.of(), Optional.of("Empty")); + } + } + + /** + * Flattens the offsets of all partitions into a single sequential list + */ + private List extractAllPartitionsOffsetsSequentially() { + var result = new ArrayList(); + // copy the list for safe concurrent access + List> history = new ArrayList<>(consumerSpy.getCommitHistoryInt()); + return history.stream() + .flatMap(commits -> + commits.values().stream().map(meta -> (int) meta.offset()) + ).collect(Collectors.toList()); + } + + public void assertCommits(List offsets) { + assertCommits(offsets, Optional.empty()); + } + + /** + * Checks a list of commits of a list of partitions + */ + public void assertCommitLists(List> offsets) { + if (isUsingTransactionalProducer()) { + KafkaTestUtils.assertCommitLists(producerSpy, offsets, Optional.empty()); + } else { + KafkaTestUtils.assertCommitLists(consumerSpy.getCommitHistoryWithGropuId(), offsets, Optional.empty()); + } + } + + protected List>> getCommitHistory() { + if (isUsingTransactionalProducer()) { + return producerSpy.consumerGroupOffsetsHistory(); + } else { + return consumerSpy.getCommitHistoryWithGropuId(); + } } - public void assertCommits(List integers) { - KafkaTestUtils.assertCommits(producerSpy, integers, Optional.empty()); + protected boolean isUsingTransactionalProducer() { + ParallelConsumerOptions.CommitMode commitMode = parallelConsumer.getWm().getOptions().getCommitMode(); + return commitMode.equals(TRANSACTIONAL_PRODUCER); } - public void assertCommitLists(List> integers) { - KafkaTestUtils.assertCommitLists(producerSpy, integers, Optional.empty()); + protected boolean isUsingAsyncCommits() { + ParallelConsumerOptions.CommitMode commitMode = parallelConsumer.getWm().getOptions().getCommitMode(); + return commitMode.equals(CONSUMER_ASYNCHRONOUS); } protected void awaitLatch(List latches, int latchIndex) { diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/WorkManagerTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/WorkManagerTest.java index 1322608a8..1dc21746c 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/WorkManagerTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/WorkManagerTest.java @@ -318,7 +318,7 @@ public void maxPerTopic() { public void maxInFlight() { // var opts = ParallelConsumerOptions.builder(); - opts.maxUncommittedMessagesToHandlePerPartition(1); + opts.maxUncommittedMessagesToHandle(1); setupWorkManager(opts.build()); // @@ -372,7 +372,7 @@ public void maxConcurrencyVsInFlightAndNoLeaks() { var opts = ParallelConsumerOptions.builder(); opts.ordering(UNORDERED); - opts.maxUncommittedMessagesToHandlePerPartition(3); + opts.maxUncommittedMessagesToHandle(3); opts.maxConcurrency(2); setupWorkManager(opts.build()); @@ -610,7 +610,7 @@ public void workQueuesEmptyWhenAllWorkComplete() { ParallelConsumerOptions build = ParallelConsumerOptions.builder() .ordering(UNORDERED) .maxConcurrency(10) - .maxUncommittedMessagesToHandlePerPartition(10) + .maxUncommittedMessagesToHandle(10) .build(); setupWorkManager(build); registerSomeWork(); diff --git a/parallel-consumer-core/src/test/resources/junit-platform.properties b/parallel-consumer-core/src/test/resources/junit-platform.properties index 7f40a1ce1..e3c240beb 100644 --- a/parallel-consumer-core/src/test/resources/junit-platform.properties +++ b/parallel-consumer-core/src/test/resources/junit-platform.properties @@ -2,4 +2,4 @@ # Copyright (C) 2020 Confluent, Inc. # -junit.jupiter.displayname.generator.default = io.confluent.csid.utils.ReplaceCamelCase \ No newline at end of file +#junit.jupiter.displayname.generator.default = io.confluent.csid.utils.ReplaceCamelCase \ No newline at end of file diff --git a/parallel-consumer-core/src/test/resources/logback-test.xml b/parallel-consumer-core/src/test/resources/logback-test.xml index aa004bb17..9f4330bfe 100644 --- a/parallel-consumer-core/src/test/resources/logback-test.xml +++ b/parallel-consumer-core/src/test/resources/logback-test.xml @@ -27,29 +27,30 @@ - + - + + - - - - - - + + + + + + - - + + + - diff --git a/parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java b/parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java index ee01ee717..290d0fc03 100644 --- a/parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java +++ b/parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java @@ -18,7 +18,11 @@ import org.apache.kafka.clients.producer.ProducerRecord; import pl.tlinkowski.unij.api.UniLists; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; import java.util.Properties; +import java.util.concurrent.atomic.AtomicInteger; import static io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder.KEY; @@ -28,8 +32,9 @@ @Slf4j public class CoreApp { - static String inputTopic = "input-topic-" + RandomUtils.nextInt(); - static String outputTopic = "output-topic-" + RandomUtils.nextInt(); + + String inputTopic = "input-topic-" + RandomUtils.nextInt(); + String outputTopic = "output-topic-" + RandomUtils.nextInt(); Consumer getKafkaConsumer() { return new KafkaConsumer<>(new Properties()); @@ -43,7 +48,7 @@ Producer getKafkaProducer() { @SuppressWarnings("UnqualifiedFieldAccess") void run() { - this.parallelConsumer = setupConsumer(); + this.parallelConsumer = setupParallelConsumer(); // tag::example[] parallelConsumer.poll(record -> @@ -53,40 +58,47 @@ void run() { } @SuppressWarnings({"FeatureEnvy", "MagicNumber"}) - ParallelStreamProcessor setupConsumer() { + ParallelStreamProcessor setupParallelConsumer() { // tag::exampleSetup[] - var options = ParallelConsumerOptions.builder() - .ordering(KEY) // <1> - .maxConcurrency(1000) // <2> - .maxUncommittedMessagesToHandlePerPartition(10000) // <3> - .build(); + ParallelConsumerOptions options = getOptions(); Consumer kafkaConsumer = getKafkaConsumer(); // <4> + Producer kafkaProducer = getKafkaProducer(); + + ParallelStreamProcessor eosStreamProcessor = ParallelStreamProcessor.createEosStreamProcessor(kafkaConsumer, kafkaProducer, options); if (!(kafkaConsumer instanceof MockConsumer)) { - kafkaConsumer.subscribe(UniLists.of(inputTopic)); // <5> + eosStreamProcessor.subscribe(UniLists.of(inputTopic)); // <5> } - return ParallelStreamProcessor.createEosStreamProcessor(kafkaConsumer, getKafkaProducer(), options); + return eosStreamProcessor; // end::exampleSetup[] } + ParallelConsumerOptions getOptions() { + var options = ParallelConsumerOptions.builder() + .ordering(KEY) // <1> + .maxConcurrency(1000) // <2> + .maxUncommittedMessagesToHandle(1000) // <3> + .build(); + return options; + } + void close() { this.parallelConsumer.close(); } void runPollAndProduce() { - this.parallelConsumer = setupConsumer(); + this.parallelConsumer = setupParallelConsumer(); // tag::exampleProduce[] this.parallelConsumer.pollAndProduce(record -> { var result = processBrokerRecord(record); - ProducerRecord produceRecord = - new ProducerRecord<>(outputTopic, "a-key", result.payload); - return UniLists.of(produceRecord); - }, consumeProduceResult -> - log.info("Message {} saved to broker at offset {}", - consumeProduceResult.getOut(), - consumeProduceResult.getMeta().offset()) + return UniLists.of(new ProducerRecord<>(outputTopic, record.key(), result.payload)); + }, consumeProduceResult -> { + log.debug("Message {} saved to broker at offset {}", + consumeProduceResult.getOut(), + consumeProduceResult.getMeta().offset()); + } ); // end::exampleProduce[] } diff --git a/parallel-consumer-examples/parallel-consumer-example-core/src/test/java/io/confluent/parallelconsumer/examples/core/CoreAppTest.java b/parallel-consumer-examples/parallel-consumer-example-core/src/test/java/io/confluent/parallelconsumer/examples/core/CoreAppTest.java index fb3b0a65c..49f13cbc9 100644 --- a/parallel-consumer-examples/parallel-consumer-example-core/src/test/java/io/confluent/parallelconsumer/examples/core/CoreAppTest.java +++ b/parallel-consumer-examples/parallel-consumer-example-core/src/test/java/io/confluent/parallelconsumer/examples/core/CoreAppTest.java @@ -4,6 +4,7 @@ * Copyright (C) 2020 Confluent, Inc. */ +import io.confluent.csid.utils.KafkaTestUtils; import io.confluent.csid.utils.LongPollingMockConsumer; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; @@ -13,7 +14,6 @@ import org.apache.kafka.clients.producer.MockProducer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.common.TopicPartition; -import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; import org.junit.jupiter.api.Test; import org.mockito.Mockito; @@ -22,28 +22,27 @@ import java.time.Duration; import java.util.HashMap; -import static io.confluent.csid.utils.KafkaTestUtils.DEFAULT_GROUP_METADATA; +import static io.confluent.parallelconsumer.ParallelEoSStreamProcessorTestBase.DEFAULT_GROUP_METADATA; import static org.mockito.Mockito.when; @Slf4j public class CoreAppTest { - TopicPartition tp = new TopicPartition(CoreApp.inputTopic, 0); - @SneakyThrows @Test public void test() { log.info("Test start"); CoreAppUnderTest coreApp = new CoreAppUnderTest(); + TopicPartition tp = new TopicPartition(coreApp.inputTopic, 0); coreApp.run(); - coreApp.mockConsumer.addRecord(new ConsumerRecord(CoreApp.inputTopic, 0, 0, "a key 1", "a value")); - coreApp.mockConsumer.addRecord(new ConsumerRecord(CoreApp.inputTopic, 0, 1, "a key 2", "a value")); - coreApp.mockConsumer.addRecord(new ConsumerRecord(CoreApp.inputTopic, 0, 2, "a key 3", "a value")); + coreApp.mockConsumer.addRecord(new ConsumerRecord(coreApp.inputTopic, 0, 0, "a key 1", "a value")); + coreApp.mockConsumer.addRecord(new ConsumerRecord(coreApp.inputTopic, 0, 1, "a key 2", "a value")); + coreApp.mockConsumer.addRecord(new ConsumerRecord(coreApp.inputTopic, 0, 2, "a key 3", "a value")); - Awaitility.await().pollInterval(Duration.ofSeconds(1)).untilAsserted(()->{ - Assertions.assertThat(coreApp.mockConsumer.position(tp)).isEqualTo(3); + Awaitility.await().pollInterval(Duration.ofSeconds(1)).untilAsserted(() -> { + KafkaTestUtils.assertLastCommitIs(coreApp.mockConsumer, 3); }); coreApp.close(); @@ -55,15 +54,16 @@ public void test() { public void testPollAndProduce() { log.info("Test start"); CoreAppUnderTest coreApp = new CoreAppUnderTest(); + TopicPartition tp = new TopicPartition(coreApp.inputTopic, 0); coreApp.runPollAndProduce(); - coreApp.mockConsumer.addRecord(new ConsumerRecord(CoreApp.inputTopic, 0, 0, "a key 1", "a value")); - coreApp.mockConsumer.addRecord(new ConsumerRecord(CoreApp.inputTopic, 0, 1, "a key 2", "a value")); - coreApp.mockConsumer.addRecord(new ConsumerRecord(CoreApp.inputTopic, 0, 2, "a key 3", "a value")); + coreApp.mockConsumer.addRecord(new ConsumerRecord(coreApp.inputTopic, 0, 0, "a key 1", "a value")); + coreApp.mockConsumer.addRecord(new ConsumerRecord(coreApp.inputTopic, 0, 1, "a key 2", "a value")); + coreApp.mockConsumer.addRecord(new ConsumerRecord(coreApp.inputTopic, 0, 2, "a key 3", "a value")); - Awaitility.await().pollInterval(Duration.ofSeconds(1)).untilAsserted(()->{ - Assertions.assertThat(coreApp.mockConsumer.position(tp)).isEqualTo(3); + Awaitility.await().pollInterval(Duration.ofSeconds(1)).untilAsserted(() -> { + KafkaTestUtils.assertLastCommitIs(coreApp.mockConsumer, 3); }); coreApp.close(); @@ -72,6 +72,7 @@ public void testPollAndProduce() { class CoreAppUnderTest extends CoreApp { LongPollingMockConsumer mockConsumer = Mockito.spy(new LongPollingMockConsumer<>(OffsetResetStrategy.EARLIEST)); + TopicPartition tp = new TopicPartition(inputTopic, 0); @Override Consumer getKafkaConsumer() { @@ -85,7 +86,7 @@ Consumer getKafkaConsumer() { @Override Producer getKafkaProducer() { - return new MockProducer<>(); + return new MockProducer<>(true, null, null); } } diff --git a/parallel-consumer-examples/parallel-consumer-example-core/src/test/resources/logback-test.xml b/parallel-consumer-examples/parallel-consumer-example-core/src/test/resources/logback-temp-test.xml similarity index 94% rename from parallel-consumer-examples/parallel-consumer-example-core/src/test/resources/logback-test.xml rename to parallel-consumer-examples/parallel-consumer-example-core/src/test/resources/logback-temp-test.xml index 1d977eeab..aab528624 100644 --- a/parallel-consumer-examples/parallel-consumer-example-core/src/test/resources/logback-test.xml +++ b/parallel-consumer-examples/parallel-consumer-example-core/src/test/resources/logback-temp-test.xml @@ -16,7 +16,7 @@ - + diff --git a/parallel-consumer-examples/parallel-consumer-example-streams/src/main/java/io/confluent/parallelconsumer/examples/streams/StreamsApp.java b/parallel-consumer-examples/parallel-consumer-example-streams/src/main/java/io/confluent/parallelconsumer/examples/streams/StreamsApp.java index d85148239..1c192f5f9 100644 --- a/parallel-consumer-examples/parallel-consumer-example-streams/src/main/java/io/confluent/parallelconsumer/examples/streams/StreamsApp.java +++ b/parallel-consumer-examples/parallel-consumer-example-streams/src/main/java/io/confluent/parallelconsumer/examples/streams/StreamsApp.java @@ -5,7 +5,6 @@ */ -import io.confluent.parallelconsumer.ParallelEoSStreamProcessor; import io.confluent.parallelconsumer.ParallelConsumerOptions; import io.confluent.parallelconsumer.ParallelStreamProcessor; import lombok.extern.slf4j.Slf4j; @@ -69,7 +68,7 @@ void startStreams(Topology topology) { } void concurrentProcess() { - setupConsumer(); + setupParallelConsumer(); parallelConsumer.poll(record -> { log.info("Concurrently processing a record: {}", record); @@ -78,16 +77,18 @@ void concurrentProcess() { } // end::example[] - private void setupConsumer() { + private void setupParallelConsumer() { var options = ParallelConsumerOptions.builder() .ordering(ParallelConsumerOptions.ProcessingOrder.KEY) .maxConcurrency(1000) - .maxUncommittedMessagesToHandlePerPartition(10000) + .maxUncommittedMessagesToHandle(10000) .build(); Consumer kafkaConsumer = getKafkaConsumer(); - parallelConsumer = ParallelStreamProcessor.createEosStreamProcessor(kafkaConsumer, getKafkaProducer(), options); + Producer kafkaProducer = getKafkaProducer(); + + parallelConsumer = ParallelStreamProcessor.createEosStreamProcessor(kafkaConsumer, kafkaProducer, options); parallelConsumer.subscribe(UniLists.of(outputTopicName)); } diff --git a/parallel-consumer-examples/parallel-consumer-example-streams/src/test/java/io/confluent/parallelconsumer/examples/streams/StreamsAppTest.java b/parallel-consumer-examples/parallel-consumer-example-streams/src/test/java/io/confluent/parallelconsumer/examples/streams/StreamsAppTest.java index d8cc84425..a0e5a5acb 100644 --- a/parallel-consumer-examples/parallel-consumer-example-streams/src/test/java/io/confluent/parallelconsumer/examples/streams/StreamsAppTest.java +++ b/parallel-consumer-examples/parallel-consumer-example-streams/src/test/java/io/confluent/parallelconsumer/examples/streams/StreamsAppTest.java @@ -52,7 +52,7 @@ Consumer getKafkaConsumer() { @Override Producer getKafkaProducer() { - return kcu.createNewProducer(true); + return kcu.createNewProducer(false); } @Override diff --git a/parallel-consumer-examples/parallel-consumer-example-vertx/pom.xml b/parallel-consumer-examples/parallel-consumer-example-vertx/pom.xml index 054f25d4a..23bd541c3 100644 --- a/parallel-consumer-examples/parallel-consumer-example-vertx/pom.xml +++ b/parallel-consumer-examples/parallel-consumer-example-vertx/pom.xml @@ -31,6 +31,13 @@ tests test + + io.confluent.parallelconsumer + parallel-consumer-vertx + ${project.version} + tests + test + org.apache.commons commons-lang3 @@ -40,6 +47,12 @@ awaitility test + + com.github.tomakehurst + wiremock + 2.27.2 + test + \ No newline at end of file diff --git a/parallel-consumer-examples/parallel-consumer-example-vertx/src/main/java/io/confluent/parallelconsumer/examples/vertx/VertxApp.java b/parallel-consumer-examples/parallel-consumer-example-vertx/src/main/java/io/confluent/parallelconsumer/examples/vertx/VertxApp.java index 0b4c1d023..62dc5b191 100644 --- a/parallel-consumer-examples/parallel-consumer-example-vertx/src/main/java/io/confluent/parallelconsumer/examples/vertx/VertxApp.java +++ b/parallel-consumer-examples/parallel-consumer-example-vertx/src/main/java/io/confluent/parallelconsumer/examples/vertx/VertxApp.java @@ -40,7 +40,7 @@ void run() { var options = ParallelConsumerOptions.builder() .ordering(ParallelConsumerOptions.ProcessingOrder.KEY) .maxConcurrency(1000) - .maxUncommittedMessagesToHandlePerPartition(10000) + .maxUncommittedMessagesToHandle(10000) .build(); Consumer kafkaConsumer = getKafkaConsumer(); @@ -49,20 +49,26 @@ void run() { this.parallelConsumer = JStreamVertxParallelStreamProcessor.createEosStreamProcessor(kafkaConsumer, getKafkaProducer(), options); + int port = getPort(); + // tag::example[] var resultStream = parallelConsumer.vertxHttpReqInfoStream(record -> { log.info("Concurrently constructing and returning RequestInfo from record: {}", record); Map params = UniMaps.of("recordKey", record.key(), "payload", record.value()); - return new RequestInfo("localhost", "/api", params); // <1> + return new RequestInfo("localhost", port, "/api", params); // <1> }); // end::example[] - resultStream.forEach(x->{ + resultStream.forEach(x -> { log.info("From result stream: {}", x); }); } + protected int getPort() { + return 8080; + } + void setupSubscription(Consumer kafkaConsumer) { kafkaConsumer.subscribe(UniLists.of(inputTopic)); } diff --git a/parallel-consumer-examples/parallel-consumer-example-vertx/src/test/java/io/confluent/parallelconsumer/examples/vertx/VertxAppTest.java b/parallel-consumer-examples/parallel-consumer-example-vertx/src/test/java/io/confluent/parallelconsumer/examples/vertx/VertxAppTest.java index 7652ad1d3..88cc778ee 100644 --- a/parallel-consumer-examples/parallel-consumer-example-vertx/src/test/java/io/confluent/parallelconsumer/examples/vertx/VertxAppTest.java +++ b/parallel-consumer-examples/parallel-consumer-example-vertx/src/test/java/io/confluent/parallelconsumer/examples/vertx/VertxAppTest.java @@ -4,7 +4,10 @@ * Copyright (C) 2020 Confluent, Inc. */ +import io.confluent.csid.utils.KafkaTestUtils; import io.confluent.csid.utils.LongPollingMockConsumer; +import io.confluent.parallelconsumer.vertx.VertxTest; +import lombok.RequiredArgsConstructor; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.Consumer; @@ -13,7 +16,6 @@ import org.apache.kafka.clients.producer.MockProducer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.common.TopicPartition; -import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -22,9 +24,8 @@ import java.time.Duration; import java.util.HashMap; -import java.util.concurrent.TimeoutException; -import static io.confluent.csid.utils.KafkaTestUtils.DEFAULT_GROUP_METADATA; +import static io.confluent.parallelconsumer.ParallelEoSStreamProcessorTestBase.DEFAULT_GROUP_METADATA; @Slf4j public class VertxAppTest { @@ -36,7 +37,10 @@ public class VertxAppTest { @Test public void test() { log.info("Test start"); - VertxAppAppUnderTest coreApp = new VertxAppAppUnderTest(); + VertxTest.setupWireMock(); + int port = VertxTest.stubServer.port(); + + VertxAppAppUnderTest coreApp = new VertxAppAppUnderTest(port); coreApp.run(); @@ -45,17 +49,17 @@ public void test() { coreApp.mockConsumer.addRecord(new ConsumerRecord(VertxApp.inputTopic, 0, 2, "a key 3", "a value")); Awaitility.await().pollInterval(Duration.ofSeconds(1)).untilAsserted(()->{ - Assertions.assertThat(coreApp.mockConsumer.position(tp)).isEqualTo(3); + KafkaTestUtils.assertLastCommitIs(coreApp.mockConsumer, 3); }); - Assertions.assertThatExceptionOfType(TimeoutException.class) - .as("no server to receive request, should timeout trying to close. Could also setup wire mock...") - .isThrownBy(coreApp::close) - .withMessageContainingAll("Waiting", "records", "flight"); + coreApp.close(); } + @RequiredArgsConstructor class VertxAppAppUnderTest extends VertxApp { + private final int port; + LongPollingMockConsumer mockConsumer = Mockito.spy(new LongPollingMockConsumer<>(OffsetResetStrategy.EARLIEST)); @Override @@ -69,12 +73,17 @@ Consumer getKafkaConsumer() { @Override Producer getKafkaProducer() { - return new MockProducer<>(); + return new MockProducer<>(true, null, null); } @Override void setupSubscription(Consumer kafkaConsumer) { mockConsumer.assign(UniLists.of(tp)); } + + @Override + protected int getPort() { + return port; + } } } diff --git a/parallel-consumer-vertx/src/main/java/io/confluent/parallelconsumer/vertx/JStreamVertxParallelEoSStreamProcessor.java b/parallel-consumer-vertx/src/main/java/io/confluent/parallelconsumer/vertx/JStreamVertxParallelEoSStreamProcessor.java index ae2eae61b..2724cedae 100644 --- a/parallel-consumer-vertx/src/main/java/io/confluent/parallelconsumer/vertx/JStreamVertxParallelEoSStreamProcessor.java +++ b/parallel-consumer-vertx/src/main/java/io/confluent/parallelconsumer/vertx/JStreamVertxParallelEoSStreamProcessor.java @@ -25,6 +25,8 @@ import java.util.function.Function; import java.util.stream.Stream; +import static io.confluent.parallelconsumer.UserFunctions.carefullyRun; + @Slf4j public class JStreamVertxParallelEoSStreamProcessor extends VertxParallelEoSStreamProcessor implements JStreamVertxParallelStreamProcessor { @@ -72,7 +74,7 @@ public Stream> vertxHttpReqInfoStream(Function, RequestInfo> requestInfoFunctionWrapped = x -> { result.in(x); - RequestInfo apply = requestInfoFunction.apply(x); + RequestInfo apply = carefullyRun(requestInfoFunction, x); result.requestInfo(Optional.of(apply)); return apply; }; @@ -97,7 +99,8 @@ public Stream> vertxHttpRequestStream(BiFunction, HttpRequest> requestInfoFunctionWrapped = (wc, x) -> { result.in(x); - HttpRequest apply = webClientRequestFunction.apply(wc, x); + HttpRequest apply = carefullyRun(webClientRequestFunction, wc, x); + result.httpReq(Optional.of(apply)); return apply; }; @@ -123,7 +126,8 @@ public Stream> vertxHttpWebClientStream( BiFunction, Future>> wrappedFunc = (x, y) -> { // capture result.in(y); - Future> apply = webClientRequestFunction.apply(x, y); + Future> apply = carefullyRun(webClientRequestFunction, x, y); + result.asr(apply); return apply; }; diff --git a/parallel-consumer-vertx/src/main/java/io/confluent/parallelconsumer/vertx/VertxParallelEoSStreamProcessor.java b/parallel-consumer-vertx/src/main/java/io/confluent/parallelconsumer/vertx/VertxParallelEoSStreamProcessor.java index 68eefbb06..070be97df 100644 --- a/parallel-consumer-vertx/src/main/java/io/confluent/parallelconsumer/vertx/VertxParallelEoSStreamProcessor.java +++ b/parallel-consumer-vertx/src/main/java/io/confluent/parallelconsumer/vertx/VertxParallelEoSStreamProcessor.java @@ -4,9 +4,7 @@ * Copyright (C) 2020 Confluent, Inc. */ -import io.confluent.parallelconsumer.ParallelEoSStreamProcessor; -import io.confluent.parallelconsumer.ParallelConsumerOptions; -import io.confluent.parallelconsumer.WorkContainer; +import io.confluent.parallelconsumer.*; import io.vertx.core.AsyncResult; import io.vertx.core.Future; import io.vertx.core.Vertx; @@ -34,6 +32,8 @@ import java.util.function.Consumer; import java.util.function.Function; +import static io.confluent.parallelconsumer.UserFunctions.carefullyRun; + @Slf4j public class VertxParallelEoSStreamProcessor extends ParallelEoSStreamProcessor implements VertxParallelStreamProcessor { @@ -90,7 +90,8 @@ public void vertxHttpReqInfo(Function, RequestInfo> request Consumer>> onSend, Consumer>> onWebRequestComplete) { vertxHttpRequest((WebClient wc, ConsumerRecord rec) -> { - RequestInfo reqInf = requestInfoFunction.apply(rec); + RequestInfo reqInf = carefullyRun(requestInfoFunction, rec); + HttpRequest req = wc.get(reqInf.getPort(), reqInf.getHost(), reqInf.getContextPath()); Map params = reqInf.getParams(); for (var entry : params.entrySet()) { @@ -106,7 +107,7 @@ public void vertxHttpRequest(BiFunction, HttpReq Consumer>> onWebRequestComplete) { // TODO remove, redundant over onSend? vertxHttpWebClient((webClient, record) -> { - HttpRequest call = webClientRequestFunction.apply(webClient, record); + HttpRequest call = carefullyRun(webClientRequestFunction, webClient, record); Future> send = call.send(); // dispatches the work to vertx @@ -124,7 +125,8 @@ public void vertxHttpWebClient(BiFunction, Futur Consumer>> onSend) { Function, List>>> userFuncWrapper = (record) -> { - Future> send = webClientRequestFunction.apply(webClient, record); + + Future> send = carefullyRun(webClientRequestFunction, webClient, record); // send callback onSend.accept(send); @@ -232,7 +234,7 @@ private boolean isVertxWork(List resultsFromUserFunction) { @Override public void close(Duration timeout, DrainingMode drainMode) { log.info("Vert.x async consumer closing..."); - waitForNoInFlight(timeout); + waitForProcessedNotCommitted(timeout); super.close(timeout, drainMode); webClient.close(); Future close = vertx.close(); diff --git a/parallel-consumer-vertx/src/test/java/io/confluent/parallelconsumer/vertx/VertxTest.java b/parallel-consumer-vertx/src/test/java/io/confluent/parallelconsumer/vertx/VertxTest.java index 22b634f02..58ff082a8 100644 --- a/parallel-consumer-vertx/src/test/java/io/confluent/parallelconsumer/vertx/VertxTest.java +++ b/parallel-consumer-vertx/src/test/java/io/confluent/parallelconsumer/vertx/VertxTest.java @@ -41,6 +41,7 @@ import static com.github.tomakehurst.wiremock.client.WireMock.*; import static com.github.tomakehurst.wiremock.core.WireMockConfiguration.wireMockConfig; +import static io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode.TRANSACTIONAL_PRODUCER; import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThat; import static org.awaitility.Awaitility.await; @@ -52,7 +53,7 @@ public class VertxTest extends ParallelEoSStreamProcessorTestBase { JStreamVertxParallelEoSStreamProcessor vertxAsync; - protected static WireMockServer stubServer; + public static WireMockServer stubServer; protected static final String stubResponse = "Good times."; @@ -68,12 +69,13 @@ VertxParallelEoSStreamProcessor.RequestInfo getBadHost() { public static void setupWireMock() { WireMockConfiguration options = wireMockConfig().dynamicPort(); stubServer = new WireMockServer(options); - stubServer.start(); MappingBuilder mappingBuilder = get(urlPathEqualTo("/")) .willReturn(aResponse() .withBody(stubResponse)); - stubServer.stubFor( - mappingBuilder); + stubServer.stubFor(mappingBuilder); + stubServer.stubFor(get(urlPathEqualTo("/api")). + willReturn(aResponse().withBody(stubResponse))); + stubServer.start(); } @Override @@ -81,7 +83,7 @@ protected ParallelEoSStreamProcessor initAsyncConsumer(ParallelConsumerOptions p VertxOptions vertxOptions = new VertxOptions(); Vertx vertx = Vertx.vertx(vertxOptions); WebClient wc = WebClient.create(vertx); - ParallelConsumerOptions build = ParallelConsumerOptions.builder().build(); + ParallelConsumerOptions build = ParallelConsumerOptions.builder().commitMode(TRANSACTIONAL_PRODUCER).usingTransactionalProducer(true).build(); vertxAsync = new JStreamVertxParallelEoSStreamProcessor<>(consumerSpy, producerSpy, vertx, wc, build); return vertxAsync; diff --git a/src/docs/README.adoc b/src/docs/README.adoc index 3501b1511..80b24821d 100644 --- a/src/docs/README.adoc +++ b/src/docs/README.adoc @@ -528,16 +528,33 @@ When your function is actually run, a result object will be streamed back to you After your operation completes, you can also choose to publish a result message back to Kafka. The message publishing metadata can be streamed back to your client code. -== Apache Kafka EoS Transaction Model +== Commit Mode -The system uses Kafka's Exactly Once Semantics (EoS) system. +The system gives you three choices for how to do offset commits. +The simplest of the three are the two Consummer commits modes. +They are of course, `sycnhronous` and `asynchronous` mode. +The `transactional` mode is explained in the next section. + +`Asychornous` mode is faster, as it doesn't block the control loop. + +`Sycnhronous` will block the processing loop until a successful commit response is received, however, `asycnhronous` will still be capped by the max processing settings in the `Options` class. + +If you're used to using the auto commit mode in the normal Kafka consumer, you can think of the `Asychornous` mode being similar to this. +We suggest starting with this mode, and it is the default. + +=== Apache Kafka EoS Transaction Model + +There is also the option to use Kafka's Exactly Once Semantics (EoS) system. This causes all messages produced as a result of processing a message to be committed within a transaction, along with their source offset. This means that even under failure, the results will exist exactly once in the Kafka output topic. -If as a part of -your processing, you create side effects in other systems, this pertains to the usual idempotency requirements when breaking of EoS Kafka boundaries. +If as a part of your processing, you create side effects in other systems, this pertains to the usual idempotency requirements when breaking of EoS Kafka boundaries. + +NOTE:: As with the `synchronous` processing mode, this will also block the processing loop until a successful transaction completes CAUTION: This cannot be true for any externally integrated third party system, unless that system is __idempotent__. +For implementations details, see the <> section. + [[streams-usage]] == Using with Kafka Streams @@ -724,6 +741,10 @@ Instead of the work thread pool count being the degree of concurrency, it is con .Vert.x Architecture image::https://lucid.app/publicSegments/view/509df410-5997-46be-98e7-ac7f241780b4/image.png[Vert.x Architecture, align="center"] +=== Transactional System Architecture + +image::https://lucid.app/publicSegments/view/7480d948-ed7d-4370-a308-8ec12e6b453b/image.png[] + === Offset Map ==== Storage From 06171986464d842f4e1179c6724845a11f86c64d Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Fri, 20 Nov 2020 12:06:52 +0000 Subject: [PATCH 03/11] feature: Make Producer instance optional, adds convenience methods --- README.adoc | 8 +- .../JStreamParallelEoSStreamProcessor.java | 9 +- .../JStreamParallelStreamProcessor.java | 9 +- .../ParallelConsumerOptions.java | 54 +++++----- .../ParallelEoSStreamProcessor.java | 76 ++++++++----- .../ParallelStreamProcessor.java | 35 ++++-- .../parallelconsumer/ProducerManager.java | 10 +- .../CloseAndOpenOffsetTest.java | 41 ++++--- .../integrationTests/LoadTest.java | 11 +- .../TransactionAndCommitModeTest.java | 14 ++- .../integrationTests/VolumeTests.java | 12 +-- ...JStreamParallelEoSStreamProcessorTest.java | 5 +- .../ParallelEoSStreamProcessorTest.java | 102 ++++++++++++++++-- .../ParallelEoSStreamProcessorTestBase.java | 17 ++- .../examples/core/CoreApp.java | 8 +- .../examples/streams/StreamsApp.java | 11 +- .../examples/vertx/VertxApp.java | 10 +- ...StreamVertxParallelEoSStreamProcessor.java | 14 +-- .../JStreamVertxParallelStreamProcessor.java | 7 +- .../VertxParallelEoSStreamProcessor.java | 7 +- .../vertx/VertxNonVertxOperations.java | 2 +- .../parallelconsumer/vertx/VertxTest.java | 6 +- 22 files changed, 308 insertions(+), 160 deletions(-) diff --git a/README.adoc b/README.adoc index e6a82eaf7..0c0c1eb7b 100644 --- a/README.adoc +++ b/README.adoc @@ -364,12 +364,12 @@ Where `${project.version}` is the version to be used: .Setup the client [source,java,indent=0] ---- - ParallelConsumerOptions options = getOptions(); - Consumer kafkaConsumer = getKafkaConsumer(); // <4> Producer kafkaProducer = getKafkaProducer(); - ParallelStreamProcessor eosStreamProcessor = ParallelStreamProcessor.createEosStreamProcessor(kafkaConsumer, kafkaProducer, options); + var options = getOptions().toBuilder().consumer(kafkaConsumer).producer(kafkaProducer).build(); + + ParallelStreamProcessor eosStreamProcessor = ParallelStreamProcessor.createEosStreamProcessor(options); if (!(kafkaConsumer instanceof MockConsumer)) { eosStreamProcessor.subscribe(UniLists.of(inputTopic)); // <5> } @@ -434,7 +434,7 @@ You can also optionally provide a callback function to be run after the message( [source,java,indent=0] this.parallelConsumer.pollAndProduce(record -> { var result = processBrokerRecord(record); - return UniLists.of(new ProducerRecord<>(outputTopic, record.key(), result.payload)); + return new ProducerRecord<>(outputTopic, record.key(), result.payload); }, consumeProduceResult -> { log.debug("Message {} saved to broker at offset {}", consumeProduceResult.getOut(), diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/JStreamParallelEoSStreamProcessor.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/JStreamParallelEoSStreamProcessor.java index c0d93c568..db76b98a2 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/JStreamParallelEoSStreamProcessor.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/JStreamParallelEoSStreamProcessor.java @@ -7,7 +7,6 @@ import io.confluent.csid.utils.Java8StreamUtils; import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; import java.util.List; @@ -22,10 +21,8 @@ public class JStreamParallelEoSStreamProcessor extends ParallelEoSStreamPr private final ConcurrentLinkedDeque> userProcessResultsStream; - public JStreamParallelEoSStreamProcessor(org.apache.kafka.clients.consumer.Consumer consumer, - Producer producer, - ParallelConsumerOptions parallelConsumerOptions) { - super(consumer, producer, parallelConsumerOptions); + public JStreamParallelEoSStreamProcessor(ParallelConsumerOptions parallelConsumerOptions) { + super(parallelConsumerOptions); this.userProcessResultsStream = new ConcurrentLinkedDeque<>(); @@ -34,7 +31,7 @@ public JStreamParallelEoSStreamProcessor(org.apache.kafka.clients.consumer.Consu @Override public Stream> pollProduceAndStream(Function, List>> userFunction) { - super.pollAndProduce(userFunction, (result) -> { + super.pollAndProduceMany(userFunction, (result) -> { log.trace("Wrapper callback applied, sending result to stream. Input: {}", result); this.userProcessResultsStream.add(result); }); diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/JStreamParallelStreamProcessor.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/JStreamParallelStreamProcessor.java index 0a27e3ef1..afd5424b0 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/JStreamParallelStreamProcessor.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/JStreamParallelStreamProcessor.java @@ -13,15 +13,12 @@ public interface JStreamParallelStreamProcessor extends DrainingCloseable { - static JStreamParallelStreamProcessor createJStreamEosStreamProcessor( - org.apache.kafka.clients.consumer.Consumer consumer, - org.apache.kafka.clients.producer.Producer producer, - ParallelConsumerOptions options) { - return new JStreamParallelEoSStreamProcessor<>(consumer, producer, options); + static JStreamParallelStreamProcessor createJStreamEosStreamProcessor(ParallelConsumerOptions options) { + return new JStreamParallelEoSStreamProcessor<>(options); } /** - * Like {@link ParallelEoSStreamProcessor#pollAndProduce} but instead of callbacks, streams the results instead, + * Like {@link ParallelEoSStreamProcessor#pollAndProduceMany} but instead of callbacks, streams the results instead, * after the produce result is ack'd by Kafka. * * @return a stream of results of applying the function to the polled records diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java index 2654020b6..00f425e9a 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java @@ -4,23 +4,31 @@ * Copyright (C) 2020 Confluent, Inc. */ -import io.confluent.csid.utils.StringUtils; import lombok.Builder; import lombok.Getter; import lombok.ToString; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.producer.Producer; -import java.util.Properties; +import java.util.Objects; import static io.confluent.csid.utils.StringUtils.msg; import static io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode.TRANSACTIONAL_PRODUCER; /** * The options for the {@link ParallelEoSStreamProcessor} system. + * + * @see #builder() + * @see ParallelConsumerOptions.ParallelConsumerOptionsBuilder */ @Getter @Builder(toBuilder = true) @ToString -public class ParallelConsumerOptions { +public class ParallelConsumerOptions { + + private final Consumer consumer; + + private final Producer producer; /** * The ordering guarantee to use. @@ -52,6 +60,9 @@ public enum CommitMode { /** * Commits through the Producer using transactions. Slowest fot he options, but no duplicates in Kafka * guaranteed (message replay may cause duplicates in external systems which is unavoidable with Kafka). + *

+ * This is separate from using an IDEMPOTENT Producer, which can be used, along with {@link + * CommitMode#CONSUMER_SYNC} or {@link CommitMode#CONSUMER_ASYNCHRONOUS}. */ TRANSACTIONAL_PRODUCER, /** @@ -75,22 +86,6 @@ public enum CommitMode { @Builder.Default private final CommitMode commitMode = CommitMode.CONSUMER_ASYNCHRONOUS; - /** - * When using a produce flow, when producing the message, either - *

- * This is separate from using an IDEMPOTENT Producer, which can be used, along with {@link - * CommitMode#CONSUMER_SYNC} or {@link CommitMode#CONSUMER_ASYNCHRONOUS}. - *

- * Must be set to true if being used with a transactional producer. Producers state. However, forcing it to be - * specified makes the choice more verbose? - *

- * TODO we could just auto detect this from the - *

- * TODO delete in favor of CommitMode - */ - @Builder.Default - private final boolean usingTransactionalProducer = false; - /** * Don't have more than this many uncommitted messages in process *

@@ -114,17 +109,20 @@ public enum CommitMode { @Builder.Default private final int numberOfThreads = 16; - // TODO remove - or include when checking passed property arguments instead of instances - // using reflection instead -// @Builder.Default -// private final Properties producerConfig = new Properties(); - public void validate() { - boolean commitModeIsTx = commitMode.equals(TRANSACTIONAL_PRODUCER); - if (this.isUsingTransactionalProducer() ^ commitModeIsTx) { - throw new IllegalArgumentException(msg("Using transaction producer mode ({}) without matching commit mode ({})", - this.isUsingTransactionalProducer(), + Objects.requireNonNull(consumer, "A consumer must be supplied"); + + if (isUsingTransactionalProducer() && producer == null) { + throw new IllegalArgumentException(msg("Wanting to use Transaction Producer mode ({}) without supplying a Producer instance", commitMode)); } } + + protected boolean isUsingTransactionalProducer() { + return commitMode.equals(TRANSACTIONAL_PRODUCER); + } + + public boolean isProducerSupplied() { + return getProducer() != null; + } } 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 ca750b6bd..205bf1714 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 @@ -56,7 +56,7 @@ public class ParallelEoSStreamProcessor implements ParallelStreamProcessor private Instant lastCommit = Instant.now(); - final ProducerManager producerManager; + private final Optional> producerManager; private final org.apache.kafka.clients.consumer.Consumer consumer; @@ -125,38 +125,37 @@ enum State { * * @see ParallelConsumerOptions */ - public ParallelEoSStreamProcessor(org.apache.kafka.clients.consumer.Consumer newConsumer, - org.apache.kafka.clients.producer.Producer newProducer, - ParallelConsumerOptions newOptions) { - log.info("Confluent Parallel Consumer initialise... Options: {}", newOptions); + public ParallelEoSStreamProcessor(ParallelConsumerOptions newOptions) { + Objects.requireNonNull(newOptions, "Options must be supplied"); - Objects.requireNonNull(newConsumer); - Objects.requireNonNull(newProducer); - Objects.requireNonNull(newOptions); + log.info("Confluent Parallel Consumer initialise... Options: {}", newOptions); options = newOptions; options.validate(); - checkNotSubscribed(newConsumer); - checkAutoCommitIsDisabled(newConsumer); + this.consumer = options.getConsumer(); - this.consumer = newConsumer; + checkNotSubscribed(consumer); + checkAutoCommitIsDisabled(consumer); this.workerPool = Executors.newFixedThreadPool(newOptions.getNumberOfThreads()); - this.wm = new WorkManager<>(newOptions, newConsumer); + this.wm = new WorkManager<>(newOptions, consumer); - ConsumerManager consumerMgr = new ConsumerManager<>(newConsumer); - this.producerManager = new ProducerManager<>(newProducer, consumerMgr, this.wm, options); + ConsumerManager consumerMgr = new ConsumerManager<>(consumer); this.brokerPollSubsystem = new BrokerPollSystem<>(consumerMgr, wm, this, newOptions); - if (options.isUsingTransactionalProducer()) { - this.committer = this.producerManager; + if (options.isProducerSupplied()) { + this.producerManager = Optional.of(new ProducerManager<>(options.getProducer(), consumerMgr, this.wm, options)); + if (options.isUsingTransactionalProducer()) + this.committer = this.producerManager.get(); + else + this.committer = this.brokerPollSubsystem; } else { + this.producerManager = Optional.empty(); this.committer = this.brokerPollSubsystem; } - } private void checkNotSubscribed(org.apache.kafka.clients.consumer.Consumer consumerToCheck) { @@ -277,8 +276,13 @@ public void poll(Consumer> usersVoidConsumptionFunction) { @Override @SneakyThrows - public void pollAndProduce(Function, List>> userFunction, - Consumer> callback) { + public void pollAndProduceMany(Function, List>> userFunction, + Consumer> callback) { + // todo refactor out the producer system to a sub class + if (!options.isProducerSupplied()) { + throw new IllegalArgumentException("To use the produce flows you must supply a Producer in the options"); + } + // wrap user func to add produce function Function, List>> wrappedUserFunc = (consumedRecord) -> { @@ -291,7 +295,7 @@ public void pollAndProduce(Function, List> results = new ArrayList<>(); for (ProducerRecord toProduce : recordListToProduce) { - RecordMetadata produceResultMeta = producerManager.produceMessage(toProduce); + RecordMetadata produceResultMeta = producerManager.get().produceMessage(toProduce); var result = new ConsumeProduceResult<>(consumedRecord, toProduce, produceResultMeta); results.add(result); } @@ -301,6 +305,31 @@ public void pollAndProduce(Function, List, List>> userFunction) { + pollAndProduceMany(userFunction, (record) -> { + // no op call back + log.trace("No-op user callback"); + }); + } + + @Override + @SneakyThrows + public void pollAndProduce(Function, ProducerRecord> userFunction) { + pollAndProduce(userFunction, (record) -> { + // no op call back + log.trace("No-op user callback"); + }); + } + + @Override + @SneakyThrows + public void pollAndProduce(Function, ProducerRecord> userFunction, + Consumer> callback) { + pollAndProduceMany((record) -> UniLists.of(userFunction.apply(record)), callback); + } + @Override public void close() { // use a longer timeout, to cover for evey other step using the default @@ -368,7 +397,7 @@ private void doClose(Duration timeout) throws TimeoutException, ExecutionExcepti maybeCloseConsumer(); - producerManager.close(timeout); + producerManager.ifPresent(x -> x.close(timeout)); log.debug("Shutting down execution pool..."); List unfinished = workerPool.shutdownNow(); @@ -417,7 +446,7 @@ private boolean isResponsibleForCommits() { */ @SneakyThrows public void waitForProcessedNotCommitted(Duration timeout) { - log.debug("Waiting processed but not commmitted..."); + log.debug("Waiting processed but not committed..."); var timer = Time.SYSTEM.timer(timeout); while (wm.isRecordsAwaitingToBeCommitted()) { log.trace("Waiting for no in processing..."); @@ -757,7 +786,6 @@ protected List, R>> userFunctionRunner(Function wc, List resultsFromUserFunction) { addToMailbox(wc); } @@ -783,7 +811,7 @@ protected void addToMailbox(WorkContainer wc) { */ void notifyNewWorkRegistered() { if (currentlyPollingWorkCompleteMailBox.get()) { - if (!producerManager.isTransactionInProgress()) { + if (producerManager.isPresent() && !producerManager.get().isTransactionInProgress()) { log.trace("Interrupting control thread: Knock knock, wake up! You've got mail (tm)!"); this.blockableControlThread.interrupt(); } else { diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelStreamProcessor.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelStreamProcessor.java index 79c6ee17f..89f063380 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelStreamProcessor.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelStreamProcessor.java @@ -18,17 +18,38 @@ * Parallel message consumer which also can optionally produce 0 or many {@link ProducerRecord} results to be published * back to Kafka. * - * @see #pollAndProduce(Function, Consumer) + * @see #pollAndProduceMany(Function, Consumer) */ public interface ParallelStreamProcessor extends ParallelConsumer, DrainingCloseable { - static ParallelStreamProcessor createEosStreamProcessor( - org.apache.kafka.clients.consumer.Consumer consumer, - org.apache.kafka.clients.producer.Producer producer, - ParallelConsumerOptions options) { - return new ParallelEoSStreamProcessor<>(consumer, producer, options); + static ParallelStreamProcessor createEosStreamProcessor(ParallelConsumerOptions options) { + return new ParallelEoSStreamProcessor(options); } + /** + * Register a function to be applied in parallel to each received message, which in turn returns one or more {@link + * ProducerRecord}s to be sent back to the broker. + * + * @param callback applied after the produced message is acknowledged by kafka + */ + @SneakyThrows + void pollAndProduceMany(Function, List>> userFunction, + Consumer> callback); + + /** + * Register a function to be applied in parallel to each received message, which in turn returns one or many {@link + * ProducerRecord}s to be sent back to the broker. + */ + @SneakyThrows + void pollAndProduceMany(Function, List>> userFunction); + + /** + * Register a function to be applied in parallel to each received message, which in turn returns a {@link + * ProducerRecord} to be sent back to the broker. + */ + @SneakyThrows + void pollAndProduce(Function, ProducerRecord> userFunction); + /** * Register a function to be applied in parallel to each received message, which in turn returns a {@link * ProducerRecord} to be sent back to the broker. @@ -36,7 +57,7 @@ static ParallelStreamProcessor createEosStreamProcessor( * @param callback applied after the produced message is acknowledged by kafka */ @SneakyThrows - void pollAndProduce(Function, List>> userFunction, + void pollAndProduce(Function, ProducerRecord> userFunction, Consumer> callback); /** diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ProducerManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ProducerManager.java index 746c40674..cab36250e 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ProducerManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ProducerManager.java @@ -48,10 +48,10 @@ public ProducerManager(final Producer newProducer, final ConsumerManager newProducer) { + private void initProducer() { producerTransactionLock = new ReentrantReadWriteLock(true); // String transactionIdProp = options.getProducerConfig().getProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG); @@ -70,7 +70,9 @@ private void initProducer(final Producer newProducer) { } } else { if (producerIsConfiguredForTransactions) { - throw new IllegalArgumentException("Not using non-transactional option, but Producer has a transaction ID - Producer must not have a transaction ID for this option"); + throw new IllegalArgumentException("Using non-transactional producer option, but Producer has a transaction ID - " + + "the Producer must not have a transaction ID for this option. This is because having such an ID forces the " + + "Producer into transactional mode - i.e. you cannot use it without using transactions."); } } } @@ -109,7 +111,7 @@ private TransactionManager getTransactionManager() { * use case where messages aren't produced. * * @see ParallelConsumer#poll - * @see ParallelStreamProcessor#pollAndProduce + * @see ParallelStreamProcessor#pollAndProduceMany */ RecordMetadata produceMessage(ProducerRecord outMsg) { // only needed if not using tx diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/CloseAndOpenOffsetTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/CloseAndOpenOffsetTest.java index 47c26403a..033279ee0 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/CloseAndOpenOffsetTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/CloseAndOpenOffsetTest.java @@ -80,17 +80,18 @@ void offsetsOpenClose() { } // 1 client - ParallelConsumerOptions options = ParallelConsumerOptions.builder() + KafkaConsumer newConsumerOne = kcu.createNewConsumer(); + KafkaProducer producerOne = kcu.createNewProducer(true); + var options = ParallelConsumerOptions.builder() .ordering(UNORDERED) - .usingTransactionalProducer(true) .commitMode(TRANSACTIONAL_PRODUCER) + .consumer(newConsumerOne) + .producer(producerOne) .build(); kcu.props.put(ConsumerConfig.CLIENT_ID_CONFIG, "ONE-my-client"); - KafkaConsumer newConsumerOne = kcu.createNewConsumer(); // - KafkaProducer producerOne = kcu.createNewProducer(true); - var asyncOne = new ParallelEoSStreamProcessor<>(newConsumerOne, producerOne, options); + var asyncOne = new ParallelEoSStreamProcessor(options); // asyncOne.subscribe(UniLists.of(rebalanceTopic)); @@ -153,7 +154,8 @@ void offsetsOpenClose() { kcu.props.put(ConsumerConfig.CLIENT_ID_CONFIG, "THREE-my-client"); KafkaConsumer newConsumerThree = kcu.createNewConsumer(); KafkaProducer producerThree = kcu.createNewProducer(true); - try (var asyncThree = new ParallelEoSStreamProcessor<>(newConsumerThree, producerThree, options)) { + var optionsThree = options.toBuilder().consumer(newConsumerThree).producer(producerThree).build(); + try (var asyncThree = new ParallelEoSStreamProcessor(optionsThree)) { asyncThree.subscribe(UniLists.of(rebalanceTopic)); // read what we're given @@ -203,13 +205,14 @@ void correctOffsetVerySimple() { KafkaConsumer consumer = kcu.createNewConsumer(); KafkaProducer producerOne = kcu.createNewProducer(true); - ParallelConsumerOptions options = ParallelConsumerOptions.builder() + var options = ParallelConsumerOptions.builder() .ordering(UNORDERED) - .usingTransactionalProducer(true) + .consumer(consumer) + .producer(producerOne) .commitMode(TRANSACTIONAL_PRODUCER) .build(); - try (var asyncOne = new ParallelEoSStreamProcessor<>(consumer, producerOne, options)) { + try (var asyncOne = new ParallelEoSStreamProcessor(options)) { asyncOne.subscribe(UniLists.of(topic)); @@ -233,8 +236,11 @@ void correctOffsetVerySimple() { kcu.props.put(ConsumerConfig.CLIENT_ID_CONFIG, "THREE-my-client"); KafkaConsumer newConsumerThree = kcu.createNewConsumer(); KafkaProducer producerThree = kcu.createNewProducer(true); - - try (var asyncThree = new ParallelEoSStreamProcessor<>(newConsumerThree, producerThree, options)) { + ParallelConsumerOptions optionsThree = options.toBuilder() + .consumer(newConsumerThree) + .producer(producerThree) + .build(); + try (var asyncThree = new ParallelEoSStreamProcessor(optionsThree)) { asyncThree.subscribe(UniLists.of(topic)); // read what we're given @@ -273,12 +279,13 @@ void largeNumberOfMessagesSmallOffsetBitmap() { KafkaConsumer consumer = kcu.createNewConsumer(); KafkaProducer producerOne = kcu.createNewProducer(true); - ParallelConsumerOptions options = ParallelConsumerOptions.builder() + var options = ParallelConsumerOptions.builder() .ordering(UNORDERED) - .usingTransactionalProducer(true) + .consumer(consumer) + .producer(producerOne) .commitMode(TRANSACTIONAL_PRODUCER) .build(); - var asyncOne = new ParallelEoSStreamProcessor<>(consumer, producerOne, options); + var asyncOne = new ParallelEoSStreamProcessor(options); asyncOne.subscribe(UniLists.of(topic)); @@ -306,7 +313,11 @@ void largeNumberOfMessagesSmallOffsetBitmap() { kcu.props.put(ConsumerConfig.CLIENT_ID_CONFIG, "THREE-my-client"); KafkaConsumer newConsumerThree = kcu.createNewConsumer(); KafkaProducer producerThree = kcu.createNewProducer(true); - try (var asyncThree = new ParallelEoSStreamProcessor<>(newConsumerThree, producerThree, options)) { + var optionsThree = options.toBuilder() + .consumer(newConsumerThree) + .producer(producerThree) + .build(); + try (var asyncThree = new ParallelEoSStreamProcessor(optionsThree)) { asyncThree.subscribe(UniLists.of(topic)); // read what we're given diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/LoadTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/LoadTest.java index 0bebc18bf..73f58c02c 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/LoadTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/LoadTest.java @@ -72,12 +72,17 @@ void asyncConsumeAndProcess() { // subscribe in advance, it can be a few seconds kcu.consumer.subscribe(UniLists.of(topic)); + KafkaConsumer newConsumer = kcu.createNewConsumer(); // boolean tx = true; - ParallelConsumerOptions options = ParallelConsumerOptions.builder().ordering(ParallelConsumerOptions.ProcessingOrder.KEY).usingTransactionalProducer(tx).commitMode(TRANSACTIONAL_PRODUCER).build(); - KafkaConsumer newConsumer = kcu.createNewConsumer(); + ParallelConsumerOptions options = ParallelConsumerOptions.builder() + .ordering(ParallelConsumerOptions.ProcessingOrder.KEY) + .commitMode(TRANSACTIONAL_PRODUCER) + .producer(kcu.createNewProducer(tx)) + .consumer(newConsumer) + .build(); newConsumer.subscribe(Pattern.compile(topic)); - var async = new ParallelEoSStreamProcessor<>(newConsumer, kcu.createNewProducer(tx), options); + ParallelEoSStreamProcessor async = new ParallelEoSStreamProcessor<>(options); AtomicInteger msgCount = new AtomicInteger(0); ProgressBar pb = new ProgressBarBuilder() diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java index d61d1af1d..fe9954a27 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java @@ -7,9 +7,9 @@ import io.confluent.parallelconsumer.ParallelConsumerOptions; import io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode; import io.confluent.parallelconsumer.ParallelEoSStreamProcessor; -import io.confluent.parallelconsumer.integrationTests.KafkaTest; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; +import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; import org.awaitility.core.ConditionTimeoutException; @@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicInteger; import static io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode.TRANSACTIONAL_PRODUCER; +import static io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder.KEY; import static java.time.Duration.ofSeconds; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.fail; @@ -102,12 +103,19 @@ private void runTest(boolean tx, int maxPoll, CommitMode commitMode) { // run parallel-consumer log.info("Starting application..."); - ParallelEoSStreamProcessor pc = new ParallelEoSStreamProcessor<>(kcu.consumer, kcu.producer, ParallelConsumerOptions.builder().build()); + KafkaProducer newProducer = kcu.createNewProducer(commitMode.equals(TRANSACTIONAL_PRODUCER)); + + var pc = new ParallelEoSStreamProcessor(ParallelConsumerOptions.builder() + .ordering(KEY) + .consumer(kcu.createNewConsumer()) + .producer(newProducer) + .commitMode(commitMode) + .build()); pc.subscribe(UniLists.of(inputName)); // <5> pc.pollAndProduce(record -> { processedCount.incrementAndGet(); - return UniLists.of(new ProducerRecord<>(outputName, record.key(), "data")); + return new ProducerRecord(outputName, record.key(), "data"); }, consumeProduceResult -> { producedCount.incrementAndGet(); processedAndProducedKeys.add(consumeProduceResult.getIn().key()); diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/VolumeTests.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/VolumeTests.java index df0f3c5c1..687d45d7c 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/VolumeTests.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/VolumeTests.java @@ -53,10 +53,8 @@ public class VolumeTests extends ParallelEoSStreamProcessorTestBase { @EnumSource(CommitMode.class) public void load(CommitMode commitMode) { setupClients(); - boolean useTxProducer = commitMode.equals(TRANSACTIONAL_PRODUCER); setupParallelConsumerInstance(ParallelConsumerOptions.builder() .ordering(UNORDERED) - .usingTransactionalProducer(useTxProducer) .commitMode(commitMode) .build()); @@ -68,7 +66,7 @@ public void load(CommitMode commitMode) { CountDownLatch allMessagesConsumedLatch = new CountDownLatch(quantityOfMessagesToProduce); - parallelConsumer.pollAndProduce((rec) -> { + parallelConsumer.pollAndProduceMany((rec) -> { ProducerRecord mock = mock(ProducerRecord.class); return UniLists.of(mock); }, (x) -> { @@ -76,7 +74,6 @@ public void load(CommitMode commitMode) { allMessagesConsumedLatch.countDown(); }); - // allMessagesConsumedLatch.await(defaultTimeoutSeconds, SECONDS); // waitAtMost(defaultTimeout).until(() -> producerSpy.consumerGroupOffsetsHistory().size() > 0); @@ -87,7 +84,7 @@ public void load(CommitMode commitMode) { List> history = producerSpy.history(); assertThat(history).hasSize(quantityOfMessagesToProduce); - if (useTxProducer) { + if (commitMode.equals(TRANSACTIONAL_PRODUCER)) { // assert order of commits assertCommitsAlwaysIncrease(); @@ -147,11 +144,8 @@ public void timingOfDifferentOrderingTypes(CommitMode commitMode) { var quantityOfMessagesToProduce = 10_00; var defaultNumKeys = 20; - boolean useTxProducer = commitMode.equals(TRANSACTIONAL_PRODUCER); - ParallelConsumerOptions baseOptions = ParallelConsumerOptions.builder() .ordering(UNORDERED) - .usingTransactionalProducer(useTxProducer) .commitMode(commitMode) .build(); @@ -219,7 +213,7 @@ private void testTiming(int numberOfKeys, int quantityOfMessagesToProduce) { Queue> processingCheck = new ConcurrentLinkedQueue>(); - parallelConsumer.pollAndProduce((rec) -> { + parallelConsumer.pollAndProduceMany((rec) -> { processingCheck.add(rec); int rangeOfTimeSimulatedProcessingTakesMs = 5; long sleepTime = (long) (Math.random() * rangeOfTimeSimulatedProcessingTakesMs); diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/JStreamParallelEoSStreamProcessorTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/JStreamParallelEoSStreamProcessorTest.java index aab0365a9..f84df4b96 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/JStreamParallelEoSStreamProcessorTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/JStreamParallelEoSStreamProcessorTest.java @@ -36,9 +36,8 @@ public void setupData() { } @Override - protected ParallelEoSStreamProcessor initAsyncConsumer(ParallelConsumerOptions parallelConsumerOptions) { - ParallelConsumerOptions options = ParallelConsumerOptions.builder().build(); - streaming = new JStreamParallelEoSStreamProcessor<>(consumerSpy, producerSpy, options); + protected ParallelEoSStreamProcessor initAsyncConsumer(ParallelConsumerOptions options) { + streaming = new JStreamParallelEoSStreamProcessor<>(options); return streaming; } 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 fbef02bdc..840f7a106 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 @@ -11,6 +11,7 @@ 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.clients.producer.ProducerRecord; import org.apache.kafka.common.TopicPartition; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Disabled; @@ -20,6 +21,7 @@ import org.junit.jupiter.params.provider.EnumSource; import org.mockito.ArgumentMatchers; import org.mockito.Mockito; +import pl.tlinkowski.unij.api.UniLists; import java.time.Duration; import java.util.*; @@ -29,6 +31,7 @@ import static io.confluent.csid.utils.GeneralTestUtils.time; import static io.confluent.csid.utils.KafkaUtils.toTP; import static io.confluent.csid.utils.Range.range; +import static io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode.CONSUMER_ASYNCHRONOUS; import static io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode.TRANSACTIONAL_PRODUCER; import static io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder.KEY; import static java.time.Duration.ofMillis; @@ -36,12 +39,13 @@ import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.*; import static org.awaitility.Awaitility.await; +import static org.awaitility.Awaitility.setDefaultConditionEvaluationListener; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.*; import static org.mockito.internal.verification.VerificationModeFactory.times; import static pl.tlinkowski.unij.api.UniLists.of; -@Timeout(value = 10, unit = SECONDS) +//@Timeout(value = 10, unit = SECONDS) @Slf4j public class ParallelEoSStreamProcessorTest extends ParallelEoSStreamProcessorTestBase { @@ -141,9 +145,10 @@ private void setupParallelConsumerInstance(final CommitMode commitMode) { } private ParallelConsumerOptions getBaseOptions(final CommitMode commitMode) { - return ParallelConsumerOptions.builder() + return ParallelConsumerOptions.builder() .commitMode(commitMode) - .usingTransactionalProducer(commitMode.equals(TRANSACTIONAL_PRODUCER)) + .consumer(consumerSpy) + .producer(producerSpy) .build(); } @@ -248,8 +253,6 @@ public void offsetsAreNeverCommittedForMessagesStillInFlightLong(CommitMode comm public void offsetCommitsAreIsolatedPerPartition(CommitMode commitMode) { setupParallelConsumerInstance(commitMode); -// @Test -// public void offsetCommitsAreIsolatedPerPartition() { sendSecondRecord(consumerSpy); // send three messages - 0,1, to one partition and 3,4 to another partition petitions @@ -276,7 +279,7 @@ public void offsetCommitsAreIsolatedPerPartition(CommitMode commitMode) { // finish processing 1 releaseAndWait(locks, 1); - waitForSomeLoopCycles(5); + waitForSomeLoopCycles(2); // make sure only base offsets are committed assertCommits(of(0, 2)); @@ -331,13 +334,11 @@ public void controlFlowException(CommitMode commitMode) { subscribeParallelConsumerAndMockConsumerTo(INPUT_TOPIC); setupData(); - // cause a control loop error parallelConsumer.addLoopEndCallBack(() -> { throw new RuntimeException("My fake control loop error"); }); - // parallelConsumer.poll((ignore) -> { log.info("Ignoring {}", ignore); @@ -431,7 +432,6 @@ public void processInKeyOrder(CommitMode commitMode) { setupParallelConsumerInstance(ParallelConsumerOptions.builder() .commitMode(commitMode) .ordering(KEY) - .usingTransactionalProducer(commitMode.equals(TRANSACTIONAL_PRODUCER)) .build()); // created a new client above, so have to send the prime record again primeFirstRecord(); @@ -710,4 +710,88 @@ public void ensureLibraryCantBeUsedTwice() { }); } + @ParameterizedTest() + @EnumSource(CommitMode.class) + void consumeFlowDoesntRequireProducer(CommitMode commitMode) { + setupClients(); + + var optionsWithClients = ParallelConsumerOptions.builder() + .consumer(consumerSpy) + .commitMode(commitMode) + .build(); + + if (commitMode.equals(TRANSACTIONAL_PRODUCER)) { + assertThatThrownBy(() -> parallelConsumer = initAsyncConsumer(optionsWithClients)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Producer", "Transaction"); + } else { + parallelConsumer = initAsyncConsumer(optionsWithClients); + attachLoopCounter(parallelConsumer); + + setupData(); + + parallelConsumer.poll((ignore) -> { + //ignore + }); + + waitForSomeLoopCycles(2); + + parallelConsumer.closeDrainFirst(); + + // + assertCommits(of(0, 0, 1)); + } + } + + @Test + void produceMessageFlowRequiresProducer() { + setupClients(); + + var optionsWithClients = ParallelConsumerOptions.builder() + .consumer(consumerSpy) + .commitMode(TRANSACTIONAL_PRODUCER) + .build(); + + assertThatThrownBy(() -> parallelConsumer = initAsyncConsumer(optionsWithClients)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Producer", "Transaction"); + } + + @Test + void cantUseProduceFlowWithWrongOptions() throws InterruptedException { + setupClients(); + + // forget to supply producer + var optionsWithClients = ParallelConsumerOptions.builder() + .consumer(consumerSpy) + .commitMode(CONSUMER_ASYNCHRONOUS) + .build(); + + setupData(); + + var parallel = initAsyncConsumer(optionsWithClients); + + assertThatThrownBy(() -> parallel.pollAndProduce((record) -> + new ProducerRecord<>(INPUT_TOPIC, "hi there"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Producer", "Transaction"); + } + + @ParameterizedTest() + @EnumSource(CommitMode.class) + public void produceMessageFlow(CommitMode commitMode) { + setupParallelConsumerInstance(commitMode); + + parallelConsumer.pollAndProduce((ignore) -> new ProducerRecord<>("Hello", "there")); + + // let it process + waitForSomeLoopCycles(2); + + parallelConsumer.closeDrainFirst(); + + // + assertCommits(of(0, 1)); + + assertThat(producerSpy.history()).hasSize(1); + } } diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorTestBase.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorTestBase.java index f02b79fb2..42b48311c 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorTestBase.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorTestBase.java @@ -12,7 +12,6 @@ import org.apache.kafka.clients.consumer.*; import org.apache.kafka.clients.producer.MockProducer; import org.apache.kafka.common.TopicPartition; -import org.assertj.core.util.Lists; import org.junit.jupiter.api.BeforeEach; import pl.tlinkowski.unij.api.UniLists; import pl.tlinkowski.unij.api.UniMaps; @@ -147,7 +146,12 @@ protected void setupParallelConsumerInstance(ProcessingOrder order) { protected void setupParallelConsumerInstance(ParallelConsumerOptions parallelConsumerOptions) { setupClients(); - parallelConsumer = initAsyncConsumer(parallelConsumerOptions); + var optionsWithClients = parallelConsumerOptions.toBuilder() + .consumer(consumerSpy) + .producer(producerSpy) + .build(); + + parallelConsumer = initAsyncConsumer(optionsWithClients); parallelConsumer.setLongPollTimeout(ofMillis(DEFAULT_BROKER_POLL_FREQUENCY_MS)); parallelConsumer.setTimeBetweenCommits(ofMillis(DEFAULT_COMMIT_INTERVAL_MAX_MS)); @@ -160,7 +164,7 @@ protected void setupParallelConsumerInstance(ParallelConsumerOptions parallelCon } protected ParallelEoSStreamProcessor initAsyncConsumer(ParallelConsumerOptions parallelConsumerOptions) { - parallelConsumer = new ParallelEoSStreamProcessor<>(consumerSpy, producerSpy, parallelConsumerOptions); + parallelConsumer = new ParallelEoSStreamProcessor<>(parallelConsumerOptions); return parallelConsumer; } @@ -223,12 +227,15 @@ protected void waitUntilTrue(Callable booleanCallable) { waitAtMost(defaultTimeout).until(booleanCallable); } + /** + * Make sure the latch is attached, if this times out unexpectedly + */ @SneakyThrows private void blockingLoopLatchTrigger(int waitForCount) { log.debug("Waiting on {} cycles on loop latch...", waitForCount); loopLatchV = new CountDownLatch(waitForCount); - boolean await = loopLatchV.await(defaultTimeoutSeconds, SECONDS); - if (!await) + boolean timeout = !loopLatchV.await(defaultTimeoutSeconds, SECONDS); + if (timeout) throw new TimeoutException(msg("Timeout {} waiting for latch", defaultTimeoutSeconds)); } diff --git a/parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java b/parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java index 290d0fc03..c425c379a 100644 --- a/parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java +++ b/parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java @@ -60,12 +60,12 @@ void run() { @SuppressWarnings({"FeatureEnvy", "MagicNumber"}) ParallelStreamProcessor setupParallelConsumer() { // tag::exampleSetup[] - ParallelConsumerOptions options = getOptions(); - Consumer kafkaConsumer = getKafkaConsumer(); // <4> Producer kafkaProducer = getKafkaProducer(); - ParallelStreamProcessor eosStreamProcessor = ParallelStreamProcessor.createEosStreamProcessor(kafkaConsumer, kafkaProducer, options); + var options = getOptions().toBuilder().consumer(kafkaConsumer).producer(kafkaProducer).build(); + + ParallelStreamProcessor eosStreamProcessor = ParallelStreamProcessor.createEosStreamProcessor(options); if (!(kafkaConsumer instanceof MockConsumer)) { eosStreamProcessor.subscribe(UniLists.of(inputTopic)); // <5> } @@ -93,7 +93,7 @@ void runPollAndProduce() { // tag::exampleProduce[] this.parallelConsumer.pollAndProduce(record -> { var result = processBrokerRecord(record); - return UniLists.of(new ProducerRecord<>(outputTopic, record.key(), result.payload)); + return new ProducerRecord<>(outputTopic, record.key(), result.payload); }, consumeProduceResult -> { log.debug("Message {} saved to broker at offset {}", consumeProduceResult.getOut(), diff --git a/parallel-consumer-examples/parallel-consumer-example-streams/src/main/java/io/confluent/parallelconsumer/examples/streams/StreamsApp.java b/parallel-consumer-examples/parallel-consumer-example-streams/src/main/java/io/confluent/parallelconsumer/examples/streams/StreamsApp.java index 1c192f5f9..1fdfa98d5 100644 --- a/parallel-consumer-examples/parallel-consumer-example-streams/src/main/java/io/confluent/parallelconsumer/examples/streams/StreamsApp.java +++ b/parallel-consumer-examples/parallel-consumer-example-streams/src/main/java/io/confluent/parallelconsumer/examples/streams/StreamsApp.java @@ -78,17 +78,18 @@ void concurrentProcess() { // end::example[] private void setupParallelConsumer() { - var options = ParallelConsumerOptions.builder() + Consumer kafkaConsumer = getKafkaConsumer(); + Producer kafkaProducer = getKafkaProducer(); + var options = ParallelConsumerOptions.builder() .ordering(ParallelConsumerOptions.ProcessingOrder.KEY) .maxConcurrency(1000) .maxUncommittedMessagesToHandle(10000) + .consumer(kafkaConsumer) + .producer(kafkaProducer) .build(); - Consumer kafkaConsumer = getKafkaConsumer(); - Producer kafkaProducer = getKafkaProducer(); - - parallelConsumer = ParallelStreamProcessor.createEosStreamProcessor(kafkaConsumer, kafkaProducer, options); + parallelConsumer = ParallelStreamProcessor.createEosStreamProcessor(options); parallelConsumer.subscribe(UniLists.of(outputTopicName)); } diff --git a/parallel-consumer-examples/parallel-consumer-example-vertx/src/main/java/io/confluent/parallelconsumer/examples/vertx/VertxApp.java b/parallel-consumer-examples/parallel-consumer-example-vertx/src/main/java/io/confluent/parallelconsumer/examples/vertx/VertxApp.java index 62dc5b191..e9991bc76 100644 --- a/parallel-consumer-examples/parallel-consumer-example-vertx/src/main/java/io/confluent/parallelconsumer/examples/vertx/VertxApp.java +++ b/parallel-consumer-examples/parallel-consumer-example-vertx/src/main/java/io/confluent/parallelconsumer/examples/vertx/VertxApp.java @@ -37,17 +37,19 @@ Producer getKafkaProducer() { void run() { - var options = ParallelConsumerOptions.builder() + Consumer kafkaConsumer = getKafkaConsumer(); + Producer kafkaProducer = getKafkaProducer(); + var options = ParallelConsumerOptions.builder() .ordering(ParallelConsumerOptions.ProcessingOrder.KEY) .maxConcurrency(1000) .maxUncommittedMessagesToHandle(10000) + .consumer(kafkaConsumer) + .producer(kafkaProducer) .build(); - Consumer kafkaConsumer = getKafkaConsumer(); setupSubscription(kafkaConsumer); - this.parallelConsumer = JStreamVertxParallelStreamProcessor.createEosStreamProcessor(kafkaConsumer, - getKafkaProducer(), options); + this.parallelConsumer = JStreamVertxParallelStreamProcessor.createEosStreamProcessor(options); int port = getPort(); diff --git a/parallel-consumer-vertx/src/main/java/io/confluent/parallelconsumer/vertx/JStreamVertxParallelEoSStreamProcessor.java b/parallel-consumer-vertx/src/main/java/io/confluent/parallelconsumer/vertx/JStreamVertxParallelEoSStreamProcessor.java index 2724cedae..1ddf16616 100644 --- a/parallel-consumer-vertx/src/main/java/io/confluent/parallelconsumer/vertx/JStreamVertxParallelEoSStreamProcessor.java +++ b/parallel-consumer-vertx/src/main/java/io/confluent/parallelconsumer/vertx/JStreamVertxParallelEoSStreamProcessor.java @@ -46,12 +46,10 @@ public class JStreamVertxParallelEoSStreamProcessor extends VertxParallelE *

* Use this to share a Vertx runtime with different systems for efficiency. */ - public JStreamVertxParallelEoSStreamProcessor(org.apache.kafka.clients.consumer.Consumer consumer, - Producer producer, - Vertx vertx, + public JStreamVertxParallelEoSStreamProcessor(Vertx vertx, WebClient webClient, - ParallelConsumerOptions options) { - super(consumer, producer, vertx, webClient, options); + ParallelConsumerOptions options) { + super(vertx, webClient, options); this.userProcessResultsStream = new ConcurrentLinkedDeque<>(); @@ -61,10 +59,8 @@ public JStreamVertxParallelEoSStreamProcessor(org.apache.kafka.clients.consumer. /** * Simple constructor. Internal Vertx objects will be created. */ - public JStreamVertxParallelEoSStreamProcessor(org.apache.kafka.clients.consumer.Consumer consumer, - Producer producer, - ParallelConsumerOptions options) { - this(consumer, producer, null, null, options); + public JStreamVertxParallelEoSStreamProcessor(ParallelConsumerOptions options) { + this(null, null, options); } @Override diff --git a/parallel-consumer-vertx/src/main/java/io/confluent/parallelconsumer/vertx/JStreamVertxParallelStreamProcessor.java b/parallel-consumer-vertx/src/main/java/io/confluent/parallelconsumer/vertx/JStreamVertxParallelStreamProcessor.java index bbc677b19..9c9b722b5 100644 --- a/parallel-consumer-vertx/src/main/java/io/confluent/parallelconsumer/vertx/JStreamVertxParallelStreamProcessor.java +++ b/parallel-consumer-vertx/src/main/java/io/confluent/parallelconsumer/vertx/JStreamVertxParallelStreamProcessor.java @@ -24,11 +24,8 @@ */ public interface JStreamVertxParallelStreamProcessor extends DrainingCloseable { - static JStreamVertxParallelStreamProcessor createEosStreamProcessor( - org.apache.kafka.clients.consumer.Consumer consumer, - org.apache.kafka.clients.producer.Producer producer, - ParallelConsumerOptions options) { - return new JStreamVertxParallelEoSStreamProcessor<>(consumer, producer, options); + static JStreamVertxParallelStreamProcessor createEosStreamProcessor(ParallelConsumerOptions options) { + return new JStreamVertxParallelEoSStreamProcessor<>(options); } /** diff --git a/parallel-consumer-vertx/src/main/java/io/confluent/parallelconsumer/vertx/VertxParallelEoSStreamProcessor.java b/parallel-consumer-vertx/src/main/java/io/confluent/parallelconsumer/vertx/VertxParallelEoSStreamProcessor.java index 070be97df..bf9df7dcb 100644 --- a/parallel-consumer-vertx/src/main/java/io/confluent/parallelconsumer/vertx/VertxParallelEoSStreamProcessor.java +++ b/parallel-consumer-vertx/src/main/java/io/confluent/parallelconsumer/vertx/VertxParallelEoSStreamProcessor.java @@ -64,7 +64,7 @@ public class VertxParallelEoSStreamProcessor extends ParallelEoSStreamProc public VertxParallelEoSStreamProcessor(org.apache.kafka.clients.consumer.Consumer consumer, Producer producer, ParallelConsumerOptions options) { - this(consumer, producer, Vertx.vertx(), null, options); + this(Vertx.vertx(), null, options); } /** @@ -72,11 +72,10 @@ public VertxParallelEoSStreamProcessor(org.apache.kafka.clients.consumer.Consume *

* Use this to share a Vertx runtime with different systems for efficiency. */ - public VertxParallelEoSStreamProcessor(org.apache.kafka.clients.consumer.Consumer consumer, Producer producer, - Vertx vertx, + public VertxParallelEoSStreamProcessor(Vertx vertx, WebClient webClient, ParallelConsumerOptions options) { - super(consumer, producer, options); + super(options); if (vertx == null) vertx = Vertx.vertx(); this.vertx = vertx; diff --git a/parallel-consumer-vertx/src/test/java/io/confluent/parallelconsumer/vertx/VertxNonVertxOperations.java b/parallel-consumer-vertx/src/test/java/io/confluent/parallelconsumer/vertx/VertxNonVertxOperations.java index df4d6516f..cf72d299f 100644 --- a/parallel-consumer-vertx/src/test/java/io/confluent/parallelconsumer/vertx/VertxNonVertxOperations.java +++ b/parallel-consumer-vertx/src/test/java/io/confluent/parallelconsumer/vertx/VertxNonVertxOperations.java @@ -20,7 +20,7 @@ public class VertxNonVertxOperations extends ParallelEoSStreamProcessorTest { protected ParallelEoSStreamProcessor initAsyncConsumer(ParallelConsumerOptions parallelConsumerOptions) { VertxOptions vertxOptions = new VertxOptions(); Vertx vertx = Vertx.vertx(vertxOptions); - parallelConsumer = new VertxParallelEoSStreamProcessor<>(consumerSpy, producerSpy, vertx, WebClient.create(vertx), parallelConsumerOptions); + parallelConsumer = new VertxParallelEoSStreamProcessor<>(vertx, WebClient.create(vertx), parallelConsumerOptions); return parallelConsumer; } diff --git a/parallel-consumer-vertx/src/test/java/io/confluent/parallelconsumer/vertx/VertxTest.java b/parallel-consumer-vertx/src/test/java/io/confluent/parallelconsumer/vertx/VertxTest.java index 58ff082a8..b8e7aca6c 100644 --- a/parallel-consumer-vertx/src/test/java/io/confluent/parallelconsumer/vertx/VertxTest.java +++ b/parallel-consumer-vertx/src/test/java/io/confluent/parallelconsumer/vertx/VertxTest.java @@ -83,8 +83,10 @@ protected ParallelEoSStreamProcessor initAsyncConsumer(ParallelConsumerOptions p VertxOptions vertxOptions = new VertxOptions(); Vertx vertx = Vertx.vertx(vertxOptions); WebClient wc = WebClient.create(vertx); - ParallelConsumerOptions build = ParallelConsumerOptions.builder().commitMode(TRANSACTIONAL_PRODUCER).usingTransactionalProducer(true).build(); - vertxAsync = new JStreamVertxParallelEoSStreamProcessor<>(consumerSpy, producerSpy, vertx, wc, build); + var build = parallelConsumerOptions.toBuilder() + .commitMode(TRANSACTIONAL_PRODUCER) // force tx + .build(); + vertxAsync = new JStreamVertxParallelEoSStreamProcessor<>(vertx, wc, build); return vertxAsync; } From b8d78474d5c6228241d386233832da0bfa243fe2 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Fri, 20 Nov 2020 14:40:10 +0000 Subject: [PATCH 04/11] docs: Explanations for concurrency limits --- README.adoc | 10 +++---- .../ParallelConsumerOptions.java | 28 +++++++++++++------ .../parallelconsumer/WorkManager.java | 12 ++++---- .../parallelconsumer/WorkManagerTest.java | 12 ++++---- .../examples/core/CoreApp.java | 4 +-- .../examples/streams/StreamsApp.java | 4 +-- .../examples/vertx/VertxApp.java | 4 +-- src/docs/README.adoc | 10 +++---- 8 files changed, 47 insertions(+), 37 deletions(-) diff --git a/README.adoc b/README.adoc index 0c0c1eb7b..23bbe917a 100644 --- a/README.adoc +++ b/README.adoc @@ -195,6 +195,7 @@ without operational burden or harming the clusters performance * Vert.x non-blocking library integration (HTTP currently) * Fair partition traversal * Zero~ dependencies (`Slf4j` and `Lombok`) for the core module +* Java 8 compatibility * Throttle control and broker liveliness management * Clean draining shutdown cycle @@ -646,21 +647,20 @@ image::https://lucid.app/publicSegments/view/43f2740c-2a7f-4b7f-909e-434a5bbe3fb === Short Term - What we're working on nowish ⏰ -* Producer is optional -* Transactions optional * Depth~ first or breadth first partition traversal * JavaRX and other streaming modules === Medium Term - What's up next ⏲ -* Automatic fanout (automatic selection of concurrency level based on downstream back pressure) +* https://github.com/confluentinc/parallel-consumer/issues/21[Automatic fanout] (automatic selection of concurrency level based on downstream back pressure) (https://github.com/confluentinc/parallel-consumer/pull/22[draft PR]) * Support for general Vert.x Verticles (non-blocking libraries) * Dead Letter Queue (DLQ) handling * Non-blocking I/O work management ** More customisable handling of HTTP interactions ** Chance to batch multiple consumer records into a single or multiple http request objects -* Distributed tracing integration -* Metrics +* https://github.com/confluentinc/parallel-consumer/issues/28[Distributed tracing integration] +* https://github.com/confluentinc/parallel-consumer/issues/24[Distributed rate limiting] +* https://github.com/confluentinc/parallel-consumer/issues/27[Metrics] === Long Term - The future ☁️ diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java index 00f425e9a..a5540b679 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java @@ -87,24 +87,36 @@ public enum CommitMode { private final CommitMode commitMode = CommitMode.CONSUMER_ASYNCHRONOUS; /** - * Don't have more than this many uncommitted messages in process + * Total max number of messages to process beyond the base committed offsets. *

- * TODO docs - needs renaming. Messages aren't "uncommitted", they're just in the comitted offset map instead of the - * committed base offset + * This acts as a sort sort of upper limit on the number of messages we should allow our system to handle, when + * working with large quantities of messages that haven't been included in the normal Broker offset commit protocol. + * I.e. if there is a single message that is failing to process, without this limit we will continue on forever with + * our system, with the actual (normal) committed offset never moving, and relying totally on our {@link + * OffsetMapCodecManager} to encode the process status of our records and store in metadata next to the committed + * offset. + *

+ * At the moment this is a sort of sanity check, and was chosen rather arbitriarly. However, one should consider + * that this is per client, and is a total across all assigned partitions. */ @Builder.Default - private final int maxUncommittedMessagesToHandle = 1000; + private final int maxNumberMessagesBeyondBaseCommitOffset = 1000; /** - * Don't process any more than this many messages concurrently + * Max number of messages to queue up in our execution system and attempt to process concurrently. + *

+ * In the core module, this will be constrained by the {@link #numberOfThreads} setting, as that is the max actual + * concurrency for processing the messages. To actually get this degree of concurrency, you would need to have a + * matching number of threads in the pool. *

- * TODO docs differentiate from thread count, vertx etc. remove to vertx module? + * However with the VertX engine, this will control how many messages at a time are being submitted to the Vert.x + * engine to process. As Vert.x isn't constrained by a thread count, this will be the actual degree of concurrency. */ @Builder.Default - private final int maxConcurrency = 100; + private final int maxMessagesToQueue = 100; /** - * TODO docs. rename to max concurrency. differentiate between this and vertx threads + * Number of threads to use in the core's thread pool. */ @Builder.Default private final int numberOfThreads = 16; diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/WorkManager.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/WorkManager.java index fe54543b1..a1ff0353d 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/WorkManager.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/WorkManager.java @@ -19,11 +19,9 @@ import pl.tlinkowski.unij.api.UniLists; import pl.tlinkowski.unij.api.UniSets; -import java.time.Duration; import java.util.*; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentSkipListMap; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Consumer; import static io.confluent.csid.utils.KafkaUtils.toTP; @@ -75,7 +73,7 @@ public class WorkManager implements ConsumerRebalanceListener { private int inFlightCount = 0; /** - * The multiple of {@link ParallelConsumerOptions#getMaxConcurrency()} that should be pre-loaded awaiting + * The multiple of {@link ParallelConsumerOptions#getMaxMessagesToQueue()} that should be pre-loaded awaiting * processing. Consumer already pipelines, so we shouldn't need to pipeline ourselves too much. *

* Note how this relates to {@link BrokerPollSystem#getLongPollTimeout()} - if longPollTimeout is high and loading @@ -258,7 +256,7 @@ private Object computeShardKey(ConsumerRecord rec) { } public List> maybeGetWork() { - return maybeGetWork(options.getMaxConcurrency()); + return maybeGetWork(options.getMaxMessagesToQueue()); } /** @@ -267,7 +265,7 @@ public List> maybeGetWork() { * @param requestedMaxWorkToRetrieve ignored unless less than {@link ParallelConsumerOptions#maxConcurrency} */ public List> maybeGetWork(int requestedMaxWorkToRetrieve) { - int minWorkToGetSetting = min(min(requestedMaxWorkToRetrieve, options.getMaxConcurrency()), options.getMaxUncommittedMessagesToHandle()); + int minWorkToGetSetting = min(min(requestedMaxWorkToRetrieve, options.getMaxMessagesToQueue()), options.getMaxNumberMessagesBeyondBaseCommitOffset()); int workToGetDelta = minWorkToGetSetting - getInFlightCount(); // optimise early @@ -542,8 +540,8 @@ public boolean shouldThrottle() { boolean isSufficientlyLoaded() { int remaining = getPartitionWorkRemainingCount(); - boolean loadedEnoughInPipeline = remaining > options.getMaxConcurrency() * loadingFactor; - boolean overMaxUncommitted = remaining > options.getMaxUncommittedMessagesToHandle(); + boolean loadedEnoughInPipeline = remaining > options.getMaxMessagesToQueue() * loadingFactor; + boolean overMaxUncommitted = remaining > options.getMaxNumberMessagesBeyondBaseCommitOffset(); boolean remainingIsSufficient = loadedEnoughInPipeline || overMaxUncommitted; if (remainingIsSufficient) { log.debug("loadedEnoughInPipeline {} || overMaxUncommitted {}", loadedEnoughInPipeline, overMaxUncommitted); diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/WorkManagerTest.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/WorkManagerTest.java index 1dc21746c..7736f7068 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/WorkManagerTest.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/WorkManagerTest.java @@ -318,7 +318,7 @@ public void maxPerTopic() { public void maxInFlight() { // var opts = ParallelConsumerOptions.builder(); - opts.maxUncommittedMessagesToHandle(1); + opts.maxNumberMessagesBeyondBaseCommitOffset(1); setupWorkManager(opts.build()); // @@ -333,7 +333,7 @@ public void maxInFlight() { public void maxConcurrency() { // var opts = ParallelConsumerOptions.builder(); - opts.maxConcurrency(1); + opts.maxMessagesToQueue(1); setupWorkManager(opts.build()); // @@ -372,8 +372,8 @@ public void maxConcurrencyVsInFlightAndNoLeaks() { var opts = ParallelConsumerOptions.builder(); opts.ordering(UNORDERED); - opts.maxUncommittedMessagesToHandle(3); - opts.maxConcurrency(2); + opts.maxNumberMessagesBeyondBaseCommitOffset(3); + opts.maxMessagesToQueue(2); setupWorkManager(opts.build()); @@ -609,8 +609,8 @@ public void treeMapOrderingCorrect() { public void workQueuesEmptyWhenAllWorkComplete() { ParallelConsumerOptions build = ParallelConsumerOptions.builder() .ordering(UNORDERED) - .maxConcurrency(10) - .maxUncommittedMessagesToHandle(10) + .maxMessagesToQueue(10) + .maxNumberMessagesBeyondBaseCommitOffset(10) .build(); setupWorkManager(build); registerSomeWork(); diff --git a/parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java b/parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java index c425c379a..36973da8e 100644 --- a/parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java +++ b/parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java @@ -77,8 +77,8 @@ ParallelStreamProcessor setupParallelConsumer() { ParallelConsumerOptions getOptions() { var options = ParallelConsumerOptions.builder() .ordering(KEY) // <1> - .maxConcurrency(1000) // <2> - .maxUncommittedMessagesToHandle(1000) // <3> + .maxMessagesToQueue(1000) // <2> + .maxNumberMessagesBeyondBaseCommitOffset(1000) // <3> .build(); return options; } diff --git a/parallel-consumer-examples/parallel-consumer-example-streams/src/main/java/io/confluent/parallelconsumer/examples/streams/StreamsApp.java b/parallel-consumer-examples/parallel-consumer-example-streams/src/main/java/io/confluent/parallelconsumer/examples/streams/StreamsApp.java index 1fdfa98d5..7bf6e00b5 100644 --- a/parallel-consumer-examples/parallel-consumer-example-streams/src/main/java/io/confluent/parallelconsumer/examples/streams/StreamsApp.java +++ b/parallel-consumer-examples/parallel-consumer-example-streams/src/main/java/io/confluent/parallelconsumer/examples/streams/StreamsApp.java @@ -82,8 +82,8 @@ private void setupParallelConsumer() { Producer kafkaProducer = getKafkaProducer(); var options = ParallelConsumerOptions.builder() .ordering(ParallelConsumerOptions.ProcessingOrder.KEY) - .maxConcurrency(1000) - .maxUncommittedMessagesToHandle(10000) + .maxMessagesToQueue(1000) + .maxNumberMessagesBeyondBaseCommitOffset(10000) .consumer(kafkaConsumer) .producer(kafkaProducer) .build(); diff --git a/parallel-consumer-examples/parallel-consumer-example-vertx/src/main/java/io/confluent/parallelconsumer/examples/vertx/VertxApp.java b/parallel-consumer-examples/parallel-consumer-example-vertx/src/main/java/io/confluent/parallelconsumer/examples/vertx/VertxApp.java index e9991bc76..e0e0e58d3 100644 --- a/parallel-consumer-examples/parallel-consumer-example-vertx/src/main/java/io/confluent/parallelconsumer/examples/vertx/VertxApp.java +++ b/parallel-consumer-examples/parallel-consumer-example-vertx/src/main/java/io/confluent/parallelconsumer/examples/vertx/VertxApp.java @@ -41,8 +41,8 @@ void run() { Producer kafkaProducer = getKafkaProducer(); var options = ParallelConsumerOptions.builder() .ordering(ParallelConsumerOptions.ProcessingOrder.KEY) - .maxConcurrency(1000) - .maxUncommittedMessagesToHandle(10000) + .maxMessagesToQueue(1000) + .maxNumberMessagesBeyondBaseCommitOffset(10000) .consumer(kafkaConsumer) .producer(kafkaProducer) .build(); diff --git a/src/docs/README.adoc b/src/docs/README.adoc index 80b24821d..3e77d0555 100644 --- a/src/docs/README.adoc +++ b/src/docs/README.adoc @@ -193,6 +193,7 @@ without operational burden or harming the clusters performance * Vert.x non-blocking library integration (HTTP currently) * Fair partition traversal * Zero~ dependencies (`Slf4j` and `Lombok`) for the core module +* Java 8 compatibility * Throttle control and broker liveliness management * Clean draining shutdown cycle @@ -573,21 +574,20 @@ image::https://lucid.app/publicSegments/view/43f2740c-2a7f-4b7f-909e-434a5bbe3fb === Short Term - What we're working on nowish ⏰ -* Producer is optional -* Transactions optional * Depth~ first or breadth first partition traversal * JavaRX and other streaming modules === Medium Term - What's up next ⏲ -* Automatic fanout (automatic selection of concurrency level based on downstream back pressure) +* https://github.com/confluentinc/parallel-consumer/issues/21[Automatic fanout] (automatic selection of concurrency level based on downstream back pressure) (https://github.com/confluentinc/parallel-consumer/pull/22[draft PR]) * Support for general Vert.x Verticles (non-blocking libraries) * Dead Letter Queue (DLQ) handling * Non-blocking I/O work management ** More customisable handling of HTTP interactions ** Chance to batch multiple consumer records into a single or multiple http request objects -* Distributed tracing integration -* Metrics +* https://github.com/confluentinc/parallel-consumer/issues/28[Distributed tracing integration] +* https://github.com/confluentinc/parallel-consumer/issues/24[Distributed rate limiting] +* https://github.com/confluentinc/parallel-consumer/issues/27[Metrics] === Long Term - The future ☁️ From fed55f05b9f02e81b99afa361b1248061afc7c78 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Fri, 20 Nov 2020 16:53:44 +0000 Subject: [PATCH 05/11] Delombok for Javadoc (allows references to Lombok elements in Javadoc) --- .gitignore | 5 ++++- pom.xml | 31 ++++++++++++++++++++++++++++++- 2 files changed, 34 insertions(+), 2 deletions(-) diff --git a/.gitignore b/.gitignore index 05e3ba15b..cb8ad9f7f 100644 --- a/.gitignore +++ b/.gitignore @@ -32,4 +32,7 @@ target/ *.versionsBackup # JENV -.java-version \ No newline at end of file +.java-version + +delombok/ +**/*.releaseBackup \ No newline at end of file diff --git a/pom.xml b/pom.xml index 8d609a8b8..984ee283d 100644 --- a/pom.xml +++ b/pom.xml @@ -4,7 +4,8 @@ Copyright (C) 2020 Confluent, Inc. --> - + 4.0.0 io.confluent.parallelconsumer @@ -75,6 +76,7 @@ -insert jvm8 location via environment variable- -insert jvm9 location via environment variable- check + ${project.basedir}/target/delombok 3.6.3 @@ -568,10 +570,37 @@ maven-project-info-reports-plugin 3.1.0 + + org.projectlombok + lombok-maven-plugin + 1.18.16.0 + + ${project.basedir}/src/main/java + ${delombok.output} + false + + + + generate-sources + + delombok + + + + org.apache.maven.plugins maven-javadoc-plugin 3.1.1 + + ${delombok.output} + ${delombok.output} + + -Xdoclint:none + attach-javadocs From 97ddead67ba5343c8ec35c8546f7801d4ec90c48 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Fri, 20 Nov 2020 21:04:46 +0000 Subject: [PATCH 06/11] bug: Fixes bug in commit linger, remove genesis offset (0) from testing (avoid races), add ability to request commit --- .../ParallelConsumerOptions.java | 8 +++ .../ParallelEoSStreamProcessor.java | 39 ++++++++-- .../ParallelStreamProcessor.java | 3 +- .../TransactionAndCommitModeTest.java | 2 +- .../confluent/csid/utils/KafkaTestUtils.java | 29 ++++++-- .../csid/utils/LongPollingMockConsumer.java | 2 +- .../ParallelEoSStreamProcessorTest.java | 71 ++++++++++--------- .../ParallelEoSStreamProcessorTestBase.java | 20 ++++-- .../src/test/resources/logback-test.xml | 2 +- .../parallelconsumer/vertx/VertxTest.java | 4 +- 10 files changed, 124 insertions(+), 56 deletions(-) diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java index a5540b679..361476010 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java @@ -26,8 +26,16 @@ @ToString public class ParallelConsumerOptions { + /** + * Required parameter for all use. + */ private final Consumer consumer; + /** + * Supplying a producer is only needed if using the produce flows. + * + * @see ParallelStreamProcessor + */ private final Producer producer; /** 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 205bf1714..d74396ddd 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 @@ -98,6 +98,11 @@ public class ParallelEoSStreamProcessor implements ParallelStreamProcessor private final OffsetCommitter committer; + /** + * Used to request a commit asap + */ + private final AtomicBoolean commitCommand = new AtomicBoolean(false); + /** * The run state of the controller. * @@ -661,7 +666,8 @@ private void processWorkCompleteMailBox() { private void commitOffsetsMaybe() { Duration elapsedSinceLast = getTimeSinceLastCommit(); boolean commitFrequencyOK = toSeconds(elapsedSinceLast) >= toSeconds(timeBetweenCommits); - if (commitFrequencyOK || lingeringOnCommitWouldBeBeneficial()) { + boolean shouldCommitNow = commitFrequencyOK || !lingeringOnCommitWouldBeBeneficial() || isCommandedToCommit(); + if (shouldCommitNow) { if (!commitFrequencyOK) { log.debug("Commit too frequent, but no benefit in lingering"); } @@ -682,14 +688,14 @@ private void commitOffsetsMaybe() { * @return */ private boolean lingeringOnCommitWouldBeBeneficial() { - // no work is waiting to be done + // work is waiting to be done boolean workIsWaitingToBeCompletedSuccessfully = wm.workIsWaitingToBeCompletedSuccessfully(); // no work is currently being done - boolean noWorkInFlight = wm.hasWorkInFlight(); + boolean noWorkInFlight = !wm.hasWorkInFlight(); // work mailbox is empty - boolean workMailBoxEmpty = workMailBox.isEmpty(); - - return workIsWaitingToBeCompletedSuccessfully || noWorkInFlight || workMailBoxEmpty; + boolean workWaitingInMailbox = !workMailBox.isEmpty(); + log.trace("workIsWaitingToBeCompletedSuccessfully {} || noWorkInFlight {} || workWaitingInMailbox {};", workIsWaitingToBeCompletedSuccessfully, noWorkInFlight, workWaitingInMailbox); + return workIsWaitingToBeCompletedSuccessfully || noWorkInFlight || workWaitingInMailbox; } private Duration getTimeToNextCommit() { @@ -840,4 +846,25 @@ public void setLongPollTimeout(Duration ofMillis) { BrokerPollSystem.setLongPollTimeout(ofMillis); } + /** + * Request a commit as soon as possible (ASAP), overriding other constraints. + */ + public void requestCommitAsap() { + log.debug("Registering command to commit next chance"); + synchronized (commitCommand) { + this.commitCommand.set(true); + } + } + + private boolean isCommandedToCommit() { + synchronized (commitCommand) { + boolean commitAsap = this.commitCommand.get(); + if (commitAsap) { + log.debug("Command to commit asap received, clearing"); + this.commitCommand.set(false); + } + return commitAsap; + } + } + } diff --git a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelStreamProcessor.java b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelStreamProcessor.java index 89f063380..902580931 100644 --- a/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelStreamProcessor.java +++ b/parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelStreamProcessor.java @@ -18,7 +18,8 @@ * Parallel message consumer which also can optionally produce 0 or many {@link ProducerRecord} results to be published * back to Kafka. * - * @see #pollAndProduceMany(Function, Consumer) + * @see #pollAndProduce + * @see #pollAndProduceMany */ public interface ParallelStreamProcessor extends ParallelConsumer, DrainingCloseable { diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java index fe9954a27..e70340fac 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java @@ -125,7 +125,7 @@ private void runTest(boolean tx, int maxPoll, CommitMode commitMode) { // wait for all pre-produced messages to be processed and produced String failureMessage = "All keys sent to input-topic should be processed and produced"; try { - waitAtMost(ofSeconds(10)).alias(failureMessage).untilAsserted(() -> { + waitAtMost(ofSeconds(30)).alias(failureMessage).untilAsserted(() -> { log.debug("Processed-count: " + processedCount.get()); log.debug("Produced-count: " + producedCount.get()); List processedAndProducedKeysCopy = new ArrayList<>(processedAndProducedKeys); // avoid concurrent-modification in assert 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 3dd09da7b..8777e1053 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 @@ -47,6 +47,19 @@ public static void assignConsumerToTopic(final MockConsumer mc) { mc.updateBeginningOffsets(beginningOffsets); } + /** + * It's a race to see if the genesis offset gets committed or not. So lets remove it if it exists, and all tests can + * assume it doesnt. + */ + public static List trimAllGeneisOffset(final List collect) { + while (!collect.isEmpty() && collect.get(0) == 0) { + int genesisOffset = collect.get(0); + if (genesisOffset == 0) + collect.remove(0); + } + return collect; + } + public ConsumerRecord makeRecord(String key, String value) { return makeRecord(0, key, value); } @@ -70,7 +83,7 @@ public static void assertCommits(MockProducer mp, List expectedOffsets, log.debug("Asserting commits of {}", expectedOffsets); List>> history = mp.consumerGroupOffsetsHistory(); - Set set = history.stream().flatMap(histories -> { + List set = history.stream().flatMap(histories -> { // get all partition offsets and flatten var results = new ArrayList(); var group = histories.get(CONSUMER_GROUP_ID); @@ -79,8 +92,9 @@ public static void assertCommits(MockProducer mp, List expectedOffsets, int offset = (int) commit.offset(); results.add(offset); } - return results.stream(); - }).collect(Collectors.toSet()); // set - ignore repeated commits ({@link OffsetMap}) + List integers = KafkaTestUtils.trimAllGeneisOffset(results); + return integers.stream(); + }).collect(Collectors.toList()); // set - ignore repeated commits ({@link OffsetMap}) assertThat(set).describedAs(description.orElse("Which offsets are committed and in the expected order")) .containsExactlyElementsOf(expectedOffsets); @@ -92,7 +106,7 @@ public static void assertCommitLists(MockProducer mp, List> expect /** * Collects into a set - ignore repeated commits ({@link OffsetMapCodecManager}). - * + *

* Ignores duplicates. * * @see OffsetMapCodecManager @@ -102,7 +116,7 @@ public static void assertCommitLists(List topicName = new AtomicReference<>(""); var partitionToCommittedOffsets = new HashMap>(); // set - ignore repeated commits ({@link OffsetMap}) - history.stream().forEachOrdered(histories -> { + new ArrayList<>(history).stream().forEachOrdered(histories -> { // get all partition offsets and flatten var partitionCommits = histories.get(CONSUMER_GROUP_ID); for (var singlePartitionCommit : partitionCommits.entrySet()) { @@ -110,7 +124,10 @@ public static void assertCommitLists(List new HashSet<>()).add(offset); + partitionToCommittedOffsets.computeIfAbsent(key, x -> new HashSet<>()); + // ignore genesis commits + if (offset != 0) + partitionToCommittedOffsets.get(key).add(offset); } }); diff --git a/parallel-consumer-core/src/test/java/io/confluent/csid/utils/LongPollingMockConsumer.java b/parallel-consumer-core/src/test/java/io/confluent/csid/utils/LongPollingMockConsumer.java index 18629f957..edb7a05e2 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/csid/utils/LongPollingMockConsumer.java +++ b/parallel-consumer-core/src/test/java/io/confluent/csid/utils/LongPollingMockConsumer.java @@ -117,7 +117,7 @@ private void revokeAssignment() throws NoSuchFieldException, IllegalAccessExcept // execute if (consumerRebalanceListener == null) { - log.debug("No rebalance listener assigned - on revoke can't fire"); + log.warn("No rebalance listener assigned - on revoke can't fire"); } else { Set assignment = super.assignment(); consumerRebalanceListener.onPartitionsRevoked(assignment); 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 840f7a106..26d9a7541 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 @@ -45,7 +45,7 @@ import static org.mockito.internal.verification.VerificationModeFactory.times; import static pl.tlinkowski.unij.api.UniLists.of; -//@Timeout(value = 10, unit = SECONDS) +@Timeout(value = 10, unit = SECONDS) @Slf4j public class ParallelEoSStreamProcessorTest extends ParallelEoSStreamProcessorTestBase { @@ -78,7 +78,7 @@ public void failingActionNothingCommitted(CommitMode commitMode) { waitForSomeLoopCycles(loops); // - assertCommits(of(0), "All erroring, nothing committed except initial"); + assertCommits(of(), "All erroring, nothing committed except initial"); List>> maps = getCommitHistory(); assertThat(maps).isNotEmpty(); List metas = new ArrayList<>(); @@ -126,8 +126,10 @@ public void offsetsAreNeverCommittedForMessagesStillInFlightSimplest(CommitMode // finish processing 1 releaseAndWait(locks, 1); + parallelConsumer.requestCommitAsap(); + // make sure no offsets are committed - assertCommits(of(0), "Partition is blocked"); + assertCommits(of(), "Partition is blocked"); // So it's data is setup can be used in other tests, finish 0 releaseAndWait(locks, 0); @@ -160,7 +162,7 @@ public void offsetsAreNeverCommittedForMessagesStillInFlightShort(CommitMode com log.info("Test start"); // next expected offset is now 2 - assertCommits(of(0, 2), "Only one of the two offsets committed, as they were coalesced for efficiency"); + assertCommits(of(2), "Only one of the two offsets committed, as they were coalesced for efficiency"); } @Disabled @@ -279,29 +281,32 @@ public void offsetCommitsAreIsolatedPerPartition(CommitMode commitMode) { // finish processing 1 releaseAndWait(locks, 1); + parallelConsumer.requestCommitAsap(); + waitForSomeLoopCycles(2); // make sure only base offsets are committed - assertCommits(of(0, 2)); - assertCommitLists(of(of(0), of(2))); +// assertCommits(of(2)); + assertCommitLists(of(of(), of(2))); // finish 2 releaseAndWait(locks, 2); + parallelConsumer.requestCommitAsap(); waitForOneLoopCycle(); // make sure only 2 on it's partition of committed - assertCommits(of(0, 2, 3)); - assertCommitLists(of(of(0), of(2, 3))); +// assertCommits(of(2, 3)); + assertCommitLists(of(of(), of(2, 3))); // finish 0 releaseAndWait(locks, 0); - // async consumer is slower to execute the commit. We could just wait, or we could add an event to the async consumer commit cycle - if (isUsingAsyncCommits()) - waitForOneLoopCycle(); + parallelConsumer.requestCommitAsap(); + + waitForOneLoopCycle(); // make sure offset 0 and 1 is committed - assertCommitLists(of(of(0, 2), of(2, 3))); + assertCommitLists(of(of(2), of(2, 3))); // finish 3 releaseAndWait(locks, 3); @@ -311,7 +316,7 @@ public void offsetCommitsAreIsolatedPerPartition(CommitMode commitMode) { waitForOneLoopCycle(); // - assertCommitLists(of(of(0, 2), of(2, 3, 4))); + assertCommitLists(of(of(2), of(2, 3, 4))); } @Test @@ -359,7 +364,6 @@ public void testVoid(CommitMode commitMode) { int expected = 1; var msgCompleteBarrier = new CountDownLatch(expected); parallelConsumer.poll((record) -> { - waitForInitialBootstrapCommit(); myRecordProcessingAction.apply(record); msgCompleteBarrier.countDown(); }); @@ -370,7 +374,7 @@ public void testVoid(CommitMode commitMode) { parallelConsumer.close(); - assertCommits(of(0, 1)); + assertCommits(of(1)); verify(myRecordProcessingAction, times(expected)).apply(any()); @@ -382,16 +386,6 @@ public void testVoid(CommitMode commitMode) { } } - /** - * Allow time for offset zero to be committed - */ - private void waitForInitialBootstrapCommit() { - await("for initial commit") - .pollDelay(ofMillis(0)) - .pollInterval(ofMillis(DEFAULT_COMMIT_INTERVAL_MAX_MS / 2)) - .untilAsserted(() -> assertCommits(of(0))); - } - @Test @Disabled public void userSucceedsButProduceToBrokerFails() { @@ -666,7 +660,6 @@ public void closeAfterSingleMessageShouldBeEventBasedFast(CommitMode commitMode) var msgCompleteBarrier = new CountDownLatch(1); parallelConsumer.poll((ignore) -> { - waitForInitialBootstrapCommit(); log.info("Message processed: {} - noop", ignore.offset()); msgCompleteBarrier.countDown(); }); @@ -676,11 +669,11 @@ public void closeAfterSingleMessageShouldBeEventBasedFast(CommitMode commitMode) // allow for offset to be committed waitForOneLoopCycle(); - if (isUsingAsyncCommits()) { - waitForOneLoopCycle(); - } + parallelConsumer.requestCommitAsap(); + + waitForOneLoopCycle(); - assertCommits(of(0, 1)); + assertCommits(of(1)); // close Duration time = time(() -> { @@ -728,6 +721,7 @@ void consumeFlowDoesntRequireProducer(CommitMode commitMode) { parallelConsumer = initAsyncConsumer(optionsWithClients); attachLoopCounter(parallelConsumer); + subscribeParallelConsumerAndMockConsumerTo(INPUT_TOPIC); setupData(); parallelConsumer.poll((ignore) -> { @@ -739,7 +733,7 @@ void consumeFlowDoesntRequireProducer(CommitMode commitMode) { parallelConsumer.closeDrainFirst(); // - assertCommits(of(0, 0, 1)); + assertCommits(of(1)); } } @@ -767,6 +761,10 @@ void cantUseProduceFlowWithWrongOptions() throws InterruptedException { .commitMode(CONSUMER_ASYNCHRONOUS) .build(); + setupParallelConsumerInstance(optionsWithClients); + + subscribeParallelConsumerAndMockConsumerTo(INPUT_TOPIC); + setupData(); var parallel = initAsyncConsumer(optionsWithClients); @@ -787,11 +785,20 @@ public void produceMessageFlow(CommitMode commitMode) { // let it process waitForSomeLoopCycles(2); + parallelConsumer.requestCommitAsap(); +// requestCommitAndPause(); + parallelConsumer.closeDrainFirst(); // - assertCommits(of(0, 1)); + assertCommits(of(1)); assertThat(producerSpy.history()).hasSize(1); } + + private void requestCommitAndPause() { + parallelConsumer.requestCommitAsap(); + waitForSomeLoopCycles(2); + } } + diff --git a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorTestBase.java b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorTestBase.java index 42b48311c..e01c3d3db 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorTestBase.java +++ b/parallel-consumer-core/src/test/java/io/confluent/parallelconsumer/ParallelEoSStreamProcessorTestBase.java @@ -22,6 +22,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; +import static io.confluent.csid.utils.KafkaTestUtils.trimAllGeneisOffset; import static io.confluent.csid.utils.Range.range; import static io.confluent.csid.utils.StringUtils.msg; import static io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode.*; @@ -48,7 +49,7 @@ public class ParallelEoSStreamProcessorTestBase { * * @see LongPollingMockConsumer#poll(Duration) */ - public static final int DEFAULT_BROKER_POLL_FREQUENCY_MS = 1000; + public static final int DEFAULT_BROKER_POLL_FREQUENCY_MS = 100; /** * The commit interval for the main {@link ParallelEoSStreamProcessor} control thread. Actually the timeout that we @@ -58,7 +59,7 @@ public class ParallelEoSStreamProcessorTestBase { * @see ParallelEoSStreamProcessor#workMailBox * @see ParallelEoSStreamProcessor#processWorkCompleteMailBox */ - public static final int DEFAULT_COMMIT_INTERVAL_MAX_MS = 1000; + public static final int DEFAULT_COMMIT_INTERVAL_MAX_MS = 100; protected LongPollingMockConsumer consumerSpy; protected MockProducer producerSpy; @@ -113,8 +114,6 @@ protected MockConsumer setupClients() { ktu = new KafkaTestUtils(consumerSpy); - KafkaTestUtils.assignConsumerToTopic(this.consumerSpy); - return consumerSpy; } @@ -153,6 +152,8 @@ protected void setupParallelConsumerInstance(ParallelConsumerOptions parallelCon parallelConsumer = initAsyncConsumer(optionsWithClients); + subscribeParallelConsumerAndMockConsumerTo(INPUT_TOPIC); + parallelConsumer.setLongPollTimeout(ofMillis(DEFAULT_BROKER_POLL_FREQUENCY_MS)); parallelConsumer.setTimeBetweenCommits(ofMillis(DEFAULT_COMMIT_INTERVAL_MAX_MS)); @@ -268,13 +269,19 @@ public void assertCommits(List offsets, Optional description) { assertThat(extractAllPartitionsOffsetsSequentially()).isEmpty(); } else { List collect = extractAllPartitionsOffsetsSequentially(); + collect = trimAllGeneisOffset(collect); // duplicates are ok // is there a nicer optional way? // {@link Optional#ifPresentOrElse} only @since 9 if (description.isPresent()) { assertThat(collect).as(description.get()).hasSameElementsAs(offsets); } else { - assertThat(collect).hasSameElementsAs(offsets); + try { + assertThat(collect).hasSameElementsAs(offsets); + } catch (AssertionError e) { + log.error("", e); + throw e; + } } KafkaTestUtils.assertCommits(producerSpy, UniLists.of(), Optional.of("Empty")); @@ -305,7 +312,8 @@ public void assertCommitLists(List> offsets) { if (isUsingTransactionalProducer()) { KafkaTestUtils.assertCommitLists(producerSpy, offsets, Optional.empty()); } else { - KafkaTestUtils.assertCommitLists(consumerSpy.getCommitHistoryWithGropuId(), offsets, Optional.empty()); + List>> commitHistoryWithGropuId = consumerSpy.getCommitHistoryWithGropuId(); + KafkaTestUtils.assertCommitLists(commitHistoryWithGropuId, offsets, Optional.empty()); } } diff --git a/parallel-consumer-core/src/test/resources/logback-test.xml b/parallel-consumer-core/src/test/resources/logback-test.xml index 9f4330bfe..ae20165e0 100644 --- a/parallel-consumer-core/src/test/resources/logback-test.xml +++ b/parallel-consumer-core/src/test/resources/logback-test.xml @@ -27,7 +27,7 @@ - + diff --git a/parallel-consumer-vertx/src/test/java/io/confluent/parallelconsumer/vertx/VertxTest.java b/parallel-consumer-vertx/src/test/java/io/confluent/parallelconsumer/vertx/VertxTest.java index b8e7aca6c..6f150099c 100644 --- a/parallel-consumer-vertx/src/test/java/io/confluent/parallelconsumer/vertx/VertxTest.java +++ b/parallel-consumer-vertx/src/test/java/io/confluent/parallelconsumer/vertx/VertxTest.java @@ -123,7 +123,7 @@ public void failingHttpCall() { awaitLatch(latch); // - assertCommits(of(0)); + assertCommits(of()); // check results are failures var res = getResults(tupleStream); @@ -189,7 +189,7 @@ public void testHttpMinimal() { // verify var res = getResults(futureStream); - KafkaTestUtils.assertCommits(producerSpy, of(0, 1)); + KafkaTestUtils.assertCommits(producerSpy, of(1)); // test results are successes assertThat(res).extracting(x -> x.result().statusCode()).containsOnly(200); From e1e9af0bcb0f0a9d34f19e6ff0b6ade245ffb5fc Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Mon, 23 Nov 2020 10:19:12 +0000 Subject: [PATCH 07/11] docs: Elaborate on Why not more consumers? FAQ --- README.adoc | 13 ++++++++++--- src/docs/README.adoc | 13 ++++++++++--- 2 files changed, 20 insertions(+), 6 deletions(-) diff --git a/README.adoc b/README.adoc index 23bbe917a..e2e154622 100644 --- a/README.adoc +++ b/README.adoc @@ -38,7 +38,7 @@ ifdef::env-github[] :warning-caption: :warning: endif::[] -image:https://travis-ci.com/astubbs/parallel-consumer.svg?branch=master["Build Status", link="https://travis-ci.com/astubbs/parallel-consumer"] image:https://codecov.io/gh/astubbs/parallel-consumer/branch/master/graph/badge.svg["Coverage",https://codecov.io/gh/astubbs/parallel-consumer] +image:https://maven-badges.herokuapp.com/maven-central/io.confluent.parallelconsumer/parallel-consumer-parent/badge.svg?style=flat[link=https://mvnrepository.com/artifact/io.confluent.parallelconsumer/parallel-consumer-parent,Latest Parallel Consumer on Maven Central] image:https://travis-ci.com/astubbs/parallel-consumer.svg?branch=master["Build Status", link="https://travis-ci.com/astubbs/parallel-consumer"] image:https://codecov.io/gh/astubbs/parallel-consumer/branch/master/graph/badge.svg["Coverage",https://codecov.io/gh/astubbs/parallel-consumer] Parallel Apache Kafka client wrapper with client side queueing, a simpler consumer/producer API with *key concurrency* and *extendable non-blocking IO* processing. @@ -113,10 +113,12 @@ This is effective in many situations, but falls short in a lot too. * Primarily: You cannot use more consumers than you have partitions available to read from. For example, if you have a topic with five partitions, you cannot use a group with more than five consumers to read from it. -* Running more extra consumers has resource implications - each consumer takes up resources on both the client and broker side +* Running more extra consumers has resource implications - each consumer takes up resources on both the client and broker side. Each consumer adds a lot of overhead in terms of memory, CPU, and network bandwidth. * Large consumer groups (especially many large groups) can cause a lot of strain on the consumer group coordination system, such as rebalance storms. -* Even with several partitions, you cannot achieve the performance levels obtainable by *per-key* ordered or unordered concurrent processing +* Even with several partitions, you cannot achieve the performance levels obtainable by *per-key* ordered or unordered concurrent processing. +* A single slow or failing message will also still block all messages behind the problematic message, ie. the entire partition. +The process may recover, but the latency of all the messages behind the problematic one will be negatively impacted severely. Why not run more consumers __within__ your application instance?:: * This is in some respects a slightly easier way of running more consumer instances, and in others a more complicated way. @@ -343,6 +345,11 @@ Latest version can be seen https://search.maven.org/artifact/io.confluent.parall Where `${project.version}` is the version to be used: +* group ID: `io.confluent.parallelconsumer` +* artifact ID: `parallel-consumer-core` +* version: image:https://maven-badges.herokuapp.com/maven-central/io.confluent.parallelconsumer/parallel-consumer-parent/badge.svg?style=flat[link=https://mvnrepository.com/artifact/io.confluent.parallelconsumer/parallel-consumer-parent,Latest Parallel Consumer on Maven Central] + + .Core Module Dependency [source,xml,indent=0] diff --git a/src/docs/README.adoc b/src/docs/README.adoc index 3e77d0555..916d56095 100644 --- a/src/docs/README.adoc +++ b/src/docs/README.adoc @@ -38,7 +38,7 @@ ifdef::env-github[] :warning-caption: :warning: endif::[] -image:https://travis-ci.com/astubbs/parallel-consumer.svg?branch=master["Build Status", link="https://travis-ci.com/astubbs/parallel-consumer"] image:https://codecov.io/gh/astubbs/parallel-consumer/branch/master/graph/badge.svg["Coverage",https://codecov.io/gh/astubbs/parallel-consumer] +image:https://maven-badges.herokuapp.com/maven-central/io.confluent.parallelconsumer/parallel-consumer-parent/badge.svg?style=flat[link=https://mvnrepository.com/artifact/io.confluent.parallelconsumer/parallel-consumer-parent,Latest Parallel Consumer on Maven Central] image:https://travis-ci.com/astubbs/parallel-consumer.svg?branch=master["Build Status", link="https://travis-ci.com/astubbs/parallel-consumer"] image:https://codecov.io/gh/astubbs/parallel-consumer/branch/master/graph/badge.svg["Coverage",https://codecov.io/gh/astubbs/parallel-consumer] Parallel Apache Kafka client wrapper with client side queueing, a simpler consumer/producer API with *key concurrency* and *extendable non-blocking IO* processing. @@ -111,10 +111,12 @@ This is effective in many situations, but falls short in a lot too. * Primarily: You cannot use more consumers than you have partitions available to read from. For example, if you have a topic with five partitions, you cannot use a group with more than five consumers to read from it. -* Running more extra consumers has resource implications - each consumer takes up resources on both the client and broker side +* Running more extra consumers has resource implications - each consumer takes up resources on both the client and broker side. Each consumer adds a lot of overhead in terms of memory, CPU, and network bandwidth. * Large consumer groups (especially many large groups) can cause a lot of strain on the consumer group coordination system, such as rebalance storms. -* Even with several partitions, you cannot achieve the performance levels obtainable by *per-key* ordered or unordered concurrent processing +* Even with several partitions, you cannot achieve the performance levels obtainable by *per-key* ordered or unordered concurrent processing. +* A single slow or failing message will also still block all messages behind the problematic message, ie. the entire partition. +The process may recover, but the latency of all the messages behind the problematic one will be negatively impacted severely. Why not run more consumers __within__ your application instance?:: * This is in some respects a slightly easier way of running more consumer instances, and in others a more complicated way. @@ -341,6 +343,11 @@ Latest version can be seen https://search.maven.org/artifact/io.confluent.parall Where `${project.version}` is the version to be used: +* group ID: `io.confluent.parallelconsumer` +* artifact ID: `parallel-consumer-core` +* version: image:https://maven-badges.herokuapp.com/maven-central/io.confluent.parallelconsumer/parallel-consumer-parent/badge.svg?style=flat[link=https://mvnrepository.com/artifact/io.confluent.parallelconsumer/parallel-consumer-parent,Latest Parallel Consumer on Maven Central] + + .Core Module Dependency [source,xml,indent=0] include::{project_root}/parallel-consumer-examples/parallel-consumer-example-core/pom.xml[tag=exampleDep] From cb83cee1073123b39968882b97b68e62e10445da Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Mon, 23 Nov 2020 11:36:19 +0000 Subject: [PATCH 08/11] test: Commit mode test improvements, bug fixes --- parallel-consumer-core/pom.xml | 6 ++ .../ParallelEoSStreamProcessor.java | 3 +- .../TransactionAndCommitModeTest.java | 64 +++++++++++-------- .../utils/KafkaClientUtils.java | 7 ++ .../utils/EnumCartesianProductTestSets.java | 30 +++++++++ .../src/test/resources/logback-test.xml | 4 +- pom.xml | 7 ++ 7 files changed, 93 insertions(+), 28 deletions(-) create mode 100644 parallel-consumer-core/src/test/java/io/confluent/csid/utils/EnumCartesianProductTestSets.java diff --git a/parallel-consumer-core/pom.xml b/parallel-consumer-core/pom.xml index ae2be14c0..f2398e056 100644 --- a/parallel-consumer-core/pom.xml +++ b/parallel-consumer-core/pom.xml @@ -38,6 +38,12 @@ ${junit.version} test + + org.junit-pioneer + junit-pioneer + 1.0.0 + test + org.testcontainers testcontainers 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 d74396ddd..7486cf930 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 @@ -386,7 +386,8 @@ private void waitForClose(Duration timeout) throws TimeoutException, ExecutionEx // ignore log.trace("Interrupted", e); } catch (ExecutionException | TimeoutException e) { - log.error("Execution or timeout exception", e); + log.error("Execution or timeout exception while waiting for the control thread to close cleanly - lock " + + "problem? If not, try increasing your time out to allow the system to drain, if closing in drain mode.", e); throw e; } log.trace("Still waiting for system to close..."); diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java index e70340fac..ac18eaee7 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java @@ -4,27 +4,33 @@ * Copyright (C) 2020 Confluent, Inc. */ +import io.confluent.csid.utils.EnumCartesianProductTestSets; import io.confluent.parallelconsumer.ParallelConsumerOptions; import io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode; +import io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder; import io.confluent.parallelconsumer.ParallelEoSStreamProcessor; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.RandomUtils; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; import org.awaitility.core.ConditionTimeoutException; import org.junit.jupiter.api.RepeatedTest; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.EnumSource; +import org.junitpioneer.jupiter.CartesianProductTest; import pl.tlinkowski.unij.api.UniLists; import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Properties; import java.util.concurrent.atomic.AtomicInteger; import static io.confluent.parallelconsumer.ParallelConsumerOptions.CommitMode.TRANSACTIONAL_PRODUCER; import static io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder.KEY; +import static io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder.PARTITION; import static java.time.Duration.ofSeconds; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.fail; @@ -54,40 +60,42 @@ public class TransactionAndCommitModeTest extends KafkaTest { public AtomicInteger producedCount = new AtomicInteger(0); // default - @ParameterizedTest() - @EnumSource(CommitMode.class) - public void testDefaultMaxPoll(CommitMode commitMode) { - runTest(DEFAULT_MAX_POLL_RECORDS_CONFIG, commitMode); + @CartesianProductTest(factory = "enumSets") + void testDefaultMaxPoll(CommitMode commitMode, ProcessingOrder order) { + runTest(DEFAULT_MAX_POLL_RECORDS_CONFIG, commitMode, order); + } + + static CartesianProductTest.Sets enumSets() { + return new EnumCartesianProductTestSets() + .add(CommitMode.class) + .add(ProcessingOrder.class); } @RepeatedTest(5) public void testTransactionalDefaultMaxPoll() { - runTest(DEFAULT_MAX_POLL_RECORDS_CONFIG, TRANSACTIONAL_PRODUCER); + runTest(DEFAULT_MAX_POLL_RECORDS_CONFIG, TRANSACTIONAL_PRODUCER, KEY); } // low - @ParameterizedTest() - @EnumSource(CommitMode.class) - public void testLowMaxPoll(CommitMode commitMode) { - runTest(LOW_MAX_POLL_RECORDS_CONFIG, commitMode); + @CartesianProductTest(factory = "enumSets") + public void testLowMaxPoll(CommitMode commitMode, ProcessingOrder order) { + runTest(LOW_MAX_POLL_RECORDS_CONFIG, commitMode, order); } // high counts - @ParameterizedTest() - @EnumSource(CommitMode.class) - public void testHighMaxPollEnum(CommitMode commitMode) { - runTest(HIGH_MAX_POLL_RECORDS_CONFIG, commitMode); + @CartesianProductTest(factory = "enumSets") + public void testHighMaxPollEnum(CommitMode commitMode, ProcessingOrder order) { + runTest(HIGH_MAX_POLL_RECORDS_CONFIG, commitMode, order); } private void runTest(int maxPoll, CommitMode commitMode) { - boolean tx = commitMode.equals(TRANSACTIONAL_PRODUCER); - runTest(tx, maxPoll, commitMode); + runTest(maxPoll, commitMode, PARTITION); } @SneakyThrows - private void runTest(boolean tx, int maxPoll, CommitMode commitMode) { - String inputName = setupTopic(this.getClass().getSimpleName() + "-input"); - String outputName = setupTopic(this.getClass().getSimpleName() + "-input"); + private void runTest(int maxPoll, CommitMode commitMode, ProcessingOrder order) { + String inputName = setupTopic(this.getClass().getSimpleName() + "-input-" + RandomUtils.nextInt()); + String outputName = setupTopic(this.getClass().getSimpleName() + "-output-" + RandomUtils.nextInt()); // pre-produce messages to input-topic List expectedKeys = new ArrayList<>(); @@ -105,9 +113,13 @@ private void runTest(boolean tx, int maxPoll, CommitMode commitMode) { log.info("Starting application..."); KafkaProducer newProducer = kcu.createNewProducer(commitMode.equals(TRANSACTIONAL_PRODUCER)); + Properties consumerProps = new Properties(); + consumerProps.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPoll); + KafkaConsumer newConsumer = kcu.createNewConsumer(true, consumerProps); + var pc = new ParallelEoSStreamProcessor(ParallelConsumerOptions.builder() - .ordering(KEY) - .consumer(kcu.createNewConsumer()) + .ordering(order) + .consumer(newConsumer) .producer(newProducer) .commitMode(commitMode) .build()); @@ -125,9 +137,8 @@ private void runTest(boolean tx, int maxPoll, CommitMode commitMode) { // wait for all pre-produced messages to be processed and produced String failureMessage = "All keys sent to input-topic should be processed and produced"; try { - waitAtMost(ofSeconds(30)).alias(failureMessage).untilAsserted(() -> { - log.debug("Processed-count: " + processedCount.get()); - log.debug("Produced-count: " + producedCount.get()); + waitAtMost(ofSeconds(10)).alias(failureMessage).untilAsserted(() -> { + log.info("Processed-count: {}, Produced-count: {}", processedCount.get(), producedCount.get()); List processedAndProducedKeysCopy = new ArrayList<>(processedAndProducedKeys); // avoid concurrent-modification in assert assertThat(processedAndProducedKeysCopy).contains(expectedKeys.toArray(new String[0])); }); @@ -145,6 +156,9 @@ private void runTest(boolean tx, int maxPoll, CommitMode commitMode) { .as("messages processed and produced by parallel-consumer should be equal") .isEqualTo(producedCount.get()); + // sanity + assertThat(expectedMessageCount).isEqualTo(processedCount.get()); + pc.close(); } diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java index 17ba965c3..333bbcfd3 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/utils/KafkaClientUtils.java @@ -14,6 +14,7 @@ import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; +import org.jetbrains.annotations.NotNull; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.testcontainers.containers.KafkaContainer; @@ -93,9 +94,15 @@ public KafkaConsumer createNewConsumer() { } public KafkaConsumer createNewConsumer(boolean newConsumerGroup) { + return createNewConsumer(newConsumerGroup, new Properties()); + } + + @NotNull + public KafkaConsumer createNewConsumer(boolean newConsumerGroup, Properties options) { if (newConsumerGroup) { props.put(ConsumerConfig.GROUP_ID_CONFIG, "group-1-" + RandomUtils.nextInt()); // new group } + props.putAll(options); KafkaConsumer kvKafkaConsumer = new KafkaConsumer<>(props); log.debug("New consume {}", kvKafkaConsumer); return kvKafkaConsumer; diff --git a/parallel-consumer-core/src/test/java/io/confluent/csid/utils/EnumCartesianProductTestSets.java b/parallel-consumer-core/src/test/java/io/confluent/csid/utils/EnumCartesianProductTestSets.java new file mode 100644 index 000000000..816d05f98 --- /dev/null +++ b/parallel-consumer-core/src/test/java/io/confluent/csid/utils/EnumCartesianProductTestSets.java @@ -0,0 +1,30 @@ +package io.confluent.csid.utils; + +import org.junitpioneer.jupiter.CartesianProductTest; + +/** + * Automatically extract enum constants + */ +public class EnumCartesianProductTestSets extends CartesianProductTest.Sets { + + /** + * Simply pass in the enum class, otherwise use as normal. + * + * @see CartesianProductTest.Sets#add + */ + @Override + public CartesianProductTest.Sets add(final Object... entries) { + Object[] finalEntries = entries; + if (entries.length == 1) { + Object entry = entries[0]; + if (entry instanceof Class) { + Class classEntry = (Class) entry; + if (classEntry.isEnum()) { + finalEntries = classEntry.getEnumConstants(); + } + } + } + return super.add(finalEntries); + } + +} diff --git a/parallel-consumer-core/src/test/resources/logback-test.xml b/parallel-consumer-core/src/test/resources/logback-test.xml index ae20165e0..3b376a4a6 100644 --- a/parallel-consumer-core/src/test/resources/logback-test.xml +++ b/parallel-consumer-core/src/test/resources/logback-test.xml @@ -49,8 +49,8 @@ - - + + diff --git a/pom.xml b/pom.xml index 984ee283d..579b2854f 100644 --- a/pom.xml +++ b/pom.xml @@ -268,6 +268,13 @@ ${junit.platform.version} test + + + org.junit.platform + junit-platform-commons + ${junit.platform.version} + test + org.assertj assertj-core From a7267fd1114f24ba9f681ec3c94f9bba8b008c87 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Mon, 23 Nov 2020 12:47:49 +0000 Subject: [PATCH 09/11] bug: By default, close without draining and when draining still poll mailbox and commit normally --- .../ParallelEoSStreamProcessor.java | 29 +++++++++++++++---- .../TransactionAndCommitModeTest.java | 6 ++-- 2 files changed, 27 insertions(+), 8 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 7486cf930..c54c6bd7a 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 @@ -109,7 +109,19 @@ public class ParallelEoSStreamProcessor implements ParallelStreamProcessor * @see #state */ enum State { - unused, running, draining, closing, closed; + unused, + running, + /** + * 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 first before closing. + * + * @see #closeDrainFirst() + * @see #close() + */ + draining, + closing, + closed; } /** @@ -335,11 +347,16 @@ public void pollAndProduce(Function, ProducerRecord> pollAndProduceMany((record) -> UniLists.of(userFunction.apply(record)), callback); } + /** + * Close the system, without draining. + * + * @see State#draining + */ @Override public void close() { // use a longer timeout, to cover for evey other step using the default Duration timeout = DrainingCloseable.DEFAULT_TIMEOUT.multipliedBy(2); - closeDrainFirst(timeout); + closeDontDrainFirst(timeout); } @Override @@ -386,8 +403,9 @@ private void waitForClose(Duration timeout) throws TimeoutException, ExecutionEx // ignore log.trace("Interrupted", e); } catch (ExecutionException | TimeoutException e) { - log.error("Execution or timeout exception while waiting for the control thread to close cleanly - lock " + - "problem? If not, try increasing your time out to allow the system to drain, if closing in drain mode.", e); + log.error("Execution or timeout exception while waiting for the control thread to close cleanly " + + "(state was {}). Try increasing your time-out to allow the system to drain, or close withing " + + "draining.", state, e); throw e; } log.trace("Still waiting for system to close..."); @@ -700,7 +718,8 @@ private boolean lingeringOnCommitWouldBeBeneficial() { } private Duration getTimeToNextCommit() { - if (state == running) { + // draining is a normal running mode for the controller + if (state == running || state == draining) { return getTimeBetweenCommits().minus(getTimeSinceLastCommit()); } else { log.debug("System not {} (state: {}), so don't wait to commit, only a small thread yield time", running, state); diff --git a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java index ac18eaee7..6d8cc66f1 100644 --- a/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java +++ b/parallel-consumer-core/src/test-integration/java/io/confluent/parallelconsumer/integrationTests/TransactionAndCommitModeTest.java @@ -126,6 +126,7 @@ private void runTest(int maxPoll, CommitMode commitMode, ProcessingOrder order) pc.subscribe(UniLists.of(inputName)); // <5> pc.pollAndProduce(record -> { + log.trace("Still going {}", record); processedCount.incrementAndGet(); return new ProducerRecord(outputName, record.key(), "data"); }, consumeProduceResult -> { @@ -135,7 +136,7 @@ private void runTest(int maxPoll, CommitMode commitMode, ProcessingOrder order) ); // wait for all pre-produced messages to be processed and produced - String failureMessage = "All keys sent to input-topic should be processed and produced"; + String failureMessage = "All keys sent to input-topic should be processed and produced, within time"; try { waitAtMost(ofSeconds(10)).alias(failureMessage).untilAsserted(() -> { log.info("Processed-count: {}, Produced-count: {}", processedCount.get(), producedCount.get()); @@ -158,8 +159,7 @@ private void runTest(int maxPoll, CommitMode commitMode, ProcessingOrder order) // sanity assertThat(expectedMessageCount).isEqualTo(processedCount.get()); - - pc.close(); + assertThat(processedAndProducedKeys).hasSameSizeAs(expectedKeys); } } From ecfa6312becf97378dea0059707451a5a90e4787 Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Mon, 23 Nov 2020 15:13:59 +0000 Subject: [PATCH 10/11] docs: Temporarily remove travis badges (travis deactivated for repo) --- README.adoc | 9 ++++++--- src/docs/README.adoc | 9 ++++++--- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/README.adoc b/README.adoc index e2e154622..ce4dc9c10 100644 --- a/README.adoc +++ b/README.adoc @@ -38,7 +38,10 @@ ifdef::env-github[] :warning-caption: :warning: endif::[] -image:https://maven-badges.herokuapp.com/maven-central/io.confluent.parallelconsumer/parallel-consumer-parent/badge.svg?style=flat[link=https://mvnrepository.com/artifact/io.confluent.parallelconsumer/parallel-consumer-parent,Latest Parallel Consumer on Maven Central] image:https://travis-ci.com/astubbs/parallel-consumer.svg?branch=master["Build Status", link="https://travis-ci.com/astubbs/parallel-consumer"] image:https://codecov.io/gh/astubbs/parallel-consumer/branch/master/graph/badge.svg["Coverage",https://codecov.io/gh/astubbs/parallel-consumer] +image:https://maven-badges.herokuapp.com/maven-central/io.confluent.parallelconsumer/parallel-consumer-parent/badge.svg?style=flat[link=https://mvnrepository.com/artifact/io.confluent.parallelconsumer/parallel-consumer-parent,Latest Parallel Consumer on Maven Central] + +// travis badges temporarily disabled as travis isn't running CI currently +//image:https://travis-ci.com/astubbs/parallel-consumer.svg?branch=master["Build Status", link="https://travis-ci.com/astubbs/parallel-consumer"] image:https://codecov.io/gh/astubbs/parallel-consumer/branch/master/graph/badge.svg["Coverage",https://codecov.io/gh/astubbs/parallel-consumer] Parallel Apache Kafka client wrapper with client side queueing, a simpler consumer/producer API with *key concurrency* and *extendable non-blocking IO* processing. @@ -201,8 +204,8 @@ without operational burden or harming the clusters performance * Throttle control and broker liveliness management * Clean draining shutdown cycle -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"] +//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"] And more <>! diff --git a/src/docs/README.adoc b/src/docs/README.adoc index 916d56095..30e4d533a 100644 --- a/src/docs/README.adoc +++ b/src/docs/README.adoc @@ -38,7 +38,10 @@ ifdef::env-github[] :warning-caption: :warning: endif::[] -image:https://maven-badges.herokuapp.com/maven-central/io.confluent.parallelconsumer/parallel-consumer-parent/badge.svg?style=flat[link=https://mvnrepository.com/artifact/io.confluent.parallelconsumer/parallel-consumer-parent,Latest Parallel Consumer on Maven Central] image:https://travis-ci.com/astubbs/parallel-consumer.svg?branch=master["Build Status", link="https://travis-ci.com/astubbs/parallel-consumer"] image:https://codecov.io/gh/astubbs/parallel-consumer/branch/master/graph/badge.svg["Coverage",https://codecov.io/gh/astubbs/parallel-consumer] +image:https://maven-badges.herokuapp.com/maven-central/io.confluent.parallelconsumer/parallel-consumer-parent/badge.svg?style=flat[link=https://mvnrepository.com/artifact/io.confluent.parallelconsumer/parallel-consumer-parent,Latest Parallel Consumer on Maven Central] + +// travis badges temporarily disabled as travis isn't running CI currently +//image:https://travis-ci.com/astubbs/parallel-consumer.svg?branch=master["Build Status", link="https://travis-ci.com/astubbs/parallel-consumer"] image:https://codecov.io/gh/astubbs/parallel-consumer/branch/master/graph/badge.svg["Coverage",https://codecov.io/gh/astubbs/parallel-consumer] Parallel Apache Kafka client wrapper with client side queueing, a simpler consumer/producer API with *key concurrency* and *extendable non-blocking IO* processing. @@ -199,8 +202,8 @@ without operational burden or harming the clusters performance * Throttle control and broker liveliness management * Clean draining shutdown cycle -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"] +//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"] And more <>! From 88261e9bb3b360b3caac05fbe068ce3bb11347bf Mon Sep 17 00:00:00 2001 From: Antony Stubbs Date: Mon, 23 Nov 2020 15:44:09 +0000 Subject: [PATCH 11/11] docs: Fix code examples --- README.adoc | 30 +++++++++------ .../csid/utils/LongPollingMockConsumer.java | 2 +- .../examples/core/CoreApp.java | 38 +++++++++++-------- .../examples/core/CoreAppTest.java | 11 +++--- src/docs/README.adoc | 10 ++--- 5 files changed, 51 insertions(+), 40 deletions(-) diff --git a/README.adoc b/README.adoc index ce4dc9c10..a4bc0a20c 100644 --- a/README.adoc +++ b/README.adoc @@ -375,25 +375,31 @@ Where `${project.version}` is the version to be used: .Setup the client [source,java,indent=0] ---- - Consumer kafkaConsumer = getKafkaConsumer(); // <4> + Consumer kafkaConsumer = getKafkaConsumer(); // <1> Producer kafkaProducer = getKafkaProducer(); - var options = getOptions().toBuilder().consumer(kafkaConsumer).producer(kafkaProducer).build(); + var options = ParallelConsumerOptions.builder() + .ordering(KEY) // <2> + .maxMessagesToQueue(1000) // <3> + .maxNumberMessagesBeyondBaseCommitOffset(1000) // <4> + .consumer(kafkaConsumer) + .producer(kafkaProducer) + .build(); - ParallelStreamProcessor eosStreamProcessor = ParallelStreamProcessor.createEosStreamProcessor(options); - if (!(kafkaConsumer instanceof MockConsumer)) { - eosStreamProcessor.subscribe(UniLists.of(inputTopic)); // <5> - } + ParallelStreamProcessor eosStreamProcessor = + ParallelStreamProcessor.createEosStreamProcessor(options); + + eosStreamProcessor.subscribe(of(inputTopic)); // <5> return eosStreamProcessor; ---- -<1> Choose your ordering type, `KEY` in this case. +<1> Setup your clients as per normal. A Producer is only required if using the `produce` flows. +<2> Choose your ordering type, `KEY` in this case. This ensures maximum concurrency, while ensuring messages are processed and committed in `KEY` order, making sure no offset is committed unless all offsets before it in it's partition, are completed also. -<2> The maximum number of concurrent processing operations to be performing at any given time -<3> Regardless of the level of concurrency, don't have more than this many messages uncommitted at any given time. +<3> The maximum number of concurrent processing operations to be performing at any given time +<4> Regardless of the level of concurrency, don't have more than this many messages uncommitted at any given time. Also, because the library coordinates offsets, `enable.auto.commit` must be disabled in your consumer. -<4> Setup your consumer client as per normal -<5> Setup your topic subscriptions - (when using the `MockConsumer` you must use the `MockConsumer#assign` method) +<5> Subscribe to your topics NOTE: Because the library coordinates offsets, `enable.auto.commit` must be disabled. @@ -443,7 +449,7 @@ You can also optionally provide a callback function to be run after the message( .Usage - print message content out to the console in parallel [source,java,indent=0] - this.parallelConsumer.pollAndProduce(record -> { + parallelConsumer.pollAndProduce(record -> { var result = processBrokerRecord(record); return new ProducerRecord<>(outputTopic, record.key(), result.payload); }, consumeProduceResult -> { diff --git a/parallel-consumer-core/src/test/java/io/confluent/csid/utils/LongPollingMockConsumer.java b/parallel-consumer-core/src/test/java/io/confluent/csid/utils/LongPollingMockConsumer.java index edb7a05e2..798ff44c6 100644 --- a/parallel-consumer-core/src/test/java/io/confluent/csid/utils/LongPollingMockConsumer.java +++ b/parallel-consumer-core/src/test/java/io/confluent/csid/utils/LongPollingMockConsumer.java @@ -136,6 +136,6 @@ public void subscribeWithRebalanceAndAssignment(final List topics, int p beginningOffsets.put(tp, 0L); } super.updateBeginningOffsets(beginningOffsets); - } + } \ No newline at end of file diff --git a/parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java b/parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java index 36973da8e..a109eabef 100644 --- a/parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java +++ b/parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java @@ -25,6 +25,7 @@ import java.util.concurrent.atomic.AtomicInteger; import static io.confluent.parallelconsumer.ParallelConsumerOptions.ProcessingOrder.KEY; +import static pl.tlinkowski.unij.api.UniLists.of; /** * Basic core examples @@ -50,6 +51,8 @@ Producer getKafkaProducer() { void run() { this.parallelConsumer = setupParallelConsumer(); + postSetup(); + // tag::example[] parallelConsumer.poll(record -> log.info("Concurrently processing a record: {}", record) @@ -57,32 +60,33 @@ void run() { // end::example[] } + protected void postSetup() { + // ignore + } + @SuppressWarnings({"FeatureEnvy", "MagicNumber"}) ParallelStreamProcessor setupParallelConsumer() { // tag::exampleSetup[] - Consumer kafkaConsumer = getKafkaConsumer(); // <4> + Consumer kafkaConsumer = getKafkaConsumer(); // <1> Producer kafkaProducer = getKafkaProducer(); - var options = getOptions().toBuilder().consumer(kafkaConsumer).producer(kafkaProducer).build(); + var options = ParallelConsumerOptions.builder() + .ordering(KEY) // <2> + .maxMessagesToQueue(1000) // <3> + .maxNumberMessagesBeyondBaseCommitOffset(1000) // <4> + .consumer(kafkaConsumer) + .producer(kafkaProducer) + .build(); + + ParallelStreamProcessor eosStreamProcessor = + ParallelStreamProcessor.createEosStreamProcessor(options); - ParallelStreamProcessor eosStreamProcessor = ParallelStreamProcessor.createEosStreamProcessor(options); - if (!(kafkaConsumer instanceof MockConsumer)) { - eosStreamProcessor.subscribe(UniLists.of(inputTopic)); // <5> - } + eosStreamProcessor.subscribe(of(inputTopic)); // <5> return eosStreamProcessor; // end::exampleSetup[] } - ParallelConsumerOptions getOptions() { - var options = ParallelConsumerOptions.builder() - .ordering(KEY) // <1> - .maxMessagesToQueue(1000) // <2> - .maxNumberMessagesBeyondBaseCommitOffset(1000) // <3> - .build(); - return options; - } - void close() { this.parallelConsumer.close(); } @@ -90,8 +94,10 @@ void close() { void runPollAndProduce() { this.parallelConsumer = setupParallelConsumer(); + postSetup(); + // tag::exampleProduce[] - this.parallelConsumer.pollAndProduce(record -> { + parallelConsumer.pollAndProduce(record -> { var result = processBrokerRecord(record); return new ProducerRecord<>(outputTopic, record.key(), result.payload); }, consumeProduceResult -> { diff --git a/parallel-consumer-examples/parallel-consumer-example-core/src/test/java/io/confluent/parallelconsumer/examples/core/CoreAppTest.java b/parallel-consumer-examples/parallel-consumer-example-core/src/test/java/io/confluent/parallelconsumer/examples/core/CoreAppTest.java index 49f13cbc9..f44ba7159 100644 --- a/parallel-consumer-examples/parallel-consumer-example-core/src/test/java/io/confluent/parallelconsumer/examples/core/CoreAppTest.java +++ b/parallel-consumer-examples/parallel-consumer-example-core/src/test/java/io/confluent/parallelconsumer/examples/core/CoreAppTest.java @@ -24,6 +24,7 @@ import static io.confluent.parallelconsumer.ParallelEoSStreamProcessorTestBase.DEFAULT_GROUP_METADATA; import static org.mockito.Mockito.when; +import static pl.tlinkowski.unij.api.UniLists.of; @Slf4j public class CoreAppTest { @@ -48,13 +49,11 @@ public void test() { coreApp.close(); } - @SneakyThrows @Test public void testPollAndProduce() { log.info("Test start"); CoreAppUnderTest coreApp = new CoreAppUnderTest(); - TopicPartition tp = new TopicPartition(coreApp.inputTopic, 0); coreApp.runPollAndProduce(); @@ -76,11 +75,7 @@ class CoreAppUnderTest extends CoreApp { @Override Consumer getKafkaConsumer() { - HashMap beginningOffsets = new HashMap<>(); - beginningOffsets.put(tp, 0L); - mockConsumer.updateBeginningOffsets(beginningOffsets); when(mockConsumer.groupMetadata()).thenReturn(DEFAULT_GROUP_METADATA); // todo fix AK mock consumer - mockConsumer.assign(UniLists.of(tp)); return mockConsumer; } @@ -89,5 +84,9 @@ Producer getKafkaProducer() { return new MockProducer<>(true, null, null); } + @Override + protected void postSetup() { + mockConsumer.subscribeWithRebalanceAndAssignment(of(inputTopic), 1); + } } } diff --git a/src/docs/README.adoc b/src/docs/README.adoc index 30e4d533a..82f8034d4 100644 --- a/src/docs/README.adoc +++ b/src/docs/README.adoc @@ -367,13 +367,13 @@ include::{project_root}/parallel-consumer-examples/parallel-consumer-example-ver ---- include::{project_root}/parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java[tag=exampleSetup] ---- -<1> Choose your ordering type, `KEY` in this case. +<1> Setup your clients as per normal. A Producer is only required if using the `produce` flows. +<2> Choose your ordering type, `KEY` in this case. This ensures maximum concurrency, while ensuring messages are processed and committed in `KEY` order, making sure no offset is committed unless all offsets before it in it's partition, are completed also. -<2> The maximum number of concurrent processing operations to be performing at any given time -<3> Regardless of the level of concurrency, don't have more than this many messages uncommitted at any given time. +<3> The maximum number of concurrent processing operations to be performing at any given time +<4> Regardless of the level of concurrency, don't have more than this many messages uncommitted at any given time. Also, because the library coordinates offsets, `enable.auto.commit` must be disabled in your consumer. -<4> Setup your consumer client as per normal -<5> Setup your topic subscriptions - (when using the `MockConsumer` you must use the `MockConsumer#assign` method) +<5> Subscribe to your topics NOTE: Because the library coordinates offsets, `enable.auto.commit` must be disabled.