[jira] [Updated] (KAFKA-10105) Regression in group coordinator dealing with flaky clients joining while leaving

2020-06-22 Thread William Reynolds (Jira)


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

William Reynolds updated KAFKA-10105:
-
Environment: Kafka 2.4.1 on jre 11 on debian 9 in docker  (was: Kafka 1.1.0 
on jre 8 on debian 9 in docker
Kafka 2.4.1 on jre 11 on debian 9 in docker)

> Regression in group coordinator dealing with flaky clients joining while 
> leaving
> 
>
> Key: KAFKA-10105
> URL: https://issues.apache.org/jira/browse/KAFKA-10105
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 2.4.1
> Environment: Kafka 2.4.1 on jre 11 on debian 9 in docker
>Reporter: William Reynolds
>Priority: Major
>
> Since upgrade of a cluster from 1.1.0 to 2.4.1 the broker no longer deals 
> correctly with a consumer sending a join after a leave correctly.
> What happens no is that if a consumer sends a leaving then follows up by 
> trying to send a join again as it is shutting down the group coordinator adds 
> the leaving member to the group but never seems to heartbeat that member.
> Since the consumer is then gone when it joins again after starting it is 
> added as a new member but the zombie member is there and is included in the 
> partition assignment which means that those partitions never get consumed 
> from. What can also happen is that one of the zombies gets group leader so 
> rebalance gets stuck forever and the group is entirely blocked.
> I have not been able to track down where this got introduced between 1.1.0 
> and 2.4.1 but I will look further into this. Unfortunately the logs are 
> essentially silent about the zombie mebers and I only had INFO level logging 
> on during the issue and by stopping all the consumers in the group and 
> restarting the broker coordinating that group we could get back to a working 
> state.



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


[jira] [Commented] (KAFKA-10105) Regression in group coordinator dealing with flaky clients joining while leaving

2020-06-22 Thread William Reynolds (Jira)


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

William Reynolds commented on KAFKA-10105:
--

[~gokul2411s]the reference to 1.1.0 was due to that be the version that we were 
on before upgrade to 2.4.1. It is a bit confusing now I look at it, apologies.

 

