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

Some partitions are fully truncated during recovery when log.message.format = 0.10.2 & inter.broker.protocol >= 0.11

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Duplicate
    • 0.11.0.0, 2.0.1, 2.1.0
    • None
    • core
    • None
    • centos 7

    Description

      On a cluster of Kafka 2.0.1, and brokers configured with

      • inter.broker.protocol.format = 2.0
      • log.message.format.version = 0.10.2

       

      In such configuration, when a broker is restarted (clean shutdown), the recovery process, for some partitions, is not taking in account the high watermark and is truncating and re-downloading the full partition.

      Typically for brokers with 500 partitions each / 5 TB of disk usage the recovery process with this configuration is during up to 1 hour whereas it usually takes less than 10 min in the same broker when (inter.broker.protocol.format = log.message.format.version)
      Partitions redownloaded seems not predictable : after several restart of the same broker, partitions redownloaded are now always the same.

      Broker log filter for one specific partition that was redownloaded ( the truncate offset : 12878451349 is corresponding to the log-start-offset) :

       

      2019-01-31 09:23:34,703 INFO [ProducerStateManager partition=my_topic-11] Writing producer snapshot at offset 13132373966 (kafka.log.ProducerStateManager)
      2019-01-31 09:25:15,245 INFO [Log partition=my_topic-11, dir=/var/lib/kafka] Loading producer state till offset 13132373966 with message format version 1 (kafka.log.Log)
      2019-01-31 09:25:15,245 INFO [ProducerStateManager partition=my_topic-11] Writing producer snapshot at offset 13130789408 (kafka.log.ProducerStateManager)
      2019-01-31 09:25:15,249 INFO [ProducerStateManager partition=my_topic-11] Writing producer snapshot at offset 13131829288 (kafka.log.ProducerStateManager)
      2019-01-31 09:25:15,388 INFO [ProducerStateManager partition=my_topic-11] Writing producer snapshot at offset 13132373966 (kafka.log.ProducerStateManager)
      
      2019-01-31 09:25:15,388 INFO [Log partition=my_topic-11, dir=/var/lib/kafka] Completed load of log with 243 segments, log start offset 12878451349 and log end offset 13132373966 in 46273 ms (kafka.log.Log)
      
      2019-01-31 09:28:38,226 INFO Replica loaded for partition my_topic-11 with initial high watermark 13132373966 (kafka.cluster.Replica)
      2019-01-31 09:28:38,226 INFO Replica loaded for partition my_topic-11 with initial high watermark 0 (kafka.cluster.Replica)
      2019-01-31 09:28:38,226 INFO Replica loaded for partition my_topic-11 with initial high watermark 0 (kafka.cluster.Replica)
      2019-01-31 09:28:42,132 INFO The cleaning for partition my_topic-11 is aborted and paused (kafka.log.LogCleaner)
      
      2019-01-31 09:28:42,133 INFO [Log partition=my_topic-11, dir=/var/lib/kafka] Truncating to offset 12878451349 (kafka.log.Log)
      
      2019-01-31 09:28:42,135 INFO [Log partition=my_topic-11, dir=/var/lib/kafka] Scheduling log segment [baseOffset 12879521312, size 536869342] for deletion. (kafka.log.Log)
      (...)
      2019-01-31 09:28:42,521 INFO [Log partition=my_topic-11, dir=/var/lib/kafka] Scheduling log segment [baseOffset 13131829288, size 280543535] for deletion. (kafka.log.Log)
      2019-01-31 09:28:43,870 WARN [ReplicaFetcher replicaId=11, leaderId=13, fetcherId=1] Truncating my_topic-11 to offset 12878451349 below high watermark 13132373966 (kafka.server.ReplicaFetcherThread)
      2019-01-31 09:29:03,703 INFO [Log partition=my_topic-11, dir=/var/lib/kafka] Found deletable segments with base offsets [12878451349] due to retention time 259200000ms breach (kafka.log.Log)
      2019-01-31 09:28:42,550 INFO Compaction for partition my_topic-11 is resumed (kafka.log.LogManager)
      

       

      We sucessfull tried to reproduce the same bug with kafka 0.11, 2.0.1 & 2.1.0

       

      Same issue appears when we are doing a rolling restart by switching log.message.format to 2.0

      Issue disappears when all brokers are with log.message.format = 2.0 & inter.broker.protocol = 2.0

      Attachments

        1. broker.log
          199 kB
          Hervé RIVIERE

        Issue Links

          Activity

            People

              Unassigned Unassigned
              herveriviere Hervé RIVIERE
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: