You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by zz...@apache.org on 2013/02/02 02:49:21 UTC

git commit: HELIX-39: add apis for registering listeners to configs of all scopes (instance, cluster, and resource)

Updated Branches:
  refs/heads/master 3b473667d -> c52baf615


HELIX-39: add apis for registering listeners to configs of all scopes (instance, cluster, and resource)


Project: http://git-wip-us.apache.org/repos/asf/incubator-helix/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-helix/commit/c52baf61
Tree: http://git-wip-us.apache.org/repos/asf/incubator-helix/tree/c52baf61
Diff: http://git-wip-us.apache.org/repos/asf/incubator-helix/diff/c52baf61

Branch: refs/heads/master
Commit: c52baf615207f75c03bb9e92dc3d8b5c9cc14a7d
Parents: 3b47366
Author: zzhang <zz...@uci.edu>
Authored: Fri Feb 1 17:49:11 2013 -0800
Committer: zzhang <zz...@uci.edu>
Committed: Fri Feb 1 17:49:11 2013 -0800

----------------------------------------------------------------------
 .../org/apache/helix/ConfigChangeListener.java     |    1 +
 .../main/java/org/apache/helix/HelixConstants.java |    1 +
 .../main/java/org/apache/helix/HelixManager.java   |   17 ++
 .../main/java/org/apache/helix/HelixProperty.java  |    2 +-
 .../apache/helix/InstanceConfigChangeListener.java |   35 +++
 .../main/java/org/apache/helix/PropertyKey.java    |   99 ++++----
 .../java/org/apache/helix/PropertyPathConfig.java  |    8 +-
 .../apache/helix/ScopedConfigChangeListener.java   |   33 +++
 .../apache/helix/manager/zk/CallbackHandler.java   |   59 +++--
 .../helix/manager/zk/ZKHelixDataAccessor.java      |   11 -
 .../apache/helix/manager/zk/ZKHelixManager.java    |  196 ++++++---------
 .../src/test/java/org/apache/helix/Mocks.java      |   15 ++
 .../org/apache/helix/TestListenerCallback.java     |  133 ++++++++++
 .../controller/stages/DummyClusterManager.java     |   16 ++
 .../helix/participant/MockZKHelixManager.java      |   16 ++
 15 files changed, 442 insertions(+), 200 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c52baf61/helix-core/src/main/java/org/apache/helix/ConfigChangeListener.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/ConfigChangeListener.java b/helix-core/src/main/java/org/apache/helix/ConfigChangeListener.java
index 54ea574..b9083ab 100644
--- a/helix-core/src/main/java/org/apache/helix/ConfigChangeListener.java
+++ b/helix-core/src/main/java/org/apache/helix/ConfigChangeListener.java
@@ -26,6 +26,7 @@ import org.apache.helix.model.InstanceConfig;
 
 /**
  * 
+ * @deprecated replaced by InstanceConfigChangeListener
  */
 public interface ConfigChangeListener
 {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c52baf61/helix-core/src/main/java/org/apache/helix/HelixConstants.java
----------------------------------------------------------------------
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 186b2e7..938b9bf 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixConstants.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixConstants.java
@@ -27,6 +27,7 @@ public interface HelixConstants
     // @formatter:off
     IDEAL_STATE,
     CONFIG,
+    INSTANCE_CONFIG,
     LIVE_INSTANCE,
     CURRENT_STATE,
     MESSAGE,

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c52baf61/helix-core/src/main/java/org/apache/helix/HelixManager.java
----------------------------------------------------------------------
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 65aa3cb..4d3adc8 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixManager.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixManager.java
@@ -21,6 +21,7 @@ package org.apache.helix;
 
 import java.util.List;
 
+import org.apache.helix.ConfigScope.ConfigScopeProperty;
 import org.apache.helix.controller.GenericHelixController;
 import org.apache.helix.healthcheck.ParticipantHealthReportCollector;
 import org.apache.helix.participant.HelixStateMachineEngine;
@@ -100,10 +101,26 @@ public interface HelixManager
   /**
    * @see ConfigChangeListener#onConfigChange(List, NotificationContext)
    * @param listener
+   * 
+   * @deprecated replaced by addInstanceConfigChangeListener()
    */
   void addConfigChangeListener(ConfigChangeListener listener) throws Exception;
 
   /**
+   * @see InstanceConfigChangeListener#onInstanceConfigChange(List, NotificationContext)
+   * @param listener
+   * 
+   */
+  void addInstanceConfigChangeListener(InstanceConfigChangeListener listener) throws Exception;
+
+  /**
+   * @see ScopedConfigChangeListener#onConfigChange(List, NotificationContext)
+   * @param listener
+   * @param scope
+   */
+  void addConfigChangeListener(ScopedConfigChangeListener listener, ConfigScopeProperty scope) throws Exception;
+  
+  /**
    * @see MessageListener#onMessage(String, List, NotificationContext)
    * @param listener
    * @param instanceName

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c52baf61/helix-core/src/main/java/org/apache/helix/HelixProperty.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/HelixProperty.java b/helix-core/src/main/java/org/apache/helix/HelixProperty.java
index 74d5e3a..080ee58 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixProperty.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixProperty.java
@@ -205,7 +205,7 @@ public class HelixProperty
   
   public boolean isValid()
   {
-    return false;
+    return true;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c52baf61/helix-core/src/main/java/org/apache/helix/InstanceConfigChangeListener.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/InstanceConfigChangeListener.java b/helix-core/src/main/java/org/apache/helix/InstanceConfigChangeListener.java
new file mode 100644
index 0000000..5d55be1
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/InstanceConfigChangeListener.java
@@ -0,0 +1,35 @@
+package org.apache.helix;
+
+/*
+ * 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.List;
+
+import org.apache.helix.model.InstanceConfig;
+
+public interface InstanceConfigChangeListener 
+{
+  /**
+   * Invoked when participant config changes
+   * 
+   * @param configs
+   * @param changeContext
+   */
+  public void onInstanceConfigChange(List<InstanceConfig> instanceConfigs, NotificationContext context);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c52baf61/helix-core/src/main/java/org/apache/helix/PropertyKey.java
----------------------------------------------------------------------
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 68fd0b6..fa26915 100644
--- a/helix-core/src/main/java/org/apache/helix/PropertyKey.java
+++ b/helix-core/src/main/java/org/apache/helix/PropertyKey.java
@@ -75,7 +75,17 @@ public class PropertyKey
   private final String[]         _params;
   Class<? extends HelixProperty> _typeClazz;
 
+  //if type is CONFIGS, set configScope; otherwise null
+  ConfigScopeProperty _configScope;
+
+  public PropertyKey(PropertyType type, Class<? extends HelixProperty> typeClazz, 
+		  String... params)
+  {
+    this(type, null, typeClazz, params);
+  }
+  
   public PropertyKey(PropertyType type,
+		  			 ConfigScopeProperty configScope,
                      Class<? extends HelixProperty> typeClazz,
                      String... params)
   {
@@ -87,6 +97,8 @@ public class PropertyKey
 
     _params = params;
     _typeClazz = typeClazz;
+    
+    _configScope = configScope;
   }
 
   @Override
@@ -140,45 +152,70 @@ public class PropertyKey
                              stateModelName);
     }
 
