[jira] [Commented] (KAFKA-8793) StickyTaskAssignor throws java.lang.ArithmeticException

2019-09-01 Thread Guozhang Wang (Jira)


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

Guozhang Wang commented on KAFKA-8793:
--

[~rocketraman] Yeah I think your reasoning is correct. If with the latest 
merged patch of KAFKA-8715 you no longer see the observed issue as it was quite 
frequently observable, please feel free to go ahead and close this issue.

> StickyTaskAssignor throws java.lang.ArithmeticException
> ---
>
> Key: KAFKA-8793
> URL: https://issues.apache.org/jira/browse/KAFKA-8793
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.3.0
>Reporter: Raman Gupta
>Assignee: Guozhang Wang
>Priority: Critical
>
> Occassionally when starting a streams consumer that uses the static consumer 
> group protocol, I get the following error:
> {code:java}
> 2019-08-13 06:06:43,527 ERROR --- [691d2-StreamThread-1] 
> org.apa.kaf.str.pro.int.StreamThread  : stream-thread 
> [prod-cisSegmenter-777489d8-6cc5-48b4-8771-868d873691d2-StreamThread-1] 
> Encountered the following er
> ror during processing:
> EXCEPTION: java.lang.ArithmeticException: / by zero
> at 
> org.apache.kafka.streams.processor.internals.assignment.StickyTaskAssignor.assignActive(StickyTaskAssignor.java:76)
>  ~[kafka-streams-2.3.0.jar:?]
> at 
> org.apache.kafka.streams.processor.internals.assignment.StickyTaskAssignor.assign(StickyTaskAssignor.java:52)
>  ~[kafka-streams-2.3.0.jar:?]
> at 
> org.apache.kafka.streams.processor.internals.StreamsPartitionAssignor.assign(StreamsPartitionAssignor.java:634)
>  ~[kafka-streams-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.performAssignment(ConsumerCoordinator.java:424)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.onJoinLeader(AbstractCoordinator.java:622)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.access$1100(AbstractCoordinator.java:107)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:544)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:527)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:978)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:958)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.RequestFuture$1.onSuccess(RequestFuture.java:204)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.RequestFuture.fireSuccess(RequestFuture.java:167)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.RequestFuture.complete(RequestFuture.java:127)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient$RequestFutureCompletionHandler.fireCompletion(ConsumerNetworkClient.java:578)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.firePendingCompletedRequests(ConsumerNetworkClient.java:388)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:294)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:233)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:212)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.joinGroupIfNeeded(AbstractCoordinator.java:415)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:358)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:353)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.KafkaConsumer.updateAssignmentMetadataIfNeeded(KafkaConsumer.java:1251)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1216) 
> ~[kafka-clients-2.3.0.jar:?]
> at 
> 

[jira] [Commented] (KAFKA-8793) StickyTaskAssignor throws java.lang.ArithmeticException

2019-08-30 Thread Raman Gupta (Jira)


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

Raman Gupta commented on KAFKA-8793:


[~guozhang] Unfortunately my logs have rolled over and I don't have this error 
recently. However, I will note that I applied an earlier patch for KAFKA-8715 
in which the patch used a timestamp and not a UUID. There was discussion on the 
pull request about this, and then it was changed to UUID, but I had already 
patched my broker, as the issue was causing me lots of trouble. So given your 
comments, a timestamp collision due to my older patch of KAFKA-8715 is probably 
the cause of this issue. There are 3 streams instances, and 2 threads per 
instance so there could easily have been a timestamp collision between each of 
the two threads connecting simultaneously to the broker. If that is the case, 
then I think it makes sense to close this.

