[jira] [Commented] (KAFKA-5600) Group loading regression causing stale metadata/offsets cache

2017-09-11 Thread Jan Burkhardt (JIRA)

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

Jan Burkhardt commented on KAFKA-5600:
--

we have build a patched 0.11.0 version
https://github.com/justsocialapps/kafka/tree/0.11.0-KAFKA-5600

you can use this if you want, we are using it in production
so long as i know there is only a scheduled v1.0.0 release in october

> Group loading regression causing stale metadata/offsets cache
> -
>
> 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
>Assignee: Jan Burkhardt
>Priority: Critical
>  Labels: regression, reliability
> Fix For: 0.11.0.1, 1.0.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 
> 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



--
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-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 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 

[jira] [Updated] (KAFKA-5600) GroupMetadataManager doesn't read Offsets of segmented log 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

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



  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. 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




> GroupMetadataManager doesn't read Offsets of segmented log 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
> 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 

[jira] [Updated] (KAFKA-5600) GroupMetadataManager doesn't read Offsets of segmented log 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

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. 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



  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 
__consumer_offsets-27. This step takes some time. The close of the consumer is 
needed to also have metadata events in the segments
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




> GroupMetadataManager doesn't read Offsets of segmented log 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
> 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. 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 
> 

[jira] [Updated] (KAFKA-5600) GroupMetadataManager doesn't read Offsets of segmented log 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 consumer 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 
__consumer_offsets-27. This step takes some time. The close of the consumer is 
needed to also have metadata events in the segments
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



  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 it starts 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 
__consumer_offsets-27. This step takes some time. The close of the consumer is 
needed to also have metadata events in the segments
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




> GroupMetadataManager doesn't read Offsets of segmented log 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 consumer 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 __consumer_offsets-27. This step takes some time. The close of the 
> consumer is needed to also have metadata events in the segments
> 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 
> 

[jira] [Updated] (KAFKA-5600) GroupMetadataManager doesn't read Offsets of segmented log 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 it starts 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 
__consumer_offsets-27. This step takes some time. The close of the consumer is 
needed to also have metadata events in the segments
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 5bd06f1d542e6b588a1d402d059bc24690017d32



  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 it starts 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 
__consumer_offsets-27. This step takes some time. The close of the consumer is 
needed to also have metadata events in the segments
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




> GroupMetadataManager doesn't read Offsets of segmented log 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 it starts 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 __consumer_offsets-27. This step takes some time. The close of the 
> consumer is needed to also have metadata events in the segments
> 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 5bd06f1d542e6b588a1d402d059bc24690017d32



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


[jira] [Updated] (KAFKA-5600) GroupMetadataManager doesn't read Offsets of segmented log 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 it starts 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 
__consumer_offsets-27. This step takes some time. The close of the consumer is 
needed to also have metadata events in the segments
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



  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 it starts 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 
__consumer_offsets-27
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




> GroupMetadataManager doesn't read Offsets of segmented log 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 it starts 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 __consumer_offsets-27. This step takes some time. The close of the 
> consumer is needed to also have metadata events in the segments
> 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



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