[jira] [Commented] (KAFKA-10772) java.lang.IllegalStateException: There are insufficient bytes available to read assignment from the sync-group response (actual byte size 0)

2020-12-08 Thread Bradley Peterson (Jira)


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

Bradley Peterson commented on KAFKA-10772:
--

[~cadonna] no, our brokers are on 2.4.1.

> java.lang.IllegalStateException: There are insufficient bytes available to 
> read assignment from the sync-group response (actual byte size 0)
> 
>
> Key: KAFKA-10772
> URL: https://issues.apache.org/jira/browse/KAFKA-10772
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.6.0
>Reporter: Levani Kokhreidze
>Priority: Blocker
> Attachments: KAFKA-10772.log
>
>
> From time to time we encounter the following exception that results in Kafka 
> Streams threads dying.
> Broker version 2.4.1, Client version 2.6.0
> {code:java}
> Nov 27 00:59:53.681 streaming-app service: prod | streaming-app-2 | 
> stream-client [cluster1-profile-stats-pipeline-client-id] State transition 
> from REBALANCING to ERROR Nov 27 00:59:53.681 streaming-app service: prod | 
> streaming-app-2 | stream-client [cluster1-profile-stats-pipeline-client-id] 
> State transition from REBALANCING to ERROR Nov 27 00:59:53.682 streaming-app 
> service: prod | streaming-app-2 | 2020-11-27 00:59:53.681 ERROR 105 --- 
> [-StreamThread-1] .KafkaStreamsBasedStreamProcessingEngine : Stream 
> processing pipeline: [profile-stats] encountered unrecoverable exception. 
> Thread: [cluster1-profile-stats-pipeline-client-id-StreamThread-1] is 
> completely dead. If all worker threads die, Kafka Streams will be moved to 
> permanent ERROR state. Nov 27 00:59:53.682 streaming-app service: prod | 
> streaming-app-2 | Stream processing pipeline: [profile-stats] encountered 
> unrecoverable exception. Thread: 
> [cluster1-profile-stats-pipeline-client-id-StreamThread-1] is completely 
> dead. If all worker threads die, Kafka Streams will be moved to permanent 
> ERROR state. java.lang.IllegalStateException: There are insufficient bytes 
> available to read assignment from the sync-group response (actual byte size 
> 0) , this is not expected; it is possible that the leader's assign function 
> is buggy and did not return any assignment for this member, or because static 
> member is configured and the protocol is buggy hence did not get the 
> assignment for this member at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.onJoinComplete(ConsumerCoordinator.java:367)
>  at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.joinGroupIfNeeded(AbstractCoordinator.java:440)
>  at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:359)
>  at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:513)
>  at 
> org.apache.kafka.clients.consumer.KafkaConsumer.updateAssignmentMetadataIfNeeded(KafkaConsumer.java:1268)
>  at 
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1230) 
> at 
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1210) 
> at 
> org.apache.kafka.streams.processor.internals.StreamThread.pollRequests(StreamThread.java:766)
>  at 
> org.apache.kafka.streams.processor.internals.StreamThread.runOnce(StreamThread.java:624)
>  at 
> org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:551)
>  at 
> org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:510)
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-10772) java.lang.IllegalStateException: There are insufficient bytes available to read assignment from the sync-group response (actual byte size 0)

2020-12-04 Thread Bradley Peterson (Jira)


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

Bradley Peterson commented on KAFKA-10772:
--

Hi Sophie, just fyi we're seeing the same error, also with static group 
membership, but we are running a 2.6.1 built from 
[1cbc4da|https://github.com/apache/kafka/commit/1cbc4da0c9d19b25ffeb04cb2b52d827fbe38684].
 Meaning we have the fix for the [KAFKA-10284].

> java.lang.IllegalStateException: There are insufficient bytes available to 
> read assignment from the sync-group response (actual byte size 0)
> 
>
> Key: KAFKA-10772
> URL: https://issues.apache.org/jira/browse/KAFKA-10772
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.6.0
>Reporter: Levani Kokhreidze
>Priority: Major
> Attachments: KAFKA-10772.log
>
>
> From time to time we encounter the following exception that results in Kafka 
> Streams threads dying.
> Broker version 2.4.1, Client version 2.6.0
> {code:java}
> Nov 27 00:59:53.681 streaming-app service: prod | streaming-app-2 | 
> stream-client [cluster1-profile-stats-pipeline-client-id] State transition 
> from REBALANCING to ERROR Nov 27 00:59:53.681 streaming-app service: prod | 
> streaming-app-2 | stream-client [cluster1-profile-stats-pipeline-client-id] 
> State transition from REBALANCING to ERROR Nov 27 00:59:53.682 streaming-app 
> service: prod | streaming-app-2 | 2020-11-27 00:59:53.681 ERROR 105 --- 
> [-StreamThread-1] .KafkaStreamsBasedStreamProcessingEngine : Stream 
> processing pipeline: [profile-stats] encountered unrecoverable exception. 
> Thread: [cluster1-profile-stats-pipeline-client-id-StreamThread-1] is 
> completely dead. If all worker threads die, Kafka Streams will be moved to 
> permanent ERROR state. Nov 27 00:59:53.682 streaming-app service: prod | 
> streaming-app-2 | Stream processing pipeline: [profile-stats] encountered 
> unrecoverable exception. Thread: 
> [cluster1-profile-stats-pipeline-client-id-StreamThread-1] is completely 
> dead. If all worker threads die, Kafka Streams will be moved to permanent 
> ERROR state. java.lang.IllegalStateException: There are insufficient bytes 
> available to read assignment from the sync-group response (actual byte size 
> 0) , this is not expected; it is possible that the leader's assign function 
> is buggy and did not return any assignment for this member, or because static 
> member is configured and the protocol is buggy hence did not get the 
> assignment for this member at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.onJoinComplete(ConsumerCoordinator.java:367)
>  at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.joinGroupIfNeeded(AbstractCoordinator.java:440)
>  at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:359)
>  at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:513)
>  at 
> org.apache.kafka.clients.consumer.KafkaConsumer.updateAssignmentMetadataIfNeeded(KafkaConsumer.java:1268)
>  at 
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1230) 
> at 
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1210) 
> at 
> org.apache.kafka.streams.processor.internals.StreamThread.pollRequests(StreamThread.java:766)
>  at 
> org.apache.kafka.streams.processor.internals.StreamThread.runOnce(StreamThread.java:624)
>  at 
> org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:551)
>  at 
> org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:510)
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-10678) Re-deploying Streams app causes rebalance and task migration

