[jira] [Updated] (KAFKA-5600) GroupMetadataManager doesn't read offsets of segmented logs correctly

2017-07-18 Thread Ismael Juma (JIRA)

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

Ismael Juma updated KAFKA-5600:
---
Description: 
After long investigation we found a Problem in Kafka.
When a __consumer_offsets partition gets segmented and Kafka is restarted and 
needs to reload offsets, consumers will start at a wrong position when metadata 
and offset events are in both segments.

Reproduction:
1.) Start zookeeper and kafka as is from the archive
{code}
KAFKA_HEAP_OPTS="-Xmx256M -Xms256M" bin/zookeeper-server-start.sh 
config/zookeeper.properties
KAFKA_HEAP_OPTS="-Xmx256M -Xms256M" bin/kafka-server-start.sh 
config/server.properties
{code}
2.) Start [^KafkaErrorProducer.java] which adds 1M log entries to the topic test
3.) Start [^KafkaErrorConsumer.java] which starts a consumer, reads 100 entries 
one by one and then closes the consumer. This leads to a 2nd segment in 
/tmp/kafka-logs/__consumer_offsets-27. This step takes some time (around 
5mins). The close of the consumer is needed to have metadata events in the 
segments too.
4.) Stop and restart the Kafka broker
5.) Start any consumer on topic test and group testgroup
{code}
bin/kafka-console-consumer.sh --from-beginning --bootstrap-server 
localhost:9092 --topic test --consumer-property group.id=testgroup
{code}

Is:
the consumer starts at the segmentation boundary
Expected:
the consumer starts at the end

The Reason for this behavior is the closing brace of the while loop in 
GroupMetadataManager#loadGroupsAndOffsets at a wrong position introduced with 
commit 
https://github.com/apache/kafka/commit/5bd06f1d542e6b588a1d402d059bc24690017d32
I will prepare a pull request.

*Edit*: The issue can happen if there are multiple reads from the same segment, 
see https://github.com/apache/kafka/pull/3538#discussion_r127759694

  was:
After long investigation we found a Problem in Kafka.
When a __consumer_offsets partition gets segmented and Kafka is restarted and 
needs to reload offsets, consumers will start at a wrong position when metadata 
and offset events are in both segments.

Reproduction:
1.) Start zookeeper and kafka as is from the archive
{code}
KAFKA_HEAP_OPTS="-Xmx256M -Xms256M" bin/zookeeper-server-start.sh 
config/zookeeper.properties
KAFKA_HEAP_OPTS="-Xmx256M -Xms256M" bin/kafka-server-start.sh 
config/server.properties
{code}
2.) Start [^KafkaErrorProducer.java] which adds 1M log entries to the topic test
3.) Start [^KafkaErrorConsumer.java] which starts a consumer, reads 100 entries 
one by one and then closes the consumer. This leads to a 2nd segment in 
/tmp/kafka-logs/__consumer_offsets-27. This step takes some time (around 
5mins). The close of the consumer is needed to have metadata events in the 
segments too.
4.) Stop and restart the Kafka broker
5.) Start any consumer on topic test and group testgroup
{code}
bin/kafka-console-consumer.sh --from-beginning --bootstrap-server 
localhost:9092 --topic test --consumer-property group.id=testgroup
{code}

Is:
the consumer starts at the segmentation boundary
Expected:
the consumer starts at the end

The Reason for this behavior is the closing brace of the while loop in 
GroupMetadataManager#loadGroupsAndOffsets at a wrong position introduced with 
commit 
https://github.com/apache/kafka/commit/5bd06f1d542e6b588a1d402d059bc24690017d32
I will prepare a pull request.

**Edit**: The issue can happen if there are multiple reads from the same 
segment, see https://github.com/apache/kafka/pull/3538#discussion_r127759694


