You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Karolis Pocius (Jira)" <ji...@apache.org> on 2019/11/02 12:42:00 UTC

[jira] [Created] (KAFKA-9133) LogCleaner thread dies with: currentLog cannot be empty on an unexpected exception

Karolis Pocius created KAFKA-9133:
-------------------------------------

             Summary: LogCleaner thread dies with: currentLog cannot be empty on an unexpected exception
                 Key: KAFKA-9133
                 URL: https://issues.apache.org/jira/browse/KAFKA-9133
             Project: Kafka
          Issue Type: Bug
          Components: log cleaner
    Affects Versions: 2.3.1
            Reporter: Karolis Pocius


Log cleaner thread dies without a clear reference to which log is causing it:
{code}
[2019-11-02 11:59:59,078] INFO Starting the log cleaner (kafka.log.LogCleaner)
[2019-11-02 11:59:59,144] INFO [kafka-log-cleaner-thread-0]: Starting (kafka.log.LogCleaner)
[2019-11-02 11:59:59,199] ERROR [kafka-log-cleaner-thread-0]: Error due to (kafka.log.LogCleaner)
java.lang.IllegalStateException: currentLog cannot be empty on an unexpected exception
 at kafka.log.LogCleaner$CleanerThread.cleanFilthiestLog(LogCleaner.scala:346)
 at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:307)
 at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:89)
Caused by: java.lang.IllegalArgumentException: Illegal request for non-active segments beginning at offset 5033130, which is larger than the active segment's base offset 5019648
 at kafka.log.Log.nonActiveLogSegmentsFrom(Log.scala:1933)
 at kafka.log.LogCleanerManager$.maxCompactionDelay(LogCleanerManager.scala:491)
 at kafka.log.LogCleanerManager.$anonfun$grabFilthiestCompactedLog$4(LogCleanerManager.scala:184)
 at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
 at scala.collection.immutable.List.foreach(List.scala:392)
 at scala.collection.TraversableLike.map(TraversableLike.scala:238)
 at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
 at scala.collection.immutable.List.map(List.scala:298)
 at kafka.log.LogCleanerManager.$anonfun$grabFilthiestCompactedLog$1(LogCleanerManager.scala:181)
 at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:253)
 at kafka.log.LogCleanerManager.grabFilthiestCompactedLog(LogCleanerManager.scala:171)
 at kafka.log.LogCleaner$CleanerThread.cleanFilthiestLog(LogCleaner.scala:321)
 ... 2 more
[2019-11-02 11:59:59,200] INFO [kafka-log-cleaner-thread-0]: Stopped (kafka.log.LogCleaner)
{code}
If I try to ressurect it by dynamically bumping {{log.cleaner.threads}} it instantly dies with the exact same error.

Not sure if this is something KAFKA-8725 is supposed to address.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)