You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "Christopher Auston (Jira)" <ji...@apache.org> on 2023/05/17 18:52:00 UTC

[jira] [Created] (KAFKA-15006) kafka-get-offsets.sh returns 0 for earliest offset on a compacted topic

Christopher Auston created KAFKA-15006:
------------------------------------------

             Summary: kafka-get-offsets.sh returns 0 for earliest offset on a compacted topic
                 Key: KAFKA-15006
                 URL: https://issues.apache.org/jira/browse/KAFKA-15006
             Project: Kafka
          Issue Type: Bug
    Affects Versions: 3.4.0
            Reporter: Christopher Auston


 
After some compactions and when the 00000000000000000000.log is empty, it seems like kafka-get-offsets.sh (Admin client?) should return a non-zero offset for the earliest offset?

A simple, compacted topic:
{code:java}
kraft-combined-logs % kafka-topics.sh --bootstrap-server :9092 -topic offtest -describeTopic: offtest	TopicId: pwmz5qPGQsyo3006M_VBqw	PartitionCount: 1	ReplicationFactor: 1	Configs: cleanup.policy=compact,segment.bytes=16384	Topic: offtest	Partition: 0	Leader: 1	Replicas: 1	Isr: 1{code}
Trying to get the earliest offset after some compactions returns 0 which is not where a consumer starts with earliest (also the 000 log segment is empty):
{code:java}
kraft-combined-logs % kafka-get-offsets.sh --bootstrap-server :9092 -topic offtest -time -2
offtest:0:0

kraft-combined-logs % kcat -b :9092 -C -t offtest -c1 -J
{"topic":"offtest","partition":0,"offset":161,"tstype":"create","ts":1684347565956,"broker":1,"key":"key1","payload":"                                                                                                                        value 34"}

kraft-combined-logs % ls -l offtest-0
total 41024
-rw-r--r--  1 ca9898  staff         0 May 17 14:16 00000000000000000000.index
-rw-r--r--  1 ca9898  staff         0 May 17 14:16 00000000000000000000.log
-rw-r--r--  1 ca9898  staff         0 May 17 14:16 00000000000000000000.timeindex
-rw-r--r--  1 ca9898  staff         0 May 17 14:19 00000000000000000081.index
-rw-r--r--  1 ca9898  staff       202 May 17 14:19 00000000000000000081.log
-rw-r--r--  1 ca9898  staff        10 May 17 14:16 00000000000000000081.snapshot
-rw-r--r--  1 ca9898  staff        12 May 17 14:19 00000000000000000081.timeindex
-rw-r--r--  1 ca9898  staff  10485760 May 17 14:19 00000000000000000162.index
-rw-r--r--  1 ca9898  staff      6060 May 17 14:19 00000000000000000162.log
-rw-r--r--  1 ca9898  staff        10 May 17 14:19 00000000000000000162.snapshot
-rw-r--r--  1 ca9898  staff  10485756 May 17 14:19 00000000000000000162.timeindex
-rw-r--r--  1 ca9898  staff         8 May 17 13:56 leader-epoch-checkpoint
-rw-r--r--  1 ca9898  staff        43 May 17 13:56 partition.metadata