[~ableegoldman] the consumers were the ruby-kafka clients [~theturtle32] 
described earlier which aren't very tightly coded to keep up with kafka version 
changes ([https://github.com/zendesk/ruby-kafka)], it just uses the new 
consumer. I think KAFKA-9935 was with one of the official clients so perhaps 
digging into that would make for an easier reproduction. If that doesn't pan 
out between brian and me I believe we could outline the ruby kafka steps to 
reproduce also.

> Regression in group coordinator dealing with flaky clients joining while 
> leaving
> 
>
> Key: KAFKA-10105
> URL: https://issues.apache.org/jira/browse/KAFKA-10105
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 2.4.1
> Environment: Kafka 1.1.0 on jre 8 on debian 9 in docker
> Kafka 2.4.1 on jre 11 on debian 9 in docker
>Reporter: William Reynolds
>Priority: Major
>
> Since upgrade of a cluster from 1.1.0 to 2.4.1 the broker no longer deals 
> correctly with a consumer sending a join after a leave correctly.
> What happens no is that if a consumer sends a leaving then follows up by 
> trying to send a join again as it is shutting down the group coordinator adds 
> the leaving member to the group but never seems to heartbeat that member.
> Since the consumer is then gone when it joins again after starting it is 
> added as a new member but the zombie member is there and is included in the 
> partition assignment which means that those partitions never get consumed 
> from. What can also happen is that one of the zombies gets group leader so 
> rebalance gets stuck forever and the group is entirely blocked.
> I have not been able to track down where this got introduced between 1.1.0 
> and 2.4.1 but I will look further into this. Unfortunately the logs are 
> essentially silent about the zombie mebers and I only had INFO level logging 
> on during the issue and by stopping all the consumers in the group and 
> restarting the broker coordinating that group we could get back to a working 
> state.



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


[jira] [Commented] (KAFKA-10107) Producer snapshots LSO used in certain situations which can lead to data loss on compacted topics as LSO breach occurs and early offsets cleaned

2020-06-09 Thread William Reynolds (Jira)


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

William Reynolds commented on KAFKA-10107:
--

Still working on getting the logs out, apologies for delay

> Producer snapshots LSO used in certain situations which can lead to data loss 
> on compacted topics as LSO breach occurs and early offsets cleaned
> 
>
> Key: KAFKA-10107
> URL: https://issues.apache.org/jira/browse/KAFKA-10107
> Project: Kafka
>  Issue Type: Bug
>  Components: core, log cleaner
>Affects Versions: 2.4.1
> Environment: Kafka 1.1.0 on jre 8 on debian 9 in docker
> Kafka 2.4.1 on jre 11 on debian 9 in docker
>Reporter: William Reynolds
>Priority: Major
>
> While upgading a 1.1.0 cluster to 2.4.1 and also adding an interbroker port 
> using SSL we ran into a situation where producer snapshot offsets get set as 
> the log start offset then logs truncate to nothing across 2 relatively unsafe 
> restarts.
>  
> Here is the timeline of what we did to trigger this
> Broker 40 is shutdown as first to go to 2.4.1 and switch to interbroker port 
> 9094.
>  As it shuts down it writes producer snapshots
>  Broker 40 starts on 2.4.1, loads the snapshots then compares checkpointed 
> offsets to log start offset and finds them to be invalid (exact reason 
> unknown but looks to be producer snapshot load related)
>  On broker 40 all topics show an offset reset like this 2020-05-18 
> 15:22:21,106] WARN Resetting first dirty offset of topic-name-60 to log start 
> offset 6009368 since the checkpointed offset 5952382 is invalid. 
> (kafka.log.LogCleanerManager$)" which then triggers log cleanup on broker 40 
> for all these topics which is where the data is lost
>  At this point only partitions led by broker 40 have lost data and would be 
> failing for client lookups on older data but this can't spread as 40 has 
> interbroker port 9094 and brokers 50 and 60 have interbroker port 9092
>  I stop start brokers 50 and 60 in quick succession to take them to 2.4.1 and 
> onto the new interbroker port 9094
>  This leaves broker 40 as the in sync replica for all but a couple of 
> partitions which aren't on 40 at all shown in the attached image
>  Brokers 50 and 60 start and then take their start offset from leader (or if 
> there was no leader pulls from recovery on returning broker 50 or 60) and so 
> all the replicas also clean logs to remove data to catch up to broker 40 as 
> that is the in sync replica
>  Then I shutdown 40 and 50 leading to 60 leading all partitions it holds and 
> then we see this happen across all of those partitions
>  "May 18, 2020 @ 
> 15:48:28.252",hostname-1,30438,apache-kafka:2.4.1,"[2020-05-18 15:48:28,251] 
> INFO [Log partition=topic-name-60, dir=/kafka-topic-data] Loading producer 
> state till offset 0 with message format version 2 (kafka.log.Log)" 
>  "May 18, 2020 @ 
> 15:48:28.252",hostname-1,30438,apache-kafka:2.4.1,"[2020-05-18 15:48:28,252] 
> INFO [Log partition=topic-name-60, dir=/kafka-topic-data] Completed load of 
> log with 1 segments, log start offset 0 and log end offset 0 in 2 ms 
> (kafka.log.Log)"
>  "May 18, 2020 @ 15:48:45.883",hostname,7805,apache-kafka:2.4.1,"[2020-05-18 
> 15:48:45,883] WARN [ReplicaFetcher replicaId=50, leaderId=60, fetcherId=0] 
> Leader or replica is on protocol version where leader epoch is not considered 
> in the OffsetsForLeaderEpoch response. The leader's offset 0 will be used for 
> truncation in topic-name-60. (kafka.server.ReplicaFetcherThread)" 
>  "May 18, 2020 @ 15:48:45.883",hostname,7805,apache-kafka:2.4.1,"[2020-05-18 
> 15:48:45,883] INFO [Log partition=topic-name-60, dir=/kafka-topic-data] 
> Truncating to offset 0 (kafka.log.Log)"
>  
> I believe the truncation has always been a problem but recent 
> https://issues.apache.org/jira/browse/KAFKA-6266 fix allowed truncation to 
> actually happen where it wouldn't have before. 
>  The producer snapshots setting as log start offset is a mystery to me so any 
> light you could shed on why that yhappened and how to avoid would be great.
>  
> I am sanitising full logs and will upload here soon



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


[jira] [Commented] (KAFKA-9935) Kafka not releasing member from Consumer Group

2020-06-09 Thread William Reynolds (Jira)


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

William Reynolds commented on KAFKA-9935:
-

