Upgrade to Pro — share decks privately, control downloads, hide ads and more …

SF Big Analytics July 2017 - Pushing the Limits of Kafka

SF Big Analytics July 2017 - Pushing the Limits of Kafka

This was a talk given at SF Big Analytics in July 2017. This talk details a subset of challenges Heroku has encountered in operating Apache Kafka as a service in production for the last several years. For each challenge and incident, we talk about the observed behaviors, followed by deep dive into Kafka internals, and then key insights, takeaways, and practical fixes.

Jeff Chao

July 07, 2017
Tweet

More Decks by Jeff Chao

Other Decks in Programming

Transcript

  1. Beyond 50,000 Partitions How Heroku Operates and Pushes the Limits

    of Kafka at Scale Jeff Chao SF Big Analytics, July 2017
  2. m 1 m 2 . . . m n m

    1 m 2 . . . m n m 1 m 2 . . . m n m 1 m 2 . . . m n m 1 m 2 . . . m n m 1 m 2 . . . m n m 1 m 2 . . . m n m 1 m 2 . . . m n m 1 m 2 . . . m n m 1 m 2 . . . m n Topic: user_events Topic: system_events Topic: alerts Topic: control_messages
  3. m 1 m 2 . . . m n m

    1 m 2 . . . m n m 1 m 2 . . . m n m 1 m 2 . . . m n m 1 m 2 . . . m n m 1 m 2 . . . m n ( low throughput ) ( high throughput ) ( oldest ) ( newest )
  4. Topic : user_events p0 p1 p2 Consumers Consumer 1 Consumer

    2 p2, offset: 25 p0, offset: 10 p1, offset: 5
  5. OS, JVM, Kafka Tuning Heap, off-heap, I/O threads, network request

    threads, controllers, leaders, logs, log segments
  6. Monitoring & Metrics VPC, SSL, JMX, RMI, Jolokia Networking Disk

    Automatic volume growth, corruption & replacement Logging
  7. Version upgrades How do you upgrade to new versions? What

    do you do when faced with breaking changes?
  8. 1

  9. Network threads getting NPEs Observed behavior 3 Observed behavior 1

    Brokers were crashing and being restarted Users were exceeding their produce quotas Observed behavior 2
  10. ( quota = 5 MB/s ) Producers { “id”: 1,

    “msg”: “...” } { “id”: 2, “msg”: “...” } { “id”: 5, “msg”: “...” } { “id”: 4, “msg”: “...” } { “id”: 2, “msg”: “...” } { “id”: 3, “msg”: “...” } { “id”: 3, “msg”: “...” } { “id”: 1, “msg”: “...” } { “id”: 2, “msg”: “...” } { “id”: 1, “msg”: “...” } ( write throughput = 10 MB/s ) ( throughput = 10 MB/s ) { “id”: 1, “msg”: “...” } { “id”: 2, “msg”: “...” } { “id”: 5, “msg”: “...” } { “id”: 4, “msg”: “...” } { “id”: 2, “msg”: “...” } User’s Perspective { “id”: 3, “msg”: “...” } { “id”: 3, “msg”: “...” } { “id”: 1, “msg”: “...” } { “id”: 2, “msg”: “...” } { “id”: 1, “msg”: “...” } ( delayed )
  11. Network threads getting NPEs Observed behavior 3 Observed behavior 1

    Brokers were crashing and being restarted Users were exceeding their produce quotas Observed behavior 2
  12. ReplicaManager#appendRecords // true if required acks = all, data to

    append, or partial success if (delayedProduceRequestRequired(requiredAcks, ...)) { ... // calls responseCallback eventually delayedProducePurgatory.tryCompleteElseWatch(...) } else { ... responseCallback(produceResponseStatus) }
  13. ClientQuotaManager#recordAndMaybeThrottle try { // trigger the callback immediately if quota

    is not violated callback(0) } catch { case _: QuotaViolationException => val throttleTimeMs = ... // Compute the delay clientSensors.throttleTimeSensor.record(throttleTimeMs) delayQueue.add(new ThrottledResponse(...)) }
  14. Some brokers booted, but suddenly fail again Observed behavior 3

    Observed behavior 1 Brokers restarted, not able to boot up Cascading failure of brokers Observed behavior 2
  15. What is Kafka’s HA Model? What happens when a broker

    goes down? What happens when a broker comes up?
  16. p0-lead user_events p2-lead p1-lead p0 writes reads ( replicated )

    p1 p2 [ p0, p1, p2 ] p1 p2 p0 p2 p1 p0 other broker other broker
  17. Example - load distribution A broker will be leader for

    10 partitions (on average, 50 partitions / 5 brokers)
  18. Example - load distribution Process read and write requests For

    10 out of 50 partitions for that given topic
  19. Example - fault tolerance Replication factor = 3 is set

    per topic 2 additional copies of each partition will live on other brokers in the cluster
  20. Each leader maintains In-Sync Replicas Leaders manage a set of

    In-Sync Replicas (ISR) that represent all followers that are caught up
  21. Leaders will evict lagging replicas If followers get stuck or

    lagged, their leader will remove them from the ISR
  22. What is Kafka’s HA Model? What happens when a broker

    goes down? What happens when a broker comes up?
  23. PartitionStateMachine.scala NonExistentPartition -> NewPartition NewPartition -> OnlinePartition OnlinePartition, OfflinePartition ->

    OnlinePartition NewPartition, OnlinePartition, OfflinePartition -> OfflinePartition OfflinePartition -> NonExistentPartition
  24. KafkaServer#shutdown controlledShutdown() brokerState.newState(BrokerShuttingDown) if (socketServer != null) socketServer.shutdown() if (requestHandlerPool

    != null) requestHandlerPool.shutdown() // ... Shutdown other managers and listeners ... info(“shut down completed”) Clean Shutdown
  25. PartitionStateMachine#handleStateChange try { targetState match { ... case OfflinePartition =>

    // pre: partition should be in New or Online state assertValidPreviousStates(...) partitionState.put(topicAndPartition, OfflinePartition) // post: partition has no alive leader ... } } Unclean Shutdown
  26. KafkaController#onBrokerFailure val deadBrokersThatWereShuttingDown = ... val partitionsWithoutLeader = ... partitionStateMachine.handleStateChanges(partitionsWithoutLeader,

    ...) partitionStateMachine.triggerOnlinePartitionStateChange() if (partitionsWithoutLeader.isEmpty) { sendUpdateMetadataRequest(context.liveOrShuttingDownBrokerIds.toSeq) } Unclean Shutdown
  27. What is Kafka’s HA Model? What happens when a broker

    goes down? What happens when a broker comes up?
  28. KafkaServer#startup (abridged) logManager.startup() metadataCache = new MetadataCache(config.brokerId) credentialProvider = new

    CredentialProvider(config.saslEnabledMechanisms) socketServer.startup() replicaManager.startup() kafkaController.startup() groupCoordinator.startup() apis = new KafkaApis(...) requestHandlerPool = new KafkaRequestHandlerPool(...) dynamicConfigManager.startup() Broker Starting
  29. KafkaServer#startup (cont.) /* tell everyone we are alive */ val

    listeners = config.advertisedListeners.map { endpoint => ... } kafkaHealthcheck.startup() checkpointBrokerId(config.brokerId) registerStats() brokerState.newState(RunningAsBroker) startupComplete.set(true) isStartingUp.set(false) AppInfoParser.registerAppInfo(jmxPrefix, config.brokerId.toString) info(“started”) Broker Starting
  30. Automation would restart brokers just fine Recovery could take longer

    than expected Automation would time out the broker
  31. Under-replicated partitions, request timeouts Observed behavior 3 Observed behavior 1

    Brokers crashing at various stages on boot Long period of stability, sudden failure Observed behavior 2
  32. Log Anatomy of a Partition (a Log) m 1 m

    2 m 3 m 4 m 5 log segment 1 m 6 m 7 m 8 m 9 m 10 log segment 2 m 11 m 12 m 13 m 14 m 15 log segment 3 00:00 - 03:00 or 0 GB - 1 GB 03:00 - 06:00 or 1 GB - 2 GB 06:00 - 09:00 or 2 GB - 3 GB
  33. fs.file-max={system_file_descriptor_limit} // sysctl.conf {system_user} - nofile {file_descriptor_limit} // limits.conf $

    echo ‘Increase file descriptors and keep a buffer for the system.’ The Fix
  34. Rule of Thumb ~1.5 file descriptors per log segment Log

    segments consists of 3 actual files on disk: .log, .index, and .timeindex However, in our testing, we found on average, ~1.5 FDs per log segment Kafka tries to hold FDs only for active segments
  35. Rule of Thumb Inputs (per broker) partitions_replicas = (topics *

    partitions * replication_factor) / brokers
  36. Rule of Thumb Inputs (per broker) partitions_replicas = (topics *

    partitions * replication_factor) / brokers segments_per_partition_replicas = retention_time / log.roll.hours
  37. Rule of Thumb Inputs (per broker) partitions_replicas = (topics *

    partitions * replication_factor) / brokers segments_per_partition_replicas = retention_time / log.roll.hours fd_buffer = tunable (e.g., 1, 1.5, 2) fd = partitions_replicas * segments_per_partition_replicas * fd_buffer
  38. Under-replicated partitions, request timeouts Observed behavior 3 Observed behavior 1

    Brokers crashing at various stages on boot Long period of stability, sudden failure Observed behavior 2
  39. pri=FATAL t=ReplicaFetcherThread-0-6 at=ReplicaFetcherThread [ReplicaFetcherThread-0-6], Disk error while replicating data for

    salmon-84320.messages4-24 kafka.common.KafkaStorageException: I/O exception in append to log
  40. pri=ERROR t=kafka-request-handler-6 at=logger Error on broker 1 while processing LeaderAndIsr

    request with correlationId 8 received from controller 1 epoch 244 java.io.IOException: Map failed at sun.nio.ch.FileChannelImpl.map ... at kafka.utils.CoreUtils$.inLock at kafka.log.AbstractIndex.resize at kafka.log.LogSegment.truncateTo
  41. pri=FATAL t=main at=KafkaServerStartable Fatal error during KafkaServerStartable startup. Prepare to

    shutdown java.io.IOException: Map failed at sun.nio.ch.FileChannelImpl.map ... at kafka.log.AbstractIndex.resize at kafka.log.AbstractIndex.trimToValidSize at kafka.log.LogSegment.recover at kafka.log.Log.recoverLog at kafka.log.Log.loadSegments
  42. [GC pause (G1 Evacuation Pause) (young), 0.0167712 secs] ... [Eden:

    420.0M(192.0M)->0.0B(998.0M) Survivors: 12.0M->12.0M Heap: 1809.5M(4096.0M)->1389.5M(4096.0M)] [Times: user=0.28 sys=0.00, real=0.03 secs]
  43. pri=ERROR t=kafka-request-handler-6 at=logger Error on broker 1 while processing LeaderAndIsr

    request with correlationId 8 received from controller 1 epoch 244 java.io.IOException: Map failed at sun.nio.ch.FileChannelImpl.map ... at kafka.utils.CoreUtils$.inLock at kafka.log.AbstractIndex.resize at kafka.log.LogSegment.truncateTo
  44. FileChannelImpl.c#Java_..._FileChannelImpl_map0 mapAddress = mmap64(...); if (mapAddress == MAP_FAILED) { if

    (errno == ENOMEM) { JNU_ThrowOutOfMemoryError(env, “Map failed”); return IOS_THROWN; } return handle(env, -1, “Map failed”); }
  45. pri=ERROR t=kafka-request-handler-6 at=logger Error on broker 1 while processing LeaderAndIsr

    request with correlationId 8 received from controller 1 epoch 244 java.io.IOException: Map failed at sun.nio.ch.FileChannelImpl.map ... at kafka.utils.CoreUtils$.inLock at kafka.log.AbstractIndex.resize at kafka.log.LogSegment.truncateTo
  46. Ran with new configuration in staging For completeness, under heavy

    load for a long period of time Ruled out a potential leak scenario No FD or mmap leaks
  47. Kafka & internals Supports many use cases, many things to

    configure, scalable Heroku offers fully-managed Apache Kafka Integrated with the Heroku ecosystem Lots to consider when operating at scale An operational undertaking
  48. Challenge 1 How quotas work Release-blocking heap-based OOM bug around

    quota management and shipped a patch upstream that landed in 0.10.2
  49. Challenge 3 How partitions are implemented What kind of resources

    are needed More partitions requires more file descriptors and memory-mapped byte buffers