2020-11-06 Thread Bradley Peterson (Jira)


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

Bradley Peterson commented on KAFKA-10678:
--

Sophie, thank you again for your thorough answers. Our logs are busy, but I was 
able to find the line you described, "Requested to schedule immediate rebalance 
to update group with new host endpoint = ...". I said we don't use IQ, but we 
used to, and neglected to remove the {{application.server}} configuration. So, 
that explains question 1).

We may want to use IQ again (we removed it while we waited on the rebalance 
changes and [KAFKA-9568]). So, I'd still like to fix question 2). I think I 
understand what you're saying about [KAFKA-10121]. The current assignment, 
after bouncing, is based on the caught-up state stores, but there is an 
"eventual" assignment which is different. Is it then true that bouncing a 
single instance would cause it to have a new processId and potentially cause a 
different eventual assignment? That's unfortunate, because we have instances 
come and go all the time, and task migration of large state stores is 
noticeable to us, due to the increased network transfer and disk usage.

For now, I think removing {{application.server}} should be sufficient for us.

> Re-deploying Streams app causes rebalance and task migration
> 
>
> Key: KAFKA-10678
> URL: https://issues.apache.org/jira/browse/KAFKA-10678
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.6.0, 2.6.1
>Reporter: Bradley Peterson
>Priority: Major
> Attachments: after, before, broker
>
>
> Re-deploying our Streams app causes a rebalance, even when using static group 
> membership. Worse, the rebalance creates standby tasks, even when the 
> previous task assignment was balanced and stable.
> Our app is currently using Streams 2.6.1-SNAPSHOT (due to [KAFKA-10633]) but 
> we saw the same behavior in 2.6.0. The app runs on 4 EC2 instances, each with 
> 4 streams threads, and data stored on persistent EBS volumes.. During a 
> redeploy, all EC2 instances are stopped, new instances are launched, and the 
> EBS volumes are attached to the new instances. We do not use interactive 
> queries. {{session.timeout.ms}} is set to 30 minutes, and the deployment 
> finishes well under that. {{num.standby.replicas}} is 0.
> h2. Expected Behavior
> Given a stable and balanced task assignment prior to deploying, we expect to 
> see the same task assignment after deploying. Even if a rebalance is 
> triggered, we do not expect to see new standby tasks.
> h2. Observed Behavior
> Attached are the "Assigned tasks to clients" log lines from before and after 
> deploying. The "before" is from over 24 hours ago, the task assignment is 
> well balanced and "Finished stable assignment of tasks, no followup 
> rebalances required." is logged. The "after" log lines show the same 
> assignment of active tasks, but some additional standby tasks. There are 
> additional log lines about adding and removing active tasks, which I don't 
> quite understand.
> I've also included logs from the broker showing the rebalance was triggered 
> for "Updating metadata".



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Resolved] (KAFKA-10633) Constant probing rebalances in Streams 2.6

2020-11-03 Thread Bradley Peterson (Jira)


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

Bradley Peterson resolved KAFKA-10633.
--
Fix Version/s: 2.6.1
   Resolution: Fixed

Closing this ticket as the issue is fixed in 2.6.1. I've opened [KAFKA-10678] 
for our other problem with unexpected rebalancing.

