[ 
https://issues.apache.org/jira/browse/KAFKA-14494?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Oleg Zhovtanyuk updated KAFKA-14494:
------------------------------------
    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.

 
{code:java}
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
{code}
 

 

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:

 
{code:java}
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)
{code}
 

I'm ready to fix the issue, just need some initial guidance - what's missing?

 

 

Both test clients are attached too.

As Java client ignores JVM proxy options (it uses custom socket factory), I've 
employed 
[socksify|https://manpages.debian.org/testing/dante-client/socksify.1.en.html] 
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
{code:java}
sudo apt-get install dante-client{code}
 

Java client can be run as
 * Kafka cluster URI + credentials -> 'client.properties',
 * 
{code:java}
mvn clean package{code}

 * 
{code:java}
./run-with-socksify.sh{code}

NodeJS client can be run as 
 * Kafka cluster URI + credentials -> 'consumer.js',
 * 
{code:java}
npm install .{code}

 * 
{code:java}
./run-with-socksify.sh {code}

 

 

 

  was:
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, performs the same, but proceeds further to the binary data exchange.

 

Both clients and their execution logs are attached.

Java client logs looks like

 

As Java client ignores JVM proxy options (it uses custom socket factory), I've 
employed 
[socksify|https://manpages.debian.org/testing/dante-client/socksify.1.en.html] 
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
{code:java}
sudo apt-get install dante-client{code}
 

Java client can be run as 
 * Kafka cluster URI + credentials -> 'client.properties',
 * 
{code:java}
mvn clean package{code}

 * 
{code:java}
./run-with-socksify.sh{code}

NodeJS client can be run as 
 * Kafka cluster URI + credentials -> 'consumer.js',
 * 
{code:java}
npm install .{code}

 * 
{code:java}
./run-with-socksify.sh {code}

 

 

 


> Kafka Java client can't send data when behind SOCKS proxy - while native 
> client can
> -----------------------------------------------------------------------------------
>
>                 Key: KAFKA-14494
>                 URL: https://issues.apache.org/jira/browse/KAFKA-14494
>             Project: Kafka
>          Issue Type: Bug
>          Components: clients
>    Affects Versions: 3.3.1
>            Reporter: Oleg Zhovtanyuk
>            Priority: Major
>         Attachments: client-java.tar.gz, client-nodejs.tar.gz, 
> kafka-client-java-direct.log.gz, kafka-client-java-socks.log.gz, 
> kafka-client-node-socks.log.gz
>
>
> 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.
>  
> {code:java}
> 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
> {code}
>  
>  
> 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:
>  
> {code:java}
> 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)
> {code}
>  
> I'm ready to fix the issue, just need some initial guidance - what's missing?
>  
>  
> Both test clients are attached too.
> As Java client ignores JVM proxy options (it uses custom socket factory), 
> I've employed 
> [socksify|https://manpages.debian.org/testing/dante-client/socksify.1.en.html]
>  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
> {code:java}
> sudo apt-get install dante-client{code}
>  
> Java client can be run as
>  * Kafka cluster URI + credentials -> 'client.properties',
>  * 
> {code:java}
> mvn clean package{code}
>  * 
> {code:java}
> ./run-with-socksify.sh{code}
> NodeJS client can be run as 
>  * Kafka cluster URI + credentials -> 'consumer.js',
>  * 
> {code:java}
> npm install .{code}
>  * 
> {code:java}
> ./run-with-socksify.sh {code}
>  
>  
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to