> StickyTaskAssignor throws java.lang.ArithmeticException
> ---
>
> Key: KAFKA-8793
> URL: https://issues.apache.org/jira/browse/KAFKA-8793
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.3.0
>Reporter: Raman Gupta
>Assignee: Guozhang Wang
>Priority: Critical
>
> Occassionally when starting a streams consumer that uses the static consumer 
> group protocol, I get the following error:
> {code:java}
> 2019-08-13 06:06:43,527 ERROR --- [691d2-StreamThread-1] 
> org.apa.kaf.str.pro.int.StreamThread  : stream-thread 
> [prod-cisSegmenter-777489d8-6cc5-48b4-8771-868d873691d2-StreamThread-1] 
> Encountered the following er
> ror during processing:
> EXCEPTION: java.lang.ArithmeticException: / by zero
> at 
> org.apache.kafka.streams.processor.internals.assignment.StickyTaskAssignor.assignActive(StickyTaskAssignor.java:76)
>  ~[kafka-streams-2.3.0.jar:?]
> at 
> org.apache.kafka.streams.processor.internals.assignment.StickyTaskAssignor.assign(StickyTaskAssignor.java:52)
>  ~[kafka-streams-2.3.0.jar:?]
> at 
> org.apache.kafka.streams.processor.internals.StreamsPartitionAssignor.assign(StreamsPartitionAssignor.java:634)
>  ~[kafka-streams-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.performAssignment(ConsumerCoordinator.java:424)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.onJoinLeader(AbstractCoordinator.java:622)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.access$1100(AbstractCoordinator.java:107)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:544)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:527)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:978)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:958)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.RequestFuture$1.onSuccess(RequestFuture.java:204)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.RequestFuture.fireSuccess(RequestFuture.java:167)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.RequestFuture.complete(RequestFuture.java:127)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient$RequestFutureCompletionHandler.fireCompletion(ConsumerNetworkClient.java:578)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.firePendingCompletedRequests(ConsumerNetworkClient.java:388)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:294)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:233)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:212)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.joinGroupIfNeeded(AbstractCoordinator.java:415)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:358)
>  

[jira] [Commented] (KAFKA-8793) StickyTaskAssignor throws java.lang.ArithmeticException

2019-08-30 Thread Guozhang Wang (Jira)


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

Guozhang Wang commented on KAFKA-8793:
--

With KAFKA-8715, the old / new memberID should be different (UUID's conflict 
possibly is very, very small), however from the observed logic it seems that 
they are still the same, and hence causing the old leader id to still be 
recognized as the current leader id and hence triggers the `assign` function.

[~rocketraman] To verify if KAFKA-8715 is indeed patched, could you search for 
the following log entry along the time this issue happens on the broker side 
server logs:

```
Static member {...} with unknown member id rejoins, assigning new member id 
XXX, while old member YYY will be removed."
```

XXX and YYY above should be different, and if they are the same this may cause 
the observed issue. Could you verify? If yes could you also paste the values of 
XXX / YYY here?

> StickyTaskAssignor throws java.lang.ArithmeticException
> ---
>
> Key: KAFKA-8793
> URL: https://issues.apache.org/jira/browse/KAFKA-8793
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.3.0
>Reporter: Raman Gupta
>Assignee: Guozhang Wang
>Priority: Critical
>
> Occassionally when starting a streams consumer that uses the static consumer 
> group protocol, I get the following error:
> {code:java}
> 2019-08-13 06:06:43,527 ERROR --- [691d2-StreamThread-1] 
> org.apa.kaf.str.pro.int.StreamThread  : stream-thread 
> [prod-cisSegmenter-777489d8-6cc5-48b4-8771-868d873691d2-StreamThread-1] 
> Encountered the following er
> ror during processing:
> EXCEPTION: java.lang.ArithmeticException: / by zero
> at 
> org.apache.kafka.streams.processor.internals.assignment.StickyTaskAssignor.assignActive(StickyTaskAssignor.java:76)
>  ~[kafka-streams-2.3.0.jar:?]
> at 
> org.apache.kafka.streams.processor.internals.assignment.StickyTaskAssignor.assign(StickyTaskAssignor.java:52)
>  ~[kafka-streams-2.3.0.jar:?]
> at 
> org.apache.kafka.streams.processor.internals.StreamsPartitionAssignor.assign(StreamsPartitionAssignor.java:634)
>  ~[kafka-streams-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.performAssignment(ConsumerCoordinator.java:424)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.onJoinLeader(AbstractCoordinator.java:622)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.access$1100(AbstractCoordinator.java:107)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:544)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:527)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:978)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:958)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.RequestFuture$1.onSuccess(RequestFuture.java:204)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.RequestFuture.fireSuccess(RequestFuture.java:167)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.RequestFuture.complete(RequestFuture.java:127)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient$RequestFutureCompletionHandler.fireCompletion(ConsumerNetworkClient.java:578)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.firePendingCompletedRequests(ConsumerNetworkClient.java:388)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:294)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:233)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:212)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.joinGroupIfNeeded(AbstractCoordinator.java:415)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> 

