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

Ben Stopford updated KAFKA-2908:
--------------------------------
    Description: 
*Context:*
Instance of the rolling upgrade test. 10s sleeps have been put around node 
restarts to ensure stability when subsequent nodes go down. Consumer timeout 
has been set to 60s. Proudcer has been throttled to 100 messages per second. 

Failure is rare: It occurred once in 60 executions (6 executions per run #276 
-> #285 of the system_test_branch_builder)

*Reported Failure:*

At least one acked message did not appear in the consumed messages. 
acked_minus_consumed: 16385, 16388, 16391, 16394, 16397, 16400, 16403, 16406, 
16409, 16412, 16415, 16418, 16421, 16424, 16427, 16430, 16433, 16436, 16439, 
16442, ...plus 1669 more

*Immediate Observations:*

* The list of messages not consumed are all in partition 1.
* Production and Consumption continues throughout the test (there is no 
complete write or read failure as we have seen elsewhere)
* The messages ARE present in the log file:
e.g. 
{quote}
Examining missing value 16385. This was written to p1,offset:5453
=> there is an entry in all data files for partition 1 for this (presumably 
meaning it was replicated correctly)

kafka/bin/kafka-run-class.sh kafka.tools.DumpLogSegments --files 
worker10/kafka-data-logs/test_topic-1/00000000000000000000.log | grep 'offset: 
5453'

worker10,9,8 repectively:
offset: 5453 position: 165346 isvalid: true payloadsize: 5 magic: 0 
compresscodec: NoCompressionCodec crc: 1502953075
offset: 5453 position: 165346 isvalid: true payloadsize: 5 magic: 0 
compresscodec: NoCompressionCodec crc: 1502953075
offset: 5453 position: 165346 isvalid: true payloadsize: 5 magic: 0 
compresscodec: NoCompressionCodec crc: 1502953075 
{quote}

These entries definitely do not appear in the consumer stdout file either. 

*Timeline:*
{quote}
15:27:02,232 - Producer sends first message
..servers 1, 2 are restarted (clean shutdown)
15:29:42,718 - Server 3 shutdown complete
15:29:42,712 - (Controller fails over): Broker 2 starting become controller 
state transition (kafka.controller.KafkaController)
15:29:42,743 - New leasder is 2 (LeaderChangeListner)
15:29:43,239 - WARN Broker 2 ignoring LeaderAndIsr request from controller 2 
with correlation id 0 epoch 7 for partition (test_topic,1) since its associated 
leader epoch 8 is old. Current leader epoch is 8 (state.change.logger)
15:29:45,642 - Producer starts writing messages that are never consumed
15:30:10,804 - Last message sent by producer
15:31:10,983 - Consumer times out after 60s wait without messages
{quote}

Logs for this run are attached



  was:
*Context:*
Instance of the rolling upgrade test. 10s sleeps have been put around node 
restarts to ensure stability when subsequent nodes go down. Consumer timeout 
has been set to 60s. Proudcer has been throttled to 100 messages per second. 

Failure is rare: It occurred once in 60 executions (6 executions per run #276 
-> #285 of the system_test_branch_builder)

*Reported Failure:*

At least one acked message did not appear in the consumed messages. 
acked_minus_consumed: 16385, 16388, 16391, 16394, 16397, 16400, 16403, 16406, 
16409, 16412, 16415, 16418, 16421, 16424, 16427, 16430, 16433, 16436, 16439, 
16442, ...plus 1669 more

*Immediate Observations:*

* The list of messages not consumed are all in partition 1.
* The messages ARE present in the log file:
e.g. 
{quote}
Examining missing value 16385. This was written to p1,offset:5453
=> there is an entry in all data files for partition 1 for this (presumably 
meaning it was replicated correctly)

kafka/bin/kafka-run-class.sh kafka.tools.DumpLogSegments --files 
worker10/kafka-data-logs/test_topic-1/00000000000000000000.log | grep 'offset: 
5453'

worker10,9,8 repectively:
offset: 5453 position: 165346 isvalid: true payloadsize: 5 magic: 0 
compresscodec: NoCompressionCodec crc: 1502953075
offset: 5453 position: 165346 isvalid: true payloadsize: 5 magic: 0 
compresscodec: NoCompressionCodec crc: 1502953075
offset: 5453 position: 165346 isvalid: true payloadsize: 5 magic: 0 
compresscodec: NoCompressionCodec crc: 1502953075 
{quote}

These entries definitely do not appear in the consumer stdout file either. 

*Timeline:*
{quote}
15:27:02,232 - Producer sends first message
..servers 1, 2 are restarted (clean shutdown)
15:29:42,718 - Server 3 shutdown complete
15:29:42,712 - (Controller fails over): Broker 2 starting become controller 
state transition (kafka.controller.KafkaController)
15:29:42,743 - New leasder is 2 (LeaderChangeListner)
15:29:43,239 - WARN Broker 2 ignoring LeaderAndIsr request from controller 2 
with correlation id 0 epoch 7 for partition (test_topic,1) since its associated 
leader epoch 8 is old. Current leader epoch is 8 (state.change.logger)
15:29:45,642 - Producer starts writing messages that are never consumed
15:30:10,804 - Last message sent by producer
15:31:10,983 - Consumer times out after 60s wait without messages
{quote}

Logs for this run are attached




> Instance of Gap in Consumption after Restart
> --------------------------------------------
>
>                 Key: KAFKA-2908
>                 URL: https://issues.apache.org/jira/browse/KAFKA-2908
>             Project: Kafka
>          Issue Type: Sub-task
>          Components: consumer
>            Reporter: Ben Stopford
>         Attachments: 2015-11-28--001.tar.gz
>
>
> *Context:*
> Instance of the rolling upgrade test. 10s sleeps have been put around node 
> restarts to ensure stability when subsequent nodes go down. Consumer timeout 
> has been set to 60s. Proudcer has been throttled to 100 messages per second. 
> Failure is rare: It occurred once in 60 executions (6 executions per run #276 
> -> #285 of the system_test_branch_builder)
> *Reported Failure:*
> At least one acked message did not appear in the consumed messages. 
> acked_minus_consumed: 16385, 16388, 16391, 16394, 16397, 16400, 16403, 16406, 
> 16409, 16412, 16415, 16418, 16421, 16424, 16427, 16430, 16433, 16436, 16439, 
> 16442, ...plus 1669 more
> *Immediate Observations:*
> * The list of messages not consumed are all in partition 1.
> * Production and Consumption continues throughout the test (there is no 
> complete write or read failure as we have seen elsewhere)
> * The messages ARE present in the log file:
> e.g. 
> {quote}
> Examining missing value 16385. This was written to p1,offset:5453
> => there is an entry in all data files for partition 1 for this (presumably 
> meaning it was replicated correctly)
> kafka/bin/kafka-run-class.sh kafka.tools.DumpLogSegments --files 
> worker10/kafka-data-logs/test_topic-1/00000000000000000000.log | grep 
> 'offset: 5453'
> worker10,9,8 repectively:
> offset: 5453 position: 165346 isvalid: true payloadsize: 5 magic: 0 
> compresscodec: NoCompressionCodec crc: 1502953075
> offset: 5453 position: 165346 isvalid: true payloadsize: 5 magic: 0 
> compresscodec: NoCompressionCodec crc: 1502953075
> offset: 5453 position: 165346 isvalid: true payloadsize: 5 magic: 0 
> compresscodec: NoCompressionCodec crc: 1502953075 
> {quote}
> These entries definitely do not appear in the consumer stdout file either. 
> *Timeline:*
> {quote}
> 15:27:02,232 - Producer sends first message
> ..servers 1, 2 are restarted (clean shutdown)
> 15:29:42,718 - Server 3 shutdown complete
> 15:29:42,712 - (Controller fails over): Broker 2 starting become controller 
> state transition (kafka.controller.KafkaController)
> 15:29:42,743 - New leasder is 2 (LeaderChangeListner)
> 15:29:43,239 - WARN Broker 2 ignoring LeaderAndIsr request from controller 2 
> with correlation id 0 epoch 7 for partition (test_topic,1) since its 
> associated leader epoch 8 is old. Current leader epoch is 8 
> (state.change.logger)
> 15:29:45,642 - Producer starts writing messages that are never consumed
> 15:30:10,804 - Last message sent by producer
> 15:31:10,983 - Consumer times out after 60s wait without messages
> {quote}
> Logs for this run are attached



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to