> Constant probing rebalances in Streams 2.6
> --
>
> Key: KAFKA-10633
> URL: https://issues.apache.org/jira/browse/KAFKA-10633
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.6.0
>Reporter: Bradley Peterson
>Priority: Major
> Fix For: 2.6.1
>
> Attachments: Discover 2020-10-21T23 34 03.867Z - 2020-10-21T23 44 
> 46.409Z.csv
>
>
> We are seeing a few issues with the new rebalancing behavior in Streams 2.6. 
> This ticket is for constant probing rebalances on one StreamThread, but I'll 
> mention the other issues, as they may be related.
> First, when we redeploy the application we see tasks being moved, even though 
> the task assignment was stable before redeploying. We would expect to see 
> tasks assigned back to the same instances and no movement. The application is 
> in EC2, with persistent EBS volumes, and we use static group membership to 
> avoid rebalancing. To redeploy the app we terminate all EC2 instances. The 
> new instances will reattach the EBS volumes and use the same group member id.
> After redeploying, we sometimes see the group leader go into a tight probing 
> rebalance loop. This doesn't happen immediately, it could be several hours 
> later. Because the redeploy caused task movement, we see expected probing 
> rebalances every 10 minutes. But, then one thread will go into a tight loop 
> logging messages like "Triggering the followup rebalance scheduled for 
> 1603323868771 ms.", handling the partition assignment (which doesn't change), 
> then "Requested to schedule probing rebalance for 1603323868771 ms." This 
> repeats several times a second until the app is restarted again. I'll attach 
> a log export from one such incident.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (KAFKA-10678) Re-deploying Streams app causes rebalance and task migration

2020-11-03 Thread Bradley Peterson (Jira)
Bradley Peterson created KAFKA-10678:


 Summary: Re-deploying Streams app causes rebalance and task 
migration
 Key: KAFKA-10678
 URL: https://issues.apache.org/jira/browse/KAFKA-10678
 Project: Kafka
  Issue Type: Bug
  Components: streams
Affects Versions: 2.6.0, 2.6.1
Reporter: Bradley Peterson
 Attachments: after, before, broker

Re-deploying our Streams app causes a rebalance, even when using static group 
membership. Worse, the rebalance creates standby tasks, even when the previous 
task assignment was balanced and stable.

Our app is currently using Streams 2.6.1-SNAPSHOT (due to [KAFKA-10633]) but we 
saw the same behavior in 2.6.0. The app runs on 4 EC2 instances, each with 4 
streams threads, and data stored on persistent EBS volumes.. During a redeploy, 
all EC2 instances are stopped, new instances are launched, and the EBS volumes 
are attached to the new instances. We do not use interactive queries. 
{{session.timeout.ms}} is set to 30 minutes, and the deployment finishes well 
under that. {{num.standby.replicas}} is 0.

h2. Expected Behavior
Given a stable and balanced task assignment prior to deploying, we expect to 
see the same task assignment after deploying. Even if a rebalance is triggered, 
we do not expect to see new standby tasks.

h2. Observed Behavior
Attached are the "Assigned tasks to clients" log lines from before and after 
deploying. The "before" is from over 24 hours ago, the task assignment is well 
balanced and "Finished stable assignment of tasks, no followup rebalances 
required." is logged. The "after" log lines show the same assignment of active 
tasks, but some additional standby tasks. There are additional log lines about 
adding and removing active tasks, which I don't quite understand.

I've also included logs from the broker showing the rebalance was triggered for 
"Updating metadata".



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Updated] (KAFKA-10678) Re-deploying Streams app causes rebalance and task migration

2020-11-03 Thread Bradley Peterson (Jira)


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

Bradley Peterson updated KAFKA-10678:
-
Attachment: broker
before
after

> Re-deploying Streams app causes rebalance and task migration
> 
>
> Key: KAFKA-10678
> URL: https://issues.apache.org/jira/browse/KAFKA-10678
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.6.0, 2.6.1
>Reporter: Bradley Peterson
>Priority: Major
> Attachments: after, before, broker
>
>
> Re-deploying our Streams app causes a rebalance, even when using static group 
> membership. Worse, the rebalance creates standby tasks, even when the 
> previous task assignment was balanced and stable.
> Our app is currently using Streams 2.6.1-SNAPSHOT (due to [KAFKA-10633]) but 
> we saw the same behavior in 2.6.0. The app runs on 4 EC2 instances, each with 
> 4 streams threads, and data stored on persistent EBS volumes.. During a 
> redeploy, all EC2 instances are stopped, new instances are launched, and the 
> EBS volumes are attached to the new instances. We do not use interactive 
> queries. {{session.timeout.ms}} is set to 30 minutes, and the deployment 
> finishes well under that. {{num.standby.replicas}} is 0.
> h2. Expected Behavior
> Given a stable and balanced task assignment prior to deploying, we expect to 
> see the same task assignment after deploying. Even if a rebalance is 
> triggered, we do not expect to see new standby tasks.
> h2. Observed Behavior
> Attached are the "Assigned tasks to clients" log lines from before and after 
> deploying. The "before" is from over 24 hours ago, the task assignment is 
> well balanced and "Finished stable assignment of tasks, no followup 
> rebalances required." is logged. The "after" log lines show the same 
> assignment of active tasks, but some additional standby tasks. There are 
> additional log lines about adding and removing active tasks, which I don't 
> quite understand.
> I've also included logs from the broker showing the rebalance was triggered 
> for "Updating metadata".



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-10633) Constant probing rebalances in Streams 2.6

