[jira] [Commented] (KAFKA-3822) Kafka Consumer close() hangs indefinitely if Kafka Broker shutdown while connected

2016-06-29 Thread Jason Gustafson (JIRA)

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

Jason Gustafson commented on KAFKA-3822:


[~singhashish] There are two reasons why we might block in close(). First, if 
autocommit is enabled, we can block waiting for the final commit to complete. 
Second, the consumer typically needs to send a LeaveGroup request to ensure 
that the group can reassign its partitions without waiting for the session 
timeout. There may be other reasons in the future why we need to block there as 
well. I think letting the user provide a timeout in close() and the other 
blocking methods gives the most flexibility, but it does clutter the API a 
little bit and it's not clear if that much flexibility is really needed. Maybe 
we just need to choose one option, list the other in the rejected alternatives, 
and see how the discussion pans out.

> Kafka Consumer close() hangs indefinitely if Kafka Broker shutdown while 
> connected
> --
>
> Key: KAFKA-3822
> URL: https://issues.apache.org/jira/browse/KAFKA-3822
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Affects Versions: 0.9.0.1, 0.10.0.0
> Environment: x86 Red Hat 6 (1 broker running zookeeper locally, 
> client running on a separate server)
>Reporter: Alexander Cook
>Assignee: Ashish K Singh
>
> I am using the KafkaConsumer java client to consume messages. My application 
> shuts down smoothly if I am connected to a Kafka broker, or if I never 
> succeed at connecting to a Kafka broker, but if the broker is shut down while 
> my consumer is connected to it, consumer.close() hangs indefinitely. 
> Here is how I reproduce it: 
> 1. Start 0.9.0.1 Kafka Broker
> 2. Start consumer application and consume messages
> 3. Stop 0.9.0.1 Kafka Broker (ctrl-c or stop script)
> 4. Try to stop application...hangs at consumer.close() indefinitely. 
> I also see this same behavior using 0.10 broker and client. 
> This is my first bug reported to Kafka, so please let me know if I should be 
> following a different format. Thanks! 



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


[jira] [Commented] (KAFKA-3822) Kafka Consumer close() hangs indefinitely if Kafka Broker shutdown while connected

2016-06-28 Thread Ashish K Singh (JIRA)

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

Ashish K Singh commented on KAFKA-3822:
---

[~hachikuji] while working out a POC, I thought more about the options we have. 
Wanted to bounce it off you, before opening it up to larger audience via KIP.

* *Adding timeout to close in {{KafkaConsumer.close()}}* In {{KafkaProducer}} 
adding timeout to close was required as while closing it has incomplete 
requests that are taken care of and timeout specifies max time user would want 
to wait for those requests to complete. However, in {{KafkaConsumer}} we only 
wait for various underlying {{closable}} components like, 
{{ConsumerCoordinator}}, etc, to close.

