You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Gregory Koshelev (Jira)" <ji...@apache.org> on 2019/08/21 08:24:00 UTC

[jira] [Created] (KAFKA-8823) Retention is partially broken on recreated topic

Gregory Koshelev created KAFKA-8823:
---------------------------------------

             Summary: Retention is partially broken on recreated topic
                 Key: KAFKA-8823
                 URL: https://issues.apache.org/jira/browse/KAFKA-8823
             Project: Kafka
          Issue Type: Bug
          Components: log cleaner
    Affects Versions: 2.2.0
            Reporter: Gregory Koshelev


I've recreated topic with 48 partitions across 6 brokers with replication factor 3 with following config:
{code}
retention.ms=86400000
retention.bytes=137438953472
{code}

Log cleaner have cleaned old segments in most partitions after 1 day as expected. But some partitions remained untouched (those partitions still have segments with base offset 0).

There are no errors in the server.log or log-cleaner.log for a week after topic creation. Also, some partitions remains untouched and grows over 1TB. There no messages for broken partitions in the server.log like:
{code}
[2019-08-14 23:55:35,465] INFO [Log partition=traces_prod-25, dir=/storage6/kafka/data] Found deletable segments with base offsets [355575520] due to retention size in bytes 137438953472 breach (kafka.log.Log)
[2019-08-14 23:55:35,465] INFO [Log partition=traces_prod-25, dir=/storage6/kafka/data] Scheduling log segment [baseOffset 355575520, size 1073739552] for deletion. (kafka.log.Log)
[2019-08-14 23:55:35,465] INFO [Log partition=traces_prod-25, dir=/storage6/kafka/data] Incrementing log start offset to 358210127 (kafka.log.Log)
{code}

So, finally, what I have tried:
# Set up another retention.ms (no success).
# Set up retention.bytes (no success).
# Restart the broker (success).

It's important to note that the topic had existed before: it was deleted and recreated again. I see some errors in the server.log about topic deletion:
{code}
[2019-08-13 18:31:24,402] INFO Deleted log /storage7/kafka/data/traces_prod-16.8395523b3a25454e8a1a7dd35c5f15d3-delete/00000000000000000000.log. (kafka.log.LogSegment)
[2019-08-13 18:31:24,406] ERROR Error unmapping index /storage7/kafka/data/traces_prod-16.8395523b3a25454e8a1a7dd35c5f15d3-delete/00000000000000000000.index (kafka.log.OffsetIndex)
java.lang.NullPointerException
        at org.apache.kafka.common.utils.MappedByteBuffers.unmap(MappedByteBuffers.java:73)
        at kafka.log.AbstractIndex.forceUnmap(AbstractIndex.scala:321)
        at kafka.log.AbstractIndex.safeForceUnmap(AbstractIndex.scala:311)
        at kafka.log.AbstractIndex.$anonfun$closeHandler$1(AbstractIndex.scala:260)
        at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
        at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:251)
        at kafka.log.AbstractIndex.closeHandler(AbstractIndex.scala:260)
        at kafka.log.AbstractIndex.deleteIfExists(AbstractIndex.scala:229)
        at kafka.log.LogSegment.$anonfun$deleteIfExists$6(LogSegment.scala:597)
        at kafka.log.LogSegment.delete$1(LogSegment.scala:585)
        at kafka.log.LogSegment.$anonfun$deleteIfExists$5(LogSegment.scala:597)
        at kafka.utils.CoreUtils$.$anonfun$tryAll$1(CoreUtils.scala:115)
        at kafka.utils.CoreUtils$.$anonfun$tryAll$1$adapted(CoreUtils.scala:114)
        at scala.collection.immutable.List.foreach(List.scala:392)
        at kafka.utils.CoreUtils$.tryAll(CoreUtils.scala:114)
        at kafka.log.LogSegment.deleteIfExists(LogSegment.scala:599)
        at kafka.log.Log.$anonfun$delete$3(Log.scala:1762)
        at kafka.log.Log.$anonfun$delete$3$adapted(Log.scala:1762)
        at scala.collection.Iterator.foreach(Iterator.scala:941)
        at scala.collection.Iterator.foreach$(Iterator.scala:941)
        at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
        at scala.collection.IterableLike.foreach(IterableLike.scala:74)
        at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
        at scala.collection.AbstractIterable.foreach(Iterable.scala:56)
        at kafka.log.Log.$anonfun$delete$2(Log.scala:1762)
        at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
        at kafka.log.Log.maybeHandleIOException(Log.scala:2013)
        at kafka.log.Log.delete(Log.scala:1759)
        at kafka.log.LogManager.deleteLogs(LogManager.scala:761)
        at kafka.log.LogManager.$anonfun$deleteLogs$6(LogManager.scala:775)
        at kafka.utils.KafkaScheduler.$anonfun$schedule$2(KafkaScheduler.scala:114)
        at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:63)
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)
{code}



--
This message was sent by Atlassian Jira
(v8.3.2#803003)