- TIP
-
Editing template file
Parallel Apache Kafka client wrapper with client side queueing, a simpler consumer/producer API with key concurrency and extendable non-blocking IO processing.
Confluent’s product page for the project is here.
💡
|
If you like this project, please ⭐ Star it in GitHub to show your appreciation, help us gauge popularity of the project and allocate resources. |
ℹ️
|
This is not a part of the Confluent commercial support offering, except through consulting engagements. See the Support and Issues section for more information. |
❗
|
This project has been stable and reached its initial target feature set in Q1 2021. It is actively maintained by the CSID team at Confluent. |
This library lets you process messages in parallel via a single Kafka Consumer meaning you can increase consumer parallelism without increasing the number of partitions in the topic you intend to process. For many use cases this improves both throughput and latency by reducing load on your brokers. It also opens up new use cases like extreme parallelism, external data enrichment, and queuing.
link:../..//parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java[role=include]
An overview article to the library can also be found on Confluent’s blog: Introducing the Confluent Parallel Consumer.
The unit of parallelism in Kafka’s consumers is the partition but sometimes you want to break away from this approach and manage parallelism yourself using threads rather than new instances of a Consumer. Notable use cases include:
-
Where partition counts are difficult to change and you need more parallelism than the current configuration allows.
-
You wish to avoid over provisioning partitions in topics due to unknown future requirements.
-
You wish to reduce the broker-side resource utilization associated with highly-parallel consumer groups.
-
You need queue-like semantics that use message level acknowledgment, for example to process a work queue with short- and long-running tasks.
When reading the below, keep in mind that the unit of concurrency and thus performance, is restricted by the number of partitions (degree of sharding / concurrency). Currently, you can’t adjust the number of partitions in your Kafka topics without jumping through a lot of hoops, or breaking your key ordering.
The core Kafka consumer client gives you a batch of messages to process one at a time. Processing these in parallel on thread pools is difficult, particularly when considering offset management and strong ordering guarantees. You also need to manage your consume loop, and commit transactions properly if using Exactly Once semantics.
This wrapper library for the Apache Kafka Java client handles all this for you, you just supply your processing function.
Another common situation where concurrent processing of messages is advantageous, is what is referred to as "competing consumers". A pattern that is often addressed in traditional messaging systems using a shared queue. Kafka doesn’t provide native queue support and this can result in a slow processing message blocking the messages behind it in the same partition. If log ordering isn’t a concern this can be an unwelcome bottleneck for users. The Parallel Consumer provides a solution to this problem.
In addition, the Vert.x extension to this library supplies non-blocking interfaces, allowing higher still levels of concurrency with a further simplified interface. Also included now is a module for Project Reactor.io.
-
Why not just run more consumers?
The typical way to address performance issues in a Kafka system, is to increase the number of consumers reading from a topic. 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. 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.
-
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. However, you are still restricted by all the per consumer restrictions as described above.
-
-
Why not use the Vert.x library yourself in your processing loop?
-
Vert.x is used in this library to provide a non-blocking IO system in the message processing step. Using Vert.x without using this library with ordered processing requires dealing with the quite complicated, and not straight forward, aspect of handling offset commits with the Vert.x asynchronous processing system.
Unordered processing with Vert.x is somewhat easier, however offset management is still quite complicated, and the Parallel Consumer also provides optimizations for message-level acknowledgment in this case. This library handles offset commits for both ordered and unordered processing cases.
-
Below are some real world use cases which illustrate concrete situations where the described advantages massively improve performance.
-
Slow consumer systems in transactional systems (online vs offline or reporting systems)
-
Notification system:
-
Notification processing system which sends push notifications for a user to acknowledge two-factor authentication requests on their mobile for authorising a website login, requires optimal end-to-end latency for a good user experience.
-
A specific message in this queue uncharacteristically takes a long time to process because the third party system is sometimes unpredictably slow to respond and so holds up the processing for ALL other notifications for other users that are in the same partition behind this message.
-
Using key order concurrent processing will allow notifications to proceed while this message either slowly succeeds or times out and retires.
-
-
Slow GPS tracking system (slow HTTP service interfaces that can scale horizontally)
-
GPS tracking messages from 100,000 different field devices pour through at a high rate into an input topic.
-
For each message, the GPS location coordinates is checked to be within allowed ranges using a legacy HTTP services, dictated by business rules behind the service.
-
The service takes 50ms to process each message, however can be scaled out horizontally without restriction.
-
The input topic only has 10 partitions and for various reasons (see above) cannot be changed.
-
With the vanilla consumer, messages on each partition must be consumed one after the other in serial order.
-
The maximum rate of message processing is then:
1 second / 50 ms * 10 partitions = 200 messages per second.
-
By using this library, the 10 partitions can all be processed in key order.
1 second / 50ms × 100,000 keys = 2,000,000 messages per second
While the HTTP system probably cannot handle 2,000,000 messages per second, your system is no longer the bottleneck.
-
-
Slow CPU bound model processing for fraud prediction
-
Consider a system where message data is passed through a fraud prediction model which takes CPU cycles, instead of an external system being slow.
-
We can scale easily the number of CPUs on our virtual machine where the processing is being run, but we choose not to scale the partitions or consumers (see above).
-
By deploying onto machines with far more CPUs available, we can run our prediction model massively parallel, increasing our throughput and reducing our end-to-end response times.
-
-
-
Spikey load with latency sensitive non-functional requirements
-
An upstream system regularly floods our input topic daily at close of business with settlement totals data from retail outlets.
-
Situations like this are common where systems are designed to comfortably handle average day time load, but are not provisioned to handle sudden increases in traffic as they don’t happen often enough to justify the increased spending on processing capacity that would otherwise remain idle.
-
Without adjusting the available partitions or running consumers, we can reduce our maximum end-to-end latency and increase throughout to get our global days outlet reports to division managers so action can be taken, before close of business.
-
-
Natural consumer behaviour
-
Consider scenarios where bursts of data flooding input topics are generated by sudden user behaviour such as sales or television events ("Oprah" moments).
-
For example, an evening, prime-time game show on TV where users send in quiz answers on their devices. The end-to-end latency of the responses to these answers needs to be as low as technically possible, even if the processing step is quick.
-
Instead of a vanilla client where each user response waits in a virtual queue with others to be processed, this library allows every single response to be processed in parallel.
-
-
-
Legacy partition structure
-
Any existing setups where we need higher performance either in throughput or latency where there are not enough partitions for needed concurrency level, the tool can be applied.
-
-
Partition overloaded brokers
-
Clusters with under-provisioned hardware and with too many partitions already - where we cannot expand partitions even if we were able to.
-
Similar to the above, but from the operations perspective, our system is already over partitioned, perhaps in order to support existing parallel workloads which aren’t using the tool (and so need large numbers of partitions).
-
We encourage our development teams to migrate to the tool, and then being a process of actually lowering the number of partitions in our partitions in order to reduce operational complexity, improve reliability and perhaps save on infrastructure costs.
-
-
Server side resources are controlled by a different team we can’t influence
-
The cluster our team is working with is not in our control, we cannot change the partition setup, or perhaps even the consumer layout.
-
We can use the tool ourselves to improve our system performance without touching the cluster / topic setup.
-
-
Kafka Streams app that had a slow stage
-
We use Kafka Streams for our message processing, but one of it’s steps have characteristics of the above and we need better performance. We can break out as described below into the tool for processing that step, then return to the Kafka Streams context.
-
-
Provisioning extra machines (either virtual machines or real machines) to run multiple clients has a cost, using this library instead avoids the need for extra instances to be deployed in any respect.
-
Have massively parallel consumption processing without running hundreds or thousands of:
-
Kafka consumer clients,
-
topic partitions,
without operational burden or harming the cluster’s performance
-
-
Client side queueing system on top of Apache Kafka consumer
-
Efficient individual message acknowledgement system (without local or third party external system state storage) to massively reduce (and usually completely eliminate) message replay upon failure - see Offset Map section for more details
-
-
Solution for the "head of line" blocking problem where continued failure of a single message, prevents progress for messages behind it in the queue
-
Per
key
concurrent processing, per partition and unordered message processing -
Offsets committed correctly, in order, of only processed messages, regardless of concurrency level or retries
-
Batch support in all versions of the API to process batches of messages in parallel instead of single messages.
-
Particularly useful for when your processing function can work with more than a single record at a time - e.g. sending records to an API which has a batch version like Elasticsearch
-
-
Vert.x and Reactor.io non-blocking library integration
-
Non-blocking I/O work management
-
Vert.x’s WebClient and general Vert.x Future support
-
Reactor.io Publisher (Mono/Flux) and Java’s CompletableFuture (through
Mono#fromFuture
)
-
-
Exactly Once bulk transaction system
-
When using the transactional mode, record processing that happens in parallel and produce records back to kafka get all grouped into a large batch transaction, and the offsets and records are submitted through the transactional producer, giving you Exactly once Semantics for parallel processing.
-
For further information, see the Apache Kafka EoS Transaction Model in BULK section.
-
-
Fair partition traversal
-
Zero~ dependencies (
Slf4j
andLombok
) for the core module -
Java 8 compatibility
-
Throttle control and broker liveliness management
-
Clean draining shutdown cycle
-
Manual global pause / resume of all partitions, without unsubscribing from topics (useful for implementing a simplistic circuit breaker)
-
Circuit breaker patterns for individual paritions or keys can be done through throwing failure exceptions in the processing function (see PR #291 Explicit terminal and retriable exceptions for further refinement)
-
Note: Pausing of a partition is also automatic, whenever back pressure has built up on a given partition
-
And more to come!
In the best case, you don’t care about ordering at all.In which case, the degree of concurrency achievable is simply set by max thread and concurrency settings, or with the Vert.x extension, the Vert.x Vertical being used - e.g. non-blocking HTTP calls.
For example, instead of having to run 1,000 consumers to process 1,000 messages at the same time, we can process all 1,000 concurrently on a single consumer instance.
More typically though you probably still want the per key ordering grantees that Kafka provides. For this there is the per key ordering setting. This will limit the library from processing any message at the same time or out of order, if they have the same key.
Massively reduce message processing latency regardless of partition count for spikey workloads where there is good key distribution. Eg 100,000 “users” all trigger an action at once. As long as the processing layer can handle the load horizontally (e.g auto scaling web service), per message latency will be massively decreased, potentially down to the time for processing a single message, if the integration point can handle the concurrency.
For example, if you have a key set of 10,000 unique keys, and you need to call an http endpoint to process each one, you can use the per key order setting, and in the best case the system will process 10,000 at the same time using the non-blocking Vert.x HTTP client library. The user just has to provide a function to extract from the message the HTTP call parameters and construct the HTTP request object.
These performance comparison results below, even though are based on real performance measurement results, are for illustrative purposes. To see how the performance of the tool is related to instance counts, partition counts, key distribution and how it would relate to the vanilla client. Actual results will vary wildly depending upon the setup being deployed into.
For example, if you have hundreds of thousands of keys in your topic, randomly distributed, even with hundreds of partitions, with only a handful of this wrapper deployed, you will probably see many orders of magnitude performance improvements - massively out performing dozens of vanilla Kafka consumer clients.
As an illustrative example of relative performance, given:
-
A random processing time between 0 and 5ms
-
10,000 messages to process
-
A single partition (simplifies comparison - a topic with 5 partitions is the same as 1 partition with a keyspace of 5)
-
Default
ParallelConsumerOptions
-
maxConcurrency = 100
-
numberOfThreads = 16
-
Ordering | Number of keys | Duration | Note |
---|---|---|---|
Partition |
20 (not relevant) |
22.221s |
This is the same as a single partition with a single normal serial consumer, as we can see: 2.5ms avg processing time * 10,000 msg / 1000ms = ~25s. |
Key |
1 |
26.743s |
Same as above |
Key |
2 |
13.576s |
|
Key |
5 |
5.916s |
|
Key |
10 |
3.310s |
|
Key |
20 |
2.242s |
|
Key |
50 |
2.204s |
|
Key |
100 |
2.178s |
|
Key |
1,000 |
2.056s |
|
Key |
10,000 |
2.128s |
As key space is t he same as the number of messages, this is similar (but restricted by max concurrency settings) as having a single consumer instance and partition per key. 10,000 msgs * avg processing time 2.5ms = ~2.5s. |
Unordered |
20 (not relevant) |
2.829s |
As there is no order restriction, this is similar (but restricted by max concurrency settings) as having a single consumer instance and partition per key. 10,000 msgs * avg processing time 2.5ms = ~2.5s. |
If you encounter any issues, or have any suggestions or future requests, please create issues in the github issue tracker. Issues will be dealt with on a good faith, best efforts basis, by the small team maintaining this library.
We also encourage participation, so if you have any feature ideas etc, please get in touch, and we will help you work on submitting a PR!
ℹ️
|
We are very interested to hear about your experiences! And please vote on your favourite issues! |
If you have questions, head over to the Confluent Slack community, or raise an issue on GitHub.
This library is copyright Confluent Inc, and licensed under the Apache License Version 2.0.
This project is available in maven central, repo1, along with SNAPSHOT builds (starting with 0.5-SNAPSHOT) in repo1’s SNAPSHOTS repo.
Latest version can be seen here.
Where ${project.version}
is the version to be used:
link:../..//parallel-consumer-examples/parallel-consumer-example-core/pom.xml[role=include]
link:../..//parallel-consumer-examples/parallel-consumer-example-reactor/pom.xml[role=include]
link:../..//parallel-consumer-examples/parallel-consumer-example-vertx/pom.xml[role=include]
link:../..//parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java[role=include]
-
Setup your clients as per normal. A Producer is only required if using the
produce
flows. -
Choose your ordering type,
KEY
in this case. This ensures maximum concurrency, while ensuring messages are processed and committed inKEY
order, making sure no offset is committed unless all offsets before it in it’s partition, are completed also. -
The maximum number of concurrent processing operations to be performing at any given time. Also, because the library coordinates offsets,
enable.auto.commit
must be disabled in your consumer. -
Subscribe to your topics
ℹ️
|
Because the library coordinates offsets, enable.auto.commit must be disabled.
|
After this setup, one then has the choice of interfaces:
-
ParallelStreamProcessor
-
VertxParallelStreamProcessor
-
JStreamParallelStreamProcessor
-
JStreamVertxParallelStreamProcessor
There is another interface: ParallelConsumer
which is integrated, however there is currently no immediate implementation.
See issue #12, and the ParallelConsumer
JavaDoc:
link:../..//parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumer.java[role=include]
This is the only thing you need to do, in order to get massively concurrent processing in your code.
link:../..//parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java[role=include]
See the core example project, and it’s test.
This interface allows you to process your message, then publish back to the broker zero, one or more result messages. You can also optionally provide a callback function to be run after the message(s) is(are) successfully published to the broker.
link:../..//parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java[role=include]
You have the option to either use callbacks to be notified of events, or use the Streaming
versions of the API, which use the java.util.stream.Stream
system:
-
JStreamParallelStreamProcessor
-
JStreamVertxParallelStreamProcessor
In future versions, we plan to look at supporting other streaming systems like RxJava via modules.
The library also supports sending a batch or records as input to the users processing function in parallel. Using this, you can process several records in your function at once.
To use it, set a batch size
in the options class.
There are then various access methods for the batch of records - see the PollContext
object for more information.
❗
|
If an exception is thrown while processing the batch, all messages in the batch will be returned to the queue, to be retried with the standard retry system. There is no guarantee that the messages will be retried again in the same batch. |
link:../..//parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java[role=include]
-
Choose your batch size.
-
If using a batch version of the API, you must choose a batch size in the options class.
-
If a batch size is chosen, the "normal" APIs cannot be used, and an error will be thrown.
link:../..//parallel-consumer-examples/parallel-consumer-example-vertx/src/main/java/io/confluent/parallelconsumer/examples/vertx/VertxApp.java[role=include]
-
Simply return an object representing the request, the Vert.x HTTP engine will handle the rest, using it’s non-blocking engine
See the Vert.x example project, and it’s test.
As per the Vert.x support, there is also a Reactor module. This means you can use Reactor’s non-blocking threading model to process your messages, allowing for orders of magnitudes higher concurrent processing than the core module’s thread per worker module.
See the Reactor example project, and it’s test.
Mono.just
to return a value, but you can use any Reactor API here.link:../..//parallel-consumer-examples/parallel-consumer-example-reactor/src/main/java/io/confluent/parallelconsumer/examples/reactor/ReactorApp.java[role=include]
Use your Streams app to process your data first, then send anything needed to be processed concurrently to an output topic, to be consumed by the parallel consumer.
link:../..//parallel-consumer-examples/parallel-consumer-example-streams/src/main/java/io/confluent/parallelconsumer/examples/streams/StreamsApp.java[role=include]
-
Setup your Kafka Streams stage as per normal, performing any type of preprocessing in Kafka Streams
-
For the slow consumer part of your Topology, drop down into the parallel consumer, and use massive concurrency
See the Kafka Streams example project, and it’s test.
-
Provision your fully managed Kafka cluster in Confluent Cloud
-
Sign up for Confluent Cloud, a fully-managed Apache Kafka service.
-
After you log in to Confluent Cloud, click on
Add cloud environment
and name the environmentlearn-kafka
. Using a new environment keeps your learning resources separate from your other Confluent Cloud resources. -
Click on LEARN and follow the instructions to launch a Kafka cluster and to enable Schema Registry.
-
-
Access the client configuration settings
-
From the Confluent Cloud Console, navigate to your Kafka cluster. From the
Clients
view, get the connection information customized to your cluster (selectJava
). -
Create new credentials for your Kafka cluster, and then Confluent Cloud will show a configuration block with your new credentials automatically populated (make sure
show API keys
is checked). -
Use these settings presented to configure your clients.
-
-
Use these clients for steps outlined in the Common Preparation section.
This version has a breaking change in the API - instead of passing in ConsumerRecord
instances, it passes in a PollContext
object which has extra information and utility methods.
See the PollContext
class for more information.
The user has the option to either choose ordered, or unordered message processing.
Either in ordered
or unordered
processing, the system will only commit offsets for messages which have been successfully processed.
🔥
|
Unordered processing could cause problems for third party integration where ordering by key is required.
|
🔥
|
Beware of third party systems which are not idempotent, or are key order sensitive. |
❗
|
The below diagrams represent a single iteration of the system and a very small number of input partitions and messages. |
Given this input topic with three partitions and a series of messages:
The normal Kafka client operations in the following manner. Note that typically offset commits are not performed after processing a single message, but is illustrated in this manner for comparison to the single pass concurrent methods below. Usually many messages are committed in a single go, which is much more efficient, but for our illustrative purposes is not really relevant, as we are demonstration sequential vs concurrent processing messages.
Unordered processing is where there is no restriction on the order of multiple messages processed per partition, allowing for highest level of concurrency.
This is the fastest option.
At most only one message from any given input partition will be in flight at any given time. This means that concurrent processing is restricted to the number of input partitions.
The advantage of ordered processing mode, is that for an assignment of 1000 partitions to a single consumer, you do not need to run 1000 consumer instances or threads, to process the partitions in parallel.
Note that for a given partition, a slow processing message will prevent messages behind it from being processed. However, messages in other partitions assigned to the consumer will continue processing.
This option is most like normal operation, except if the consumer is assigned more than one partition, it is free to process all partitions in parallel.
Note that in PARTITION ordered mode it may be necessary to tune Consumer fetch bytes per partition and ParallelConsumer message buffer size in ParallelConsumerOptions
(either through specifying relatively high initialLoadFactor
and maximumLoadFactor
or explicitly setting messageBufferSize
).
As default buffer size is calculated as maxConcurrency * batchSize * loadFactor
- it can be quite small in PARTITION order by default (as concurrency is typically low) and processing threads can be starved if incoming message rate is higher than processing rate as small buffer gets filled with messages from only subset of subscribed partitions before back-pressure kicks in.
It may be not enough to just increase the buffer size - so tuning of underlying Kafka Consumer max.partition.fetch.bytes
is recommended - there are two approaches that could be used to tune it - depending on data distribution.
One - have a large enough buffer to smooth out spikes in specific partitions - this approach may still lead to thread starvation in cases when processing is relatively slow and some partitions have no data for periods of time or data is consistently unevenly partitioned - as buffer will get filled and will take some time to resume polling, but is acceptable when data flow is more or less consistent. Rough target to return X number of messages so that ParallelConsumer buffer is 2 * partitionCount * X. That way back pressure will only kick in after Consumer done at least 2 polls from each subscribed partition.
For example - with 5 subscribed partitions and 1KB message size - can use 500KB max.partition.fetch.bytes
to get a cap of maximum 500 records per partition fetch - so using guide above set messageBufferSize
to 5000 ( 2*5*500 ) as a starting point - and tune from there depending on processing speed - but keeping similar (or higher) ratio of messageBufferSize
to maximum number of records fetched per partition.
Two - have a small buffer and small max.partition.fetch.bytes
- for scenarios when processing is slow and there is no goal to maximize message polling throughput - setting those values low - will allow to drain buffer faster in cases where data flow is inconsistent and some partitions may have no data for periods of time. As a rough starting point buffer can be set to same 2 * fetch per partition * number of partitions - but partition fetch size set to a low value - for example for messages that take 1 second to process - 5-10 messages per fetch per partition would give reasonable buffer drain time and not poll excessively.
- Note
-
Kafka Consumer option
max.poll.records
does not change number of records actually fetched by Kafka Consumer - so it is not really useful for this tuning.
Refer to PartitionOrderProcessingTest
integration tests for example scenario.
Most similar to ordered by partition, this mode ensures process ordering by key (per partition).
The advantage of this mode, is that a given input topic may not have many partitions, it may have a ~large number of unique keys. Each of these key → message sets can actually be processed concurrently, bringing concurrent processing to a per key level, without having to increase the number of input partitions, whilst keeping strong ordering by key.
As usual, the offset tracking will be correct, regardless of the ordering of unique keys on the partition or adjacency to the committed offset, such that after failure or rebalance, the system will not replay messages already marked as successful.
This option provides the performance of maximum concurrency, while maintaining message processing order per key, which is sufficient for many applications.
Even during retries, offsets will always be committed only after successful processing, and in order.
If processing of a record fails, the record will be placed back into it’s queue and retried with a configurable delay (see the ParallelConsumerOptions
class).
Ordering guarantees will always be adhered to, regardless of failure.
A failure is denoted by any exception being thrown from the user’s processing function. The system catches these exceptions, logs them and replaces the record in the queue for processing later. All types of Exceptions thrown are considered retriable. To not retry a record, do not throw an exception from your processing function.
- TIP
-
To avoid the system logging an error, throw an exception which extends PCRetriableException.
- TIP
-
If there was an error processing a record, and you’d like to skip it - do not throw an exception, and the system will mark the record as succeeded.
If for some reason you want to proactively fail a record, without relying on some other system throwing an exception which you don’t catch - simply throw an exception of your own design, which the system will treat the same way.
To configure the retry delay, see ParallelConsumerOptions#defaultRetryDelay
.
At the moment there is no terminal error support, so messages will continue to be retried forever as long as an exception continues to be thrown from the user function (see Skipping Records).
But still this will not hold up the queues in KEY
or UNORDERED
modes, however in PARTITION
mode it will block progress.
Offsets will also continue to be committed (see Commit Mode and Offset Map).
As part of the enhanced retry epic, the ability to dynamically determine the retry delay was added. This can be used to customise retry delay for a record, such as exponential back off or have different delays for different types of records, or have the delay determined by the status of a system etc.
You can access the retry count of a record through it’s wrapped WorkContainer
class, which is the input variable to the retry delay function.
link:../..//parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java[role=include]
If for whatever reason you want to skip a record, simply do not throw an exception, or catch any exception being thrown, log and swallow it and return from the user function normally. The system will treat this as a record processing success, mark the record as completed and move on as though it was a normal operation.
A user may choose to skip a record for example, if it has been retried too many times or if the record is invalid or doesn’t need processing.
Implementing a max retries feature as a part of the system is planned.
link:../..//parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java[role=include]
Although the system doesn’t have an explicit circuit breaker pattern feature, one can be created by combining the custom retry delay function and proactive failure. For example, the retry delay can be calculated based upon the status of an external system - i.e. if the external system is currently out of action, use a higher retry. Then in the processing function, again check the status of the external system first, and if it’s still offline, throw an exception proactively without attempting to process the message. This will put the message back in the queue.
link:../..//parallel-consumer-examples/parallel-consumer-example-core/src/main/java/io/confluent/parallelconsumer/examples/core/CoreApp.java[role=include]
In order to have a failing record not block progress of a partition, one of the ordering modes other than PARTITION
must be used, so that the system is allowed to process other messages that are perhaps in KEY
order or in the case of UNORDERED
processing - any message.
This is because in PARTITION
ordering mode, records are always processed in order of partition, and so the Head of Line blocking feature is effectively disabled.
Improvements to this system are planned, see the following issues:
-
Void
Processing is complete simply when your provided function finishes, and the offsets are committed.
-
Streaming User Results
When your function is actually run, a result object will be streamed back to your client code, with information about the operation completion.
-
Streaming Message Publishing Results
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.
The system gives you three choices for how to do offset commits.
The simplest of the three are the two Consumer commits modes.
They are of course, synchronous
and asynchronous
mode.
The transactional
mode is explained in the next section.
Asynchronous
mode is faster, as it doesn’t block the control loop.
Synchronous
will block the processing loop until a successful commit response is received, however, Asynchronous
will still be capped by the max processing settings in the ParallelConsumerOptions
class.
If you’re used to using the auto commit mode in the normal Kafka consumer, you can think of the Asynchronous
mode being similar to this.
We suggest starting with this mode, and it is the default.
There is also the option to use Kafka’s Exactly Once Semantics (EoS) system. This causes all messages produced, by all workers in parallel, as a result of processing their messages, to be committed within a SINGLE, BULK transaction, along with their source offset.
Note importantly - this is a BULK transaction, not a per input record transaction.
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.
- CAUTION
-
This is a BULK transaction, not a per input record transaction. There is not a single transaction per input record and per worker "thread", but one LARGE transaction that gets used by all parallel processing, until the commit interval.
- NOTE
-
As with the
synchronous
processing mode, this will also block the processing loop until a successful transaction completes
🔥
|
This cannot be true for any externally integrated third party system, unless that system is idempotent. |
For implementations details, see the Transactional System Architecture section.
link:../..//parallel-consumer-core/src/main/java/io/confluent/parallelconsumer/ParallelConsumerOptions.java[role=include]
Kafka Streams (KS) doesn’t yet (KIP-311, KIP-408) have parallel processing of messages. However, any given preprocessing can be done in KS, preparing the messages. One can then use this library to consume from an input topic, produced by KS to process the messages in parallel.
For a code example, see the Kafka Streams Concurrent Processing section.
Metrics collection subsystem is implemented using Micrometer. This allows for flexible configuration of target metrics backend to be used. See below on example of how to configure MeterRegistry for Parallel Consumer to use for metrics collection.
Following meters are defined by Parallel Consumer - grouped by Subsystem
Number Of Partitions
Gauge pc.partitions.number{subsystem=partitions}
Number of partitions
Partition Incomplete Offsets
Gauge pc.partition.incomplete.offsets{subsystem=partitions, topic="topicName", partition="partitionNumber"}
Number of incomplete offsets in the partition
Partition Highest Completed Offset
Gauge pc.partition.highest.completed.offset{subsystem=partitions, topic="topicName", partition="partitionNumber"}
Highest completed offset in the partition
Partition Highest Sequential Succeeded Offset
Gauge pc.partition.highest.sequential.succeeded.offset{subsystem=partitions, topic="topicName", partition="partitionNumber"}
Highest sequential succeeded offset in the partition
Partition Highest Seen Offset
Gauge pc.partition.highest.seen.offset{subsystem=partitions, topic="topicName", partition="partitionNumber"}
Highest seen / consumed offset in the partition
Partition Last Committed Offset
Gauge pc.partition.latest.committed.offset{subsystem=partitions, topic="topicName", partition="partitionNumber"}
Latest committed offset in the partition
Partition Assignment Epoch
Gauge pc.partition.assignment.epoch{subsystem=partitions, topic="topicName", partition="partitionNumber"}
Epoch of partition assignment
User Function Processing Time
Timer pc.user.function.processing.time{subsystem=processor}
User function processing time
Dynamic Extra Load Factor
Gauge pc.dynamic.load.factor{subsystem=processor}
Dynamic load factor - load of processing buffers
Pc Status
Gauge pc.status{subsystem=processor}
PC Status, reported as number with following mapping - 0:UNUSED, 1:RUNNING, 2:PAUSED, 3:DRAINING, 4:CLOSING, 5:CLOSED
Number Of Shards
Gauge pc.shards{subsystem=shardmanager}
Number of shards
Incomplete Offsets Total
Gauge pc.incomplete.offsets.total{subsystem=shardmanager}
Total number of incomplete offsets
Shards Size
Gauge pc.shards.size{subsystem=shardmanager}
Number of records queued for processing across all shards
Inflight Records
Gauge pc.inflight.records{subsystem=workmanager}
Total number of records currently being processed or waiting for retry
Waiting Records
Gauge pc.waiting.records{subsystem=workmanager}
Total number of records waiting to be selected for processing
Processed Records
Counter pc.processed.records{subsystem=workmanager, topic="topicName", partition="partitionNumber"}
Total number of records successfully processed
Failed Records
Counter pc.failed.records{subsystem=workmanager, topic="topicName", partition="partitionNumber"}
Total number of records failed to be processed
Slow Records
Counter pc.slow.records{subsystem=workmanager, topic="topicName", partition="partitionNumber"}
Total number of records that spent more than the configured time threshold in the waiting queue. This setting defaults to 10 seconds
Pc Poller Status
Gauge pc.poller.status{subsystem=poller}
PC Broker Poller Status, reported as number with following mapping - 0:UNUSED, 1:RUNNING, 2:PAUSED, 3:DRAINING, 4:CLOSING, 5:CLOSED
Num Paused Partitions
Gauge pc.partitions.paused{subsystem=poller}
Number of paused partitions
Offsets Encoding Time
Timer pc.offsets.encoding.time{subsystem=offsetencoder}
Time spend encoding offsets
Offsets Encoding Usage
Counter pc.offsets.encoding.usage{subsystem=offsetencoder, codec="BitSet|BitSetCompressed|BitSetV2Compressed|RunLength"}
Offset encoding usage per encoding type
Metadata Space Used
Distribution Summary pc.metadata.space.used{subsystem=offsetencoder}
Ratio between offset metadata payload size and available space
Payload Ratio Used
Distribution Summary pc.payload.ratio.used{subsystem=offsetencoder}
Ratio between offset metadata payload size and offsets encoded
Meter registry that metrics should be bound has to be set using Parallel Consumer Options along with any common tags that identify the PC instance. In addition, if desired - Kafka Consumer, Producer can be bound to the registry as well as general JVM metric, logging system and other common binders.
Following example illustrates setup of Parallel Consumer with Meter Registry and binds Kafka Consumer to that same registry as well.
link:../..//parallel-consumer-examples/parallel-consumer-example-metrics/src/main/java/io/confluent/parallelconsumer/examples/metrics/CoreApp.java[role=include]
-
- Meter Registry is set through ParallelConsumerOptions.builder(), if not specified - will default to CompositeMeterRegistry - which is No-op.
-
- Optional - common tags can be specified through same builder - they will be added to all Parallel Consumer meters
-
- Optional - instance tag value can be specified - it has to be unique to ensure meter uniqueness in cases when multiple parallel consumer instances are recording metrics to the same meter registry. If instance tag is not specified - unique UUID value will be generated and used. Tag is created with tag key 'pcinstance'.
-
- Optional - Kafka Consumer Micrometer metrics object created for Kafka Consumer that is later used for Parallel Consumer.
-
- Optional - Kafka Consumer Micrometer metrics are bound to Meter Registry.
- NOTE
-
any additional binders / metrics need to be cleaned up appropriately - for example the Kafka Consumer Metrics registered above - need to be closed using
kafkaClientMetrics.close()
after calling shutting down Parallel Consumer as Parallel Consumer will close Kafka Consumer on shutdown.
For released changes, see the CHANGELOG.
For features in development and a more accurate view on the roadmap, have a look at the GitHub issues, and clone Antony’s fork.
-
Client side
-
JDK 8
-
SLF4J
-
Apache Kafka (AK) Client libraries 2.5
-
Supports all features of the AK client (e.g. security setups, schema registry etc)
-
For use with Streams, see Using with Kafka Streams section
-
For use with Connect:
-
Source: simply consume from the topic that your Connect plugin is publishing to
-
Sink: use the poll and producer style API and publish the records to the topic that the connector is sinking from
-
-
-
Server side
-
Should work with any cluster that the linked AK client library works with
-
If using EoS/Transactions, needs a cluster setup that supports EoS/transactions
-
-
-
Integration tests require a running locally accessible Docker host.
-
Has a Maven
profile
setup for IntelliJ Idea, but not Eclipse for example.
The unit test code is set to run at a very high frequency, which can make it difficult to read debug logs (or impossible). If you want to debug the code or view the main logs, consider changing the below:
ParallelEoSStreamProcessorTestBase#DEFAULT_BROKER_POLL_FREQUENCY_MS
ParallelEoSStreamProcessorTestBase#DEFAULT_COMMIT_INTERVAL_MAX_MS
-
AsciiDoc
-
CheckStyle
-
CodeGlance
-
EditorConfig
-
Rainbow Brackets
-
SonarLint
-
Lombok
The README
uses a special custom maven processor plugin to import live code blocks into the root readme, so that GitHub can show the real code as includes in the README
.
This is because GitHub doesn’t properly support the include directive.
The source of truth readme is in ../..//src/docs/README_TEMPLATE.adoc.
-
Compile and run all tests
mvn verify
-
Run tests excluding the integration tests
mvn test
-
Run all tests
mvn verify
-
Run any goal skipping tests (replace
<goalName>
e.g.install
)mvn <goalName> -DskipTests
-
See what profiles are active
mvn help:active-profiles
-
See what plugins or dependencies are available to be updated
mvn versions:display-plugin-updates versions:display-property-updates versions:display-dependency-updates
-
Run a single unit test
mvn -Dtest=TestCircle test
-
Run a specific integration test method in a submodule project, skipping unit tests
mvn -Dit.test=TransactionAndCommitModeTest#testLowMaxPoll -DskipUTs=true verify -DfailIfNoTests=false --projects parallel-consumer-core
-
Run
git bisect
to find a bad commit, edit the Maven command inbisect.sh
and run
git bisect start good bad git bisect run ./bisect.sh
- Note
-
mvn compile
- Due to a bug in Maven’s handling of test-jar dependencies - runningmvn compile
fails, usemvn test-compile
instead. See issue #162 and this Stack Overflow question.
The project has good automated test coverage, of all features.
Including integration tests running against real Kafka broker and database.
If you want to run the tests yourself, clone the repository and run the command: mvn test
.
The tests require an active docker server on localhost
.
We use the excellent Testcontainers library for integration testing with JUnit.
To speed up test execution, you can enable container reuse across test runs by setting the following in your ~/.testcontainers.properties
file:
testcontainers.reuse.enable=true
This will leave the container running after the JUnit test is complete for reuse by subsequent runs.
ℹ️The container will only be left running if it is not explicitly stopped by the JUnit rule. For this reason, we use a variant of the singleton container pattern instead of the JUnit rule.
Testcontainers detects if a container is reusable by hashing the container creation parameters from the JUnit test. If an existing container is not reusable, a new container will be created, but the old container will not be removed.
Target | Description --- | ---
testcontainers-list
| List all containers labeled as testcontainers
testcontainers-clean
| Remove all containers labeled as testcontainers
org.testcontainers=true
docker container ls --filter 'label=org.testcontainers=true' --format '{{.ID}}' \
| $(XARGS) docker container rm --force
org.testcontainers=true
docker container ls --filter 'label=org.testcontainers=true'
ℹ️testcontainers-clean
removes all docker containers on your system with theio.testcontainers=true
label > (including the most recent container which may be reusable).
See this testcontainers PR for details on the reusable containers feature.
Concurrency is controlled by the size of the thread pool (worker pool
in the diagram).
Work is performed in a blocking manner, by the users submitted lambda functions.
These are the main sub systems:
-
controller thread
-
broker poller thread
-
work pool thread
-
work management
-
offset map manipulation
Each thread collaborates with the others through thread safe Java collections.
The Vert.x module is an optional extension to the core module. As depicted in the diagram, the architecture extends the core architecture.
Instead of the work thread pool count being the degree of concurrency, it is controlled by a max parallel requests setting, and work is performed asynchronously on the Vert.x engine by a core count aligned Vert.x managed thread pool using Vert.x asynchronous IO plugins (verticles).
Unlike a traditional queue, messages are not deleted on an acknowledgement. However, offsets are tracked per message, per consumer group - there is no message replay for successful messages, even over clean restarts.
Across a system failure, only completed messages not stored as such in the last offset payload commit will be replayed. This is not an exactly once guarantee, as message replay cannot be prevented across failure.
🔥
|
Note that Kafka’s Exactly Once Semantics (EoS) (transactional processing) also does not prevent duplicate message replay - it presents an effectively once result messages in Kafka topics.
Messages may still be replayed when using EoS .
This is an important consideration when using it, especially when integrating with thrid party systems, which is a very common pattern for utilising this project.
|
As mentioned previously, offsets are always committed in the correct order and only once all previous messages have been successfully processed; regardless of ordering mode selected. We call this the "highest committable offset".
However, because messages can be processed out of order, messages beyond the highest committable offset must also be tracked for success and not replayed upon restart of failure. To achieve this the system goes a step further than normal Kafka offset commits.
When messages beyond the highest committable offset are successfully processed;
-
they are stored as such in an internal memory map.
-
when the system then next commits offsets
-
if there are any messages beyond the highest offset which have been marked as succeeded
-
the offset map is serialised and encoded into a base 64 string, and added to the commit message metadata.
-
-
upon restore, if needed, the system then deserializes this offset map and loads it back into memory
-
when each messages is polled into the system
-
it checks if it’s already been previously completed
-
at which point it is then skipped.
-
This ensures that no message is reprocessed if it’s been previously completed.
❗
|
Successful messages beyond the highest committable offset are still recorded as such in a specially constructed metadata payload stored alongside the Kafka committed offset. These messages are not replayed upon restore/restart. |
The offset map is compressed in parallel using two different compression techniques - run length encoding and bitmap encoding.
The sizes of the compressed maps are then compared, and the smallest chosen for serialization.
If both serialised formats are significantly large, they are then both compressed using zstd
compression, and if that results in a smaller serialization then the compressed form is used instead.
-
Runtime data model creates list of incomplete offsets
-
Continuously builds a full complete / not complete bit map from the base offset to be committed
-
Dynamically switching storage
-
encodes into a
BitSet
, and aRunLength
, then compresses both using zstd, then uses the smallest and tags as such in the encoded String -
Which is smallest can depend on the size and information density of the offset map
-
Smaller maps fit better into uncompressed
BitSets
~(30 entry map bitset: compressed: 13 Bytes, uncompressed: 4 Bytes) -
Larger maps with continuous sections usually better in compressed
RunLength
-
Completely random offset maps, compressed and uncompressed
BitSet
is roughly the same (2000 entries, uncompressed bitset: 250, compressed: 259, compressed bytes array: 477) -
Very large maps (20,000 entries), a compressed
BitSet
seems to be significantly smaller again if random.
-
-
-
Gets stored along with base offset for each partition, in the offset
commitsync
metadata
string -
The offset commit metadata has a hardcoded limit of 4096 bytes (4 kb) per partition (@see
kafka.coordinator.group.OffsetConfig#DefaultMaxMetadataSize = 4096
)-
Because of this, if our map doesn’t fit into this, we have to drop it and not use it, losing the shorter replay benefits. However, with runlength encoding and typical offset patterns this should be quite rare.
-
Work is being done on continuous and predictive space requirements, which will optionally prevent the system from continuing past a point by introducing local backpressure which it can’t proceed without dropping the encoded map information - see Exact continuous offset encoding for precise offset payload size back pressure.
-
-
Not being able to fit the map into the metadata, depends on message acknowledgement patterns in the use case and the numbers of messages involved. Also, the information density in the map (i.e. a single not yet completed message in 4000 completed ones will be a tiny map and will fit very large amounts of messages)
-
-
If for example, offset 5 cannot be processed for whatever reason, does it cause the committed offset to stick to 5?
Yes - the committed offset would "stick" to 5, with the metadata payload containing all the per msg ack’s beyond 5.
(Reference: #415)
-
In the above scenario, would the system eventually exceed the OffsetMap size limit?
No, as if the payload size hits 75% or more of the limit (4kB), the back pressure system kicks in, and no more records will be taken for processing, until it drops below 75% again. Instead, it will keep retrying existing records.
However, note that if the only record to continually fail is 5, and all others succeed, let’s say offset 6-50,000, then the metadata payload is only ~2 shorts (1 and (50,000-6=) 49,994), as it will use run length encoding. So it’s very efficient.
(Reference: #415)
Apache®, Apache Kafka, and Kafka® are either registered trademarks or trademarks of the Apache Software Foundation in the United States and/or other countries.
Quite simply the best profiler for Java, and the only one I use. I have been using it for decades. Quick, easy to use but soo powerful.
YourKit supports open source projects with innovative and intelligent tools for monitoring and profiling Java and .NET applications.
YourKit is the creator of YourKit Java Profiler, YourKit .NET Profiler, and YourKit YouMonitor.