[jira] [Commented] (KAFKA-8793) StickyTaskAssignor throws java.lang.ArithmeticException

2019-08-30 Thread Guozhang Wang (Jira)


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

Guozhang Wang commented on KAFKA-8793:
--

[~rocketraman] could you let me know how many streams instances (i.e. 
processes) are there that belongs to the same app and how many threads per 
instance is configured?

> StickyTaskAssignor throws java.lang.ArithmeticException
> ---
>
> Key: KAFKA-8793
> URL: https://issues.apache.org/jira/browse/KAFKA-8793
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.3.0
>Reporter: Raman Gupta
>Assignee: Guozhang Wang
>Priority: Critical
>
> Occassionally when starting a streams consumer that uses the static consumer 
> group protocol, I get the following error:
> {code:java}
> 2019-08-13 06:06:43,527 ERROR --- [691d2-StreamThread-1] 
> org.apa.kaf.str.pro.int.StreamThread  : stream-thread 
> [prod-cisSegmenter-777489d8-6cc5-48b4-8771-868d873691d2-StreamThread-1] 
> Encountered the following er
> ror during processing:
> EXCEPTION: java.lang.ArithmeticException: / by zero
> at 
> org.apache.kafka.streams.processor.internals.assignment.StickyTaskAssignor.assignActive(StickyTaskAssignor.java:76)
>  ~[kafka-streams-2.3.0.jar:?]
> at 
> org.apache.kafka.streams.processor.internals.assignment.StickyTaskAssignor.assign(StickyTaskAssignor.java:52)
>  ~[kafka-streams-2.3.0.jar:?]
> at 
> org.apache.kafka.streams.processor.internals.StreamsPartitionAssignor.assign(StreamsPartitionAssignor.java:634)
>  ~[kafka-streams-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.performAssignment(ConsumerCoordinator.java:424)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.onJoinLeader(AbstractCoordinator.java:622)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.access$1100(AbstractCoordinator.java:107)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:544)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:527)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:978)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:958)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.RequestFuture$1.onSuccess(RequestFuture.java:204)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.RequestFuture.fireSuccess(RequestFuture.java:167)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.RequestFuture.complete(RequestFuture.java:127)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient$RequestFutureCompletionHandler.fireCompletion(ConsumerNetworkClient.java:578)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.firePendingCompletedRequests(ConsumerNetworkClient.java:388)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:294)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:233)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:212)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.joinGroupIfNeeded(AbstractCoordinator.java:415)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:358)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:353)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.KafkaConsumer.updateAssignmentMetadataIfNeeded(KafkaConsumer.java:1251)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1216) 
> ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1201) 
> ~[kafka-clients-2.3.0.jar:?]
> at 
> 

[jira] [Commented] (KAFKA-8793) StickyTaskAssignor throws java.lang.ArithmeticException

2019-08-29 Thread Guozhang Wang (Jira)


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

Guozhang Wang commented on KAFKA-8793:
--

[~rocketraman] I think I found out the root cause of this issue, which is 
correlated to version probing such that if all clients are excluded as future 
consumers, the client involving in the actual assignment would be empty and 
hence the summed total capacity being zero. Will file a PR for fixing this 
issue asap.