It does look related to me. Your reproduction looks similar to what we hit and 
also what it looks like to the tools

> Kafka not releasing member from Consumer Group
> --
>
> Key: KAFKA-9935
> URL: https://issues.apache.org/jira/browse/KAFKA-9935
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 2.4.1
> Environment: Linux
>Reporter: Steve Kecskes
>Priority: Major
>
> Hello. I am experiencing an issue where Kafka is not releasing members from a 
> consumer group when the member crashes. The consumer group is then stuck in 
> rebalancing state indefinitely.
> In this consumer group, there is only 1 client. This client has the following 
> related settings:
> {code:java}
> auto.commit.interval.ms = 5000
>  auto.offset.reset = earliest
>  bootstrap.servers = [austgkafka01.hk.eclipseoptions.com:9092]
>  check.crcs = true
>  client.dns.lookup = default
>  client.id = TraderAutomationViewServer_workaround_stuck_rebalance_20200427-0
>  connections.max.idle.ms = 54
>  default.api.timeout.ms = 6
>  enable.auto.commit = true
>  exclude.internal.topics = true
>  fetch.max.bytes = 52428800
>  fetch.max.wait.ms = 500
>  fetch.min.bytes = 1
>  group.id = TraderAutomationViewServer_workaround_stuck_rebalance_20200427
>  heartbeat.interval.ms = 3000
>  interceptor.classes = []
>  internal.leave.group.on.close = true
>  isolation.level = read_uncommitted
>  key.deserializer = class 
> org.apache.kafka.common.serialization.ByteArrayDeserializer
>  max.partition.fetch.bytes = 1048576
>  max.poll.interval.ms = 30
>  max.poll.records = 1
>  metadata.max.age.ms = 30
>  metric.reporters = []
>  metrics.num.samples = 2
>  metrics.recording.level = INFO
>  metrics.sample.window.ms = 3
>  partition.assignment.strategy = [class 
> org.apache.kafka.clients.consumer.RangeAssignor]
>  receive.buffer.bytes = 16777216
>  reconnect.backoff.max.ms = 1000
>  reconnect.backoff.ms = 50
>  request.timeout.ms = 3
>  retry.backoff.ms = 100
>  sasl.client.callback.handler.class = null
>  sasl.jaas.config = null
>  sasl.kerberos.kinit.cmd = /usr/bin/kinit
>  sasl.kerberos.min.time.before.relogin = 6
>  sasl.kerberos.service.name = null
>  sasl.kerberos.ticket.renew.jitter = 0.05
>  sasl.kerberos.ticket.renew.window.factor = 0.8
>  sasl.login.callback.handler.class = null
>  sasl.login.class = null
>  sasl.login.refresh.buffer.seconds = 300
>  sasl.login.refresh.min.period.seconds = 60
>  sasl.login.refresh.window.factor = 0.8
>  sasl.login.refresh.window.jitter = 0.05
>  sasl.mechanism = GSSAPI
>  security.protocol = PLAINTEXT
>  send.buffer.bytes = 131072
>  session.timeout.ms = 1
>  ssl.cipher.suites = null
>  ssl.enabled.protocols = [TLSv1.2, TLSv1.1, TLSv1]
>  ssl.endpoint.identification.algorithm = https
>  ssl.key.password = null
>  ssl.keymanager.algorithm = SunX509
>  ssl.keystore.location = null
>  ssl.keystore.password = null
>  ssl.keystore.type = JKS
>  ssl.protocol = TLS
>  ssl.provider = null
>  ssl.secure.random.implementation = null
>  ssl.trustmanager.algorithm = PKIX
>  ssl.truststore.location = null
>  ssl.truststore.password = null
>  ssl.truststore.type = JKS
>  value.deserializer = class 
> org.apache.kafka.common.serialization.ByteArrayDeserializer
> {code}
> If the client crashes (not a graceful exit from group) the group remains in 
> the following state indefinitely.
> {code}
> Warning: Consumer group 
> 'TraderAutomationViewServer_workaround_stuck_rebalance' is rebalancing.
> GROUP TOPIC 
> PARTITION  CURRENT-OFFSET  LOG-END-OFFSET  LAG CONSUMER-ID 
> HOSTCLIENT-ID
> TraderAutomationViewServer_workaround_stuck_rebalance EventAdjustedVols 10
>  6984061 7839599 855538  -   -
>-
> TraderAutomationViewServer_workaround_stuck_rebalance VolMetrics8 
>  128459531   143736443   15276912-   -
>-
> TraderAutomationViewServer_workaround_stuck_rebalance EventAdjustedVols 12
>  7216495 8106030 889535  -   -
>-
> TraderAutomationViewServer_workaround_stuck_rebalance VolMetrics6 
>  122921729   137377358   14455629-   -
>-
> TraderAutomationViewServer_workaround_stuck_rebalance EventAdjustedVols 14
>  5457618 6171142 713524  -   -
>-
> TraderAutomationViewServer_workaround_stuck_rebalance VolMetrics4 
>  125647891   140542566 