-    public PropertyKey clusterConfig()
+    public PropertyKey clusterConfigs()
     {
       return new PropertyKey(CONFIGS,
-                             null,
+    		  				 ConfigScopeProperty.CLUSTER,
+                             HelixProperty.class,
                              _clusterName,
                              ConfigScopeProperty.CLUSTER.toString());
     }
+    
+    public PropertyKey clusterConfig()
+    {
+    	return new PropertyKey(CONFIGS,
+ 				 ConfigScopeProperty.CLUSTER,
+                HelixProperty.class,
+                _clusterName,
+                ConfigScopeProperty.CLUSTER.toString(),
+                _clusterName);    	
+    }
 
     public PropertyKey instanceConfigs()
     {
-      return new PropertyKey(CONFIGS,
-                             InstanceConfig.class,
-                             _clusterName,
-                             ConfigScopeProperty.PARTICIPANT.toString());
+        return new PropertyKey(CONFIGS,
+ 				 ConfigScopeProperty.PARTICIPANT,
+                InstanceConfig.class,
+                _clusterName,
+                ConfigScopeProperty.PARTICIPANT.toString());        
     }
 
     public PropertyKey instanceConfig(String instanceName)
     {
       return new PropertyKey(CONFIGS,
+    		  	ConfigScopeProperty.PARTICIPANT,
                              InstanceConfig.class,
                              _clusterName,
                              ConfigScopeProperty.PARTICIPANT.toString(),
                              instanceName);
     }
 
+    public PropertyKey resourceConfigs()
+    {
+      return new PropertyKey(CONFIGS,
+				 			 ConfigScopeProperty.RESOURCE,
+    		  				 HelixProperty.class,
+                             _clusterName,
+                             ConfigScopeProperty.RESOURCE.toString());
+    }
+    
     public PropertyKey resourceConfig(String resourceName)
     {
       return new PropertyKey(CONFIGS,
-                             null,
+    		  				ConfigScopeProperty.RESOURCE,
+    		  				HelixProperty.class,
                              _clusterName,
                              ConfigScopeProperty.RESOURCE.toString(),
                              resourceName);
     }
 
+    
     public PropertyKey resourceConfig(String instanceName, String resourceName)
     {
       return new PropertyKey(CONFIGS,
-                             null,
-                             _clusterName,
+    		  				 ConfigScopeProperty.RESOURCE,
+    		  				 HelixProperty.class,
+    		  				 _clusterName,
                              ConfigScopeProperty.RESOURCE.toString(),
                              resourceName);
     }
@@ -186,7 +223,8 @@ public class PropertyKey
     public PropertyKey partitionConfig(String resourceName, String partitionName)
     {
       return new PropertyKey(CONFIGS,
-                             null,
+    		  				ConfigScopeProperty.RESOURCE,
+    		  				HelixProperty.class,
                              _clusterName,
                              ConfigScopeProperty.RESOURCE.toString(),
                              resourceName);
@@ -197,8 +235,9 @@ public class PropertyKey
                                        String partitionName)
     {
       return new PropertyKey(CONFIGS,
-                             null,
-                             _clusterName,
+    		  				ConfigScopeProperty.RESOURCE,
+				 			 HelixProperty.class,
+				 			 _clusterName,
                              ConfigScopeProperty.RESOURCE.toString(),
                              resourceName);
     }
@@ -308,14 +347,6 @@ public class PropertyKey
       }
     }
 
-    // addEntry(PropertyType.STATUSUPDATES, 2,
-    // "/{clusterName}/INSTANCES/{instanceName}/STATUSUPDATES");
-    // addEntry(PropertyType.STATUSUPDATES, 3,
-    // "/{clusterName}/INSTANCES/{instanceName}/STATUSUPDATES/{sessionId}");
-    // addEntry(PropertyType.STATUSUPDATES, 4,
-    // "/{clusterName}/INSTANCES/{instanceName}/STATUSUPDATES/{sessionId}/{subPath}");
-    // addEntry(PropertyType.STATUSUPDATES, 5,
-    // "/{clusterName}/INSTANCES/{instanceName}/STATUSUPDATES/{sessionId}/{subPath}/{recordName}");
     public PropertyKey stateTransitionStatus(String instanceName,
                                              String sessionId,
                                              String resourceName,
@@ -433,22 +464,6 @@ public class PropertyKey
       return new PropertyKey(EXTERNALVIEW, ExternalView.class, _clusterName, resourceName);
     }
 
