Browse Source

KAFKA-5987: Maintain order of metric tags in generated documentation

The `MetricNameTemplate` is changed to used a `LinkedHashSet` to maintain the same order of the tags that are passed in. This tag order is then maintained when `Metrics.toHtmlTable` generates the MBean names for each of the metrics.

The `SenderMetricsRegistry` and `FetcherMetricsRegistry` both contain templates used in the producer and consumer, respectively, and these were changed to use a `LinkedHashSet` to maintain the order of the tags.

Before this change, the generated HTML documentation might use MBean names like the following and order them:

```
kafka.connect:type=sink-task-metrics,connector={connector},partition={partition},task={task},topic={topic}
kafka.connect:type=sink-task-metrics,connector={connector},task={task}
```
However, after this change, the documentation would use the following order:
```
kafka.connect:type=sink-task-metrics,connector={connector},task={task}
kafka.connect:type=sink-task-metrics,connector={connector},task={task},topic={topic},partition={partition}
```

This is more readable as the code that is creating the templates has control over the order of the tags.

Note that JMX MBean names use ObjectName that does not maintain order of the properties (tags), so this change should have no impact on the actual JMX MBean names used in the metrics.

cc wushujames

Author: Randall Hauch <rhauch@gmail.com>

Reviewers: James Cheng <jylcheng@yahoo.com>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes #3985 from rhauch/kafka-5987
pull/3985/merge
Randall Hauch 7 years ago committed by Ewen Cheslack-Postava
parent
commit
447c64fffb
  1. 3
      clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetcherMetricsRegistry.java
  2. 8
      clients/src/main/java/org/apache/kafka/clients/producer/internals/SenderMetricsRegistry.java
  3. 58
      clients/src/main/java/org/apache/kafka/common/MetricNameTemplate.java
  4. 14
      clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java

3
clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetcherMetricsRegistry.java

