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

NETWORK_EXCEPTION and REQUEST_TIMED_OUT in mirrormaker producer after 1.0 broker upgrade

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Blocker
    • Resolution: Fixed
    • 1.0.0
    • 1.1.1
    • core, network

    Description

      We have 2 clusters A and B with 4 brokers each, we use mirrormaker to replicate topics from A to B.
      We recently upgraded our brokers from 0.10.2.0 to 1.0.0, after the upgrade we started seeing the mirrormaker task showing producer errors and intermittently dying.
      We tried using 1.0.0 and 0.10.2.0 mirrormaker, both have the same problem. Downgrading cluster B brokers back to 0.10.2.0 and the problem went away, so we think it's a server side problem.

      There are 2 types of errors: REQUEST_TIMED_OUT and NETWORK_EXCEPTION. For testing, I used a topic logging with 20 partitions and 3 replicas (same on cluster A and B), the source topic has 50+ million msg.

      (this is from mirrormaker 1.0 at info level, the 0.10.2.0 log is very similar)

      22 Mar 2018 02:16:07.407 [kafka-producer-network-thread | producer-1] WARN org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer clientId=producer-1] Got error produce response with correlation id 35122 on topic-partition logging-7, retrying (2147483646 attempts left). Error: REQUEST_TIMED_OUT
       22 Mar 2018 02:17:49.731 [kafka-producer-network-thread | producer-1] WARN org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer clientId=producer-1] Got error produce response with correlation id 51572 on topic-partition logging-7, retrying (2147483646 attempts left). Error: REQUEST_TIMED_OUT
       22 Mar 2018 02:18:33.903 [kafka-producer-network-thread | producer-1] WARN org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer clientId=producer-1] Got error produce response with correlation id 57785 on topic-partition logging-5, retrying (2147483646 attempts left). Error: REQUEST_TIMED_OUT
       22 Mar 2018 02:21:21.399 [kafka-producer-network-thread | producer-1] WARN org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer clientId=producer-1] Got error produce response with correlation id 85406 on topic-partition logging-18, retrying (2147483646 attempts left). Error: REQUEST_TIMED_OUT
       22 Mar 2018 02:25:22.278 [kafka-producer-network-thread | producer-1] WARN org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer clientId=producer-1] Got error produce response with correlation id 128047 on topic-partition logging-5, retrying (2147483646 attempts left). Error: REQUEST_TIMED_OUT
       22 Mar 2018 02:26:17.154 [kafka-producer-network-thread | producer-1] WARN org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer clientId=producer-1] Got error produce response with correlation id 137049 on topic-partition logging-18, retrying (2147483646 attempts left). Error: REQUEST_TIMED_OUT
       22 Mar 2018 02:27:57.358 [kafka-producer-network-thread | producer-1] WARN org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer clientId=producer-1] Got error produce response with correlation id 153976 on topic-partition logging-5, retrying (2147483646 attempts left). Error: REQUEST_TIMED_OUT
       22 Mar 2018 02:27:57.779 [kafka-producer-network-thread | producer-1] WARN org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer clientId=producer-1] Got error produce response with correlation id 154077 on topic-partition logging-2, retrying (2147483646 attempts left). Error: NETWORK_EXCEPTION
       22 Mar 2018 02:27:57.780 [kafka-producer-network-thread | producer-1] WARN org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer clientId=producer-1] Got error produce response with correlation id 154077 on topic-partition logging-10, retrying (2147483646 attempts left). Error: NETWORK_EXCEPTION
       22 Mar 2018 02:27:57.780 [kafka-producer-network-thread | producer-1] WARN org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer clientId=producer-1] Got error produce response with correlation id 154077 on topic-partition logging-18, retrying (2147483646 attempts left). Error: NETWORK_EXCEPTION
       22 Mar 2018 02:27:57.781 [kafka-producer-network-thread | producer-1] WARN org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer clientId=producer-1] Got error produce response with correlation id 154077 on topic-partition logging-14, retrying (2147483646 attempts left). Error: NETWORK_EXCEPTION
       22 Mar 2018 02:27:57.781 [kafka-producer-network-thread | producer-1] WARN org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer clientId=producer-1] Got error produce response with correlation id 154077 on topic-partition logging-6, retrying (2147483646 attempts left). Error: NETWORK_EXCEPTION
       22 Mar 2018 02:29:12.378 [kafka-producer-network-thread | producer-1] WARN org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer clientId=producer-1] Got error produce response with correlation id 163175 on topic-partition logging-18, retrying (2147483646 attempts left). Error: REQUEST_TIMED_OUT
       22 Mar 2018 02:31:47.067 [kafka-producer-network-thread | producer-1] WARN org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer clientId=producer-1] Got error produce response with correlation id 192333 on topic-partition logging-17, retrying (2147483646 attempts left). Error: REQUEST_TIMED_OUT
       22 Mar 2018 02:32:26.198 [kafka-producer-network-thread | producer-1] WARN org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer clientId=producer-1] Got error produce response with correlation id 195400 on topic-partition logging-18, retrying (2147483646 attempts left). Error: REQUEST_TIMED_OUT
       22 Mar 2018 02:33:16.280 [kafka-producer-network-thread | producer-1] WARN org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer clientId=producer-1] Got error produce response with correlation id 206708 on topic-partition logging-18, retrying (2147483646 attempts left). Error: REQUEST_TIMED_OUT
       22 Mar 2018 02:33:46.428 [kafka-producer-network-thread | producer-1] WARN org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer clientId=producer-1] Got error produce response with correlation id 206830 on topic-partition logging-18, retrying (2147483646 attempts left). Error: REQUEST_TIMED_OUT
       22 Mar 2018 12:53:50.848 [kafka-producer-network-thread | producer-1] WARN org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer clientId=producer-1] Got error produce response with correlation id 734439 on topic-partition logging-3, retrying (2147483646 attempts left). Error: REQUEST_TIMED_OUT
       22 Mar 2018 16:21:45.557 [kafka-producer-network-thread | producer-1] WARN org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer clientId=producer-1] Got error produce response with correlation id 941307 on topic-partition logging-6, retrying (2147483646 attempts left). Error: REQUEST_TIMED_OUT

      The NETWORK_EXCEPTION always comes in group of 5 with partition leaders on the same broker (there are 5 partition leader/broker).
      The REQUEST_TIMED_OUT seems to be from random individual partition.

      I turned on trace on mirrormaker and it showed how the NETWORK_EXCEPTION is causing the task to die.
      (I filtered out the important lines in this log segment)

      22 Mar 2018 02:06:20.154 [kafka-producer-network-thread | producer-1] TRACE org.apache.kafka.clients.NetworkClient trace(line:135) [Producer clientId=producer-1] Completed receive from node 44 for PRODUCE with correlation id 71596, received {responses=[{topic=logging,partition_responses=[{partition=17,error_code=0,base_offset=28466678,log_append_time=-1,log_start_offset=24408600},{partition=1,error_code=0,base_offset=11611076,log_append_time=-1,log_start_offset=7562939},{partition=5,error_code=0,base_offset=28467700,log_append_time=-1,log_start_offset=24416182},{partition=9,error_code=0,base_offset=32762076,log_append_time=-1,log_start_offset=30108807},{partition=13,error_code=0,base_offset=11612712,log_append_time=-1,log_start_offset=7564490}]}],throttle_time_ms=0}
      22 Mar 2018 02:06:20.154 [kafka-producer-network-thread | producer-1] TRACE org.apache.kafka.clients.producer.internals.Sender trace(line:129) [Producer clientId=producer-1] Received produce response from node 44 with correlation id 71596
      22 Mar 2018 02:06:20.154 [kafka-producer-network-thread | producer-1] TRACE org.apache.kafka.clients.producer.internals.ProducerBatch done(line:171) Successfully produced messages to logging-9 with base offset 32762076.
      22 Mar 2018 02:06:20.154 [kafka-producer-network-thread | producer-1] TRACE org.apache.kafka.clients.producer.internals.ProducerBatch done(line:171) Successfully produced messages to logging-13 with base offset 11612712.
      22 Mar 2018 02:06:20.154 [kafka-producer-network-thread | producer-1] TRACE org.apache.kafka.clients.producer.internals.ProducerBatch done(line:171) Successfully produced messages to logging-17 with base offset 28466678.
      22 Mar 2018 02:06:20.155 [kafka-producer-network-thread | producer-1] TRACE org.apache.kafka.clients.producer.internals.ProducerBatch done(line:171) Successfully produced messages to logging-1 with base offset 11611076.
      22 Mar 2018 02:06:20.155 [kafka-producer-network-thread | producer-1] TRACE org.apache.kafka.clients.producer.internals.ProducerBatch done(line:171) Successfully produced messages to logging-5 with base offset 28467700.
      22 Mar 2018 02:06:20.155 [kafka-producer-network-thread | producer-1] TRACE org.apache.kafka.clients.producer.internals.Sender trace(line:123) [Producer clientId=producer-1] Nodes with data ready to send: [broker44:9020 (id: 44 rack: null)]
      22 Mar 2018 02:06:20.155 [kafka-producer-network-thread | producer-1] TRACE org.apache.kafka.clients.NetworkClient trace(line:135) [Producer clientId=producer-1] Sending PRODUCE {acks=-1,timeout=30000,partitionSizes=[logging-1=647,logging-9=645,logging-5=647,logging-13=645]} with correlation id 71599 to node 44
      22 Mar 2018 02:06:20.155 [kafka-producer-network-thread | producer-1] TRACE org.apache.kafka.clients.producer.internals.Sender trace(line:129) [Producer clientId=producer-1] Sent produce request to 44: (type=ProduceRequest, acks=-1, timeout=30000, partitionRecords=({logging-1=[(record=DefaultRecord(offset=0, timestamp=1521353381705, key=0 bytes, value=1048 bytes))], logging-9=[(record=DefaultRecord(offset=0, timestamp=1521353381983, key=0 bytes, value=1048 bytes))], logging-5=[(record=DefaultRecord(offset=0, timestamp=1521353380881, key=0 bytes, value=1048 bytes))], logging-13=[(record=DefaultRecord(offset=0, timestamp=1521353381288, key=0 bytes, value=1048 bytes))]}), transactionalId=''
      22 Mar 2018 02:06:20.156 [mirrormaker-thread-0] TRACE org.apache.kafka.clients.producer.KafkaProducer trace(line:135) [Producer clientId=producer-1] Sending record ProducerRecord(topic=logging, partition=null, headers=RecordHeaders(headers = [], isReadOnly = true), key=null, value=[B@cf56dcea, timestamp=1521353382731) with callback kafka.tools.MirrorMaker$MirrorMakerProducerCallback@e7d47b69 to topic logging partition 17
      22 Mar 2018 02:06:20.156 [mirrormaker-thread-0] TRACE org.apache.kafka.clients.producer.internals.RecordAccumulator trace(line:135) [Producer clientId=producer-1] Allocating a new 16384 byte message buffer for topic logging partition 17
      22 Mar 2018 02:06:20.156 [mirrormaker-thread-0] TRACE org.apache.kafka.clients.producer.KafkaProducer trace(line:129) [Producer clientId=producer-1] Waking up the sender since topic logging partition 17 is either full or getting a new batch
      22 Mar 2018 02:06:20.164 [mirrormaker-thread-0] TRACE org.apache.kafka.clients.producer.KafkaProducer trace(line:135) [Producer clientId=producer-1] Sending record ProducerRecord(topic=logging, partition=null, headers=RecordHeaders(headers = [], isReadOnly = true), key=null, value=[B@d6e47439, timestamp=1521353385041) with callback kafka.tools.MirrorMaker$MirrorMakerProducerCallback@60717e86 to topic logging partition 17
      22 Mar 2018 02:06:20.172 [mirrormaker-thread-0] TRACE org.apache.kafka.clients.producer.KafkaProducer trace(line:135) [Producer clientId=producer-1] Sending record ProducerRecord(topic=logging, partition=null, headers=RecordHeaders(headers = [], isReadOnly = true), key=null, value=[B@613a02ed, timestamp=1521353387109) with callback kafka.tools.MirrorMaker$MirrorMakerProducerCallback@563a082b to topic logging partition 17
      ...
      30s later (request.timeout.ms=30000)
      ...
      22 Mar 2018 02:06:50.156 [kafka-producer-network-thread | producer-1] DEBUG org.apache.kafka.clients.NetworkClient debug(line:183) [Producer clientId=producer-1] Disconnecting from node 44 due to request timeout.
      22 Mar 2018 02:06:50.156 [kafka-producer-network-thread | producer-1] TRACE org.apache.kafka.clients.NetworkClient trace(line:135) [Producer clientId=producer-1] Cancelled request {acks=-1,timeout=30000,partitionSizes=[logging-1=647,logging-9=645,logging-5=647,logging-13=645]} with correlation id 71599 due to node 44 being disconnected
      22 Mar 2018 02:06:50.156 [kafka-producer-network-thread | producer-1] TRACE org.apache.kafka.clients.producer.internals.Sender trace(line:129) [Producer clientId=producer-1] Cancelled request with header RequestHeader(apiKey=PRODUCE, apiVersion=5, clientId=producer-1, correlationId=71599) due to node 44 being disconnected
      22 Mar 2018 02:06:50.156 [kafka-producer-network-thread | producer-1] WARN  org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer clientId=producer-1] Got error produce response with correlation id 71599 on topic-partition logging-1, retrying (2147483646 attempts left). Error: NETWORK_EXCEPTION
      22 Mar 2018 02:06:50.157 [kafka-producer-network-thread | producer-1] WARN  org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer clientId=producer-1] Got error produce response with correlation id 71599 on topic-partition logging-9, retrying (2147483646 attempts left). Error: NETWORK_EXCEPTION
      22 Mar 2018 02:06:50.157 [kafka-producer-network-thread | producer-1] WARN  org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer clientId=producer-1] Got error produce response with correlation id 71599 on topic-partition logging-5, retrying (2147483646 attempts left). Error: NETWORK_EXCEPTION
      22 Mar 2018 02:06:50.157 [kafka-producer-network-thread | producer-1] WARN  org.apache.kafka.clients.producer.internals.Sender warn(line:251) [Producer clientId=producer-1] Got error produce response with correlation id 71599 on topic-partition logging-13, retrying (2147483646 attempts left). Error: NETWORK_EXCEPTION
      22 Mar 2018 02:06:50.157 [kafka-producer-network-thread | producer-1] TRACE org.apache.kafka.clients.producer.internals.Sender trace(line:123) [Producer clientId=producer-1] Expired 1 batches in accumulator
      22 Mar 2018 02:06:50.157 [kafka-producer-network-thread | producer-1] TRACE org.apache.kafka.clients.producer.internals.ProducerBatch done(line:174) Failed to produce messages to logging-17.
      org.apache.kafka.common.errors.TimeoutException: Expiring 25 record(s) for logging-17: 30001 ms has passed since batch creation plus linger time
      22 Mar 2018 02:06:50.162 [kafka-producer-network-thread | producer-1] ERROR org.apache.kafka.clients.producer.internals.ErrorLoggingCallback onCompletion(line:52) Error when sending message to topic logging with key: null, value: 1047 bytes with error:
      org.apache.kafka.common.errors.TimeoutException: Expiring 25 record(s) for logging-17: 30001 ms has passed since batch creation plus linger time
      22 Mar 2018 02:06:50.162 [kafka-producer-network-thread | producer-1] INFO  kafka.tools.MirrorMaker$ info(line:72) Closing producer due to send failure.
      22 Mar 2018 02:06:50.163 [kafka-producer-network-thread | producer-1] INFO  org.apache.kafka.clients.producer.KafkaProducer info(line:341) [Producer clientId=producer-1] Closing the Kafka producer with timeoutMillis = 0 ms.
      22 Mar 2018 02:06:50.163 [kafka-producer-network-thread | producer-1] INFO  org.apache.kafka.clients.producer.KafkaProducer info(line:341) [Producer clientId=producer-1] Proceeding to force close the producer since pending requests could not be completed within timeout 0 ms.
      22 Mar 2018 02:06:50.164 [kafka-producer-network-thread | producer-1] DEBUG org.apache.kafka.clients.producer.KafkaProducer debug(line:177) [Producer clientId=producer-1] Kafka producer has been closed
      

       The first few lines shows that the producer just received a response from its previous produce request to broker 44 with partitions {1,5,9,13,17} (leaders).
      The next produce request it sent only contained batches for partitions {1,5,9,13}, however 1ms later new data arrived for partition 17 but since it missed the current request to that broker it had to sit in the producer and wait for the next send round.
      Somehow the produce request for {1,5,9,13} never got a response and timed out. Normally these batches can be retried and mirrormaker will keep working, however because of the leftover batch for partition 17 expired, mirrormaker detected data loss and will shutdown.

      So I think the root problem is the NETWORK_EXCEPTION where the server intermittently times out for the produce request after upgrading to 1.0.
      In the lucky case where all batches can be retried (they seem to always succeed on the first retry anyway), it can randomly introduce 30s of latency to data replication, and if unlucky a batch get left out for the initial request and expires, the task immediately dies.
      We observed that this usually happens under high load with large number of msg from the source topic to mirror (I can reliably recreate the errors by restarting the task with a new consumer group id and auto.offset.reset=earliest so it got 50 mil msg to mirror from the start of the topic)

      Increasing producer linger.ms and batch.size seems to some degree reduce the error rate under same heavy workload.

      I tried to enable trace on the server but all I see are logs (massive) from ReplicaFetcher and lines like this:

      Bytes written as part of multi-send call: 2431, total bytes written so far: 2431, expected bytes to write: 2431

      Is there any log on the server side that can be correlated to the producer request and help debugging this?

      Also I still got no idea what the REQUEST_TIMED_OUT meant and what's causing it. 

      This is the trace log config I tried to use on the server but couldn't see anything useful:

       rootLogger.level=trace
       appender.console.type=Console
       appender.console.name=STDOUT
       rootLogger.appenderRef.stdout.ref=STDOUT

       

      Here's the mirrormaker producer config:

      	acks = all
      	batch.size = 16384
      	bootstrap.servers = [...]
      	buffer.memory = 33554432
      	client.id = 
      	compression.type = gzip
      	connections.max.idle.ms = 540000
      	enable.idempotence = false
      	interceptor.classes = null
      	key.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer
      	linger.ms = 0
      	max.block.ms = 9223372036854775807
      	max.in.flight.requests.per.connection = 1
      	max.request.size = 2097152
      	metadata.max.age.ms = 300000
      	metric.reporters = []
      	metrics.num.samples = 2
      	metrics.recording.level = INFO
      	metrics.sample.window.ms = 30000
      	partitioner.class = class org.apache.kafka.clients.producer.internals.DefaultPartitioner
      	receive.buffer.bytes = 32768
      	reconnect.backoff.max.ms = 1000
      	reconnect.backoff.ms = 50
      	request.timeout.ms = 30000
      	retries = 2147483647
      	retry.backoff.ms = 100
      	sasl.jaas.config = null
      	sasl.kerberos.kinit.cmd = /usr/bin/kinit
      	sasl.kerberos.min.time.before.relogin = 60000
      	sasl.kerberos.service.name = null
      	sasl.kerberos.ticket.renew.jitter = 0.05
      	sasl.kerberos.ticket.renew.window.factor = 0.8
      	sasl.mechanism = GSSAPI
      	security.protocol = PLAINTEXT
      	send.buffer.bytes = 131072
      	ssl.cipher.suites = null
      	ssl.enabled.protocols = [TLSv1.2, TLSv1.1, TLSv1]
      	ssl.endpoint.identification.algorithm = null
      	ssl.key.password = null
      	ssl.keymanager.algorithm = IbmX509
      	ssl.keystore.location = null
      	ssl.keystore.password = null
      	ssl.keystore.type = JKS
      	ssl.protocol = TLS
      	ssl.provider = null
      	ssl.secure.random.implementation = null
      	ssl.trustmanager.algorithm = PKIX
      	ssl.truststore.location = null
      	ssl.truststore.password = null
      	ssl.truststore.type = JKS
      	transaction.timeout.ms = 60000
      	transactional.id = null
      	value.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer

      Attachments

        Activity

          People

            Unassigned Unassigned
            shangdi Di Shang
            Votes:
            4 Vote for this issue
            Watchers:
            8 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: