You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@zookeeper.apache.org by GitBox <gi...@apache.org> on 2021/03/17 00:33:40 UTC

[GitHub] [zookeeper] li4wang opened a new pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

li4wang opened a new pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644


   Motivation
   
   In 3.7, Quota limit can be enforced and the quota related stats are captured.  From the "listquota" CLI command, we can the quota limit and usage info. This is an addition to that so we can collect the quota metrics per top namespace and expose them to the Prometheus for monitor and alert purpose. 
   
   Summary of Changes
   
   - added 5 quota metrics 
   - added GaugeSet metric type to group gauge metrics by key
   - changed PrometheusMetricsProvider to to support the GaugeSet
   - changed ZookeeperServer and DataTree to collect/publish quota metrics
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r598222993



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }
+    }
+
+    static boolean shouldCollect(final String path, final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        return path.endsWith(LIMIT_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT == type)
+                || path.endsWith(STATS_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE == type);
+    }
+
+    static void collectQuotaLimitOrUsage(final String path,
+                                         final DataNode node,
+                                         final Map<String, Number> metricsMap,
+                                         final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final String namespace = PathUtils.getTopNamespace(Quotas.trimQuotaPath(path));
+        if (namespace == null) {
+            return;
+        }
+        if (node.getData() == null) {
+            return;
+        }
+        final StatsTrack statsTrack = new StatsTrack(node.getData());

Review comment:
       yes, you are right. It's better this way. I will change it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r598217478



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java
##########
@@ -259,6 +260,8 @@ private ServerMetrics(MetricsProvider metricsProvider) {
         WATCH_BYTES = metricsContext.getCounter("watch_bytes");
 
         JVM_PAUSE_TIME = metricsContext.getSummary("jvm_pause_time_ms", DetailLevel.ADVANCED);
+
+        QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = metricsContext.getSummarySet(QuotaMetricsUtils.QUOTA_EXCEEDED_ERROR_PER_NAMESPACE, DetailLevel.BASIC);

Review comment:
       SummarySet is used because it provides count (i.e. cnt) and the grouping capability.  Yes, you are right, in this case, since the count is always added by 1, the cnt and sum are the same with PrometheusMetricsProvider and the min/max/avg with DefaultMetricsProvider are always 1 in this case.  User can just use the cnt and ignore the others.
   
   Is there any reason that CounterSet is not supported? Are you think we should add a CounterSet in this PR?
   
   I added GaugeSet for other quota metrics, as SummarySet can not be used for them.
   
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r632273912



##########
File path: zookeeper-server/src/test/java/org/apache/zookeeper/test/EnforceQuotaTest.java
##########
@@ -54,14 +56,16 @@ public void tearDown() throws Exception {
 
     @Test
     public void testSetQuotaDisableWhenExceedBytesHardQuota() throws Exception {
-        final String path = "/c1";
+        final String namespace = UUID.randomUUID().toString();

Review comment:
       Yes, the reason is that we added validation on the per-namespace quota exceeded count in the test cases. The namespace need to be dynamic/random to make sure the counter is not interfered by other test cases.
   
   `ZooKeeperQuotaTest.validateNoQuotaExceededMetrics(namespace);`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] eolivelli commented on pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
eolivelli commented on pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#issuecomment-858328402


   The failure seems related to this patch.
   I would like not to add a new flaky test


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] ztzg commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
ztzg commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r605547937



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }

Review comment:
       Thank you for your updated PR. Still on my list of things to review! But regarding this point:
   
   > I thought about it as I saw the synchronized block is used when DataNode.getChildren() is called in some other places, but I don\'t think synchronized is needed here because the getChildren() API is already synchronized on the DataNode instance object and it returns an UnmodifiableSet.
   
   Yes, `getChildren()` is synchronized. And the `EMPTY_SET` case is totally fine as that set is indeed immutable.
   
   The problem is with `Collections.unmodifiableSet()`. This function creates a wrapper preventing **external** modifications to the set, but still delegates all of the allowed operations to the original object.
   
   It **does not** provide a way of make the *inner* collection immutable nor synchronized (it has no way of doing that) and it does **not** lock the object when accessing it (a design decision).
   
   As you mention:
   
   > With UnmodifiableSet, the underneath \"children\" set can still be modified by other ZK operations while I am iterating on it, but I think this should be okay, as we are just reading not mutating data, so there is no race condition on write.
   
   Yes, `DataTree` may very well continue modifying the (locked) object via `addChildren()`, and the code above does not honor that lock. `DataNode` is backed a `HashSet`, whose documentation clearly says:
   
   > **Note that this implementation is not synchronized.** If multiple threads access a hash set concurrently, and at least one of the threads modifies the set, it *must* be synchronized externally.
   
   <https://docs.oracle.com/javase/8/docs/api/java/util/HashSet.html>
   
   So I still believe that code breaks its contract with the JDK.
   
   > Metrics data is a snapshot. While iterating on it, nodes could be added or removed. If nodes are deleted, it is okay as both collectQuotaLimitOrUsage() API and getQuotaLimitOrUsage() API check whether a node exists or not before processing it. If nodes are added, it is okay too because the updates will be reflected when the MetricsProvider collects data next time.
   
   Right: I agree that there is currently no way of atomically summing over the tree, and that it is not necessary from the metrics collection point of view.
   
   But this is not where my objection lies. The JDK threading model and collection contracts are not honored, which will result---at best!---in seemingly random `ConcurrentModificationException` log messages.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r598224282



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }
+    }
+
+    static boolean shouldCollect(final String path, final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        return path.endsWith(LIMIT_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT == type)
+                || path.endsWith(STATS_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE == type);
+    }
+
+    static void collectQuotaLimitOrUsage(final String path,
+                                         final DataNode node,
+                                         final Map<String, Number> metricsMap,
+                                         final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final String namespace = PathUtils.getTopNamespace(Quotas.trimQuotaPath(path));
+        if (namespace == null) {
+            return;
+        }
+        if (node.getData() == null) {
+            return;
+        }
+        final StatsTrack statsTrack = new StatsTrack(node.getData());
+        switch (type) {
+            case QUOTA_COUNT_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getCountHardLimit(), statsTrack.getCount()));
+                break;
+            case QUOTA_BYTES_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getByteHardLimit(), statsTrack.getBytes()));
+                break;
+            case QUOTA_COUNT_USAGE:
+                metricsMap.put(namespace, statsTrack.getCount());
+                break;
+            case QUOTA_BYTES_USAGE:
+                metricsMap.put(namespace, statsTrack.getBytes());
+                break;
+            default:
+        }
+    }
+
+    // hard limit takes precedence if specified
+    static long getQuotaLimit(final long hardLimit, final long limit) {
+        return hardLimit > -1 ? hardLimit : limit;
+    }