[jira] [Commented] (KAFKA-10105) Regression in group coordinator dealing with flaky clients joining while leaving

2020-06-09 Thread William Reynolds (Jira)


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

William Reynolds commented on KAFKA-10105:
--

Hi James, that looks really similar. If it isn't the exact thing I would 
suspect that it is another symptom of the group coord change that we hit

> Regression in group coordinator dealing with flaky clients joining while 
> leaving
> 
>
> Key: KAFKA-10105
> URL: https://issues.apache.org/jira/browse/KAFKA-10105
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 2.4.1
> Environment: Kafka 1.1.0 on jre 8 on debian 9 in docker
> Kafka 2.4.1 on jre 11 on debian 9 in docker
>Reporter: William Reynolds
>Priority: Major
>
> Since upgrade of a cluster from 1.1.0 to 2.4.1 the broker no longer deals 
> correctly with a consumer sending a join after a leave correctly.
> What happens no is that if a consumer sends a leaving then follows up by 
> trying to send a join again as it is shutting down the group coordinator adds 
> the leaving member to the group but never seems to heartbeat that member.
> Since the consumer is then gone when it joins again after starting it is 
> added as a new member but the zombie member is there and is included in the 
> partition assignment which means that those partitions never get consumed 
> from. What can also happen is that one of the zombies gets group leader so 
> rebalance gets stuck forever and the group is entirely blocked.
> I have not been able to track down where this got introduced between 1.1.0 
> and 2.4.1 but I will look further into this. Unfortunately the logs are 
> essentially silent about the zombie mebers and I only had INFO level logging 
> on during the issue and by stopping all the consumers in the group and 
> restarting the broker coordinating that group we could get back to a working 
> state.



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


[jira] [Updated] (KAFKA-10107) Producer snapshots LSO used in certain situations which can lead to data loss on compacted topics as LSO breach occurs and early offsets cleaned

2020-06-05 Thread William Reynolds (Jira)


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

William Reynolds updated KAFKA-10107:
-
Environment: 
Kafka 1.1.0 on jre 8 on debian 9 in docker
Kafka 2.4.1 on jre 11 on debian 9 in docker