2020-11-01 Thread Bradley Peterson (Jira)


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

Bradley Peterson commented on KAFKA-10633:
--

[~eran-levy] The fix did seem to work for us, we've been running it for a few 
days. We ended up using the latest commit for 2.6.1-SNAPSHOT. We just built it 
locally from https://github.com/apache/kafka/tree/2.6. As you said, 2.6.1 is 
not released, and doesn't have a planned release date yet.

> Constant probing rebalances in Streams 2.6
> --
>
> Key: KAFKA-10633
> URL: https://issues.apache.org/jira/browse/KAFKA-10633
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.6.0
>Reporter: Bradley Peterson
>Priority: Major
> Attachments: Discover 2020-10-21T23 34 03.867Z - 2020-10-21T23 44 
> 46.409Z.csv
>
>
> We are seeing a few issues with the new rebalancing behavior in Streams 2.6. 
> This ticket is for constant probing rebalances on one StreamThread, but I'll 
> mention the other issues, as they may be related.
> First, when we redeploy the application we see tasks being moved, even though 
> the task assignment was stable before redeploying. We would expect to see 
> tasks assigned back to the same instances and no movement. The application is 
> in EC2, with persistent EBS volumes, and we use static group membership to 
> avoid rebalancing. To redeploy the app we terminate all EC2 instances. The 
> new instances will reattach the EBS volumes and use the same group member id.
> After redeploying, we sometimes see the group leader go into a tight probing 
> rebalance loop. This doesn't happen immediately, it could be several hours 
> later. Because the redeploy caused task movement, we see expected probing 
> rebalances every 10 minutes. But, then one thread will go into a tight loop 
> logging messages like "Triggering the followup rebalance scheduled for 
> 1603323868771 ms.", handling the partition assignment (which doesn't change), 
> then "Requested to schedule probing rebalance for 1603323868771 ms." This 
> repeats several times a second until the app is restarted again. I'll attach 
> a log export from one such incident.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-10633) Constant probing rebalances in Streams 2.6

2020-10-28 Thread Bradley Peterson (Jira)


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

Bradley Peterson commented on KAFKA-10633:
--

It's possible that the task assignment has never been balanced, and that this 
issue was hiding that fact. After deploying the fix for [KAFKA-10455], I 
haven't seen this issue recur, but I have seen steady task movement. So, it may 
be that will have to wait a while before tasks are balanced.

> Constant probing rebalances in Streams 2.6
> --
>
> Key: KAFKA-10633
> URL: https://issues.apache.org/jira/browse/KAFKA-10633
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.6.0
>Reporter: Bradley Peterson
>Priority: Major
> Attachments: Discover 2020-10-21T23 34 03.867Z - 2020-10-21T23 44 
> 46.409Z.csv
>
>
> We are seeing a few issues with the new rebalancing behavior in Streams 2.6. 
> This ticket is for constant probing rebalances on one StreamThread, but I'll 
> mention the other issues, as they may be related.
> First, when we redeploy the application we see tasks being moved, even though 
> the task assignment was stable before redeploying. We would expect to see 
> tasks assigned back to the same instances and no movement. The application is 
> in EC2, with persistent EBS volumes, and we use static group membership to 
> avoid rebalancing. To redeploy the app we terminate all EC2 instances. The 
> new instances will reattach the EBS volumes and use the same group member id.
> After redeploying, we sometimes see the group leader go into a tight probing 
> rebalance loop. This doesn't happen immediately, it could be several hours 
> later. Because the redeploy caused task movement, we see expected probing 
> rebalances every 10 minutes. But, then one thread will go into a tight loop 
> logging messages like "Triggering the followup rebalance scheduled for 
> 1603323868771 ms.", handling the partition assignment (which doesn't change), 
> then "Requested to schedule probing rebalance for 1603323868771 ms." This 
> repeats several times a second until the app is restarted again. I'll attach 
> a log export from one such incident.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-10633) Constant probing rebalances in Streams 2.6

2020-10-23 Thread Bradley Peterson (Jira)


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

Bradley Peterson commented on KAFKA-10633:
--

Sophie (et al.), do you have any thoughts about what would cause rebalances 
(and task movements) after redeploying? If we could fix that, then this would 
be less of a problem, because we would rarely have probing rebalances. We do 
have another problem where state directories are not deleted, but their 
contents are (almost like KAFKA-6647, but not quite the same). Is it possible 
that is confusing the task assignor?