Review comment:
       Hmm, not sure what you meant. This is the limit used for both soft quota errors and hard quota errors.  If hardLimit is specified, then the client will get QuotaExceededException, otherwise it just log as WARN.  
   
   We are not preventing people from monitoring  the soft quota errors. In fact, we publish the "proper" metric data so they can set up the alerts based on different usage level.  
   
   The limit is published in this way because
   
   1. It matches up with what's been used in the checkQuota() logic, so user doesn't need to know how the soft limit and hard limit are used internally.  
   2. This is the limit used by the server to check quota and this is what they need to monitor and alert on. It can be soft limit or hard limit.
   3. In the future if the quota check logic is changed in term of how the soft limit and hard limit are used, we just need to change the server side code. User's monitoring and alert rule do not need to be changed.
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r598217478



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java
##########
@@ -259,6 +260,8 @@ private ServerMetrics(MetricsProvider metricsProvider) {
         WATCH_BYTES = metricsContext.getCounter("watch_bytes");
 
         JVM_PAUSE_TIME = metricsContext.getSummary("jvm_pause_time_ms", DetailLevel.ADVANCED);
+
+        QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = metricsContext.getSummarySet(QuotaMetricsUtils.QUOTA_EXCEEDED_ERROR_PER_NAMESPACE, DetailLevel.BASIC);

Review comment:
       SummarySet is used because it provides count (i.e. cnt) and the grouping capability.  Yes, you are right, in this case, since the count is always added by 1, the cnt and sum are the same with PrometheusMetricsProvider and the min/max/avg with DefaultMetricsProvider are always 1 in this case.  User can just use the cnt and ignore the others.
   
   I added GaugeSet for other quota metrics, as SummarySet really can not be used for them.
   
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] ztzg commented on pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
ztzg commented on pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#issuecomment-888963304


   The only CI failure is in the C client, which this changeset doesn't touch:
   
   ``` example
   [exec] /home/runner/work/zookeeper/zookeeper/zookeeper-client/zookeeper-client-c/tests/TestOperations.cc:296: Assertion: equality assertion failed [Expected: 1, Actual  : 0]
   [exec] Failures !!!
   [exec] Run: 48   Failure total: 1   Failures: 1   Errors: 0
   [exec] FAIL: zktest-st
   ```
   
   Moreover, the failed assertion is related to a ping count, and running the C test suite on the same commit succeeded locally—so I'm chalking it up to flakiness.  (I really need to find some time & energy I can spend on improving that test suite…)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@zookeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] ztzg commented on pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
ztzg commented on pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#issuecomment-888966677


   Okay, @li4wang—this is now in `master`.  Thank you for your contribution and patience!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@zookeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r598217478



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java
##########
@@ -259,6 +260,8 @@ private ServerMetrics(MetricsProvider metricsProvider) {
         WATCH_BYTES = metricsContext.getCounter("watch_bytes");
 
         JVM_PAUSE_TIME = metricsContext.getSummary("jvm_pause_time_ms", DetailLevel.ADVANCED);
+
+        QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = metricsContext.getSummarySet(QuotaMetricsUtils.QUOTA_EXCEEDED_ERROR_PER_NAMESPACE, DetailLevel.BASIC);

Review comment:
       SummarySet is used because it provides count (i.e. cnt) and the grouping capability.  Yes, you are right, in this case, since the count is always added by 1, the cnt and sum are the same with PrometheusMetricsProvider  and the min/max/avg with DefaultMetricsProvider are always 1 in this case. 
   
   Is there any reason that CounterSet is not supported?
   
   I added GaugeSet for other quota metrics, as SummarySet can not be used.
   
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r598222690



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }

Review comment:
       I thought about it as I saw the synchronized block is used when  DataNode.getChildren() is called in some other places, but I don't think synchronized is necessary here because the getChildren() API is already synchronized on the DataNode instance object and it returns an UnmodifiableSet. 
   
   `public synchronized Set<String> getChildren() {
           if (children == null) {
               return EMPTY_SET;
           }
   
           return Collections.unmodifiableSet(children);
       }
   `
   
   
   
     




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r598224282



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }
+    }
+
+    static boolean shouldCollect(final String path, final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        return path.endsWith(LIMIT_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT == type)
+                || path.endsWith(STATS_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE == type);
+    }
+
+    static void collectQuotaLimitOrUsage(final String path,
+                                         final DataNode node,
+                                         final Map<String, Number> metricsMap,
+                                         final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final String namespace = PathUtils.getTopNamespace(Quotas.trimQuotaPath(path));
+        if (namespace == null) {
+            return;
+        }
+        if (node.getData() == null) {
+            return;
+        }
+        final StatsTrack statsTrack = new StatsTrack(node.getData());
+        switch (type) {
+            case QUOTA_COUNT_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getCountHardLimit(), statsTrack.getCount()));
+                break;
+            case QUOTA_BYTES_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getByteHardLimit(), statsTrack.getBytes()));
+                break;
+            case QUOTA_COUNT_USAGE:
+                metricsMap.put(namespace, statsTrack.getCount());
+                break;
+            case QUOTA_BYTES_USAGE:
+                metricsMap.put(namespace, statsTrack.getBytes());
+                break;
+            default:
+        }
+    }
+
+    // hard limit takes precedence if specified
+    static long getQuotaLimit(final long hardLimit, final long limit) {
+        return hardLimit > -1 ? hardLimit : limit;
+    }

Review comment:
       Hmm, not sure what you meant. This is the limit used for both soft quota errors and hard quota errors.  If hardLimit is specified, then the client will get QuotaExceededException, otherwise it just log as WARN. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] ztzg commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
ztzg commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r605548947



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }
+    }
+
+    static boolean shouldCollect(final String path, final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        return path.endsWith(LIMIT_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT == type)
+                || path.endsWith(STATS_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE == type);
+    }
+
+    static void collectQuotaLimitOrUsage(final String path,
+                                         final DataNode node,
+                                         final Map<String, Number> metricsMap,
+                                         final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final String namespace = PathUtils.getTopNamespace(Quotas.trimQuotaPath(path));
+        if (namespace == null) {
+            return;
+        }
+        if (node.getData() == null) {
+            return;
+        }
+        final StatsTrack statsTrack = new StatsTrack(node.getData());
+        switch (type) {
+            case QUOTA_COUNT_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getCountHardLimit(), statsTrack.getCount()));
+                break;
+            case QUOTA_BYTES_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getByteHardLimit(), statsTrack.getBytes()));
+                break;
+            case QUOTA_COUNT_USAGE:
+                metricsMap.put(namespace, statsTrack.getCount());
+                break;
+            case QUOTA_BYTES_USAGE:
+                metricsMap.put(namespace, statsTrack.getBytes());
+                break;
+            default:
+        }
+    }
+
+    // hard limit takes precedence if specified
+    static long getQuotaLimit(final long hardLimit, final long limit) {
+        return hardLimit > -1 ? hardLimit : limit;
+    }

Review comment:
       Sorry---I was indeed too quick (and not very clear to boot).
   
   > We are not preventing people from monitoring the soft quota errors. In fact, we publish the \"proper\" metric data so they can set up the alerts based on different usage level.
   >
   > The limit is published in this way because
   >
   > -   It matches up with what\'s been used in the checkQuota() logic, so user doesn\'t need to know how the soft limit and hard limit are used internally.
   > -   This is the limit used by the server to check quota and this is what they need to monitor and alert on. It can be soft limit or hard limit.
   > -   In the future if the quota check logic is changed in term of how the soft limit and hard limit are used, we just need to change the server side code. User\'s monitoring and alert rule do not need to be changed.
   
   You are right.
   
   I had in mind that soft/hard quotas had the same semantics as, e.g., disk quotas under Unix---but that is not what is implemented as of today.
   
   (That being said: it is something I would like to see improved at some point—I believe the current semantics are accidental, and perhaps not as useful as they could be—but my wishes do no make reality.
   
   IMO, server administrators should be informed when a node is above the soft quota, even if no errors are generated. Similarly, clients should have a way of monitoring the \"warnings\" which are being issued against them.
   
   So it seems that published metrics will have to be updated when specifying both a soft and hard limit on a node becomes useful—e.g., by adding `quota_count_soft_limit_per_namespace` and `quota_bytes_soft_limit_per_namespace`—so that dashboards can show watermarks as actually configured in the quota subsystem.)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r598224282



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }
+    }
+
+    static boolean shouldCollect(final String path, final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        return path.endsWith(LIMIT_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT == type)
+                || path.endsWith(STATS_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE == type);
+    }
+
+    static void collectQuotaLimitOrUsage(final String path,
+                                         final DataNode node,
+                                         final Map<String, Number> metricsMap,
+                                         final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final String namespace = PathUtils.getTopNamespace(Quotas.trimQuotaPath(path));
+        if (namespace == null) {
+            return;
+        }
+        if (node.getData() == null) {
+            return;
+        }
+        final StatsTrack statsTrack = new StatsTrack(node.getData());
+        switch (type) {
+            case QUOTA_COUNT_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getCountHardLimit(), statsTrack.getCount()));
+                break;
+            case QUOTA_BYTES_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getByteHardLimit(), statsTrack.getBytes()));
+                break;
+            case QUOTA_COUNT_USAGE:
+                metricsMap.put(namespace, statsTrack.getCount());
+                break;
+            case QUOTA_BYTES_USAGE:
+                metricsMap.put(namespace, statsTrack.getBytes());
+                break;
+            default:
+        }
+    }
+
+    // hard limit takes precedence if specified
+    static long getQuotaLimit(final long hardLimit, final long limit) {
+        return hardLimit > -1 ? hardLimit : limit;
+    }