-    // * addEntry(PropertyType.STATUSUPDATES_CONTROLLER, 4,
-    // *
-    // "/{clusterName}/CONTROLLER/STATUSUPDATES/{sessionId}/{subPath}/{recordName}"
-    // * ); addEntry(PropertyType.LEADER, 1,
-    // "/{clusterName}/CONTROLLER/LEADER");
-    // * addEntry(PropertyType.HISTORY, 1, "/{clusterName}/CONTROLLER/HISTORY");
-    // * addEntry(PropertyType.PAUSE, 1, "/{clusterName}/CONTROLLER/PAUSE");
-    // * addEntry(PropertyType.PERSISTENTSTATS, 1,
-    // * "/{clusterName}/CONTROLLER/PERSISTENTSTATS");
-    // addEntry(PropertyType.ALERTS,
-    // * 1, "/{clusterName}/CONTROLLER/ALERTS");
-    // addEntry(PropertyType.ALERT_STATUS,
-    // * 1, "/{clusterName}/CONTROLLER/ALERT_STATUS");
-    // * addEntry(PropertyType.ALERT_HISTORY, 1,
-    // * "/{clusterName}/CONTROLLER/ALERT_HISTORY"); // @formatter:on
-
     public PropertyKey controller()
     {
       return new PropertyKey(CONTROLLER, null, _clusterName);
@@ -557,16 +572,8 @@ public class PropertyKey
     return _typeClazz;
   }
 
