[jira] [Commented] (STORM-2494) KafkaSpout does not handle CommitFailedException
[ https://issues.apache.org/jira/browse/STORM-2494?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16129179#comment-16129179 ] Stig Rohde Døssing commented on STORM-2494: --- [~hmclouro] Okay, I just wanted to make sure you didn't spend time debugging to find out why this happens :) Regarding fixing this, I think it might not be easy. We'd need to make sure that the spout always calls poll before commit in nextTuple, so it can trigger rebalance in case the spout has been idle for longer than the session timeout, and we'd probably also need to change the rebalance listener logic. It currently assumes that when onPartitionsRevoked runs, the spout was assigned the partitions it has offset managers for. Users who don't want to upgrade storm-kafka-client for some reason can still work around this by setting a higher session timeout, and newer Kafka versions allow you to set it pretty high without losing fast detection of dead consumers due to KIP-62. > KafkaSpout does not handle CommitFailedException > > > Key: STORM-2494 > URL: https://issues.apache.org/jira/browse/STORM-2494 > Project: Apache Storm > Issue Type: Bug > Components: storm-kafka-client >Affects Versions: 1.1.0 >Reporter: Yuri Barseghyan >Assignee: Hugo Louro > > In situations when tuple processing takes longer than session timeout, we get > CommitFailedException and instead of recovering from it Storm worker dies. > {code} > 2017-04-26 11:07:04.902 o.a.s.util [ERROR] Async loop died! > org.apache.kafka.clients.consumer.CommitFailedException: Commit cannot be > completed since the group has already rebalanced and assigned the partitions > to another member. This means that the time between subsequent calls to > poll() was longer than the configured session.timeout.ms, which typically > implies that the poll loop is spending too much time message processing. You > can address this either by increasing the session timeout or by reducing the > maximum size of batches returned in poll() with max.poll.records. > \tat > org.apache.kafka.clients.consumer.internals.ConsumerCoordinator$OffsetCommitResponseHandler.handle(ConsumerCoordinator.java:578) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.ConsumerCoordinator$OffsetCommitResponseHandler.handle(ConsumerCoordinator.java:519) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:679) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:658) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.RequestFuture$1.onSuccess(RequestFuture.java:167) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.RequestFuture.fireSuccess(RequestFuture.java:133) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.RequestFuture.complete(RequestFuture.java:107) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient$RequestFutureCompletionHandler.onComplete(ConsumerNetworkClient.java:426) > ~[stormjar.jar:3.0.2] > \tat org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:278) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.clientPoll(ConsumerNetworkClient.java:360) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:224) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:192) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:163) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.commitOffsetsSync(ConsumerCoordinator.java:404) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.KafkaConsumer.commitSync(KafkaConsumer.java:1058) > ~[stormjar.jar:3.0.2] > \tat > org.apache.storm.kafka.spout.KafkaSpout.commitOffsetsForAckedTuples(KafkaSpout.java:384) > ~[stormjar.jar:3.0.2] > \tat org.apache.storm.kafka.spout.KafkaSpout.nextTuple(KafkaSpout.java:219) > ~[stormjar.jar:3.0.2] > \tat > org.apache.storm.daemon.executor$fn__4976$fn__4991$fn__5022.invoke(executor.clj:644) > ~[storm-core-1.1.0.jar:1.1.0] > \tat org.apache.storm.util$async_loop$fn__557.invoke(util.clj:484) > [storm-core-1.1.0.jar:1.1.0] > \tat clojure.lang.AFn.run(AFn.java:22) [clojure-1.7.0.jar:?] > \tat java.lang.Thread.run(Thread.java:748) [?:1.8.0_131] > 2017-04-26 11:07:04.909 o.a.s.d.executor
[jira] [Commented] (STORM-2494) KafkaSpout does not handle CommitFailedException
[ https://issues.apache.org/jira/browse/STORM-2494?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16128979#comment-16128979 ] Hugo Louro commented on STORM-2494: --- [~Srdo] I understand that. The only reason I decided to put some thought into a possible fix is to cover the scenario where a customer that is in Storm 1.1.0 does not want to upgrade to 1.2.0. But my idea all along is to advise to move manual partition assignment (i.e. upgrade to Storm 1.2.0). > KafkaSpout does not handle CommitFailedException > > > Key: STORM-2494 > URL: https://issues.apache.org/jira/browse/STORM-2494 > Project: Apache Storm > Issue Type: Bug > Components: storm-kafka-client >Affects Versions: 1.1.0 >Reporter: Yuri Barseghyan >Assignee: Hugo Louro > > In situations when tuple processing takes longer than session timeout, we get > CommitFailedException and instead of recovering from it Storm worker dies. > {code} > 2017-04-26 11:07:04.902 o.a.s.util [ERROR] Async loop died! > org.apache.kafka.clients.consumer.CommitFailedException: Commit cannot be > completed since the group has already rebalanced and assigned the partitions > to another member. This means that the time between subsequent calls to > poll() was longer than the configured session.timeout.ms, which typically > implies that the poll loop is spending too much time message processing. You > can address this either by increasing the session timeout or by reducing the > maximum size of batches returned in poll() with max.poll.records. > \tat > org.apache.kafka.clients.consumer.internals.ConsumerCoordinator$OffsetCommitResponseHandler.handle(ConsumerCoordinator.java:578) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.ConsumerCoordinator$OffsetCommitResponseHandler.handle(ConsumerCoordinator.java:519) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:679) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:658) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.RequestFuture$1.onSuccess(RequestFuture.java:167) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.RequestFuture.fireSuccess(RequestFuture.java:133) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.RequestFuture.complete(RequestFuture.java:107) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient$RequestFutureCompletionHandler.onComplete(ConsumerNetworkClient.java:426) > ~[stormjar.jar:3.0.2] > \tat org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:278) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.clientPoll(ConsumerNetworkClient.java:360) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:224) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:192) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:163) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.commitOffsetsSync(ConsumerCoordinator.java:404) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.KafkaConsumer.commitSync(KafkaConsumer.java:1058) > ~[stormjar.jar:3.0.2] > \tat > org.apache.storm.kafka.spout.KafkaSpout.commitOffsetsForAckedTuples(KafkaSpout.java:384) > ~[stormjar.jar:3.0.2] > \tat org.apache.storm.kafka.spout.KafkaSpout.nextTuple(KafkaSpout.java:219) > ~[stormjar.jar:3.0.2] > \tat > org.apache.storm.daemon.executor$fn__4976$fn__4991$fn__5022.invoke(executor.clj:644) > ~[storm-core-1.1.0.jar:1.1.0] > \tat org.apache.storm.util$async_loop$fn__557.invoke(util.clj:484) > [storm-core-1.1.0.jar:1.1.0] > \tat clojure.lang.AFn.run(AFn.java:22) [clojure-1.7.0.jar:?] > \tat java.lang.Thread.run(Thread.java:748) [?:1.8.0_131] > 2017-04-26 11:07:04.909 o.a.s.d.executor [ERROR] > org.apache.kafka.clients.consumer.CommitFailedException: Commit cannot be > completed since the group has already rebalanced and assigned the partitions > to another member. This means that the time between subsequent calls to > poll() was longer than the configured session.timeout.ms, which typically > implies that the poll loop is spending too much time message processing. You > can address this either by increasing the session timeout or by reducing the > maximum size of batches returned in
[jira] [Commented] (STORM-2494) KafkaSpout does not handle CommitFailedException
[ https://issues.apache.org/jira/browse/STORM-2494?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16128296#comment-16128296 ] Stig Rohde Døssing commented on STORM-2494: --- [~hmclouro] This is caused by using the subscribe API. I don't think we should try to make the code work around this. Upgrading to 1.2.0 for https://issues.apache.org/jira/browse/STORM-2640 will solve this. > KafkaSpout does not handle CommitFailedException > > > Key: STORM-2494 > URL: https://issues.apache.org/jira/browse/STORM-2494 > Project: Apache Storm > Issue Type: Bug > Components: storm-kafka-client >Affects Versions: 1.1.0 >Reporter: Yuri Barseghyan >Assignee: Hugo Louro > > In situations when tuple processing takes longer than session timeout, we get > CommitFailedException and instead of recovering from it Storm worker dies. > {code} > 2017-04-26 11:07:04.902 o.a.s.util [ERROR] Async loop died! > org.apache.kafka.clients.consumer.CommitFailedException: Commit cannot be > completed since the group has already rebalanced and assigned the partitions > to another member. This means that the time between subsequent calls to > poll() was longer than the configured session.timeout.ms, which typically > implies that the poll loop is spending too much time message processing. You > can address this either by increasing the session timeout or by reducing the > maximum size of batches returned in poll() with max.poll.records. > \tat > org.apache.kafka.clients.consumer.internals.ConsumerCoordinator$OffsetCommitResponseHandler.handle(ConsumerCoordinator.java:578) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.ConsumerCoordinator$OffsetCommitResponseHandler.handle(ConsumerCoordinator.java:519) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:679) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:658) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.RequestFuture$1.onSuccess(RequestFuture.java:167) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.RequestFuture.fireSuccess(RequestFuture.java:133) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.RequestFuture.complete(RequestFuture.java:107) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient$RequestFutureCompletionHandler.onComplete(ConsumerNetworkClient.java:426) > ~[stormjar.jar:3.0.2] > \tat org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:278) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.clientPoll(ConsumerNetworkClient.java:360) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:224) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:192) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:163) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.commitOffsetsSync(ConsumerCoordinator.java:404) > ~[stormjar.jar:3.0.2] > \tat > org.apache.kafka.clients.consumer.KafkaConsumer.commitSync(KafkaConsumer.java:1058) > ~[stormjar.jar:3.0.2] > \tat > org.apache.storm.kafka.spout.KafkaSpout.commitOffsetsForAckedTuples(KafkaSpout.java:384) > ~[stormjar.jar:3.0.2] > \tat org.apache.storm.kafka.spout.KafkaSpout.nextTuple(KafkaSpout.java:219) > ~[stormjar.jar:3.0.2] > \tat > org.apache.storm.daemon.executor$fn__4976$fn__4991$fn__5022.invoke(executor.clj:644) > ~[storm-core-1.1.0.jar:1.1.0] > \tat org.apache.storm.util$async_loop$fn__557.invoke(util.clj:484) > [storm-core-1.1.0.jar:1.1.0] > \tat clojure.lang.AFn.run(AFn.java:22) [clojure-1.7.0.jar:?] > \tat java.lang.Thread.run(Thread.java:748) [?:1.8.0_131] > 2017-04-26 11:07:04.909 o.a.s.d.executor [ERROR] > org.apache.kafka.clients.consumer.CommitFailedException: Commit cannot be > completed since the group has already rebalanced and assigned the partitions > to another member. This means that the time between subsequent calls to > poll() was longer than the configured session.timeout.ms, which typically > implies that the poll loop is spending too much time message processing. You > can address this either by increasing the session timeout or by reducing the > maximum size of batches returned in poll() with max.poll.records. > \tat >