{code}
Server console log:
{code:java}
[2023-05-17 13:50:52,461] INFO [SocketServer listenerType=BROKER, nodeId=1] Enabling request processing. (kafka.network.SocketServer)
[2023-05-17 13:50:52,496] INFO [BrokerLifecycleManager id=1] The broker has been unfenced. Transitioning from RECOVERY to RUNNING. (kafka.server.BrokerLifecycleManager)
[2023-05-17 13:50:52,496] INFO [BrokerServer id=1] Transition from STARTING to STARTED (kafka.server.BrokerServer)
[2023-05-17 13:50:52,496] INFO Kafka version: 3.4.0 (org.apache.kafka.common.utils.AppInfoParser)
[2023-05-17 13:50:52,500] INFO Kafka commitId: 2e1947d240607d53 (org.apache.kafka.common.utils.AppInfoParser)
[2023-05-17 13:50:52,500] INFO Kafka startTimeMs: 1684345852496 (org.apache.kafka.common.utils.AppInfoParser)
[2023-05-17 13:50:52,501] INFO [KafkaRaftServer nodeId=1] Kafka Server started (kafka.server.KafkaRaftServer)
[2023-05-17 13:56:18,521] INFO [ReplicaFetcherManager on broker 1] Removed fetcher for partitions Set(offtest-0) (kafka.server.ReplicaFetcherManager)
[2023-05-17 13:56:18,530] INFO [LogLoader partition=offtest-0, dir=/Users/ca9898/kraft-combined-logs] Loading producer state till offset 0 with message format version 2 (kafka.log.UnifiedLog$)
[2023-05-17 13:56:18,531] INFO Created log for partition offtest-0 in /Users/ca9898/kraft-combined-logs/offtest-0 with properties {} (kafka.log.LogManager)
[2023-05-17 13:56:18,536] INFO [Partition offtest-0 broker=1] No checkpointed highwatermark is found for partition offtest-0 (kafka.cluster.Partition)
[2023-05-17 13:56:18,536] INFO [Partition offtest-0 broker=1] Log loaded for partition offtest-0 with initial high watermark 0 (kafka.cluster.Partition)
[2023-05-17 14:06:25,254] INFO [DynamicConfigPublisher nodeType=broker id=1] Updating topic offtest with new configuration : cleanup.policy -> compact,segment.bytes -> 16384 (kafka.server.metadata.DynamicConfigPublisher)
[2023-05-17 14:16:30,990] INFO [BrokerToControllerChannelManager broker=1 name=forwarding] Node 1 disconnected. (org.apache.kafka.clients.NetworkClient)
[2023-05-17 14:16:53,142] INFO [LocalLog partition=offtest-0, dir=/Users/ca9898/kraft-combined-logs] Rolled new log segment at offset 81 in 2 ms. (kafka.log.LocalLog)
[2023-05-17 14:16:53,147] INFO [ProducerStateManager partition=offtest-0] Wrote producer snapshot at offset 81 with 0 producer ids in 5 ms. (kafka.log.ProducerStateManager)
[2023-05-17 14:18:08,020] INFO [UnifiedLog partition=offtest-0, dir=/Users/ca9898/kraft-combined-logs] Deleting segment files LogSegment(baseOffset=0, size=16362, lastModifiedTime=1684347413121, largestRecordTimestamp=Some(1684347413113)) (kafka.log.LocalLog$)
[2023-05-17 14:18:08,028] INFO Deleted log /Users/ca9898/kraft-combined-logs/offtest-0/00000000000000000000.log.deleted. (kafka.log.LogSegment)
[2023-05-17 14:18:08,033] INFO Deleted offset index /Users/ca9898/kraft-combined-logs/offtest-0/00000000000000000000.index.deleted. (kafka.log.LogSegment)
[2023-05-17 14:18:08,033] INFO Deleted time index /Users/ca9898/kraft-combined-logs/offtest-0/00000000000000000000.timeindex.deleted. (kafka.log.LogSegment)
[2023-05-17 14:19:25,983] INFO [LocalLog partition=offtest-0, dir=/Users/ca9898/kraft-combined-logs] Rolled new log segment at offset 162 in 1 ms. (kafka.log.LocalLog)
[2023-05-17 14:19:25,987] INFO [ProducerStateManager partition=offtest-0] Wrote producer snapshot at offset 162 with 0 producer ids in 5 ms. (kafka.log.ProducerStateManager)
[2023-05-17 14:20:38,102] INFO [UnifiedLog partition=offtest-0, dir=/Users/ca9898/kraft-combined-logs] Deleting segment files LogSegment(baseOffset=0, size=202, lastModifiedTime=1684347413121, largestRecordTimestamp=Some(1684347413113)) (kafka.log.LocalLog$)
[2023-05-17 14:20:38,104] INFO Deleted log /Users/ca9898/kraft-combined-logs/offtest-0/00000000000000000000.log.deleted. (kafka.log.LogSegment)
[2023-05-17 14:20:38,104] INFO Deleted offset index /Users/ca9898/kraft-combined-logs/offtest-0/00000000000000000000.index.deleted. (kafka.log.LogSegment)
[2023-05-17 14:20:38,105] INFO Deleted time index /Users/ca9898/kraft-combined-logs/offtest-0/00000000000000000000.timeindex.deleted. (kafka.log.LogSegment)
[2023-05-17 14:20:38,116] INFO [UnifiedLog partition=offtest-0, dir=/Users/ca9898/kraft-combined-logs] Deleting segment files LogSegment(baseOffset=81, size=16362, lastModifiedTime=1684347565963, largestRecordTimestamp=Some(1684347565956)) (kafka.log.LocalLog$)
[2023-05-17 14:20:38,117] INFO Deleted log /Users/ca9898/kraft-combined-logs/offtest-0/00000000000000000081.log.deleted. (kafka.log.LogSegment)
[2023-05-17 14:20:38,117] INFO Deleted offset index /Users/ca9898/kraft-combined-logs/offtest-0/00000000000000000081.index.deleted. (kafka.log.LogSegment)
[2023-05-17 14:20:38,117] INFO Deleted time index /Users/ca9898/kraft-combined-logs/offtest-0/00000000000000000081.timeindex.deleted. (kafka.log.LogSegment) {code}
 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)