You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 5 Next »

Status

Current state: Under Discussion

Discussion thread: <TODO>

JIRA: Unable to render Jira issues macro, execution error.

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum changes the way cluster metadata is stored and managed in a Kafka cluster. It introduces the concept of a replicated log that is maintained using a custom version of the Raft consensus protocol described in KIP-595: A Raft Protocol for the Metadata Quorum. The controller now utilizes this log to persist and broadcast all metadata related actions in the cluster as described in KIP-631: The Quorum-based Kafka Controller.

With these changes in place, the replicated log containing all metadata changes (henceforth called metadata log) is the source of metadata related information for all nodes in the cluster. Any errors that occur while processing the log could lead to the in-memory state for the node becoming inconsistent. It is important that such errors are made visible. The metrics proposed in the following section aim at doing so. These metrics can be used to set up alerts so that affected nodes can be discovered and needed remedial actions can be performed on them.

Public Interfaces

We propose adding the following new metrics:

NameDescription
kafka.server:type=broker-metadata-metrics,name=publisher-error-countReports the number of errors encountered by the BrokerMetadataPublisher while publishing a new MetadataImage based on the MetadataDelta 
kafka.server:type=broker-metadata-metrics,name=listener-batch-load-error-countReports the number of errors encountered by the BrokerMetadataListener while generating a new MetadataDelta based on the log it has received thus far.
kafka.controller:type=KafkaController,name=ForceRenounceCountReports the number of times this controller node has renounced leadership of the metadata quorum owing to an error encountered during event processing

Proposed Changes

ControllerForceRenounceCount and MetadataPublisherErrorCount 

The MetadataPublisherErrorCount metric reflects the count of errors encountered while publishing a new version of the MetadataImage using the metadata log.


Both these metrics can be used to set up alerts so that affected nodes are visible and needed remedial actions can be performed on them.

Controllers

Any errors during metadata processing on the Active Controller cause it to renounce the quorum leadership. These are different than the general Raft elections being triggered due to other reasons like a roll. Repeated elections being caused due to errors in the active controller could point to issues in the metadata log generation handling logic and having visibility into these would be helpful. The ControllerForceRenounceCount metric reflects the number of times a controller node has had to renounce quorum leadership due to an error in the event processing logic.

The ForceRenounceCount metric will be incremented anytime the controller is going to resign as a result of handling exceptions in Event Processing

https://github.com/apache/kafka/blob/14d2269471141067dc3c45300187f20a0a051777/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java#L409 

handleEventException
 private Throwable handleEventException(String name,
                                           OptionalLong startProcessingTimeNs,
                                           Throwable exception) {
        if (!startProcessingTimeNs.isPresent()) {
        ...
        ...
        renounce();
		//**** Increment ForceRenounceCount
        return new UnknownServerException(exception);
    }

Brokers

The publisher-error-count metric will be incremented by one every time there is an error in publishing a new MetadataImage 

https://github.com/apache/kafka/blob/14d2269471141067dc3c45300187f20a0a051777/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala#L125

Publish
override def publish(delta: MetadataDelta, newImage: MetadataImage): Unit = {
    val highestOffsetAndEpoch = newImage.highestOffsetAndEpoch()

    try {
      trace(s"Publishing delta $delta with highest offset $highestOffsetAndEpoch")

      // Publish the new metadata image to the metadata cache.
      metadataCache.setImage(newImage)
   	  ...
	  ...
      publishedOffsetAtomic.set(newImage.highestOffsetAndEpoch().offset)
    } catch {
	  //**** Increment publisher-error-count
      case t: Throwable => error(s"Error publishing broker metadata at $highestOffsetAndEpoch", t)	
        throw t
    } finally {
      _firstPublish = false
    }
  }

 

The listener-batch-load-error-count metric will be incremented every time there is an error in loading batches and generating MetadataDelta from them.

https://github.com/apache/kafka/blob/14d2269471141067dc3c45300187f20a0a051777/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala#L112

HandleCommitsEvent
class HandleCommitsEvent(reader: BatchReader[ApiMessageAndVersion])
      extends EventQueue.FailureLoggingEvent(log) {
    override def run(): Unit = {
      val results = try {
        val loadResults = loadBatches(_delta, reader, None, None, None)
        ...
        loadResults
      } catch {
		//**** Increment listener-batch-load-error-count
	  } finally {
        reader.close()
      }

      ...
      _publisher.foreach(publish)
    }
  }


https://github.com/apache/kafka/blob/14d2269471141067dc3c45300187f20a0a051777/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala#L162

HandleSnapshotEvent
  class HandleSnapshotEvent(reader: SnapshotReader[ApiMessageAndVersion])
    extends EventQueue.FailureLoggingEvent(log) {
    override def run(): Unit = {
      try {
        info(s"Loading snapshot ${reader.snapshotId().offset}-${reader.snapshotId().epoch}.")
        _delta = new MetadataDelta(_image) // Discard any previous deltas.
        val loadResults = loadBatches(
       ...
      } catch {
		//**** Increment listener-batch-load-error-count
	  } finally {
        reader.close()
      }
      _publisher.foreach(publish)
    }
  }


Compatibility, Deprecation, and Migration Plan

These will be newly exposed metrics and there will be no impact on existing kafka versions.

Rejected Alternatives

Instead of adding the specific metrics, we could have added a more generic MetadataProcessingErrorCount Metric which would be incremented when either of these (and any other similar) or any other similar errors are hit. The downside to this approach would be the loss in granularity on what exactly failed on a given node. The specific metrics are more meaningful and give better control over any alarming that might be setup on these metrics.

  • No labels