> Constant probing rebalances in Streams 2.6
> --
>
> Key: KAFKA-10633
> URL: https://issues.apache.org/jira/browse/KAFKA-10633
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.6.0
>Reporter: Bradley Peterson
>Priority: Major
> Attachments: Discover 2020-10-21T23 34 03.867Z - 2020-10-21T23 44 
> 46.409Z.csv
>
>
> We are seeing a few issues with the new rebalancing behavior in Streams 2.6. 
> This ticket is for constant probing rebalances on one StreamThread, but I'll 
> mention the other issues, as they may be related.
> First, when we redeploy the application we see tasks being moved, even though 
> the task assignment was stable before redeploying. We would expect to see 
> tasks assigned back to the same instances and no movement. The application is 
> in EC2, with persistent EBS volumes, and we use static group membership to 
> avoid rebalancing. To redeploy the app we terminate all EC2 instances. The 
> new instances will reattach the EBS volumes and use the same group member id.
> After redeploying, we sometimes see the group leader go into a tight probing 
> rebalance loop. This doesn't happen immediately, it could be several hours 
> later. Because the redeploy caused task movement, we see expected probing 
> rebalances every 10 minutes. But, then one thread will go into a tight loop 
> logging messages like "Triggering the followup rebalance scheduled for 
> 1603323868771 ms.", handling the partition assignment (which doesn't change), 
> then "Requested to schedule probing rebalance for 1603323868771 ms." This 
> repeats several times a second until the app is restarted again. I'll attach 
> a log export from one such incident.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-10633) Constant probing rebalances in Streams 2.6

2020-10-23 Thread Bradley Peterson (Jira)


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

Bradley Peterson commented on KAFKA-10633:
--

Thank you, Sophie! I'll do a build with the PR for KAFKA-10455 and confirm that 
it fixes this issue.

> Constant probing rebalances in Streams 2.6
> --
>
> Key: KAFKA-10633
> URL: https://issues.apache.org/jira/browse/KAFKA-10633
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.6.0
>Reporter: Bradley Peterson
>Priority: Major
> Attachments: Discover 2020-10-21T23 34 03.867Z - 2020-10-21T23 44 
> 46.409Z.csv
>
>
> We are seeing a few issues with the new rebalancing behavior in Streams 2.6. 
> This ticket is for constant probing rebalances on one StreamThread, but I'll 
> mention the other issues, as they may be related.
> First, when we redeploy the application we see tasks being moved, even though 
> the task assignment was stable before redeploying. We would expect to see 
> tasks assigned back to the same instances and no movement. The application is 
> in EC2, with persistent EBS volumes, and we use static group membership to 
> avoid rebalancing. To redeploy the app we terminate all EC2 instances. The 
> new instances will reattach the EBS volumes and use the same group member id.
> After redeploying, we sometimes see the group leader go into a tight probing 
> rebalance loop. This doesn't happen immediately, it could be several hours 
> later. Because the redeploy caused task movement, we see expected probing 
> rebalances every 10 minutes. But, then one thread will go into a tight loop 
> logging messages like "Triggering the followup rebalance scheduled for 
> 1603323868771 ms.", handling the partition assignment (which doesn't change), 
> then "Requested to schedule probing rebalance for 1603323868771 ms." This 
> repeats several times a second until the app is restarted again. I'll attach 
> a log export from one such incident.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-10633) Constant probing rebalances in Streams 2.6

2020-10-22 Thread Bradley Peterson (Jira)


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

Bradley Peterson commented on KAFKA-10633:
--

/cc [~vvcephei]

> Constant probing rebalances in Streams 2.6
> --
>
> Key: KAFKA-10633
> URL: https://issues.apache.org/jira/browse/KAFKA-10633
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.6.0
>Reporter: Bradley Peterson
>Priority: Major
> Attachments: Discover 2020-10-21T23 34 03.867Z - 2020-10-21T23 44 
> 46.409Z.csv
>
>
> We are seeing a few issues with the new rebalancing behavior in Streams 2.6. 
> This ticket is for constant probing rebalances on one StreamThread, but I'll 
> mention the other issues, as they may be related.
> First, when we redeploy the application we see tasks being moved, even though 
> the task assignment was stable before redeploying. We would expect to see 
> tasks assigned back to the same instances and no movement. The application is 
> in EC2, with persistent EBS volumes, and we use static group membership to 
> avoid rebalancing. To redeploy the app we terminate all EC2 instances. The 
> new instances will reattach the EBS volumes and use the same group member id.
> After redeploying, we sometimes see the group leader go into a tight probing 
> rebalance loop. This doesn't happen immediately, it could be several hours 
> later. Because the redeploy caused task movement, we see expected probing 
> rebalances every 10 minutes. But, then one thread will go into a tight loop 
> logging messages like "Triggering the followup rebalance scheduled for 
> 1603323868771 ms.", handling the partition assignment (which doesn't change), 
> then "Requested to schedule probing rebalance for 1603323868771 ms." This 
> repeats several times a second until the app is restarted again. I'll attach 
> a log export from one such incident.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (KAFKA-10633) Constant probing rebalances in Streams 2.6

