Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

Public Interfaces

We will introduce a two new metric metrics and a new TopicCommand option to identify AtMinIsr partitions.

Code Block
# New metric
ReplicaManager.AtMinIsrPartitionsmetrics
- kafka.server:name=AtMinIsrPartitions,type=ReplicaManager
- kafka.cluster:name=AtMinIsr,type=Partition,topic={topic},partition={partition}

# New TopicCommand option
--at-min-isr-partitions

...

Code Block
newGauge("AtMinIsr",
  new Gauge[Int] {
    def value = {
      if (isAtMinIsr) 1 else 0
    }
  },
  tags
)


...


def isAtMinIsr: Boolean = {
  leaderReplicaIfLocal match {
    case Some(leaderReplica) =>
      inSyncReplicas.size == leaderReplica.log.get.config.minInSyncReplicas
    case None =>
      false
  }
}

And the similar change to ReplicaManager.scala:

Code Block
val atMinIsrPartitionCount = newGauge(
  "AtMinIsrPartitionCount",
  new Gauge[Int] {
    def value = leaderPartitionsIterator.count(_.isAtMinIsr)
  }
)

And TopicCommand.scala:

Code Block
private val reportAtMinIsrPartitionsOpt = parser.accepts("at-min-isr-partitions",
  "if set when describing topics, only show partitions whose isr count is equal to the configured minimum. Not supported with the --zookeeper option.")

private def hasAtMinIsrPartitions(partitionDescription: PartitionDescription) = {
  partitionDescription.isr.size == partitionDescription.minIsrCount
}

private def shouldPrintAtMinIsrPartitions(partitionDescription: PartitionDescription) = {
  opts.reportAtMinIsrPartitions && hasAtMinIsrPartitions(partitionDescription)
}

# Some other minor changes omitted

...