@ -18,6 +18,7 @@ package org.apache.kafka.clients.consumer.internals; @@ -18,6 +18,7 @@ package org.apache.kafka.clients.consumer.internals;
import java.util.Arrays;
import java.util.HashSet;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Set;
@ -102,7 +103,7 @@ public class FetcherMetricsRegistry { @@ -102,7 +103,7 @@ public class FetcherMetricsRegistry {
"The maximum throttle time in ms", tags);
/***** Topic level *****/
Set<String> topicTags = new HashSet<>(tags);
Set<String> topicTags = new LinkedHashSet<>(tags);
topicTags.add("topic");
this.topicFetchSizeAvg = new MetricNameTemplate("fetch-size-avg", groupName,

8
clients/src/main/java/org/apache/kafka/clients/producer/internals/SenderMetricsRegistry.java

@ -17,7 +17,7 @@ @@ -17,7 +17,7 @@
package org.apache.kafka.clients.producer.internals;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -70,12 +70,12 @@ public class SenderMetricsRegistry { @@ -70,12 +70,12 @@ public class SenderMetricsRegistry {
private final Metrics metrics;
private final Set<String> tags;
private final HashSet<String> topicTags;
private final LinkedHashSet<String> topicTags;
public SenderMetricsRegistry(Metrics metrics) {
this.metrics = metrics;
this.tags = this.metrics.config().tags().keySet();
this.allTemplates = new ArrayList<MetricNameTemplate>();
this.allTemplates = new ArrayList<>();
/***** Client level *****/
@ -126,7 +126,7 @@ public class SenderMetricsRegistry { @@ -126,7 +126,7 @@ public class SenderMetricsRegistry {
"The maximum time in ms a request was throttled by a broker");
/***** Topic level *****/
this.topicTags = new HashSet<String>(tags);
this.topicTags = new LinkedHashSet<>(tags);
this.topicTags.add("topic");
// We can't create the MetricName up front for these, because we don't know the topic name yet.

58
clients/src/main/java/org/apache/kafka/common/MetricNameTemplate.java

@ -16,7 +16,7 @@ @@ -16,7 +16,7 @@
*/
package org.apache.kafka.common;
import java.util.HashSet;
import java.util.LinkedHashSet;
import java.util.Objects;
import java.util.Set;
@ -26,27 +26,45 @@ import org.apache.kafka.common.utils.Utils; @@ -26,27 +26,45 @@ import org.apache.kafka.common.utils.Utils;
* A template for a MetricName. It contains a name, group, and description, as
* well as all the tags that will be used to create the mBean name. Tag values
* are omitted from the template, but are filled in at runtime with their
* specified values.
* specified values. The order of the tags is maintained, if an ordered set
* is provided, so that the mBean names can be compared and sorted lexicographically.
*/
public class MetricNameTemplate {
private final String name;
private final String group;
private final String description;
private Set<String> tags;
private LinkedHashSet<String> tags;
public MetricNameTemplate(String name, String group, String description, Set<String> tags) {
/**
* Create a new template. Note that the order of the tags will be preserved if the supplied
* {@code tagsNames} set has an order.
*
* @param name the name of the metric; may not be null
* @param group the name of the group; may not be null
* @param description the description of the metric; may not be null
* @param tagsNames the set of metric tag names, which can/should be a set that maintains order; may not be null
*/
public MetricNameTemplate(String name, String group, String description, Set<String> tagsNames) {
this.name = Utils.notNull(name);
this.group = Utils.notNull(group);
this.description = Utils.notNull(description);
this.tags = Utils.notNull(tags);
this.tags = new LinkedHashSet<>(Utils.notNull(tagsNames));
}
public MetricNameTemplate(String name, String group, String description, String... keys) {
this(name, group, description, getTags(keys));
/**
* Create a new template. Note that the order of the tags will be preserved.
*
* @param name the name of the metric; may not be null
* @param group the name of the group; may not be null
* @param description the description of the metric; may not be null
* @param tagsNames the names of the metric tags in the preferred order; none of the tag names should be null
*/
public MetricNameTemplate(String name, String group, String description, String... tagsNames) {
this(name, group, description, getTags(tagsNames));
}
private static Set<String> getTags(String... keys) {
Set<String> tags = new HashSet<String>();
private static LinkedHashSet<String> getTags(String... keys) {
LinkedHashSet<String> tags = new LinkedHashSet<>();
for (int i = 0; i < keys.length; i++)
tags.add(keys[i]);
@ -54,18 +72,38 @@ public class MetricNameTemplate { @@ -54,18 +72,38 @@ public class MetricNameTemplate {
return tags;
}
/**
* Get the name of the metric.
*
* @return the metric name; never null
*/
public String name() {
return this.name;
}
/**
* Get the name of the group.
*
* @return the group name; never null
*/
public String group() {
return this.group;
}
/**
* Get the description of the metric.
*
* @return the metric description; never null
*/
public String description() {
return this.description;
}
/**
* Get the set of tag names for the metric.
*
* @return the ordered set of tag names; never null but possibly empty
*/
public Set<String> tags() {
return tags;
}

14
clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java

@ -232,12 +232,22 @@ public class Metrics implements Closeable { @@ -232,12 +232,22 @@ public class Metrics implements Closeable {
return tags;
}
public static String toHtmlTable(String domain, List<MetricNameTemplate> allMetrics) {
/**
* Use the specified domain and metric name templates to generate an HTML table documenting the metrics. A separate table section
* will be generated for each of the MBeans and the associated attributes. The MBean names are lexicographically sorted to
* determine the order of these sections. This order is therefore dependent upon the order of the
* tags in each {@link MetricNameTemplate}.
*
* @param domain the domain or prefix for the JMX MBean names; may not be null
* @param allMetrics the collection of all {@link MetricNameTemplate} instances each describing one metric; may not be null
* @return the string containing the HTML table; never null
*/
public static String toHtmlTable(String domain, Iterable<MetricNameTemplate> allMetrics) {
Map<String, Map<String, String>> beansAndAttributes = new TreeMap<String, Map<String, String>>();
try (Metrics metrics = new Metrics()) {
for (MetricNameTemplate template : allMetrics) {
Map<String, String> tags = new TreeMap<String, String>();
Map<String, String> tags = new LinkedHashMap<>();
for (String s : template.tags()) {
tags.put(s, "{" + s + "}");
}

Loading…
Cancel
Save