2020-10-22 Thread Bradley Peterson (Jira)
Bradley Peterson created KAFKA-10633:


 Summary: Constant probing rebalances in Streams 2.6
 Key: KAFKA-10633
 URL: https://issues.apache.org/jira/browse/KAFKA-10633
 Project: Kafka
  Issue Type: Bug
  Components: streams
Affects Versions: 2.6.0
Reporter: Bradley Peterson
 Attachments: Discover 2020-10-21T23 34 03.867Z - 2020-10-21T23 44 
46.409Z.csv

We are seeing a few issues with the new rebalancing behavior in Streams 2.6. 
This ticket is for constant probing rebalances on one StreamThread, but I'll 
mention the other issues, as they may be related.

First, when we redeploy the application we see tasks being moved, even though 
the task assignment was stable before redeploying. We would expect to see tasks 
assigned back to the same instances and no movement. The application is in EC2, 
with persistent EBS volumes, and we use static group membership to avoid 
rebalancing. To redeploy the app we terminate all EC2 instances. The new 
instances will reattach the EBS volumes and use the same group member id.

After redeploying, we sometimes see the group leader go into a tight probing 
rebalance loop. This doesn't happen immediately, it could be several hours 
later. Because the redeploy caused task movement, we see expected probing 
rebalances every 10 minutes. But, then one thread will go into a tight loop 
logging messages like "Triggering the followup rebalance scheduled for 
1603323868771 ms.", handling the partition assignment (which doesn't change), 
then "Requested to schedule probing rebalance for 1603323868771 ms." This 
repeats several times a second until the app is restarted again. I'll attach a 
log export from one such incident.





--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9062) Handle stalled writes to RocksDB

2020-07-01 Thread Bradley Peterson (Jira)


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

Bradley Peterson commented on KAFKA-9062:
-

[~ableegoldman] great, thanks. That bug does sound like a match, so I'll keep 
an eye on it.

> Handle stalled writes to RocksDB
> 
>
> Key: KAFKA-9062
> URL: https://issues.apache.org/jira/browse/KAFKA-9062
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Reporter: Sophie Blee-Goldman
>Priority: Major
>
> RocksDB may stall writes at times when background compactions or flushes are 
> having trouble keeping up. This means we can effectively end up blocking 
> indefinitely during a StateStore#put call within Streams, and may get kicked 
> from the group if the throttling does not ease up within the max poll 
> interval.
> Example: when restoring large amounts of state from scratch, we use the 
> strategy recommended by RocksDB of turning off automatic compactions and 
> dumping everything into L0. We do batch somewhat, but do not sort these small 
> batches before loading into the db, so we end up with a large number of 
> unsorted L0 files.
> When restoration is complete and we toggle the db back to normal (not bulk 
> loading) settings, a background compaction is triggered to merge all these 
> into the next level. This background compaction can take a long time to merge 
> unsorted keys, especially when the amount of data is quite large.
> Any new writes while the number of L0 files exceeds the max will be stalled 
> until the compaction can finish, and processing after restoring from scratch 
> can block beyond the polling interval



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9062) Handle stalled writes to RocksDB

2020-07-01 Thread Bradley Peterson (Jira)


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

Bradley Peterson commented on KAFKA-9062:
-

I have just a question -- would these stalls block heartbeats or just polling? 
We have hit this issue in the past and raised out max.poll.interval.ms very 
high (12 hours) to work around it. But, sometimes after a state restoration we 
still instances kicked out of the group by the group coordinator. Our 
session.timeout.ms is 30 minutes, so that suggests to me that something is 
blocking heartbeats for over 30 minutes.

> Handle stalled writes to RocksDB
> 
>
> Key: KAFKA-9062
> URL: https://issues.apache.org/jira/browse/KAFKA-9062
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Reporter: Sophie Blee-Goldman
>Priority: Major
>
> RocksDB may stall writes at times when background compactions or flushes are 
> having trouble keeping up. This means we can effectively end up blocking 
> indefinitely during a StateStore#put call within Streams, and may get kicked 
> from the group if the throttling does not ease up within the max poll 
> interval.
> Example: when restoring large amounts of state from scratch, we use the 
> strategy recommended by RocksDB of turning off automatic compactions and 
> dumping everything into L0. We do batch somewhat, but do not sort these small 
> batches before loading into the db, so we end up with a large number of 
> unsorted L0 files.
> When restoration is complete and we toggle the db back to normal (not bulk 
> loading) settings, a background compaction is triggered to merge all these 
> into the next level. This background compaction can take a long time to merge 
> unsorted keys, especially when the amount of data is quite large.
> Any new writes while the number of L0 files exceeds the max will be stalled 
> until the compaction can finish, and processing after restoring from scratch 
> can block beyond the polling interval



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9867) Log cleaner throws InvalidOffsetException for some partitions

2020-04-16 Thread Bradley Peterson (Jira)


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

Bradley Peterson commented on KAFKA-9867:
-

After upgrade to 2.4.1, I see the same error logged slightly differently.