Review comment:
       Hmm, not sure what you meant. This is the limit used for both soft quota errors and hard quota errors.  If hardLimit is specified, then the client will get QuotaExceededException, otherwise it just log as WARN.  
   
   We are not preventing people from monitoring  the soft quota errors. In fact, we publish the "proper" metric data so they can set up the alerts based on different usage level.  
   
   The limit is published in this way because
   
   1. It matches up with what's been used in the checkQuota() logic, so user doesn't need to know how the soft limit and hard limit are used internally.  
   2. This is the limit used by the server to check quota and this is what they need to monitor and alert on. It can be soft limit or hard limit.
   3. In the future if the quota check logic is changed in term of how the soft limit and hard limit are used, we just change the server side code. User's monitoring and alert rule doesn't need to be changed.
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang edited a comment on pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang edited a comment on pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#issuecomment-858346284


   > Could you have a look? Do you know what is going on?
   
   Thanks a lot for reviewing it, @ztzg 
   
   The test case has been fixed. 
   
   The expected number of metrics should be 1 instead of 5 after the QUOTA_EXCEEDED_ERROR_PER_NAMESPACE is changed from SummarySet to CounterSet.
   
   All the test cases passed when I ran on my local. Can you please take a quick look at it? Thanks.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] ztzg commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
ztzg commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r632066681



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java
##########
@@ -259,6 +260,8 @@ private ServerMetrics(MetricsProvider metricsProvider) {
         WATCH_BYTES = metricsContext.getCounter("watch_bytes");
 
         JVM_PAUSE_TIME = metricsContext.getSummary("jvm_pause_time_ms", DetailLevel.ADVANCED);
+
+        QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = metricsContext.getSummarySet(QuotaMetricsUtils.QUOTA_EXCEEDED_ERROR_PER_NAMESPACE, DetailLevel.BASIC);

Review comment:
       Okay!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r605845846



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }
+    }
+
+    static boolean shouldCollect(final String path, final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        return path.endsWith(LIMIT_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT == type)
+                || path.endsWith(STATS_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE == type);
+    }
+
+    static void collectQuotaLimitOrUsage(final String path,
+                                         final DataNode node,
+                                         final Map<String, Number> metricsMap,
+                                         final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final String namespace = PathUtils.getTopNamespace(Quotas.trimQuotaPath(path));
+        if (namespace == null) {
+            return;
+        }
+        if (node.getData() == null) {
+            return;
+        }
+        final StatsTrack statsTrack = new StatsTrack(node.getData());
+        switch (type) {
+            case QUOTA_COUNT_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getCountHardLimit(), statsTrack.getCount()));
+                break;
+            case QUOTA_BYTES_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getByteHardLimit(), statsTrack.getBytes()));
+                break;
+            case QUOTA_COUNT_USAGE:
+                metricsMap.put(namespace, statsTrack.getCount());
+                break;
+            case QUOTA_BYTES_USAGE:
+                metricsMap.put(namespace, statsTrack.getBytes());
+                break;
+            default:
+        }
+    }
+
+    // hard limit takes precedence if specified
+    static long getQuotaLimit(final long hardLimit, final long limit) {
+        return hardLimit > -1 ? hardLimit : limit;
+    }

Review comment:
       Yes, if we change the semantics of soft and hard quotas in the future, we will update the the published metrics along with the changes.  Changing the semantics is a fundamental thing, so it makes sense to make all the changes that are impacted at that time.
   
   There are two concerns for publishing both soft and hard limit at this point.
   
   1. It doesn't match with the current semantics of soft and hard quotas and make it harder for user t o use the metrics. 
   2. Given the constraint we have with the current interface of metrics framework, it means the we need to iterate two more times on the /zookeeper/quota tree. IMO, we pay something up-to-front that doesn't give us too much benefits.
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] muse-dev[bot] commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
muse-dev[bot] commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r605980043



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final List<String> children;
+        synchronized (node) {
+            children = new ArrayList<>(node.getChildren());
+        }
+        if (children.size() == 0) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);

Review comment:
       *THREAD_SAFETY_VIOLATION:*  Unprotected write. Non-private method `QuotaMetricsUtils.getQuotaLimitOrUsage(...)` indirectly mutates container `metricsMap` via call to `Map.put(...)` outside of synchronization.
    Reporting because this access may occur on a background thread.
   (at-me [in a reply](https://docs.muse.dev/docs/talk-to-muse/) with `help` or `ignore`)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang edited a comment on pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang edited a comment on pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#issuecomment-858346284


   > Could you have a look? Do you know what is going on?
   
   Thanks a lot for reviewing it, @ztzg 
   
   The test case has been fixed. 
   
   The expected number of metrics should be 1 instead of 5 after the QUOTA_EXCEEDED_ERROR_PER_NAMESPACE is changed from SummarySet to CounterSet.
   
   Can you please take a quick look at it? Thanks.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] ztzg edited a comment on pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
ztzg edited a comment on pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#issuecomment-871197634


   @eolivelli, @hanm: Agree that the patch is a bit difficult to digest.  It would be great to have another set of eyes, but it takes a substantial amount of effort.  On the other hand, it would be better to merge it and deal with the potential consequences of something we would have missed than to have the patch linger for too long.  I would merge it into `master`, but I understand that @li4wang would also like to see it in `branch-3.7`, which is a tad more risky.  What do you think?  (I would be able to take care of the merge later today or tomorrow.)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@zookeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r605877838



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }

Review comment:
       Good point from the JDK threading model point of view. Let me add code to honor it to avoid the ConcurrentModificationException.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r630457777



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }
+    }
+
+    static boolean shouldCollect(final String path, final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        return path.endsWith(LIMIT_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT == type)
+                || path.endsWith(STATS_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE == type);
+    }
+
+    static void collectQuotaLimitOrUsage(final String path,
+                                         final DataNode node,
+                                         final Map<String, Number> metricsMap,
+                                         final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final String namespace = PathUtils.getTopNamespace(Quotas.trimQuotaPath(path));
+        if (namespace == null) {
+            return;
+        }
+        if (node.getData() == null) {
+            return;
+        }
+        final StatsTrack statsTrack = new StatsTrack(node.getData());
+        switch (type) {
+            case QUOTA_COUNT_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getCountHardLimit(), statsTrack.getCount()));
+                break;
+            case QUOTA_BYTES_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getByteHardLimit(), statsTrack.getBytes()));
+                break;
+            case QUOTA_COUNT_USAGE:
+                metricsMap.put(namespace, statsTrack.getCount());
+                break;
+            case QUOTA_BYTES_USAGE:
+                metricsMap.put(namespace, statsTrack.getBytes());
+                break;

Review comment:
       changed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] ztzg commented on pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