> StickyTaskAssignor throws java.lang.ArithmeticException
> ---
>
> Key: KAFKA-8793
> URL: https://issues.apache.org/jira/browse/KAFKA-8793
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.3.0
>Reporter: Raman Gupta
>Assignee: Guozhang Wang
>Priority: Critical
>
> Occassionally when starting a streams consumer that uses the static consumer 
> group protocol, I get the following error:
> {code:java}
> 2019-08-13 06:06:43,527 ERROR --- [691d2-StreamThread-1] 
> org.apa.kaf.str.pro.int.StreamThread  : stream-thread 
> [prod-cisSegmenter-777489d8-6cc5-48b4-8771-868d873691d2-StreamThread-1] 
> Encountered the following er
> ror during processing:
> EXCEPTION: java.lang.ArithmeticException: / by zero
> at 
> org.apache.kafka.streams.processor.internals.assignment.StickyTaskAssignor.assignActive(StickyTaskAssignor.java:76)
>  ~[kafka-streams-2.3.0.jar:?]
> at 
> org.apache.kafka.streams.processor.internals.assignment.StickyTaskAssignor.assign(StickyTaskAssignor.java:52)
>  ~[kafka-streams-2.3.0.jar:?]
> at 
> org.apache.kafka.streams.processor.internals.StreamsPartitionAssignor.assign(StreamsPartitionAssignor.java:634)
>  ~[kafka-streams-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.performAssignment(ConsumerCoordinator.java:424)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.onJoinLeader(AbstractCoordinator.java:622)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.access$1100(AbstractCoordinator.java:107)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:544)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:527)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:978)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:958)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.RequestFuture$1.onSuccess(RequestFuture.java:204)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.RequestFuture.fireSuccess(RequestFuture.java:167)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.RequestFuture.complete(RequestFuture.java:127)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient$RequestFutureCompletionHandler.fireCompletion(ConsumerNetworkClient.java:578)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.firePendingCompletedRequests(ConsumerNetworkClient.java:388)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:294)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:233)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:212)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.joinGroupIfNeeded(AbstractCoordinator.java:415)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:358)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:353)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.KafkaConsumer.updateAssignmentMetadataIfNeeded(KafkaConsumer.java:1251)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1216) 
> 

[jira] [Commented] (KAFKA-8793) StickyTaskAssignor throws java.lang.ArithmeticException

2019-08-20 Thread Matthias J. Sax (Jira)


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

Matthias J. Sax commented on KAFKA-8793:


This is a weird issue. It seem that the computed capacity is zero – but I don't 
understand how this could happen, as the capacity should only be zero if we 
have zero threads. However, if there are zero thread there is no consumer 
group...? \cc [~guozhang] Any idea?

> StickyTaskAssignor throws java.lang.ArithmeticException
> ---
>
> Key: KAFKA-8793
> URL: https://issues.apache.org/jira/browse/KAFKA-8793
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.3.0
>Reporter: Raman Gupta
>Assignee: Bill Bejeck
>Priority: Critical
>
> Occassionally when starting a streams consumer that uses the static consumer 
> group protocol, I get the following error:
> {code:java}
> 2019-08-13 06:06:43,527 ERROR --- [691d2-StreamThread-1] 
> org.apa.kaf.str.pro.int.StreamThread  : stream-thread 
> [prod-cisSegmenter-777489d8-6cc5-48b4-8771-868d873691d2-StreamThread-1] 
> Encountered the following er
> ror during processing:
> EXCEPTION: java.lang.ArithmeticException: / by zero
> at 
> org.apache.kafka.streams.processor.internals.assignment.StickyTaskAssignor.assignActive(StickyTaskAssignor.java:76)
>  ~[kafka-streams-2.3.0.jar:?]
> at 
> org.apache.kafka.streams.processor.internals.assignment.StickyTaskAssignor.assign(StickyTaskAssignor.java:52)
>  ~[kafka-streams-2.3.0.jar:?]
> at 
> org.apache.kafka.streams.processor.internals.StreamsPartitionAssignor.assign(StreamsPartitionAssignor.java:634)
>  ~[kafka-streams-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.performAssignment(ConsumerCoordinator.java:424)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.onJoinLeader(AbstractCoordinator.java:622)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.access$1100(AbstractCoordinator.java:107)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:544)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:527)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:978)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:958)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.RequestFuture$1.onSuccess(RequestFuture.java:204)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.RequestFuture.fireSuccess(RequestFuture.java:167)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.RequestFuture.complete(RequestFuture.java:127)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient$RequestFutureCompletionHandler.fireCompletion(ConsumerNetworkClient.java:578)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.firePendingCompletedRequests(ConsumerNetworkClient.java:388)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:294)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:233)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:212)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.joinGroupIfNeeded(AbstractCoordinator.java:415)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:358)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:353)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.KafkaConsumer.updateAssignmentMetadataIfNeeded(KafkaConsumer.java:1251)
>  ~[kafka-clients-2.3.0.jar:?]
> at 
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1216) 
> ~[kafka-clients-2.3.0.jar:?]
> at 
>