> GroupMetadataManager doesn't read offsets of segmented logs correctly
> -
>
> Key: KAFKA-5600
> URL: https://issues.apache.org/jira/browse/KAFKA-5600
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 0.10.2.1, 0.11.0.0
> Environment: any
>Reporter: Jan Burkhardt
>Priority: Critical
>  Labels: regression, reliability
> Fix For: 0.11.0.1, 0.11.1.0
>
> Attachments: KafkaErrorConsumer.java, KafkaErrorProducer.java
>
>
> After long investigation we found a Problem in Kafka.
> When a __consumer_offsets partition gets segmented and Kafka is restarted and 
> needs to reload offsets, consumers will start at a wrong position when 
> metadata and offset events are in both segments.
> Reproduction:
> 1.) Start zookeeper and kafka as is from the archive
> {code}
> KAFKA_HEAP_OPTS="-Xmx256M -Xms256M" bin/zookeeper-server-start.sh 
> config/zookeeper.properties
> KAFKA_HEAP_OPTS="-Xmx256M -Xms256M" bin/kafka-server-start.sh 
> config/server.properties
> {code}
> 2.) Start [^KafkaErrorProducer.java] which adds 1M log entries to the topic 
> test
> 3.) Start [^KafkaErrorConsumer.java] which starts a consumer, reads 100 
> entries one by one and then closes the consumer. This leads to a 2nd segment 

[jira] [Updated] (KAFKA-5600) GroupMetadataManager doesn't read offsets of segmented logs correctly

2017-07-18 Thread Jan Burkhardt (JIRA)

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

Jan Burkhardt updated KAFKA-5600:
-
Affects Version/s: (was: 0.10.2.0)
   0.10.2.1

> GroupMetadataManager doesn't read offsets of segmented logs correctly
> -
>
> Key: KAFKA-5600
> URL: https://issues.apache.org/jira/browse/KAFKA-5600
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 0.10.2.1, 0.11.0.0
> Environment: any
>Reporter: Jan Burkhardt
>Priority: Critical
>  Labels: regression, reliability
> Fix For: 0.11.0.1
>
> Attachments: KafkaErrorConsumer.java, KafkaErrorProducer.java
>
>
> After long investigation we found a Problem in Kafka.
> When a __consumer_offsets partition gets segmented and Kafka is restarted and 
> needs to reload offsets, consumers will start at a wrong position when 
> metadata and offset events are in both segments.
> Reproduction:
> 1.) Start zookeeper and kafka as is from the archive
> {code}
> KAFKA_HEAP_OPTS="-Xmx256M -Xms256M" bin/zookeeper-server-start.sh 
> config/zookeeper.properties
> KAFKA_HEAP_OPTS="-Xmx256M -Xms256M" bin/kafka-server-start.sh 
> config/server.properties
> {code}
> 2.) Start [^KafkaErrorProducer.java] which adds 1M log entries to the topic 
> test
> 3.) Start [^KafkaErrorConsumer.java] which starts a consumer, reads 100 
> entries one by one and then closes the consumer. This leads to a 2nd segment 
> in /tmp/kafka-logs/__consumer_offsets-27. This step takes some time (around 
> 5mins). The close of the consumer is needed to have metadata events in the 
> segments too.
> 4.) Stop and restart the Kafka broker
> 5.) Start any consumer on topic test and group testgroup
> {code}
> bin/kafka-console-consumer.sh --from-beginning --bootstrap-server 
> localhost:9092 --topic test --consumer-property group.id=testgroup
> {code}
> Is:
> the consumer starts at the segmentation boundary
> Expected:
> the consumer starts at the end
> The Reason for this behavior is the closing brace of the while loop in 
> GroupMetadataManager#loadGroupsAndOffsets at a wrong position introduced with 
> commit 
> https://github.com/apache/kafka/commit/5bd06f1d542e6b588a1d402d059bc24690017d32
> I will prepare a pull request.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Updated] (KAFKA-5600) GroupMetadataManager doesn't read offsets of segmented logs correctly

2017-07-17 Thread Ismael Juma (JIRA)

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

Ismael Juma updated KAFKA-5600:
---
Labels: reliability  (was: )