ztzg commented on pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#issuecomment-858413588


   Hi @li4wang,
   
   > > Could you have a look? Do you know what is going on?
   > 
   > Thanks a lot for reviewing it, @ztzg
   > 
   > The test case has been fixed.
   > 
   > The expected number of metrics should be 1 instead of 5 after the QUOTA_EXCEEDED_ERROR_PER_NAMESPACE is changed from SummarySet to CounterSet.
   
   Right; makes sense.  The update looks good.
    
   > All the test cases passed when I ran on my local. Can you please take a quick look at it? Thanks.
   
   LGTM.  But I will let CI finish.
   
   @eolivelli wrote:
   > The failure seems related to this patch.
   > I would like not to add a new flaky test
   
   That was a failure was deterministic; the test had not been updated after the introduction of the `CounterSet`.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r598217478



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java
##########
@@ -259,6 +260,8 @@ private ServerMetrics(MetricsProvider metricsProvider) {
         WATCH_BYTES = metricsContext.getCounter("watch_bytes");
 
         JVM_PAUSE_TIME = metricsContext.getSummary("jvm_pause_time_ms", DetailLevel.ADVANCED);
+
+        QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = metricsContext.getSummarySet(QuotaMetricsUtils.QUOTA_EXCEEDED_ERROR_PER_NAMESPACE, DetailLevel.BASIC);

Review comment:
       SummarySet is used because it provides count (i.e. cnt) and the grouping capability.  Yes, you are right, in this case, since the count is always added by 1, the cnt and sum are the same with PrometheusMetricsProvider and the min/max/avg with DefaultMetricsProvider are always 1 in this case.  User can just use the cnt and ignore the others.
   
   Is there any reason that CounterSet is not supported? Are you think we should add a CounterSet in this PR?
   
   I added GaugeSet for other quota metrics, as SummarySet really can not be used for them.
   
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r632274802



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final List<String> children;
+        synchronized (node) {
+            children = new ArrayList<>(node.getChildren());
+        }
+        if (children.size() == 0) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);

Review comment:
       Good catch. It should have `private`. I probably planned to add unit test case for this API at that time nd changed the scope to 'package'.  
   
   Added `private` back.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r630456856



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java
##########
@@ -259,6 +260,8 @@ private ServerMetrics(MetricsProvider metricsProvider) {
         WATCH_BYTES = metricsContext.getCounter("watch_bytes");
 
         JVM_PAUSE_TIME = metricsContext.getSummary("jvm_pause_time_ms", DetailLevel.ADVANCED);
+
+        QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = metricsContext.getSummarySet(QuotaMetricsUtils.QUOTA_EXCEEDED_ERROR_PER_NAMESPACE, DetailLevel.BASIC);

Review comment:
       Added `CounterSet` and used it for `QUOTA_EXCEEDED_ERROR_PER_NAMESPACE` 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] ztzg commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
ztzg commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r632067144



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }
+    }
+
+    static boolean shouldCollect(final String path, final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        return path.endsWith(LIMIT_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT == type)
+                || path.endsWith(STATS_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE == type);
+    }
+
+    static void collectQuotaLimitOrUsage(final String path,
+                                         final DataNode node,
+                                         final Map<String, Number> metricsMap,
+                                         final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final String namespace = PathUtils.getTopNamespace(Quotas.trimQuotaPath(path));
+        if (namespace == null) {
+            return;
+        }
+        if (node.getData() == null) {
+            return;
+        }
+        final StatsTrack statsTrack = new StatsTrack(node.getData());
+        switch (type) {
+            case QUOTA_COUNT_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getCountHardLimit(), statsTrack.getCount()));
+                break;
+            case QUOTA_BYTES_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getByteHardLimit(), statsTrack.getBytes()));
+                break;
+            case QUOTA_COUNT_USAGE:
+                metricsMap.put(namespace, statsTrack.getCount());
+                break;
+            case QUOTA_BYTES_USAGE:
+                metricsMap.put(namespace, statsTrack.getBytes());
+                break;

Review comment:
       Okay!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] ztzg commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
ztzg commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r598156394



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }

Review comment:
       Hmm… I understand that you are constrained by the current interface, but it also means we iterate four times on the `/zookeeper/quota` tree.  It also "forces" the implementation to report unset (count or byte) quotas as `-1` instead of just omitting them, as it cannot correlate limits and values without doing an even larger amount of duplicate work.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r598223377



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }
+    }
+
+    static boolean shouldCollect(final String path, final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        return path.endsWith(LIMIT_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT == type)
+                || path.endsWith(STATS_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE == type);
+    }
+
+    static void collectQuotaLimitOrUsage(final String path,
+                                         final DataNode node,
+                                         final Map<String, Number> metricsMap,
+                                         final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final String namespace = PathUtils.getTopNamespace(Quotas.trimQuotaPath(path));
+        if (namespace == null) {
+            return;
+        }
+        if (node.getData() == null) {
+            return;
+        }
+        final StatsTrack statsTrack = new StatsTrack(node.getData());
+        switch (type) {
+            case QUOTA_COUNT_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getCountHardLimit(), statsTrack.getCount()));
+                break;
+            case QUOTA_BYTES_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getByteHardLimit(), statsTrack.getBytes()));
+                break;
+            case QUOTA_COUNT_USAGE:
+                metricsMap.put(namespace, statsTrack.getCount());
+                break;
+            case QUOTA_BYTES_USAGE:
+                metricsMap.put(namespace, statsTrack.getBytes());
+                break;

Review comment:
       Yes, good suggestion. In our case, we only set the quota on the top namespace, so no need to sum. But for general use case, we want to do sum. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] ztzg commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
ztzg commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r605548947



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }
+    }
+
+    static boolean shouldCollect(final String path, final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        return path.endsWith(LIMIT_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT == type)
+                || path.endsWith(STATS_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE == type);
+    }
+
+    static void collectQuotaLimitOrUsage(final String path,
+                                         final DataNode node,
+                                         final Map<String, Number> metricsMap,
+                                         final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final String namespace = PathUtils.getTopNamespace(Quotas.trimQuotaPath(path));
+        if (namespace == null) {
+            return;
+        }
+        if (node.getData() == null) {
+            return;
+        }
+        final StatsTrack statsTrack = new StatsTrack(node.getData());
+        switch (type) {
+            case QUOTA_COUNT_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getCountHardLimit(), statsTrack.getCount()));
+                break;
+            case QUOTA_BYTES_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getByteHardLimit(), statsTrack.getBytes()));
+                break;
+            case QUOTA_COUNT_USAGE:
+                metricsMap.put(namespace, statsTrack.getCount());
+                break;
+            case QUOTA_BYTES_USAGE:
+                metricsMap.put(namespace, statsTrack.getBytes());
+                break;
+            default:
+        }
+    }
+
+    // hard limit takes precedence if specified
+    static long getQuotaLimit(final long hardLimit, final long limit) {
+        return hardLimit > -1 ? hardLimit : limit;
+    }

Review comment:
       Sorry---I was indeed too quick (and not very clear to boot).
   
   > We are not preventing people from monitoring the soft quota errors. In fact, we publish the \"proper\" metric data so they can set up the alerts based on different usage level.
   >
   > The limit is published in this way because
   >
   > -   It matches up with what\'s been used in the checkQuota() logic, so user doesn\'t need to know how the soft limit and hard limit are used internally.
   > -   This is the limit used by the server to check quota and this is what they need to monitor and alert on. It can be soft limit or hard limit.
   > -   In the future if the quota check logic is changed in term of how the soft limit and hard limit are used, we just need to change the server side code. User\'s monitoring and alert rule do not need to be changed.
   
   You are right.
   
   I had in mind that soft/hard quotas had the same semantics as, e.g., disk quotas under Unix---but that is not what is implemented as of today.
   
   (That being said: it is something I would like to see improved at some point—I believe the current semantics are accidental, and perhaps not as useful as they could be—but my wishes do no make reality.
   
   IMO, server administrators should be informed when a node is above the soft quota, even if no errors are generated. Similarly, clients should have a way of monitoring the \"warnings\" which are being issued against them.
   
   So it seems that published metrics will have to be updated when specifying both a soft and hard limit on a node becomes useful—e.g., by adding `quota_count_soft_limit_per_namespace` and `quota_bytes_soft_limit_per_namespace`—so that dashboards can show both watermarks as actually configured in the quota subsystem.)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r598226025



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }

Review comment:
       Yes, I tried to avoid iterating four times on the /zookeeper/quota tree, but constrained by the current interface. The callback is for each individual gauge. 
   
   The implementation is not the most optimal due to the above constraint, but I thought it's probably okay considering 
   
   1. All the data stored in the memory and the quota tree should not too big in general (it's just the top namespace level in our use case) 
   
   2) What it impacts is the latency of the metrics endpoint call. It doesn't impact the ZK client operations.
   
   What do you think?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang edited a comment on pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang edited a comment on pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#issuecomment-888479191


   The conflicts have been resolved.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@zookeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] ztzg commented on pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
ztzg commented on pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#issuecomment-888286034


   > @eolivelli, @hanm: [...] I would merge it into `master`, but I understand that @li4wang would also like to see it in `branch-3.7`, which is a tad more risky. What do you think? (I would be able to take care of the merge later today or tomorrow.)
   
   Unless somebody hollers, I'm willing to merge this to both branches without waiting any further.  The patch cherry-picks cleanly into `branch-3.7`, but conflicts on `master`.  @li4wang: would you mind changing the target branch for this PR to `branch-3.7`, and submitting a refreshed one for `master`?
   
   Cheers, -D


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@zookeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] eolivelli commented on pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
eolivelli commented on pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#issuecomment-888304345


   This is an API change (about the Metrics Provider API) and in my opinion it is better to not pick it on a released branch, in which the API must be stable.
   
   So I would keep this only on master branch


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@zookeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r598224282



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }
+    }
+
+    static boolean shouldCollect(final String path, final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        return path.endsWith(LIMIT_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT == type)
+                || path.endsWith(STATS_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE == type);
+    }
+
+    static void collectQuotaLimitOrUsage(final String path,
+                                         final DataNode node,
+                                         final Map<String, Number> metricsMap,
+                                         final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final String namespace = PathUtils.getTopNamespace(Quotas.trimQuotaPath(path));
+        if (namespace == null) {
+            return;
+        }
+        if (node.getData() == null) {
+            return;
+        }
+        final StatsTrack statsTrack = new StatsTrack(node.getData());
+        switch (type) {
+            case QUOTA_COUNT_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getCountHardLimit(), statsTrack.getCount()));
+                break;
+            case QUOTA_BYTES_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getByteHardLimit(), statsTrack.getBytes()));
+                break;
+            case QUOTA_COUNT_USAGE:
+                metricsMap.put(namespace, statsTrack.getCount());
+                break;
+            case QUOTA_BYTES_USAGE:
+                metricsMap.put(namespace, statsTrack.getBytes());
+                break;
+            default:
+        }
+    }
+
+    // hard limit takes precedence if specified
+    static long getQuotaLimit(final long hardLimit, final long limit) {
+        return hardLimit > -1 ? hardLimit : limit;
+    }

Review comment:
       Hmm, not sure what you meant. This is the limit used for both soft quota errors and hard quota errors.  If hardLimit is specified, then the client will get QuotaExceededException, otherwise it just log as WARN.  We are not preventing people from monitoring  the soft quota errors. In fact, we publish the "proper" metric data so they can set up the alerts based on different usage level.  The limit we publish matches up with what's been used in the checkQuota() logic, so user doesn't need to know how the soft limit and hard limit are used internally. This is the limit is used by the server to check quota and this is what they need to monitor and alert on.
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r598222690



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }

Review comment:
       I thought about it as I saw the synchronized block is used when  DataNode.getChildren() is called in some other places, but I don't think synchronized is needed here because the getChildren() API is already synchronized on the DataNode instance object and it returns an UnmodifiableSet. 
   
   `public synchronized Set<String> getChildren() {
           if (children == null) {
               return EMPTY_SET;
           }
   
           return Collections.unmodifiableSet(children);
       }
   `
   With UnmodifiableSet, the underneath "children" set can still be modified by other ZK operations while I am iterating on it, but I think this should be okay, as we are just reading not mutating data, so there is no race condition on write.  
   
   Metrics data is a snapshot. While iterating on it, nodes could be added or removed. If nodes are deleted, it is okay as both collectQuotaLimitOrUsage() API and getQuotaLimitOrUsage() API check whether a node exists or not before processing it.  If nodes are added, it is okay too because the updates will be reflected when the MetricsProvider collects data next time.
   
   What do you think?
   
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r630457279



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }

Review comment:
       changed.

##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }
+    }
+
+    static boolean shouldCollect(final String path, final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        return path.endsWith(LIMIT_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT == type)
+                || path.endsWith(STATS_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE == type);
+    }
+
+    static void collectQuotaLimitOrUsage(final String path,
+                                         final DataNode node,
+                                         final Map<String, Number> metricsMap,
+                                         final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final String namespace = PathUtils.getTopNamespace(Quotas.trimQuotaPath(path));
+        if (namespace == null) {
+            return;
+        }
+        if (node.getData() == null) {
+            return;
+        }
+        final StatsTrack statsTrack = new StatsTrack(node.getData());

Review comment:
       changed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#issuecomment-858346284


   > Could you have a look? Do you know what is going on?
   
   The test case has been fixed. 
   
   The expected number of metrics should be 1 instead of 5 after the QUOTA_EXCEEDED_ERROR_PER_NAMESPACE is changed from SummarySet to CounterSet.
   
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r598217478



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java
##########
@@ -259,6 +260,8 @@ private ServerMetrics(MetricsProvider metricsProvider) {
         WATCH_BYTES = metricsContext.getCounter("watch_bytes");
 
         JVM_PAUSE_TIME = metricsContext.getSummary("jvm_pause_time_ms", DetailLevel.ADVANCED);
+
+        QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = metricsContext.getSummarySet(QuotaMetricsUtils.QUOTA_EXCEEDED_ERROR_PER_NAMESPACE, DetailLevel.BASIC);

Review comment:
       Yes,  I used SummarySet because it provides count (i.e. cnt) and also the grouping capability.  
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#issuecomment-889249252


   @ztzg Thank you so much for your inputs and time. This is awesome!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@zookeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] ztzg commented on pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
ztzg commented on pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#issuecomment-871197634


   @eolivelli, @hanm: Agree that the patch is a bit difficult to digest.  It would be great to have another set of eyes, but it takes a substantial amount of effort.  On the other hand, it would be better to merge it and deal with the potential consequences of something we would have missed than to have the patch linger for too long.  I would merge it into `master`, but I understand that @li4wang would also like it in `branch-3.7`, which is a tad more risky.  What do you think?  (I would be able to take care of the merge later today or tomorrow.)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@zookeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] ztzg commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
ztzg commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r632069739



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final List<String> children;
+        synchronized (node) {
+            children = new ArrayList<>(node.getChildren());
+        }
+        if (children.size() == 0) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);

Review comment:
       Agree.  Nit: is there a reason the 4-argument overload is not `private`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] ztzg commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
ztzg commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r605907995



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }
+    }
+
+    static boolean shouldCollect(final String path, final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        return path.endsWith(LIMIT_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT == type)
+                || path.endsWith(STATS_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE == type);
+    }
+
+    static void collectQuotaLimitOrUsage(final String path,
+                                         final DataNode node,
+                                         final Map<String, Number> metricsMap,
+                                         final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final String namespace = PathUtils.getTopNamespace(Quotas.trimQuotaPath(path));
+        if (namespace == null) {
+            return;
+        }
+        if (node.getData() == null) {
+            return;
+        }
+        final StatsTrack statsTrack = new StatsTrack(node.getData());
+        switch (type) {
+            case QUOTA_COUNT_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getCountHardLimit(), statsTrack.getCount()));
+                break;
+            case QUOTA_BYTES_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getByteHardLimit(), statsTrack.getBytes()));
+                break;
+            case QUOTA_COUNT_USAGE:
+                metricsMap.put(namespace, statsTrack.getCount());
+                break;
+            case QUOTA_BYTES_USAGE:
+                metricsMap.put(namespace, statsTrack.getBytes());
+                break;
+            default:
+        }
+    }
+
+    // hard limit takes precedence if specified
+    static long getQuotaLimit(final long hardLimit, final long limit) {
+        return hardLimit > -1 ? hardLimit : limit;
+    }

Review comment:
       I agree.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r605845846



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }
+    }
+
+    static boolean shouldCollect(final String path, final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        return path.endsWith(LIMIT_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT == type)
+                || path.endsWith(STATS_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE == type);
+    }
+
+    static void collectQuotaLimitOrUsage(final String path,
+                                         final DataNode node,
+                                         final Map<String, Number> metricsMap,
+                                         final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final String namespace = PathUtils.getTopNamespace(Quotas.trimQuotaPath(path));
+        if (namespace == null) {
+            return;
+        }
+        if (node.getData() == null) {
+            return;
+        }
+        final StatsTrack statsTrack = new StatsTrack(node.getData());
+        switch (type) {
+            case QUOTA_COUNT_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getCountHardLimit(), statsTrack.getCount()));
+                break;
+            case QUOTA_BYTES_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getByteHardLimit(), statsTrack.getBytes()));
+                break;
+            case QUOTA_COUNT_USAGE:
+                metricsMap.put(namespace, statsTrack.getCount());
+                break;
+            case QUOTA_BYTES_USAGE:
+                metricsMap.put(namespace, statsTrack.getBytes());
+                break;
+            default:
+        }
+    }
+
+    // hard limit takes precedence if specified
+    static long getQuotaLimit(final long hardLimit, final long limit) {
+        return hardLimit > -1 ? hardLimit : limit;
+    }

Review comment:
       Yes, if we change the semantics of soft and hard quotas in the future, we will update the the published metrics along with the changes.  Changing the semantics is a fundamental thing, so it makes sense to make all the changes that are impacted at that time.
   
   There are two concerns for publishing both soft and hard limit at this point.
   
   1. It doesn't match with the current semantics of soft and hard quotas and make it harder for user t o use the metrics. 
   2. Given the constraint we have with the current interface of metrics framework, it means the we need to iterate two more times on the /zookeeper/quota tree. IMO, we pay something that doesn't give us too much benefits.
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r598222690



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }

Review comment:
       I thought about it as I saw the synchronized block is used when  DataNode.getChildren() is called in some other places, but I don't think synchronized is necessary here because getChildren() is already synchronized on the DatNode object and it's thread-safe.  Am I missing something?
   
   public synchronized Set<String> getChildren() {
           if (children == null) {
               return EMPTY_SET;
           }
   
           return Collections.unmodifiableSet(children);
       }
   
   
   
     




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] ztzg edited a comment on pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
ztzg edited a comment on pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#issuecomment-858413588


   Hi @li4wang,
   
   > > Could you have a look? Do you know what is going on?
   > 
   > Thanks a lot for reviewing it, @ztzg
   > 
   > The test case has been fixed.
   > 
   > The expected number of metrics should be 1 instead of 5 after the QUOTA_EXCEEDED_ERROR_PER_NAMESPACE is changed from SummarySet to CounterSet.
   
   Right; makes sense.  The update looks good.
    
   > All the test cases passed when I ran on my local. Can you please take a quick look at it? Thanks.
   
   LGTM.  But I will let CI finish.
   
   @eolivelli wrote:
   > The failure seems related to this patch.
   > I would like not to add a new flaky test
   
   That failure was deterministic; the test had not been updated after the introduction of the `CounterSet`.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r632274802



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final List<String> children;
+        synchronized (node) {
+            children = new ArrayList<>(node.getChildren());
+        }
+        if (children.size() == 0) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);

Review comment:
       Good catch. It should have `private`. I probably thought I need to add unit test case for this API at that time and changed the scope to 'package'.  I will change it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] ztzg commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
ztzg commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r632066881



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }

Review comment:
       Okay!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r598217478



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java
##########
@@ -259,6 +260,8 @@ private ServerMetrics(MetricsProvider metricsProvider) {
         WATCH_BYTES = metricsContext.getCounter("watch_bytes");
 
         JVM_PAUSE_TIME = metricsContext.getSummary("jvm_pause_time_ms", DetailLevel.ADVANCED);
+
+        QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = metricsContext.getSummarySet(QuotaMetricsUtils.QUOTA_EXCEEDED_ERROR_PER_NAMESPACE, DetailLevel.BASIC);

Review comment:
       SummarySet is used because it provides count (i.e. cnt) and the grouping capability.  Yes, you are right, in this case, since the count is always added by 1, the cnt and sum are the same with PrometheusMetricsProvider  and the min/max/avg with DefaultMetricsProvider are always 1 in this case. 
   
   Is there any reason that CounterSet is not supported?
   
   I added GaugeSet for other quota metrics, as SummarySet can not be used for them.
   
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] ztzg closed pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
ztzg closed pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@zookeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] li4wang commented on pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#issuecomment-866251017


   @ztzg @hanm Is there anything that I can help to get the PR merged? We are preparing for 3.7 rollout and it would be great if the changes can be included.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#issuecomment-858349421


   > The failure seems related to this patch.
   > I would like not to add a new flaky test
   
   @eolivelli the test case has been fixed. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r598222690



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }

Review comment:
       I thought about it as I saw the synchronized block is used when  DataNode.getChildren() is called in some other places, but I don't think synchronized is necessary because getChildren() is already synchronized on the DatNode object and it's thread-safe.  Am I missing something?
   
   public synchronized Set<String> getChildren() {
           if (children == null) {
               return EMPTY_SET;
           }
   
           return Collections.unmodifiableSet(children);
       }
   
   
   
     




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] ztzg commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
ztzg commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r598155234



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }
+    }
+
+    static boolean shouldCollect(final String path, final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        return path.endsWith(LIMIT_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT == type)
+                || path.endsWith(STATS_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE == type);
+    }
+
+    static void collectQuotaLimitOrUsage(final String path,
+                                         final DataNode node,
+                                         final Map<String, Number> metricsMap,
+                                         final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final String namespace = PathUtils.getTopNamespace(Quotas.trimQuotaPath(path));
+        if (namespace == null) {
+            return;
+        }
+        if (node.getData() == null) {
+            return;
+        }
+        final StatsTrack statsTrack = new StatsTrack(node.getData());
+        switch (type) {
+            case QUOTA_COUNT_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getCountHardLimit(), statsTrack.getCount()));
+                break;
+            case QUOTA_BYTES_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getByteHardLimit(), statsTrack.getBytes()));
+                break;
+            case QUOTA_COUNT_USAGE:
+                metricsMap.put(namespace, statsTrack.getCount());
+                break;
+            case QUOTA_BYTES_USAGE:
+                metricsMap.put(namespace, statsTrack.getBytes());
+                break;

