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/08/05 05:26:09 UTC

[GitHub] [helix] zhangmeng916 commented on a change in pull request #1187: Add latency metric support for customized view aggregation

zhangmeng916 commented on a change in pull request #1187:
URL: https://github.com/apache/helix/pull/1187#discussion_r465472688



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/CustomizedStateOutput.java
##########
@@ -24,32 +24,58 @@
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.helix.model.CustomizedState;
 import org.apache.helix.model.Partition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 public class CustomizedStateOutput {
+  private static final Logger LOG = LoggerFactory.getLogger(CustomizedStateOutput.class);
+
   // stateType -> (resourceName -> (Partition -> (instanceName -> customizedState)))
   private final Map<String, Map<String, Map<Partition, Map<String, String>>>> _customizedStateMap;
+  // stateType -> (resourceName -> (Partition -> (instanceName -> startTime)))
+  private final Map<String, Map<String, Map<Partition, Map<String, String>>>> _startTimeMap;
 
   public CustomizedStateOutput() {
     _customizedStateMap = new HashMap<>();
+    _startTimeMap = new HashMap<>();
   }
 
   public void setCustomizedState(String stateType, String resourceName, Partition partition,
       String instanceName, String state) {
-    if (!_customizedStateMap.containsKey(stateType)) {
-      _customizedStateMap
-          .put(stateType, new HashMap<String, Map<Partition, Map<String, String>>>());
-    }
-    if (!_customizedStateMap.get(stateType).containsKey(resourceName)) {
-      _customizedStateMap.get(stateType)
-          .put(resourceName, new HashMap<Partition, Map<String, String>>());
-    }
-    if (!_customizedStateMap.get(stateType).get(resourceName).containsKey(partition)) {
-      _customizedStateMap.get(stateType).get(resourceName)
-          .put(partition, new HashMap<String, String>());
+    setCustomizedStateProperty(CustomizedState.CustomizedStateProperty.CURRENT_STATE, stateType,
+        resourceName, partition, instanceName, state);
+  }
+
+  public void setStartTime(String stateType, String resourceName, Partition partition,
+      String instanceName, String state) {
+    setCustomizedStateProperty(CustomizedState.CustomizedStateProperty.START_TIME, stateType,
+        resourceName, partition, instanceName, state);
+  }
+
+  private void setCustomizedStateProperty(CustomizedState.CustomizedStateProperty propertyName,
+      String stateType, String resourceName, Partition partition, String instanceName,
+      String state) {
+    Map<String, Map<String, Map<Partition, Map<String, String>>>> mapToUpdate;

Review comment:
       Can we rename `mapToUpdate` and the following `readFromMap` to be more meaningful and following convention.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/CustomizedViewAggregationStage.java
##########
@@ -97,7 +103,8 @@ public void execute(final ClusterEvent event) throws Exception {
       for (Resource resource : resourceMap.values()) {
         try {
           computeCustomizedStateView(resource, stateType, customizedStateOutput, curCustomizedViews,
-              updatedCustomizedViews);
+              updatedCustomizedViews, updatedStartTimestamps);
+          Map<String, CustomizedView> curCustomizedViewsCopy = new HashMap<>(curCustomizedViews);

Review comment:
       As you never use this variable in other places, just move it to line 120.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/CustomizedStateOutput.java
##########
@@ -24,32 +24,58 @@
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.helix.model.CustomizedState;
 import org.apache.helix.model.Partition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 public class CustomizedStateOutput {
+  private static final Logger LOG = LoggerFactory.getLogger(CustomizedStateOutput.class);
+
   // stateType -> (resourceName -> (Partition -> (instanceName -> customizedState)))
   private final Map<String, Map<String, Map<Partition, Map<String, String>>>> _customizedStateMap;
+  // stateType -> (resourceName -> (Partition -> (instanceName -> startTime)))
+  private final Map<String, Map<String, Map<Partition, Map<String, String>>>> _startTimeMap;
 
   public CustomizedStateOutput() {
     _customizedStateMap = new HashMap<>();
+    _startTimeMap = new HashMap<>();
   }
 
   public void setCustomizedState(String stateType, String resourceName, Partition partition,
       String instanceName, String state) {
-    if (!_customizedStateMap.containsKey(stateType)) {
-      _customizedStateMap
-          .put(stateType, new HashMap<String, Map<Partition, Map<String, String>>>());
-    }
-    if (!_customizedStateMap.get(stateType).containsKey(resourceName)) {
-      _customizedStateMap.get(stateType)
-          .put(resourceName, new HashMap<Partition, Map<String, String>>());
-    }
-    if (!_customizedStateMap.get(stateType).get(resourceName).containsKey(partition)) {
-      _customizedStateMap.get(stateType).get(resourceName)
-          .put(partition, new HashMap<String, String>());
+    setCustomizedStateProperty(CustomizedState.CustomizedStateProperty.CURRENT_STATE, stateType,
+        resourceName, partition, instanceName, state);
+  }
+
+  public void setStartTime(String stateType, String resourceName, Partition partition,
+      String instanceName, String state) {
+    setCustomizedStateProperty(CustomizedState.CustomizedStateProperty.START_TIME, stateType,
+        resourceName, partition, instanceName, state);
+  }
+
+  private void setCustomizedStateProperty(CustomizedState.CustomizedStateProperty propertyName,
+      String stateType, String resourceName, Partition partition, String instanceName,
+      String state) {
+    Map<String, Map<String, Map<Partition, Map<String, String>>>> mapToUpdate;
+    switch (propertyName) {
+      case CURRENT_STATE:
+        mapToUpdate = _customizedStateMap;
+        break;
+      case START_TIME:
+        mapToUpdate = _startTimeMap;
+        break;
+      default:
+        LOG.warn(
+            "The customized state property is not supported, could not update customized state output.");
+        return;
     }
-    _customizedStateMap.get(stateType).get(resourceName).get(partition).put(instanceName, state);
+
+    mapToUpdate.computeIfAbsent(stateType, k -> new HashMap<>())
+        .computeIfAbsent(resourceName, k -> new HashMap<>())
+        .computeIfAbsent(partition, k -> new HashMap<>())
+        .put(instanceName, state);

Review comment:
       Good change!

##########
File path: helix-core/src/test/java/org/apache/helix/controller/stages/TestCustomizedViewStage.java
##########
@@ -112,4 +110,60 @@ public void testCachedCustomizedViews() throws Exception {
     deleteLiveInstances(clusterName);
     deleteCluster(clusterName);
   }
+
+  @Test
+  public void testLatencyMetricReporting() throws Exception {
+    String clusterName = "CLUSTER_" + TestHelper.getTestMethodName();
+
+    HelixDataAccessor accessor =
+        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<>(_gZkClient));
+    HelixManager manager = new DummyClusterManager(clusterName, accessor);
+
+    // ideal state: node0 is MASTER, node1 is SLAVE
+    // replica=2 means 1 master and 1 slave
+    setupIdealState(clusterName, new int[]{0, 1}, new String[]{"TestDB"}, 1, 2);
+    setupLiveInstances(clusterName, new int[]{0, 1});
+    setupStateModel(clusterName);
+
+    ClusterEvent event = new ClusterEvent(clusterName, ClusterEventType.Unknown);
+    ResourceControllerDataProvider cache = new ResourceControllerDataProvider(clusterName);
+    event.addAttribute(AttributeName.helixmanager.name(), manager);
+    event.addAttribute(AttributeName.ControllerDataProvider.name(), cache);
+
+    CustomizedStateConfig config = new CustomizedStateConfig();
+    List<String> aggregationEnabledTypes = new ArrayList<>();
+    aggregationEnabledTypes.add(CUSTOMIZED_STATE_NAME);
+    config.setAggregationEnabledTypes(aggregationEnabledTypes);
+
+    PropertyKey.Builder keyBuilder = accessor.keyBuilder();
+    accessor.setProperty(keyBuilder.customizedStateConfig(), config);
+
+    CustomizedState customizedState = new CustomizedState(RESOURCE_NAME);
+    customizedState.setState(PARTITION_NAME, "STATE");
+    customizedState.setStartTime(PARTITION_NAME, 0);
+    accessor.setProperty(
+        keyBuilder.customizedState(INSTANCE_NAME, CUSTOMIZED_STATE_NAME, RESOURCE_NAME),
+        customizedState);
+
+    Pipeline dataRefresh = new Pipeline();
+    dataRefresh.addStage(new ReadClusterDataStage());
+    runPipeline(event, dataRefresh);
+    runStage(event, new ResourceComputationStage());
+    runStage(event, new CustomizedStateComputationStage());
+    runStage(event, new CustomizedViewAggregationStage());
+
+    ObjectName objectName = new ObjectName(String
+        .format("%s:%s=%s", MonitorDomainNames.CustomizedView.name(), "Cluster", clusterName));
+    ObjectInstance monitor = _server.getObjectInstance(objectName);
+    Assert.assertNotNull(monitor);
+    TestHelper.verify(() -> (long) _server.getAttribute(objectName,
+        CustomizedViewMonitor.UPDATE_TO_AGGREGATION_LATENCY_GAUGE + ".Max") == 0,

Review comment:
       Can you explain what is verified as 0?

##########
File path: helix-core/src/main/java/org/apache/helix/monitoring/mbeans/CustomizedViewMonitor.java
##########
@@ -0,0 +1,151 @@
+package org.apache.helix.monitoring.mbeans;
+
+/*
+ * 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.
+ */
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import javax.management.JMException;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.SlidingTimeWindowArrayReservoir;
+import org.apache.helix.model.CustomizedView;
+import org.apache.helix.model.Partition;
+import org.apache.helix.monitoring.mbeans.dynamicMBeans.DynamicMBeanProvider;
+import org.apache.helix.monitoring.mbeans.dynamicMBeans.DynamicMetric;
+import org.apache.helix.monitoring.mbeans.dynamicMBeans.HistogramDynamicMetric;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CustomizedViewMonitor extends DynamicMBeanProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(CustomizedViewMonitor.class);
+
+  private static final String MBEAN_DESCRIPTION = "Helix Customized View Aggregation Monitor";
+  private final String _clusterName;
+  private final String _sensorName;
+  private HistogramDynamicMetric _updateToAggregationLatencyGauge;
+  public static final String UPDATE_TO_AGGREGATION_LATENCY_GAUGE =
+      "UpdateToAggregationLatencyGauge";
+
+  public CustomizedViewMonitor(String clusterName) {
+    _clusterName = clusterName;
+    _sensorName = String.format("%s.%s", MonitorDomainNames.CustomizedView.name(), _clusterName);
+    _updateToAggregationLatencyGauge =
+        new HistogramDynamicMetric(UPDATE_TO_AGGREGATION_LATENCY_GAUGE, new Histogram(
+            new SlidingTimeWindowArrayReservoir(getResetIntervalInMs(), TimeUnit.MILLISECONDS)));
+  }
+
+  @Override
+  public DynamicMBeanProvider register() throws JMException {
+    List<DynamicMetric<?, ?>> attributeList = new ArrayList<>();
+    attributeList.add(_updateToAggregationLatencyGauge);
+    doRegister(attributeList, MBEAN_DESCRIPTION, getMBeanName());
+    return this;
+  }
+
+  private ObjectName getMBeanName() throws MalformedObjectNameException {
+    return new ObjectName(String
+        .format("%s:%s=%s", MonitorDomainNames.CustomizedView.name(), "Cluster", _clusterName));
+  }
+
+  @Override
+  public String getSensorName() {
+    return _sensorName;
+  }
+
+  void recordUpdateToAggregationLatency(long latency) {
+    if (_updateToAggregationLatencyGauge != null) {
+      _updateToAggregationLatencyGauge.updateValue(latency);
+    }
+  }
+
+  /**
+   * Find updated customized states and report the aggregation latency of each customized state
+   * @param updatedCustomizedViews Customized views that have been updated, obtained from CustomizedStateOutput
+   * @param curCustomizedViews Current customized view values from the CustomizedViewCache
+   * @param updatedStartTimestamps All customized state START_TIME property values from CustomizedStateOutput
+   * @param updateSuccess If the customized view update to ZK is successful or not
+   * @param endTime The timestamp when the new customized view is updated to ZK
+   */
+  public void reportLatency(List<CustomizedView> updatedCustomizedViews,
+      Map<String, CustomizedView> curCustomizedViews,
+      Map<String, Map<Partition, Map<String, String>>> updatedStartTimestamps,
+      boolean[] updateSuccess, long endTime) {
+    if (updatedCustomizedViews == null || curCustomizedViews == null
+        || updatedStartTimestamps == null) {
+      LOG.warn("Cannot find updated time stamps for customized states, input parameter is null.");
+      return;
+    }
+
+    List<Long> collectedTimestamps = new ArrayList<>();
+
+    for (int i = 0; i < updatedCustomizedViews.size(); i++) {
+      if (!updateSuccess[i]) {
+        continue;
+      }
+      CustomizedView newCV = updatedCustomizedViews.get(i);
+      String resourceName = newCV.getResourceName();
+      CustomizedView oldCV =
+          curCustomizedViews.getOrDefault(resourceName, new CustomizedView(resourceName));
+      if (newCV.getRecord().equals(oldCV.getRecord())) {

Review comment:
       Is this still necessary? The updated customized views should have been compared and filtered. 

##########
File path: helix-core/src/main/java/org/apache/helix/monitoring/mbeans/CustomizedViewMonitor.java
##########
@@ -0,0 +1,151 @@
+package org.apache.helix.monitoring.mbeans;
+
+/*
+ * 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.
+ */
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import javax.management.JMException;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.SlidingTimeWindowArrayReservoir;
+import org.apache.helix.model.CustomizedView;
+import org.apache.helix.model.Partition;
+import org.apache.helix.monitoring.mbeans.dynamicMBeans.DynamicMBeanProvider;
+import org.apache.helix.monitoring.mbeans.dynamicMBeans.DynamicMetric;
+import org.apache.helix.monitoring.mbeans.dynamicMBeans.HistogramDynamicMetric;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CustomizedViewMonitor extends DynamicMBeanProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(CustomizedViewMonitor.class);
+
+  private static final String MBEAN_DESCRIPTION = "Helix Customized View Aggregation Monitor";
+  private final String _clusterName;
+  private final String _sensorName;
+  private HistogramDynamicMetric _updateToAggregationLatencyGauge;
+  public static final String UPDATE_TO_AGGREGATION_LATENCY_GAUGE =
+      "UpdateToAggregationLatencyGauge";
+
+  public CustomizedViewMonitor(String clusterName) {
+    _clusterName = clusterName;
+    _sensorName = String.format("%s.%s", MonitorDomainNames.CustomizedView.name(), _clusterName);
+    _updateToAggregationLatencyGauge =
+        new HistogramDynamicMetric(UPDATE_TO_AGGREGATION_LATENCY_GAUGE, new Histogram(
+            new SlidingTimeWindowArrayReservoir(getResetIntervalInMs(), TimeUnit.MILLISECONDS)));
+  }
+
+  @Override
+  public DynamicMBeanProvider register() throws JMException {
+    List<DynamicMetric<?, ?>> attributeList = new ArrayList<>();
+    attributeList.add(_updateToAggregationLatencyGauge);
+    doRegister(attributeList, MBEAN_DESCRIPTION, getMBeanName());
+    return this;
+  }
+
+  private ObjectName getMBeanName() throws MalformedObjectNameException {
+    return new ObjectName(String
+        .format("%s:%s=%s", MonitorDomainNames.CustomizedView.name(), "Cluster", _clusterName));
+  }
+
+  @Override
+  public String getSensorName() {
+    return _sensorName;
+  }
+
+  void recordUpdateToAggregationLatency(long latency) {
+    if (_updateToAggregationLatencyGauge != null) {
+      _updateToAggregationLatencyGauge.updateValue(latency);
+    }
+  }
+
+  /**
+   * Find updated customized states and report the aggregation latency of each customized state
+   * @param updatedCustomizedViews Customized views that have been updated, obtained from CustomizedStateOutput
+   * @param curCustomizedViews Current customized view values from the CustomizedViewCache
+   * @param updatedStartTimestamps All customized state START_TIME property values from CustomizedStateOutput
+   * @param updateSuccess If the customized view update to ZK is successful or not
+   * @param endTime The timestamp when the new customized view is updated to ZK
+   */
+  public void reportLatency(List<CustomizedView> updatedCustomizedViews,
+      Map<String, CustomizedView> curCustomizedViews,
+      Map<String, Map<Partition, Map<String, String>>> updatedStartTimestamps,
+      boolean[] updateSuccess, long endTime) {
+    if (updatedCustomizedViews == null || curCustomizedViews == null
+        || updatedStartTimestamps == null) {
+      LOG.warn("Cannot find updated time stamps for customized states, input parameter is null.");
+      return;
+    }
+
+    List<Long> collectedTimestamps = new ArrayList<>();
+
+    for (int i = 0; i < updatedCustomizedViews.size(); i++) {
+      if (!updateSuccess[i]) {
+        continue;
+      }
+      CustomizedView newCV = updatedCustomizedViews.get(i);
+      String resourceName = newCV.getResourceName();
+      CustomizedView oldCV =
+          curCustomizedViews.getOrDefault(resourceName, new CustomizedView(resourceName));
+      if (newCV.getRecord().equals(oldCV.getRecord())) {
+        continue;
+      }
+
+      Map<String, Map<String, String>> newPartitionStateMaps = newCV.getRecord().getMapFields();
+      Map<String, Map<String, String>> oldPartitionStateMaps = oldCV.getRecord().getMapFields();
+      Map<Partition, Map<String, String>> partitionStartTimeMaps =
+          updatedStartTimestamps.getOrDefault(resourceName, Collections.emptyMap());
+
+      for (Map.Entry<String, Map<String, String>> partitionStateMapEntry : newPartitionStateMaps
+          .entrySet()) {
+        String partitionName = partitionStateMapEntry.getKey();
+        Map<String, String> newStateMap = partitionStateMapEntry.getValue();
+        Map<String, String> oldStateMap =
+            oldPartitionStateMaps.getOrDefault(partitionName, Collections.emptyMap());
+        if (!newStateMap.equals(oldStateMap)) {
+          Map<String, String> partitionStartTimeMap = partitionStartTimeMaps
+              .getOrDefault(new Partition(partitionName), Collections.emptyMap());
+
+          for (Map.Entry<String, String> stateMapEntry : newStateMap.entrySet()) {
+            String instanceName = stateMapEntry.getKey();
+            if (!stateMapEntry.getValue().equals(oldStateMap.get(instanceName))) {
+              try {
+                long timestamp = Long.parseLong(partitionStartTimeMap.get(instanceName));
+                if (timestamp > 0) {
+                  collectedTimestamps.add(timestamp);
+                } else {
+                  LOG.warn(
+                      "Failed to find customized state update time stamp for reos, the number should be positive.");
+                }
+              } catch (NumberFormatException e) {
+                LOG.warn("Error occurs while parsing customized state update time stamp");

Review comment:
       -> timestamp, one word.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/CustomizedViewAggregationStage.java
##########
@@ -107,8 +114,11 @@ public void execute(final ClusterEvent event) throws Exception {
           }
           // add/update customized-views from zk and cache
           if (updatedCustomizedViews.size() > 0) {
-            dataAccessor.setChildren(keys, updatedCustomizedViews);
+            boolean[] success = dataAccessor.setChildren(keys, updatedCustomizedViews);
             cache.updateCustomizedViews(stateType, updatedCustomizedViews);
+            asyncReportLatency(cache.getAsyncTasksThreadPool(), getOrCreateMonitor(event),
+                new ArrayList<>(updatedCustomizedViews), curCustomizedViewsCopy,

Review comment:
       Have you run the whole integration tests? I'm a bit concerned about the extra memory cost. Do you need to whole curCustomizedViewCopy? The ones that are corresponding to the updatedViews should be fine, right? as you only need those timestamps to compare.

##########
File path: helix-core/src/main/java/org/apache/helix/monitoring/mbeans/CustomizedViewMonitor.java
##########
@@ -0,0 +1,151 @@
+package org.apache.helix.monitoring.mbeans;
+
+/*
+ * 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.
+ */
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import javax.management.JMException;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.SlidingTimeWindowArrayReservoir;
+import org.apache.helix.model.CustomizedView;
+import org.apache.helix.model.Partition;
+import org.apache.helix.monitoring.mbeans.dynamicMBeans.DynamicMBeanProvider;
+import org.apache.helix.monitoring.mbeans.dynamicMBeans.DynamicMetric;
+import org.apache.helix.monitoring.mbeans.dynamicMBeans.HistogramDynamicMetric;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CustomizedViewMonitor extends DynamicMBeanProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(CustomizedViewMonitor.class);
+
+  private static final String MBEAN_DESCRIPTION = "Helix Customized View Aggregation Monitor";
+  private final String _clusterName;
+  private final String _sensorName;
+  private HistogramDynamicMetric _updateToAggregationLatencyGauge;
+  public static final String UPDATE_TO_AGGREGATION_LATENCY_GAUGE =
+      "UpdateToAggregationLatencyGauge";
+
+  public CustomizedViewMonitor(String clusterName) {
+    _clusterName = clusterName;
+    _sensorName = String.format("%s.%s", MonitorDomainNames.CustomizedView.name(), _clusterName);
+    _updateToAggregationLatencyGauge =
+        new HistogramDynamicMetric(UPDATE_TO_AGGREGATION_LATENCY_GAUGE, new Histogram(
+            new SlidingTimeWindowArrayReservoir(getResetIntervalInMs(), TimeUnit.MILLISECONDS)));
+  }
+
+  @Override
+  public DynamicMBeanProvider register() throws JMException {
+    List<DynamicMetric<?, ?>> attributeList = new ArrayList<>();
+    attributeList.add(_updateToAggregationLatencyGauge);
+    doRegister(attributeList, MBEAN_DESCRIPTION, getMBeanName());
+    return this;
+  }
+
+  private ObjectName getMBeanName() throws MalformedObjectNameException {
+    return new ObjectName(String
+        .format("%s:%s=%s", MonitorDomainNames.CustomizedView.name(), "Cluster", _clusterName));
+  }
+
+  @Override
+  public String getSensorName() {
+    return _sensorName;
+  }
+
+  void recordUpdateToAggregationLatency(long latency) {
+    if (_updateToAggregationLatencyGauge != null) {
+      _updateToAggregationLatencyGauge.updateValue(latency);
+    }
+  }
+
+  /**
+   * Find updated customized states and report the aggregation latency of each customized state
+   * @param updatedCustomizedViews Customized views that have been updated, obtained from CustomizedStateOutput
+   * @param curCustomizedViews Current customized view values from the CustomizedViewCache
+   * @param updatedStartTimestamps All customized state START_TIME property values from CustomizedStateOutput
+   * @param updateSuccess If the customized view update to ZK is successful or not
+   * @param endTime The timestamp when the new customized view is updated to ZK
+   */
+  public void reportLatency(List<CustomizedView> updatedCustomizedViews,
+      Map<String, CustomizedView> curCustomizedViews,
+      Map<String, Map<Partition, Map<String, String>>> updatedStartTimestamps,
+      boolean[] updateSuccess, long endTime) {
+    if (updatedCustomizedViews == null || curCustomizedViews == null
+        || updatedStartTimestamps == null) {
+      LOG.warn("Cannot find updated time stamps for customized states, input parameter is null.");
+      return;
+    }
+
+    List<Long> collectedTimestamps = new ArrayList<>();
+
+    for (int i = 0; i < updatedCustomizedViews.size(); i++) {
+      if (!updateSuccess[i]) {
+        continue;
+      }
+      CustomizedView newCV = updatedCustomizedViews.get(i);
+      String resourceName = newCV.getResourceName();
+      CustomizedView oldCV =
+          curCustomizedViews.getOrDefault(resourceName, new CustomizedView(resourceName));
+      if (newCV.getRecord().equals(oldCV.getRecord())) {
+        continue;
+      }
+
+      Map<String, Map<String, String>> newPartitionStateMaps = newCV.getRecord().getMapFields();
+      Map<String, Map<String, String>> oldPartitionStateMaps = oldCV.getRecord().getMapFields();
+      Map<Partition, Map<String, String>> partitionStartTimeMaps =
+          updatedStartTimestamps.getOrDefault(resourceName, Collections.emptyMap());
+
+      for (Map.Entry<String, Map<String, String>> partitionStateMapEntry : newPartitionStateMaps
+          .entrySet()) {
+        String partitionName = partitionStateMapEntry.getKey();
+        Map<String, String> newStateMap = partitionStateMapEntry.getValue();
+        Map<String, String> oldStateMap =
+            oldPartitionStateMaps.getOrDefault(partitionName, Collections.emptyMap());
+        if (!newStateMap.equals(oldStateMap)) {
+          Map<String, String> partitionStartTimeMap = partitionStartTimeMaps
+              .getOrDefault(new Partition(partitionName), Collections.emptyMap());
+
+          for (Map.Entry<String, String> stateMapEntry : newStateMap.entrySet()) {
+            String instanceName = stateMapEntry.getKey();
+            if (!stateMapEntry.getValue().equals(oldStateMap.get(instanceName))) {
+              try {
+                long timestamp = Long.parseLong(partitionStartTimeMap.get(instanceName));
+                if (timestamp > 0) {
+                  collectedTimestamps.add(timestamp);
+                } else {
+                  LOG.warn(
+                      "Failed to find customized state update time stamp for reos, the number should be positive.");

Review comment:
       typo?




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



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