> GroupMetadataManager doesn't read offsets of segmented logs correctly
> -
>
> Key: KAFKA-5600
> URL: https://issues.apache.org/jira/browse/KAFKA-5600
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 0.10.2.0, 0.11.0.0
> Environment: any
>Reporter: Jan Burkhardt
>  Labels: reliability
> Fix For: 0.11.0.1
>
> Attachments: KafkaErrorConsumer.java, KafkaErrorProducer.java
>
>
> After long investigation we found a Problem in Kafka.
> When a __consumer_offsets partition gets segmented and Kafka is restarted and 
> needs to reload offsets, consumers will start at a wrong position when 
> metadata and offset events are in both segments.
> Reproduction:
> 1.) Start zookeeper and kafka as is from the archive
> {code}
> KAFKA_HEAP_OPTS="-Xmx256M -Xms256M" bin/zookeeper-server-start.sh 
> config/zookeeper.properties
> KAFKA_HEAP_OPTS="-Xmx256M -Xms256M" bin/kafka-server-start.sh 
> config/server.properties
> {code}
> 2.) Start [^KafkaErrorProducer.java] which adds 1M log entries to the topic 
> test
> 3.) Start [^KafkaErrorConsumer.java] which starts a consumer, reads 100 
> entries one by one and then closes the consumer. This leads to a 2nd segment 
> in /tmp/kafka-logs/__consumer_offsets-27. This step takes some time (around 
> 5mins). The close of the consumer is needed to have metadata events in the 
> segments too.
> 4.) Stop and restart the Kafka broker
> 5.) Start any consumer on topic test and group testgroup
> {code}
> bin/kafka-console-consumer.sh --from-beginning --bootstrap-server 
> localhost:9092 --topic test --consumer-property group.id=testgroup
> {code}
> Is:
> the consumer starts at the segmentation boundary
> Expected:
> the consumer starts at the end
> The Reason for this behavior is the closing brace of the while loop in 
> GroupMetadataManager#loadGroupsAndOffsets at a wrong position introduced with 
> commit 
> https://github.com/apache/kafka/commit/5bd06f1d542e6b588a1d402d059bc24690017d32
> I will prepare a pull request.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Updated] (KAFKA-5600) GroupMetadataManager doesn't read offsets of segmented logs correctly

2017-07-17 Thread Ismael Juma (JIRA)

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

Ismael Juma updated KAFKA-5600:
---
Fix Version/s: 0.11.0.1

> GroupMetadataManager doesn't read offsets of segmented logs correctly
> -
>
> Key: KAFKA-5600
> URL: https://issues.apache.org/jira/browse/KAFKA-5600
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 0.10.2.0, 0.11.0.0
> Environment: any
>Reporter: Jan Burkhardt
>  Labels: reliability
> Fix For: 0.11.0.1
>
> Attachments: KafkaErrorConsumer.java, KafkaErrorProducer.java
>
>
> After long investigation we found a Problem in Kafka.
> When a __consumer_offsets partition gets segmented and Kafka is restarted and 
> needs to reload offsets, consumers will start at a wrong position when 
> metadata and offset events are in both segments.
> Reproduction:
> 1.) Start zookeeper and kafka as is from the archive
> {code}
> KAFKA_HEAP_OPTS="-Xmx256M -Xms256M" bin/zookeeper-server-start.sh 
> config/zookeeper.properties
> KAFKA_HEAP_OPTS="-Xmx256M -Xms256M" bin/kafka-server-start.sh 
> config/server.properties
> {code}
> 2.) Start [^KafkaErrorProducer.java] which adds 1M log entries to the topic 
> test
> 3.) Start [^KafkaErrorConsumer.java] which starts a consumer, reads 100 
> entries one by one and then closes the consumer. This leads to a 2nd segment 
> in /tmp/kafka-logs/__consumer_offsets-27. This step takes some time (around 
> 5mins). The close of the consumer is needed to have metadata events in the 
> segments too.
> 4.) Stop and restart the Kafka broker
> 5.) Start any consumer on topic test and group testgroup
> {code}
> bin/kafka-console-consumer.sh --from-beginning --bootstrap-server 
> localhost:9092 --topic test --consumer-property group.id=testgroup
> {code}
> Is:
> the consumer starts at the segmentation boundary
> Expected:
> the consumer starts at the end
> The Reason for this behavior is the closing brace of the while loop in 
> GroupMetadataManager#loadGroupsAndOffsets at a wrong position introduced with 
> commit 
> https://github.com/apache/kafka/commit/5bd06f1d542e6b588a1d402d059bc24690017d32
> I will prepare a pull request.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Updated] (KAFKA-5600) GroupMetadataManager doesn't read offsets of segmented logs correctly

2017-07-17 Thread Jan Burkhardt (JIRA)

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

Jan Burkhardt updated KAFKA-5600:
-
Description: 
After long investigation we found a Problem in Kafka.
When a __consumer_offsets partition gets segmented and Kafka is restarted and 
needs to reload offsets, consumers will start at a wrong position when metadata 
and offset events are in both segments.

