Description
Description
It is possible to get duplicated messages in a single topic-partition. It happens despite configuring the producer for exactly-once delivery. It occurs sporadicly during the partition leader change, where for some reason the producer epoch is also being incremented. It is not trivial to reproduce, but it is fairly doable. (I can provide repro-steps, but I think it is not helpful as the attached pcap file contains enough details to analyze this issue).
The essential configuration bits for the producer are:
--replication-factor=3
--min-isr=2
--config request.required.acks=-1
--config enable.idempotence=true
--config max.in.flight.requests.per.connection=5
Details
Below is the excerpt from the captured packets demonstrating the problem.
The producer app was writing a sequence of numbers (strictly incrementing) into my-topic-0188
. Because of the topic-partition leadership change, the values 0x0a90
and 0x0a91
were written twice. This breaks exactly-once delivery guarantees and can cause data inconsistencies in applications relying on message deduplication.
It is unclear why these values were written twice as there is only one successful confirmation of writing them.
What is also interesting, the response from frame 1566024, has a valid offset but also a non-zero error code.
Confluent.Kafka(Librdkafka) version
Confluent.Kafka, Version=2.6.1.0, Culture=neutral, PublicKeyToken=12c514ca49093d1em
librdkafka Version: 2.6.1 (20601FF)
Produce requests
1560411, 15:37:11.568610, 192.168.65.3:65494->3.70.194.188:40001 - Kafka Produce v10 Request
Kafka (Produce v10 Request)
Length: 129
API Key: Produce (0)
API Version: 10
Correlation ID: 3561
Client ID: rdkafka
Tagged fields
Transactional ID: [ Null ]
Required Acks: Full ISR (-1)
Timeout: 30000
Topic (Name=my-topic-0188)
Topic Name: my-topic-0188
Partition (ID=0)
Partition ID: 0
Message Set
Record Batch
Offset: 0
Message Size: 68
Leader Epoch: 0
Magic Byte: 2
CRC32: 0x1ee6e86f
.... .... .... .000 = Compression Codec: None (0)
.... .... .... 0... = Timestamp Type: CreateTime (0)
.... .... ...0 .... = Transactional: Non-transactional (0)
.... .... ..0. .... = Control Batch: Data batch (0)
Last Offset Delta: 0
First Timestamp: Jan 1, 1970 00:06:44.-1728723361 UTC
Last Timestamp: Jan 1, 1970 00:06:44.-1728723361 UTC
Producer ID: 7078
Producer Epoch: 0
Base Sequence: 2703
Size: 1
Record
Record Attributes (reserved): 0
Timestamp: Jan 24, 2025 15:37:11.519000000 UTC
Offset: 0
Key: 00000000
Value: 0000000000000a8f
Headers
Tagged fields
Tagged fields
Tagged fields
1564645, 15:37:12.298735, 3.70.194.188:40001->192.168.65.3:65494 - Kafka Produce v10 Response
Kafka (Produce v10 Response)
Length: 60
Correlation ID: 3561
[Request Frame: 1560411]
[API Key: Produce (0)]
[API Version: 10]
Tagged fields
Topic
Topic Name: my-topic-0188
Partitions
Partition (ID=0, Offset=71228)
Partition ID: 0
Error: No Error (0)
Offset: 71228
Time: -1 (latest)
Log Start Offset: 0
Record Errors
Error Message: [ Null ]
Tagged fields
Tagged fields
Throttle time: 0
Tagged fields
1561308, 15:37:11.655040, 192.168.65.3:65494->3.70.194.188:40001 - Kafka Produce v10 Request
Kafka (Produce v10 Request)
Length: 148
API Key: Produce (0)
API Version: 10
Correlation ID: 3564
Client ID: rdkafka
Tagged fields
Transactional ID: [ Null ]
Required Acks: Full ISR (-1)
Timeout: 30000
Topic (Name=my-topic-0188)
Topic Name: my-topic-0188
Partition (ID=0)
Partition ID: 0
Message Set
Record Batch
Offset: 0
Message Size: 87
Leader Epoch: 0
Magic Byte: 2
CRC32: 0xe97ffa59
.... .... .... .000 = Compression Codec: None (0)
.... .... .... 0... = Timestamp Type: CreateTime (0)
.... .... ...0 .... = Transactional: Non-transactional (0)
.... .... ..0. .... = Control Batch: Data batch (0)
Last Offset Delta: 1
First Timestamp: Jan 1, 1970 00:06:44.-1728723327 UTC
Last Timestamp: Jan 1, 1970 00:06:44.-1728723293 UTC
Producer ID: 7078
Producer Epoch: 0
Base Sequence: 2704
Size: 2
Record
Record Attributes (reserved): 0
Timestamp: Jan 24, 2025 15:37:11.553000000 UTC
Offset: 0
Key: 00000000
Value: 0000000000000a90
Headers
Record
Record Attributes (reserved): 0
Timestamp: Jan 24, 2025 15:37:11.587000000 UTC
Offset: 1
Key: 00000000
Value: 0000000000000a91
Headers
Tagged fields
Tagged fields
Tagged fields
1566024, 15:37:12.623878, 3.70.194.188:40001->192.168.65.3:65494 - Kafka Produce v10 Response [Not Leader For Partition]
Kafka (Produce v10 Response)
Length: 97
Correlation ID: 3564
[Request Frame: 1561308]
[API Key: Produce (0)]
[API Version: 10]
Tagged fields
Topic
Topic Name: my-topic-0188
Partitions
Partition (ID=0, Offset=71229)
Partition ID: 0
Error: Not Leader For Partition (6)
Offset: 71229
Time: -1 (latest)
Log Start Offset: 0
Record Errors
Error Message: [ Null ]
Tagged fields
Field
Tag Value: 0x0000000000000000
Tag Data: 000000030000001200
Tagged fields
Throttle time: 0
Tagged fields
Field
Tag Value: 0x0000000000000000
Tag Data: 02000000030d332e37302e3139342e31383800009c430000
1566591, 15:37:12.769799, 192.168.65.3:58732->3.70.194.188:40003 - Kafka Produce v10 Request
Kafka (Produce v10 Request)
Length: 148
API Key: Produce (0)
API Version: 10
Correlation ID: 2474
Client ID: rdkafka
Tagged fields
Transactional ID: [ Null ]
Required Acks: Full ISR (-1)
Timeout: 30000
Topic (Name=my-topic-0188)
Topic Name: my-topic-0188
Partition (ID=0)
Partition ID: 0
Message Set
Record Batch
Offset: 0
Message Size: 87
Leader Epoch: 0
Magic Byte: 2
CRC32: 0xe97ffa59
.... .... .... .000 = Compression Codec: None (0)
.... .... .... 0... = Timestamp Type: CreateTime (0)
.... .... ...0 .... = Transactional: Non-transactional (0)
.... .... ..0. .... = Control Batch: Data batch (0)
Last Offset Delta: 1
First Timestamp: Jan 1, 1970 00:06:44.-1728723327 UTC
Last Timestamp: Jan 1, 1970 00:06:44.-1728723293 UTC
Producer ID: 7078
Producer Epoch: 0
Base Sequence: 2704
Size: 2
Record
Record Attributes (reserved): 0
Timestamp: Jan 24, 2025 15:37:11.553000000 UTC
Offset: 0
Key: 00000000
Value: 0000000000000a90
Headers
Record
Record Attributes (reserved): 0
Timestamp: Jan 24, 2025 15:37:11.587000000 UTC
Offset: 1
Key: 00000000
Value: 0000000000000a91
Headers
Tagged fields
Tagged fields
Tagged fields
1567506, 15:37:12.925713, 3.70.194.188:40003->192.168.65.3:58732 - Kafka Produce v10 Response [Not Leader For Partition]
Kafka (Produce v10 Response)
Length: 97
Correlation ID: 2474
[Request Frame: 1566591]
[API Key: Produce (0)]
[API Version: 10]
Tagged fields
Topic
Topic Name: my-topic-0188
Partitions
Partition (ID=0, Offset=-1)
Partition ID: 0
Error: Not Leader For Partition (6)
Offset: -1
Time: -1 (latest)
Log Start Offset: -1
Record Errors
Error Message: [ Null ]
Tagged fields
Field
Tag Value: 0x0000000000000000
Tag Data: 000000010000001100
Tagged fields
Throttle time: 0
Tagged fields
Field
Tag Value: 0x0000000000000000
Tag Data: 02000000010d332e37302e3139342e31383800009c410000
1568303, 15:37:13.040103, 192.168.65.3:58732->3.70.194.188:40003 - Kafka Produce v10 Request
Kafka (Produce v10 Request)
Length: 148
API Key: Produce (0)
API Version: 10
Correlation ID: 2478
Client ID: rdkafka
Tagged fields
Transactional ID: [ Null ]
Required Acks: Full ISR (-1)
Timeout: 30000
Topic (Name=my-topic-0188)
Topic Name: my-topic-0188
Partition (ID=0)
Partition ID: 0
Message Set
Record Batch
Offset: 0
Message Size: 87
Leader Epoch: 0
Magic Byte: 2
CRC32: 0xe97ffa59
.... .... .... .000 = Compression Codec: None (0)
.... .... .... 0... = Timestamp Type: CreateTime (0)
.... .... ...0 .... = Transactional: Non-transactional (0)
.... .... ..0. .... = Control Batch: Data batch (0)
Last Offset Delta: 1
First Timestamp: Jan 1, 1970 00:06:44.-1728723327 UTC
Last Timestamp: Jan 1, 1970 00:06:44.-1728723293 UTC
Producer ID: 7078
Producer Epoch: 0
Base Sequence: 2704
Size: 2
Record
Record Attributes (reserved): 0
Timestamp: Jan 24, 2025 15:37:11.553000000 UTC
Offset: 0
Key: 00000000
Value: 0000000000000a90
Headers
Record
Record Attributes (reserved): 0
Timestamp: Jan 24, 2025 15:37:11.587000000 UTC
Offset: 1
Key: 00000000
Value: 0000000000000a91
Headers
Tagged fields
Tagged fields
Tagged fields
1568678, 15:37:13.113566, 3.70.194.188:40003->192.168.65.3:58732 - Kafka Produce v10 Response [Not Leader For Partition]
Kafka (Produce v10 Response)
Length: 97
Correlation ID: 2478
[Request Frame: 1568303]
[API Key: Produce (0)]
[API Version: 10]
Tagged fields
Topic
Topic Name: my-topic-0188
Partitions
Partition (ID=0, Offset=-1)
Partition ID: 0
Error: Not Leader For Partition (6)
Offset: -1
Time: -1 (latest)
Log Start Offset: -1
Record Errors
Error Message: [ Null ]
Tagged fields
Field
Tag Value: 0x0000000000000000
Tag Data: 000000010000001100
Tagged fields
Throttle time: 0
Tagged fields
Field
Tag Value: 0x0000000000000000
Tag Data: 02000000010d332e37302e3139342e31383800009c410000
1569730, 15:37:13.280079, 192.168.65.3:58732->3.70.194.188:40003 - Kafka Produce v10 Request
Kafka (Produce v10 Request)
Length: 148
API Key: Produce (0)
API Version: 10
Correlation ID: 2482
Client ID: rdkafka
Tagged fields
Transactional ID: [ Null ]
Required Acks: Full ISR (-1)
Timeout: 30000
Topic (Name=my-topic-0188)
Topic Name: my-topic-0188
Partition (ID=0)
Partition ID: 0
Message Set
Record Batch
Offset: 0
Message Size: 87
Leader Epoch: 0
Magic Byte: 2
CRC32: 0xe97ffa59
.... .... .... .000 = Compression Codec: None (0)
.... .... .... 0... = Timestamp Type: CreateTime (0)
.... .... ...0 .... = Transactional: Non-transactional (0)
.... .... ..0. .... = Control Batch: Data batch (0)
Last Offset Delta: 1
First Timestamp: Jan 1, 1970 00:06:44.-1728723327 UTC
Last Timestamp: Jan 1, 1970 00:06:44.-1728723293 UTC
Producer ID: 7078
Producer Epoch: 0
Base Sequence: 2704
Size: 2
Record
Record Attributes (reserved): 0
Timestamp: Jan 24, 2025 15:37:11.553000000 UTC
Offset: 0
Key: 00000000
Value: 0000000000000a90
Headers
Record
Record Attributes (reserved): 0
Timestamp: Jan 24, 2025 15:37:11.587000000 UTC
Offset: 1
Key: 00000000
Value: 0000000000000a91
Headers
Tagged fields
Tagged fields
Tagged fields
1570055, 15:37:13.330732, 3.70.194.188:40003->192.168.65.3:58732 - Kafka Produce v10 Response [Not Leader For Partition]
Kafka (Produce v10 Response)
Length: 97
Correlation ID: 2482
[Request Frame: 1569730]
[API Key: Produce (0)]
[API Version: 10]
Tagged fields
Topic
Topic Name: my-topic-0188
Partitions
Partition (ID=0, Offset=-1)
Partition ID: 0
Error: Not Leader For Partition (6)
Offset: -1
Time: -1 (latest)
Log Start Offset: -1
Record Errors
Error Message: [ Null ]
Tagged fields
Field
Tag Value: 0x0000000000000000
Tag Data: 000000010000001100
Tagged fields
Throttle time: 0
Tagged fields
Field
Tag Value: 0x0000000000000000
Tag Data: 02000000010d332e37302e3139342e31383800009c410000
1570954, 15:37:13.507840, 192.168.65.3:58732->3.70.194.188:40003 - Kafka Produce v10 Request
Kafka (Produce v10 Request)
Length: 148
API Key: Produce (0)
API Version: 10
Correlation ID: 2489
Client ID: rdkafka
Tagged fields
Transactional ID: [ Null ]
Required Acks: Full ISR (-1)
Timeout: 30000
Topic (Name=my-topic-0188)
Topic Name: my-topic-0188
Partition (ID=0)
Partition ID: 0
Message Set
Record Batch
Offset: 0
Message Size: 87
Leader Epoch: 0
Magic Byte: 2
CRC32: 0xe97ffa59
.... .... .... .000 = Compression Codec: None (0)
.... .... .... 0... = Timestamp Type: CreateTime (0)
.... .... ...0 .... = Transactional: Non-transactional (0)
.... .... ..0. .... = Control Batch: Data batch (0)
Last Offset Delta: 1
First Timestamp: Jan 1, 1970 00:06:44.-1728723327 UTC
Last Timestamp: Jan 1, 1970 00:06:44.-1728723293 UTC
Producer ID: 7078
Producer Epoch: 0
Base Sequence: 2704
Size: 2
Record
Record Attributes (reserved): 0
Timestamp: Jan 24, 2025 15:37:11.553000000 UTC
Offset: 0
Key: 00000000
Value: 0000000000000a90
Headers
Record
Record Attributes (reserved): 0
Timestamp: Jan 24, 2025 15:37:11.587000000 UTC
Offset: 1
Key: 00000000
Value: 0000000000000a91
Headers
Tagged fields
Tagged fields
Tagged fields
1571293, 15:37:13.567244, 3.70.194.188:40003->192.168.65.3:58732 - Kafka Produce v10 Response [Not Leader For Partition]
Kafka (Produce v10 Response)
Length: 97
Correlation ID: 2489
[Request Frame: 1570954]
[API Key: Produce (0)]
[API Version: 10]
Tagged fields
Topic
Topic Name: my-topic-0188
Partitions
Partition (ID=0, Offset=-1)
Partition ID: 0
Error: Not Leader For Partition (6)
Offset: -1
Time: -1 (latest)
Log Start Offset: -1
Record Errors
Error Message: [ Null ]
Tagged fields
Field
Tag Value: 0x0000000000000000
Tag Data: 000000010000001100
Tagged fields
Throttle time: 0
Tagged fields
Field
Tag Value: 0x0000000000000000
Tag Data: 02000000010d332e37302e3139342e31383800009c410000
Metadata requests
1571548, 15:37:13.608869, 192.168.65.3:58732->3.70.194.188:40003 - Kafka Metadata v12 Request
Kafka (Metadata v12 Request)
Length: 84
API Key: Metadata (3)
API Version: 12
Correlation ID: 2492
Client ID: rdkafka
Tagged fields
Topic
Topic Name: my-topic-0188
Tagged fields
Topic ID: 00000000-0000-0000-0000-000000000000
Allow Auto Topic Creation: False
Include Topic Authorized Operations: False
Tagged fields
1571750, 15:37:13.642273, 3.70.194.188:40003->192.168.65.3:58732 - Kafka Metadata v12 Response
Kafka (Metadata v12 Response)
Length: 270
Correlation ID: 2492
[Request Frame: 1571548]
[API Key: Metadata (3)]
[API Version: 12]
Tagged fields
Throttle time: 0
Broker Metadata
Broker (node 1: 3.70.194.188:40001)
Node ID: 1
Host: 3.70.194.188
Port: 40001
Rack: [ Null ]
Tagged fields
Broker (node 2: 3.70.194.188:40002)
Node ID: 2
Host: 3.70.194.188
Port: 40002
Rack: [ Null ]
Tagged fields
Broker (node 3: 3.70.194.188:40003)
Node ID: 3
Host: 3.70.194.188
Port: 40003
Rack: [ Null ]
Tagged fields
Cluster ID: 4L6g3nShT-eMCtK--X86sw
Controller ID: 3
Topic Metadata
Topic (my-topic-0188)
Error: No Error (0)
Topic Name: my-topic-0188
Topic ID: f49b160c-8440-4ffb-8211-44a649a35366
Is Internal: False
Partition (ID=0)
Error: No Error (0)
Partition ID: 0
Leader ID: 3
Leader Epoch: 18
Replicas
Replica ID: 3
Replica ID: 1
Replica ID: 2
Caught-Up Replicas
Caught-Up Replica ID: 1
Caught-Up Replica ID: 2
Caught-Up Replica ID: 3
Offline Replicas
Tagged fields
Topic Authorized Operations: 0x80000000
Tagged fields
Tagged fields
Final Produce request (with incremented producer epoch)
1572355, 15:37:13.710099, 192.168.65.3:58732->3.70.194.188:40003 - Kafka Produce v10 Request
Kafka (Produce v10 Request)
Length: 148
API Key: Produce (0)
API Version: 10
Correlation ID: 2496
Client ID: rdkafka
Tagged fields
Transactional ID: [ Null ]
Required Acks: Full ISR (-1)
Timeout: 30000
Topic (Name=my-topic-0188)
Topic Name: my-topic-0188
Partition (ID=0)
Partition ID: 0
Message Set
Record Batch
Offset: 0
Message Size: 87
Leader Epoch: 0
Magic Byte: 2
CRC32: 0x1cbeb4b1
.... .... .... .000 = Compression Codec: None (0)
.... .... .... 0... = Timestamp Type: CreateTime (0)
.... .... ...0 .... = Transactional: Non-transactional (0)
.... .... ..0. .... = Control Batch: Data batch (0)
Last Offset Delta: 1
First Timestamp: Jan 1, 1970 00:06:44.-1728723327 UTC
Last Timestamp: Jan 1, 1970 00:06:44.-1728723293 UTC
Producer ID: 7078
Producer Epoch: 1
Base Sequence: 0
Size: 2
Record
Record Attributes (reserved): 0
Timestamp: Jan 24, 2025 15:37:11.553000000 UTC
Offset: 0
Key: 00000000
Value: 0000000000000a90
Headers
Record
Record Attributes (reserved): 0
Timestamp: Jan 24, 2025 15:37:11.587000000 UTC
Offset: 1
Key: 00000000
Value: 0000000000000a91
Headers
Tagged fields
Tagged fields
Tagged fields
1591784, 15:37:15.822886, 3.70.194.188:40003->192.168.65.3:58732 - Kafka Produce v10 Response
Kafka (Produce v10 Response)
Length: 60
Correlation ID: 2496
[Request Frame: 1572355]
[API Key: Produce (0)]
[API Version: 10]
Tagged fields
Topic
Topic Name: my-topic-0188
Partitions
Partition (ID=0, Offset=71231)
Partition ID: 0
Error: No Error (0)
Offset: 71231
Time: -1 (latest)
Log Start Offset: 0
Record Errors
Error Message: [ Null ]
Tagged fields
Tagged fields
Throttle time: 0
Tagged fields
OffsetForLeaderEpoch requests
1572938, 15:37:13.770089, 192.168.65.3:58626->3.70.194.188:40003 - Kafka OffsetForLeaderEpoch v2 Request
Kafka (OffsetForLeaderEpoch v2 Request)
Length: 52
API Key: OffsetForLeaderEpoch (23)
API Version: 2
Correlation ID: 875
Client ID: rdkafka
Topics
Topic (Name=my-topic-0188)
Topic Name: my-topic-0188
Partitions
Partition (ID=0)
Partition ID: 0
Leader Epoch: 18
Leader Epoch: 17
1573309, 15:37:13.802351, 3.70.194.188:40003->192.168.65.3:58626 - Kafka OffsetForLeaderEpoch v2 Response
Kafka (OffsetForLeaderEpoch v2 Response)
Length: 49
Correlation ID: 875
[Request Frame: 1572938]
[API Key: OffsetForLeaderEpoch (23)]
[API Version: 2]
Throttle time: 0
Topics
Topic (Name=my-topic-0188)
Topic Name: my-topic-0188
Partitions
Partition (ID=0, Error=Unknown Leader Epoch)
Error: Unknown Leader Epoch (75)
Partition ID: 0
Leader Epoch: -1
Offset: -1
1583043, 15:37:14.817771, 192.168.65.3:58626->3.70.194.188:40003 - Kafka OffsetForLeaderEpoch v2 Request
Kafka (OffsetForLeaderEpoch v2 Request)
Length: 52
API Key: OffsetForLeaderEpoch (23)
API Version: 2
Correlation ID: 957
Client ID: rdkafka
Topics
Topic (Name=my-topic-0188)
Topic Name: my-topic-0188
Partitions
Partition (ID=0)
Partition ID: 0
Leader Epoch: 18
Leader Epoch: 17
1583379, 15:37:14.848090, 3.70.194.18840003->192.168.65.3:58626 - Kafka OffsetForLeaderEpoch v2 Response
Kafka (OffsetForLeaderEpoch v2 Response)
Length: 49
Correlation ID: 957
[Request Frame: 1583043]
[API Key: OffsetForLeaderEpoch (23)]
[API Version: 2]
Throttle time: 0
Topics
Topic (Name=my-topic-0188)
Topic Name: my-topic-0188
Partitions
Partition (ID=0)
Error: No Error (0)
Partition ID: 0
Leader Epoch: 17
Offset: 71231
Fetch requests
1586668, 15:37:15.212955, 3.70.194.188:40003->192.168.65.3:58626 - Kafka Fetch v16 Response
Kafka (Fetch v16 Response)
Length: 5300
Correlation ID: 992
[Request Frame: 1585869]
[API Key: Fetch (1)]
[API Version: 16]
Tagged fields
Throttle time: 0
Error: No Error (0)
Fetch Session ID: 0
Topics
Topic (1 partitions)
Topic ID: f49b160c-8440-4ffb-8211-44a649a35366
Partitions
Partition (ID=0, Offset=71231)
Partition ID: 0
Error: No Error (0)
Offset: 71231
Last Stable Offset: 71231
Log Start Offset: 0
Aborted Transactions
Replica ID: -1
Message Set
Record Batch
Offset: 71229
Message Size: 87
Leader Epoch: 17
Magic Byte: 2
CRC32: 0xe97ffa59
.... .... .... .000 = Compression Codec: None (0)
.... .... .... 0... = Timestamp Type: CreateTime (0)
.... .... ...0 .... = Transactional: Non-transactional (0)
.... .... ..0. .... = Control Batch: Data batch (0)
Last Offset Delta: 1
First Timestamp: Jan 1, 1970 00:06:44.-1728723327 UTC
Last Timestamp: Jan 1, 1970 00:06:44.-1728723293 UTC
Producer ID: 7078
Producer Epoch: 0
Base Sequence: 2704
Size: 2
Record
Record Attributes (reserved): 0
Timestamp: Jan 24, 2025 15:37:11.553000000 UTC
Offset: 71229
Key: 00000000
Value: 0000000000000a90
Headers
Record
Record Attributes (reserved): 0
Timestamp: Jan 24, 2025 15:37:11.587000000 UTC
Offset: 71230
Key: 00000000
Value: 0000000000000a91
Headers
Tagged fields
Tagged fields
1592076, 15:37:15.844683, 3.70.194.188:40003->192.168.65.3:58626 - Kafka Fetch v16 Response
Kafka (Fetch v16 Response)
Length: 12501
Correlation ID: 999
[Request Frame: 1591563]
[API Key: Fetch (1)]
[API Version: 16]
Tagged fields
Throttle time: 0
Error: No Error (0)
Fetch Session ID: 0
Topics
Topic (1 partitions)
Topic ID: f49b160c-8440-4ffb-8211-44a649a35366
Partitions
Partition (ID=0, Offset=71233)
Partition ID: 0
Error: No Error (0)
Offset: 71233
Last Stable Offset: 71233
Log Start Offset: 0
Aborted Transactions
Replica ID: -1
Message Set
Record Batch
Offset: 71231
Message Size: 87
Leader Epoch: 18
Magic Byte: 2
CRC32: 0x1cbeb4b1
.... .... .... .000 = Compression Codec: None (0)
.... .... .... 0... = Timestamp Type: CreateTime (0)
.... .... ...0 .... = Transactional: Non-transactional (0)
.... .... ..0. .... = Control Batch: Data batch (0)
Last Offset Delta: 1
First Timestamp: Jan 1, 1970 00:06:44.-1728723327 UTC
Last Timestamp: Jan 1, 1970 00:06:44.-1728723293 UTC
Producer ID: 7078
Producer Epoch: 1
Base Sequence: 0
Size: 2
Record
Record Attributes (reserved): 0
Timestamp: Jan 24, 2025 15:37:11.553000000 UTC
Offset: 71231
Key: 00000000
Value: 0000000000000a90
Headers
Record
Record Attributes (reserved): 0
Timestamp: Jan 24, 2025 15:37:11.587000000 UTC
Offset: 71232
Key: 00000000
Value: 0000000000000a91
Headers
Tagged fields
Tagged fields