Hi,

Your log segment dump and the producer log don't correlate. The producer
log shows the producerId == 4001. But your log segment dumps don't have
this producerId. Please share data from the same run where you reproduce
this issue.

For the producerId's 0-4 (shown in the dump), there seem to be no
transaction markers (because these would have sequence number == -1). So if
your messages from producerId 4001 are behind these messages, they would
never be read in read committed mode.

Thanks,
Apurva

On Mon, Nov 6, 2017 at 9:44 PM, Abhishek Verma <abhishekverma3...@gmail.com>
wrote:

> Hi Matthis J. Sax,
>
> Thank you for your suggestions.
>
> I tried the same in kafka 1.0.0 version also. Same issue is coming.
>
> I am attaching log segment below please let me know what might be the
> problem.
>
> Regards,
> Abhishek Verma
>
> <dump Log segment>
>
>
>
> Dumping 00000000000000000000.index
>
> offset: 0 position: 0
>
> Dumping 00000000000000000000.log
>
> Starting offset: 0
>
> baseOffset: 0 lastOffset: 0 baseSequence: 0 lastSequence: 0 producerId: 0
> producerEpoch: 0 partitionLeaderEpoch: 0 isTransactional: true position: 0
> CreateTime: 1509605714710 isvalid: true size: 103 magic: 2 compresscodec:
> NONE crc:344974185
>
> baseOffset: 1 lastOffset: 1 baseSequence: 1 lastSequence: 1 producerId: 0
> producerEpoch: 0 partitionLeaderEpoch: 0 isTransactional: true position:
> 103 CreateTime: 1509605714863 isvalid: true size: 103 magic: 2
> compresscodec: NONE crc:102431214
>
> baseOffset: 2 lastOffset: 2 baseSequence: 0 lastSequence: 0 producerId: 1
> producerEpoch: 0 partitionLeaderEpoch: 0 isTransactional: true position:
> 206 CreateTime: 1509607351944 isvalid: true size: 103 magic: 2
> compresscodec: NONE crc:1129944557
>
> baseOffset: 3 lastOffset: 3 baseSequence: 0 lastSequence: 0 producerId: 2
> producerEpoch: 0 partitionLeaderEpoch: 0 isTransactional: true position:
> 309 CreateTime: 1509616649669 isvalid: true size: 110 magic: 2
> compresscodec: NONE crc:630443129
>
> baseOffset: 4 lastOffset: 4 baseSequence: 0 lastSequence: 0 producerId: 3
> producerEpoch: 0 partitionLeaderEpoch: 0 isTransactional: true position:
> 419 CreateTime: 1509616850564 isvalid: true size: 110 magic: 2
> compresscodec: NONE crc:3357473778
>
> baseOffset: 5 lastOffset: 5 baseSequence: 0 lastSequence: 0 producerId: 4
> producerEpoch: 0 partitionLeaderEpoch: 0 isTransactional: true position:
> 529 CreateTime: 1509624206511 isvalid: true size: 110 magic: 2
> compresscodec: NONE crc:1193735168
>
> baseOffset: 6 lastOffset: 6 baseSequence: 0 lastSequence: 0 producerId: 5
> producerEpoch: 0 partitionLeaderEpoch: 0 isTransactional: true position:
> 639 CreateTime: 1509624453377 isvalid: true size: 110 magic: 2
> compresscodec: NONE crc:3859361029
>
> Dumping 00000000000000000000.timeindex
>
> timestamp: 0 offset: 0
>
> Found timestamp mismatch in :D:\tmp\kafka-logs-0\topic-0\
> 00000000000000000000.timeindex
>
> Index timestamp: 0, log timestamp: 1509605714710
>
> Index timestamp: 0, log timestamp: 1509605714710
>
> Found out of order timestamp in :D:\tmp\kafka-logs-0\topic-0\
> 00000000000000000000.timeindex
>
> Index timestamp: 0, Previously indexed timestamp: 0
>
>
>
> </dumpet log segment>
> ________________________________
> From: Matthias J. Sax <matth...@confluent.io>
> Sent: Saturday, November 4, 2017 8:11:07 PM
> To: users@kafka.apache.org
> Subject: Re: Reg. Kafka transactional producer and consumer
>
> Hi,
>
> this consumer log line indicates that there is an open/pending
> transaction (ie, neither committed nor aborted) and thus, the broker
> does not deliver the data to the consumer.
>
> -> highWaterMark = 5, but lastStableOffset = 0
>
>
> On 11/2/17 5:25 AM, Abhishek Verma wrote:
> > 1871 [main] DEBUG org.apache.kafka.clients.consumer.internals.Fetcher
> - Fetch READ_COMMITTED at offset 0 for partition topic-4-0 returned fetch
> data (error=NONE, highWaterMark=5, lastStableOffset = 0, logStartOffset =
> 0, abortedTransactions = [], recordsSizeInBytes=0)
>
>
> Thus, there must be an issue on the producer side, that the transactions
> does not get committed. Not sure why though, as producer logs indicate
> that the TX was committed successfully.
>
> Maybe you can dump the log segments to see what is in them?
>
> Btw: Kafka 1.0.0 was release recently, containing several bug fixes for
> transactions. Maybe you can try if it fixed in 1.0.0.
>
>
> -Matthias
>
>

Reply via email to