{noformat}
[2020-04-16 20:53:14,748] WARN [kafka-log-cleaner-thread-0]: Unexpected 
exception thrown when cleaning log Log(dir=/var/lib/kafka/x-10, 
topic=x, partition=10, highWatermark=1191729811, 
lastStableOffset=1191729811, logStartOffset=0, logEndOffset=1191729811). 
Marking its partition (x-10) as uncleanable (kafka.log.LogCleaner)
kafka.log.LogCleaningException: Attempt to append an offset (279327173) to 
position 4077 no larger than the last offset appended (279327173) to 
/var/lib/kafka/x-10/000278373839.index.cleaned.
at 
kafka.log.LogCleaner$CleanerThread.cleanFilthiestLog(LogCleaner.scala:349)
at 
kafka.log.LogCleaner$CleanerThread.tryCleanFilthiestLog(LogCleaner.scala:325)
at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:314)
at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:96)
Caused by: org.apache.kafka.common.errors.InvalidOffsetException: Attempt to 
append an offset (279327173) to position 4077 no larger than the last offset 
appended (279327173) to 
/var/lib/kafka/x-10/000278373839.index.cleaned.
{noformat}

> Log cleaner throws InvalidOffsetException for some partitions
> -
>
> Key: KAFKA-9867
> URL: https://issues.apache.org/jira/browse/KAFKA-9867
> Project: Kafka
>  Issue Type: Bug
>  Components: log cleaner
>Affects Versions: 2.3.0
> Environment: AWS EC2 with data on EBS volumes.
>Reporter: Bradley Peterson
>Priority: Major
>
> About half the partitions for one topic are marked "uncleanable". This is 
> consistent across broker replicas – if a partition is uncleanable on one 
> broker, its replicas are also uncleanable.
> The log-cleaner log seems to suggest out of order offsets. We've seen corrupt 
> indexes before, so I removed the indexes from the affected segments and let 
> Kafka rebuild them, but it hit the same error.
> I don't know when the error first occurred because we've restarted the 
> brokers and rotated logs, but it is possible the broker's crashed at some 
> point.
> How can I repair these partitions?
> {noformat}
> [2020-04-09 00:14:16,979] INFO Cleaner 0: Cleaning segment 223293473 in log 
> x-9 (largest timestamp Wed Nov 13 20:35:57 UTC 2019
> ) into 223293473, retaining deletes. (kafka.log.LogCleaner)
> [2020-04-09 00:14:29,486] INFO Cleaner 0: Cleaning segment 226762315 in log 
> x-9 (largest timestamp Wed Nov 06 18:17:10 UTC 2019
> ) into 223293473, retaining deletes. (kafka.log.LogCleaner)
> [2020-04-09 00:14:29,502] WARN [kafka-log-cleaner-thread-0]: Unexpected 
> exception thrown when cleaning log Log(/var/lib/kafka/x
> -9). Marking its partition (x-9) as uncleanable (kafka.log.LogCleaner)
> org.apache.kafka.common.errors.InvalidOffsetException: Attempt to append an 
> offset (226765178) to position 941 no larger than the last offset appended 
> (228774155) to /var/lib/kafka/x-9/000223293473.index.cleaned.
> {noformat}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9867) Log cleaner throws InvalidOffsetException for some partitions

2020-04-15 Thread Bradley Peterson (Jira)


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

Bradley Peterson commented on KAFKA-9867:
-

My mistake, the brokers are actually 2.3.0. I'll upgrade to 2.4.1.

> Log cleaner throws InvalidOffsetException for some partitions
> -
>
> Key: KAFKA-9867
> URL: https://issues.apache.org/jira/browse/KAFKA-9867
> Project: Kafka
>  Issue Type: Bug
>  Components: log cleaner
>Affects Versions: 2.3.0
> Environment: AWS EC2 with data on EBS volumes.
>Reporter: Bradley Peterson
>Priority: Major
>
> About half the partitions for one topic are marked "uncleanable". This is 
> consistent across broker replicas – if a partition is uncleanable on one 
> broker, its replicas are also uncleanable.
> The log-cleaner log seems to suggest out of order offsets. We've seen corrupt 
> indexes before, so I removed the indexes from the affected segments and let 
> Kafka rebuild them, but it hit the same error.
> I don't know when the error first occurred because we've restarted the 
> brokers and rotated logs, but it is possible the broker's crashed at some 
> point.
> How can I repair these partitions?
> {noformat}
> [2020-04-09 00:14:16,979] INFO Cleaner 0: Cleaning segment 223293473 in log 
> x-9 (largest timestamp Wed Nov 13 20:35:57 UTC 2019
> ) into 223293473, retaining deletes. (kafka.log.LogCleaner)
> [2020-04-09 00:14:29,486] INFO Cleaner 0: Cleaning segment 226762315 in log 
> x-9 (largest timestamp Wed Nov 06 18:17:10 UTC 2019
> ) into 223293473, retaining deletes. (kafka.log.LogCleaner)
> [2020-04-09 00:14:29,502] WARN [kafka-log-cleaner-thread-0]: Unexpected 
> exception thrown when cleaning log Log(/var/lib/kafka/x
> -9). Marking its partition (x-9) as uncleanable (kafka.log.LogCleaner)
> org.apache.kafka.common.errors.InvalidOffsetException: Attempt to append an 
> offset (226765178) to position 941 no larger than the last offset appended 
> (228774155) to /var/lib/kafka/x-9/000223293473.index.cleaned.
> {noformat}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Updated] (KAFKA-9867) Log cleaner throws InvalidOffsetException for some partitions