> Producer snapshots LSO used in certain situations which can lead to data loss 
> on compacted topics as LSO breach occurs and early offsets cleaned
> 
>
> Key: KAFKA-10107
> URL: https://issues.apache.org/jira/browse/KAFKA-10107
> Project: Kafka
>  Issue Type: Bug
>  Components: core, log cleaner
>Affects Versions: 2.4.1
> Environment: Kafka 1.1.0 on jre 8 on debian 9 in docker
> Kafka 2.4.1 on jre 11 on debian 9 in docker
>Reporter: William Reynolds
>Priority: Major
>
> While upgading a 1.1.0 cluster to 2.4.1 and also adding an interbroker port 
> using SSL we ran into a situation where producer snapshot offsets get set as 
> the log start offset then logs truncate to nothing across 2 relatively unsafe 
> restarts.
>  
> Here is the timeline of what we did to trigger this
> Broker 40 is shutdown as first to go to 2.4.1 and switch to interbroker port 
> 9094.
>  As it shuts down it writes producer snapshots
>  Broker 40 starts on 2.4.1, loads the snapshots then compares checkpointed 
> offsets to log start offset and finds them to be invalid (exact reason 
> unknown but looks to be producer snapshot load related)
>  On broker 40 all topics show an offset reset like this 2020-05-18 
> 15:22:21,106] WARN Resetting first dirty offset of topic-name-60 to log start 
> offset 6009368 since the checkpointed offset 5952382 is invalid. 
> (kafka.log.LogCleanerManager$)" which then triggers log cleanup on broker 40 
> for all these topics which is where the data is lost
>  At this point only partitions led by broker 40 have lost data and would be 
> failing for client lookups on older data but this can't spread as 40 has 
> interbroker port 9094 and brokers 50 and 60 have interbroker port 9092
>  I stop start brokers 50 and 60 in quick succession to take them to 2.4.1 and 
> onto the new interbroker port 9094
>  This leaves broker 40 as the in sync replica for all but a couple of 
> partitions which aren't on 40 at all shown in the attached image
>  Brokers 50 and 60 start and then take their start offset from leader (or if 
> there was no leader pulls from recovery on returning broker 50 or 60) and so 
> all the replicas also clean logs to remove data to catch up to broker 40 as 
> that is the in sync replica
>  Then I shutdown 40 and 50 leading to 60 leading all partitions it holds and 
> then we see this happen across all of those partitions
>  "May 18, 2020 @ 
> 15:48:28.252",hostname-1,30438,apache-kafka:2.4.1,"[2020-05-18 15:48:28,251] 
> INFO [Log partition=topic-name-60, dir=/kafka-topic-data] Loading producer 
> state till offset 0 with message format version 2 (kafka.log.Log)" 
>  "May 18, 2020 @ 
> 15:48:28.252",hostname-1,30438,apache-kafka:2.4.1,"[2020-05-18 15:48:28,252] 
> INFO [Log partition=topic-name-60, dir=/kafka-topic-data] Completed load of 
> log with 1 segments, log start offset 0 and log end offset 0 in 2 ms 
> (kafka.log.Log)"
>  "May 18, 2020 @ 15:48:45.883",hostname,7805,apache-kafka:2.4.1,"[2020-05-18 
> 15:48:45,883] WARN [ReplicaFetcher replicaId=50, leaderId=60, fetcherId=0] 
> Leader or replica is on protocol version where leader epoch is not considered 
> in the OffsetsForLeaderEpoch response. The leader's offset 0 will be used for 
> truncation in topic-name-60. (kafka.server.ReplicaFetcherThread)" 
>  "May 18, 2020 @ 15:48:45.883",hostname,7805,apache-kafka:2.4.1,"[2020-05-18 
> 15:48:45,883] INFO [Log partition=topic-name-60, dir=/kafka-topic-data] 
> Truncating to offset 0 (kafka.log.Log)"
>  
> I believe the truncation has always been a problem but recent 
> https://issues.apache.org/jira/browse/KAFKA-6266 fix allowed truncation to 
> actually happen where it wouldn't have before. 
>  The producer snapshots setting as log start offset is a mystery to me so any 
> light you could shed on why that yhappened and how to avoid would be great.
>  
> I am sanitising full logs and will upload here soon



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


[jira] [Created] (KAFKA-10107) Producer snapshots LSO used in certain situations which can lead to data loss on compacted topics as LSO breach occurs and early offsets cleaned

2020-06-05 Thread William Reynolds (Jira)
William Reynolds created KAFKA-10107:


 Summary: Producer snapshots LSO used in certain situations which 
can lead to data loss on compacted topics as LSO breach occurs and early 
offsets cleaned
 Key: KAFKA-10107
 URL: https://issues.apache.org/jira/browse/KAFKA-10107
 Project: Kafka
  Issue Type: Bug
  Components: core, log cleaner
Affects Versions: 2.4.1
Reporter: William Reynolds


While upgading a 1.1.0 cluster to 2.4.1 and also adding an interbroker port 
using SSL we ran into a situation where producer snapshot offsets get set as 
the log start offset then logs truncate to nothing across 2 relatively unsafe 
restarts.

 

Here is the timeline of what we did to trigger this

Broker 40 is shutdown as first to go to 2.4.1 and switch to interbroker port 
9094.
 As it shuts down it writes producer snapshots
 Broker 40 starts on 2.4.1, loads the snapshots then compares checkpointed 
offsets to log start offset and finds them to be invalid (exact reason unknown 
but looks to be producer snapshot load related)
 On broker 40 all topics show an offset reset like this 2020-05-18 
15:22:21,106] WARN Resetting first dirty offset of topic-name-60 to log start 
offset 6009368 since the checkpointed offset 5952382 is invalid. 
(kafka.log.LogCleanerManager$)" which then triggers log cleanup on broker 40 
for all these topics which is where the data is lost
 At this point only partitions led by broker 40 have lost data and would be 
