[jira] [Commented] (KAFKA-1894) Avoid long or infinite blocking in the consumer

2016-11-09 Thread Catalina-Alina Dobrica (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1894?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15650814#comment-15650814
 ] 

Catalina-Alina Dobrica commented on KAFKA-1894:
---

This issue also prevents the consumer's thread from being interrupted. This is 
particularly relevant when the consumer is integrated in an external system - 
like a camel ecosystem. Trying to force the shutdown of the ExecutorService 
that manages the thread or to terminate the thread itself has no effect and the 
thread is in the infinite loop. This eventually leads to OOME if enough such 
threads are started.
I found this issue when providing an incorrect SSL protocol to the consumer in 
version 0.10.1.0, but it can occur in any circumstance where the channel is not 
established - such as not having kafka enabled. The thread loops infinitely to 
check if this connection was established, which, in some cases, will never 
happen.

> Avoid long or infinite blocking in the consumer
> ---
>
> Key: KAFKA-1894
> URL: https://issues.apache.org/jira/browse/KAFKA-1894
> Project: Kafka
>  Issue Type: Sub-task
>  Components: consumer
>Reporter: Jay Kreps
>Assignee: Jason Gustafson
> Fix For: 0.10.2.0
>
>
> The new consumer has a lot of loops that look something like
> {code}
>   while(!isThingComplete())
> client.poll();
> {code}
> This occurs both in KafkaConsumer but also in NetworkClient.completeAll. 
> These retry loops are actually mostly the behavior we want but there are 
> several cases where they may cause problems:
>  - In the case of a hard failure we may hang for a long time or indefinitely 
> before realizing the connection is lost.
>  - In the case where the cluster is malfunctioning or down we may retry 
> forever.
> It would probably be better to give a timeout to these. The proposed approach 
> would be to add something like retry.time.ms=6 and only continue retrying 
> for that period of time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Issue Comment Deleted] (KAFKA-4189) Consumer poll hangs forever if kafka is disabled

2016-11-09 Thread Catalina-Alina Dobrica (JIRA)

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

Catalina-Alina Dobrica updated KAFKA-4189:
--
Comment: was deleted

(was: This issue also prevents the consumer's thread from being interrupted. 
This is particularly relevant when the consumer is integrated in an external 
system - like a camel ecosystem. Trying to force the shutdown of the 
ExecutorService that manages the thread or to terminate the thread itself has 
no effect and the thread is in an infinite loop in 
org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient#awaitMetadataUpdate().
 This eventually leads to OOME if enough such threads are started.
I found this issue when providing an incorrect SSL protocol to the consumer in 
version 0.10.1.0, but it can occur in any circumstance where the channel is not 
established - such as not having kafka enabled. The thread loops infinitely to 
check if this connection was established, which, in some cases, will never 
happen.)

> Consumer poll hangs forever if kafka is disabled
> 
>
> Key: KAFKA-4189
> URL: https://issues.apache.org/jira/browse/KAFKA-4189
> Project: Kafka
>  Issue Type: Bug
>  Components: clients, consumer
>Affects Versions: 0.9.0.1, 0.10.0.1
>Reporter: Tomas Benc
>Priority: Critical
>
> We develop web application, where client sends REST request and our 
> application downloads messages from Kafka and sends those messages back to 
> client. In our web application we use "New Consumer API" (not High Level nor 
> Simple Consumer API).
> Problem occurs in case of disabling Kafka and web application is running on. 
> Application receives request and tries to poll messages from Kafka. 
> Processing is on that line blocked until Kafka is enabled.
> ConsumerRecords records = consumer.poll(1000);
> Timeout parameter of the poll method has no influence in such case. I expect 
> poll method could throw some Exception describing about connection issues.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-4189) Consumer poll hangs forever if kafka is disabled

2016-11-09 Thread Catalina-Alina Dobrica (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-4189?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15650781#comment-15650781
 ] 

Catalina-Alina Dobrica commented on KAFKA-4189:
---

This issue also prevents the consumer's thread from being interrupted. This is 
particularly relevant when the consumer is integrated in an external system - 
like a camel ecosystem. Trying to force the shutdown of the ExecutorService 
that manages the thread or to terminate the thread itself has no effect and the 
thread is in an infinite loop in 
org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient#awaitMetadataUpdate().
 This eventually leads to OOME if enough such threads are started.
I found this issue when providing an incorrect SSL protocol to the consumer in 
version 0.10.1.0, but it can occur in any circumstance where the channel is not 
established - such as not having kafka enabled. The thread loops infinitely to 
check if this connection was established, which, in some cases, will never 
happen.

> Consumer poll hangs forever if kafka is disabled
> 
>
> Key: KAFKA-4189
> URL: https://issues.apache.org/jira/browse/KAFKA-4189
> Project: Kafka
>  Issue Type: Bug
>  Components: clients, consumer
>Affects Versions: 0.9.0.1, 0.10.0.1
>Reporter: Tomas Benc
>Priority: Critical
>
> We develop web application, where client sends REST request and our 
> application downloads messages from Kafka and sends those messages back to 
> client. In our web application we use "New Consumer API" (not High Level nor 
> Simple Consumer API).
> Problem occurs in case of disabling Kafka and web application is running on. 
> Application receives request and tries to poll messages from Kafka. 
> Processing is on that line blocked until Kafka is enabled.
> ConsumerRecords records = consumer.poll(1000);
> Timeout parameter of the poll method has no influence in such case. I expect 
> poll method could throw some Exception describing about connection issues.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)