You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/02/05 20:16:00 UTC

[jira] [Commented] (KAFKA-5987) Kafka metrics templates used in document generation should maintain order of tags

    [ https://issues.apache.org/jira/browse/KAFKA-5987?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16352874#comment-16352874 ] 

ASF GitHub Bot commented on KAFKA-5987:
---------------------------------------

ewencp closed pull request #3985: KAFKA-5987: Maintain order of metric tags in generated documentation
URL: https://github.com/apache/kafka/pull/3985
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetcherMetricsRegistry.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetcherMetricsRegistry.java
index acf42ec339f..0a2d43a4b61 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetcherMetricsRegistry.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetcherMetricsRegistry.java
@@ -18,6 +18,7 @@
 
 import java.util.Arrays;
 import java.util.HashSet;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Set;
 
@@ -98,7 +99,7 @@ public FetcherMetricsRegistry(Set<String> tags, String metricGrpPrefix) {
                 "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, 
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/SenderMetricsRegistry.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/SenderMetricsRegistry.java
index 21dbca61830..b01236f6218 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/SenderMetricsRegistry.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/SenderMetricsRegistry.java
@@ -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 @@
     
     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 SenderMetricsRegistry(Metrics metrics) {
                 "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.
diff --git a/clients/src/main/java/org/apache/kafka/common/MetricNameTemplate.java b/clients/src/main/java/org/apache/kafka/common/MetricNameTemplate.java
index e3ea9950ef1..1b1de71037d 100644
--- a/clients/src/main/java/org/apache/kafka/common/MetricNameTemplate.java
+++ b/clients/src/main/java/org/apache/kafka/common/MetricNameTemplate.java
@@ -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 @@
  * 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 MetricNameTemplate(String name, String group, String description, String.
         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;
     }
diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java b/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java
index 0b4507b3aef..9b7a9f38460 100644
--- a/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java
@@ -232,12 +232,22 @@ public MetricName metricName(String name, String group, Map<String, String> tags
         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 + "}");
                 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


> Kafka metrics templates used in document generation should maintain order of tags
> ---------------------------------------------------------------------------------
>
>                 Key: KAFKA-5987
>                 URL: https://issues.apache.org/jira/browse/KAFKA-5987
>             Project: Kafka
>          Issue Type: Improvement
>          Components: clients
>    Affects Versions: 1.0.0
>            Reporter: Randall Hauch
>            Assignee: Randall Hauch
>            Priority: Blocker
>             Fix For: 1.1.0, 1.0.1, 1.2.0
>
>
> KAFKA-5191 recently added a new {{MetricNameTemplate}} that is used to create the {{MetricName}} objects in the producer and consumer, as we as in the newly-added generation of metric documentation. The {{MetricNameTemplate}} and the {{Metric.toHtmlTable}} do not maintain the order of the tags, which means the resulting HTML documentation will order the table of MBean attributes based upon the lexicographical ordering of the MBeans, each of which uses the lexicographical ordering of its tags. This can result in the following order:
> {noformat}
> kafka.connect:type=sink-task-metrics,connector={connector},partition={partition},task={task},topic={topic}
> kafka.connect:type=sink-task-metrics,connector={connector},task={task}
> {noformat}
> However, if the MBeans maintained the order of the tags then the documentation would use the following order:
> {noformat}
> kafka.connect:type=sink-task-metrics,connector={connector},task={task}
> kafka.connect:type=sink-task-metrics,connector={connector},task={task},topic={topic},partition={partition}
> {noformat}
> This would be more readable, and the code that is creating the templates would have control over the order of the tags. 
> To maintain order, {{MetricNameTemplate}} should used a {{LinkedHashSet}} for the tags, and the {{Metrics.toHtmlTable}} method should also use a {{LinkedHashMap}} when building up the tags used in the MBean name.
> Note that JMX MBean names use {{ObjectName}} that does not maintain order, so this change should have no impact on JMX MBean names.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)