Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Low Throughput with Kafka Source if Parallelism is a Multiple of Kafka Broker #26293

Open
SoerenHenning opened this issue Mar 18, 2024 · 0 comments

Comments

@SoerenHenning
Copy link

TL;DR: The use of consumer.poll(Duration.ZERO) in the Kafka Source causes a very low throughput of Hazelcast if the number of Hazelcast instances (or global parallelism) is a multiple of the number of Kafka brokers.

We recently conducted an extensive benchmarking study, where we observed a surprisingly low throughput of Hazelcast Jet compared to other stream processing engines. In summary, our setup consisted of an AWS-Kubernetes cluster with 3 Kafka brokers and 9 Hazelcast Pods, which were consuming the data from Kafka. For our pipeline, Hazelcast was able to process around 100 MB/s, while other engines process data at rates in the order of 300-400 MB/s. After additional experiments, we were able to find that the throughput is always only that low if the number of Hazelcast Pods is a multiple of the number of Kafka brokers. For example, with a 3-broker Kafka cluster, the overall throughput of Hazelcast was quite low with 3, 6 and 9 Hazelcast Pods (compared to a e.g. 2, 5 and 8). Likewise, with a 4-broker cluster, the throughput with 4, 8 and 12 Hazelcast Pods was very low. Here are two plots showing these results (one records is 1024 bytes):

What became apparent is that we observe the low throughput always if Kafka consumers talk to only a single Kafka broker, because both the assignment of partitions to brokers and the assignment of partitions to consumers happens in a round robin fashion. See the following diagram for an illustration.

Interaction of Kafka consumers with Kafka brokers

In a series of experiments, we were able to rule out that this is related to:

  • Underlying infrastructure, such as network, Kubernetes cluster size, node sizes, etc. (The finding were reproducible in various configurations.)
  • Size of Kafka cluster/brokers (Kafka was not the bottleneck.)
  • The use of Pods with only a single CPU (For example, 3 Pods with local parallelism of 3 yields basically the same result as 9 Pods with local parallelism of 1.)
  • CPU (or memory) constraints within a Pod (Setting local parallelism to 1, but giving unlimited CPU resources did not significantly improve the results.)
  • The pipeline itself (A simplified pipeline, which only reads the data, filters null record (not possible) and writes those to disk (does never happen) yields the same throughput pattern.)
  • The size of record (We used records of 1 KB size, but also with smaller record the total throughput in terms of MB/s was the same.)

(I can share more details if required.)

Eventually, we found that this is related to calling the Kafka consumer poll(..) method with a timeout of Duration.ZERO in the Kafka Source. This setting was changed with commit 04fe8d3 and PR hazelcast/hazelcast-jet#2732 in the hazelcast/hazelcast-jet project from Duration.ofMillis(50). Changing it again to consumer.poll(Duration.ofMillis(50)) improves the throughput benchmark results as follows:

(We can see that still 9 instances perform slightly worse than 8 instances, but this is at least in line with our results for, e.g., Kafka Streams.)

I do not know enough about the inner workings of the Kafka consumer poll method to explain why reducing the timeout has such a strong impact only if the number of Kafka consumers is a multiple of the the number of Kafka brokers, but would surely be interested in any explanation. Moreover, I do not know what the implications of changing the poll timeout in Hazelcast (or making it configurable) are. So I do not want to propose any change, but would like raise awareness of our observation. We are also available for further discussions.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

No branches or pull requests

1 participant