Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-8764

LogCleanerManager endless loop while compacting/cleaning segments

    XMLWordPrintableJSON

Details

    Description

      LogCleanerManager stuck in endless loop while clearing segments for one partition resulting with many log outputs and heavy disk read/writes/IOPS.

       

      Issue appeared on follower brokers, and it happens on every (new) broker if partition assignment is changed.

       

      Original issue setup:

      • kafka_2.12-2.2.1 deployed as statefulset on kubernetes, 5 brokers
      • log directory is (AWS) EBS mounted PV, gp2 (ssd) kind of 750GiB
      • 5 zookeepers
      • topic created with config:
        • name = "backup_br_domain_squad"
          partitions = 36
          replication_factor = 3

      config =

      { "cleanup.policy" = "compact" "min.compaction.lag.ms" = "86400000" "min.cleanable.dirty.ratio" = "0.3" }

       

       

      Log excerpt:

      [2019-08-07 12:10:53,895] INFO [Log partition=backup_br_domain_squad-14, dir=/var/lib/kafka/data/topics] Deleting segment 0 (kafka.log.Log)
      [2019-08-07 12:10:53,895] INFO Deleted log /var/lib/kafka/data/topics/backup_br_domain_squad-14/00000000000000000000.log.deleted. (kafka.log.LogSegment)
      [2019-08-07 12:10:53,896] INFO Deleted offset index /var/lib/kafka/data/topics/backup_br_domain_squad-14/00000000000000000000.index.deleted. (kafka.log.LogSegment)
      [2019-08-07 12:10:53,896] INFO Deleted time index /var/lib/kafka/data/topics/backup_br_domain_squad-14/00000000000000000000.timeindex.deleted. (kafka.log.LogSegment)
      [2019-08-07 12:10:53,964] INFO [Log partition=backup_br_domain_squad-14, dir=/var/lib/kafka/data/topics] Deleting segment 0 (kafka.log.Log)
      [2019-08-07 12:10:53,964] INFO Deleted log /var/lib/kafka/data/topics/backup_br_domain_squad-14/00000000000000000000.log.deleted. (kafka.log.LogSegment)
      [2019-08-07 12:10:53,964] INFO Deleted offset index /var/lib/kafka/data/topics/backup_br_domain_squad-14/00000000000000000000.index.deleted. (kafka.log.LogSegment)
      [2019-08-07 12:10:53,964] INFO Deleted time index /var/lib/kafka/data/topics/backup_br_domain_squad-14/00000000000000000000.timeindex.deleted. (kafka.log.LogSegment)
      [2019-08-07 12:10:54,031] INFO [Log partition=backup_br_domain_squad-14, dir=/var/lib/kafka/data/topics] Deleting segment 0 (kafka.log.Log)
      [2019-08-07 12:10:54,032] INFO Deleted log /var/lib/kafka/data/topics/backup_br_domain_squad-14/00000000000000000000.log.deleted. (kafka.log.LogSegment)
      [2019-08-07 12:10:54,032] INFO Deleted offset index /var/lib/kafka/data/topics/backup_br_domain_squad-14/00000000000000000000.index.deleted. (kafka.log.LogSegment)
      [2019-08-07 12:10:54,032] INFO Deleted time index /var/lib/kafka/data/topics/backup_br_domain_squad-14/00000000000000000000.timeindex.deleted. (kafka.log.LogSegment)
      [2019-08-07 12:10:54,101] INFO [Log partition=backup_br_domain_squad-14, dir=/var/lib/kafka/data/topics] Deleting segment 0 (kafka.log.Log)
      [2019-08-07 12:10:54,101] INFO Deleted log /var/lib/kafka/data/topics/backup_br_domain_squad-14/00000000000000000000.log.deleted. (kafka.log.LogSegment)
      [2019-08-07 12:10:54,101] INFO Deleted offset index /var/lib/kafka/data/topics/backup_br_domain_squad-14/00000000000000000000.index.deleted. (kafka.log.LogSegment)
      [2019-08-07 12:10:54,101] INFO Deleted time index /var/lib/kafka/data/topics/backup_br_domain_squad-14/00000000000000000000.timeindex.deleted. (kafka.log.LogSegment)
      [2019-08-07 12:10:54,173] INFO [Log partition=backup_br_domain_squad-14, dir=/var/lib/kafka/data/topics] Deleting segment 0 (kafka.log.Log)
      [2019-08-07 12:10:54,173] INFO Deleted log /var/lib/kafka/data/topics/backup_br_domain_squad-14/00000000000000000000.log.deleted. (kafka.log.LogSegment)
      [2019-08-07 12:10:54,173] INFO Deleted offset index /var/lib/kafka/data/topics/backup_br_domain_squad-14/00000000000000000000.index.deleted. (kafka.log.LogSegment)
      [2019-08-07 12:10:54,173] INFO Deleted time index /var/lib/kafka/data/topics/backup_br_domain_squad-14/00000000000000000000.timeindex.deleted. (kafka.log.LogSegment)

       

      And such log keeps repeating forever.

       

       

      I've been able to extract segment files from (running) leader broker, and replicated same behaviour locally.

       

      Reproduction setup:

      • start single broker kafka_2.12-2.2.1
      • create topic
        • ./bin/kafka-topics.sh --bootstrap-server _BOOTSTRAP_SERVER_:9092 --create --topic backup_br_domain_squad --partitions 1 --replication-factor 1 --config cleanup.policy=compact --config min.compaction.lag.ms=86400000 --config min.cleanable.dirty.ratio=0.3
      • stop broker
      • copy segment files (attachment content, under segments) to backup_br_domain_squad-0 log folder
      • rerun broker again

       

      After rerun, fun starts with endless repeating logging outputs:

      [2019-08-07 13:05:37,610] DEBUG Checking if log segment may be cleaned: log='backup_br_domain_squad-0' segment.baseOffset=0 segment.largestTimestamp=1563203503987; now - compactionLag=1565096737609; is uncleanable=false (kafka.log.LogCleanerManager$)
      [2019-08-07 13:05:37,610] DEBUG Finding range of cleanable offsets for log=backup_br_domain_squad-0 topicPartition=backup_br_domain_squad-0. Last clean offset=Some(0) now=1565183137609 => firstDirtyOffset=0 firstUncleanableOffset=233 activeSegment.baseOffset=233 (kafka.log.LogCleanerManager$)
      [2019-08-07 13:05:37,613] INFO Cleaner 0: Beginning cleaning of log backup_br_domain_squad-0. (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,613] INFO Cleaner 0: Building offset map for backup_br_domain_squad-0... (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,628] INFO Cleaner 0: Building offset map for log backup_br_domain_squad-0 for 1 segments in offset range [0, 233). (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,643] INFO Cleaner 0: Offset map for log backup_br_domain_squad-0 complete. (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,644] INFO Cleaner 0: Cleaning log backup_br_domain_squad-0 (cleaning prior to Wed Aug 07 13:05:22 GMT 2019, discarding tombstones prior to Thu Jan 01 00:00:00 GMT 1970)... (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,645] DEBUG Loaded index file /runtime/logs/kafka_1/backup_br_domain_squad-0/00000000000000000000.index.cleaned with maxEntries = 1310720, maxIndexSize = 10485760, entries = 0, lastOffset = 0, file position = 0 (kafka.log.OffsetIndex)
      [2019-08-07 13:05:37,646] DEBUG Loaded index file /runtime/logs/kafka_1/backup_br_domain_squad-0/00000000000000000000.timeindex.cleaned with maxEntries = 873813, maxIndexSize = 10485760, entries = 0, lastOffset = TimestampOffset(-1,0), file position = 0 (kafka.log.TimeIndex)
      [2019-08-07 13:05:37,647] INFO Cleaner 0: Cleaning segment 0 in log backup_br_domain_squad-0 (largest timestamp Mon Jul 15 15:11:43 GMT 2019) into 0, retaining deletes. (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,656] DEBUG Resized /runtime/logs/kafka_1/backup_br_domain_squad-0/00000000000000000000.index.cleaned to 32, position is 32 and limit is 32 (kafka.log.OffsetIndex)
      [2019-08-07 13:05:37,657] DEBUG Resized /runtime/logs/kafka_1/backup_br_domain_squad-0/00000000000000000000.timeindex.cleaned to 36, position is 36 and limit is 36 (kafka.log.TimeIndex)
      [2019-08-07 13:05:37,669] INFO Cleaner 0: Swapping in cleaned segment LogSegment(baseOffset=0, size=1044021) for segment(s) List(LogSegment(baseOffset=0, size=1044021)) in log Log(/runtime/logs/kafka_1/backup_br_domain_squad-0) (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,672] INFO [kafka-log-cleaner-thread-0]:
      {{ Log cleaner thread 0 cleaned log backup_br_domain_squad-0 (dirty section = [0, 0])}}
      {{ 1.0 MB of log processed in 0.1 seconds (17.2 MB/sec).}}
      {{ Indexed 1.0 MB in 0.0 seconds (33.2 Mb/sec, 51.7% of total time)}}
      {{ Buffer utilization: 0.0%}}
      {{ Cleaned 1.0 MB in 0.0 seconds (35.6 Mb/sec, 48.3% of total time)}}
      {{ Start size: 1.0 MB (231 messages)}}
      {{ End size: 1.0 MB (231 messages)}}
      {{ 0.0% size reduction (0.0% fewer messages)}}
      {{ (kafka.log.LogCleaner)}}
      [2019-08-07 13:05:37,676] DEBUG Checking if log segment may be cleaned: log='backup_br_domain_squad-0' segment.baseOffset=0 segment.largestTimestamp=1563203503987; now - compactionLag=1565096737676; is uncleanable=false (kafka.log.LogCleanerManager$)
      [2019-08-07 13:05:37,677] DEBUG Finding range of cleanable offsets for log=backup_br_domain_squad-0 topicPartition=backup_br_domain_squad-0. Last clean offset=Some(232) now=1565183137676 => firstDirtyOffset=232 firstUncleanableOffset=233 activeSegment.baseOffset=233 (kafka.log.LogCleanerManager$)
      [2019-08-07 13:05:37,677] INFO Cleaner 0: Beginning cleaning of log backup_br_domain_squad-0. (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,678] INFO Cleaner 0: Building offset map for backup_br_domain_squad-0... (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,680] INFO [SocketServer brokerId=1001] Created data-plane acceptor and processors for endpoint : EndPoint(null,9082,ListenerName(INSIDE_DOCKER),PLAINTEXT) (kafka.network.SocketServer)
      [2019-08-07 13:05:37,680] INFO [SocketServer brokerId=1001] Created data-plane acceptor and processors for endpoint : EndPoint(null,9082,ListenerName(INSIDE_DOCKER),PLAINTEXT) (kafka.network.SocketServer)
      [2019-08-07 13:05:37,681] INFO Awaiting socket connections on s0.0.0.0:9092. (kafka.network.Acceptor)
      [2019-08-07 13:05:37,681] INFO Awaiting socket connections on s0.0.0.0:9092. (kafka.network.Acceptor)
      [2019-08-07 13:05:37,700] INFO [SocketServer brokerId=1001] Created data-plane acceptor and processors for endpoint : EndPoint(null,9092,ListenerName(OUTSIDE_DOCKER),PLAINTEXT) (kafka.network.SocketServer)
      [2019-08-07 13:05:37,700] INFO [SocketServer brokerId=1001] Created data-plane acceptor and processors for endpoint : EndPoint(null,9092,ListenerName(OUTSIDE_DOCKER),PLAINTEXT) (kafka.network.SocketServer)
      [2019-08-07 13:05:37,701] INFO Awaiting socket connections on s0.0.0.0:19092. (kafka.network.Acceptor)
      [2019-08-07 13:05:37,701] INFO Cleaner 0: Building offset map for log backup_br_domain_squad-0 for 1 segments in offset range [232, 233). (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,701] INFO Awaiting socket connections on s0.0.0.0:19092. (kafka.network.Acceptor)
      [2019-08-07 13:05:37,702] INFO Cleaner 0: Offset map for log backup_br_domain_squad-0 complete. (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,702] INFO Cleaner 0: Cleaning log backup_br_domain_squad-0 (cleaning prior to Wed Aug 07 13:05:22 GMT 2019, discarding tombstones prior to Tue Aug 06 13:05:22 GMT 2019)... (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,703] INFO [kafka-log-cleaner-thread-0]:
      {{ Log cleaner thread 0 cleaned log backup_br_domain_squad-0 (dirty section = [232, 232])}}
      {{ 0.0 MB of log processed in 0.0 seconds (0.0 MB/sec).}}
      {{ Indexed 0.0 MB in 0.0 seconds (0.3 Mb/sec, 96.2% of total time)}}
      {{ Buffer utilization: 0.0%}}
      {{ Cleaned 0.0 MB in 0.0 seconds (0.0 Mb/sec, 3.8% of total time)}}
      {{ Start size: 0.0 MB (0 messages)}}
      {{ End size: 0.0 MB (0 messages)}}
      {{ NaN% size reduction (NaN% fewer messages)}}
      {{ (kafka.log.LogCleaner)}}
      [2019-08-07 13:05:37,707] DEBUG Checking if log segment may be cleaned: log='backup_br_domain_squad-0' segment.baseOffset=0 segment.largestTimestamp=1563203503987; now - compactionLag=1565096737707; is uncleanable=false (kafka.log.LogCleanerManager$)
      [2019-08-07 13:05:37,708] DEBUG Finding range of cleanable offsets for log=backup_br_domain_squad-0 topicPartition=backup_br_domain_squad-0. Last clean offset=Some(0) now=1565183137707 => firstDirtyOffset=0 firstUncleanableOffset=233 activeSegment.baseOffset=233 (kafka.log.LogCleanerManager$)
      [2019-08-07 13:05:37,708] INFO Cleaner 0: Beginning cleaning of log backup_br_domain_squad-0. (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,708] INFO Cleaner 0: Building offset map for backup_br_domain_squad-0... (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,719] INFO [SocketServer brokerId=1001] Created data-plane acceptor and processors for endpoint : EndPoint(null,19092,ListenerName(MAC_OS_DOCKER),PLAINTEXT) (kafka.network.SocketServer)
      [2019-08-07 13:05:37,719] INFO [SocketServer brokerId=1001] Created data-plane acceptor and processors for endpoint : EndPoint(null,19092,ListenerName(MAC_OS_DOCKER),PLAINTEXT) (kafka.network.SocketServer)
      [2019-08-07 13:05:37,721] INFO Cleaner 0: Building offset map for log backup_br_domain_squad-0 for 1 segments in offset range [0, 233). (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,722] INFO [SocketServer brokerId=1001] Started 3 acceptor threads for data-plane (kafka.network.SocketServer)
      [2019-08-07 13:05:37,722] INFO [SocketServer brokerId=1001] Started 3 acceptor threads for data-plane (kafka.network.SocketServer)
      [2019-08-07 13:05:37,730] INFO Cleaner 0: Offset map for log backup_br_domain_squad-0 complete. (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,730] INFO Cleaner 0: Cleaning log backup_br_domain_squad-0 (cleaning prior to Wed Aug 07 13:05:22 GMT 2019, discarding tombstones prior to Thu Jan 01 00:00:00 GMT 1970)... (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,732] DEBUG Loaded index file /runtime/logs/kafka_1/backup_br_domain_squad-0/00000000000000000000.index.cleaned with maxEntries = 1310720, maxIndexSize = 10485760, entries = 0, lastOffset = 0, file position = 0 (kafka.log.OffsetIndex)
      [2019-08-07 13:05:37,733] DEBUG Loaded index file /runtime/logs/kafka_1/backup_br_domain_squad-0/00000000000000000000.timeindex.cleaned with maxEntries = 873813, maxIndexSize = 10485760, entries = 0, lastOffset = TimestampOffset(-1,0), file position = 0 (kafka.log.TimeIndex)
      [2019-08-07 13:05:37,733] INFO Cleaner 0: Cleaning segment 0 in log backup_br_domain_squad-0 (largest timestamp Mon Jul 15 15:11:43 GMT 2019) into 0, retaining deletes. (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,739] DEBUG Resized /runtime/logs/kafka_1/backup_br_domain_squad-0/00000000000000000000.index.cleaned to 32, position is 32 and limit is 32 (kafka.log.OffsetIndex)
      [2019-08-07 13:05:37,739] DEBUG Resized /runtime/logs/kafka_1/backup_br_domain_squad-0/00000000000000000000.timeindex.cleaned to 36, position is 36 and limit is 36 (kafka.log.TimeIndex)
      [2019-08-07 13:05:37,764] INFO Cleaner 0: Swapping in cleaned segment LogSegment(baseOffset=0, size=1044021) for segment(s) List(LogSegment(baseOffset=0, size=1044021)) in log Log(/runtime/logs/kafka_1/backup_br_domain_squad-0) (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,767] INFO [kafka-log-cleaner-thread-0]:
      {{ Log cleaner thread 0 cleaned log backup_br_domain_squad-0 (dirty section = [0, 0])}}
      {{ 1.0 MB of log processed in 0.1 seconds (17.2 MB/sec).}}
      {{ Indexed 1.0 MB in 0.0 seconds (45.3 Mb/sec, 37.9% of total time)}}
      {{ Buffer utilization: 0.0%}}
      {{ Cleaned 1.0 MB in 0.0 seconds (27.7 Mb/sec, 62.1% of total time)}}
      {{ Start size: 1.0 MB (231 messages)}}
      {{ End size: 1.0 MB (231 messages)}}
      {{ 0.0% size reduction (0.0% fewer messages)}}
      {{ (kafka.log.LogCleaner)}}
      [2019-08-07 13:05:37,772] DEBUG Checking if log segment may be cleaned: log='backup_br_domain_squad-0' segment.baseOffset=0 segment.largestTimestamp=1563203503987; now - compactionLag=1565096737771; is uncleanable=false (kafka.log.LogCleanerManager$)
      [2019-08-07 13:05:37,772] DEBUG Finding range of cleanable offsets for log=backup_br_domain_squad-0 topicPartition=backup_br_domain_squad-0. Last clean offset=Some(232) now=1565183137771 => firstDirtyOffset=232 firstUncleanableOffset=233 activeSegment.baseOffset=233 (kafka.log.LogCleanerManager$)
      [2019-08-07 13:05:37,773] INFO Cleaner 0: Beginning cleaning of log backup_br_domain_squad-0. (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,774] INFO Cleaner 0: Building offset map for backup_br_domain_squad-0... (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,795] INFO [ExpirationReaper-1001-Produce]: Starting (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
      [2019-08-07 13:05:37,795] INFO [ExpirationReaper-1001-Produce]: Starting (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
      [2019-08-07 13:05:37,800] INFO [ExpirationReaper-1001-Fetch]: Starting (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
      [2019-08-07 13:05:37,800] INFO [ExpirationReaper-1001-Fetch]: Starting (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
      [2019-08-07 13:05:37,803] INFO [ExpirationReaper-1001-DeleteRecords]: Starting (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
      [2019-08-07 13:05:37,803] INFO [ExpirationReaper-1001-DeleteRecords]: Starting (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
      [2019-08-07 13:05:37,805] INFO [ExpirationReaper-1001-ElectPreferredLeader]: Starting (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
      [2019-08-07 13:05:37,805] INFO [ExpirationReaper-1001-ElectPreferredLeader]: Starting (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
      [2019-08-07 13:05:37,806] INFO Cleaner 0: Building offset map for log backup_br_domain_squad-0 for 1 segments in offset range [232, 233). (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,807] INFO Cleaner 0: Offset map for log backup_br_domain_squad-0 complete. (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,808] INFO Cleaner 0: Cleaning log backup_br_domain_squad-0 (cleaning prior to Wed Aug 07 13:05:22 GMT 2019, discarding tombstones prior to Tue Aug 06 13:05:22 GMT 2019)... (kafka.log.LogCleaner)
      [2019-08-07 13:05:37,809] INFO [kafka-log-cleaner-thread-0]:
      {{ Log cleaner thread 0 cleaned log backup_br_domain_squad-0 (dirty section = [232, 232])}}
      {{ 0.0 MB of log processed in 0.0 seconds (0.0 MB/sec).}}
      {{ Indexed 0.0 MB in 0.0 seconds (0.2 Mb/sec, 97.1% of total time)}}
      {{ Buffer utilization: 0.0%}}
      {{ Cleaned 0.0 MB in 0.0 seconds (0.0 Mb/sec, 2.9% of total time)}}
      {{ Start size: 0.0 MB (0 messages)}}
      {{ End size: 0.0 MB (0 messages)}}
      {{ NaN% size reduction (NaN% fewer messages)}}
      {{ (kafka.log.LogCleaner)}}

       

       

      Attachments

        1. Screen Shot 2020-01-10 at 8.38.25 AM.png
          313 kB
          Jeff Nadler
        2. kafka2.4.0-KAFKA-8764.patch
          1 kB
          Tomislav Rajakovic
        3. kafka2.4.0-KAFKA-8764.patch
          1 kB
          Tomislav Rajakovic
        4. log-cleaner-bug-reproduction.zip
          382 kB
          Tomislav Rajakovic

        Issue Links

          Activity

            People

              Unassigned Unassigned
              trajakovic Tomislav Rajakovic
              Votes:
              1 Vote for this issue
              Watchers:
              8 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: