You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@helix.apache.org by GitBox <gi...@apache.org> on 2020/01/16 22:41:57 UTC

[GitHub] [helix] pkuwm opened a new pull request #686: Add partition weight gauge.

pkuwm opened a new pull request #686: Add partition weight gauge.
URL: https://github.com/apache/helix/pull/686
 
 
   ### Issues
   
   - [ ] My PR addresses the following Helix issues and references them in the PR description:
   
   Fixes #685
   
   ### Description
   
   - [ ] Here are some details about my PR, including screenshots of any UI changes:
   
   We would like to monitor the usage of each capacity for the resource partitions: gauge of the average partition weight for each CAPACITY key.
   
   ### Tests
   
   - [ ] The following tests are written for this issue:
   
   - testUpdatePartitionWeight
   - testCalculateAveragePartitionWeight
   
   - [ ] The following is the result of the "mvn test" command on the appropriate module:
   
   Running...
   
   ### Commits
   
   - [ ] My commits all reference appropriate Apache Helix GitHub issues in their subject lines, and I have squashed multiple commits if they address the same issue. In addition, my commits follow the guidelines from "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)":
     1. Subject is separated from body by a blank line
     1. Subject is limited to 50 characters (not including Jira issue reference)
     1. Subject does not end with a period
     1. Subject uses the imperative mood ("add", not "adding")
     1. Body wraps at 72 characters
     1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation in the following wiki page:
   
   (Link the GitHub wiki you added)
   
   ### Code Quality
   
   - [ ] My diff has been formatted using helix-style.xml

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] pkuwm commented on a change in pull request #686: Add resource partition weight gauge

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #686: Add resource partition weight gauge
URL: https://github.com/apache/helix/pull/686#discussion_r369315560
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestResourceMonitor.java
 ##########
 @@ -217,10 +217,13 @@ public void testReportData() throws JMException {
     monitor.setRebalanceState(ResourceMonitor.RebalanceStatus.INTERMEDIATE_STATE_CAL_FAILED);
     Assert.assertEquals(monitor.getRebalanceState(),
         ResourceMonitor.RebalanceStatus.INTERMEDIATE_STATE_CAL_FAILED.name());
+
+    // Has to unregister this monitor to clean up. Otherwise, later tests may be affected and fail.
+    monitor.unregister();
 
 Review comment:
   Done.

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #686: Add resource partition weight gauge

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #686: Add resource partition weight gauge
URL: https://github.com/apache/helix/pull/686#discussion_r369312412
 
 

 ##########
 File path: helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestResourceMonitor.java
 ##########
 @@ -217,10 +217,13 @@ public void testReportData() throws JMException {
     monitor.setRebalanceState(ResourceMonitor.RebalanceStatus.INTERMEDIATE_STATE_CAL_FAILED);
     Assert.assertEquals(monitor.getRebalanceState(),
         ResourceMonitor.RebalanceStatus.INTERMEDIATE_STATE_CAL_FAILED.name());
+
+    // Has to unregister this monitor to clean up. Otherwise, later tests may be affected and fail.
+    monitor.unregister();
 
 Review comment:
   In this case, put it in finally block and check if monitor is null before unregister.

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] pkuwm commented on a change in pull request #686: Add resource partition weight gauge

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #686: Add resource partition weight gauge
URL: https://github.com/apache/helix/pull/686#discussion_r369220159
 
 

 ##########
 File path: helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ResourceMonitor.java
 ##########
 @@ -382,6 +367,37 @@ public void updateRebalancerStats(long numPendingRecoveryRebalancePartitions,
     _numLoadRebalanceThrottledPartitions.updateValue(numLoadRebalanceThrottledPartitions);
   }
 
+  /**
+   * Updates partition weight metric. If the partition capacity keys are changed, all MBean
+   * attributes will be updated accordingly: old capacity keys will be replaced with new capacity
+   * keys in MBean server.
+   *
+   * @param partitionWeightMap A map of partition weight: capacity key -> partition weight
+   */
+  void updatePartitionWeightStats(Map<String, Integer> partitionWeightMap) {
+    synchronized (_dynamicCapacityMetricsMap) {
+      if (_dynamicCapacityMetricsMap.keySet().equals(partitionWeightMap.keySet())) {
+        for (Map.Entry<String, Integer> entry : partitionWeightMap.entrySet()) {
+          _dynamicCapacityMetricsMap.get(entry.getKey()).updateValue((long) entry.getValue());
+        }
+        return;
+      }
+
+      // Capacity keys are changed, so capacity attribute map needs to be updated.
+      _dynamicCapacityMetricsMap.clear();
+      final String gaugeMetricSuffix = "Gauge";
 
 Review comment:
   OK.

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #686: Add resource partition weight gauge

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #686: Add resource partition weight gauge
URL: https://github.com/apache/helix/pull/686#discussion_r368193248
 
 

 ##########
 File path: helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/ResourceUsageCalculator.java
 ##########
 @@ -142,4 +142,97 @@ public static double measureBaselineDivergence(Map<String, ResourceAssignment> b
     return numTotalBestPossibleReplicas == 0 ? 1.0d
         : (1.0d - (double) numMatchedReplicas / (double) numTotalBestPossibleReplicas);
   }
+
+  /**
+   * Calculates average partition weight per capacity key for a resource config. Example as below:
+   * Input =
+   * {
+   *   "partition1": {
+   *     "capacity1": 20,
+   *     "capacity2": 40
+   *   },
+   *   "partition2": {
+   *     "capacity1": 30,
+   *     "capacity2": 50
+   *   },
+   *   "partition3": {
+   *     "capacity1": 16,
+   *     "capacity2": 30
+   *   }
+   * }
+   *
+   * Total weight for key "capacity1" = 20 + 30 + 16 = 66;
+   * Total weight for key "capacity2" = 40 + 50 + 30 = 120;
+   * Total partitions = 3;
+   * Average partition weight for "capacity1" = 66 / 3 = 22;
+   * Average partition weight for "capacity2" = 120 / 3 = 40;
+   *
+   * Output =
+   * {
+   *   "capacity1": 22,
+   *   "capacity2": 40
+   * }
+   *
+   * @param partitionCapacityMap A map of partition capacity:
+   *        <PartitionName or DEFAULT_PARTITION_KEY, <Capacity Key, Capacity Number>>
+   * @return A map of partition weight: capacity key -> average partition weight
+   */
+  public static Map<String, Integer> calculateAveragePartitionWeight(
 
 Review comment:
   Let's simplify this by merging the 2 methods?

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang merged pull request #686: Add resource partition weight gauge

Posted by GitBox <gi...@apache.org>.
jiajunwang merged pull request #686: Add resource partition weight gauge
URL: https://github.com/apache/helix/pull/686
 
 
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] pkuwm commented on issue #686: Add resource partition weight gauge

Posted by GitBox <gi...@apache.org>.
pkuwm commented on issue #686: Add resource partition weight gauge
URL: https://github.com/apache/helix/pull/686#issuecomment-577333169
 
 
   This PR is ready to be merged, approved by @jiajunwang 
   
   ```
   We would like to monitor the usage of each capacity for the resource partitions: gauge of the average partition weight for each CAPACITY key.
   
   Change list:
   - Add partition weight gauge metric to resource monitor.
   - Add two unit tests to cover new code.
   ```

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #686: Add resource partition weight gauge

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #686: Add resource partition weight gauge
URL: https://github.com/apache/helix/pull/686#discussion_r368192940
 
 

 ##########
 File path: helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/ResourceUsageCalculator.java
 ##########
 @@ -142,4 +142,97 @@ public static double measureBaselineDivergence(Map<String, ResourceAssignment> b
     return numTotalBestPossibleReplicas == 0 ? 1.0d
         : (1.0d - (double) numMatchedReplicas / (double) numTotalBestPossibleReplicas);
   }
+
+  /**
+   * Calculates average partition weight per capacity key for a resource config. Example as below:
+   * Input =
+   * {
+   *   "partition1": {
+   *     "capacity1": 20,
+   *     "capacity2": 40
+   *   },
+   *   "partition2": {
+   *     "capacity1": 30,
+   *     "capacity2": 50
+   *   },
+   *   "partition3": {
+   *     "capacity1": 16,
+   *     "capacity2": 30
+   *   }
+   * }
+   *
+   * Total weight for key "capacity1" = 20 + 30 + 16 = 66;
+   * Total weight for key "capacity2" = 40 + 50 + 30 = 120;
+   * Total partitions = 3;
+   * Average partition weight for "capacity1" = 66 / 3 = 22;
+   * Average partition weight for "capacity2" = 120 / 3 = 40;
+   *
+   * Output =
+   * {
+   *   "capacity1": 22,
+   *   "capacity2": 40
+   * }
+   *
+   * @param partitionCapacityMap A map of partition capacity:
+   *        <PartitionName or DEFAULT_PARTITION_KEY, <Capacity Key, Capacity Number>>
+   * @return A map of partition weight: capacity key -> average partition weight
+   */
+  public static Map<String, Integer> calculateAveragePartitionWeight(
+      Map<String, Map<String, Integer>> partitionCapacityMap) {
+    Map<String, PartitionWeightCounterEntry> countPartitionWeightMap =
+        aggregatePartitionWeight(partitionCapacityMap);
+
+    // capacity key -> average partition weight
+    Map<String, Integer> averagePartitionWeightMap = new HashMap<>();
+
+    // Calculate average partition weight for each capacity key.
+    // Per capacity key level:
+    // average partition weight = (total partition weight) / (number of partitions)
+    for (Map.Entry<String, PartitionWeightCounterEntry> entry
+        : countPartitionWeightMap.entrySet()) {
+      String capacityKey = entry.getKey();
+      PartitionWeightCounterEntry weightEntry = entry.getValue();
+      int averageWeight = weightEntry.getWeight() / weightEntry.getPartitions();
+      averagePartitionWeightMap.put(capacityKey, averageWeight);
+    }
+
+    return averagePartitionWeightMap;
+  }
+
+  /*
+   * Aggregates partition weight for each capacity key.
+   */
+  private static Map<String, PartitionWeightCounterEntry> aggregatePartitionWeight(
+      Map<String, Map<String, Integer>> partitionCapacityMap) {
+    // capacity key -> [number of partitions, total weight per capacity key]
+    Map<String, PartitionWeightCounterEntry> countPartitionWeightMap = new HashMap<>();
+
+    partitionCapacityMap.values().forEach(partitionCapacityEntry ->
+        partitionCapacityEntry.forEach((capacityKey, weight) -> countPartitionWeightMap
+            .computeIfAbsent(capacityKey, counterEntry -> new PartitionWeightCounterEntry())
+            .increase(1, weight)));
+
+    return countPartitionWeightMap;
+  }
+
+  /*
+   * Represents total number of partitions and total partition weight for a capacity key.
+   */
+  private static class PartitionWeightCounterEntry {
+    private int partitions;
+    private int weight;
 
 Review comment:
   long would be safer.

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #686: Add resource partition weight gauge

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #686: Add resource partition weight gauge
URL: https://github.com/apache/helix/pull/686#discussion_r368191477
 
 

 ##########
 File path: helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterStatusMonitor.java
 ##########
 @@ -519,7 +538,7 @@ public void updateRebalancerStats(String resourceName, long numPendingRecoveryRe
     }
   }
 
-  private ResourceMonitor getOrCreateResourceMonitor(String resourceName) {
+  ResourceMonitor getOrCreateResourceMonitor(String resourceName) {
 
 Review comment:
   Why?

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] pkuwm commented on a change in pull request #686: Add resource partition weight gauge

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #686: Add resource partition weight gauge
URL: https://github.com/apache/helix/pull/686#discussion_r369219486
 
 

 ##########
 File path: helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/ResourceUsageCalculator.java
 ##########
 @@ -142,4 +142,97 @@ public static double measureBaselineDivergence(Map<String, ResourceAssignment> b
     return numTotalBestPossibleReplicas == 0 ? 1.0d
         : (1.0d - (double) numMatchedReplicas / (double) numTotalBestPossibleReplicas);
   }
+
+  /**
+   * Calculates average partition weight per capacity key for a resource config. Example as below:
+   * Input =
+   * {
+   *   "partition1": {
+   *     "capacity1": 20,
+   *     "capacity2": 40
+   *   },
+   *   "partition2": {
+   *     "capacity1": 30,
+   *     "capacity2": 50
+   *   },
+   *   "partition3": {
+   *     "capacity1": 16,
+   *     "capacity2": 30
+   *   }
+   * }
+   *
+   * Total weight for key "capacity1" = 20 + 30 + 16 = 66;
+   * Total weight for key "capacity2" = 40 + 50 + 30 = 120;
+   * Total partitions = 3;
+   * Average partition weight for "capacity1" = 66 / 3 = 22;
+   * Average partition weight for "capacity2" = 120 / 3 = 40;
+   *
+   * Output =
+   * {
+   *   "capacity1": 22,
+   *   "capacity2": 40
+   * }
+   *
+   * @param partitionCapacityMap A map of partition capacity:
+   *        <PartitionName or DEFAULT_PARTITION_KEY, <Capacity Key, Capacity Number>>
+   * @return A map of partition weight: capacity key -> average partition weight
+   */
+  public static Map<String, Integer> calculateAveragePartitionWeight(
 
 Review comment:
   Done

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #686: Add resource partition weight gauge

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #686: Add resource partition weight gauge
URL: https://github.com/apache/helix/pull/686#discussion_r368191391
 
 

 ##########
 File path: helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterStatusMonitor.java
 ##########
 @@ -497,6 +497,25 @@ public void setResourceStatus(ExternalView externalView, IdealState idealState,
     }
   }
 
+  /**
+   * Updates metrics of average partition weight per capacity key for a resource. If a resource
+   * monitor is not yet existed for this resource, a new resource monitor will be created for this
+   * resource.
+   *
+   * @param resourceName The resource name for which partition weight is updated
+   * @param averageWeightMap A map of average partition weight of each capacity key:
+   *                         capacity key -> average partition weight
+   */
+  public void updatePartitionWeight(String resourceName, Map<String, Integer> averageWeightMap) {
+    ResourceMonitor monitor = _resourceMonitorMap.get(resourceName);
 
 Review comment:
   getOrCreateResourceMonitor()

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] pkuwm commented on a change in pull request #686: Add resource partition weight gauge

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #686: Add resource partition weight gauge
URL: https://github.com/apache/helix/pull/686#discussion_r369219969
 
 

 ##########
 File path: helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterStatusMonitor.java
 ##########
 @@ -519,7 +538,7 @@ public void updateRebalancerStats(String resourceName, long numPendingRecoveryRe
     }
   }
 
-  private ResourceMonitor getOrCreateResourceMonitor(String resourceName) {
+  ResourceMonitor getOrCreateResourceMonitor(String resourceName) {
 
 Review comment:
   For test purpose. As I discussed offline, we will keep this private and just call monitor's register().

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] pkuwm commented on a change in pull request #686: Add resource partition weight gauge

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #686: Add resource partition weight gauge
URL: https://github.com/apache/helix/pull/686#discussion_r369219403
 
 

 ##########
 File path: helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/ResourceUsageCalculator.java
 ##########
 @@ -142,4 +142,97 @@ public static double measureBaselineDivergence(Map<String, ResourceAssignment> b
     return numTotalBestPossibleReplicas == 0 ? 1.0d
         : (1.0d - (double) numMatchedReplicas / (double) numTotalBestPossibleReplicas);
   }
+
+  /**
+   * Calculates average partition weight per capacity key for a resource config. Example as below:
+   * Input =
+   * {
+   *   "partition1": {
+   *     "capacity1": 20,
+   *     "capacity2": 40
+   *   },
+   *   "partition2": {
+   *     "capacity1": 30,
+   *     "capacity2": 50
+   *   },
+   *   "partition3": {
+   *     "capacity1": 16,
+   *     "capacity2": 30
+   *   }
+   * }
+   *
+   * Total weight for key "capacity1" = 20 + 30 + 16 = 66;
+   * Total weight for key "capacity2" = 40 + 50 + 30 = 120;
+   * Total partitions = 3;
+   * Average partition weight for "capacity1" = 66 / 3 = 22;
+   * Average partition weight for "capacity2" = 120 / 3 = 40;
+   *
+   * Output =
+   * {
+   *   "capacity1": 22,
+   *   "capacity2": 40
+   * }
+   *
+   * @param partitionCapacityMap A map of partition capacity:
+   *        <PartitionName or DEFAULT_PARTITION_KEY, <Capacity Key, Capacity Number>>
+   * @return A map of partition weight: capacity key -> average partition weight
+   */
+  public static Map<String, Integer> calculateAveragePartitionWeight(
+      Map<String, Map<String, Integer>> partitionCapacityMap) {
+    Map<String, PartitionWeightCounterEntry> countPartitionWeightMap =
+        aggregatePartitionWeight(partitionCapacityMap);
+
+    // capacity key -> average partition weight
+    Map<String, Integer> averagePartitionWeightMap = new HashMap<>();
+
+    // Calculate average partition weight for each capacity key.
+    // Per capacity key level:
+    // average partition weight = (total partition weight) / (number of partitions)
+    for (Map.Entry<String, PartitionWeightCounterEntry> entry
+        : countPartitionWeightMap.entrySet()) {
+      String capacityKey = entry.getKey();
+      PartitionWeightCounterEntry weightEntry = entry.getValue();
+      int averageWeight = weightEntry.getWeight() / weightEntry.getPartitions();
+      averagePartitionWeightMap.put(capacityKey, averageWeight);
+    }
+
+    return averagePartitionWeightMap;
+  }
+
+  /*
+   * Aggregates partition weight for each capacity key.
+   */
+  private static Map<String, PartitionWeightCounterEntry> aggregatePartitionWeight(
+      Map<String, Map<String, Integer>> partitionCapacityMap) {
+    // capacity key -> [number of partitions, total weight per capacity key]
+    Map<String, PartitionWeightCounterEntry> countPartitionWeightMap = new HashMap<>();
+
+    partitionCapacityMap.values().forEach(partitionCapacityEntry ->
+        partitionCapacityEntry.forEach((capacityKey, weight) -> countPartitionWeightMap
+            .computeIfAbsent(capacityKey, counterEntry -> new PartitionWeightCounterEntry())
+            .increase(1, weight)));
+
+    return countPartitionWeightMap;
+  }
+
+  /*
+   * Represents total number of partitions and total partition weight for a capacity key.
+   */
+  private static class PartitionWeightCounterEntry {
+    private int partitions;
+    private int weight;
 
 Review comment:
   Done.

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #686: Add resource partition weight gauge

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #686: Add resource partition weight gauge
URL: https://github.com/apache/helix/pull/686#discussion_r368193942
 
 

 ##########
 File path: helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ResourceMonitor.java
 ##########
 @@ -382,6 +367,37 @@ public void updateRebalancerStats(long numPendingRecoveryRebalancePartitions,
     _numLoadRebalanceThrottledPartitions.updateValue(numLoadRebalanceThrottledPartitions);
   }
 
+  /**
+   * Updates partition weight metric. If the partition capacity keys are changed, all MBean
+   * attributes will be updated accordingly: old capacity keys will be replaced with new capacity
+   * keys in MBean server.
+   *
+   * @param partitionWeightMap A map of partition weight: capacity key -> partition weight
+   */
+  void updatePartitionWeightStats(Map<String, Integer> partitionWeightMap) {
+    synchronized (_dynamicCapacityMetricsMap) {
+      if (_dynamicCapacityMetricsMap.keySet().equals(partitionWeightMap.keySet())) {
+        for (Map.Entry<String, Integer> entry : partitionWeightMap.entrySet()) {
+          _dynamicCapacityMetricsMap.get(entry.getKey()).updateValue((long) entry.getValue());
+        }
+        return;
+      }
+
+      // Capacity keys are changed, so capacity attribute map needs to be updated.
+      _dynamicCapacityMetricsMap.clear();
+      final String gaugeMetricSuffix = "Gauge";
 
 Review comment:
   nit, static final and put to the class level?

----------------------------------------------------------------
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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org