Review comment:
       There is a mismatch, in general, between the notions of quota path (which cover subtrees, potentially deep in the hierarchy) and that of "top namespace."  I know you did not introduce the latter, and I understand that splitting just below the root is convenient in most deployments—but that mismatch is bound to rear its ugly head at the most inopportune moments :)
   
   The above suffers from it, as:
   
   ```
   create /ZOOKEEPER-4211
   create /ZOOKEEPER-4211/a
   create /ZOOKEEPER-4211/a/b
   setquota -B 32 /ZOOKEEPER-4211/a/b
   
   create /ZOOKEEPER-4211/a/z
   setquota -B 128 /ZOOKEEPER-4211/a/z
   
   set /ZOOKEEPER-4211/a/b 012345678901234567890123456789
   ```
   
   causes the `/a/z` subtree to "cover" `/a/b` and report the wrong results:
   
   ```
   quota_bytes_limit_per_namespace{key="ZOOKEEPER-4211",} 128.0
   quota_bytes_usage_per_namespace{key="ZOOKEEPER-4211",} 0.0
   ```
   
   I would suggest *summing* the values found in multiple subtrees of a given "namespace."




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#issuecomment-803518407


   Hi Damien,
   
   Thanks for reviewing the PR and the valuable comments.  Please see my response.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r598223377



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }
+    }
+
+    static boolean shouldCollect(final String path, final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        return path.endsWith(LIMIT_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT == type)
+                || path.endsWith(STATS_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE == type);
+    }
+
+    static void collectQuotaLimitOrUsage(final String path,
+                                         final DataNode node,
+                                         final Map<String, Number> metricsMap,
+                                         final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final String namespace = PathUtils.getTopNamespace(Quotas.trimQuotaPath(path));
+        if (namespace == null) {
+            return;
+        }
+        if (node.getData() == null) {
+            return;
+        }
+        final StatsTrack statsTrack = new StatsTrack(node.getData());
+        switch (type) {
+            case QUOTA_COUNT_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getCountHardLimit(), statsTrack.getCount()));
+                break;
+            case QUOTA_BYTES_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getByteHardLimit(), statsTrack.getBytes()));
+                break;
+            case QUOTA_COUNT_USAGE:
+                metricsMap.put(namespace, statsTrack.getCount());
+                break;
+            case QUOTA_BYTES_USAGE:
+                metricsMap.put(namespace, statsTrack.getBytes());
+                break;

Review comment:
       Yes, good suggestion. In our case, we only set the quota on the top namespace, so no need to sum. But for general use case, we want to do sum. I will change it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] ztzg commented on pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
ztzg commented on pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#issuecomment-888947183


   @eolivelli wrote:
   
   > This is an API change (about the Metrics Provider API) and in my opinion it is better to not pick it on a released branch, in which the API must be stable.
   >
   > So I would keep this only on master branch
   
   You're right, of course; the change to `MetricsContext` would break third-party metrics providers.
   
   @li4wang wrote:
   
   > The conflicts have been resolved.
   
   Thanks! Looking into it now…
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@zookeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] li4wang commented on pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#issuecomment-888479191


   I will take care of the conflict.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@zookeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] ztzg commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
ztzg commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r632071331



##########
File path: zookeeper-server/src/test/java/org/apache/zookeeper/test/EnforceQuotaTest.java
##########
@@ -54,14 +56,16 @@ public void tearDown() throws Exception {
 
     @Test
     public void testSetQuotaDisableWhenExceedBytesHardQuota() throws Exception {
-        final String path = "/c1";
+        final String namespace = UUID.randomUUID().toString();

Review comment:
       Why not `namespace = "c1"`?  Is there a specific reason to change from a static path to a dynamic/random one?
   
   (I haven't looked into the tests yet, so may be missing something obvious.)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang edited a comment on pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang edited a comment on pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#issuecomment-858346284


   > Could you have a look? Do you know what is going on?
   
   Thanks a lot for reviewing it, @ztzg 
   
   The test case has been fixed. 
   
   The expected number of metrics should be 1 instead of 5 after the QUOTA_EXCEEDED_ERROR_PER_NAMESPACE is changed from SummarySet to CounterSet.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] ztzg commented on pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
ztzg commented on pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#issuecomment-857981721


   Closing/reopening to trigger Jenkins job.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r605985813



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final List<String> children;
+        synchronized (node) {
+            children = new ArrayList<>(node.getChildren());
+        }
+        if (children.size() == 0) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);

Review comment:
       getQuotaLimitOrUsage () is not a public API. The metricsMap passed is a ConcurrentHashMap and it's thread safe.
   
   I think this can be ignored.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] ztzg commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
ztzg commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r598152755



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java
##########
@@ -259,6 +260,8 @@ private ServerMetrics(MetricsProvider metricsProvider) {
         WATCH_BYTES = metricsContext.getCounter("watch_bytes");
 
         JVM_PAUSE_TIME = metricsContext.getSummary("jvm_pause_time_ms", DetailLevel.ADVANCED);
+
+        QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = metricsContext.getSummarySet(QuotaMetricsUtils.QUOTA_EXCEEDED_ERROR_PER_NAMESPACE, DetailLevel.BASIC);

Review comment:
       I suppose you used a `SummarySet` because we are missing a `CounterSet`, but this generates strange results with default endpoint as well as the Prometheus one. `cnt` and `sum` are effectively duplicate, whereas `min`, `max`, `avg` are just nonsensical:
   
   ```
   zk_avg_ZOOKEEPER-4211_quota_exceeded_error_per_namespace	1.0
   zk_min_ZOOKEEPER-4211_quota_exceeded_error_per_namespace	1
   zk_max_ZOOKEEPER-4211_quota_exceeded_error_per_namespace	1
   zk_cnt_ZOOKEEPER-4211_quota_exceeded_error_per_namespace	5
   zk_sum_ZOOKEEPER-4211_quota_exceeded_error_per_namespace	5
   ```
   

##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }

Review comment:
       Hmm… I understand that you are constrained by the current interface, but it also means we iterate four times on the `/zookeeper/quota` tree.  It also "forces" the implementation to report unset (count or byte) quotas as `-1`, as it cannot correlate limits and values without doing an even larger amount of duplicate work.

##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }
+    }
+
+    static boolean shouldCollect(final String path, final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        return path.endsWith(LIMIT_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT == type)
+                || path.endsWith(STATS_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE == type);
+    }
+
+    static void collectQuotaLimitOrUsage(final String path,
+                                         final DataNode node,
+                                         final Map<String, Number> metricsMap,
+                                         final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final String namespace = PathUtils.getTopNamespace(Quotas.trimQuotaPath(path));
+        if (namespace == null) {
+            return;
+        }
+        if (node.getData() == null) {
+            return;
+        }
+        final StatsTrack statsTrack = new StatsTrack(node.getData());
+        switch (type) {
+            case QUOTA_COUNT_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getCountHardLimit(), statsTrack.getCount()));
+                break;
+            case QUOTA_BYTES_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getByteHardLimit(), statsTrack.getBytes()));
+                break;
+            case QUOTA_COUNT_USAGE:
+                metricsMap.put(namespace, statsTrack.getCount());
+                break;
+            case QUOTA_BYTES_USAGE:
+                metricsMap.put(namespace, statsTrack.getBytes());
+                break;
+            default:
+        }
+    }
+
+    // hard limit takes precedence if specified
+    static long getQuotaLimit(final long hardLimit, final long limit) {
+        return hardLimit > -1 ? hardLimit : limit;
+    }

Review comment:
       This is another case of policy and mechanism getting mixed up; do we really want to prevent people from monitoring their soft quota errors?

##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }

Review comment:
       I believe this is illegal: the `Set<String>` returned by `DataNode.getChildren()` is not synchronized, and other ZooKeeper operations could be adding or removing nodes while you are iterating on it.  You need something like:
   
   ```java
   synchronized(node) {
       children = new ArrayList<>(node.getChildren())
   }
   ```
   
   @eolivelli: do you agree, or am I missing something?