2020-04-15 Thread Bradley Peterson (Jira)


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

Bradley Peterson updated KAFKA-9867:

Affects Version/s: (was: 2.4.0)
   2.3.0

> Log cleaner throws InvalidOffsetException for some partitions
> -
>
> Key: KAFKA-9867
> URL: https://issues.apache.org/jira/browse/KAFKA-9867
> Project: Kafka
>  Issue Type: Bug
>  Components: log cleaner
>Affects Versions: 2.3.0
> Environment: AWS EC2 with data on EBS volumes.
>Reporter: Bradley Peterson
>Priority: Major
>
> About half the partitions for one topic are marked "uncleanable". This is 
> consistent across broker replicas – if a partition is uncleanable on one 
> broker, its replicas are also uncleanable.
> The log-cleaner log seems to suggest out of order offsets. We've seen corrupt 
> indexes before, so I removed the indexes from the affected segments and let 
> Kafka rebuild them, but it hit the same error.
> I don't know when the error first occurred because we've restarted the 
> brokers and rotated logs, but it is possible the broker's crashed at some 
> point.
> How can I repair these partitions?
> {noformat}
> [2020-04-09 00:14:16,979] INFO Cleaner 0: Cleaning segment 223293473 in log 
> x-9 (largest timestamp Wed Nov 13 20:35:57 UTC 2019
> ) into 223293473, retaining deletes. (kafka.log.LogCleaner)
> [2020-04-09 00:14:29,486] INFO Cleaner 0: Cleaning segment 226762315 in log 
> x-9 (largest timestamp Wed Nov 06 18:17:10 UTC 2019
> ) into 223293473, retaining deletes. (kafka.log.LogCleaner)
> [2020-04-09 00:14:29,502] WARN [kafka-log-cleaner-thread-0]: Unexpected 
> exception thrown when cleaning log Log(/var/lib/kafka/x
> -9). Marking its partition (x-9) as uncleanable (kafka.log.LogCleaner)
> org.apache.kafka.common.errors.InvalidOffsetException: Attempt to append an 
> offset (226765178) to position 941 no larger than the last offset appended 
> (228774155) to /var/lib/kafka/x-9/000223293473.index.cleaned.
> {noformat}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9867) Log cleaner throws InvalidOffsetException for some partitions

2020-04-14 Thread Bradley Peterson (Jira)


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

Bradley Peterson commented on KAFKA-9867:
-

I haven't. I didn't see any related changes in 2.4.1 so I was waiting for 2.5.0 
to upgrade. But I'll try 2.4.1 if there's reason to believe it will help.

> Log cleaner throws InvalidOffsetException for some partitions
> -
>
> Key: KAFKA-9867
> URL: https://issues.apache.org/jira/browse/KAFKA-9867
> Project: Kafka
>  Issue Type: Bug
>  Components: log cleaner
>Affects Versions: 2.4.0
> Environment: AWS EC2 with data on EBS volumes.
>Reporter: Bradley Peterson
>Priority: Major
>
> About half the partitions for one topic are marked "uncleanable". This is 
> consistent across broker replicas – if a partition is uncleanable on one 
> broker, its replicas are also uncleanable.
> The log-cleaner log seems to suggest out of order offsets. We've seen corrupt 
> indexes before, so I removed the indexes from the affected segments and let 
> Kafka rebuild them, but it hit the same error.
> I don't know when the error first occurred because we've restarted the 
> brokers and rotated logs, but it is possible the broker's crashed at some 
> point.
> How can I repair these partitions?
> {noformat}
> [2020-04-09 00:14:16,979] INFO Cleaner 0: Cleaning segment 223293473 in log 
> x-9 (largest timestamp Wed Nov 13 20:35:57 UTC 2019
> ) into 223293473, retaining deletes. (kafka.log.LogCleaner)
> [2020-04-09 00:14:29,486] INFO Cleaner 0: Cleaning segment 226762315 in log 
> x-9 (largest timestamp Wed Nov 06 18:17:10 UTC 2019
> ) into 223293473, retaining deletes. (kafka.log.LogCleaner)
> [2020-04-09 00:14:29,502] WARN [kafka-log-cleaner-thread-0]: Unexpected 
> exception thrown when cleaning log Log(/var/lib/kafka/x
> -9). Marking its partition (x-9) as uncleanable (kafka.log.LogCleaner)
> org.apache.kafka.common.errors.InvalidOffsetException: Attempt to append an 
> offset (226765178) to position 941 no larger than the last offset appended 
> (228774155) to /var/lib/kafka/x-9/000223293473.index.cleaned.
> {noformat}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)