Details
-
Bug
-
Status: Open
-
Major
-
Resolution: Unresolved
-
3.3.1
-
None
-
None
Description
When Kafka Java client sits behind the SOCK5 proxy, it connects to the cluster, gets the list of brokers, but enters the infinite loop trying to detect the least loaded broker.
To the contrary, NodeJS client (a wrapper for librdkafka) with exactly the same setup, proceeds further to the binary data exchange.
The TRACE logs for Java client (with and without proxy) and for the NodeJS client (with proxy) are attached.
Diff'ing the logs highlights the issue.
grep ' Found least loaded connecting node' kafka-client-java-direct.log | wc -l 28 grep ' Found least loaded connecting node' kafka-client-java-socks.log | wc -l 434
Some debugging pointed to the place where it stucks - https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java#L712
Call stack:
leastLoadedNode:672, NetworkClient (org.apache.kafka.clients) leastLoadedNode:141, ConsumerNetworkClient (org.apache.kafka.clients.consumer.internals) lookupCoordinator:300, AbstractCoordinator (org.apache.kafka.clients.consumer.internals) ensureCoordinatorReady:264, AbstractCoordinator (org.apache.kafka.clients.consumer.internals) ensureCoordinatorReady:240, AbstractCoordinator (org.apache.kafka.clients.consumer.internals) coordinatorUnknownAndUnreadySync:492, ConsumerCoordinator (org.apache.kafka.clients.consumer.internals) poll:524, ConsumerCoordinator (org.apache.kafka.clients.consumer.internals) updateAssignmentMetadataIfNeeded:1276, KafkaConsumer (org.apache.kafka.clients.consumer) poll:1240, KafkaConsumer (org.apache.kafka.clients.consumer) poll:1220, KafkaConsumer (org.apache.kafka.clients.consumer) main:32, AppConsumer (com.example.test.kafka)
I'm ready to fix the issue, just need some initial guidance...
Both test clients are attached too.
As Java client ignores JVM proxy options (it uses custom socket factory), I've employed socksify native library, that forcefully forwards all socket calls to the proxy server.
It can be installed with the package manager on any UNIX, e.g. for Ubuntu
sudo apt-get install dante-client
Java client can be run as
- Kafka cluster URI + credentials -> 'client.properties',
mvn clean package
./run-with-socksify.sh
NodeJS client can be run as
- Kafka cluster URI + credentials -> 'consumer.js',
npm install .
./run-with-socksify.sh