failing for client lookups on older data but this can't spread as 40 has 
interbroker port 9094 and brokers 50 and 60 have interbroker port 9092
 I stop start brokers 50 and 60 in quick succession to take them to 2.4.1 and 
onto the new interbroker port 9094
 This leaves broker 40 as the in sync replica for all but a couple of 
partitions which aren't on 40 at all shown in the attached image
 Brokers 50 and 60 start and then take their start offset from leader (or if 
there was no leader pulls from recovery on returning broker 50 or 60) and so 
all the replicas also clean logs to remove data to catch up to broker 40 as 
that is the in sync replica
 Then I shutdown 40 and 50 leading to 60 leading all partitions it holds and 
then we see this happen across all of those partitions
 "May 18, 2020 @ 15:48:28.252",hostname-1,30438,apache-kafka:2.4.1,"[2020-05-18 
15:48:28,251] INFO [Log partition=topic-name-60, dir=/kafka-topic-data] Loading 
producer state till offset 0 with message format version 2 (kafka.log.Log)" 
 "May 18, 2020 @ 15:48:28.252",hostname-1,30438,apache-kafka:2.4.1,"[2020-05-18 
15:48:28,252] INFO [Log partition=topic-name-60, dir=/kafka-topic-data] 
Completed load of log with 1 segments, log start offset 0 and log end offset 0 
in 2 ms (kafka.log.Log)"
 "May 18, 2020 @ 15:48:45.883",hostname,7805,apache-kafka:2.4.1,"[2020-05-18 
15:48:45,883] WARN [ReplicaFetcher replicaId=50, leaderId=60, fetcherId=0] 
Leader or replica is on protocol version where leader epoch is not considered 
in the OffsetsForLeaderEpoch response. The leader's offset 0 will be used for 
truncation in topic-name-60. (kafka.server.ReplicaFetcherThread)" 
 "May 18, 2020 @ 15:48:45.883",hostname,7805,apache-kafka:2.4.1,"[2020-05-18 
15:48:45,883] INFO [Log partition=topic-name-60, dir=/kafka-topic-data] 
Truncating to offset 0 (kafka.log.Log)"

 

I believe the truncation has always been a problem but recent 
https://issues.apache.org/jira/browse/KAFKA-6266 fix allowed truncation to 
actually happen where it wouldn't have before. 
 The producer snapshots setting as log start offset is a mystery to me so any 
light you could shed on why that yhappened and how to avoid would be great.

 

I am sanitising full logs and will upload here soon



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


[jira] [Created] (KAFKA-10105) Regression in group coordinator dealing with flaky clients joining while leaving

2020-06-04 Thread William Reynolds (Jira)
William Reynolds created KAFKA-10105:


 Summary: Regression in group coordinator dealing with flaky 
clients joining while leaving
 Key: KAFKA-10105
 URL: https://issues.apache.org/jira/browse/KAFKA-10105
 Project: Kafka
  Issue Type: Bug
  Components: core
Affects Versions: 2.4.1
 Environment: Kafka 1.1.0 on jre 8 on debian 9 in docker
Kafka 2.4.1 on jre 11 on debian 9 in docker
Reporter: William Reynolds


Since upgrade of a cluster from 1.1.0 to 2.4.1 the broker no longer deals 
correctly with a consumer sending a join after a leave correctly.

What happens no is that if a consumer sends a leaving then follows up by trying 
to send a join again as it is shutting down the group coordinator adds the 
leaving member to the group but never seems to heartbeat that member.

Since the consumer is then gone when it joins again after starting it is added 
as a new member but the zombie member is there and is included in the partition 
assignment which means that those partitions never get consumed from. What can 
also happen is that one of the zombies gets group leader so rebalance gets 
stuck forever and the group is entirely blocked.

I have not been able to track down where this got introduced between 1.1.0 and 
2.4.1 but I will look further into this. Unfortunately the logs are essentially 
silent about the zombie mebers and I only had INFO level logging on during the 
issue and by stopping all the consumers in the group and restarting the broker 
coordinating that group we could get back to a working state.



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


[jira] [Commented] (KAFKA-6266) Kafka 1.0.0 : Repeated occurrence of WARN Resetting first dirty offset of __consumer_offsets-xx to log start offset 203569 since the checkpointed offset 120955 is inval

2020-04-17 Thread William Reynolds (Jira)


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

William Reynolds commented on KAFKA-6266:
-

