You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by ji...@apache.org on 2020/04/23 19:27:52 UTC

[helix] 06/23: Add basic functionalities for RoutingTableProvider for CustomizedView (#814)

This is an automated email from the ASF dual-hosted git repository.

jiajunwang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/helix.git

commit 84d0fe72c0948500e8ee69a4466ee4fadd516e9e
Author: Ali Reza Zamani Zadeh Najari <an...@linkedin.com>
AuthorDate: Wed Feb 26 16:29:27 2020 -0800

    Add basic functionalities for RoutingTableProvider for CustomizedView (#814)
    
    This commit contains the basic functionalities for
    CustomizedView RoutingTableProvider.
    
    Here are the new added functionalities:
    1- CustomizedViewChangeListener
    2- Addition of CustomizedView to helix PropertyType and PropertyKey
    3- Implementation of CustomizedView cache.
    4- Registering CallbackHandler for CustomizedView.
---
 .../main/java/org/apache/helix/HelixConstants.java |   1 +
 .../main/java/org/apache/helix/HelixManager.java   |   7 +
 .../main/java/org/apache/helix/PropertyKey.java    |  30 +++++
 .../main/java/org/apache/helix/PropertyType.java   |   1 +
 .../listeners/CustomizedViewChangeListener.java}   |  37 +++---
 .../helix/common/caches/CustomizedViewCache.java   | 146 +++++++++++++++++++++
 .../helix/controller/stages/ClusterEventType.java  |   1 +
 .../apache/helix/manager/zk/CallbackHandler.java   |  13 ++
 .../apache/helix/manager/zk/ZKHelixManager.java    |  10 ++
 .../controller/stages/DummyClusterManager.java     |   8 ++
 .../manager/TestParticipantManager.java            |   2 +-
 .../java/org/apache/helix/mock/MockManager.java    |   8 ++
 .../helix/participant/MockZKHelixManager.java      |   8 ++
 13 files changed, 253 insertions(+), 19 deletions(-)

diff --git a/helix-core/src/main/java/org/apache/helix/HelixConstants.java b/helix-core/src/main/java/org/apache/helix/HelixConstants.java
index f84173b..b0783bf 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixConstants.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixConstants.java
@@ -35,6 +35,7 @@ public interface HelixConstants {
     CURRENT_STATE (PropertyType.CURRENTSTATES),
     MESSAGE (PropertyType.MESSAGES),
     EXTERNAL_VIEW (PropertyType.EXTERNALVIEW),
+    CUSTOMIZED_VIEW (PropertyType.CUSTOMIZEDVIEW),
     TARGET_EXTERNAL_VIEW (PropertyType.TARGETEXTERNALVIEW),
     CONTROLLER (PropertyType.CONTROLLER),
     MESSAGES_CONTROLLER (PropertyType.MESSAGES_CONTROLLER),
diff --git a/helix-core/src/main/java/org/apache/helix/HelixManager.java b/helix-core/src/main/java/org/apache/helix/HelixManager.java
index 2191153..2be5ce4 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixManager.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixManager.java
@@ -27,6 +27,7 @@ import org.apache.helix.api.listeners.ConfigChangeListener;
 import org.apache.helix.api.listeners.ControllerChangeListener;
 import org.apache.helix.api.listeners.CurrentStateChangeListener;
 import org.apache.helix.api.listeners.ExternalViewChangeListener;
+import org.apache.helix.api.listeners.CustomizedViewChangeListener;
 import org.apache.helix.api.listeners.IdealStateChangeListener;
 import org.apache.helix.api.listeners.InstanceConfigChangeListener;
 import org.apache.helix.api.listeners.LiveInstanceChangeListener;
@@ -220,6 +221,12 @@ public interface HelixManager {
   void addExternalViewChangeListener(ExternalViewChangeListener listener) throws Exception;
 
   /**
+   * @see CustomizedViewChangeListener#onCustomizedViewChange(List, NotificationContext)
+   * @param listener
+   */
+  void addCustomizedViewChangeListener(CustomizedViewChangeListener listener, String aggregationType) throws Exception;
+
+  /**
    * @see ExternalViewChangeListener#onExternalViewChange(List, NotificationContext)
    * @param listener
    */
diff --git a/helix-core/src/main/java/org/apache/helix/PropertyKey.java b/helix-core/src/main/java/org/apache/helix/PropertyKey.java
index 39f18c9..fc343a8 100644
--- a/helix-core/src/main/java/org/apache/helix/PropertyKey.java
+++ b/helix-core/src/main/java/org/apache/helix/PropertyKey.java
@@ -29,6 +29,7 @@ import org.apache.helix.model.ControllerHistory;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.CustomizedStateAggregationConfig;
 import org.apache.helix.model.CustomizedState;
+import org.apache.helix.model.CustomizedView;
 import org.apache.helix.model.Error;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.HealthStat;
@@ -58,6 +59,7 @@ import static org.apache.helix.PropertyType.CUSTOMIZEDSTATES;
 import static org.apache.helix.PropertyType.ERRORS;
 import static org.apache.helix.PropertyType.ERRORS_CONTROLLER;
 import static org.apache.helix.PropertyType.EXTERNALVIEW;
+import static org.apache.helix.PropertyType.CUSTOMIZEDVIEW;
 import static org.apache.helix.PropertyType.HISTORY;
 import static org.apache.helix.PropertyType.IDEALSTATES;
 import static org.apache.helix.PropertyType.INSTANCE_HISTORY;
@@ -73,6 +75,7 @@ import static org.apache.helix.PropertyType.STATUSUPDATES;
 import static org.apache.helix.PropertyType.STATUSUPDATES_CONTROLLER;
 import static org.apache.helix.PropertyType.TARGETEXTERNALVIEW;
 
+
 /**
  * Key allowing for type-safe lookups of and conversions to {@link HelixProperty} objects.
  */
@@ -639,6 +642,33 @@ public class PropertyKey {
     }
 
     /**
+     * Get a property key associated with all {@link CustomizedView}
+     * @return {@link PropertyKey}
+     */
+    public PropertyKey customizedView() {
+      return new PropertyKey(CUSTOMIZEDVIEW, CustomizedView.class, _clusterName);
+    }
+
+    /**
+     * Get a property key associated with an {@link CustomizedView} of a type
+     * @param aggregationType
+     * @return {@link PropertyKey}
+     */
+    public PropertyKey customizedView(String aggregationType) {
+      return new PropertyKey(CUSTOMIZEDVIEW, CustomizedView.class, _clusterName, aggregationType);
+    }
+
+    /**
+     * Get a property key associated with an {@link CustomizedView} of a type and resource
+     * @param aggregationType
+     * @return {@link PropertyKey}
+     */
+    public PropertyKey customizedView(String aggregationType, String resourceName) {
+      return new PropertyKey(CUSTOMIZEDVIEW, CustomizedView.class, _clusterName, aggregationType,
+          resourceName);
+    }
+
+    /**
      * Get a property key associated with all target external view
      * @return {@link PropertyKey}
      */
diff --git a/helix-core/src/main/java/org/apache/helix/PropertyType.java b/helix-core/src/main/java/org/apache/helix/PropertyType.java
index e076322..363db21 100644
--- a/helix-core/src/main/java/org/apache/helix/PropertyType.java
+++ b/helix-core/src/main/java/org/apache/helix/PropertyType.java
@@ -43,6 +43,7 @@ public enum PropertyType {
   INSTANCES(Type.CLUSTER, true, false),
   IDEALSTATES(Type.CLUSTER, true, false, false, false, true),
   EXTERNALVIEW(Type.CLUSTER, true, false),
+  CUSTOMIZEDVIEW(Type.CLUSTER, true, false),
   TARGETEXTERNALVIEW(Type.CLUSTER, true, false),
   STATEMODELDEFS(Type.CLUSTER, true, false, false, false, true),
   CONTROLLER(Type.CLUSTER, true, false),
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterEventType.java b/helix-core/src/main/java/org/apache/helix/api/listeners/CustomizedViewChangeListener.java
similarity index 59%
copy from helix-core/src/main/java/org/apache/helix/controller/stages/ClusterEventType.java
copy to helix-core/src/main/java/org/apache/helix/api/listeners/CustomizedViewChangeListener.java
index 7e9ab15..4cd4e7a 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterEventType.java
+++ b/helix-core/src/main/java/org/apache/helix/api/listeners/CustomizedViewChangeListener.java
@@ -1,4 +1,4 @@
-package org.apache.helix.controller.stages;
+package org.apache.helix.api.listeners;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -19,21 +19,22 @@ package org.apache.helix.controller.stages;
  * under the License.
  */
 
-public enum ClusterEventType {
-  IdealStateChange,
-  CurrentStateChange,
-  ConfigChange,
-  ClusterConfigChange,
-  ResourceConfigChange,
-  InstanceConfigChange,
-  LiveInstanceChange,
-  MessageChange,
-  ExternalViewChange,
-  TargetExternalViewChange,
-  Resume,
-  PeriodicalRebalance,
-  OnDemandRebalance,
-  RetryRebalance,
-  StateVerifier,
-  Unknown
+import java.util.List;
+
+import org.apache.helix.NotificationContext;
+import org.apache.helix.model.CustomizedView;
+
+/**
+ * Interface to implement to respond to changes in the CustomizedView
+ */
+public interface CustomizedViewChangeListener {
+
+  /**
+   * Invoked when customized view changes
+   * @param customizedViewList a list of CustomizedViews
+   * @param changeContext the change event and state
+   */
+  void onCustomizedViewChange(List<CustomizedView> customizedViewList,
+      NotificationContext changeContext);
+
 }
diff --git a/helix-core/src/main/java/org/apache/helix/common/caches/CustomizedViewCache.java b/helix-core/src/main/java/org/apache/helix/common/caches/CustomizedViewCache.java
new file mode 100644
index 0000000..493ca69
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/common/caches/CustomizedViewCache.java
@@ -0,0 +1,146 @@
+package org.apache.helix.common.caches;
+
+/*
+ * 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.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixException;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.PropertyType;
+import org.apache.helix.model.CustomizedView;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Maps;
+
+/**
+ * Cache to hold all CustomizedView of a specific type.
+ */
+public class CustomizedViewCache extends AbstractDataCache<CustomizedView> {
+  private static final Logger LOG = LoggerFactory.getLogger(CustomizedViewCache.class.getName());
+
+  protected Map<String, CustomizedView> _customizedViewMap;
+  protected Map<String, CustomizedView> _customizedViewCache;
+  protected String _clusterName;
+  private PropertyType _propertyType;
+  private String _aggregationType;
+
+  public CustomizedViewCache(String clusterName, String aggregationType) {
+    this(clusterName, PropertyType.CUSTOMIZEDVIEW, aggregationType);
+  }
+
+  protected CustomizedViewCache(String clusterName, PropertyType propertyType, String aggregationType) {
+    super(createDefaultControlContextProvider(clusterName));
+    _clusterName = clusterName;
+    _customizedViewMap = Collections.emptyMap();
+    _customizedViewCache = Collections.emptyMap();
+    _propertyType = propertyType;
+    _aggregationType = aggregationType;
+  }
+
+
+  /**
+   * This refreshes the CustomizedView data by re-fetching the data from zookeeper in an efficient
+   * way
+   * @param accessor
+   * @return
+   */
+  public void refresh(HelixDataAccessor accessor) {
+    long startTime = System.currentTimeMillis();
+    PropertyKey.Builder keyBuilder = accessor.keyBuilder();
+    Set<PropertyKey> currentPropertyKeys = new HashSet<>();
+
+    List<String> resources = accessor.getChildNames(customizedViewsKey(keyBuilder));
+
+    for (String resource : resources) {
+      currentPropertyKeys.add(customizedViewKey(keyBuilder, resource));
+    }
+
+    Set<PropertyKey> cachedKeys = new HashSet<>();
+    Map<PropertyKey, CustomizedView> cachedCustomizedViewMap = Maps.newHashMap();
+    for (String resource : _customizedViewCache.keySet()) {
+      PropertyKey key = customizedViewKey(keyBuilder, resource);
+      cachedKeys.add(key);
+      cachedCustomizedViewMap.put(key, _customizedViewCache.get(resource));
+    }
+    cachedKeys.retainAll(currentPropertyKeys);
+
+    Set<PropertyKey> reloadKeys = new HashSet<>(currentPropertyKeys);
+    reloadKeys.removeAll(cachedKeys);
+
+    Map<PropertyKey, CustomizedView> updatedMap =
+        refreshProperties(accessor, reloadKeys, new ArrayList<>(cachedKeys),
+            cachedCustomizedViewMap, new HashSet<>());
+
+    Map<String, CustomizedView> newCustomizedViewMap = Maps.newHashMap();
+    for (CustomizedView customizedView : updatedMap.values()) {
+      newCustomizedViewMap.put(customizedView.getResourceName(), customizedView);
+    }
+
+    _customizedViewCache = new HashMap<>(newCustomizedViewMap);
+    _customizedViewMap = new HashMap<>(newCustomizedViewMap);
+
+    long endTime = System.currentTimeMillis();
+    LOG.info("Refresh " + _customizedViewMap.size() + " CustomizedViews of type " + _aggregationType
+        + " for cluster " + _clusterName + ", took " + (endTime - startTime) + " ms");
+  }
+
+  private PropertyKey customizedViewsKey(PropertyKey.Builder keyBuilder) {
+    PropertyKey customizedViewPropertyKey;
+    if (_propertyType.equals(PropertyType.CUSTOMIZEDVIEW)){
+      customizedViewPropertyKey = keyBuilder.customizedView(_aggregationType);
+    } else {
+      throw new HelixException(
+          "Failed to refresh CustomizedViewCache, Wrong property type " + _propertyType + "!");
+    }
+    return customizedViewPropertyKey;
+  }
+
+  private PropertyKey customizedViewKey(PropertyKey.Builder keyBuilder, String resource) {
+    PropertyKey customizedViewPropertyKey;
+    if (_propertyType.equals(PropertyType.CUSTOMIZEDVIEW)) {
+      customizedViewPropertyKey = keyBuilder.customizedView(_aggregationType, resource);
+    } else {
+      throw new HelixException(
+          "Failed to refresh CustomizedViewCache, Wrong property type " + _propertyType + "!");
+    }
+    return customizedViewPropertyKey;
+  }
+
+  /**
+   * Return CustomizedView map for all resources.
+   * @return
+   */
+  public Map<String, CustomizedView> getCustomizedViewMap() {
+    return Collections.unmodifiableMap(_customizedViewMap);
+  }
+
+  public void clear() {
+    _customizedViewCache.clear();
+    _customizedViewMap.clear();
+  }
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterEventType.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterEventType.java
index 7e9ab15..2fd015e 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterEventType.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterEventType.java
@@ -29,6 +29,7 @@ public enum ClusterEventType {
   LiveInstanceChange,
   MessageChange,
   ExternalViewChange,
+  CustomizedViewChange,
   TargetExternalViewChange,
   Resume,
   PeriodicalRebalance,
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/CallbackHandler.java b/helix-core/src/main/java/org/apache/helix/manager/zk/CallbackHandler.java
index bbb8788..dc91028 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/CallbackHandler.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/CallbackHandler.java
@@ -45,6 +45,7 @@ import org.apache.helix.api.listeners.ClusterConfigChangeListener;
 import org.apache.helix.api.listeners.ConfigChangeListener;
 import org.apache.helix.api.listeners.ControllerChangeListener;
 import org.apache.helix.api.listeners.CurrentStateChangeListener;
+import org.apache.helix.api.listeners.CustomizedViewChangeListener;
 import org.apache.helix.api.listeners.ExternalViewChangeListener;
 import org.apache.helix.api.listeners.IdealStateChangeListener;
 import org.apache.helix.api.listeners.InstanceConfigChangeListener;
@@ -56,6 +57,7 @@ import org.apache.helix.api.listeners.ScopedConfigChangeListener;
 import org.apache.helix.common.DedupEventProcessor;
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.CurrentState;
+import org.apache.helix.model.CustomizedView;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
@@ -78,6 +80,7 @@ import static org.apache.helix.HelixConstants.ChangeType.CONFIG;
 import static org.apache.helix.HelixConstants.ChangeType.CONTROLLER;
 import static org.apache.helix.HelixConstants.ChangeType.CURRENT_STATE;
 import static org.apache.helix.HelixConstants.ChangeType.EXTERNAL_VIEW;
+import static org.apache.helix.HelixConstants.ChangeType.CUSTOMIZED_VIEW;
 import static org.apache.helix.HelixConstants.ChangeType.IDEAL_STATE;
 import static org.apache.helix.HelixConstants.ChangeType.INSTANCE_CONFIG;
 import static org.apache.helix.HelixConstants.ChangeType.LIVE_INSTANCE;
@@ -86,6 +89,7 @@ import static org.apache.helix.HelixConstants.ChangeType.MESSAGES_CONTROLLER;
 import static org.apache.helix.HelixConstants.ChangeType.RESOURCE_CONFIG;
 import static org.apache.helix.HelixConstants.ChangeType.TARGET_EXTERNAL_VIEW;
 
+
 @PreFetch(enabled = false)
 public class CallbackHandler implements IZkChildListener, IZkDataListener {
   private static Logger logger = LoggerFactory.getLogger(CallbackHandler.class);
@@ -290,6 +294,9 @@ public class CallbackHandler implements IZkChildListener, IZkDataListener {
     case TARGET_EXTERNAL_VIEW:
       listenerClass = ExternalViewChangeListener.class;
       break;
+    case CUSTOMIZED_VIEW:
+      listenerClass = CustomizedViewChangeListener.class;
+      break;
     case CONTROLLER:
       listenerClass = ControllerChangeListener.class;
     }
@@ -434,6 +441,11 @@ public class CallbackHandler implements IZkChildListener, IZkDataListener {
         List<ExternalView> externalViewList = preFetch(_propertyKey);
         externalViewListener.onExternalViewChange(externalViewList, changeContext);
 
+      } else if (_changeType == CUSTOMIZED_VIEW) {
+        CustomizedViewChangeListener customizedViewListener = (CustomizedViewChangeListener) _listener;
+        List<CustomizedView> customizedViewListList = preFetch(_propertyKey);
+        customizedViewListener.onCustomizedViewChange(customizedViewListList, changeContext);
+
       } else if (_changeType == CONTROLLER) {
         ControllerChangeListener controllerChangelistener = (ControllerChangeListener) _listener;
         controllerChangelistener.onControllerChange(changeContext);
@@ -524,6 +536,7 @@ public class CallbackHandler implements IZkChildListener, IZkDataListener {
           case CURRENT_STATE:
           case IDEAL_STATE:
           case EXTERNAL_VIEW:
+          case CUSTOMIZED_VIEW:
           case TARGET_EXTERNAL_VIEW: {
             // check if bucketized
             BaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<>(_zkClient);
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java
index 72ce07f..c06ddf1 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java
@@ -57,6 +57,7 @@ import org.apache.helix.api.listeners.ClusterConfigChangeListener;
 import org.apache.helix.api.listeners.ConfigChangeListener;
 import org.apache.helix.api.listeners.ControllerChangeListener;
 import org.apache.helix.api.listeners.CurrentStateChangeListener;
+import org.apache.helix.api.listeners.CustomizedViewChangeListener;
 import org.apache.helix.api.listeners.ExternalViewChangeListener;
 import org.apache.helix.api.listeners.IdealStateChangeListener;
 import org.apache.helix.api.listeners.InstanceConfigChangeListener;
@@ -581,6 +582,15 @@ public class ZKHelixManager implements HelixManager, IZkStateListener {
   }
 
   @Override
+  public void addCustomizedViewChangeListener(CustomizedViewChangeListener listener, String aggregationType)
+      throws Exception {
+    addListener(listener, new Builder(_clusterName).customizedView(aggregationType),
+        ChangeType.CUSTOMIZED_VIEW, new EventType[] {
+            EventType.NodeChildrenChanged
+        });
+  }
+
+  @Override
   public void addTargetExternalViewChangeListener(ExternalViewChangeListener listener) throws Exception {
     addListener(listener, new Builder(_clusterName).externalViews(), ChangeType.TARGET_EXTERNAL_VIEW,
         new EventType[] { EventType.NodeChildrenChanged });
diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/DummyClusterManager.java b/helix-core/src/test/java/org/apache/helix/controller/stages/DummyClusterManager.java
index 05631c8..076f115 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/DummyClusterManager.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/stages/DummyClusterManager.java
@@ -19,6 +19,7 @@ package org.apache.helix.controller.stages;
  * under the License.
  */
 
+import java.util.List;
 import java.util.Set;
 
 import org.apache.helix.ClusterMessagingService;
@@ -37,6 +38,7 @@ import org.apache.helix.api.listeners.ConfigChangeListener;
 import org.apache.helix.api.listeners.ControllerChangeListener;
 import org.apache.helix.api.listeners.CurrentStateChangeListener;
 import org.apache.helix.api.listeners.ExternalViewChangeListener;
+import org.apache.helix.api.listeners.CustomizedViewChangeListener;
 import org.apache.helix.api.listeners.IdealStateChangeListener;
 import org.apache.helix.api.listeners.InstanceConfigChangeListener;
 import org.apache.helix.api.listeners.LiveInstanceChangeListener;
@@ -136,6 +138,12 @@ public class DummyClusterManager implements HelixManager {
   }
 
   @Override
+  public void addCustomizedViewChangeListener(CustomizedViewChangeListener listener, String aggregationType) throws Exception {
+    // TODO Auto-generated method stub
+
+  }
+
+  @Override
   public void addTargetExternalViewChangeListener(ExternalViewChangeListener listener) throws Exception {
     // TODO Auto-generated method stub
   }
diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/TestParticipantManager.java b/helix-core/src/test/java/org/apache/helix/integration/manager/TestParticipantManager.java
index aef7ff6..6e972c5 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/manager/TestParticipantManager.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/manager/TestParticipantManager.java
@@ -151,7 +151,7 @@ public class TestParticipantManager extends ZkTestBase {
     // check HelixCallback Monitor
     Set<ObjectInstance> objs =
         _server.queryMBeans(buildCallbackMonitorObjectName(type, clusterName, instanceName), null);
-    Assert.assertEquals(objs.size(), 13);
+    Assert.assertEquals(objs.size(), 14);
 
     // check HelixZkClient Monitors
     objs =
diff --git a/helix-core/src/test/java/org/apache/helix/mock/MockManager.java b/helix-core/src/test/java/org/apache/helix/mock/MockManager.java
index dfb90ec..01e6b9e 100644
--- a/helix-core/src/test/java/org/apache/helix/mock/MockManager.java
+++ b/helix-core/src/test/java/org/apache/helix/mock/MockManager.java
@@ -19,6 +19,7 @@ package org.apache.helix.mock;
  * under the License.
  */
 
+import java.util.List;
 import java.util.Set;
 import java.util.UUID;
 
@@ -38,6 +39,7 @@ import org.apache.helix.api.listeners.ClusterConfigChangeListener;
 import org.apache.helix.api.listeners.ConfigChangeListener;
 import org.apache.helix.api.listeners.ControllerChangeListener;
 import org.apache.helix.api.listeners.CurrentStateChangeListener;
+import org.apache.helix.api.listeners.CustomizedViewChangeListener;
 import org.apache.helix.api.listeners.ExternalViewChangeListener;
 import org.apache.helix.api.listeners.IdealStateChangeListener;
 import org.apache.helix.api.listeners.InstanceConfigChangeListener;
@@ -140,6 +142,12 @@ public class MockManager implements HelixManager {
   }
 
   @Override
+  public void addCustomizedViewChangeListener(CustomizedViewChangeListener listener, String aggregationType) {
+    // TODO Auto-generated method stub
+
+  }
+
+  @Override
   public void addTargetExternalViewChangeListener(ExternalViewChangeListener listener) throws Exception {
     // TODO Auto-generated method stub
   }
diff --git a/helix-core/src/test/java/org/apache/helix/participant/MockZKHelixManager.java b/helix-core/src/test/java/org/apache/helix/participant/MockZKHelixManager.java
index 85b376f..67efbd5 100644
--- a/helix-core/src/test/java/org/apache/helix/participant/MockZKHelixManager.java
+++ b/helix-core/src/test/java/org/apache/helix/participant/MockZKHelixManager.java
@@ -19,6 +19,7 @@ package org.apache.helix.participant;
  * under the License.
  */
 
+import java.util.List;
 import java.util.Set;
 import java.util.UUID;
 
@@ -37,6 +38,7 @@ import org.apache.helix.api.listeners.ClusterConfigChangeListener;
 import org.apache.helix.api.listeners.ConfigChangeListener;
 import org.apache.helix.api.listeners.ControllerChangeListener;
 import org.apache.helix.api.listeners.CurrentStateChangeListener;
+import org.apache.helix.api.listeners.CustomizedViewChangeListener;
 import org.apache.helix.api.listeners.ExternalViewChangeListener;
 import org.apache.helix.api.listeners.IdealStateChangeListener;
 import org.apache.helix.api.listeners.InstanceConfigChangeListener;
@@ -165,6 +167,12 @@ public class MockZKHelixManager implements HelixManager {
   }
 
   @Override
+  public void addCustomizedViewChangeListener(CustomizedViewChangeListener listener, String aggregationType) throws Exception {
+    // TODO Auto-generated method stub
+
+  }
+
+  @Override
   public void addTargetExternalViewChangeListener(ExternalViewChangeListener listener) throws Exception {
     // TODO Auto-generated method stub
   }