-  public static void main(String[] args)
+  public ConfigScopeProperty getConfigScope()
   {
-    ZkClient zkClient = new ZkClient("localhost:2181");
-    zkClient.waitUntilConnected(10, TimeUnit.SECONDS);
-    BaseDataAccessor baseDataAccessor = new ZkBaseDataAccessor(zkClient);
-    HelixDataAccessor accessor =
-        new ZKHelixDataAccessor("test-cluster", baseDataAccessor);
-    Builder builder = new PropertyKey.Builder("test-cluster");
-    HelixProperty value = new IdealState("test-resource");
-    accessor.createProperty(builder.idealStates("test-resource"), value);
+	  return _configScope;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c52baf61/helix-core/src/main/java/org/apache/helix/PropertyPathConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/PropertyPathConfig.java b/helix-core/src/main/java/org/apache/helix/PropertyPathConfig.java
index 11c0807..d2f9650 100644
--- a/helix-core/src/main/java/org/apache/helix/PropertyPathConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/PropertyPathConfig.java
@@ -59,8 +59,8 @@ public class PropertyPathConfig
 {
   private static Logger logger = Logger.getLogger(PropertyPathConfig.class);
 
-  static Map<PropertyType, Map<Integer, String>> templateMap = new HashMap<PropertyType, Map<Integer, String>>();
-  static Map<PropertyType, Class<? extends HelixProperty>> typeToClassMapping= new HashMap<PropertyType, Class<? extends HelixProperty>>();
+  static final Map<PropertyType, Map<Integer, String>> templateMap = new HashMap<PropertyType, Map<Integer, String>>();
+  static final Map<PropertyType, Class<? extends HelixProperty>> typeToClassMapping= new HashMap<PropertyType, Class<? extends HelixProperty>>();
   static{
     typeToClassMapping.put(LIVEINSTANCES, LiveInstance.class);
     typeToClassMapping.put(IDEALSTATES, IdealState.class);
@@ -75,9 +75,7 @@ public class PropertyPathConfig
     typeToClassMapping.put(ALERTS, Alerts.class);
     typeToClassMapping.put(ALERT_STATUS, AlertStatus.class);
     typeToClassMapping.put(PAUSE, PauseSignal.class);
-  }
-  static
-  {
+
     // @formatter:off
     addEntry(PropertyType.CONFIGS, 1, "/{clusterName}/CONFIGS");
     addEntry(PropertyType.CONFIGS, 2, "/{clusterName}/CONFIGS/{scope}");

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c52baf61/helix-core/src/main/java/org/apache/helix/ScopedConfigChangeListener.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/ScopedConfigChangeListener.java b/helix-core/src/main/java/org/apache/helix/ScopedConfigChangeListener.java
new file mode 100644
index 0000000..e690d38
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/ScopedConfigChangeListener.java
@@ -0,0 +1,33 @@
+package org.apache.helix;
+
+/*
+ * 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.List;
+
+public interface ScopedConfigChangeListener 
+{
+    /**
+     * Invoked when configs of a scope (cluster, participant, or resource) change
+     * 
+     * @param configs
+     * @param context
+     */
+    public void onConfigChange(List<HelixProperty> configs, NotificationContext context);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c52baf61/helix-core/src/main/java/org/apache/helix/manager/zk/CallbackHandler.java
----------------------------------------------------------------------
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 0cc72b5..4e08ab0 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
@@ -44,11 +44,14 @@ import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixProperty;
 import org.apache.helix.IdealStateChangeListener;
+import org.apache.helix.InstanceConfigChangeListener;
 import org.apache.helix.LiveInstanceChangeListener;
 import org.apache.helix.MessageListener;
 import org.apache.helix.NotificationContext;
+import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.PropertyPathConfig;
+import org.apache.helix.ScopedConfigChangeListener;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.ExternalView;
@@ -74,10 +77,11 @@ public class CallbackHandler implements IZkChildListener, IZkDataListener
   private final ZkClient          _zkClient;
   private final AtomicLong        _lastNotificationTimeStamp;
   private final HelixManager      _manager;
+  private final PropertyKey 	  _propertyKey;
 
   public CallbackHandler(HelixManager manager,
                          ZkClient client,
-                         String path,
+                         PropertyKey propertyKey,
                          Object listener,
                          EventType[] eventTypes,
                          ChangeType changeType)
@@ -85,7 +89,8 @@ public class CallbackHandler implements IZkChildListener, IZkDataListener
     this._manager = manager;
     this._accessor = manager.getHelixDataAccessor();
     this._zkClient = client;
-    this._path = path;
+    this._propertyKey = propertyKey;
+    this._path = propertyKey.getPath();
     this._listener = listener;
     this._eventTypes = eventTypes;
     this._changeType = changeType;
@@ -108,7 +113,7 @@ public class CallbackHandler implements IZkChildListener, IZkDataListener
     // This allows the listener to work with one change at a time
     synchronized (_manager)
     {
-      Builder keyBuilder = _accessor.keyBuilder();
+      // Builder keyBuilder = _accessor.keyBuilder();
       long start = System.currentTimeMillis();
       if (logger.isInfoEnabled())
       {
@@ -122,21 +127,32 @@ public class CallbackHandler implements IZkChildListener, IZkDataListener
         IdealStateChangeListener idealStateChangeListener =
             (IdealStateChangeListener) _listener;
         subscribeForChanges(changeContext, _path, true, true);
-        List<IdealState> idealStates = _accessor.getChildValues(keyBuilder.idealStates());
+        List<IdealState> idealStates = _accessor.getChildValues(_propertyKey);
 
         idealStateChangeListener.onIdealStateChange(idealStates, changeContext);
 
       }
-      else if (_changeType == CONFIG)
+      else if (_changeType == ChangeType.INSTANCE_CONFIG)
       {
-
-        ConfigChangeListener configChangeListener = (ConfigChangeListener) _listener;
         subscribeForChanges(changeContext, _path, true, true);
-        List<InstanceConfig> configs =
-            _accessor.getChildValues(keyBuilder.instanceConfigs());
-
-        configChangeListener.onConfigChange(configs, changeContext);
-
+      	if (_listener instanceof ConfigChangeListener)
+      	{
+      		ConfigChangeListener configChangeListener = (ConfigChangeListener) _listener;
+      		List<InstanceConfig> configs = _accessor.getChildValues(_propertyKey);
+      		configChangeListener.onConfigChange(configs, changeContext);
+      	} else if (_listener instanceof InstanceConfigChangeListener)
+      	{
+      		InstanceConfigChangeListener listener = (InstanceConfigChangeListener) _listener;
+      		List<InstanceConfig> configs = _accessor.getChildValues(_propertyKey);
+      		listener.onInstanceConfigChange(configs, changeContext);    		
+      	}	  
+      }
+      else if (_changeType == CONFIG)
+      {
+            subscribeForChanges(changeContext, _path, true, true);
+      		ScopedConfigChangeListener listener = (ScopedConfigChangeListener) _listener;
+      		List<HelixProperty> configs = _accessor.getChildValues(_propertyKey);
+      		listener.onConfigChange(configs, changeContext);
       }
       else if (_changeType == LIVE_INSTANCE)
       {
@@ -144,23 +160,18 @@ public class CallbackHandler implements IZkChildListener, IZkDataListener
             (LiveInstanceChangeListener) _listener;
         subscribeForChanges(changeContext, _path, true, true);
         List<LiveInstance> liveInstances =
-            _accessor.getChildValues(keyBuilder.liveInstances());
+            _accessor.getChildValues(_propertyKey);
 
         liveInstanceChangeListener.onLiveInstanceChange(liveInstances, changeContext);
 
       }
       else if (_changeType == CURRENT_STATE)
       {
-        CurrentStateChangeListener currentStateChangeListener;
-        currentStateChangeListener = (CurrentStateChangeListener) _listener;
+        CurrentStateChangeListener currentStateChangeListener = (CurrentStateChangeListener) _listener;
         subscribeForChanges(changeContext, _path, true, true);
         String instanceName = PropertyPathConfig.getInstanceNameFromPath(_path);
-        String[] pathParts = _path.split("/");
 
-        // TODO: fix this
-        List<CurrentState> currentStates =
-            _accessor.getChildValues(keyBuilder.currentStates(instanceName,
-                                                              pathParts[pathParts.length - 1]));
+        List<CurrentState> currentStates = _accessor.getChildValues(_propertyKey);
 
         currentStateChangeListener.onStateChange(instanceName,
                                                  currentStates,
@@ -173,7 +184,7 @@ public class CallbackHandler implements IZkChildListener, IZkDataListener
         subscribeForChanges(changeContext, _path, true, false);
         String instanceName = PropertyPathConfig.getInstanceNameFromPath(_path);
         List<Message> messages =
-            _accessor.getChildValues(keyBuilder.messages(instanceName));
+            _accessor.getChildValues(_propertyKey);
 
         messageListener.onMessage(instanceName, messages, changeContext);
 
@@ -183,7 +194,7 @@ public class CallbackHandler implements IZkChildListener, IZkDataListener
         MessageListener messageListener = (MessageListener) _listener;
         subscribeForChanges(changeContext, _path, true, false);
         List<Message> messages =
-            _accessor.getChildValues(keyBuilder.controllerMessages());
+            _accessor.getChildValues(_propertyKey);
 
         messageListener.onMessage(_manager.getInstanceName(), messages, changeContext);
 
@@ -194,7 +205,7 @@ public class CallbackHandler implements IZkChildListener, IZkDataListener
             (ExternalViewChangeListener) _listener;
         subscribeForChanges(changeContext, _path, true, true);
         List<ExternalView> externalViewList =
-            _accessor.getChildValues(keyBuilder.externalViews());
+            _accessor.getChildValues(_propertyKey);
 
         externalViewListener.onExternalViewChange(externalViewList, changeContext);
       }
@@ -214,7 +225,7 @@ public class CallbackHandler implements IZkChildListener, IZkDataListener
         String instanceName = PropertyPathConfig.getInstanceNameFromPath(_path);
 
         List<HealthStat> healthReportList =
-            _accessor.getChildValues(keyBuilder.healthReports(instanceName));
+            _accessor.getChildValues(_propertyKey);
 
         healthStateChangeListener.onHealthChange(instanceName,
                                                  healthReportList,

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c52baf61/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixDataAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixDataAccessor.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixDataAccessor.java
index 46274b8..3d49c01 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixDataAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixDataAccessor.java
@@ -431,17 +431,6 @@ public class ZKHelixDataAccessor implements HelixDataAccessor, ControllerChangeL
       options = options | AccessOption.EPHEMERAL;
     }
 
-    // if (type == PropertyType.CURRENTSTATES && _instanceType ==
-    // InstanceType.PARTICIPANT)
-    // {
-    // options = options | BaseDataAccessor.Option.WRITE_THROUGH;
-    // }
-    // else if (type == PropertyType.EXTERNALVIEW
-    // && _instanceType == InstanceType.CONTROLLER)
-    // {
-    // options = options | BaseDataAccessor.Option.WRITE_THROUGH;
-    // }
-
     return options;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c52baf61/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java
----------------------------------------------------------------------
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 1cbdec9..f542993 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
@@ -56,13 +56,16 @@ import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixTimerTask;
 import org.apache.helix.IdealStateChangeListener;
+import org.apache.helix.InstanceConfigChangeListener;
 import org.apache.helix.InstanceType;
 import org.apache.helix.LiveInstanceChangeListener;
 import org.apache.helix.MessageListener;
 import org.apache.helix.PreConnectCallback;
+import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.PropertyPathConfig;
 import org.apache.helix.PropertyType;
+import org.apache.helix.ScopedConfigChangeListener;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.controller.restlet.ZKPropertyTransferServer;
 import org.apache.helix.healthcheck.HealthStatsAggregationTask;
@@ -242,87 +245,94 @@ public class ZKHelixManager implements HelixManager
     }
     return true;
   }
+  
+  private void addListener(Object listener, PropertyKey propertyKey, ChangeType changeType, EventType[] eventType)
+  {
+    checkConnected();
 
+    PropertyType type = propertyKey.getType();
+    CallbackHandler handler =
+        createCallBackHandler(propertyKey, listener, eventType, changeType);
+
+    synchronized (this)
+    {
+      _handlers.add(handler);
+      logger.info("Add listener: " + listener + " for type: " + type + " to path: " + handler.getPath());
+    }
+  }
+  
   @Override
   public void addIdealStateChangeListener(final IdealStateChangeListener listener) throws Exception
   {
-    logger.info("ClusterManager.addIdealStateChangeListener()");
-    checkConnected();
-    final String path =
-        PropertyPathConfig.getPath(PropertyType.IDEALSTATES, _clusterName);
-    CallbackHandler callbackHandler =
-        createCallBackHandler(path,
-                              listener,
-                              new EventType[] { EventType.NodeDataChanged,
-                                  EventType.NodeDeleted, EventType.NodeCreated },
-                              IDEAL_STATE);
-    addListener(callbackHandler);
+	  addListener(listener, new Builder(_clusterName).idealStates(), ChangeType.IDEAL_STATE, 
+	    new EventType[] { EventType.NodeDataChanged, EventType.NodeDeleted, EventType.NodeCreated });
   }
 
   @Override
   public void addLiveInstanceChangeListener(LiveInstanceChangeListener listener) throws Exception
   {
-    logger.info("ClusterManager.addLiveInstanceChangeListener()");
-    checkConnected();
-    final String path = _helixAccessor.keyBuilder().liveInstances().getPath();
-    CallbackHandler callbackHandler =
-        createCallBackHandler(path,
-                              listener,
-                              new EventType[] { EventType.NodeDataChanged, EventType.NodeChildrenChanged,
-                                  EventType.NodeDeleted, EventType.NodeCreated },
-                              LIVE_INSTANCE);
-    addListener(callbackHandler);
+	  addListener(listener, new Builder(_clusterName).liveInstances(), ChangeType.LIVE_INSTANCE, 
+	    new EventType[] { EventType.NodeDataChanged, EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated });
   }
 
   @Override
   public void addConfigChangeListener(ConfigChangeListener listener)
   {
-    logger.info("ClusterManager.addConfigChangeListener()");
-    checkConnected();
-    final String path =
-        PropertyPathConfig.getPath(PropertyType.CONFIGS,
-                                   _clusterName,
-                                   ConfigScopeProperty.PARTICIPANT.toString());
-
-    CallbackHandler callbackHandler =
-        createCallBackHandler(path,
-                              listener,
-                              new EventType[] { EventType.NodeChildrenChanged },
-                              CONFIG);
-    addListener(callbackHandler);
+	 addListener(listener, new Builder(_clusterName).instanceConfigs(), ChangeType.INSTANCE_CONFIG, 
+	   new EventType[] { EventType.NodeChildrenChanged });
+  }
 
+  @Override
+  public void addInstanceConfigChangeListener(InstanceConfigChangeListener listener)
+  {
+	 addListener(listener, new Builder(_clusterName).instanceConfigs(), ChangeType.INSTANCE_CONFIG, 
+			 new EventType[] { EventType.NodeChildrenChanged });
   }
 
+  @Override
+  public void addConfigChangeListener(ScopedConfigChangeListener listener, ConfigScopeProperty scope)
+  {
+	Builder keyBuilder = new Builder(_clusterName);
+
+	PropertyKey propertyKey = null;
+	switch(scope)
+	{
+	case CLUSTER:
+		propertyKey = keyBuilder.clusterConfigs();
+		break;
+	case PARTICIPANT:
+		propertyKey = keyBuilder.instanceConfigs();
+		break;
+	case RESOURCE:
+		propertyKey = keyBuilder.resourceConfigs();
+		break;
+	default:
+		break;
+	}
+
+	if (propertyKey != null)
+	{
+		addListener(listener, propertyKey, ChangeType.CONFIG, 
+				new EventType[] { EventType.NodeChildrenChanged });
+	} else
+	{
+		logger.error("Can't add listener to config scope: " + scope);
+	}
+  }
+  
   // TODO: Decide if do we still need this since we are exposing
   // ClusterMessagingService
   @Override
   public void addMessageListener(MessageListener listener, String instanceName)
   {
-    logger.info("ClusterManager.addMessageListener() " + instanceName);
-    checkConnected();
-    final String path = _helixAccessor.keyBuilder().messages(instanceName).getPath();
-    CallbackHandler callbackHandler =
-        createCallBackHandler(path,
-                              listener,
-                              new EventType[] { EventType.NodeChildrenChanged,
-                                  EventType.NodeDeleted, EventType.NodeCreated },
-                              MESSAGE);
-    addListener(callbackHandler);
+	 addListener(listener, new Builder(_clusterName).messages(instanceName), ChangeType.MESSAGE, 
+	    		new EventType[] { EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated });
   }
 
   void addControllerMessageListener(MessageListener listener)
   {
-    logger.info("ClusterManager.addControllerMessageListener()");
-    checkConnected();
-    final String path = _helixAccessor.keyBuilder().controllerMessages().getPath();
-
-    CallbackHandler callbackHandler =
-        createCallBackHandler(path,
-                              listener,
-                              new EventType[] { EventType.NodeChildrenChanged,
-                                  EventType.NodeDeleted, EventType.NodeCreated },
-                              MESSAGES_CONTROLLER);
-    addListener(callbackHandler);
+	 addListener(listener, new Builder(_clusterName).controllerMessages(), ChangeType.MESSAGES_CONTROLLER,
+	    		new EventType[] { EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated });
   }
 
   @Override
@@ -330,52 +340,30 @@ public class ZKHelixManager implements HelixManager
                                             String instanceName,
                                             String sessionId)
   {
-    logger.info("ClusterManager.addCurrentStateChangeListener() " + instanceName + " "
-        + sessionId);
-    checkConnected();
-    final String path =
-        _helixAccessor.keyBuilder().currentStates(instanceName, sessionId).getPath();
-
-    CallbackHandler callbackHandler =
-        createCallBackHandler(path,
-                              listener,
-                              new EventType[] { EventType.NodeChildrenChanged,
-                                  EventType.NodeDeleted, EventType.NodeCreated },
-                              CURRENT_STATE);
-    addListener(callbackHandler);
+	 addListener(listener, new Builder(_clusterName).currentStates(instanceName, sessionId), ChangeType.CURRENT_STATE,
+	    		new EventType[] { EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated });
   }
 
   @Override
   public void addHealthStateChangeListener(HealthStateChangeListener listener,
                                            String instanceName)
   {
-    // System.out.println("ZKClusterManager.addHealthStateChangeListener()");
-    // TODO: re-form this for stats checking
-    logger.info("ClusterManager.addHealthStateChangeListener()" + instanceName);
-    checkConnected();
-    final String path = _helixAccessor.keyBuilder().healthReports(instanceName).getPath();
-
-    CallbackHandler callbackHandler =
-        createCallBackHandler(path, listener, new EventType[] {
-            EventType.NodeChildrenChanged, EventType.NodeDataChanged,
-            EventType.NodeDeleted, EventType.NodeCreated }, HEALTH);
-    addListener(callbackHandler);
+	  addListener(listener, new Builder(_clusterName).healthReports(instanceName), ChangeType.HEALTH,
+	    		new EventType[] { EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated });
   }
 
   @Override
   public void addExternalViewChangeListener(ExternalViewChangeListener listener)
   {
-    logger.info("ClusterManager.addExternalViewChangeListener()");
-    checkConnected();
-    final String path = _helixAccessor.keyBuilder().externalViews().getPath();
-
-    CallbackHandler callbackHandler =
-        createCallBackHandler(path,
-                              listener,
-                              new EventType[] { EventType.NodeDataChanged,
-                                  EventType.NodeDeleted, EventType.NodeCreated },
-                              EXTERNAL_VIEW);
-    addListener(callbackHandler);
+	  addListener(listener, new Builder(_clusterName).externalViews(), ChangeType.EXTERNAL_VIEW,
+	    		new EventType[] { EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated });
+  }
+
+  @Override
+  public void addControllerListener(ControllerChangeListener listener)
+  {
+	  addListener(listener, new Builder(_clusterName).controller(), ChangeType.CONTROLLER,
+	    		new EventType[] { EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated });
   }
 
   @Override
@@ -507,28 +495,10 @@ public class ZKHelixManager implements HelixManager
   }
 
   @Override
-  public void addControllerListener(ControllerChangeListener listener)
-  {
-    checkConnected();
-    final String path = _helixAccessor.keyBuilder().controller().getPath();
-    logger.info("Add controller listener at: " + path);
-    CallbackHandler callbackHandler =
-        createCallBackHandler(path,
-                              listener,
-                              new EventType[] { EventType.NodeChildrenChanged,
-                                  EventType.NodeDeleted, EventType.NodeCreated },
-                              ChangeType.CONTROLLER);
-
-    // System.out.println("add controller listeners to " + _instanceName +
-    // " for " + _clusterName);
-    // _handlers.add(callbackHandler);
-    addListener(callbackHandler);
-  }
-
-  @Override
   public boolean removeListener(Object listener)
   {
-    logger.info("remove listener: " + listener + " from " + _instanceName);
+	logger.info("remove listener: " + listener + " from cluster: " + _clusterName 
+			+ ", instance: " + _instanceName);
 
     synchronized (this)
     {
@@ -672,7 +642,7 @@ public class ZKHelixManager implements HelixManager
     }
   }
 
-  private CallbackHandler createCallBackHandler(String path,
+  private CallbackHandler createCallBackHandler(PropertyKey propertyKey,
                                                 Object listener,
                                                 EventType[] eventTypes,
                                                 ChangeType changeType)
@@ -681,7 +651,7 @@ public class ZKHelixManager implements HelixManager
     {
       throw new HelixException("Listener cannot be null");
     }
-    return new CallbackHandler(this, _zkClient, path, listener, eventTypes, changeType);
+    return new CallbackHandler(this, _zkClient, propertyKey, listener, eventTypes, changeType);
   }
 
   /**
@@ -766,7 +736,7 @@ public class ZKHelixManager implements HelixManager
             PropertyPathConfig.getPath(PropertyType.CONTROLLER, _clusterName);
 
         _leaderElectionHandler =
-            createCallBackHandler(path,
+            createCallBackHandler(new Builder(_clusterName).controller(),
                                   new DistClusterControllerElection(_zkConnectString),
                                   new EventType[] { EventType.NodeChildrenChanged,
                                       EventType.NodeDeleted, EventType.NodeCreated },

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c52baf61/helix-core/src/test/java/org/apache/helix/Mocks.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/Mocks.java b/helix-core/src/test/java/org/apache/helix/Mocks.java
index e961ef6..2603272 100644
--- a/helix-core/src/test/java/org/apache/helix/Mocks.java
+++ b/helix-core/src/test/java/org/apache/helix/Mocks.java
@@ -34,6 +34,7 @@ import org.apache.helix.BaseDataAccessor;
 import org.apache.helix.ClusterMessagingService;
 import org.apache.helix.ConfigAccessor;
 import org.apache.helix.ConfigChangeListener;
+import org.apache.helix.ConfigScope.ConfigScopeProperty;
 import org.apache.helix.ControllerChangeListener;
 import org.apache.helix.Criteria;
 import org.apache.helix.CurrentStateChangeListener;
@@ -475,6 +476,20 @@ public class Mocks {
       // TODO Auto-generated method stub
       return null;
     }
+    
+    @Override
+    public void addInstanceConfigChangeListener(InstanceConfigChangeListener listener)
+            throws Exception {
+	    // TODO Auto-generated method stub
+
+    }
+
+	@Override
+    public void addConfigChangeListener(ScopedConfigChangeListener listener,
+            ConfigScopeProperty scope) throws Exception {
+	    // TODO Auto-generated method stub
+
+    }
 
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c52baf61/helix-core/src/test/java/org/apache/helix/TestListenerCallback.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/TestListenerCallback.java b/helix-core/src/test/java/org/apache/helix/TestListenerCallback.java
new file mode 100644
index 0000000..7465f7c
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/TestListenerCallback.java
@@ -0,0 +1,133 @@
+package org.apache.helix;
+
+/*
+ * 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.Date;
+import java.util.List;
+
+import org.apache.helix.ConfigScope.ConfigScopeProperty;
+import org.apache.helix.PropertyKey.Builder;
+import org.apache.helix.model.InstanceConfig;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+
+public class TestListenerCallback extends ZkUnitTestBase 
+{
+	class TestListener implements InstanceConfigChangeListener, ScopedConfigChangeListener
+	{
+		boolean _instanceConfigChanged = false;
+		boolean _configChanged = false;
+
+		@Override
+        public void onConfigChange(List<HelixProperty> configs, NotificationContext context) {
+			_configChanged = true;
+			System.out.println("onConfigChange invoked: " + configs.size() + ", " + configs);
+        }
+
+		@Override
+        public void onInstanceConfigChange(List<InstanceConfig> instanceConfigs,
+                NotificationContext context) {
+			_instanceConfigChanged = true;
+			System.out.println("onInstanceConfigChange invoked: " + instanceConfigs);
+        }
+
+	}
+
+	@Test
+	public void testBasic() throws Exception
+	{
+	    // Logger.getRootLogger().setLevel(Level.INFO);
+	    String className = TestHelper.getTestClassName();
+	    String methodName = TestHelper.getTestMethodName();
+	    String clusterName = className + "_" + methodName;
+	    int n = 2;
+
+	    System.out.println("START " + clusterName + " at "
+	        + new Date(System.currentTimeMillis()));
+
+	    TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port
+                "localhost", // participant name prefix
+                "TestDB", // resource name prefix
+                1, // resources
+                32, // partitions per resource
+                n, // number of nodes
+                2, // replicas
+                "MasterSlave",
+                true); // do rebalance
+
+	    HelixManager manager = HelixManagerFactory.getZKHelixManager(clusterName, "localhost", InstanceType.SPECTATOR, ZK_ADDR);
+
+	    manager.connect();
+
+	    TestListener listener = new TestListener();
+	    listener._instanceConfigChanged = false;
+	    manager.addInstanceConfigChangeListener(listener);
+	    Assert.assertTrue(listener._instanceConfigChanged, "Should get initial instanceConfig callback invoked");
+
+	    listener._configChanged = false;
+	    manager.addConfigChangeListener(listener, ConfigScopeProperty.CLUSTER);
+	    Assert.assertTrue(listener._configChanged, "Should get initial clusterConfig callback invoked");
+
+	    listener._configChanged = false;
+	    manager.addConfigChangeListener(listener, ConfigScopeProperty.RESOURCE);
+	    Assert.assertTrue(listener._configChanged, "Should get initial resourceConfig callback invoked");
+
+	    // test change content
+	    HelixDataAccessor accessor = manager.getHelixDataAccessor();
+	    Builder keyBuilder = accessor.keyBuilder();
+	    String instanceName = "localhost_12918";
+	    HelixProperty value = accessor.getProperty(keyBuilder.instanceConfig(instanceName));
+	    value._record.setSimpleField("" + System.currentTimeMillis(), "newValue");
+	    listener._instanceConfigChanged = false;
+	    accessor.setProperty(keyBuilder.instanceConfig(instanceName), value);
+	    Thread.sleep(1000);	// wait zk callback
+	    Assert.assertTrue(listener._instanceConfigChanged, "Should get instanceConfig callback invoked since we change instanceConfig");
+
+	    value = accessor.getProperty(keyBuilder.clusterConfig());
+	    value._record.setSimpleField("" + System.currentTimeMillis(), "newValue");
+	    listener._configChanged = false;
+	    accessor.setProperty(keyBuilder.clusterConfig(), value);
+	    Thread.sleep(1000);	// wait zk callback
+	    Assert.assertTrue(listener._configChanged, "Should get clusterConfig callback invoked since we change clusterConfig");
+
+	    String resourceName = "TestDB_0";
+	    value = new HelixProperty(resourceName);
+	    value._record.setSimpleField("" + System.currentTimeMillis(), "newValue");
+	    listener._configChanged = false;
+	    accessor.setProperty(keyBuilder.resourceConfig(resourceName), value);
+	    Thread.sleep(1000);	// wait zk callback
+	    Assert.assertTrue(listener._configChanged, "Should get resourceConfig callback invoked since we add resourceConfig");
+
+	    value._record.setSimpleField("" + System.currentTimeMillis(), "newValue");
+	    listener._configChanged = false;
+	    accessor.setProperty(keyBuilder.resourceConfig(resourceName), value);
+	    Thread.sleep(1000);	// wait zk callback
+	    Assert.assertTrue(listener._configChanged, "Should get resourceConfig callback invoked since we change resourceConfig");
+
+	    listener._configChanged = false;
+	    accessor.removeProperty(keyBuilder.resourceConfig(resourceName));
+	    Thread.sleep(1000);	// wait zk callback
+	    Assert.assertTrue(listener._configChanged, "Should get resourceConfig callback invoked since we delete resourceConfig");
+
+	    System.out.println("END " + clusterName + " at "
+	            + new Date(System.currentTimeMillis()));
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c52baf61/helix-core/src/test/java/org/apache/helix/controller/stages/DummyClusterManager.java
----------------------------------------------------------------------
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 9d035fb..25c139b 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
@@ -22,6 +22,7 @@ package org.apache.helix.controller.stages;
 import org.apache.helix.ClusterMessagingService;
 import org.apache.helix.ConfigAccessor;
 import org.apache.helix.ConfigChangeListener;
+import org.apache.helix.ConfigScope.ConfigScopeProperty;
 import org.apache.helix.ControllerChangeListener;
 import org.apache.helix.CurrentStateChangeListener;
 import org.apache.helix.ExternalViewChangeListener;
@@ -30,10 +31,12 @@ import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.IdealStateChangeListener;
+import org.apache.helix.InstanceConfigChangeListener;
 import org.apache.helix.InstanceType;
 import org.apache.helix.LiveInstanceChangeListener;
 import org.apache.helix.MessageListener;
 import org.apache.helix.PreConnectCallback;
+import org.apache.helix.ScopedConfigChangeListener;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.healthcheck.ParticipantHealthReportCollector;
 import org.apache.helix.participant.StateMachineEngine;
@@ -254,4 +257,17 @@ public class DummyClusterManager implements HelixManager
     // TODO Auto-generated method stub
     return null;
   }
+  
+  @Override
+  public void addInstanceConfigChangeListener(InstanceConfigChangeListener listener) throws Exception {
+  	// TODO Auto-generated method stub
+
+  }
+
+  @Override
+  public void addConfigChangeListener(ScopedConfigChangeListener listener, ConfigScopeProperty scope)
+          throws Exception {
+  	// TODO Auto-generated method stub
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c52baf61/helix-core/src/test/java/org/apache/helix/participant/MockZKHelixManager.java
----------------------------------------------------------------------
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 b616e22..517dbdf 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
@@ -24,6 +24,7 @@ import java.util.UUID;
 import org.apache.helix.ClusterMessagingService;
 import org.apache.helix.ConfigAccessor;
 import org.apache.helix.ConfigChangeListener;
+import org.apache.helix.ConfigScope.ConfigScopeProperty;
 import org.apache.helix.ControllerChangeListener;
 import org.apache.helix.CurrentStateChangeListener;
 import org.apache.helix.ExternalViewChangeListener;
@@ -32,10 +33,12 @@ import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.IdealStateChangeListener;
+import org.apache.helix.InstanceConfigChangeListener;
 import org.apache.helix.InstanceType;
 import org.apache.helix.LiveInstanceChangeListener;
 import org.apache.helix.MessageListener;
 import org.apache.helix.PreConnectCallback;
+import org.apache.helix.ScopedConfigChangeListener;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.healthcheck.ParticipantHealthReportCollector;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
@@ -264,5 +267,18 @@ public class MockZKHelixManager implements HelixManager
     // TODO Auto-generated method stub
     return null;
   }
+  
+  @Override
+  public void addInstanceConfigChangeListener(InstanceConfigChangeListener listener) throws Exception {
+  	// TODO Auto-generated method stub
+
+  }
+
+  @Override
+  public void addConfigChangeListener(ScopedConfigChangeListener listener, ConfigScopeProperty scope)
+          throws Exception {
+  	// TODO Auto-generated method stub
+
+  }
 
 }