If anyone runs into this and needs a workaround before getting 2.4.1/trunk. 
Might need some tweaking based on where/how you log and obviously different if 
you aren't using systemd, will need user/dirs tweaked to be a general workaround
{code:java}
journalctl -u kafka --since 'yesterday' | grep 'Resetting first dirty offset 
of' | awk '{print $14 ,$19}' | sort -u > /home/user/checkpoints
cp /kafka-topic-data/cleaner-offset-checkpoint /home/user
cat /home/user/checkpoints | sed -r 's/(.*)-/\1 /g' > 
/home/user/checkpoints-processed
cat /home/user/checkpoints-processed cleaner-offset-checkpoint | sort --key=1,2 
-u > /home/user/cleaner-offset-checkpoint.clean
sudo systemctl stop kafka; sudo mv /kafka-topic-data/cleaner-offset-checkpoint 
/home/user/; sudo mv /home/user/cleaner-offset-checkpoint.clean 
/kafka-topic-data/cleaner-offset-checkpoint; sudo chown -R kafka:kafka 
/kafka-topic-data/cleaner-offset-checkpoint; sleep 5; sudo systemctl start kafka
{code}

> Kafka 1.0.0 : Repeated occurrence of WARN Resetting first dirty offset of 
> __consumer_offsets-xx to log start offset 203569 since the checkpointed 
> offset 120955 is invalid. (kafka.log.LogCleanerManager$)
> --
>
> Key: KAFKA-6266
> URL: https://issues.apache.org/jira/browse/KAFKA-6266
> Project: Kafka
>  Issue Type: Bug
>  Components: offset manager
>Affects Versions: 1.0.0, 1.0.1
> Environment: CentOS 7, Apache kafka_2.12-1.0.0
>Reporter: VinayKumar
>Assignee: David Mao
>Priority: Major
> Fix For: 2.5.0, 2.4.1
>
>
> I upgraded Kafka from 0.10.2.1 to 1.0.0 version. From then, I see the below 
> warnings in the log.
>  I'm seeing these continuously in the log, and want these to be fixed- so 
> that they wont repeat. Can someone please help me in fixing the below 
> warnings.
> {code}
> WARN Resetting first dirty offset of __consumer_offsets-17 to log start 
> offset 3346 since the checkpointed offset 3332 is invalid. 
> (kafka.log.LogCleanerManager$)
>  WARN Resetting first dirty offset of __consumer_offsets-23 to log start 
> offset 4 since the checkpointed offset 1 is invalid. 
> (kafka.log.LogCleanerManager$)
>  WARN Resetting first dirty offset of __consumer_offsets-19 to log start 
> offset 203569 since the checkpointed offset 120955 is invalid. 
> (kafka.log.LogCleanerManager$)
>  WARN Resetting first dirty offset of __consumer_offsets-35 to log start 
> offset 16957 since the checkpointed offset 7 is invalid. 
> (kafka.log.LogCleanerManager$)
> {code}



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


[jira] [Commented] (KAFKA-8815) Kafka broker blocked on I/O primitive

2019-11-21 Thread William Reynolds (Jira)


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

William Reynolds commented on KAFKA-8815:
-

Hi Alexandre, I suspect we may have run into this but we didn't manage to get 
dumps like you did. Do you by any chance have the network in and out pattern 
after the blocking starts? Also what the network processor (type=SocketServer 
name=NetworkProcessorAvgIdlePercent) does after blocking?