##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }
+    }
+
+    static boolean shouldCollect(final String path, final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        return path.endsWith(LIMIT_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT == type)
+                || path.endsWith(STATS_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE == type);
+    }
+
+    static void collectQuotaLimitOrUsage(final String path,
+                                         final DataNode node,
+                                         final Map<String, Number> metricsMap,
+                                         final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final String namespace = PathUtils.getTopNamespace(Quotas.trimQuotaPath(path));
+        if (namespace == null) {
+            return;
+        }
+        if (node.getData() == null) {
+            return;
+        }
+        final StatsTrack statsTrack = new StatsTrack(node.getData());

Review comment:
       I believe you want this:
   
   ```java
   final byte[] data = node.getData();
   if (data == null) { ... }
   ...
   // use data
   ```
   
   as unless I am missing something, the second call to `getData()` could return `null` even if the first did not.

##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final Set<String> children = node.getChildren();
+        if (children.isEmpty()) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);
+            }
+            return;
+        }
+        for (final String child : children) {
+            getQuotaLimitOrUsage(path + "/" + child, metricsMap, type, dataTree);
+        }
+    }
+
+    static boolean shouldCollect(final String path, final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        return path.endsWith(LIMIT_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT == type)
+                || path.endsWith(STATS_END_STRING)
+                && (QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE == type || QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE == type);
+    }
+
+    static void collectQuotaLimitOrUsage(final String path,
+                                         final DataNode node,
+                                         final Map<String, Number> metricsMap,
+                                         final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final String namespace = PathUtils.getTopNamespace(Quotas.trimQuotaPath(path));
+        if (namespace == null) {
+            return;
+        }
+        if (node.getData() == null) {
+            return;
+        }
+        final StatsTrack statsTrack = new StatsTrack(node.getData());
+        switch (type) {
+            case QUOTA_COUNT_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getCountHardLimit(), statsTrack.getCount()));
+                break;
+            case QUOTA_BYTES_LIMIT:
+                metricsMap.put(namespace, getQuotaLimit(statsTrack.getByteHardLimit(), statsTrack.getBytes()));
+                break;
+            case QUOTA_COUNT_USAGE:
+                metricsMap.put(namespace, statsTrack.getCount());
+                break;
+            case QUOTA_BYTES_USAGE:
+                metricsMap.put(namespace, statsTrack.getBytes());
+                break;

Review comment:
       There is a mismatch, in general, between the notions of quota path (which cover subtrees, potentially deep in the hierarchy) and that of "top namespace."  I know you did not introduce the latter, and I understand that splitting just below the root is convenient in most deployments—but that mismatch is bound to rear its ugly head at the most inopportune moments :)
   
   The above suffers from it, as:
   
   ```
   create /ZOOKEEPER-4211
   create /ZOOKEEPER-4211/a
   create /ZOOKEEPER-4211/a/b
   setquota -B 32 /ZOOKEEPER-4211/a/b
   
   create /ZOOKEEPER-4211/a/z
   setquota -B 128 /ZOOKEEPER-4211/a/z
   
   set /ZOOKEEPER-4211/a/b 0123456789012345678901234567890123456789
   ```
   
   causes the `/a/z` subtree to "cover" `/a/b` and report the wrong results:
   
   ```
   quota_bytes_limit_per_namespace{key="ZOOKEEPER-4211",} 128.0
   quota_bytes_usage_per_namespace{key="ZOOKEEPER-4211",} 0.0
   ```
   
   I would suggest *summing* the values found in multiple subtrees of a given "namespace."




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] ztzg closed pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
ztzg closed pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] hanm commented on pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
hanm commented on pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#issuecomment-859251149


   > LGTM
   > 
   > It would be better that @lvfangmin or @hanm to take a look to this patch as well
   
   This is a fairly big patch, I will try to find sometime to take a look in next few days, but please don't let me block this landing as I see this already gets at least two committer reviews.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r598217478



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java
##########
@@ -259,6 +260,8 @@ private ServerMetrics(MetricsProvider metricsProvider) {
         WATCH_BYTES = metricsContext.getCounter("watch_bytes");
 
         JVM_PAUSE_TIME = metricsContext.getSummary("jvm_pause_time_ms", DetailLevel.ADVANCED);
+
+        QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = metricsContext.getSummarySet(QuotaMetricsUtils.QUOTA_EXCEEDED_ERROR_PER_NAMESPACE, DetailLevel.BASIC);

Review comment:
       SummarySet is used because it provides count (i.e. cnt) and the grouping capability.  Yes, you are right, in this case, since the count is always added by 1, the cnt and sum are the same with PrometheusMetricsProvider and the min/max/avg with DefaultMetricsProvider are always 1 in this case.  User can just use the cnt and ignore the others.
   
   Is any reason that CounterSet not supported? Do you think we need to add CounterSet in this PR?
   
   I added GaugeSet for other quota metrics, as SummarySet really can not be used for them. For 
   
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



[GitHub] [zookeeper] li4wang commented on a change in pull request #1644: ZOOKEEPER-4211: Expose Per Namespace Quota Metrics to Prometheus

Posted by GitBox <gi...@apache.org>.
li4wang commented on a change in pull request #1644:
URL: https://github.com/apache/zookeeper/pull/1644#discussion_r632274802



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/QuotaMetricsUtils.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.util;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.zookeeper.Quotas;
+import org.apache.zookeeper.StatsTrack;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+
+public final class QuotaMetricsUtils {
+    public static final String QUOTA_COUNT_LIMIT_PER_NAMESPACE = "quota_count_limit_per_namespace";
+    public static final String QUOTA_BYTES_LIMIT_PER_NAMESPACE = "quota_bytes_limit_per_namespace";
+    public static final String QUOTA_COUNT_USAGE_PER_NAMESPACE = "quota_count_usage_per_namespace";
+    public static final String QUOTA_BYTES_USAGE_PER_NAMESPACE = "quota_bytes_usage_per_namespace";
+    public static final String QUOTA_EXCEEDED_ERROR_PER_NAMESPACE = "quota_exceeded_error_per_namespace";
+
+    enum QUOTA_LIMIT_USAGE_METRIC_TYPE {QUOTA_COUNT_LIMIT, QUOTA_BYTES_LIMIT, QUOTA_COUNT_USAGE, QUOTA_BYTES_USAGE}
+    static final String LIMIT_END_STRING = "/" + Quotas.limitNode;
+    static final String STATS_END_STRING = "/" + Quotas.statNode;
+
+    private QuotaMetricsUtils() {
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count limit
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes limit
+     *`
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota bytes limit as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesLimit(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_LIMIT);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota count usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return a map with top namespace as the key and quota count usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaCountUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_COUNT_USAGE);
+    }
+
+    /**
+     * Traverse the quota subtree and return per namespace quota bytes usage
+     *
+     * @param dataTree dataTree that contains the quota limit and usage data
+     * @return  a map with top namespace as the key and quota bytes usage as the value
+     *
+     */
+    public static Map<String, Number> getQuotaBytesUsage(final DataTree dataTree) {
+        return getQuotaLimitOrUsage(dataTree, QUOTA_LIMIT_USAGE_METRIC_TYPE.QUOTA_BYTES_USAGE);
+    }
+
+    // traverse the quota subtree and read the quota limit or usage data
+    private static Map<String, Number> getQuotaLimitOrUsage(final DataTree dataTree,
+                                                            final QUOTA_LIMIT_USAGE_METRIC_TYPE type) {
+        final Map<String, Number> metricsMap = new ConcurrentHashMap<>();
+        if (dataTree != null) {
+            getQuotaLimitOrUsage(Quotas.quotaZookeeper, metricsMap, type, dataTree);
+        }
+        return metricsMap;
+    }
+
+    static void getQuotaLimitOrUsage(final String path,
+                                     final Map<String, Number> metricsMap,
+                                     final QUOTA_LIMIT_USAGE_METRIC_TYPE type,
+                                     final DataTree dataTree) {
+        final DataNode node = dataTree.getNode(path);
+        if (node == null) {
+            return;
+        }
+        final List<String> children;
+        synchronized (node) {
+            children = new ArrayList<>(node.getChildren());
+        }
+        if (children.size() == 0) {
+            if (shouldCollect(path, type)) {
+                collectQuotaLimitOrUsage(path, node, metricsMap, type);

Review comment:
       Good catch. It should have `private`. I probably thought I need to add unit test case for this API at that time and changed the scope to 'package'.  I will add it back.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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