* *Adding {{max.block.ms}} config to {{ConsumerConfig}}*  this config can be 
accessed by ConsumerCoordinator while doing synchronous waiting, like during 
{{ensureCoordinatorReady}}. 
[Here|https://github.com/SinghAsDev/kafka/commit/3469a9fd7f9b9310652845f0b853fe3eda07b8c4]
 is a small POC.

Thoughts?

> Kafka Consumer close() hangs indefinitely if Kafka Broker shutdown while 
> connected
> --
>
> Key: KAFKA-3822
> URL: https://issues.apache.org/jira/browse/KAFKA-3822
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Affects Versions: 0.9.0.1, 0.10.0.0
> Environment: x86 Red Hat 6 (1 broker running zookeeper locally, 
> client running on a separate server)
>Reporter: Alexander Cook
>Assignee: Ashish K Singh
>
> I am using the KafkaConsumer java client to consume messages. My application 
> shuts down smoothly if I am connected to a Kafka broker, or if I never 
> succeed at connecting to a Kafka broker, but if the broker is shut down while 
> my consumer is connected to it, consumer.close() hangs indefinitely. 
> Here is how I reproduce it: 
> 1. Start 0.9.0.1 Kafka Broker
> 2. Start consumer application and consume messages
> 3. Stop 0.9.0.1 Kafka Broker (ctrl-c or stop script)
> 4. Try to stop application...hangs at consumer.close() indefinitely. 
> I also see this same behavior using 0.10 broker and client. 
> This is my first bug reported to Kafka, so please let me know if I should be 
> following a different format. Thanks! 



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


[jira] [Commented] (KAFKA-3822) Kafka Consumer close() hangs indefinitely if Kafka Broker shutdown while connected

2016-06-27 Thread Ashish K Singh (JIRA)

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

Ashish K Singh commented on KAFKA-3822:
---

[~hachikuji] yeah makes sense. Will have a KIP up for review today.

> Kafka Consumer close() hangs indefinitely if Kafka Broker shutdown while 
> connected
> --
>
> Key: KAFKA-3822
> URL: https://issues.apache.org/jira/browse/KAFKA-3822
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Affects Versions: 0.9.0.1, 0.10.0.0
> Environment: x86 Red Hat 6 (1 broker running zookeeper locally, 
> client running on a separate server)
>Reporter: Alexander Cook
>Assignee: Ashish K Singh
>
> I am using the KafkaConsumer java client to consume messages. My application 
> shuts down smoothly if I am connected to a Kafka broker, or if I never 
> succeed at connecting to a Kafka broker, but if the broker is shut down while 
> my consumer is connected to it, consumer.close() hangs indefinitely. 
> Here is how I reproduce it: 
> 1. Start 0.9.0.1 Kafka Broker
> 2. Start consumer application and consume messages
> 3. Stop 0.9.0.1 Kafka Broker (ctrl-c or stop script)
> 4. Try to stop application...hangs at consumer.close() indefinitely. 
> I also see this same behavior using 0.10 broker and client. 
> This is my first bug reported to Kafka, so please let me know if I should be 
> following a different format. Thanks! 



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


[jira] [Commented] (KAFKA-3822) Kafka Consumer close() hangs indefinitely if Kafka Broker shutdown while connected

2016-06-23 Thread Jason Gustafson (JIRA)

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

Jason Gustafson commented on KAFKA-3822:


[~singhashish] Yeah, that might make the most sense. On the other hand, there 
are already quite a lot of configurations for the consumer, so I'm curious how 
people feel about overloading. The producer is also kind of a weird mix. It has 
both {{max.block.ms}} and a close() which accepts a timeout. Was there a reason 
close() needed to be treated differently? In any case, I think it's time to do 
a KIP to collect wider feedback since this is one of the common problems users 
face with the consumer.

> Kafka Consumer close() hangs indefinitely if Kafka Broker shutdown while 
> connected
> --
>
> Key: KAFKA-3822
> URL: https://issues.apache.org/jira/browse/KAFKA-3822
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Affects Versions: 0.9.0.1, 0.10.0.0
> Environment: x86 Red Hat 6 (1 broker running zookeeper locally, 
> client running on a separate server)
>Reporter: Alexander Cook
>Assignee: Ashish K Singh
>
> I am using the KafkaConsumer java client to consume messages. My application 
> shuts down smoothly if I am connected to a Kafka broker, or if I never 
> succeed at connecting to a Kafka broker, but if the broker is shut down while 
> my consumer is connected to it, consumer.close() hangs indefinitely. 
> Here is how I reproduce it: 
> 1. Start 0.9.0.1 Kafka Broker
> 2. Start consumer application and consume messages
> 3. Stop 0.9.0.1 Kafka Broker (ctrl-c or stop script)
> 4. Try to stop application...hangs at consumer.close() indefinitely. 
> I also see this same behavior using 0.10 broker and client. 
> This is my first bug reported to Kafka, so please let me know if I should be 
> following a different format. Thanks! 



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


[jira] [Commented] (KAFKA-3822) Kafka Consumer close() hangs indefinitely if Kafka Broker shutdown while connected

2016-06-23 Thread Jason Gustafson (JIRA)

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

Jason Gustafson commented on KAFKA-3822:


[~alex-cook4] I'm hoping we can fix the behavior of poll() separately so that 
it enforces the timeout passed by the user. I'm trying to deal with this 
piecemeal (e.g. see KAFKA-3834).

> Kafka Consumer close() hangs indefinitely if Kafka Broker shutdown while 
> connected
> --
>
> Key: KAFKA-3822
> URL: https://issues.apache.org/jira/browse/KAFKA-3822
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Affects Versions: 0.9.0.1, 0.10.0.0
> Environment: x86 Red Hat 6 (1 broker running zookeeper locally, 
> client running on a separate server)
>Reporter: Alexander Cook
>Assignee: Ashish K Singh
>
> I am using the KafkaConsumer java client to consume messages. My application 
> shuts down smoothly if I am connected to a Kafka broker, or if I never 
> succeed at connecting to a Kafka broker, but if the broker is shut down while 
> my consumer is connected to it, consumer.close() hangs indefinitely. 
> Here is how I reproduce it: 
> 1. Start 0.9.0.1 Kafka Broker
> 2. Start consumer application and consume messages
> 3. Stop 0.9.0.1 Kafka Broker (ctrl-c or stop script)
> 4. Try to stop application...hangs at consumer.close() indefinitely. 
> I also see this same behavior using 0.10 broker and client. 
> This is my first bug reported to Kafka, so please let me know if I should be 
> following a different format. Thanks! 



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


[jira] [Commented] (KAFKA-3822) Kafka Consumer close() hangs indefinitely if Kafka Broker shutdown while connected

2016-06-23 Thread Ashish K Singh (JIRA)

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

Ashish K Singh commented on KAFKA-3822:
---

Turns out KAFKA-3879 is a dup of this, had some discussion with [~hachikuji] on 
the JIRA regarding possible routes we can take here. I am more inclined towards 
having something like {{max.block.ms}}.

> Kafka Consumer close() hangs indefinitely if Kafka Broker shutdown while 
> connected
> --
>
> Key: KAFKA-3822
> URL: https://issues.apache.org/jira/browse/KAFKA-3822
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Affects Versions: 0.9.0.1, 0.10.0.0
> Environment: x86 Red Hat 6 (1 broker running zookeeper locally, 
> client running on a separate server)
>Reporter: Alexander Cook
>
> I am using the KafkaConsumer java client to consume messages. My application 
> shuts down smoothly if I am connected to a Kafka broker, or if I never 
> succeed at connecting to a Kafka broker, but if the broker is shut down while 
> my consumer is connected to it, consumer.close() hangs indefinitely. 
> Here is how I reproduce it: 
> 1. Start 0.9.0.1 Kafka Broker
> 2. Start consumer application and consume messages
> 3. Stop 0.9.0.1 Kafka Broker (ctrl-c or stop script)
> 4. Try to stop application...hangs at consumer.close() indefinitely. 
> I also see this same behavior using 0.10 broker and client. 
> This is my first bug reported to Kafka, so please let me know if I should be 
> following a different format. Thanks! 



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


[jira] [Commented] (KAFKA-3822) Kafka Consumer close() hangs indefinitely if Kafka Broker shutdown while connected

2016-06-15 Thread Alexander Cook (JIRA)

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

Alexander Cook commented on KAFKA-3822:
---

I got to try this out today, and you are correct. This only happens when 
enable.auto.commit=true. max.block.ms would be great. Would that cover 
consumer.poll as well? 

> Kafka Consumer close() hangs indefinitely if Kafka Broker shutdown while 
> connected
> --
>
> Key: KAFKA-3822
> URL: https://issues.apache.org/jira/browse/KAFKA-3822
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Affects Versions: 0.9.0.1, 0.10.0.0
> Environment: x86 Red Hat 6 (1 broker running zookeeper locally, 
> client running on a separate server)
>Reporter: Alexander Cook
>
> I am using the KafkaConsumer java client to consume messages. My application 
> shuts down smoothly if I am connected to a Kafka broker, or if I never 
> succeed at connecting to a Kafka broker, but if the broker is shut down while 
> my consumer is connected to it, consumer.close() hangs indefinitely. 
> Here is how I reproduce it: 
> 1. Start 0.9.0.1 Kafka Broker
> 2. Start consumer application and consume messages
> 3. Stop 0.9.0.1 Kafka Broker (ctrl-c or stop script)
> 4. Try to stop application...hangs at consumer.close() indefinitely. 
> I also see this same behavior using 0.10 broker and client. 
> This is my first bug reported to Kafka, so please let me know if I should be 
> following a different format. Thanks! 



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


[jira] [Commented] (KAFKA-3822) Kafka Consumer close() hangs indefinitely if Kafka Broker shutdown while connected

2016-06-14 Thread Jason Gustafson (JIRA)

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

Jason Gustafson commented on KAFKA-3822:


[~alex-cook4] Thanks for the report. Just to clarify, this only occurs when 
autocommit is enabled, right? What we've talked about doing previously is 
overloading close to let the user pass a timeout (similar to the producer). 
I've also been thinking of adding max.block.ms to the configuration (also like 
the producer) to bound the other blocking calls. We could potentially use that 
here. Probably either change will require a short KIP.

> Kafka Consumer close() hangs indefinitely if Kafka Broker shutdown while 
> connected
> --
>
> Key: KAFKA-3822
> URL: https://issues.apache.org/jira/browse/KAFKA-3822
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Affects Versions: 0.9.0.1, 0.10.0.0
> Environment: x86 Red Hat 6 (1 broker running zookeeper locally, 
> client running on a separate server)
>Reporter: Alexander Cook
>
> I am using the KafkaConsumer java client to consume messages. My application 
> shuts down smoothly if I am connected to a Kafka broker, or if I never 
> succeed at connecting to a Kafka broker, but if the broker is shut down while 
> my consumer is connected to it, consumer.close() hangs indefinitely. 
> Here is how I reproduce it: 
> 1. Start 0.9.0.1 Kafka Broker
> 2. Start consumer application and consume messages
> 3. Stop 0.9.0.1 Kafka Broker (ctrl-c or stop script)
> 4. Try to stop application...hangs at consumer.close() indefinitely. 
> I also see this same behavior using 0.10 broker and client. 
> This is my first bug reported to Kafka, so please let me know if I should be 
> following a different format. Thanks! 



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