> Kafka broker blocked on I/O primitive
> -
>
> Key: KAFKA-8815
> URL: https://issues.apache.org/jira/browse/KAFKA-8815
> Project: Kafka
>  Issue Type: Bug
>  Components: core, log
>Affects Versions: 1.1.1
>Reporter: Alexandre Dupriez
>Priority: Major
>
> This JIRA is for tracking a problem we run into on a production cluster.
> *Scenario*
> Cluster of 15 brokers and an average ingress throughput of ~4 MB/s and egress 
> of ~4 MB/s per broker.
> Brokers are running on OpenJDK 8. They are configured with a heap size of 1 
> GB.
> There is around ~1,000 partition replicas per broker. Load is evenly 
> balanced. Each broker instance is under fair CPU load, but not overloaded 
> (50-60%). G1 is used for garbage collection and doesn't exhibit any pressure, 
> with mostly short young GC observed and an heap-after-GC usage of 70%.
> Replication factor is 3.
> *Symptom*
> One broker on the cluster suddenly became "unresponsive". Other brokers, 
> Zookeeper and producers/consumers requests were failing with timeouts. The 
> Kafka process, however, was still alive and doing some background work 
> (truncating logs and rolling segments) This lasted for hours. At some point, 
> several thread dumps were taken at few minutes interval. Most of the threads 
> were "blocked". Deadlock was ruled out. The most suspicious stack is the 
> following 
> {code:java}
> Thread 7801: (state = BLOCKED)
>  - sun.nio.ch.FileChannelImpl.write(java.nio.ByteBuffer) @bci=25, line=202 
> (Compiled frame)
>  - 
> org.apache.kafka.common.record.MemoryRecords.writeFullyTo(java.nio.channels.GatheringByteChannel)
>  @bci=24, line=93 (Compiled frame)
>  - 
> org.apache.kafka.common.record.FileRecords.append(org.apache.kafka.common.record.MemoryRecords)
>  @bci=5, line=152 (Compiled frame)
>  - kafka.log.LogSegment.append(long, long, long, long, 
> org.apache.kafka.common.record.MemoryRecords) @bci=82, line=136 (Compiled 
> frame)
>  - kafka.log.Log.$anonfun$append$2(kafka.log.Log, 
> org.apache.kafka.common.record.MemoryRecords, boolean, boolean, int, 
> java.lang.Object) @bci=1080, line=757 (Compiled frame)
>  - kafka.log.Log$$Lambda$614.apply() @bci=24 (Compiled frame)
>  - kafka.log.Log.maybeHandleIOException(scala.Function0, scala.Function0) 
> @bci=1, line=1696 (Compiled frame)
>  - kafka.log.Log.append(org.apache.kafka.common.record.MemoryRecords, 
> boolean, boolean, int) @bci=29, line=642 (Compiled frame)
>  - kafka.log.Log.appendAsLeader(org.apache.kafka.common.record.MemoryRecords, 
> int, boolean) @bci=5, line=612 (Compiled frame)
>  - 
> kafka.cluster.Partition.$anonfun$appendRecordsToLeader$1(kafka.cluster.Partition,
>  org.apache.kafka.common.record.MemoryRecords, boolean, int) @bci=148, 
> line=609 (Compiled frame)
>  - kafka.cluster.Partition$$Lambda$837.apply() @bci=16 (Compiled frame)
>  - kafka.utils.CoreUtils$.inLock(java.util.concurrent.locks.Lock, 
> scala.Function0) @bci=7, line=250 (Compiled frame)
>  - 
> kafka.utils.CoreUtils$.inReadLock(java.util.concurrent.locks.ReadWriteLock, 
> scala.Function0) @bci=8, line=256 (Compiled frame)
>  - 
> kafka.cluster.Partition.appendRecordsToLeader(org.apache.kafka.common.record.MemoryRecords,
>  boolean, int) @bci=16, line=597 (Compiled frame)
>  - 
> kafka.server.ReplicaManager.$anonfun$appendToLocalLog$2(kafka.server.ReplicaManager,
>  boolean, boolean, short, scala.Tuple2) @bci=295, line=739 (Compiled frame)
>  - kafka.server.ReplicaManager$$Lambda$836.apply(java.lang.Object) @bci=20 
> (Compiled frame)
>  - scala.collection.TraversableLike.$anonfun$map$1(scala.Function1, 
> scala.collection.mutable.Builder, java.lang.Object) @bci=3, line=234 
> (Compiled frame)
>  - scala.collection.TraversableLike$$Lambda$14.apply(java.lang.Object) @bci=9 
> (Compiled frame)
>  - scala.collection.mutable.HashMap.$anonfun$foreach$1(scala.Function1, 
> scala.collection.mutable.DefaultEntry) @bci=16, line=138 (Compiled frame)
>  - scala.collection.mutable.HashMap$$Lambda$31.apply(java.lang.Object) @bci=8 
> (Compiled frame)
>  - scala.collection.mutable.HashTable.foreachEntry(scala.Function1) @bci=39, 
> line=236 (Compiled frame)
>  - 
> scala.collection.mutable.HashTable.foreachEntry$(scala.collection.mutable.HashTable,
>  scala.Function1) @bci=2, line=229 (Compiled frame)
>  - scala.collection.mutable.HashMap.foreachEntry(scala.Function1)