Reproduction:
1.) Start zookeeper and kafka as is from the archive
{code}
KAFKA_HEAP_OPTS="-Xmx256M -Xms256M" bin/zookeeper-server-start.sh 
config/zookeeper.properties
KAFKA_HEAP_OPTS="-Xmx256M -Xms256M" bin/kafka-server-start.sh 
config/server.properties
{code}
2.) Start [^KafkaErrorProducer.java] which adds 1M log entries to the topic test
3.) Start [^KafkaErrorConsumer.java] which starts a consumer, reads 100 entries 
one by one and then closes the consumer. This leads to a 2nd segment in 
/tmp/kafka-logs/__consumer_offsets-27. This step takes some time (around 
5mins). The close of the consumer is needed to have metadata events in the 
segments too.
4.) Stop and restart the Kafka broker
5.) Start any consumer on topic test and group testgroup
{code}
bin/kafka-console-consumer.sh --from-beginning --bootstrap-server 
localhost:9092 --topic test --consumer-property group.id=testgroup
{code}

Is:
the consumer starts at the segmentation boundary
Expected:
the consumer starts at the end

The Reason for this behavior is the closing brace of the while loop in 
GroupMetadataManager#loadGroupsAndOffsets at a wrong position introduced with 
commit 
https://github.com/apache/kafka/commit/5bd06f1d542e6b588a1d402d059bc24690017d32
I will prepare a pull request.


  was:
After long investigation we found a Problem in Kafka.
When a __consumer_offsets partition gets segmented and Kafka is restarted and 
needs to reload offsets, consumers will start at a wrong position

Reproduction:
1.) Start zookeeper and kafka as is from the archive
{code}
KAFKA_HEAP_OPTS="-Xmx256M -Xms256M" bin/zookeeper-server-start.sh 
config/zookeeper.properties
KAFKA_HEAP_OPTS="-Xmx256M -Xms256M" bin/kafka-server-start.sh 
config/server.properties
{code}
2.) Start [^KafkaErrorProducer.java] which adds 1M log entries to the topic test
3.) Start [^KafkaErrorConsumer.java] which starts a consumer, reads 100 entries 
one by one and then closes the consumer. This leads to a 2nd segment in 
/tmp/kafka-logs/__consumer_offsets-27. This step takes some time (around 
5mins). The close of the consumer is needed to have metadata events in the 
segments too.
4.) Stop and restart the Kafka broker
5.) Start any consumer on topic test and group testgroup
{code}
bin/kafka-console-consumer.sh --from-beginning --bootstrap-server 
localhost:9092 --topic test --consumer-property group.id=testgroup
{code}

Is:
the consumer starts at the segmentation boundary
Expected:
the consumer starts at the end

The Reason for this behavior is the closing brace of the while loop in 
GroupMetadataManager#loadGroupsAndOffsets at a wrong position introduced with 
commit 
https://github.com/apache/kafka/commit/5bd06f1d542e6b588a1d402d059bc24690017d32
I will prepare a pull request.



> GroupMetadataManager doesn't read offsets of segmented logs correctly
> -
>
> Key: KAFKA-5600
> URL: https://issues.apache.org/jira/browse/KAFKA-5600
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 0.10.2.0, 0.11.0.0
> Environment: any
>Reporter: Jan Burkhardt
> Attachments: KafkaErrorConsumer.java, KafkaErrorProducer.java
>
>
> After long investigation we found a Problem in Kafka.
> When a __consumer_offsets partition gets segmented and Kafka is restarted and 
> needs to reload offsets, consumers will start at a wrong position when 
> metadata and offset events are in both segments.
> Reproduction:
> 1.) Start zookeeper and kafka as is from the archive
> {code}
> KAFKA_HEAP_OPTS="-Xmx256M -Xms256M" bin/zookeeper-server-start.sh 
> config/zookeeper.properties
> KAFKA_HEAP_OPTS="-Xmx256M -Xms256M" bin/kafka-server-start.sh 
> config/server.properties
> {code}
> 2.) Start [^KafkaErrorProducer.java] which adds 1M log entries to the topic 
> test
> 3.) Start [^KafkaErrorConsumer.java] which starts a consumer, reads 100 
> entries one by one and then closes the consumer. This leads to a 2nd segment 
> in /tmp/kafka-logs/__consumer_offsets-27. This step takes some time (around 
> 5mins). The close of the consumer is needed to have metadata events in the 
> segments too.
> 4.) Stop and restart the Kafka broker
> 5.) Start any consumer on topic test and group testgroup
> {code}
> bin/kafka-console-consumer.sh --from-beginning --bootstrap-server 
> localhost:9092 --topic test --consumer-property group.id=testgroup
> {code}
> Is:
> the consumer starts at the