[ https://issues.apache.org/jira/browse/KAFKA-5396?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16040309#comment-16040309 ]
Apurva Mehta commented on KAFKA-5396: ------------------------------------- Running the following {noformat} amehta-macbook-pro:3 apurva$ grep output-topic-1 ConsoleConsumer-0-139710924924944/knode03/console_consumer.log | grep 'Returning fetched records' | less {noformat} Show shat the fetch position for this partition resets: {noformat} [2017-06-07 05:50:34,474] TRACE Returning fetched records at offset 0 for assigned partition output-topic-1 and update position to 250 (org.apache.kafka.clients.consumer.internals.Fetcher) [2017-06-07 05:50:34,613] TRACE Returning fetched records at offset 250 for assigned partition output-topic-1 and update position to 500 (org.apache.kafka.clients.consumer.internals.Fetcher) [2017-06-07 05:50:34,648] TRACE Returning fetched records at offset 500 for assigned partition output-topic-1 and update position to 502 (org.apache.kafka.clients.consumer.internals.Fetcher) [2017-06-07 05:50:34,883] TRACE Returning fetched records at offset 502 for assigned partition output-topic-1 and update position to 753 (org.apache.kafka.clients.consumer.internals.Fetcher) [2017-06-07 05:50:35,033] TRACE Returning fetched records at offset 753 for assigned partition output-topic-1 and update position to 1004 (org.apache.kafka.clients.consumer.internals.Fetcher) [2017-06-07 05:50:35,285] TRACE Returning fetched records at offset 1004 for assigned partition output-topic-1 and update position to 1255 (org.apache.kafka.clients.consumer.internals.Fetcher) [2017-06-07 05:50:35,442] TRACE Returning fetched records at offset 1255 for assigned partition output-topic-1 and update position to 1505 (org.apache.kafka.clients.consumer.internals.Fetcher) [2017-06-07 05:50:35,449] TRACE Returning fetched records at offset 1505 for assigned partition output-topic-1 and update position to 1506 (org.apache.kafka.clients.consumer.internals.Fetcher) [2017-06-07 05:50:35,583] TRACE Returning fetched records at offset 1506 for assigned partition output-topic-1 and update position to 1757 (org.apache.kafka.clients.consumer.internals.Fetcher) [2017-06-07 05:50:35,741] TRACE Returning fetched records at offset 1757 for assigned partition output-topic-1 and update position to 2008 (org.apache.kafka.clients.consumer.internals.Fetcher) [2017-06-07 05:50:36,048] TRACE Returning fetched records at offset 2008 for assigned partition output-topic-1 and update position to 2508 (org.apache.kafka.clients.consumer.internals.Fetcher) [2017-06-07 05:50:36,102] TRACE Returning fetched records at offset 2508 for assigned partition output-topic-1 and update position to 2510 (org.apache.kafka.clients.consumer.internals.Fetcher) {noformat} It continues like this upto position 12801, and then goes back to 0: {noformat} [2017-06-07 05:51:32,922] TRACE Returning fetched records at offset 12551 for assigned partition output-topic-1 and update position to 12801 (org.apache.kafka.clients.consumer.internals.Fetcher) [2017-06-07 05:51:32,951] TRACE Returning fetched records at offset 12801 for assigned partition output-topic-1 and update position to 12801 (org.apache.kafka.clients.consumer.internals.Fetcher) [2017-06-07 05:51:36,391] TRACE Returning fetched records at offset 0 for assigned partition output-topic-1 and update position to 500 (org.apache.kafka.clients.consumer.internals.Fetcher) [2017-06-07 05:51:36,590] TRACE Returning fetched records at offset 500 for assigned partition output-topic-1 and update position to 1003 (org.apache.kafka.clients.consumer.internals.Fetcher) [2017-06-07 05:51:36,598] TRACE Returning fetched records at offset 1003 for assigned partition output-topic-1 and update position to 1505 (org.apache.kafka.clients.consumer.internals.Fetcher) [2017-06-07 05:51:36,610] TRACE Returning fetched records at offset 1505 for assigned partition output-topic-1 and update position to 2007 (org.apache.kafka.clients.consumer.internals.Fetcher) [2017-06-07 05:51:36,619] TRACE Returning fetched records at offset 2007 for assigned partition output-topic-1 and update position to 2508 (org.apache.kafka.clients.consumer.internals.Fetcher) [2017-06-07 05:51:36,624] TRACE Returning fetched records at offset 2508 for assigned partition output-topic-1 and update position to 3011 (org.apache.kafka.clients.consumer.internals.Fetcher) [2017-06-07 05:51:36,631] TRACE Returning fetched records at offset 3011 for assigned partition output-topic-1 and update position to 3513 (org.apache.kafka.clients.consumer.internals.Fetcher) [2017-06-07 05:51:36,635] TRACE Returning fetched records at offset 3513 for assigned partition output-topic-1 and update position to 4014 (org.apache.kafka.clients.consumer.internals.Fetcher) {noformat} > Consumer reading from beginning of log can read the same message multiple > times. > -------------------------------------------------------------------------------- > > Key: KAFKA-5396 > URL: https://issues.apache.org/jira/browse/KAFKA-5396 > Project: Kafka > Issue Type: Bug > Reporter: Apurva Mehta > Attachments: KAFKA-5396.tar.gz > > > I noticed this when running the transactions system test with hard broker > bounces. We have a consumer in READ_COMMITTED mode reading from the tail of > the log as the writes are appended. > This test has failed once because the concurrent consumer returned duplicate > data. The actual log has no duplicates, so the problem is in the consumer. > One of the duplicate values is '0', and is at offset 250 in output-topic-1. > The first time it is read, we see the following. > {noformat} > [2017-06-07 05:50:34,601] TRACE Returning fetched records at offset 0 for > assigned partition output-topic-0 and update position to 250 > (org.apache.kafka.clients.consumer.internals.Fetcher) > [2017-06-07 05:50:34,602] TRACE Preparing to read 2967 bytes of data for > partition output-topic-1 with offset 250 > (org.apache.kafka.clients.consumer.internals.Fetcher) > [2017-06-07 05:50:34,602] TRACE Updating high watermark for partition > output-topic-1 to 502 (org.apache.kafka.clients.consumer.internals.Fetcher) > [2017-06-07 05:50:34,613] TRACE Returning fetched records at offset 250 for > assigned partition output-topic-1 and update position to 500 > (org.apache.kafka.clients.consumer.internals.Fetcher) > {noformat} > The next time it is read, we see this > {noformat} > [2017-06-07 05:51:36,386] TRACE Preparing to read 169858 bytes of data for > partition output-topic-1 with offset 0 > (org.apache.kafka.clients.consumer.internals.Fetcher) > [2017-06-07 05:51:36,389] TRACE Updating high watermark for partition > output-topic-1 to 13053 (org.apache.kafka.clients.consumer.internals.Fetcher) > [2017-06-07 05:51:36,391] TRACE Returning fetched records at offset 0 for > assigned partition output-topic-1 and update position to 500 > (org.apache.kafka.clients.consumer.internals.Fetcher) > {noformat} > For some reason, the fetcher re-sent the data from offset 0, an reset the > position to 500. > This is the plain consumer doing 'poll' in a loop until it is killed. So this > position reset is puzzling. -- This message was sent by Atlassian JIRA (v6.3.15#6346)