You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by ka...@apache.org on 2014/07/11 21:57:56 UTC

[01/17] git commit: [HELIX-443] Race condition in Helix register/unregister MessageHandlerFactory, rb=21248

Repository: helix
Updated Branches:
  refs/heads/master 6d30c9c58 -> 63f084bd2


[HELIX-443] Race condition in Helix register/unregister MessageHandlerFactory, rb=21248


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

Branch: refs/heads/master
Commit: fdee2dd73dcb074a1e689e2431134edee9104768
Parents: 6d30c9c
Author: zzhang <zz...@apache.org>
Authored: Wed May 21 15:09:14 2014 -0700
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Fri Jul 11 10:30:10 2014 -0700

----------------------------------------------------------------------
 .../messaging/handling/HelixTaskExecutor.java   | 115 +++++++++++--------
 1 file changed, 64 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/fdee2dd7/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
index 8da53ea..e2af382 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
@@ -49,6 +49,9 @@ import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.api.id.SessionId;
 import org.apache.helix.model.ConfigScope;
 import org.apache.helix.model.CurrentState;
+import org.apache.helix.model.HelixConfigScope;
+import org.apache.helix.model.builder.HelixConfigScopeBuilder;
+import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.MessageState;
 import org.apache.helix.model.Message.MessageType;
@@ -103,19 +106,22 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
   @Override
   public void registerMessageHandlerFactory(String type, MessageHandlerFactory factory,
       int threadpoolSize) {
-    if (!_handlerFactoryMap.containsKey(type)) {
-      if (!type.equalsIgnoreCase(factory.getMessageType())) {
-        throw new HelixException("Message factory type mismatch. Type: " + type + " factory : "
-            + factory.getMessageType());
+    if (!type.equalsIgnoreCase(factory.getMessageType())) {
+      throw new HelixException("Message factory type mismatch. Type: " + type + ", factory: "
+          + factory.getMessageType());
+    }
 
+    MessageHandlerFactory prevFactory = _handlerFactoryMap.putIfAbsent(type, factory);
+    if (prevFactory == null) {
+      if (!_executorMap.contains(type)) {
+        _executorMap.put(type, Executors.newFixedThreadPool(threadpoolSize));
+      } else {
+        LOG.error("Skip to create new thread pool for type: " + type);
       }
-      _handlerFactoryMap.put(type, factory);
-      ExecutorService executorSvc = Executors.newFixedThreadPool(threadpoolSize);
-      _executorMap.put(type, executorSvc);
-
-      LOG.info("Added msg-factory for type: " + type + ", threadpool size " + threadpoolSize);
+      LOG.info("Registered message handler factory for type: " + type + ", poolSize: "
+          + threadpoolSize + ", factory: " + factory + ", pool: " + _executorMap.get(type));
     } else {
-      LOG.warn("Fail to register msg-handler-factory for type: " + type + ", pool-size: "
+      LOG.warn("Fail to register message handler factory for type: " + type + ", poolSize: "
           + threadpoolSize + ", factory: " + factory);
     }
   }
@@ -133,9 +139,9 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
       int threadpoolSize = -1;
       ConfigAccessor configAccessor = manager.getConfigAccessor();
       if (configAccessor != null) {
-        ConfigScope scope =
-            new ConfigScopeBuilder().forCluster(manager.getClusterName()).forResource(resourceName)
-                .build();
+        HelixConfigScope scope =
+            new HelixConfigScopeBuilder(ConfigScopeProperty.RESOURCE)
+                .forCluster(manager.getClusterName()).forResource(resourceName).build();
 
         String threadpoolSizeStr = configAccessor.get(scope, MAX_THREADS);
         try {
@@ -242,6 +248,8 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
       synchronized (_lock) {
         if (!_taskMap.containsKey(taskId)) {
           ExecutorService exeSvc = findExecutorServiceForMsg(message);
+
+          LOG.info("Submit task: " + taskId + " to pool: " + exeSvc);
           Future<HelixTaskResult> future = exeSvc.submit(task);
 
           TimerTask timerTask = null;
@@ -296,8 +304,7 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
 
         // If the thread is still running it will be interrupted if cancel(true)
         // is called. So state transition callbacks should implement logic to
-        // return
-        // if it is interrupted.
+        // return if it is interrupted.
         if (future.cancel(true)) {
           _statusUpdateUtil.logInfo(message, HelixTaskExecutor.class, "Canceled task: " + taskId,
               notificationContext.getManager().getHelixDataAccessor());
@@ -346,42 +353,63 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
     accessor.setChildren(readMsgKeys, readMsgs);
   }
 
+  private void shutdownAndAwaitTermination(ExecutorService pool) {
+    LOG.info("Shutting down pool: " + pool);
+    pool.shutdown(); // Disable new tasks from being submitted
+    try {
+      // Wait a while for existing tasks to terminate
+      if (!pool.awaitTermination(200, TimeUnit.MILLISECONDS)) {
+        List<Runnable> waitingTasks = pool.shutdownNow(); // Cancel currently executing tasks
+        LOG.info("Tasks that never commenced execution: " + waitingTasks);
+        // Wait a while for tasks to respond to being cancelled
+        if (!pool.awaitTermination(200, TimeUnit.MILLISECONDS)) {
+          LOG.error("Pool did not fully terminate in 200ms. pool: " + pool);
+        }
+      }
+    } catch (InterruptedException ie) {
+      // (Re-)Cancel if current thread also interrupted
+      LOG.error("Interruped when waiting for shutdown pool: " + pool, ie);
+      pool.shutdownNow();
+      // Preserve interrupt status
+      Thread.currentThread().interrupt();
+    }
+  }
+
   /**
    * remove message-handler factory from map, shutdown the associated executor
    * @param type
    */
   void unregisterMessageHandlerFactory(String type) {
-    // shutdown executor-service. disconnect if fail
-    ExecutorService executorSvc = _executorMap.remove(type);
-    if (executorSvc != null) {
-      List<Runnable> tasksLeft = executorSvc.shutdownNow();
-      LOG.info(tasksLeft.size() + " tasks never executed for msgType: " + type + ". tasks: "
-          + tasksLeft);
-      try {
-        if (!executorSvc.awaitTermination(200, TimeUnit.MILLISECONDS)) {
-          LOG.error("executor-service for msgType: " + type
-              + " is not fully terminated in 200ms. will disconnect helix-participant");
-          throw new HelixException("fail to unregister msg-handler for msgType: " + type);
-        }
-      } catch (InterruptedException e) {
-        LOG.error("interruped when waiting for executor-service shutdown for msgType: " + type, e);
-      }
+    ExecutorService pool = _executorMap.remove(type);
+    MessageHandlerFactory handlerFty = _handlerFactoryMap.remove(type);
+
+    LOG.info("Unregistering message handler factory for type: " + type + ", factory: " + handlerFty
+        + ", pool: " + pool);
+
+    if (pool != null) {
+      shutdownAndAwaitTermination(pool);
     }
 
     // reset state-model
-    MessageHandlerFactory handlerFty = _handlerFactoryMap.remove(type);
     if (handlerFty != null) {
       handlerFty.reset();
     }
+
+    LOG.info("Unregistered message handler factory for type: " + type + ", factory: " + handlerFty
+        + ", pool: " + pool);
   }
 
   void reset() {
-    LOG.info("Get FINALIZE notification");
+    LOG.info("Reset HelixTaskExecutor");
     for (String msgType : _executorMap.keySet()) {
       unregisterMessageHandlerFactory(msgType);
     }
 
-    // clear task-map, all tasks should be terminated by now
+    // Log all tasks that fail to terminate
+    for (String taskId : _taskMap.keySet()) {
+      MessageTaskInfo info = _taskMap.get(taskId);
+      LOG.warn("Task: " + taskId + " fails to terminate. Message: " + info._task.getMessage());
+    }
     _taskMap.clear();
 
   }
@@ -559,26 +587,11 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
 
   @Override
   public void shutdown() {
-    LOG.info("shutting down TaskExecutor");
+    LOG.info("Shutting down HelixTaskExecutor");
     _timer.cancel();
 
-    synchronized (_lock) {
-      for (String msgType : _executorMap.keySet()) {
-        List<Runnable> tasksLeft = _executorMap.get(msgType).shutdownNow();
-        LOG.info(tasksLeft.size() + " tasks are still in the threadpool for msgType " + msgType);
-      }
-      for (String msgType : _executorMap.keySet()) {
-        try {
-          if (!_executorMap.get(msgType).awaitTermination(200, TimeUnit.MILLISECONDS)) {
-            LOG.warn(msgType + " is not fully termimated in 200 MS");
-            System.out.println(msgType + " is not fully termimated in 200 MS");
-          }
-        } catch (InterruptedException e) {
-          LOG.error("Interrupted", e);
-        }
-      }
-    }
+    reset();
     _monitor.shutDown();
-    LOG.info("shutdown finished");
+    LOG.info("Shutdown HelixTaskExecutor finished");
   }
 }


[17/17] git commit: [HELIX-464] rabbitmq recipe is broken, rb=23099

Posted by ka...@apache.org.
[HELIX-464] rabbitmq recipe is broken, rb=23099


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

Branch: refs/heads/master
Commit: 63f084bd2d1ebb06d41edc1aa74754410998878b
Parents: 28a8f20
Author: zzhang <zz...@apache.org>
Authored: Thu Jun 26 17:10:19 2014 -0700
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Fri Jul 11 12:56:56 2014 -0700

----------------------------------------------------------------------
 recipes/rabbitmq-consumer-group/bin/log4j.xml   | 37 --------------------
 .../rabbitmq-consumer-group/bin/send-message.sh | 26 --------------
 .../bin/setup-cluster.sh                        | 26 --------------
 .../bin/start-cluster-manager.sh                | 26 --------------
 .../bin/start-consumer.sh                       | 26 --------------
 recipes/rabbitmq-consumer-group/pom.xml         | 30 +++++++++++++++-
 .../src/main/config/log4j.properties            | 31 ++++++++++++++++
 7 files changed, 60 insertions(+), 142 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/63f084bd/recipes/rabbitmq-consumer-group/bin/log4j.xml
----------------------------------------------------------------------
diff --git a/recipes/rabbitmq-consumer-group/bin/log4j.xml b/recipes/rabbitmq-consumer-group/bin/log4j.xml
deleted file mode 100644
index d4325e9..0000000
--- a/recipes/rabbitmq-consumer-group/bin/log4j.xml
+++ /dev/null
@@ -1,37 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" ?>
-<!--
-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.
--->
-<!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
-
-<log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
-  
-  <appender name="ConsoleAppender" class="org.apache.log4j.ConsoleAppender">
-    <param name="Target" value="System.err" />
-    <layout class="org.apache.log4j.PatternLayout">
-      <param name="ConversionPattern" value="%d{yyyy/MM/dd HH:mm:ss.SSS} %p [%c{1}] [perf-test] %m%n" />
-    </layout>
-  </appender>
-	  
-  <root>
-    <priority value="debug"/>
-    <appender-ref ref="ConsoleAppender"/>
-  </root>
-  
-</log4j:configuration>
-

http://git-wip-us.apache.org/repos/asf/helix/blob/63f084bd/recipes/rabbitmq-consumer-group/bin/send-message.sh
----------------------------------------------------------------------
diff --git a/recipes/rabbitmq-consumer-group/bin/send-message.sh b/recipes/rabbitmq-consumer-group/bin/send-message.sh
deleted file mode 100755
index 2ad6a2c..0000000
--- a/recipes/rabbitmq-consumer-group/bin/send-message.sh
+++ /dev/null
@@ -1,26 +0,0 @@
-#!/bin/bash
-#
-# 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.
-#
-
-script_dir=`dirname $0`
-LIB=$script_dir/../lib
-CLASSPATH=$script_dir/../target/classes:"$LIB"/helix-core-0.1-SNAPSHOT-incubating.jar:"$LIB"/rabbitmq-client.jar:"$LIB"/commons-cli-1.1.jar:"$LIB"/commons-io-1.2.jar:"$LIB"/commons-math-2.1.jar:"$LIB"/jackson-core-asl-1.8.5.jar:"$LIB"/jackson-mapper-asl-1.8.5.jar:"$LIB"/log4j-1.2.15.jar:"$LIB"/org.restlet-1.1.10.jar:"$LIB"/zkclient-0.1.jar:"$LIB"/zookeeper-3.3.4.jar
-# echo $CLASSPATH
-
-java -cp "$CLASSPATH" org.apache.helix.recipes.rabbitmq.Emitter $@

http://git-wip-us.apache.org/repos/asf/helix/blob/63f084bd/recipes/rabbitmq-consumer-group/bin/setup-cluster.sh
----------------------------------------------------------------------
diff --git a/recipes/rabbitmq-consumer-group/bin/setup-cluster.sh b/recipes/rabbitmq-consumer-group/bin/setup-cluster.sh
deleted file mode 100755
index ab5bff1..0000000
--- a/recipes/rabbitmq-consumer-group/bin/setup-cluster.sh
+++ /dev/null
@@ -1,26 +0,0 @@
-#!/bin/bash
-#
-# 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.
-#
-
-script_dir=`dirname $0`
-LIB=$script_dir/../lib
-CLASSPATH=$script_dir/../target/classes:"$LIB"/helix-core-0.1-SNAPSHOT-incubating.jar:"$LIB"/rabbitmq-client.jar:"$LIB"/commons-cli-1.1.jar:"$LIB"/commons-io-1.2.jar:"$LIB"/commons-math-2.1.jar:"$LIB"/jackson-core-asl-1.8.5.jar:"$LIB"/jackson-mapper-asl-1.8.5.jar:"$LIB"/log4j-1.2.15.jar:"$LIB"/org.restlet-1.1.10.jar:"$LIB"/zkclient-0.1.jar:"$LIB"/zookeeper-3.3.4.jar
-# echo $CLASSPATH
-
-java -cp "$CLASSPATH" org.apache.helix.recipes.rabbitmq.SetupConsumerCluster $@

http://git-wip-us.apache.org/repos/asf/helix/blob/63f084bd/recipes/rabbitmq-consumer-group/bin/start-cluster-manager.sh
----------------------------------------------------------------------
diff --git a/recipes/rabbitmq-consumer-group/bin/start-cluster-manager.sh b/recipes/rabbitmq-consumer-group/bin/start-cluster-manager.sh
deleted file mode 100755
index 5b4ac2c..0000000
--- a/recipes/rabbitmq-consumer-group/bin/start-cluster-manager.sh
+++ /dev/null
@@ -1,26 +0,0 @@
-#!/bin/bash
-#
-# 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.
-#
-
-script_dir=`dirname $0`
-LIB=$script_dir/../lib
-CLASSPATH=$script_dir/../target/classes:"$LIB"/helix-core-0.1-SNAPSHOT-incubating.jar:"$LIB"/rabbitmq-client.jar:"$LIB"/commons-cli-1.1.jar:"$LIB"/commons-io-1.2.jar:"$LIB"/commons-math-2.1.jar:"$LIB"/jackson-core-asl-1.8.5.jar:"$LIB"/jackson-mapper-asl-1.8.5.jar:"$LIB"/log4j-1.2.15.jar:"$LIB"/org.restlet-1.1.10.jar:"$LIB"/zkclient-0.1.jar:"$LIB"/zookeeper-3.3.4.jar
-# echo $CLASSPATH
-
-java -cp "$CLASSPATH" org.apache.helix.recipes.rabbitmq.StartClusterManager $@

http://git-wip-us.apache.org/repos/asf/helix/blob/63f084bd/recipes/rabbitmq-consumer-group/bin/start-consumer.sh
----------------------------------------------------------------------
diff --git a/recipes/rabbitmq-consumer-group/bin/start-consumer.sh b/recipes/rabbitmq-consumer-group/bin/start-consumer.sh
deleted file mode 100755
index e93707b..0000000
--- a/recipes/rabbitmq-consumer-group/bin/start-consumer.sh
+++ /dev/null
@@ -1,26 +0,0 @@
-#!/bin/bash
-#
-# 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.
-#
-
-script_dir=`dirname $0`
-LIB=$script_dir/../lib
-CLASSPATH=$script_dir/../target/classes:"$LIB"/helix-core-0.1-SNAPSHOT-incubating.jar:"$LIB"/rabbitmq-client.jar:"$LIB"/commons-cli-1.1.jar:"$LIB"/commons-io-1.2.jar:"$LIB"/commons-math-2.1.jar:"$LIB"/jackson-core-asl-1.8.5.jar:"$LIB"/jackson-mapper-asl-1.8.5.jar:"$LIB"/log4j-1.2.15.jar:"$LIB"/org.restlet-1.1.10.jar:"$LIB"/zkclient-0.1.jar:"$LIB"/zookeeper-3.3.4.jar
-# echo $CLASSPATH
-
-java -cp "$CLASSPATH" org.apache.helix.recipes.rabbitmq.Consumer $@

http://git-wip-us.apache.org/repos/asf/helix/blob/63f084bd/recipes/rabbitmq-consumer-group/pom.xml
----------------------------------------------------------------------
diff --git a/recipes/rabbitmq-consumer-group/pom.xml b/recipes/rabbitmq-consumer-group/pom.xml
index a70947d..9b749bb 100644
--- a/recipes/rabbitmq-consumer-group/pom.xml
+++ b/recipes/rabbitmq-consumer-group/pom.xml
@@ -37,7 +37,7 @@ under the License.
       org.apache.log4j,
       *
     </osgi.import>
-    <osgi.export>org.apache.helix.rabbitmq*;version="${project.version};-noimport:=true</osgi.export>
+    <osgi.export>org.apache.helix.recipes.rabbitmq*;version="${project.version};-noimport:=true</osgi.export>
   </properties>
 
   <dependencies>
@@ -88,4 +88,32 @@ under the License.
       <version>0.1</version>
     </dependency>
   </dependencies>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>appassembler-maven-plugin</artifactId>
+        <configuration>
+          <programs>
+            <program>
+              <mainClass>org.apache.helix.recipes.rabbitmq.SetupConsumerCluster</mainClass>
+              <name>setup-cluster</name>
+            </program>
+            <program>
+              <mainClass>org.apache.helix.recipes.rabbitmq.StartClusterManager</mainClass>
+              <name>start-cluster-manager</name>
+            </program>
+            <program>
+              <mainClass>org.apache.helix.recipes.rabbitmq.Consumer</mainClass>
+              <name>start-consumer</name>
+            </program>
+            <program>
+              <mainClass>org.apache.helix.recipes.rabbitmq.Emitter</mainClass>
+              <name>send-message</name>
+            </program>
+          </programs>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
 </project>

http://git-wip-us.apache.org/repos/asf/helix/blob/63f084bd/recipes/rabbitmq-consumer-group/src/main/config/log4j.properties
----------------------------------------------------------------------
diff --git a/recipes/rabbitmq-consumer-group/src/main/config/log4j.properties b/recipes/rabbitmq-consumer-group/src/main/config/log4j.properties
new file mode 100644
index 0000000..4b3dc31
--- /dev/null
+++ b/recipes/rabbitmq-consumer-group/src/main/config/log4j.properties
@@ -0,0 +1,31 @@
+#
+# 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.
+#
+
+# Set root logger level to DEBUG and its only appender to A1.
+log4j.rootLogger=ERROR,A1
+
+# A1 is set to be a ConsoleAppender.
+log4j.appender.A1=org.apache.log4j.ConsoleAppender
+
+# A1 uses PatternLayout.
+log4j.appender.A1.layout=org.apache.log4j.PatternLayout
+log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
+
+log4j.logger.org.I0Itec=ERROR
+log4j.logger.org.apache=ERROR


[02/17] git commit: [HELIX-380] Incompatibility issue with HELIX_PROPERTYSTORE, rb=18053

Posted by ka...@apache.org.
[HELIX-380] Incompatibility issue with HELIX_PROPERTYSTORE, rb=18053


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

Branch: refs/heads/master
Commit: 35e3ca10ef16f8e594ee5823978e5041afdbf987
Parents: fdee2dd
Author: zzhang <zz...@uci.edu>
Authored: Thu Feb 13 16:19:46 2014 -0800
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Fri Jul 11 10:42:06 2014 -0700

----------------------------------------------------------------------
 .../apache/helix/manager/zk/ZKHelixManager.java |   7 +-
 .../helix/manager/zk/ZkBaseDataAccessor.java    |   3 +
 .../manager/zk/ZkCacheBaseDataAccessor.java     |   5 +-
 .../store/zk/AutoFallbackPropertyStore.java     | 332 ++++++++++
 .../store/zk/TestAutoFallbackPropertyStore.java | 625 +++++++++++++++++++
 .../zk/TestZkManagerWithAutoFallbackStore.java  | 115 ++++
 6 files changed, 1081 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/35e3ca10/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 d446430..3bc1985 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
@@ -70,6 +70,7 @@ import org.apache.helix.model.builder.ConfigScopeBuilder;
 import org.apache.helix.monitoring.ZKPathDataDumpTask;
 import org.apache.helix.participant.HelixStateMachineEngine;
 import org.apache.helix.participant.StateMachineEngine;
+import org.apache.helix.store.zk.AutoFallbackPropertyStore;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
@@ -622,10 +623,10 @@ public class ZKHelixManager implements HelixManager, IZkStateListener {
 
     if (_helixPropertyStore == null) {
       String path = PropertyPathConfig.getPath(PropertyType.PROPERTYSTORE, _clusterName);
-
+      String fallbackPath = String.format("/%s/%s", _clusterName, "HELIX_PROPERTYSTORE");
       _helixPropertyStore =
-          new ZkHelixPropertyStore<ZNRecord>(new ZkBaseDataAccessor<ZNRecord>(_zkclient), path,
-              null);
+          new AutoFallbackPropertyStore<ZNRecord>(new ZkBaseDataAccessor<ZNRecord>(_zkclient), path,
+              fallbackPath);
     }
 
     return _helixPropertyStore;

http://git-wip-us.apache.org/repos/asf/helix/blob/35e3ca10/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java
index 724e299..e978a23 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java
@@ -84,6 +84,9 @@ public class ZkBaseDataAccessor<T> implements BaseDataAccessor<T> {
   private final ZkClient _zkClient;
 
   public ZkBaseDataAccessor(ZkClient zkClient) {
+    if (zkClient == null) {
+      throw new NullPointerException("zkclient is null");
+    }
     _zkClient = zkClient;
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/35e3ca10/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java
index 57bdc21..1fe5b8d 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java
@@ -506,11 +506,10 @@ public class ZkCacheBaseDataAccessor<T> implements HelixPropertyStore<T> {
   @Override
   public boolean[] exists(List<String> paths, int options) {
     final int size = paths.size();
-    List<String> serverPaths = prependChroot(paths);
 
     boolean exists[] = new boolean[size];
     for (int i = 0; i < size; i++) {
-      exists[i] = exists(serverPaths.get(i), options);
+      exists[i] = exists(paths.get(i), options);
     }
     return exists;
   }
@@ -656,7 +655,7 @@ public class ZkCacheBaseDataAccessor<T> implements HelixPropertyStore<T> {
 
     List<String> paths = new ArrayList<String>();
     for (String childName : childNames) {
-      String path = parentPath + "/" + childName;
+      String path = parentPath.equals("/") ? "/" + childName : parentPath + "/" + childName;
       paths.add(path);
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/35e3ca10/helix-core/src/main/java/org/apache/helix/store/zk/AutoFallbackPropertyStore.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/store/zk/AutoFallbackPropertyStore.java b/helix-core/src/main/java/org/apache/helix/store/zk/AutoFallbackPropertyStore.java
new file mode 100644
index 0000000..02fa5bc
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/store/zk/AutoFallbackPropertyStore.java
@@ -0,0 +1,332 @@
+package org.apache.helix.store.zk;
+
+/*
+ * 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.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.I0Itec.zkclient.DataUpdater;
+import org.apache.helix.AccessOption;
+import org.apache.helix.manager.zk.ZkBaseDataAccessor;
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ * Property store that does auto fallback to an old location.
+ * Assuming no concurrent updates
+ */
+public class AutoFallbackPropertyStore<T> extends ZkHelixPropertyStore<T> {
+  private static Logger LOG = Logger.getLogger(AutoFallbackPropertyStore.class);
+
+  private final ZkHelixPropertyStore<T> _fallbackStore;
+
+  public AutoFallbackPropertyStore(ZkBaseDataAccessor<T> accessor, String root, String fallbackRoot) {
+    super(accessor, root, null);
+
+    if (accessor.exists(fallbackRoot, 0)) {
+      _fallbackStore = new ZkHelixPropertyStore<T>(accessor, fallbackRoot, null);
+    } else {
+      LOG.info("fallbackRoot: " + fallbackRoot
+          + " doesn't exist, skip creating fallback property store");
+      _fallbackStore = null;
+    }
+
+  }
+
+  @Override
+  public boolean update(String path, DataUpdater<T> updater, int options) {
+    if (_fallbackStore == null) {
+      return super.update(path, updater, options);
+    } else {
+      Stat stat = super.getStat(path, options);
+      if (stat == null) {
+        // create znode at new location with fallback-value
+        T fallbackValue = _fallbackStore.get(path, null, options);
+        boolean succeed = super.create(path, fallbackValue, AccessOption.PERSISTENT);
+        if (!succeed) {
+          LOG.error("Can't update " + path + " since there are concurrent updates");
+          return false;
+        }
+      }
+      return super.update(path, updater, options);
+    }
+  }
+
+  @Override
+  public boolean exists(String path, int options) {
+    if (_fallbackStore == null) {
+      return super.exists(path, options);
+    } else {
+      boolean exist = super.exists(path, options);
+      if (!exist) {
+        exist = _fallbackStore.exists(path, options);
+      }
+      return exist;
+    }
+  }
+
+  @Override
+  public boolean remove(String path, int options) {
+    if (_fallbackStore != null) {
+      _fallbackStore.remove(path, options);
+    }
+    return super.remove(path, options);
+  }
+
+  @Override
+  public T get(String path, Stat stat, int options) {
+    if (_fallbackStore == null) {
+      return super.get(path, stat, options);
+    } else {
+      T value = super.get(path, stat, options);
+      if (value == null) {
+        value = _fallbackStore.get(path, stat, options);
+      }
+
+      return value;
+    }
+  }
+
+  @Override
+  public Stat getStat(String path, int options) {
+    if (_fallbackStore == null) {
+      return super.getStat(path, options);
+    } else {
+      Stat stat = super.getStat(path, options);
+
+      if (stat == null) {
+        stat = _fallbackStore.getStat(path, options);
+      }
+      return stat;
+    }
+  }
+
+  @Override
+  public boolean[] updateChildren(List<String> paths, List<DataUpdater<T>> updaters, int options) {
+    if (_fallbackStore == null) {
+      return super.updateChildren(paths, updaters, options);
+    } else {
+      Stat[] stats = super.getStats(paths, options);
+      Map<String, Integer> fallbackMap = new HashMap<String, Integer>();
+      Map<String, Integer> updateMap = new HashMap<String, Integer>();
+      for (int i = 0; i < paths.size(); i++) {
+        String path = paths.get(i);
+        if (stats[i] == null) {
+          fallbackMap.put(path, i);
+        } else {
+          updateMap.put(path, i);
+        }
+      }
+
+      if (fallbackMap.size() > 0) {
+        List<String> fallbackPaths = new ArrayList<String>(fallbackMap.keySet());
+        List<T> fallbackValues = _fallbackStore.get(fallbackPaths, null, options);
+        boolean createSucceed[] =
+            super.createChildren(fallbackPaths, fallbackValues, AccessOption.PERSISTENT);
+
+        for (int i = 0; i < fallbackPaths.size(); i++) {
+          String fallbackPath = fallbackPaths.get(i);
+          if (createSucceed[i]) {
+            updateMap.put(fallbackPath, fallbackMap.get(fallbackPath));
+          } else {
+            LOG.error("Can't update " + fallbackPath + " since there are concurrent updates");
+          }
+        }
+      }
+
+      boolean succeed[] = new boolean[paths.size()]; // all init'ed to false
+      if (updateMap.size() > 0) {
+        List<String> updatePaths = new ArrayList<String>(updateMap.keySet());
+        List<DataUpdater<T>> subUpdaters = new ArrayList<DataUpdater<T>>();
+        for (int i = 0; i < updatePaths.size(); i++) {
+          String updatePath = updatePaths.get(i);
+          subUpdaters.add(updaters.get(updateMap.get(updatePath)));
+        }
+
+        boolean updateSucceed[] = super.updateChildren(updatePaths, subUpdaters, options);
+        for (int i = 0; i < updatePaths.size(); i++) {
+          String updatePath = updatePaths.get(i);
+          if (updateSucceed[i]) {
+            succeed[updateMap.get(updatePath)] = true;
+          }
+        }
+      }
+
+      return succeed;
+    }
+  }
+
+  @Override
+  public boolean[] exists(List<String> paths, int options) {
+    if (_fallbackStore == null) {
+      return super.exists(paths, options);
+    } else {
+      boolean[] exists = super.exists(paths, options);
+
+      Map<String, Integer> fallbackMap = new HashMap<String, Integer>();
+      for (int i = 0; i < paths.size(); i++) {
+        boolean exist = exists[i];
+        if (!exist) {
+          fallbackMap.put(paths.get(i), i);
+        }
+      }
+
+      if (fallbackMap.size() > 0) {
+        List<String> fallbackPaths = new ArrayList<String>(fallbackMap.keySet());
+
+        boolean[] fallbackExists = _fallbackStore.exists(fallbackPaths, options);
+        for (int i = 0; i < fallbackPaths.size(); i++) {
+          String fallbackPath = fallbackPaths.get(i);
+          int j = fallbackMap.get(fallbackPath);
+          exists[j] = fallbackExists[i];
+        }
+      }
+
+      return exists;
+    }
+  }
+
+  @Override
+  public boolean[] remove(List<String> paths, int options) {
+    if (_fallbackStore != null) {
+      _fallbackStore.remove(paths, options);
+    }
+    return super.remove(paths, options);
+  }
+
+  @Override
+  public List<T> get(List<String> paths, List<Stat> stats, int options) {
+    if (_fallbackStore == null) {
+      return super.get(paths, stats, options);
+    } else {
+      List<T> values = super.get(paths, stats, options);
+
+      Map<String, Integer> fallbackMap = new HashMap<String, Integer>();
+      for (int i = 0; i < paths.size(); i++) {
+        T value = values.get(i);
+        if (value == null) {
+          fallbackMap.put(paths.get(i), i);
+        }
+      }
+
+      if (fallbackMap.size() > 0) {
+        List<String> fallbackPaths = new ArrayList<String>(fallbackMap.keySet());
+        List<Stat> fallbackStats = new ArrayList<Stat>();
+        List<T> fallbackValues = _fallbackStore.get(fallbackPaths, fallbackStats, options);
+        for (int i = 0; i < fallbackPaths.size(); i++) {
+          String fallbackPath = fallbackPaths.get(i);
+          int j = fallbackMap.get(fallbackPath);
+          values.set(j, fallbackValues.get(i));
+          if (stats != null) {
+            stats.set(j, fallbackStats.get(i));
+          }
+        }
+      }
+
+      return values;
+    }
+  }
+
+  @Override
+  public Stat[] getStats(List<String> paths, int options) {
+    if (_fallbackStore == null) {
+      return super.getStats(paths, options);
+    } else {
+      Stat[] stats = super.getStats(paths, options);
+
+      Map<String, Integer> fallbackMap = new HashMap<String, Integer>();
+      for (int i = 0; i < paths.size(); i++) {
+        Stat stat = stats[i];
+        if (stat == null) {
+          fallbackMap.put(paths.get(i), i);
+        }
+      }
+
+      if (fallbackMap.size() > 0) {
+        List<String> fallbackPaths = new ArrayList<String>(fallbackMap.keySet());
+
+        Stat[] fallbackStats = _fallbackStore.getStats(fallbackPaths, options);
+        for (int i = 0; i < fallbackPaths.size(); i++) {
+          String fallbackPath = fallbackPaths.get(i);
+          int j = fallbackMap.get(fallbackPath);
+          stats[j] = fallbackStats[i];
+        }
+      }
+
+      return stats;
+    }
+  }
+
+  @Override
+  public List<String> getChildNames(String parentPath, int options) {
+    if (_fallbackStore == null) {
+      return super.getChildNames(parentPath, options);
+    } else {
+      List<String> childs = super.getChildNames(parentPath, options);
+      List<String> fallbackChilds = _fallbackStore.getChildNames(parentPath, options);
+
+      if (childs == null && fallbackChilds == null) {
+        return null;
+      }
+
+      // merge two child lists
+      Set<String> allChildSet = new HashSet<String>();
+      if (childs != null) {
+        allChildSet.addAll(childs);
+      }
+
+      if (fallbackChilds != null) {
+        allChildSet.addAll(fallbackChilds);
+      }
+
+      List<String> allChilds = new ArrayList<String>(allChildSet);
+      return allChilds;
+    }
+  }
+
+  @Override
+  public void start() {
+    if (_fallbackStore != null) {
+      _fallbackStore.start();
+    }
+  }
+
+  @Override
+  public void stop() {
+    if (_fallbackStore != null) {
+      _fallbackStore.stop();
+    }
+
+    super.stop();
+  }
+
+  @Override
+  public void reset() {
+    if (_fallbackStore != null) {
+      _fallbackStore.reset();
+    }
+
+    super.reset();
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/35e3ca10/helix-core/src/test/java/org/apache/helix/store/zk/TestAutoFallbackPropertyStore.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/store/zk/TestAutoFallbackPropertyStore.java b/helix-core/src/test/java/org/apache/helix/store/zk/TestAutoFallbackPropertyStore.java
new file mode 100644
index 0000000..d4cb658
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/store/zk/TestAutoFallbackPropertyStore.java
@@ -0,0 +1,625 @@
+package org.apache.helix.store.zk;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+
+import org.I0Itec.zkclient.DataUpdater;
+import org.apache.helix.AccessOption;
+import org.apache.helix.PropertyType;
+import org.apache.helix.TestHelper;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.ZkUnitTestBase;
+import org.apache.helix.manager.zk.ZkBaseDataAccessor;
+import org.apache.zookeeper.data.Stat;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class TestAutoFallbackPropertyStore extends ZkUnitTestBase {
+
+  class MyDataUpdater implements DataUpdater<ZNRecord> {
+    final String _id;
+
+    public MyDataUpdater(String id) {
+      _id = id;
+    }
+
+    @Override
+    public ZNRecord update(ZNRecord currentData) {
+      if (currentData == null) {
+        currentData = new ZNRecord(_id);
+      } else {
+        currentData.setSimpleField("key", "value");
+      }
+      return currentData;
+    }
+  }
+
+  @Test
+  public void testSingleUpdateUsingFallbackPath() {
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
+    String root = String.format("/%s/%s", clusterName, PropertyType.PROPERTYSTORE.name());
+    String fallbackRoot = String.format("/%s/%s", clusterName, "HELIX_PROPERTYSTORE");
+    ZkBaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<ZNRecord>(_gZkClient);
+
+    // create 0 under fallbackRoot
+    for (int i = 0; i < 1; i++) {
+      String path = String.format("%s/%d", fallbackRoot, i);
+      baseAccessor.create(path, new ZNRecord(Integer.toString(i)), AccessOption.PERSISTENT);
+    }
+
+    AutoFallbackPropertyStore<ZNRecord> store =
+        new AutoFallbackPropertyStore<ZNRecord>(baseAccessor, root, fallbackRoot);
+
+    String path = String.format("/%d", 0);
+    Assert.assertFalse(baseAccessor.exists(String.format("%s%s", root, path), 0),
+        "Should not exist under new location");
+    Assert.assertTrue(baseAccessor.exists(String.format("%s%s", fallbackRoot, path), 0),
+        "Should exist under fallback location");
+
+    boolean succeed = store.update(path, new MyDataUpdater("new0"), AccessOption.PERSISTENT);
+    Assert.assertTrue(succeed);
+
+    // fallback path should remain unchanged
+    ZNRecord record = baseAccessor.get(String.format("%s%s", fallbackRoot, path), null, 0);
+    Assert.assertNotNull(record);
+    Assert.assertEquals(record.getId(), "0");
+    Assert.assertNull(record.getSimpleField("key"));
+
+    // new path should have simple field set
+    record = baseAccessor.get(String.format("%s%s", root, path), null, 0);
+    Assert.assertNotNull(record);
+    Assert.assertEquals(record.getId(), "0");
+    Assert.assertNotNull(record.getSimpleField("key"));
+    Assert.assertEquals(record.getSimpleField("key"), "value");
+
+    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
+  }
+
+  @Test
+  public void testSingleUpdateUsingNewPath() {
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
+    String root = String.format("/%s/%s", clusterName, PropertyType.PROPERTYSTORE.name());
+    String fallbackRoot = String.format("/%s/%s", clusterName, "HELIX_PROPERTYSTORE");
+    ZkBaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<ZNRecord>(_gZkClient);
+
+    // create 0 under both fallbackRoot and root
+    for (int i = 0; i < 1; i++) {
+      String path = String.format("%s/%d", root, i);
+      baseAccessor.create(path, new ZNRecord("new" + i), AccessOption.PERSISTENT);
+
+      path = String.format("%s/%d", fallbackRoot, i);
+      baseAccessor.create(path, new ZNRecord(Integer.toString(i)), AccessOption.PERSISTENT);
+    }
+
+    AutoFallbackPropertyStore<ZNRecord> store =
+        new AutoFallbackPropertyStore<ZNRecord>(baseAccessor, root, fallbackRoot);
+
+    String path = String.format("/%d", 0);
+    Assert.assertTrue(baseAccessor.exists(String.format("%s%s", root, path), 0),
+        "Should exist under new location");
+    Assert.assertTrue(baseAccessor.exists(String.format("%s%s", fallbackRoot, path), 0),
+        "Should exist under fallback location");
+
+    boolean succeed = store.update(path, new MyDataUpdater("0"), AccessOption.PERSISTENT);
+    Assert.assertTrue(succeed);
+
+    ZNRecord record = baseAccessor.get(String.format("%s%s", fallbackRoot, path), null, 0);
+    Assert.assertNotNull(record);
+    Assert.assertEquals(record.getId(), "0");
+    Assert.assertNull(record.getSimpleField("key"));
+
+    record = baseAccessor.get(String.format("%s%s", root, path), null, 0);
+    Assert.assertNotNull(record);
+    Assert.assertEquals(record.getId(), "new0");
+    Assert.assertNotNull(record.getSimpleField("key"));
+    Assert.assertEquals(record.getSimpleField("key"), "value");
+
+    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
+  }
+
+  @Test
+  public void testMultiUpdateUsingFallbackPath() {
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
+    String root = String.format("/%s/%s", clusterName, PropertyType.PROPERTYSTORE.name());
+    String fallbackRoot = String.format("/%s/%s", clusterName, "HELIX_PROPERTYSTORE");
+    ZkBaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<ZNRecord>(_gZkClient);
+
+    // create 0-9 under fallbackRoot
+    for (int i = 0; i < 10; i++) {
+      String path = String.format("%s/%d", fallbackRoot, i);
+      baseAccessor.create(path, new ZNRecord(Integer.toString(i)), AccessOption.PERSISTENT);
+    }
+
+    AutoFallbackPropertyStore<ZNRecord> store =
+        new AutoFallbackPropertyStore<ZNRecord>(baseAccessor, root, fallbackRoot);
+
+    List<String> paths = new ArrayList<String>();
+    List<DataUpdater<ZNRecord>> updaters = new ArrayList<DataUpdater<ZNRecord>>();
+    for (int i = 0; i < 10; i++) {
+      String path = String.format("/%d", i);
+      Assert.assertFalse(baseAccessor.exists(String.format("%s%s", root, path), 0),
+          "Should not exist under new location");
+      Assert.assertTrue(baseAccessor.exists(String.format("%s%s", fallbackRoot, path), 0),
+          "Should exist under fallback location");
+      paths.add(path);
+      updaters.add(new MyDataUpdater("new" + i));
+    }
+
+    boolean succeed[] = store.updateChildren(paths, updaters, AccessOption.PERSISTENT);
+    for (int i = 0; i < 10; i++) {
+      Assert.assertTrue(succeed[i]);
+      String path = paths.get(i);
+
+      // fallback path should remain unchanged
+      ZNRecord record = baseAccessor.get(String.format("%s%s", fallbackRoot, path), null, 0);
+      Assert.assertNotNull(record);
+      Assert.assertEquals(record.getId(), "" + i);
+      Assert.assertNull(record.getSimpleField("key"));
+
+      // new path should have simple field set
+      record = baseAccessor.get(String.format("%s%s", root, path), null, 0);
+      Assert.assertNotNull(record);
+      Assert.assertEquals(record.getId(), "" + i);
+      Assert.assertNotNull(record.getSimpleField("key"));
+      Assert.assertEquals(record.getSimpleField("key"), "value");
+    }
+
+    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
+  }
+
+  @Test
+  public void testMultiUpdateUsingNewath() {
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
+    String root = String.format("/%s/%s", clusterName, PropertyType.PROPERTYSTORE.name());
+    String fallbackRoot = String.format("/%s/%s", clusterName, "HELIX_PROPERTYSTORE");
+    ZkBaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<ZNRecord>(_gZkClient);
+
+    // create 0-9 under both fallbackRoot and new root
+    for (int i = 0; i < 10; i++) {
+      String path = String.format("%s/%d", fallbackRoot, i);
+      baseAccessor.create(path, new ZNRecord(Integer.toString(i)), AccessOption.PERSISTENT);
+
+      path = String.format("%s/%d", root, i);
+      baseAccessor.create(path, new ZNRecord("new" + i), AccessOption.PERSISTENT);
+    }
+
+    AutoFallbackPropertyStore<ZNRecord> store =
+        new AutoFallbackPropertyStore<ZNRecord>(baseAccessor, root, fallbackRoot);
+
+    List<String> paths = new ArrayList<String>();
+    List<DataUpdater<ZNRecord>> updaters = new ArrayList<DataUpdater<ZNRecord>>();
+    for (int i = 0; i < 20; i++) {
+      String path = String.format("/%d", i);
+      if (i < 10) {
+        Assert.assertTrue(baseAccessor.exists(String.format("%s%s", root, path), 0),
+            "Should exist under new location");
+        Assert.assertTrue(baseAccessor.exists(String.format("%s%s", fallbackRoot, path), 0),
+            "Should exist under fallback location");
+      } else {
+        Assert.assertFalse(baseAccessor.exists(String.format("%s%s", root, path), 0),
+            "Should not exist under new location");
+        Assert.assertFalse(baseAccessor.exists(String.format("%s%s", fallbackRoot, path), 0),
+            "Should not exist under fallback location");
+      }
+      paths.add(path);
+      updaters.add(new MyDataUpdater("new" + i));
+    }
+
+    boolean succeed[] = store.updateChildren(paths, updaters, AccessOption.PERSISTENT);
+    for (int i = 0; i < 10; i++) {
+      Assert.assertTrue(succeed[i]);
+      String path = paths.get(i);
+
+      // fallback path should remain unchanged
+      if (i < 10) {
+        ZNRecord record = baseAccessor.get(String.format("%s%s", fallbackRoot, path), null, 0);
+        Assert.assertNotNull(record);
+        Assert.assertEquals(record.getId(), "" + i);
+        Assert.assertNull(record.getSimpleField("key"));
+      } else {
+        Assert.assertFalse(baseAccessor.exists(String.format("%s%s", fallbackRoot, path), 0),
+            "Should not exist under fallback location");
+      }
+
+      // new path should have simple field set
+      ZNRecord record = baseAccessor.get(String.format("%s%s", root, path), null, 0);
+      Assert.assertNotNull(record);
+      Assert.assertEquals(record.getId(), "new" + i);
+      if (i < 10) {
+        Assert.assertNotNull(record.getSimpleField("key"));
+        Assert.assertEquals(record.getSimpleField("key"), "value");
+      } else {
+        Assert.assertNull(record.getSimpleField("key"));
+      }
+
+    }
+
+    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
+  }
+
+  @Test
+  public void testSingleSet() {
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
+    String root = String.format("/%s/%s", clusterName, PropertyType.PROPERTYSTORE.name());
+    String fallbackRoot = String.format("/%s/%s", clusterName, "HELIX_PROPERTYSTORE");
+    ZkBaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<ZNRecord>(_gZkClient);
+
+    // create 0 under fallbackRoot
+    for (int i = 0; i < 1; i++) {
+      String path = String.format("%s/%d", fallbackRoot, i);
+      baseAccessor.create(path, new ZNRecord(Integer.toString(i)), AccessOption.PERSISTENT);
+    }
+
+    AutoFallbackPropertyStore<ZNRecord> store =
+        new AutoFallbackPropertyStore<ZNRecord>(baseAccessor, root, fallbackRoot);
+
+    String path = String.format("/%d", 0);
+    Assert.assertFalse(baseAccessor.exists(String.format("%s%s", root, path), 0),
+        "Should not exist under new location");
+    Assert.assertTrue(baseAccessor.exists(String.format("%s%s", fallbackRoot, path), 0),
+        "Should exist under fallback location");
+    ZNRecord record = new ZNRecord("new0");
+    boolean succeed = store.set(path, record, AccessOption.PERSISTENT);
+    Assert.assertTrue(succeed);
+
+    record = baseAccessor.get(String.format("%s%s", fallbackRoot, path), null, 0);
+    Assert.assertNotNull(record);
+    Assert.assertEquals(record.getId(), "0");
+
+    record = baseAccessor.get(String.format("%s%s", root, path), null, 0);
+    Assert.assertNotNull(record);
+    Assert.assertEquals(record.getId(), "new0");
+
+    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
+  }
+
+  @Test
+  public void testMultiSet() {
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
+    String root = String.format("/%s/%s", clusterName, PropertyType.PROPERTYSTORE.name());
+    String fallbackRoot = String.format("/%s/%s", clusterName, "HELIX_PROPERTYSTORE");
+    ZkBaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<ZNRecord>(_gZkClient);
+
+    // create 0-9 under fallbackRoot
+    for (int i = 0; i < 10; i++) {
+      String path = String.format("%s/%d", fallbackRoot, i);
+      baseAccessor.create(path, new ZNRecord(Integer.toString(i)), AccessOption.PERSISTENT);
+    }
+
+    AutoFallbackPropertyStore<ZNRecord> store =
+        new AutoFallbackPropertyStore<ZNRecord>(baseAccessor, root, fallbackRoot);
+
+    List<String> paths = new ArrayList<String>();
+    List<ZNRecord> records = new ArrayList<ZNRecord>();
+    for (int i = 0; i < 10; i++) {
+      String path = String.format("/%d", i);
+      Assert.assertFalse(baseAccessor.exists(String.format("%s%s", root, path), 0),
+          "Should not exist under new location");
+      Assert.assertTrue(baseAccessor.exists(String.format("%s%s", fallbackRoot, path), 0),
+          "Should exist under fallback location");
+      paths.add(path);
+      ZNRecord record = new ZNRecord("new" + i);
+      records.add(record);
+    }
+
+    boolean succeed[] = store.setChildren(paths, records, AccessOption.PERSISTENT);
+    for (int i = 0; i < 10; i++) {
+      Assert.assertTrue(succeed[i]);
+      String path = String.format("/%d", i);
+      ZNRecord record = baseAccessor.get(String.format("%s%s", fallbackRoot, path), null, 0);
+      Assert.assertNotNull(record);
+      Assert.assertEquals(record.getId(), Integer.toString(i));
+
+      record = baseAccessor.get(String.format("%s%s", root, path), null, 0);
+      Assert.assertNotNull(record);
+      Assert.assertEquals(record.getId(), "new" + i);
+    }
+
+    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
+  }
+
+  @Test
+  public void testSingleGetOnFallbackPath() {
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
+    String root = String.format("/%s/%s", clusterName, PropertyType.PROPERTYSTORE.name());
+    String fallbackRoot = String.format("/%s/%s", clusterName, "HELIX_PROPERTYSTORE");
+    ZkBaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<ZNRecord>(_gZkClient);
+
+    // create 0 under fallbackRoot
+    for (int i = 0; i < 1; i++) {
+      String path = String.format("%s/%d", fallbackRoot, i);
+      baseAccessor.create(path, new ZNRecord(Integer.toString(i)), AccessOption.PERSISTENT);
+    }
+
+    AutoFallbackPropertyStore<ZNRecord> store =
+        new AutoFallbackPropertyStore<ZNRecord>(baseAccessor, root, fallbackRoot);
+
+    String path = String.format("/%d", 0);
+    Assert.assertFalse(baseAccessor.exists(String.format("%s%s", root, path), 0),
+        "Should not exist under new location");
+    Assert.assertTrue(baseAccessor.exists(String.format("%s%s", fallbackRoot, path), 0),
+        "Should exist under fallback location");
+
+    // test single exist
+    boolean exist = store.exists(path, 0);
+    Assert.assertTrue(exist);
+
+    // test single getStat
+    Stat stat = store.getStat(path, 0);
+    Assert.assertNotNull(stat);
+
+    // test single get
+    ZNRecord record = store.get(path, null, 0);
+    Assert.assertNotNull(record);
+    Assert.assertEquals(record.getId(), "0");
+    Assert.assertFalse(baseAccessor.exists(String.format("%s%s", root, path), 0),
+        "Should not exist under new location after get");
+
+    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
+
+  }
+
+  @Test
+  void testMultiGetOnFallbackPath() {
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
+    String root = String.format("/%s/%s", clusterName, PropertyType.PROPERTYSTORE.name());
+    String fallbackRoot = String.format("/%s/%s", clusterName, "HELIX_PROPERTYSTORE");
+    ZkBaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<ZNRecord>(_gZkClient);
+
+    // create 0-9 under fallbackRoot
+    for (int i = 0; i < 10; i++) {
+      String path = String.format("%s/%d", fallbackRoot, i);
+      baseAccessor.create(path, new ZNRecord(Integer.toString(i)), AccessOption.PERSISTENT);
+    }
+
+    AutoFallbackPropertyStore<ZNRecord> store =
+        new AutoFallbackPropertyStore<ZNRecord>(baseAccessor, root, fallbackRoot);
+
+    List<String> paths = new ArrayList<String>();
+    for (int i = 0; i < 10; i++) {
+      String path = String.format("/%d", i);
+      Assert.assertFalse(baseAccessor.exists(String.format("%s%s", root, path), 0),
+          "Should not exist under new location");
+      Assert.assertTrue(baseAccessor.exists(String.format("%s%s", fallbackRoot, path), 0),
+          "Should exist under fallback location");
+      paths.add(path);
+    }
+
+    // test multi-exist
+    boolean exists[] = store.exists(paths, 0);
+    for (int i = 0; i < paths.size(); i++) {
+      Assert.assertTrue(exists[i]);
+    }
+
+    // test multi-getStat
+    Stat stats[] = store.getStats(paths, 0);
+    for (int i = 0; i < paths.size(); i++) {
+      Assert.assertNotNull(stats[i]);
+    }
+
+    // test multi-get
+    List<ZNRecord> records = store.get(paths, null, 0);
+    Assert.assertNotNull(records);
+    Assert.assertEquals(records.size(), 10);
+    for (int i = 0; i < 10; i++) {
+      ZNRecord record = records.get(i);
+      String path = paths.get(i);
+      Assert.assertNotNull(record);
+      Assert.assertEquals(record.getId(), Integer.toString(i));
+      Assert.assertFalse(baseAccessor.exists(String.format("%s%s", root, path), 0),
+          "Should not exist under new location after get");
+    }
+
+    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
+  }
+
+  @Test
+  public void testFailOnSingleGet() {
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
+    String root = String.format("/%s/%s", clusterName, PropertyType.PROPERTYSTORE.name());
+    String fallbackRoot = String.format("/%s/%s", clusterName, "HELIX_PROPERTYSTORE");
+    ZkBaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<ZNRecord>(_gZkClient);
+
+    AutoFallbackPropertyStore<ZNRecord> store =
+        new AutoFallbackPropertyStore<ZNRecord>(baseAccessor, root, fallbackRoot);
+
+    String path = String.format("/%d", 0);
+    Assert.assertFalse(baseAccessor.exists(String.format("%s%s", root, path), 0),
+        "Should not exist under new location");
+    Assert.assertFalse(baseAccessor.exists(String.format("%s%s", fallbackRoot, path), 0),
+        "Should not exist under fallback location");
+
+    // test single exist
+    boolean exist = store.exists(path, 0);
+    Assert.assertFalse(exist);
+
+    // test single getStat
+    Stat stat = store.getStat(path, 0);
+    Assert.assertNull(stat);
+
+    // test single get
+    ZNRecord record = store.get(path, null, 0);
+    Assert.assertNull(record);
+
+    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
+  }
+
+  @Test
+  public void testFailOnMultiGet() {
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
+    String root = String.format("/%s/%s", clusterName, PropertyType.PROPERTYSTORE.name());
+    String fallbackRoot = String.format("/%s/%s", clusterName, "HELIX_PROPERTYSTORE");
+    ZkBaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<ZNRecord>(_gZkClient);
+
+    // create 0-9 under fallbackRoot
+    for (int i = 0; i < 10; i++) {
+      String path = String.format("%s/%d", fallbackRoot, i);
+      baseAccessor.create(path, new ZNRecord(Integer.toString(i)), AccessOption.PERSISTENT);
+    }
+
+    AutoFallbackPropertyStore<ZNRecord> store =
+        new AutoFallbackPropertyStore<ZNRecord>(baseAccessor, root, fallbackRoot);
+
+    List<String> paths = new ArrayList<String>();
+    for (int i = 0; i < 20; i++) {
+      String path = String.format("/%d", i);
+      Assert.assertFalse(baseAccessor.exists(String.format("%s%s", root, path), 0),
+          "Should not exist under new location");
+      if (i < 10) {
+        Assert.assertTrue(baseAccessor.exists(String.format("%s%s", fallbackRoot, path), 0),
+            "Should exist under fallback location");
+      } else {
+        Assert.assertFalse(baseAccessor.exists(String.format("%s%s", fallbackRoot, path), 0),
+            "Should not exist under fallback location");
+      }
+      paths.add(path);
+    }
+
+    // test multi-exist
+    boolean exists[] = store.exists(paths, 0);
+    for (int i = 0; i < paths.size(); i++) {
+      if (i < 10) {
+        Assert.assertTrue(exists[i]);
+      } else {
+        Assert.assertFalse(exists[i]);
+      }
+    }
+
+    // test multi-getStat
+    Stat stats[] = store.getStats(paths, 0);
+    for (int i = 0; i < paths.size(); i++) {
+      if (i < 10) {
+        Assert.assertNotNull(stats[i]);
+      } else {
+        Assert.assertNull(stats[i]);
+      }
+    }
+
+    // test multi-get
+    List<ZNRecord> records = store.get(paths, null, 0);
+    Assert.assertNotNull(records);
+    Assert.assertEquals(records.size(), 20);
+    for (int i = 0; i < 20; i++) {
+      ZNRecord record = records.get(i);
+      String path = paths.get(i);
+      if (i < 10) {
+        Assert.assertNotNull(record);
+        Assert.assertEquals(record.getId(), Integer.toString(i));
+      } else {
+        Assert.assertNull(record);
+      }
+      Assert.assertFalse(baseAccessor.exists(String.format("%s%s", root, path), 0),
+          "Should not exist under new location after get");
+    }
+
+    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
+  }
+
+  @Test
+  public void testGetChildren() {
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
+    String root = String.format("/%s/%s", clusterName, PropertyType.PROPERTYSTORE.name());
+    String fallbackRoot = String.format("/%s/%s", clusterName, "HELIX_PROPERTYSTORE");
+    ZkBaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<ZNRecord>(_gZkClient);
+
+    // create 0-9 under fallbackRoot and 10-19 under root
+    for (int i = 0; i < 20; i++) {
+
+      if (i < 10) {
+        String path = String.format("%s/%d", fallbackRoot, i);
+        baseAccessor.create(path, new ZNRecord(Integer.toString(i)), AccessOption.PERSISTENT);
+      } else {
+        String path = String.format("%s/%d", root, i);
+        baseAccessor.create(path, new ZNRecord(Integer.toString(i)), AccessOption.PERSISTENT);
+      }
+    }
+
+    AutoFallbackPropertyStore<ZNRecord> store =
+        new AutoFallbackPropertyStore<ZNRecord>(baseAccessor, root, fallbackRoot);
+
+    List<String> paths = new ArrayList<String>();
+    for (int i = 0; i < 20; i++) {
+      String path = String.format("/%d", i);
+      if (i < 10) {
+        Assert.assertTrue(baseAccessor.exists(String.format("%s%s", fallbackRoot, path), 0),
+            "Should exist under fallback location");
+        Assert.assertFalse(baseAccessor.exists(String.format("%s%s", root, path), 0),
+            "Should not exist under new location");
+
+      } else {
+        Assert.assertFalse(baseAccessor.exists(String.format("%s%s", fallbackRoot, path), 0),
+            "Should not exist under fallback location");
+        Assert.assertTrue(baseAccessor.exists(String.format("%s%s", root, path), 0),
+            "Should exist under new location");
+
+      }
+      paths.add(path);
+    }
+
+    List<String> childs = store.getChildNames("/", 0);
+    Assert.assertNotNull(childs);
+    Assert.assertEquals(childs.size(), 20);
+    for (int i = 0; i < 20; i++) {
+      String child = childs.get(i);
+      Assert.assertTrue(childs.contains(child));
+    }
+
+    List<ZNRecord> records = store.getChildren("/", null, 0);
+    Assert.assertNotNull(records);
+    Assert.assertEquals(records.size(), 20);
+    for (int i = 0; i < 20; i++) {
+      ZNRecord record = records.get(i);
+      Assert.assertNotNull(record);
+      String id = record.getId();
+      Assert.assertTrue(childs.contains(id));
+    }
+
+    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/35e3ca10/helix-core/src/test/java/org/apache/helix/store/zk/TestZkManagerWithAutoFallbackStore.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/store/zk/TestZkManagerWithAutoFallbackStore.java b/helix-core/src/test/java/org/apache/helix/store/zk/TestZkManagerWithAutoFallbackStore.java
new file mode 100644
index 0000000..549ce70
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/store/zk/TestZkManagerWithAutoFallbackStore.java
@@ -0,0 +1,115 @@
+package org.apache.helix.store.zk;
+
+import java.util.Date;
+
+import org.I0Itec.zkclient.DataUpdater;
+import org.apache.helix.AccessOption;
+import org.apache.helix.BaseDataAccessor;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.TestHelper;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.ZkUnitTestBase;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.ZkBaseDataAccessor;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class TestZkManagerWithAutoFallbackStore extends ZkUnitTestBase {
+  @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", false); // do rebalance
+
+    // start participants
+    MockParticipantManager[] participants = new MockParticipantManager[n];
+    for (int i = 0; i < 1; i++) {
+      String instanceName = "localhost_" + (12918 + i);
+
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+      participants[i].syncStart();
+    }
+
+    // add some data to fallback path: HELIX_PROPERTYSTORE
+    BaseDataAccessor<ZNRecord> accessor =
+        participants[0].getHelixDataAccessor().getBaseDataAccessor();
+    for (int i = 0; i < 10; i++) {
+      String path = String.format("/%s/HELIX_PROPERTYSTORE/%d", clusterName, i);
+      ZNRecord record = new ZNRecord("" + i);
+      record.setSimpleField("key1", "value1");
+      accessor.set(path, record, AccessOption.PERSISTENT);
+    }
+
+    ZkHelixPropertyStore<ZNRecord> store = participants[0].getHelixPropertyStore();
+
+    // read shall use fallback paths
+    for (int i = 0; i < 10; i++) {
+      String path = String.format("/%d", i);
+      ZNRecord record = store.get(path, null, 0);
+      Assert.assertNotNull(record);
+      Assert.assertEquals(record.getId(), "" + i);
+      Assert.assertNotNull(record.getSimpleField("key1"));
+      Assert.assertEquals(record.getSimpleField("key1"), "value1");
+    }
+
+    // update shall update new paths
+    for (int i = 0; i < 10; i++) {
+      String path = String.format("/%d", i);
+      store.update(path, new DataUpdater<ZNRecord>() {
+
+        @Override
+        public ZNRecord update(ZNRecord currentData) {
+          if (currentData != null) {
+            currentData.setSimpleField("key2", "value2");
+          }
+          return currentData;
+        }
+      }, AccessOption.PERSISTENT);
+    }
+
+    for (int i = 0; i < 10; i++) {
+      String path = String.format("/%d", i);
+      ZNRecord record = store.get(path, null, 0);
+      Assert.assertNotNull(record);
+      Assert.assertEquals(record.getId(), "" + i);
+      Assert.assertNotNull(record.getSimpleField("key1"));
+      Assert.assertEquals(record.getSimpleField("key1"), "value1");
+      Assert.assertNotNull(record.getSimpleField("key2"));
+      Assert.assertEquals(record.getSimpleField("key2"), "value2");
+    }
+
+    // set shall use new path
+    for (int i = 10; i < 20; i++) {
+      String path = String.format("/%d", i);
+      ZNRecord record = new ZNRecord("" + i);
+      record.setSimpleField("key3", "value3");
+      store.set(path, record, AccessOption.PERSISTENT);
+    }
+
+    for (int i = 10; i < 20; i++) {
+      String path = String.format("/%d", i);
+      ZNRecord record = store.get(path, null, 0);
+      Assert.assertNotNull(record);
+      Assert.assertEquals(record.getId(), "" + i);
+      Assert.assertNotNull(record.getSimpleField("key3"));
+      Assert.assertEquals(record.getSimpleField("key3"), "value3");
+    }
+
+    participants[0].syncStop();
+    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
+  }
+
+}


[09/17] git commit: [HELIX-395] Remove old Helix alert/stat modules

Posted by ka...@apache.org.
[HELIX-395] Remove old Helix alert/stat modules


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

Branch: refs/heads/master
Commit: 38b439651b4f2ce33f1dbdb96b0f626a1ca49514
Parents: 9524e51
Author: zzhang <zz...@apache.org>
Authored: Tue May 20 13:22:02 2014 -0700
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Fri Jul 11 12:40:20 2014 -0700

----------------------------------------------------------------------
 .../apache/helix/HealthStateChangeListener.java |  39 --
 .../main/java/org/apache/helix/HelixAdmin.java  |  28 --
 .../java/org/apache/helix/HelixConnection.java  |  10 -
 .../java/org/apache/helix/HelixManager.java     |  24 +-
 .../main/java/org/apache/helix/PropertyKey.java |  61 ---
 .../org/apache/helix/PropertyPathConfig.java    |  13 -
 .../java/org/apache/helix/PropertyType.java     |   4 -
 .../helix/alerts/AccumulateAggregator.java      |  67 ---
 .../org/apache/helix/alerts/Aggregator.java     |  40 --
 .../java/org/apache/helix/alerts/Alert.java     |  52 --
 .../apache/helix/alerts/AlertComparator.java    |  30 --
 .../org/apache/helix/alerts/AlertParser.java    | 138 ------
 .../org/apache/helix/alerts/AlertProcessor.java | 308 ------------
 .../helix/alerts/AlertValueAndStatus.java       |  42 --
 .../org/apache/helix/alerts/AlertsHolder.java   | 261 ----------
 .../apache/helix/alerts/DecayAggregator.java    |  76 ---
 .../org/apache/helix/alerts/DivideOperator.java |  40 --
 .../org/apache/helix/alerts/ExpandOperator.java |  39 --
 .../helix/alerts/ExpressionOperatorType.java    |  44 --
 .../apache/helix/alerts/ExpressionParser.java   | 494 ------------------
 .../helix/alerts/GreaterAlertComparator.java    |  45 --
 .../apache/helix/alerts/MultiplyOperator.java   |  60 ---
 .../java/org/apache/helix/alerts/Operator.java  | 111 -----
 .../main/java/org/apache/helix/alerts/Stat.java |  44 --
 .../org/apache/helix/alerts/StatsHolder.java    | 306 ------------
 .../apache/helix/alerts/SumEachOperator.java    |  51 --
 .../org/apache/helix/alerts/SumOperator.java    |  59 ---
 .../java/org/apache/helix/alerts/Tuple.java     |  85 ----
 .../apache/helix/alerts/WindowAggregator.java   |  91 ----
 .../org/apache/helix/alerts/package-info.java   |  22 -
 .../helix/api/accessor/ClusterAccessor.java     | 161 ------
 .../controller/GenericHelixController.java      |  59 +--
 .../controller/stages/HealthDataCache.java      |  95 ----
 .../controller/stages/ReadHealthDataStage.java  |  51 --
 .../stages/StatsAggregationStage.java           | 399 ---------------
 .../healthcheck/AccumulateAggregationType.java  |  41 --
 .../helix/healthcheck/AggregationType.java      |  31 --
 .../healthcheck/AggregationTypeFactory.java     |  69 ---
 .../helix/healthcheck/DecayAggregationType.java |  57 ---
 .../DefaultHealthReportProvider.java            |  86 ----
 .../helix/healthcheck/DefaultPerfCounters.java  |  95 ----
 .../helix/healthcheck/HealthReportProvider.java |  39 --
 .../healthcheck/HealthStatsAggregationTask.java |  89 ----
 .../healthcheck/HealthStatsAggregator.java      | 141 ------
 .../ParticipantHealthReportCollector.java       |  32 --
 .../ParticipantHealthReportCollectorImpl.java   | 115 -----
 .../ParticipantHealthReportTask.java            |  71 ---
 .../PerformanceHealthReportProvider.java        | 138 ------
 .../java/org/apache/helix/healthcheck/Stat.java | 125 -----
 .../healthcheck/StatHealthReportProvider.java   | 159 ------
 .../healthcheck/WindowAggregationType.java      |  67 ---
 .../apache/helix/healthcheck/package-info.java  |  23 -
 .../helix/manager/zk/CallbackHandler.java       |  11 -
 .../manager/zk/HelixConnectionAdaptor.java      |  14 -
 .../manager/zk/ParticipantManagerHelper.java    |  11 -
 .../apache/helix/manager/zk/ZKHelixAdmin.java   | 141 ------
 .../apache/helix/manager/zk/ZKHelixManager.java |  46 +-
 .../helix/manager/zk/ZkCallbackHandler.java     |  11 -
 .../helix/manager/zk/ZkHelixConnection.java     |  11 -
 .../helix/manager/zk/ZkHelixController.java     |   3 -
 .../helix/manager/zk/ZkHelixParticipant.java    |  19 -
 .../java/org/apache/helix/model/HealthStat.java | 147 ------
 .../monitoring/mbeans/ClusterAlertItem.java     |  88 ----
 .../mbeans/ClusterAlertItemMBean.java           |  30 --
 .../mbeans/ClusterAlertMBeanCollection.java     | 275 ----------
 .../org/apache/helix/tools/ClusterSetup.java    |  52 --
 .../org/apache/helix/tools/NewClusterSetup.java |  40 --
 .../java/org/apache/helix/util/HelixUtil.java   |  16 -
 .../src/test/java/org/apache/helix/Mocks.java   | 123 +----
 ...estParticipantHealthReportCollectorImpl.java |  78 ---
 .../java/org/apache/helix/TestPerfCounters.java |  70 ---
 .../TestPerformanceHealthReportProvider.java    | 158 ------
 .../org/apache/helix/alerts/TestAddAlerts.java  | 114 -----
 .../helix/alerts/TestAddPersistentStats.java    | 210 --------
 .../helix/alerts/TestAlertValidation.java       | 166 -------
 .../alerts/TestArrivingParticipantStats.java    | 496 -------------------
 .../helix/alerts/TestBaseStatsValidation.java   | 169 -------
 .../apache/helix/alerts/TestEvaluateAlerts.java | 388 ---------------
 .../org/apache/helix/alerts/TestOperators.java  | 325 ------------
 .../org/apache/helix/alerts/TestStatsMatch.java | 103 ----
 .../controller/stages/DummyClusterManager.java  |  15 -
 .../stages/TestParseInfoFromAlert.java          |  56 ---
 .../helix/healthcheck/TestAddDropAlert.java     | 172 -------
 .../helix/healthcheck/TestDummyAlerts.java      | 148 ------
 .../helix/healthcheck/TestExpandAlert.java      | 186 -------
 .../helix/healthcheck/TestSimpleAlert.java      | 200 --------
 .../healthcheck/TestSimpleWildcardAlert.java    | 246 ---------
 .../helix/healthcheck/TestStalenessAlert.java   | 180 -------
 .../helix/healthcheck/TestWildcardAlert.java    | 296 -----------
 .../helix/manager/zk/TestZkClusterManager.java  |   3 -
 .../MockEspressoHealthReportProvider.java       |  81 ---
 .../MockHealthReportParticipant.java            | 248 ----------
 .../TestClusterAlertItemMBeanCollection.java    | 289 -----------
 .../helix/participant/MockZKHelixManager.java   |  15 -
 94 files changed, 38 insertions(+), 10239 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/HealthStateChangeListener.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/HealthStateChangeListener.java b/helix-core/src/main/java/org/apache/helix/HealthStateChangeListener.java
deleted file mode 100644
index 9202330..0000000
--- a/helix-core/src/main/java/org/apache/helix/HealthStateChangeListener.java
+++ /dev/null
@@ -1,39 +0,0 @@
-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.HealthStat;
-
-/**
- * Interface to implement to listen for when a health status event is triggered.
- */
-public interface HealthStateChangeListener {
-
-  /**
-   * Invoked when health stats change
-   * @param instanceName the instance where the health status changed
-   * @param reports the health statuses
-   * @param changeContext event properties and state
-   */
-  public void onHealthChange(String instanceName, List<HealthStat> reports,
-      NotificationContext changeContext);
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/HelixAdmin.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/HelixAdmin.java b/helix-core/src/main/java/org/apache/helix/HelixAdmin.java
index dce3893..5337cdd 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixAdmin.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixAdmin.java
@@ -243,34 +243,6 @@ public interface HelixAdmin {
   void dropResource(String clusterName, String resourceName);
 
   /**
-   * Add a statistics to a cluster
-   * @param clusterName
-   * @param statName
-   */
-  void addStat(String clusterName, String statName);
-
-  /**
-   * Add an alert to a cluster
-   * @param clusterName
-   * @param alertName
-   */
-  void addAlert(String clusterName, String alertName);
-
-  /**
-   * Drop statistics from a cluster
-   * @param clusterName
-   * @param statName
-   */
-  void dropStat(String clusterName, String statName);
-
-  /**
-   * Drop an alert from a cluster
-   * @param clusterName
-   * @param alertName
-   */
-  void dropAlert(String clusterName, String alertName);
-
-  /**
    * Get a list of state model definitions in a cluster
    * @param clusterName
    * @return

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/HelixConnection.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/HelixConnection.java b/helix-core/src/main/java/org/apache/helix/HelixConnection.java
index 0e674d2..c56b01a 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixConnection.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixConnection.java
@@ -25,7 +25,6 @@ import org.apache.helix.api.accessor.ResourceAccessor;
 import org.apache.helix.api.id.ClusterId;
 import org.apache.helix.api.id.ControllerId;
 import org.apache.helix.api.id.ParticipantId;
-import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.api.id.SessionId;
 import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.store.HelixPropertyStore;
@@ -222,15 +221,6 @@ public interface HelixConnection {
       ClusterId clusterId, ParticipantId participantId, SessionId sessionId);
 
   /**
-   * add health state change listener
-   * @see HealthStateChangeListener#onHealthChange(String, List, NotificationContext)
-   * @param listener
-   * @param instanceName
-   */
-  void addHealthStateChangeListener(HelixRole role, HealthStateChangeListener listener,
-      ClusterId clusterId, ParticipantId participantId);
-
-  /**
    * add external view change listener
    * @see ExternalViewChangeListener#onExternalViewChange(List, NotificationContext)
    * @param listener

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/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 7c34b00..6901715 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixManager.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixManager.java
@@ -22,7 +22,6 @@ package org.apache.helix;
 import java.util.List;
 
 import org.apache.helix.controller.GenericHelixController;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollector;
 import org.apache.helix.manager.zk.ZKHelixManager;
 import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.participant.HelixStateMachineEngine;
@@ -34,7 +33,11 @@ import org.apache.helix.store.zk.ZkHelixPropertyStore;
  * Class that represents the Helix Agent.
  * First class Object any process will interact with<br/>
  * General flow <blockquote>
+<<<<<<< HEAD
  * 
+=======
+ *
+>>>>>>> 77cc651... [HELIX-395] Remove old Helix alert/stat modules
  * <pre>
  * manager = HelixManagerFactory.getZKHelixManager(
  *    clusterName, instanceName, ROLE, zkAddr);
@@ -50,7 +53,11 @@ import org.apache.helix.store.zk.ZkHelixPropertyStore;
  * FINALIZE -> will be invoked when listener is removed or session expires
  * manager.disconnect()
  * </pre>
+<<<<<<< HEAD
  * 
+=======
+ *
+>>>>>>> 77cc651... [HELIX-395] Remove old Helix alert/stat modules
  * </blockquote> Default implementations available
  * @see HelixStateMachineEngine HelixStateMachineEngine for participant
  * @see RoutingTableProvider RoutingTableProvider for spectator
@@ -137,14 +144,6 @@ public interface HelixManager {
       String sessionId) throws Exception;
 
   /**
-   * @see HealthStateChangeListener#onHealthChange(String, List, NotificationContext)
-   * @param listener
-   * @param instanceName
-   */
-  void addHealthStateChangeListener(HealthStateChangeListener listener, String instanceName)
-      throws Exception;
-
-  /**
    * @see ExternalViewChangeListener#onExternalViewChange(List, NotificationContext)
    * @param listener
    */
@@ -237,13 +236,6 @@ public interface HelixManager {
   ClusterMessagingService getMessagingService();
 
   /**
-   * Participant only component that periodically update participant health
-   * report to cluster manager server.
-   * @return ParticipantHealthReportCollector
-   */
-  ParticipantHealthReportCollector getHealthReportCollector();
-
-  /**
    * Get cluster manager instance type
    * @return instance type (e.g. PARTICIPANT, CONTROLLER, SPECTATOR)
    */

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/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 2af63a5..fb3e1d1 100644
--- a/helix-core/src/main/java/org/apache/helix/PropertyKey.java
+++ b/helix-core/src/main/java/org/apache/helix/PropertyKey.java
@@ -19,9 +19,6 @@ package org.apache.helix;
  * under the License.
  */
 
-import static org.apache.helix.PropertyType.ALERTS;
-import static org.apache.helix.PropertyType.ALERT_HISTORY;
-import static org.apache.helix.PropertyType.ALERT_STATUS;
 import static org.apache.helix.PropertyType.CLUSTER;
 import static org.apache.helix.PropertyType.CONFIGS;
 import static org.apache.helix.PropertyType.CONTEXT;
@@ -30,7 +27,6 @@ import static org.apache.helix.PropertyType.CURRENTSTATES;
 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.HEALTHREPORT;
 import static org.apache.helix.PropertyType.HISTORY;
 import static org.apache.helix.PropertyType.IDEALSTATES;
 import static org.apache.helix.PropertyType.LEADER;
@@ -38,7 +34,6 @@ import static org.apache.helix.PropertyType.LIVEINSTANCES;
 import static org.apache.helix.PropertyType.MESSAGES;
 import static org.apache.helix.PropertyType.MESSAGES_CONTROLLER;
 import static org.apache.helix.PropertyType.PAUSE;
-import static org.apache.helix.PropertyType.PERSISTENTSTATS;
 import static org.apache.helix.PropertyType.PROPERTYSTORE;
 import static org.apache.helix.PropertyType.RESOURCEASSIGNMENTS;
 import static org.apache.helix.PropertyType.STATEMODELDEFS;
@@ -48,15 +43,11 @@ import static org.apache.helix.PropertyType.STATUSUPDATES_CONTROLLER;
 import java.util.Arrays;
 
 import org.apache.helix.controller.context.ControllerContextHolder;
-import org.apache.helix.model.AlertHistory;
-import org.apache.helix.model.AlertStatus;
-import org.apache.helix.model.Alerts;
 import org.apache.helix.model.ClusterConfiguration;
 import org.apache.helix.model.ClusterConstraints;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.Error;
 import org.apache.helix.model.ExternalView;
-import org.apache.helix.model.HealthStat;
 import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
@@ -65,7 +56,6 @@ import org.apache.helix.model.LiveInstance;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.PartitionConfiguration;
 import org.apache.helix.model.PauseSignal;
-import org.apache.helix.model.PersistentStats;
 import org.apache.helix.model.ResourceAssignment;
 import org.apache.helix.model.ResourceConfiguration;
 import org.apache.helix.model.StateModelDefinition;
@@ -666,57 +656,6 @@ public class PropertyKey {
     }
 
     /**
-     * Get a property key associated with {@link PersistentStats}
-     * @return {@link PropertyKey}
-     */
-    public PropertyKey persistantStat() {
-      return new PropertyKey(PERSISTENTSTATS, PersistentStats.class, _clusterName);
-    }
-
-    /**
-     * Get a property key associated with {@link Alerts}
-     * @return {@link PropertyKey}
-     */
-    public PropertyKey alerts() {
-      return new PropertyKey(ALERTS, Alerts.class, _clusterName);
-    }
-
-    /**
-     * Get a property key associated with {@link AlertStatus}
-     * @return {@link PropertyKey}
-     */
-    public PropertyKey alertStatus() {
-      return new PropertyKey(ALERT_STATUS, AlertStatus.class, _clusterName);
-    }
-
-    /**
-     * Get a property key associated with {@link AlertHistory}
-     * @return {@link PropertyKey}
-     */
-    public PropertyKey alertHistory() {
-      return new PropertyKey(ALERT_HISTORY, AlertHistory.class, _clusterName);
-    }
-
-    /**
-     * Get a property key associated with a {@link HealthStat} for an instance
-     * @param instanceName
-     * @param id identifies the statistics
-     * @return {@link PropertyKey}
-     */
-    public PropertyKey healthReport(String instanceName, String id) {
-      return new PropertyKey(HEALTHREPORT, HealthStat.class, _clusterName, instanceName, id);
-    }
-
-    /**
-     * Get a property key associated with {@link HealthStat}s for an instance
-     * @param instanceName
-     * @return {@link PropertyKey}
-     */
-    public PropertyKey healthReports(String instanceName) {
-      return new PropertyKey(HEALTHREPORT, HealthStat.class, _clusterName, instanceName);
-    }
-
-    /**
      * Get a propertykey associated with the root of the Helix property store
      * @return {@link PropertyStore}
      */

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/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 d66e7d9..b56a23a 100644
--- a/helix-core/src/main/java/org/apache/helix/PropertyPathConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/PropertyPathConfig.java
@@ -19,13 +19,10 @@ package org.apache.helix;
  * under the License.
  */
 
-import static org.apache.helix.PropertyType.ALERTS;
-import static org.apache.helix.PropertyType.ALERT_STATUS;
 import static org.apache.helix.PropertyType.CONFIGS;
 import static org.apache.helix.PropertyType.CONTEXT;
 import static org.apache.helix.PropertyType.CURRENTSTATES;
 import static org.apache.helix.PropertyType.EXTERNALVIEW;
-import static org.apache.helix.PropertyType.HEALTHREPORT;
 import static org.apache.helix.PropertyType.HISTORY;
 import static org.apache.helix.PropertyType.IDEALSTATES;
 import static org.apache.helix.PropertyType.LIVEINSTANCES;
@@ -42,11 +39,8 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.helix.controller.context.ControllerContextHolder;
-import org.apache.helix.model.AlertStatus;
-import org.apache.helix.model.Alerts;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.ExternalView;
-import org.apache.helix.model.HealthStat;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.LeaderHistory;
@@ -78,9 +72,6 @@ public class PropertyPathConfig {
     typeToClassMapping.put(CURRENTSTATES, CurrentState.class);
     typeToClassMapping.put(STATUSUPDATES, StatusUpdate.class);
     typeToClassMapping.put(HISTORY, LeaderHistory.class);
-    typeToClassMapping.put(HEALTHREPORT, HealthStat.class);
-    typeToClassMapping.put(ALERTS, Alerts.class);
-    typeToClassMapping.put(ALERT_STATUS, AlertStatus.class);
     typeToClassMapping.put(PAUSE, PauseSignal.class);
     typeToClassMapping.put(RESOURCEASSIGNMENTS, ResourceAssignment.class);
     typeToClassMapping.put(CONTEXT, ControllerContextHolder.class);
@@ -147,10 +138,6 @@ public class PropertyPathConfig {
     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");
     addEntry(PropertyType.CONTEXT, 1, "/{clusterName}/CONTROLLER/CONTEXT");
     addEntry(PropertyType.CONTEXT, 2, "/{clusterName}/CONTROLLER/CONTEXT/{contextId}");
     // @formatter:on

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/PropertyType.java
----------------------------------------------------------------------
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 579f454..fd3c5ef 100644
--- a/helix-core/src/main/java/org/apache/helix/PropertyType.java
+++ b/helix-core/src/main/java/org/apache/helix/PropertyType.java
@@ -61,10 +61,6 @@ public enum PropertyType {
   MESSAGES_CONTROLLER(Type.CONTROLLER, true, false, true),
   STATUSUPDATES_CONTROLLER(Type.CONTROLLER, true, true, true),
   ERRORS_CONTROLLER(Type.CONTROLLER, true, true, true),
-  PERSISTENTSTATS(Type.CONTROLLER, true, false, false, false),
-  ALERTS(Type.CONTROLLER, true, false, false, false),
-  ALERT_STATUS(Type.CONTROLLER, true, false, false, false),
-  ALERT_HISTORY(Type.CONTROLLER, true, false, false, false),
   CONTEXT(Type.CONTROLLER, true, false);
 
   // @formatter:on

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/alerts/AccumulateAggregator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/alerts/AccumulateAggregator.java b/helix-core/src/main/java/org/apache/helix/alerts/AccumulateAggregator.java
deleted file mode 100644
index 2967f05..0000000
--- a/helix-core/src/main/java/org/apache/helix/alerts/AccumulateAggregator.java
+++ /dev/null
@@ -1,67 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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 org.apache.helix.HelixException;
-
-public class AccumulateAggregator extends Aggregator {
-
-  public AccumulateAggregator() {
-    _numArgs = 0;
-  }
-
-  @Override
-  public void merge(Tuple<String> currValTup, Tuple<String> newValTup, Tuple<String> currTimeTup,
-      Tuple<String> newTimeTup, String... args) {
-
-    double currVal = 0;
-    double currTime = -1;
-    double newVal;
-    double newTime;
-    double mergedVal;
-    double mergedTime;
-
-    if (currValTup == null || newValTup == null || currTimeTup == null || newTimeTup == null) {
-      throw new HelixException("Tuples cannot be null");
-    }
-
-    // old tuples may be empty, indicating no value/time exist
-    if (currValTup.size() > 0 && currTimeTup.size() > 0) {
-      currVal = Double.parseDouble(currValTup.iterator().next());
-      currTime = Double.parseDouble(currTimeTup.iterator().next());
-    }
-    newVal = Double.parseDouble(newValTup.iterator().next());
-    newTime = Double.parseDouble(newTimeTup.iterator().next());
-
-    if (newTime > currTime) { // if old doesn't exist, we end up here
-      mergedVal = currVal + newVal; // if old doesn't exist, it has value "0"
-      mergedTime = newTime;
-    } else {
-      mergedVal = currVal;
-      mergedTime = currTime;
-    }
-
-    currValTup.clear();
-    currValTup.add(Double.toString(mergedVal));
-    currTimeTup.clear();
-    currTimeTup.add(Double.toString(mergedTime));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/alerts/Aggregator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/alerts/Aggregator.java b/helix-core/src/main/java/org/apache/helix/alerts/Aggregator.java
deleted file mode 100644
index 65a4285..0000000
--- a/helix-core/src/main/java/org/apache/helix/alerts/Aggregator.java
+++ /dev/null
@@ -1,40 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.
- */
-
-public abstract class Aggregator {
-
-  int _numArgs;
-
-  public Aggregator() {
-
-  }
-
-  /*
-   * Take curr and new values. Update curr.
-   */
-  public abstract void merge(Tuple<String> currVal, Tuple<String> newVal, Tuple<String> currTime,
-      Tuple<String> newTime, String... args);
-
-  public int getRequiredNumArgs() {
-    return _numArgs;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/alerts/Alert.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/alerts/Alert.java b/helix-core/src/main/java/org/apache/helix/alerts/Alert.java
deleted file mode 100644
index 17af060..0000000
--- a/helix-core/src/main/java/org/apache/helix/alerts/Alert.java
+++ /dev/null
@@ -1,52 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.
- */
-
-public class Alert {
-
-  String _name;
-
-  String _expression;
-  String _comparator;
-  Tuple<String> _constant;
-
-  public Alert(String name, String expression, String comparator, Tuple<String> constant) {
-    _name = name;
-    _expression = expression;
-    _comparator = comparator;
-    _constant = constant;
-  }
-
-  public String getName() {
-    return _name;
-  }
-
-  public String getExpression() {
-    return _expression;
-  }
-
-  public String getComparator() {
-    return _comparator;
-  }
-
-  public Tuple<String> getConstant() {
-    return _constant;
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/alerts/AlertComparator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/alerts/AlertComparator.java b/helix-core/src/main/java/org/apache/helix/alerts/AlertComparator.java
deleted file mode 100644
index c0330c5..0000000
--- a/helix-core/src/main/java/org/apache/helix/alerts/AlertComparator.java
+++ /dev/null
@@ -1,30 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.
- */
-
-public abstract class AlertComparator {
-
-  public AlertComparator() {
-
-  }
-
-  public abstract boolean evaluate(Tuple<String> leftTup, Tuple<String> rightTup);
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/alerts/AlertParser.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/alerts/AlertParser.java b/helix-core/src/main/java/org/apache/helix/alerts/AlertParser.java
deleted file mode 100644
index ceb4d2e..0000000
--- a/helix-core/src/main/java/org/apache/helix/alerts/AlertParser.java
+++ /dev/null
@@ -1,138 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.HashMap;
-import java.util.Map;
-
-import org.apache.helix.HelixException;
-import org.apache.helix.manager.zk.DefaultParticipantErrorMessageHandlerFactory.ActionOnError;
-import org.apache.log4j.Logger;
-
-public class AlertParser {
-  private static Logger logger = Logger.getLogger(AlertParser.class);
-
-  public static final String EXPRESSION_NAME = "EXP";
-  public static final String COMPARATOR_NAME = "CMP";
-  public static final String CONSTANT_NAME = "CON";
-  public static final String ACTION_NAME = "ACTION";
-
-  static Map<String, AlertComparator> comparatorMap = new HashMap<String, AlertComparator>();
-
-  static {
-
-    addComparatorEntry("GREATER", new GreaterAlertComparator());
-  }
-
-  private static void addComparatorEntry(String label, AlertComparator comp) {
-    if (!comparatorMap.containsKey(label)) {
-      comparatorMap.put(label, comp);
-    }
-    logger.info("Adding comparator: " + comp);
-  }
-
-  public static AlertComparator getComparator(String compName) {
-    compName = compName.replaceAll("\\s+", ""); // remove white space
-    if (!comparatorMap.containsKey(compName)) {
-      throw new HelixException("Comparator type <" + compName + "> unknown");
-    }
-    return comparatorMap.get(compName);
-  }
-
-  public static String getComponent(String component, String alert) throws HelixException {
-    // find EXP and keep going until paren are closed
-    int expStartPos = alert.indexOf(component);
-    if (expStartPos < 0) {
-      throw new HelixException(alert + " does not contain component " + component);
-    }
-    expStartPos += (component.length() + 1); // advance length of string and one for open paren
-    int expEndPos = expStartPos;
-    int openParenCount = 1;
-    while (openParenCount > 0) {
-      if (alert.charAt(expEndPos) == '(') {
-        openParenCount++;
-      } else if (alert.charAt(expEndPos) == ')') {
-        openParenCount--;
-      }
-      expEndPos++;
-    }
-    if (openParenCount != 0) {
-      throw new HelixException(alert + " does not contain valid " + component + " component, "
-          + "parentheses do not close");
-    }
-    // return what is in between paren
-    return alert.substring(expStartPos, expEndPos - 1);
-  }
-
-  public static boolean validateAlert(String alert) throws HelixException {
-    // TODO: decide if toUpperCase is going to cause problems with stuff like db name
-    alert = alert.replaceAll("\\s+", ""); // remove white space
-    String exp = getComponent(EXPRESSION_NAME, alert);
-    String cmp = getComponent(COMPARATOR_NAME, alert);
-    String val = getComponent(CONSTANT_NAME, alert);
-    logger.debug("exp: " + exp);
-    logger.debug("cmp: " + cmp);
-    logger.debug("val: " + val);
-
-    // separately validate each portion
-    ExpressionParser.validateExpression(exp);
-
-    // validate comparator
-    if (!comparatorMap.containsKey(cmp.toUpperCase())) {
-      throw new HelixException("Unknown comparator type " + cmp);
-    }
-    String actionValue = null;
-    try {
-      actionValue = AlertParser.getComponent(AlertParser.ACTION_NAME, alert);
-    } catch (Exception e) {
-      logger.info("No action specified in " + alert);
-    }
-
-    if (actionValue != null) {
-      validateActionValue(actionValue);
-    }
-    // ValParser. Probably don't need this. Just make sure it's a valid tuple. But would also be
-    // good
-    // to validate that the tuple is same length as exp's output...maybe leave that as future todo
-    // not sure we can really do much here though...anything can be in a tuple.
-
-    // TODO: try to compare tuple width of CON against tuple width of agg type! Not a good idea,
-    // what if
-    // is not at full width yet, like with window
-
-    // if all of this passes, then we can safely record the alert in zk. still need to implement zk
-    // location
-
-    return false;
-  }
-
-  public static void validateActionValue(String actionValue) {
-    try {
-      ActionOnError actionVal = ActionOnError.valueOf(actionValue);
-    } catch (Exception e) {
-      String validActions = "";
-      for (ActionOnError action : ActionOnError.values()) {
-        validActions = validActions + action + " ";
-      }
-      throw new HelixException("Unknown cmd type " + actionValue + ", valid types : "
-          + validActions);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/alerts/AlertProcessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/alerts/AlertProcessor.java b/helix-core/src/main/java/org/apache/helix/alerts/AlertProcessor.java
deleted file mode 100644
index 26cabdf..0000000
--- a/helix-core/src/main/java/org/apache/helix/alerts/AlertProcessor.java
+++ /dev/null
@@ -1,308 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.helix.HelixException;
-import org.apache.log4j.Logger;
-
-public class AlertProcessor {
-  private static Logger logger = Logger.getLogger(AlertProcessor.class);
-
-  private static final String bindingDelim = ",";
-  public static final String noWildcardAlertKey = "*";
-
-  StatsHolder _statsHolder;
-
-  // AlertsHolder _alertsHolder;
-
-  /*
-   * public AlertProcessor(StatHealthReportProvider statProvider) {
-   * }
-   */
-
-  public AlertProcessor(StatsHolder sh) {
-    _statsHolder = sh;
-  }
-
-  public static Map<String, List<Tuple<String>>> initAlertStatTuples(Alert alert) {
-    // get the stats out of the alert
-    String[] alertStats = ExpressionParser.getBaseStats(alert.getExpression());
-    // init a tuple list for each alert stat
-    Map<String, List<Tuple<String>>> alertStatTuples = new HashMap<String, List<Tuple<String>>>();
-    for (String currAlertStat : alertStats) {
-      List<Tuple<String>> currList = new ArrayList<Tuple<String>>();
-      alertStatTuples.put(currAlertStat, currList);
-    }
-    return alertStatTuples;
-  }
-
-  /*
-   * //this function is all messed up!!! public static void
-   * populateAlertStatTuples(Map<String,List<Tuple<String>>> tupleLists,
-   * List<Stat> persistentStats) { Set<String> alertStatNames =
-   * tupleLists.keySet(); for (Stat persistentStat : persistentStats) { //ignore
-   * stats with wildcards, they don't have values...they are just there to catch
-   * new actual stats if
-   * (ExpressionParser.statContainsWildcards(persistentStat.getName())) {
-   * continue; } Iterator<String> alertStatIter = alertStatNames.iterator();
-   * while (alertStatIter.hasNext()) { String currAlertStat =
-   * alertStatIter.next(); if
-   * (ExpressionParser.isAlertStatExactMatch(currAlertStat,
-   * persistentStat.getName()) ||
-   * ExpressionParser.isAlertStatWildcardMatch(currAlertStat,
-   * persistentStat.getName())) {
-   * tupleLists.get(currAlertStat).add(persistentStat.getValue()); } } } }
-   */
-
-  public static String formAlertKey(ArrayList<String> bindings) {
-    if (bindings.size() == 0) {
-      return null;
-    }
-    StringBuilder alertKey = new StringBuilder();
-    boolean emptyKey = true;
-    for (String binding : bindings) {
-      if (!emptyKey) {
-        alertKey.append(bindingDelim);
-      }
-      alertKey.append(binding);
-      emptyKey = false;
-    }
-    return alertKey.toString();
-  }
-
-  // XXX: major change here. return ArrayList of Stats instead of ArrayList of
-  // Tuple<String>'s
-  public static Map<String, ArrayList<Tuple<String>>> populateAlertStatTuples(String[] alertStats,
-      List<Stat> persistentStats) {
-    Map<String, ArrayList<Tuple<String>>> tupleSets =
-        new HashMap<String, ArrayList<Tuple<String>>>();
-
-    // check each persistentStat, alertStat pair
-    for (Stat persistentStat : persistentStats) {
-      // ignore stats with wildcards, they don't have values...they are just
-      // there to catch new actual stats
-      if (ExpressionParser.statContainsWildcards(persistentStat.getName())) {
-        continue;
-      }
-      for (int i = 0; i < alertStats.length; i++) {
-        String alertStat = alertStats[i];
-        ArrayList<String> wildcardBindings = new ArrayList<String>();
-        // if match, then proceed. If the match is wildcard, additionally fill
-        // in the wildcard bindings
-        if (ExpressionParser.isAlertStatExactMatch(alertStat, persistentStat.getName())
-            || ExpressionParser.isAlertStatWildcardMatch(alertStat, persistentStat.getName(),
-                wildcardBindings)) {
-          String alertKey;
-          if (wildcardBindings.size() == 0) {
-            alertKey = noWildcardAlertKey;
-          } else {
-            alertKey = formAlertKey(wildcardBindings);
-          }
-          if (!tupleSets.containsKey(alertKey)) { // don't have an entry for alertKey yet, create
-                                                  // one
-            ArrayList<Tuple<String>> tuples = new ArrayList<Tuple<String>>(alertStats.length);
-            for (int j = 0; j < alertStats.length; j++) { // init all entries to null
-              tuples.add(j, null);
-            }
-            tupleSets.put(alertKey, tuples); // add to map
-          }
-          tupleSets.get(alertKey).set(i, persistentStat.getValue());
-        }
-      }
-    }
-
-    // post-processing step to discard any rows with null vals...
-    // TODO: decide if this is best thing to do with incomplete rows
-    List<String> selectedKeysToRemove = new ArrayList<String>();
-    for (String setKey : tupleSets.keySet()) {
-      ArrayList<Tuple<String>> tupleSet = tupleSets.get(setKey);
-      for (Tuple<String> tup : tupleSet) {
-        if (tup == null) {
-          selectedKeysToRemove.add(setKey);
-          break; // move on to next setKey
-        }
-      }
-    }
-    for (String keyToRemove : selectedKeysToRemove) {
-      tupleSets.remove(keyToRemove);
-    }
-
-    // convert above to a series of iterators
-
-    return tupleSets;
-  }
-
-  public static List<Iterator<Tuple<String>>> convertTupleRowsToTupleColumns(
-      Map<String, ArrayList<Tuple<String>>> tupleMap) {
-    // input is a map of key -> list of tuples. each tuple list is same length
-    // output should be a list of iterators. each column in input becomes
-    // iterator in output
-
-    ArrayList<ArrayList<Tuple<String>>> columns = new ArrayList<ArrayList<Tuple<String>>>();
-    ArrayList<Iterator<Tuple<String>>> columnIters = new ArrayList<Iterator<Tuple<String>>>();
-    for (String currStat : tupleMap.keySet()) {
-      List<Tuple<String>> currSet = tupleMap.get(currStat);
-      for (int i = 0; i < currSet.size(); i++) {
-        if (columns.size() < (i + 1)) {
-          ArrayList<Tuple<String>> col = new ArrayList<Tuple<String>>();
-          columns.add(col);
-        }
-        columns.get(i).add(currSet.get(i));
-      }
-    }
-    for (ArrayList<Tuple<String>> al : columns) {
-      columnIters.add(al.iterator());
-    }
-    return columnIters;
-
-  }
-
-  public static Iterator<Tuple<String>> executeOperatorPipeline(
-      List<Iterator<Tuple<String>>> tupleIters, String[] operators) {
-    List<Iterator<Tuple<String>>> nextIters = tupleIters;
-    if (operators != null) {
-      for (String opName : operators) {
-        Operator op = ExpressionParser.getOperator(opName);
-        nextIters = op.execute(nextIters);
-      }
-    }
-
-    if (nextIters.size() != 1) {
-      throw new HelixException("operator pipeline produced " + nextIters.size()
-          + " tuple sets instead of exactly 1");
-    }
-
-    return nextIters.get(0);
-  }
-
-  /*
-   * TODO: consider returning actual values, rather than bools. Could just
-   * return the triggered alerts
-   */
-  public static ArrayList<AlertValueAndStatus> executeComparator(Iterator<Tuple<String>> tuples,
-      String comparatorName, Tuple<String> constant) {
-    ArrayList<AlertValueAndStatus> results = new ArrayList<AlertValueAndStatus>();
-    AlertComparator cmp = AlertParser.getComparator(comparatorName);
-
-    while (tuples.hasNext()) {
-      Tuple<String> currTup = tuples.next();
-      boolean fired = cmp.evaluate(currTup, constant);
-      results.add(new AlertValueAndStatus(currTup, fired));
-      // results.add(cmp.evaluate(currTup, constant));
-    }
-    return results;
-
-  }
-
-  /*
-   * public static void executeAlert(Alert alert, List<Stat> stats) { //init
-   * tuple lists and populate them Map<String,List<Tuple<String>>>
-   * alertStatTupleSets = initAlertStatTuples(alert);
-   * populateAlertStatTuples(alertStatTupleSets, stats); //TODO: not sure I am
-   * being careful enough with sticking stats that match each other in this
-   * list! //convert to operator friendly format List<Iterator<Tuple<String>>>
-   * tupleIters = convertTupleSetsToTupleIterators(alertStatTupleSets); //get
-   * the operators String[] operators =
-   * ExpressionParser.getOperators(alert.getExpression()); //do operator
-   * pipeline Iterator<Tuple<String>> opResultTuples =
-   * executeOperatorPipeline(tupleIters, operators); //execute comparator for
-   * tuple list ArrayList<Boolean> evalResults =
-   * executeComparator(opResultTuples, alert.getComparator(),
-   * alert.getConstant());
-   * //TODO: convey this back to execute all
-   * }
-   */
-
-  public static HashMap<String, AlertValueAndStatus> generateResultMap(
-      Set<String> alertStatBindings, ArrayList<AlertValueAndStatus> evalResults) {
-    HashMap<String, AlertValueAndStatus> resultMap = new HashMap<String, AlertValueAndStatus>();
-    Iterator<String> bindingIter = alertStatBindings.iterator();
-    Iterator<AlertValueAndStatus> resultIter = evalResults.iterator();
-    if (alertStatBindings.size() != evalResults.size()) {
-      // can't match up alerts bindings to results
-      while (resultIter.hasNext()) {
-        resultMap.put(noWildcardAlertKey, resultIter.next());
-      }
-    } else {
-      // they do match up
-      while (resultIter.hasNext()) {
-        resultMap.put(bindingIter.next(), resultIter.next());
-      }
-    }
-    return resultMap;
-  }
-
-  public static HashMap<String, AlertValueAndStatus> executeAlert(Alert alert,
-      List<Stat> persistedStats) {
-    // init tuple lists and populate them
-    // Map<String,List<Tuple<String>>> alertStatTupleSets =
-    // initAlertStatTuples(alert);
-
-    String[] alertStats = ExpressionParser.getBaseStats(alert.getExpression());
-
-    Map<String, ArrayList<Tuple<String>>> alertsToTupleRows =
-        populateAlertStatTuples(alertStats, persistedStats);
-
-    if (alertsToTupleRows.size() == 0) {
-      return null;
-    }
-    // convert to operator friendly format
-    List<Iterator<Tuple<String>>> tupleIters = convertTupleRowsToTupleColumns(alertsToTupleRows);
-    // get the operators
-    String[] operators = ExpressionParser.getOperators(alert.getExpression());
-    // do operator pipeline
-    Iterator<Tuple<String>> opResultTuples = executeOperatorPipeline(tupleIters, operators);
-    // execute comparator for tuple list
-    ArrayList<AlertValueAndStatus> evalResults =
-        executeComparator(opResultTuples, alert.getComparator(), alert.getConstant());
-
-    // stitch alert bindings back together with final result
-    // XXX: there is a non-critical bug here. if we have an aggregating
-    // operator, but that operator only takes one input,
-    // we bind to original wildcard binding, instead of to "*"
-
-    HashMap<String, AlertValueAndStatus> alertBindingsToResult =
-        generateResultMap(alertsToTupleRows.keySet(), evalResults);
-
-    return alertBindingsToResult;
-
-  }
-
-  public static Map<String, Map<String, AlertValueAndStatus>> executeAllAlerts(List<Alert> alerts,
-      List<Stat> stats) {
-    Map<String, Map<String, AlertValueAndStatus>> alertsResults =
-        new HashMap<String, Map<String, AlertValueAndStatus>>();
-
-    for (Alert alert : alerts) {
-      HashMap<String, AlertValueAndStatus> result = executeAlert(alert, stats);
-      // TODO: decide if sticking null results in here is ok
-      alertsResults.put(alert.getName(), result);
-    }
-
-    return alertsResults;
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/alerts/AlertValueAndStatus.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/alerts/AlertValueAndStatus.java b/helix-core/src/main/java/org/apache/helix/alerts/AlertValueAndStatus.java
deleted file mode 100644
index 1582312..0000000
--- a/helix-core/src/main/java/org/apache/helix/alerts/AlertValueAndStatus.java
+++ /dev/null
@@ -1,42 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.
- */
-
-public class AlertValueAndStatus {
-  public final static String VALUE_NAME = "value";
-  public final static String FIRED_NAME = "fired";
-
-  private Tuple<String> value;
-  private boolean fired;
-
-  public AlertValueAndStatus(Tuple<String> value, boolean fired) {
-    this.value = value;
-    this.fired = fired;
-  }
-
-  public Tuple<String> getValue() {
-    return value;
-  }
-
-  public boolean isFired() {
-    return fired;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/alerts/AlertsHolder.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/alerts/AlertsHolder.java b/helix-core/src/main/java/org/apache/helix/alerts/AlertsHolder.java
deleted file mode 100644
index 80f8561..0000000
--- a/helix-core/src/main/java/org/apache/helix/alerts/AlertsHolder.java
+++ /dev/null
@@ -1,261 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.HashMap;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixException;
-import org.apache.helix.HelixManager;
-import org.apache.helix.PropertyKey;
-import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.controller.stages.HealthDataCache;
-import org.apache.helix.model.AlertStatus;
-import org.apache.helix.model.Alerts;
-import org.apache.log4j.Logger;
-
-public class AlertsHolder {
-
-  private static final Logger logger = Logger.getLogger(AlertsHolder.class.getName());
-
-  HelixDataAccessor _accessor;
-  HealthDataCache _cache;
-  Map<String, Map<String, String>> _alertsMap; // not sure if map or set yet
-  Map<String, Map<String, String>> _alertStatusMap;
-  // Alerts _alerts;
-  HashSet<String> alerts;
-  StatsHolder _statsHolder;
-
-  private final HelixManager _manager;
-
-  private Builder _keyBuilder;
-
-  public AlertsHolder(HelixManager manager, HealthDataCache cache) {
-    this(manager, cache, new StatsHolder(manager, cache));
-  }
-
-  public AlertsHolder(HelixManager manager, HealthDataCache cache, StatsHolder statHolder) {
-    _manager = manager;
-    _accessor = manager.getHelixDataAccessor();
-    _cache = cache;
-    _statsHolder = statHolder;
-    _keyBuilder = new PropertyKey.Builder(_manager.getClusterName());
-    updateCache(_cache);
-  }
-
-  public void refreshAlerts() {
-    _cache.refresh(_accessor);
-    updateCache(_cache);
-
-    /*
-     * _alertsMap = _cache.getAlerts();
-     * //TODO: confirm this a good place to init the _statMap when null
-     * if (_alertsMap == null) {
-     * _alertsMap = new HashMap<String, Map<String,String>>();
-     * }\
-     */
-  }
-
-  public void refreshAlertStatus() {
-    AlertStatus alertStatusRecord = _cache.getAlertStatus();
-    if (alertStatusRecord != null) {
-      _alertStatusMap = alertStatusRecord.getMapFields();
-    } else {
-      _alertStatusMap = new HashMap<String, Map<String, String>>();
-    }
-  }
-
-  public void persistAlerts() {
-    // XXX: Am I using _accessor too directly here?
-
-    Alerts alerts = _accessor.getProperty(_keyBuilder.alerts());
-    if (alerts == null) {
-      alerts = new Alerts(Alerts.nodeName); // TODO: fix naming of this record, if it matters
-    }
-    alerts.getRecord().setMapFields(_alertsMap);
-    boolean retVal = _accessor.setProperty(_keyBuilder.alerts(), alerts);
-    logger.debug("persistAlerts retVal: " + retVal);
-  }
-
-  public void persistAlertStatus() {
-    // XXX: Am I using _accessor too directly here?
-    AlertStatus alertStatus = _accessor.getProperty(_keyBuilder.alertStatus());
-    if (alertStatus == null) {
-      alertStatus = new AlertStatus(AlertStatus.nodeName); // TODO: fix naming of this record, if it
-                                                           // matters
-    }
-    alertStatus.getRecord().setMapFields(_alertStatusMap);
-    boolean retVal = _accessor.setProperty(_keyBuilder.alertStatus(), alertStatus);
-    logger.debug("persistAlerts retVal: " + retVal);
-  }
-
-  // read alerts from cm state
-  private void readExistingAlerts() {
-
-  }
-
-  public void addAlert(String alert) throws HelixException {
-    alert = alert.replaceAll("\\s+", ""); // remove white space
-    AlertParser.validateAlert(alert);
-    refreshAlerts();
-    // stick the 3 alert fields in map
-    Map<String, String> alertFields = new HashMap<String, String>();
-    alertFields.put(AlertParser.EXPRESSION_NAME,
-        AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert));
-    alertFields.put(AlertParser.COMPARATOR_NAME,
-        AlertParser.getComponent(AlertParser.COMPARATOR_NAME, alert));
-    alertFields.put(AlertParser.CONSTANT_NAME,
-        AlertParser.getComponent(AlertParser.CONSTANT_NAME, alert));
-    try {
-      alertFields.put(AlertParser.ACTION_NAME,
-          AlertParser.getComponent(AlertParser.ACTION_NAME, alert));
-    } catch (Exception e) {
-      logger.info("No action specified in " + alert);
-    }
-    // store the expression as stat
-    _statsHolder.addStat(alertFields.get(AlertParser.EXPRESSION_NAME));
-    _statsHolder.persistStats();
-
-    // naming the alert with the full name
-    _alertsMap.put(alert, alertFields);
-    persistAlerts();
-  }
-
-  /*
-   * Add a set of alert statuses to ZK
-   */
-  public void addAlertStatusSet(Map<String, Map<String, AlertValueAndStatus>> statusSet)
-      throws HelixException {
-    if (_alertStatusMap == null) {
-      _alertStatusMap = new HashMap<String, Map<String, String>>();
-    }
-    _alertStatusMap.clear(); // clear map. all alerts overwrite old alerts
-    for (String alert : statusSet.keySet()) {
-      Map<String, AlertValueAndStatus> currStatus = statusSet.get(alert);
-      if (currStatus != null) {
-        addAlertStatus(alert, currStatus);
-      }
-    }
-
-    AlertStatus alertStatus = _accessor.getProperty(_keyBuilder.alertStatus());
-    int alertStatusSize = 0;
-    if (alertStatus != null) {
-      alertStatusSize = alertStatus.getMapFields().size();
-    }
-    // no need to persist alerts if there are none to persist and none are currently persisted
-    if (_alertStatusMap.size() > 0 || alertStatusSize > 0) {
-      persistAlertStatus(); // save statuses in zk
-    }
-  }
-
-  private void addAlertStatus(String parentAlertKey, Map<String, AlertValueAndStatus> alertStatus)
-      throws HelixException {
-    // _alertStatusMap = new HashMap<String,Map<String,String>>();
-    for (String alertName : alertStatus.keySet()) {
-      String mapAlertKey;
-      mapAlertKey = parentAlertKey;
-      if (!alertName.equals(ExpressionParser.wildcardChar)) {
-        mapAlertKey = mapAlertKey + " : (" + alertName + ")";
-      }
-      AlertValueAndStatus vs = alertStatus.get(alertName);
-      Map<String, String> alertFields = new HashMap<String, String>();
-      alertFields.put(AlertValueAndStatus.VALUE_NAME, vs.getValue().toString());
-      alertFields.put(AlertValueAndStatus.FIRED_NAME, String.valueOf(vs.isFired()));
-      _alertStatusMap.put(mapAlertKey, alertFields);
-    }
-  }
-
-  public AlertValueAndStatus getAlertValueAndStatus(String alertName) {
-    Map<String, String> alertFields = _alertStatusMap.get(alertName);
-    String val = alertFields.get(AlertValueAndStatus.VALUE_NAME);
-    Tuple<String> valTup = new Tuple<String>();
-    valTup.add(val);
-    boolean fired = Boolean.valueOf(alertFields.get(AlertValueAndStatus.FIRED_NAME));
-    AlertValueAndStatus vs = new AlertValueAndStatus(valTup, fired);
-    return vs;
-  }
-
-  public static void parseAlert(String alert, StringBuilder statsName,
-      Map<String, String> alertFields) throws HelixException {
-    alert = alert.replaceAll("\\s+", ""); // remove white space
-    AlertParser.validateAlert(alert);
-    // alertFields = new HashMap<String,String>();
-    alertFields.put(AlertParser.EXPRESSION_NAME,
-        AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert));
-    alertFields.put(AlertParser.COMPARATOR_NAME,
-        AlertParser.getComponent(AlertParser.COMPARATOR_NAME, alert));
-    alertFields.put(AlertParser.CONSTANT_NAME,
-        AlertParser.getComponent(AlertParser.CONSTANT_NAME, alert));
-    try {
-      alertFields.put(AlertParser.ACTION_NAME,
-          AlertParser.getComponent(AlertParser.ACTION_NAME, alert));
-    } catch (Exception e) {
-      logger.info("No action specified in " + alert);
-    }
-    statsName.append(alertFields.get(AlertParser.EXPRESSION_NAME));
-  }
-
-  /*
-   * public void evaluateAllAlerts()
-   * {
-   * for (String alert : _alertsMap.keySet()) {
-   * Map<String,String> alertFields = _alertsMap.get(alert);
-   * String exp = alertFields.get(AlertParser.EXPRESSION_NAME);
-   * String comp = alertFields.get(AlertParser.COMPARATOR_NAME);
-   * String con = alertFields.get(AlertParser.CONSTANT_NAME);
-   * //TODO: test the fields for null and fail if needed
-   * AlertProcessor.execute(exp, comp, con, sh);
-   * }
-   * }
-   */
-
-  public List<Alert> getAlertList() {
-    List<Alert> alerts = new LinkedList<Alert>();
-    for (String alert : _alertsMap.keySet()) {
-      Map<String, String> alertFields = _alertsMap.get(alert);
-      String exp = alertFields.get(AlertParser.EXPRESSION_NAME);
-      String comp = alertFields.get(AlertParser.COMPARATOR_NAME);
-      Tuple<String> con = Tuple.fromString(alertFields.get(AlertParser.CONSTANT_NAME));
-      // TODO: test the fields for null and fail if needed
-
-      Alert a = new Alert(alert, exp, comp, con);
-      alerts.add(a);
-    }
-    return alerts;
-  }
-
-  public void updateCache(HealthDataCache cache) {
-    _cache = cache;
-    Alerts alertsRecord = _cache.getAlerts();
-    if (alertsRecord != null) {
-      _alertsMap = alertsRecord.getMapFields();
-    } else {
-      _alertsMap = new HashMap<String, Map<String, String>>();
-    }
-  }
-
-  public Map<String, Map<String, String>> getAlertsMap() {
-    return _alertsMap;
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/alerts/DecayAggregator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/alerts/DecayAggregator.java b/helix-core/src/main/java/org/apache/helix/alerts/DecayAggregator.java
deleted file mode 100644
index fd9f2a6..0000000
--- a/helix-core/src/main/java/org/apache/helix/alerts/DecayAggregator.java
+++ /dev/null
@@ -1,76 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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 org.apache.helix.HelixException;
-
-public class DecayAggregator extends Aggregator {
-
-  double _decayWeight;
-
-  public DecayAggregator(double weight) {
-    _decayWeight = weight;
-  }
-
-  public DecayAggregator() {
-    _numArgs = 1;
-  }
-
-  @Override
-  public void merge(Tuple<String> currValTup, Tuple<String> newValTup, Tuple<String> currTimeTup,
-      Tuple<String> newTimeTup, String... args) {
-
-    _decayWeight = Double.parseDouble(args[0]);
-
-    double currVal = 0;
-    double currTime = -1;
-    double newVal;
-    double newTime;
-    double mergedVal;
-    double mergedTime;
-
-    if (currValTup == null || newValTup == null || currTimeTup == null || newTimeTup == null) {
-      throw new HelixException("Tuples cannot be null");
-    }
-
-    // old tuples may be empty, indicating no value/time exist
-    if (currValTup.size() > 0 && currTimeTup.size() > 0) {
-      currVal = Double.parseDouble(currValTup.iterator().next());
-      currTime = Double.parseDouble(currTimeTup.iterator().next());
-    }
-    newVal = Double.parseDouble(newValTup.iterator().next());
-    newTime = Double.parseDouble(newTimeTup.iterator().next());
-
-    if (newTime > currTime) { // if old doesn't exist, we end up here
-      mergedVal = (1 - _decayWeight) * currVal + _decayWeight * newVal; // if old doesn't exist, it
-                                                                        // has value "0"
-      mergedTime = newTime;
-    } else {
-      mergedVal = currVal;
-      mergedTime = currTime;
-    }
-
-    currValTup.clear();
-    currValTup.add(Double.toString(mergedVal));
-    currTimeTup.clear();
-    currTimeTup.add(Double.toString(mergedTime));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/alerts/DivideOperator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/alerts/DivideOperator.java b/helix-core/src/main/java/org/apache/helix/alerts/DivideOperator.java
deleted file mode 100644
index 122d4c1..0000000
--- a/helix-core/src/main/java/org/apache/helix/alerts/DivideOperator.java
+++ /dev/null
@@ -1,40 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.Iterator;
-import java.util.List;
-
-public class DivideOperator extends Operator {
-
-  public DivideOperator() {
-    minInputTupleLists = 2;
-    maxInputTupleLists = 2;
-    inputOutputTupleListsCountsEqual = false;
-    numOutputTupleLists = 1;
-  }
-
-  @Override
-  public List<Iterator<Tuple<String>>> execute(List<Iterator<Tuple<String>>> input) {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/alerts/ExpandOperator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/alerts/ExpandOperator.java b/helix-core/src/main/java/org/apache/helix/alerts/ExpandOperator.java
deleted file mode 100644
index ecd4791..0000000
--- a/helix-core/src/main/java/org/apache/helix/alerts/ExpandOperator.java
+++ /dev/null
@@ -1,39 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.Iterator;
-import java.util.List;
-
-public class ExpandOperator extends Operator {
-
-  public ExpandOperator() {
-    minInputTupleLists = 1;
-    maxInputTupleLists = Integer.MAX_VALUE;
-    inputOutputTupleListsCountsEqual = true;
-  }
-
-  @Override
-  public List<Iterator<Tuple<String>>> execute(List<Iterator<Tuple<String>>> input) {
-    // TODO: confirm this is a no-op operator
-    return input;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/alerts/ExpressionOperatorType.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/alerts/ExpressionOperatorType.java b/helix-core/src/main/java/org/apache/helix/alerts/ExpressionOperatorType.java
deleted file mode 100644
index 6c4f913..0000000
--- a/helix-core/src/main/java/org/apache/helix/alerts/ExpressionOperatorType.java
+++ /dev/null
@@ -1,44 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.
- */
-
-public enum ExpressionOperatorType {
-  // each
-  EACH(true),
-  // standard math
-  SUM(false),
-  MULTIPLY(false),
-  SUBTRACT(false),
-  DIVIDE(false),
-  // aggregation types
-  ACCUMULATE(true),
-  DECAY(false),
-  WINDOW(false);
-
-  boolean isBase;
-
-  private ExpressionOperatorType(boolean isBase) {
-    this.isBase = isBase;
-  }
-
-  boolean isBaseOp() {
-    return isBase;
-  }
-}


[03/17] git commit: [HELIX-380] Incompatibility issue with HELIX_PROPERTYSTORE, HelixConnection port

Posted by ka...@apache.org.
[HELIX-380] Incompatibility issue with HELIX_PROPERTYSTORE, HelixConnection port


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

Branch: refs/heads/master
Commit: 9524e510304152f249b51a19b12d287810be4712
Parents: 35e3ca1
Author: Kanak Biscuitwala <ka...@apache.org>
Authored: Fri Jul 11 10:44:41 2014 -0700
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Fri Jul 11 10:44:41 2014 -0700

----------------------------------------------------------------------
 .../java/org/apache/helix/manager/zk/ZkHelixConnection.java   | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/9524e510/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixConnection.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixConnection.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixConnection.java
index 7bc5fa5..8c9c376 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixConnection.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixConnection.java
@@ -68,7 +68,7 @@ import org.apache.helix.api.id.SessionId;
 import org.apache.helix.messaging.DefaultMessagingService;
 import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.store.HelixPropertyStore;
-import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.helix.store.zk.AutoFallbackPropertyStore;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.Watcher.Event.EventType;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
@@ -251,8 +251,9 @@ public class ZkHelixConnection implements HelixConnection, IZkStateListener {
   @Override
   public HelixPropertyStore<ZNRecord> createPropertyStore(ClusterId clusterId) {
     PropertyKey key = new PropertyKey.Builder(clusterId.stringify()).propertyStore();
-    return new ZkHelixPropertyStore<ZNRecord>(new ZkBaseDataAccessor<ZNRecord>(_zkclient),
-        key.getPath(), null);
+    String fallbackPath = String.format("/%s/%s", clusterId.toString(), "HELIX_PROPERTYSTORE");
+    return new AutoFallbackPropertyStore<ZNRecord>(new ZkBaseDataAccessor<ZNRecord>(_zkclient),
+        key.getPath(), fallbackPath);
   }
 
   @Override


[08/17] [HELIX-395] Remove old Helix alert/stat modules

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/alerts/ExpressionParser.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/alerts/ExpressionParser.java b/helix-core/src/main/java/org/apache/helix/alerts/ExpressionParser.java
deleted file mode 100644
index 2fd95e9..0000000
--- a/helix-core/src/main/java/org/apache/helix/alerts/ExpressionParser.java
+++ /dev/null
@@ -1,494 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.HashMap;
-import java.util.Map;
-import java.util.StringTokenizer;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import org.apache.helix.HelixException;
-import org.apache.log4j.Logger;
-
-public class ExpressionParser {
-  private static Logger logger = Logger.getLogger(ExpressionParser.class);
-
-  final static String opDelim = "|";
-  final static String opDelimForSplit = "\\|";
-  final static String argDelim = ",";
-  final public static String statFieldDelim = ".";
-  final static String wildcardChar = "*";
-
-  // static Map<String, ExpressionOperatorType> operatorMap = new
-  // HashMap<String, ExpressionOperatorType>();
-
-  static Map<String, Operator> operatorMap = new HashMap<String, Operator>();
-  static Map<String, Aggregator> aggregatorMap = new HashMap<String, Aggregator>();
-
-  static {
-
-    addOperatorEntry("EXPAND", new ExpandOperator());
-    addOperatorEntry("DIVIDE", new DivideOperator());
-    addOperatorEntry("SUM", new SumOperator());
-    addOperatorEntry("SUMEACH", new SumEachOperator());
-
-    addAggregatorEntry("ACCUMULATE", new AccumulateAggregator());
-    addAggregatorEntry("DECAY", new DecayAggregator());
-    addAggregatorEntry("WINDOW", new WindowAggregator());
-    /*
-     * addEntry("EACH", ExpressionOperatorType.EACH); addEntry("SUM",
-     * ExpressionOperatorType.SUM); addEntry("DIVIDE",
-     * ExpressionOperatorType.DIVIDE); addEntry("ACCUMULATE",
-     * ExpressionOperatorType.ACCUMULATE);
-     */
-  }
-
-  // static Pattern pattern = Pattern.compile("(\\{.+?\\})");
-
-  private static void addOperatorEntry(String label, Operator op) {
-    if (!operatorMap.containsKey(label)) {
-      operatorMap.put(label, op);
-    }
-    logger.info("Adding operator: " + op);
-  }
-
-  private static void addAggregatorEntry(String label, Aggregator agg) {
-    if (!aggregatorMap.containsKey(label.toUpperCase())) {
-      aggregatorMap.put(label.toUpperCase(), agg);
-    }
-    logger.info("Adding aggregator: " + agg);
-  }
-
-  /*
-   * private static void addEntry(String label, ExpressionOperatorType type) {
-   * if (!operatorMap.containsKey(label)) { operatorMap.put(label, type); }
-   * logger.info("Adding operator type: "+type); }
-   */
-
-  public static boolean isExpressionNested(String expression) {
-    return expression.contains("(");
-  }
-
-  /*
-   * public static Operator getOperatorType(String expression) throws Exception
-   * { String op = expression.substring(0,expression.indexOf("(")); if
-   * (!operatorMap.containsKey(op)) { throw new
-   * Exception(op+" is not a valid op type"); } return operatorMap.get(op); }
-   */
-
-  public static String getInnerExpression(String expression) {
-    return expression.substring(expression.indexOf("(") + 1, expression.lastIndexOf(")"));
-  }
-
-  /*
-   * public static String[] getBaseStats(ExpressionOperatorType type, String
-   * expression) throws Exception { String[] items = null; if
-   * (isExpressionNested(expression)) { ExpressionOperatorType nextType =
-   * getOperatorType(expression); String innerExp =
-   * getInnerExpression(expression); items = getBaseStats(nextType, innerExp); }
-   * else { //base class, no nesting items = expression.split(","); }
-   * if (type != null && type.isBaseOp()) { //surround items with type. for (int
-   * i=0; i<items.length; i++) { items[i] = type + "(" + items[i] + ")"; //!!!!
-   * NEED type to behave like string here
-   * logger.debug("Forming item "+items[i]); } } return items; }
-   * public static String[] getBaseStats(String expression) throws Exception {
-   * expression = expression.replaceAll("\\s+", ""); return getBaseStats(null,
-   * expression); }
-   */
-
-  /*
-   * Validate 2 sets of parenthesis exist, all before first opDelim
-   * extract agg type and validate it exists. validate number of args passed in
-   */
-  public static void validateAggregatorFormat(String expression) throws HelixException {
-    logger.debug("validating aggregator for expression: " + expression);
-    // have 0 or more args, 1 or more stats...e.g. ()(x) or (2)(x,y)
-    Pattern pattern = Pattern.compile("\\(.*?\\)");
-    Matcher matcher = pattern.matcher(expression);
-    String aggComponent = null;
-    String statComponent = null;
-    int lastMatchEnd = -1;
-    if (matcher.find()) {
-      aggComponent = matcher.group();
-      aggComponent = aggComponent.substring(1, aggComponent.length() - 1);
-      if (aggComponent.contains(")") || aggComponent.contains("(")) {
-        throw new HelixException(expression + " has invalid aggregate component");
-      }
-    } else {
-      throw new HelixException(expression + " has invalid aggregate component");
-    }
-    if (matcher.find()) {
-      statComponent = matcher.group();
-      statComponent = statComponent.substring(1, statComponent.length() - 1);
-      // statComponent must have at least 1 arg between paren
-      if (statComponent.contains(")") || statComponent.contains("(") || statComponent.length() == 0) {
-        throw new HelixException(expression + " has invalid stat component");
-      }
-      lastMatchEnd = matcher.end();
-    } else {
-      throw new HelixException(expression + " has invalid stat component");
-    }
-    if (matcher.find()) {
-      throw new HelixException(expression + " has too many parenthesis components");
-    }
-
-    if (expression.length() >= lastMatchEnd + 1) { // lastMatchEnd is pos 1 past the pattern. check
-                                                   // if there are paren there
-      if (expression.substring(lastMatchEnd).contains("(")
-          || expression.substring(lastMatchEnd).contains(")")) {
-        throw new HelixException(expression + " has extra parenthesis");
-      }
-    }
-
-    // check wildcard locations. each part can have at most 1 wildcard, and must
-    // be at end
-    // String expStatNamePart = expression.substring(expression.)
-    StringTokenizer fieldTok = new StringTokenizer(statComponent, statFieldDelim);
-    while (fieldTok.hasMoreTokens()) {
-      String currTok = fieldTok.nextToken();
-      if (currTok.contains(wildcardChar)) {
-        if (currTok.indexOf(wildcardChar) != currTok.length() - 1
-            || currTok.lastIndexOf(wildcardChar) != currTok.length() - 1) {
-          throw new HelixException(currTok
-              + " is illegal stat name.  Single wildcard must appear at end.");
-        }
-      }
-    }
-  }
-
-  public static boolean statContainsWildcards(String stat) {
-    return stat.contains(wildcardChar);
-  }
-
-  /*
-   * Return true if stat name matches exactly...incomingStat has no agg type
-   * currentStat can have any
-   * Function can match for 2 cases extractStatFromAgg=false. Match
-   * accumulate()(dbFoo.partition10.latency) with
-   * accumulate()(dbFoo.partition10.latency)...trival extractStatFromAgg=true.
-   * Match accumulate()(dbFoo.partition10.latency) with
-   * dbFoo.partition10.latency
-   */
-  public static boolean isExactMatch(String currentStat, String incomingStat,
-      boolean extractStatFromAgg) {
-    String currentStatName = currentStat;
-    if (extractStatFromAgg) {
-      currentStatName = getSingleAggregatorStat(currentStat);
-    }
-    return (incomingStat.equals(currentStatName));
-  }
-
-  /*
-   * Return true if incomingStat matches wildcardStat except currentStat has 1+
-   * fields with "*" a*.c* matches a5.c7 a*.c* does not match a5.b6.c7
-   * Function can match for 2 cases extractStatFromAgg=false. Match
-   * accumulate()(dbFoo.partition*.latency) with
-   * accumulate()(dbFoo.partition10.latency) extractStatFromAgg=true. Match
-   * accumulate()(dbFoo.partition*.latency) with dbFoo.partition10.latency
-   */
-  public static boolean isWildcardMatch(String currentStat, String incomingStat,
-      boolean statCompareOnly, ArrayList<String> bindings) {
-    if (!statCompareOnly) { // need to check for match on agg type and stat
-      String currentStatAggType = (currentStat.split("\\)"))[0];
-      String incomingStatAggType = (incomingStat.split("\\)"))[0];
-      if (!currentStatAggType.equals(incomingStatAggType)) {
-        return false;
-      }
-    }
-    // now just get the stats
-    String currentStatName = getSingleAggregatorStat(currentStat);
-    String incomingStatName = getSingleAggregatorStat(incomingStat);
-
-    if (!currentStatName.contains(wildcardChar)) { // no wildcards in stat name
-      return false;
-    }
-
-    String currentStatNamePattern = currentStatName.replace(".", "\\.");
-    currentStatNamePattern = currentStatNamePattern.replace("*", ".*");
-    boolean result = Pattern.matches(currentStatNamePattern, incomingStatName);
-    if (result && bindings != null) {
-      bindings.add(incomingStatName);
-    }
-    return result;
-    /*
-     * StringTokenizer currentStatTok = new StringTokenizer(currentStatName,
-     * statFieldDelim);
-     * StringTokenizer incomingStatTok = new StringTokenizer(incomingStatName,
-     * statFieldDelim);
-     * if (currentStatTok.countTokens() != incomingStatTok.countTokens())
-     * { // stat names different numbers of fields
-     * return false;
-     * }
-     * // for each token, if not wildcarded, must be an exact match
-     * while (currentStatTok.hasMoreTokens())
-     * {
-     * String currTok = currentStatTok.nextToken();
-     * String incomingTok = incomingStatTok.nextToken();
-     * logger.debug("curTok: " + currTok);
-     * logger.debug("incomingTok: " + incomingTok);
-     * if (!currTok.contains(wildcardChar))
-     * { // no wildcard, but have exact match
-     * if (!currTok.equals(incomingTok))
-     * { // not exact match
-     * return false;
-     * }
-     * }
-     * else
-     * { // currTok has a wildcard
-     * if (currTok.indexOf(wildcardChar) != currTok.length() - 1
-     * || currTok.lastIndexOf(wildcardChar) != currTok.length() - 1)
-     * {
-     * throw new HelixException(currTok
-     * + " is illegal stat name.  Single wildcard must appear at end.");
-     * }
-     * // for wildcard matching, need to escape parentheses on currTok, so
-     * // regex works
-     * // currTok = currTok.replace("(", "\\(");
-     * // currTok = currTok.replace(")", "\\)");
-     * // incomingTok = incomingTok.replace("(", "\\(");
-     * // incomingTok = incomingTok.replace(")", "\\)");
-     * String currTokPreWildcard = currTok.substring(0, currTok.length() - 1);
-     * // TODO: if current token has a "(" in it, pattern compiling throws
-     * // error
-     * // Pattern pattern = Pattern.compile(currTokPreWildcard+".+"); //form
-     * // pattern...wildcard part can be anything
-     * // Matcher matcher = pattern.matcher(incomingTok); //see if incomingTok
-     * // matches
-     * if (incomingTok.indexOf(currTokPreWildcard) != 0)
-     * {
-     * // if (!matcher.find()) { //no match on one tok, return false
-     * return false;
-     * }
-     * // get the binding
-     * if (bindings != null)
-     * {
-     * // TODO: debug me!
-     * String wildcardBinding = incomingTok.substring(incomingTok
-     * .indexOf(currTokPreWildcard) + currTokPreWildcard.length());
-     * bindings.add(wildcardBinding);
-     * }
-     * }
-     * }
-     * // all fields match or wildcard match...return true!
-     * return true;
-     */
-  }
-
-  /*
-   * For checking if an incoming stat (no agg type defined) matches a persisted
-   * stat (with agg type defined)
-   */
-  public static boolean isIncomingStatExactMatch(String currentStat, String incomingStat) {
-    return isExactMatch(currentStat, incomingStat, true);
-  }
-
-  /*
-   * For checking if an incoming stat (no agg type defined) wildcard matches a
-   * persisted stat (with agg type defined) The persisted stat may have
-   * wildcards
-   */
-  public static boolean isIncomingStatWildcardMatch(String currentStat, String incomingStat) {
-    return isWildcardMatch(currentStat, incomingStat, true, null);
-  }
-
-  /*
-   * For checking if a persisted stat matches a stat defined in an alert
-   */
-  public static boolean isAlertStatExactMatch(String alertStat, String currentStat) {
-    return isExactMatch(alertStat, currentStat, false);
-  }
-
-  /*
-   * For checking if a maintained stat wildcard matches a stat defined in an
-   * alert. The alert may have wildcards
-   */
-  public static boolean isAlertStatWildcardMatch(String alertStat, String currentStat,
-      ArrayList<String> wildcardBindings) {
-    return isWildcardMatch(alertStat, currentStat, false, wildcardBindings);
-  }
-
-  public static Aggregator getAggregator(String aggStr) throws HelixException {
-    aggStr = aggStr.toUpperCase();
-    Aggregator agg = aggregatorMap.get(aggStr);
-    if (agg == null) {
-      throw new HelixException("Unknown aggregator type " + aggStr);
-    }
-    return agg;
-  }
-
-  public static String getAggregatorStr(String expression) throws HelixException {
-    if (!expression.contains("(")) {
-      throw new HelixException(expression
-          + " does not contain a valid aggregator.  No parentheses found");
-    }
-    String aggName = expression.substring(0, expression.indexOf("("));
-    if (!aggregatorMap.containsKey(aggName.toUpperCase())) {
-      throw new HelixException("aggregator <" + aggName + "> is unknown type");
-    }
-    return aggName;
-  }
-
-  public static String[] getAggregatorArgs(String expression) throws HelixException {
-    String aggregator = getAggregatorStr(expression);
-    String argsStr = getAggregatorArgsStr(expression);
-    String[] args = argsStr.split(argDelim);
-    logger.debug("args size: " + args.length);
-    int numArgs = (argsStr.length() == 0) ? 0 : args.length;
-    // String[] argList = (expression.substring(expression.indexOf("(")+1,
-    // expression.indexOf(")"))).split(argDelim);
-    // verify correct number of args
-    int requiredNumArgs = aggregatorMap.get(aggregator.toUpperCase()).getRequiredNumArgs();
-    if (numArgs != requiredNumArgs) {
-      throw new HelixException(expression + " contains " + args.length
-          + " arguments, but requires " + requiredNumArgs);
-    }
-    return args;
-  }
-
-  /*
-   * public static String[] getAggregatorArgsList(String expression) { String
-   * argsStr = getAggregatorArgsStr(expression); String[] args =
-   * argsStr.split(argDelim); return args; }
-   */
-
-  public static String getAggregatorArgsStr(String expression) {
-    return expression.substring(expression.indexOf("(") + 1, expression.indexOf(")"));
-  }
-
-  public static String[] getAggregatorStats(String expression) throws HelixException {
-    String justStats = expression;
-    if (expression.contains("(") && expression.contains(")")) {
-      justStats =
-          (expression.substring(expression.lastIndexOf("(") + 1, expression.lastIndexOf(")")));
-    }
-    String[] statList = justStats.split(argDelim);
-    if (statList.length < 1) {
-      throw new HelixException(expression + " does not contain any aggregator stats");
-    }
-    return statList;
-  }
-
-  public static String getSingleAggregatorStat(String expression) throws HelixException {
-    String[] stats = getAggregatorStats(expression);
-    if (stats.length > 1) {
-      throw new HelixException(expression + " contains more than 1 stat");
-    }
-    return stats[0];
-  }
-
-  public static String getWildcardStatSubstitution(String wildcardStat, String fixedStat) {
-    int lastOpenParenLoc = wildcardStat.lastIndexOf("(");
-    int lastCloseParenLoc = wildcardStat.lastIndexOf(")");
-    StringBuilder builder = new StringBuilder();
-    builder.append(wildcardStat.substring(0, lastOpenParenLoc + 1));
-    builder.append(fixedStat);
-    builder.append(")");
-    logger.debug("wildcardStat: " + wildcardStat);
-    logger.debug("fixedStat: " + fixedStat);
-    logger.debug("subbedStat: " + builder.toString());
-    return builder.toString();
-  }
-
-  // XXX: each op type should have number of inputs, number of outputs. do
-  // validation.
-  // (dbFoo.partition*.latency, dbFoo.partition*.count)|EACH|ACCUMULATE|DIVIDE
-  public static String[] getBaseStats(String expression) throws HelixException {
-    expression = expression.replaceAll("\\s+", "");
-    validateAggregatorFormat(expression);
-
-    String aggName = getAggregatorStr(expression);
-    String[] aggArgs = getAggregatorArgs(expression);
-    String[] aggStats = getAggregatorStats(expression);
-
-    // form aggArgs
-    String aggArgList = getAggregatorArgsStr(expression);
-
-    String[] baseStats = new String[aggStats.length];
-    for (int i = 0; i < aggStats.length; i++) {
-      StringBuilder stat = new StringBuilder();
-      stat.append(aggName);
-      stat.append("(");
-      stat.append(aggArgList);
-      stat.append(")");
-      stat.append("(");
-      stat.append(aggStats[i]);
-      stat.append(")");
-      baseStats[i] = stat.toString();
-    }
-    return baseStats;
-  }
-
-  public static String[] getOperators(String expression) throws HelixException {
-    String[] ops = null;
-    int numAggStats = (getAggregatorStats(expression)).length;
-    int opDelimLoc = expression.indexOf(opDelim);
-    if (opDelimLoc < 0) {
-      return null;
-    }
-    logger.debug("ops str: " + expression.substring(opDelimLoc + 1));
-    ops = expression.substring(opDelimLoc + 1).split(opDelimForSplit);
-
-    // validate this string of ops
-    // verify each op exists
-    // take num input tuples sets and verify ops will output exactly 1 tuple
-    // sets
-    int currNumTuples = numAggStats;
-    for (String op : ops) {
-      logger.debug("op: " + op);
-      if (!operatorMap.containsKey(op.toUpperCase())) {
-        throw new HelixException("<" + op + "> is not a valid operator type");
-      }
-      Operator currOpType = operatorMap.get(op.toUpperCase());
-      if (currNumTuples < currOpType.minInputTupleLists
-          || currNumTuples > currOpType.maxInputTupleLists) {
-        throw new HelixException("<" + op + "> cannot process " + currNumTuples + " input tuples");
-      }
-      // reset num tuples to this op's output size
-      if (!currOpType.inputOutputTupleListsCountsEqual) { // if equal, this number does not change
-        currNumTuples = currOpType.numOutputTupleLists;
-      }
-    }
-    if (currNumTuples != 1) {
-      throw new HelixException(expression + " does not terminate in a single tuple set");
-    }
-    return ops;
-  }
-
-  public static void validateOperators(String expression) throws HelixException {
-    getOperators(expression);
-  }
-
-  public static Operator getOperator(String opName) throws HelixException {
-    if (!operatorMap.containsKey(opName)) {
-      throw new HelixException(opName + " is unknown op type");
-    }
-    return operatorMap.get(opName);
-  }
-
-  public static void validateExpression(String expression) throws HelixException {
-    // 1. extract stats part and validate
-    validateAggregatorFormat(expression);
-    // 2. extract ops part and validate the ops exist and the inputs/outputs are
-    // correct
-    validateOperators(expression);
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/alerts/GreaterAlertComparator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/alerts/GreaterAlertComparator.java b/helix-core/src/main/java/org/apache/helix/alerts/GreaterAlertComparator.java
deleted file mode 100644
index 0e9c8f1..0000000
--- a/helix-core/src/main/java/org/apache/helix/alerts/GreaterAlertComparator.java
+++ /dev/null
@@ -1,45 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.Iterator;
-
-public class GreaterAlertComparator extends AlertComparator {
-
-  @Override
-  /*
-   * Returns true if any element left tuple exceeds any element in right tuple
-   */
-  public boolean evaluate(Tuple<String> leftTup, Tuple<String> rightTup) {
-    Iterator<String> leftIter = leftTup.iterator();
-    while (leftIter.hasNext()) {
-      double leftVal = Double.parseDouble(leftIter.next());
-      Iterator<String> rightIter = rightTup.iterator();
-      while (rightIter.hasNext()) {
-        double rightVal = Double.parseDouble(rightIter.next());
-        if (leftVal > rightVal) {
-          return true;
-        }
-      }
-    }
-    return false;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/alerts/MultiplyOperator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/alerts/MultiplyOperator.java b/helix-core/src/main/java/org/apache/helix/alerts/MultiplyOperator.java
deleted file mode 100644
index 74a4688..0000000
--- a/helix-core/src/main/java/org/apache/helix/alerts/MultiplyOperator.java
+++ /dev/null
@@ -1,60 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.Iterator;
-import java.util.List;
-
-public class MultiplyOperator extends Operator {
-
-  public MultiplyOperator() {
-    minInputTupleLists = 1;
-    maxInputTupleLists = Integer.MAX_VALUE;
-    inputOutputTupleListsCountsEqual = false;
-    numOutputTupleLists = 1;
-  }
-
-  public List<Iterator<Tuple<String>>> singleSetToIter(ArrayList<Tuple<String>> input) {
-    List out = new ArrayList();
-    out.add(input.iterator());
-    return out;
-  }
-
-  @Override
-  public List<Iterator<Tuple<String>>> execute(List<Iterator<Tuple<String>>> input) {
-    ArrayList<Tuple<String>> output = new ArrayList<Tuple<String>>();
-    if (input == null || input.size() == 0) {
-      return singleSetToIter(output);
-    }
-    while (true) { // loop through set of iters, return when 1 runs out (not completing the row in
-                   // progress)
-      Tuple<String> rowProduct = null;
-      for (Iterator<Tuple<String>> it : input) {
-        if (!it.hasNext()) { // when any iterator runs out, we are done
-          return singleSetToIter(output);
-        }
-        rowProduct = multiplyTuples(rowProduct, it.next());
-      }
-      output.add(rowProduct);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/alerts/Operator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/alerts/Operator.java b/helix-core/src/main/java/org/apache/helix/alerts/Operator.java
deleted file mode 100644
index 0612cf3..0000000
--- a/helix-core/src/main/java/org/apache/helix/alerts/Operator.java
+++ /dev/null
@@ -1,111 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.Iterator;
-import java.util.List;
-
-public abstract class Operator {
-
-  public int minInputTupleLists;
-  public int maxInputTupleLists;
-  public int numOutputTupleLists = -1;
-  public boolean inputOutputTupleListsCountsEqual = false;
-
-  public Operator() {
-
-  }
-
-  public Tuple<String> multiplyTuples(Tuple<String> tup1, Tuple<String> tup2) {
-    if (tup1 == null) {
-      return tup2;
-    }
-    if (tup2 == null) {
-      return tup1;
-    }
-    Tuple<String> outputTup = new Tuple<String>();
-
-    // sum staggers if the tuples are same length
-    // e.g. 1,2,3 + 4,5 = 1,6,8
-    // so this is a bit tricky
-    Tuple<String> largerTup;
-    Tuple<String> smallerTup;
-    if (tup1.size() >= tup2.size()) {
-      largerTup = tup1;
-      smallerTup = tup2;
-    } else {
-      largerTup = tup2;
-      smallerTup = tup1;
-    }
-    int gap = largerTup.size() - smallerTup.size();
-
-    for (int i = 0; i < largerTup.size(); i++) {
-      if (i < gap) {
-        outputTup.add(largerTup.getElement(i));
-      } else {
-        double elementProduct = 0;
-        elementProduct =
-            Double.parseDouble(largerTup.getElement(i))
-                * Double.parseDouble(smallerTup.getElement(i - gap));
-        outputTup.add(String.valueOf(elementProduct));
-      }
-    }
-    return outputTup;
-  }
-
-  public Tuple<String> sumTuples(Tuple<String> tup1, Tuple<String> tup2) {
-    if (tup1 == null) {
-      return tup2;
-    }
-    if (tup2 == null) {
-      return tup1;
-    }
-    Tuple<String> outputTup = new Tuple<String>();
-
-    // sum staggers if the tuples are same length
-    // e.g. 1,2,3 + 4,5 = 1,6,8
-    // so this is a bit tricky
-    Tuple<String> largerTup;
-    Tuple<String> smallerTup;
-    if (tup1.size() >= tup2.size()) {
-      largerTup = tup1;
-      smallerTup = tup2;
-    } else {
-      largerTup = tup2;
-      smallerTup = tup1;
-    }
-    int gap = largerTup.size() - smallerTup.size();
-
-    for (int i = 0; i < largerTup.size(); i++) {
-      if (i < gap) {
-        outputTup.add(largerTup.getElement(i));
-      } else {
-        double elementSum = 0;
-        elementSum =
-            Double.parseDouble(largerTup.getElement(i))
-                + Double.parseDouble(smallerTup.getElement(i - gap));
-        outputTup.add(String.valueOf(elementSum));
-      }
-    }
-    return outputTup;
-  }
-
-  public abstract List<Iterator<Tuple<String>>> execute(List<Iterator<Tuple<String>>> input);
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/alerts/Stat.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/alerts/Stat.java b/helix-core/src/main/java/org/apache/helix/alerts/Stat.java
deleted file mode 100644
index 6895128..0000000
--- a/helix-core/src/main/java/org/apache/helix/alerts/Stat.java
+++ /dev/null
@@ -1,44 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.
- */
-
-public class Stat {
-  String _name;
-  Tuple<String> _value;
-  Tuple<String> _timestamp;
-
-  public Stat(String name, Tuple<String> value, Tuple<String> timestamp) {
-    _name = name;
-    _value = value;
-    _timestamp = timestamp;
-  }
-
-  public String getName() {
-    return _name;
-  }
-
-  public Tuple<String> getValue() {
-    return _value;
-  }
-
-  public Tuple<String> getTimestamp() {
-    return _timestamp;
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/alerts/StatsHolder.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/alerts/StatsHolder.java b/helix-core/src/main/java/org/apache/helix/alerts/StatsHolder.java
deleted file mode 100644
index 1538eb8..0000000
--- a/helix-core/src/main/java/org/apache/helix/alerts/StatsHolder.java
+++ /dev/null
@@ -1,306 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixException;
-import org.apache.helix.HelixManager;
-import org.apache.helix.PropertyKey;
-import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.controller.stages.HealthDataCache;
-import org.apache.helix.model.PersistentStats;
-import org.apache.log4j.Logger;
-
-public class StatsHolder {
-  enum MatchResult {
-    WILDCARDMATCH,
-    EXACTMATCH,
-    NOMATCH
-  };
-
-  private static final Logger logger = Logger.getLogger(StatsHolder.class.getName());
-
-  public static final String VALUE_NAME = "value";
-  public static final String TIMESTAMP_NAME = "TimeStamp";
-
-  HelixDataAccessor _accessor;
-  HealthDataCache _cache;
-
-  Map<String, Map<String, String>> _statMap;
-  Map<String, Map<String, MatchResult>> _statAlertMatchResult;
-
-  private Builder _keyBuilder;
-
-  // PersistentStats _persistentStats;
-
-  public StatsHolder(HelixManager manager, HealthDataCache cache) {
-    _accessor = manager.getHelixDataAccessor();
-    _cache = cache;
-    _keyBuilder = new PropertyKey.Builder(manager.getClusterName());
-    updateCache(_cache);
-    _statAlertMatchResult = new HashMap<String, Map<String, MatchResult>>();
-
-  }
-
-  public void refreshStats() {
-    logger.info("Refreshing cached stats");
-    _cache.refresh(_accessor);
-    updateCache(_cache);
-  }
-
-  public void persistStats() {
-    // XXX: Am I using _accessor too directly here?
-    // took around 35 ms from desktop to ESV4 machine
-    PersistentStats stats = _accessor.getProperty(_keyBuilder.persistantStat());
-    if (stats == null) {
-      stats = new PersistentStats(PersistentStats.nodeName); // TODO: fix naming of
-      // this record, if it
-      // matters
-    }
-    stats.getRecord().setMapFields(_statMap);
-    boolean retVal = _accessor.setProperty(_keyBuilder.persistantStat(), stats);
-  }
-
-  public void getStatsFromCache(boolean refresh) {
-    long refreshStartTime = System.currentTimeMillis();
-    if (refresh) {
-      _cache.refresh(_accessor);
-    }
-    PersistentStats persistentStatRecord = _cache.getPersistentStats();
-    if (persistentStatRecord != null) {
-      _statMap = persistentStatRecord.getMapFields();
-    } else {
-      _statMap = new HashMap<String, Map<String, String>>();
-    }
-    /*
-     * if (_cache.getPersistentStats() != null) {
-     * _statMap = _cache.getPersistentStats();
-     * }
-     */
-    // TODO: confirm this a good place to init the _statMap when null
-    /*
-     * if (_statMap == null) {
-     * _statMap = new HashMap<String, Map<String, String>>();
-     * }
-     */
-    System.out.println("Refresh stats done: " + (System.currentTimeMillis() - refreshStartTime));
-  }
-
-  public Iterator<String> getAllStats() {
-    return null;
-  }
-
-  /*
-   * TODO: figure out pre-conditions here. I think not allowing anything to be
-   * null on input
-   */
-  public Map<String, String> mergeStats(String statName, Map<String, String> existingStat,
-      Map<String, String> incomingStat) throws HelixException {
-    if (existingStat == null) {
-      throw new HelixException("existing stat for merge is null");
-    }
-    if (incomingStat == null) {
-      throw new HelixException("incoming stat for merge is null");
-    }
-    // get agg type and arguments, then get agg object
-    String aggTypeStr = ExpressionParser.getAggregatorStr(statName);
-    String[] aggArgs = ExpressionParser.getAggregatorArgs(statName);
-    Aggregator agg = ExpressionParser.getAggregator(aggTypeStr);
-    // XXX: some of below lines might fail with null exceptions
-
-    // get timestamps, values out of zk maps
-    String existingTime = existingStat.get(TIMESTAMP_NAME);
-    String existingVal = existingStat.get(VALUE_NAME);
-    String incomingTime = incomingStat.get(TIMESTAMP_NAME);
-    String incomingVal = incomingStat.get(VALUE_NAME);
-    // parse values into tuples, if the values exist. else, tuples are null
-    Tuple<String> existingTimeTuple =
-        (existingTime != null) ? Tuple.fromString(existingTime) : null;
-    Tuple<String> existingValueTuple = (existingVal != null) ? Tuple.fromString(existingVal) : null;
-    Tuple<String> incomingTimeTuple =
-        (incomingTime != null) ? Tuple.fromString(incomingTime) : null;
-    Tuple<String> incomingValueTuple = (incomingVal != null) ? Tuple.fromString(incomingVal) : null;
-
-    // dp merge
-    agg.merge(existingValueTuple, incomingValueTuple, existingTimeTuple, incomingTimeTuple, aggArgs);
-    // put merged tuples back in map
-    Map<String, String> mergedMap = new HashMap<String, String>();
-    if (existingTimeTuple.size() == 0) {
-      throw new HelixException("merged time tuple has size zero");
-    }
-    if (existingValueTuple.size() == 0) {
-      throw new HelixException("merged value tuple has size zero");
-    }
-
-    mergedMap.put(TIMESTAMP_NAME, existingTimeTuple.toString());
-    mergedMap.put(VALUE_NAME, existingValueTuple.toString());
-    return mergedMap;
-  }
-
-  /*
-   * Find all persisted stats this stat matches. Update those stats. An incoming
-   * stat can match multiple stats exactly (if that stat has multiple agg types)
-   * An incoming stat can match multiple wildcard stats
-   */
-
-  // need to do a time check here!
-
-  public void applyStat(String incomingStatName, Map<String, String> statFields) {
-    // TODO: consider locking stats here
-    // refreshStats(); //will have refreshed by now during stage
-
-    Map<String, Map<String, String>> pendingAdds = new HashMap<String, Map<String, String>>();
-
-    if (!_statAlertMatchResult.containsKey(incomingStatName)) {
-      _statAlertMatchResult.put(incomingStatName, new HashMap<String, MatchResult>());
-    }
-    Map<String, MatchResult> resultMap = _statAlertMatchResult.get(incomingStatName);
-    // traverse through all persistent stats
-    for (String key : _statMap.keySet()) {
-      if (resultMap.containsKey(key)) {
-        MatchResult cachedMatchResult = resultMap.get(key);
-        if (cachedMatchResult == MatchResult.EXACTMATCH) {
-          processExactMatch(key, statFields);
-        } else if (cachedMatchResult == MatchResult.WILDCARDMATCH) {
-          processWildcardMatch(incomingStatName, key, statFields, pendingAdds);
-        }
-        // don't care about NOMATCH
-        continue;
-      }
-      // exact match on stat and stat portion of persisted stat, just update
-      if (ExpressionParser.isIncomingStatExactMatch(key, incomingStatName)) {
-        processExactMatch(key, statFields);
-        resultMap.put(key, MatchResult.EXACTMATCH);
-      }
-      // wildcard match
-      else if (ExpressionParser.isIncomingStatWildcardMatch(key, incomingStatName)) {
-        processWildcardMatch(incomingStatName, key, statFields, pendingAdds);
-        resultMap.put(key, MatchResult.WILDCARDMATCH);
-      } else {
-        resultMap.put(key, MatchResult.NOMATCH);
-      }
-    }
-    _statMap.putAll(pendingAdds);
-  }
-
-  void processExactMatch(String key, Map<String, String> statFields) {
-    Map<String, String> mergedStat = mergeStats(key, _statMap.get(key), statFields);
-    // update in place, no problem with hash map
-    _statMap.put(key, mergedStat);
-  }
-
-  void processWildcardMatch(String incomingStatName, String key, Map<String, String> statFields,
-      Map<String, Map<String, String>> pendingAdds) {
-
-    // make sure incoming stat doesn't already exist, either in previous
-    // round or this round
-    // form new key (incomingStatName with agg type from the wildcarded
-    // stat)
-    String statToAdd = ExpressionParser.getWildcardStatSubstitution(key, incomingStatName);
-    // if the stat already existed in _statMap, we have/will apply it as an
-    // exact match
-    // if the stat was added this round to pendingAdds, no need to recreate
-    // (it would have same value)
-    if (!_statMap.containsKey(statToAdd) && !pendingAdds.containsKey(statToAdd)) {
-      // add this stat to persisted stats
-      Map<String, String> mergedStat = mergeStats(statToAdd, getEmptyStat(), statFields);
-      // add to pendingAdds so we don't mess up ongoing traversal of
-      // _statMap
-      pendingAdds.put(statToAdd, mergedStat);
-    }
-  }
-
-  // add parsing of stat (or is that in expression holder?) at least add
-  // validate
-  public void addStat(String exp) throws HelixException {
-    refreshStats(); // get current stats
-
-    String[] parsedStats = ExpressionParser.getBaseStats(exp);
-
-    for (String stat : parsedStats) {
-      if (_statMap.containsKey(stat)) {
-        logger.debug("Stat " + stat + " already exists; not adding");
-        continue;
-      }
-      _statMap.put(stat, getEmptyStat()); // add new stat to map
-    }
-  }
-
-  public static Map<String, Map<String, String>> parseStat(String exp) throws HelixException {
-    String[] parsedStats = ExpressionParser.getBaseStats(exp);
-    Map<String, Map<String, String>> statMap = new HashMap<String, Map<String, String>>();
-
-    for (String stat : parsedStats) {
-      if (statMap.containsKey(stat)) {
-        logger.debug("Stat " + stat + " already exists; not adding");
-        continue;
-      }
-      statMap.put(stat, getEmptyStat()); // add new stat to map
-    }
-    return statMap;
-  }
-
-  public static Map<String, String> getEmptyStat() {
-    Map<String, String> statFields = new HashMap<String, String>();
-    statFields.put(TIMESTAMP_NAME, "");
-    statFields.put(VALUE_NAME, "");
-    return statFields;
-  }
-
-  public List<Stat> getStatsList() {
-    List<Stat> stats = new LinkedList<Stat>();
-    for (String stat : _statMap.keySet()) {
-      Map<String, String> statFields = _statMap.get(stat);
-      Tuple<String> valTup = Tuple.fromString(statFields.get(VALUE_NAME));
-      Tuple<String> timeTup = Tuple.fromString(statFields.get(TIMESTAMP_NAME));
-      Stat s = new Stat(stat, valTup, timeTup);
-      stats.add(s);
-    }
-    return stats;
-  }
-
-  public Map<String, Tuple<String>> getStatsMap() {
-    // refreshStats(); //don't refresh, stage will have refreshed by this time
-    HashMap<String, Tuple<String>> stats = new HashMap<String, Tuple<String>>();
-    for (String stat : _statMap.keySet()) {
-      Map<String, String> statFields = _statMap.get(stat);
-      Tuple<String> valTup = Tuple.fromString(statFields.get(VALUE_NAME));
-      Tuple<String> timeTup = Tuple.fromString(statFields.get(TIMESTAMP_NAME));
-      stats.put(stat, valTup);
-    }
-    return stats;
-  }
-
-  public void updateCache(HealthDataCache cache) {
-    _cache = cache;
-    PersistentStats persistentStatRecord = _cache.getPersistentStats();
-    if (persistentStatRecord != null) {
-      _statMap = persistentStatRecord.getMapFields();
-    } else {
-      _statMap = new HashMap<String, Map<String, String>>();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/alerts/SumEachOperator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/alerts/SumEachOperator.java b/helix-core/src/main/java/org/apache/helix/alerts/SumEachOperator.java
deleted file mode 100644
index 2cc733f..0000000
--- a/helix-core/src/main/java/org/apache/helix/alerts/SumEachOperator.java
+++ /dev/null
@@ -1,51 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.Iterator;
-import java.util.List;
-
-public class SumEachOperator extends Operator {
-
-  public SumEachOperator() {
-    minInputTupleLists = 1;
-    maxInputTupleLists = Integer.MAX_VALUE;
-    inputOutputTupleListsCountsEqual = true;
-    numOutputTupleLists = -1;
-  }
-
-  // for each column, generate sum
-  @Override
-  public List<Iterator<Tuple<String>>> execute(List<Iterator<Tuple<String>>> input) {
-    List<Iterator<Tuple<String>>> out = new ArrayList<Iterator<Tuple<String>>>();
-    for (Iterator<Tuple<String>> currIt : input) {
-      Tuple<String> currSum = null;
-      while (currIt.hasNext()) {
-        currSum = sumTuples(currSum, currIt.next());
-      }
-      ArrayList<Tuple<String>> currOutList = new ArrayList<Tuple<String>>();
-      currOutList.add(currSum);
-      out.add(currOutList.iterator());
-    }
-    return out;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/alerts/SumOperator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/alerts/SumOperator.java b/helix-core/src/main/java/org/apache/helix/alerts/SumOperator.java
deleted file mode 100644
index 90c9ab0..0000000
--- a/helix-core/src/main/java/org/apache/helix/alerts/SumOperator.java
+++ /dev/null
@@ -1,59 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.Iterator;
-import java.util.List;
-
-public class SumOperator extends Operator {
-
-  public SumOperator() {
-    minInputTupleLists = 1;
-    maxInputTupleLists = Integer.MAX_VALUE;
-    inputOutputTupleListsCountsEqual = false;
-    numOutputTupleLists = 1;
-  }
-
-  public List<Iterator<Tuple<String>>> singleSetToIter(ArrayList<Tuple<String>> input) {
-    List out = new ArrayList();
-    out.add(input.iterator());
-    return out;
-  }
-
-  @Override
-  public List<Iterator<Tuple<String>>> execute(List<Iterator<Tuple<String>>> input) {
-    ArrayList<Tuple<String>> output = new ArrayList<Tuple<String>>();
-    if (input == null || input.size() == 0) {
-      return singleSetToIter(output);
-    }
-    while (true) { // loop through set of iters, return when 1 runs out (not completing the row in
-                   // progress)
-      Tuple<String> rowSum = null;
-      for (Iterator<Tuple<String>> it : input) {
-        if (!it.hasNext()) { // when any iterator runs out, we are done
-          return singleSetToIter(output);
-        }
-        rowSum = sumTuples(rowSum, it.next());
-      }
-      output.add(rowSum);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/alerts/Tuple.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/alerts/Tuple.java b/helix-core/src/main/java/org/apache/helix/alerts/Tuple.java
deleted file mode 100644
index e57f088..0000000
--- a/helix-core/src/main/java/org/apache/helix/alerts/Tuple.java
+++ /dev/null
@@ -1,85 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.Iterator;
-import java.util.List;
-
-public class Tuple<T> {
-  List<T> elements;
-
-  public Tuple() {
-    elements = new ArrayList<T>();
-  }
-
-  public int size() {
-    return elements.size();
-  }
-
-  public void add(T entry) {
-    elements.add(entry);
-  }
-
-  public void addAll(Tuple<T> incoming) {
-    elements.addAll(incoming.getElements());
-  }
-
-  public Iterator<T> iterator() {
-    return elements.listIterator();
-  }
-
-  public T getElement(int ind) {
-    return elements.get(ind);
-  }
-
-  public List<T> getElements() {
-    return elements;
-  }
-
-  public void clear() {
-    elements.clear();
-  }
-
-  public static Tuple<String> fromString(String in) {
-    Tuple<String> tup = new Tuple<String>();
-    if (in.length() > 0) {
-      String[] elements = in.split(",");
-      for (String element : elements) {
-        tup.add(element);
-      }
-    }
-    return tup;
-  }
-
-  public String toString() {
-    StringBuilder out = new StringBuilder();
-    Iterator<T> it = iterator();
-    boolean outEmpty = true;
-    while (it.hasNext()) {
-      if (!outEmpty) {
-        out.append(",");
-      }
-      out.append(it.next());
-      outEmpty = false;
-    }
-    return out.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/alerts/WindowAggregator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/alerts/WindowAggregator.java b/helix-core/src/main/java/org/apache/helix/alerts/WindowAggregator.java
deleted file mode 100644
index 6ef4cfe..0000000
--- a/helix-core/src/main/java/org/apache/helix/alerts/WindowAggregator.java
+++ /dev/null
@@ -1,91 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.Iterator;
-
-public class WindowAggregator extends Aggregator {
-
-  int _windowSize;
-
-  public WindowAggregator(String windowSize) {
-    _windowSize = Integer.parseInt(windowSize);
-    _numArgs = 1;
-  }
-
-  public WindowAggregator() {
-    this("1");
-  }
-
-  @Override
-  public void merge(Tuple<String> currValTup, Tuple<String> newValTup, Tuple<String> currTimeTup,
-      Tuple<String> newTimeTup, String... args) {
-
-    _windowSize = Integer.parseInt(args[0]);
-
-    // figure out how many curr tuple values we displace
-    Tuple<String> mergedTimeTuple = new Tuple<String>();
-    Tuple<String> mergedValTuple = new Tuple<String>();
-
-    Iterator<String> currTimeIter = currTimeTup.iterator();
-    Iterator<String> currValIter = currValTup.iterator();
-    Iterator<String> newTimeIter = newTimeTup.iterator();
-    Iterator<String> newValIter = newValTup.iterator();
-    int currCtr = 0;
-    // traverse current vals
-    double currTime = -1;
-    double currVal;
-    while (currTimeIter.hasNext()) {
-      currTime = Double.parseDouble(currTimeIter.next());
-      currVal = Double.parseDouble(currValIter.next());
-      currCtr++;
-      // number of evicted currVals equal to total size of both minus _windowSize
-      if (currCtr > (newTimeTup.size() + currTimeTup.size() - _windowSize)) { // non-evicted
-                                                                              // element, just bump
-                                                                              // down
-        mergedTimeTuple.add(String.valueOf(currTime));
-        mergedValTuple.add(String.valueOf(currVal));
-      }
-    }
-
-    double newVal;
-    double newTime;
-    while (newTimeIter.hasNext()) {
-      newVal = Double.parseDouble(newValIter.next());
-      newTime = Double.parseDouble(newTimeIter.next());
-      if (newTime <= currTime) { // oldest new time older than newest curr time. we will not apply
-                                 // new tuple!
-        return; // curr tuples remain the same
-      }
-      currCtr++;
-      if (currCtr > (newTimeTup.size() + currTimeTup.size() - _windowSize)) { // non-evicted element
-        mergedTimeTuple.add(String.valueOf(newTime));
-        mergedValTuple.add(String.valueOf(newVal));
-      }
-    }
-    // set curr tuples to merged tuples
-    currTimeTup.clear();
-    currTimeTup.addAll(mergedTimeTuple);
-    currValTup.clear();
-    currValTup.addAll(mergedValTuple);
-    // TODO: see if we can do merger in place on curr
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/alerts/package-info.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/alerts/package-info.java b/helix-core/src/main/java/org/apache/helix/alerts/package-info.java
deleted file mode 100644
index bf1d9a6..0000000
--- a/helix-core/src/main/java/org/apache/helix/alerts/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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.
- */
-/**
- * Classes for Helix alerts
- */
-package org.apache.helix.alerts;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java b/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java
index c85dd0b..92fb636 100644
--- a/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java
@@ -26,15 +26,10 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.I0Itec.zkclient.DataUpdater;
 import org.apache.helix.AccessOption;
 import org.apache.helix.BaseDataAccessor;
 import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixException;
 import org.apache.helix.PropertyKey;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.alerts.AlertsHolder;
-import org.apache.helix.alerts.StatsHolder;
 import org.apache.helix.api.Cluster;
 import org.apache.helix.api.Controller;
 import org.apache.helix.api.Participant;
@@ -61,7 +56,6 @@ import org.apache.helix.controller.rebalancer.config.PartitionedRebalancerConfig
 import org.apache.helix.controller.rebalancer.config.RebalancerConfig;
 import org.apache.helix.controller.rebalancer.config.RebalancerConfigHolder;
 import org.apache.helix.controller.stages.ClusterDataCache;
-import org.apache.helix.model.Alerts;
 import org.apache.helix.model.ClusterConfiguration;
 import org.apache.helix.model.ClusterConstraints;
 import org.apache.helix.model.ClusterConstraints.ConstraintType;
@@ -73,7 +67,6 @@ import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.PauseSignal;
-import org.apache.helix.model.PersistentStats;
 import org.apache.helix.model.ProvisionerConfigHolder;
 import org.apache.helix.model.ResourceAssignment;
 import org.apache.helix.model.ResourceConfiguration;
@@ -521,14 +514,6 @@ public class ClusterAccessor {
   }
 
   /**
-   * Get the stats persisted on this cluster
-   * @return PersistentStats, or null if none persisted
-   */
-  public PersistentStats readStats() {
-    return _accessor.getProperty(_keyBuilder.persistantStat());
-  }
-
-  /**
    * Read the persisted controller contexts
    * @return map of context id to controller context
    */
@@ -556,152 +541,6 @@ public class ClusterAccessor {
   }
 
   /**
-   * Get the current cluster stats
-   * @return PersistentStats
-   */
-  public PersistentStats getStats() {
-    return _accessor.getProperty(_keyBuilder.persistantStat());
-  }
-
-  /**
-   * Get the current cluster alerts
-   * @return Alerts
-   */
-  public Alerts getAlerts() {
-    return _accessor.getProperty(_keyBuilder.alerts());
-  }
-
-  /**
-   * Add a statistic specification to the cluster. Existing stat specifications will not be
-   * overwritten
-   * @param statName string representing a stat specification
-   * @return true if the stat spec was added, false otherwise
-   */
-  public boolean addStat(final String statName) {
-    if (!isClusterStructureValid()) {
-      LOG.error("cluster " + _clusterId + " is not setup yet");
-      return false;
-    }
-
-    String persistentStatsPath = _keyBuilder.persistantStat().getPath();
-    BaseDataAccessor<ZNRecord> baseAccessor = _accessor.getBaseDataAccessor();
-    return baseAccessor.update(persistentStatsPath, new DataUpdater<ZNRecord>() {
-      @Override
-      public ZNRecord update(ZNRecord statsRec) {
-        if (statsRec == null) {
-          statsRec = new ZNRecord(PersistentStats.nodeName);
-        }
-        Map<String, Map<String, String>> currStatMap = statsRec.getMapFields();
-        Map<String, Map<String, String>> newStatMap = StatsHolder.parseStat(statName);
-        for (String newStat : newStatMap.keySet()) {
-          if (!currStatMap.containsKey(newStat)) {
-            currStatMap.put(newStat, newStatMap.get(newStat));
-          }
-        }
-        statsRec.setMapFields(currStatMap);
-        return statsRec;
-      }
-    }, AccessOption.PERSISTENT);
-  }
-
-  /**
-   * Remove a statistic specification from the cluster
-   * @param statName string representing a statistic specification
-   * @return true if stats removed, false otherwise
-   */
-  public boolean dropStat(final String statName) {
-    if (!isClusterStructureValid()) {
-      LOG.error("cluster " + _clusterId + " is not setup yet");
-      return false;
-    }
-
-    String persistentStatsPath = _keyBuilder.persistantStat().getPath();
-    BaseDataAccessor<ZNRecord> baseAccessor = _accessor.getBaseDataAccessor();
-    return baseAccessor.update(persistentStatsPath, new DataUpdater<ZNRecord>() {
-      @Override
-      public ZNRecord update(ZNRecord statsRec) {
-        if (statsRec == null) {
-          throw new HelixException("No stats record in ZK, nothing to drop");
-        }
-        Map<String, Map<String, String>> currStatMap = statsRec.getMapFields();
-        Map<String, Map<String, String>> newStatMap = StatsHolder.parseStat(statName);
-        // delete each stat from stat map
-        for (String newStat : newStatMap.keySet()) {
-          if (currStatMap.containsKey(newStat)) {
-            currStatMap.remove(newStat);
-          }
-        }
-        statsRec.setMapFields(currStatMap);
-        return statsRec;
-      }
-    }, AccessOption.PERSISTENT);
-  }
-
-  /**
-   * Add an alert specification to the cluster
-   * @param alertName string representing the alert spec
-   * @return true if added, false otherwise
-   */
-  public boolean addAlert(final String alertName) {
-    if (!isClusterStructureValid()) {
-      LOG.error("cluster " + _clusterId + " is not setup yet");
-      return false;
-    }
-
-    BaseDataAccessor<ZNRecord> baseAccessor = _accessor.getBaseDataAccessor();
-    String alertsPath = _keyBuilder.alerts().getPath();
-    return baseAccessor.update(alertsPath, new DataUpdater<ZNRecord>() {
-      @Override
-      public ZNRecord update(ZNRecord alertsRec) {
-        if (alertsRec == null) {
-          alertsRec = new ZNRecord(Alerts.nodeName);
-        }
-        Map<String, Map<String, String>> currAlertMap = alertsRec.getMapFields();
-        StringBuilder newStatName = new StringBuilder();
-        Map<String, String> newAlertMap = new HashMap<String, String>();
-
-        // use AlertsHolder to get map of new stats and map for this alert
-        AlertsHolder.parseAlert(alertName, newStatName, newAlertMap);
-
-        // add stat
-        addStat(newStatName.toString());
-
-        // add alert
-        currAlertMap.put(alertName, newAlertMap);
-        alertsRec.setMapFields(currAlertMap);
-        return alertsRec;
-      }
-    }, AccessOption.PERSISTENT);
-  }
-
-  /**
-   * Remove an alert specification from the cluster
-   * @param alertName string representing an alert specification
-   * @return true if removed, false otherwise
-   */
-  public boolean dropAlert(final String alertName) {
-    if (!isClusterStructureValid()) {
-      LOG.error("cluster " + _clusterId + " is not setup yet");
-      return false;
-    }
-
-    String alertsPath = _keyBuilder.alerts().getPath();
-    BaseDataAccessor<ZNRecord> baseAccessor = _accessor.getBaseDataAccessor();
-    return baseAccessor.update(alertsPath, new DataUpdater<ZNRecord>() {
-      @Override
-      public ZNRecord update(ZNRecord alertsRec) {
-        if (alertsRec == null) {
-          throw new HelixException("No alerts record persisted, nothing to drop");
-        }
-        Map<String, Map<String, String>> currAlertMap = alertsRec.getMapFields();
-        currAlertMap.remove(alertName);
-        alertsRec.setMapFields(currAlertMap);
-        return alertsRec;
-      }
-    }, AccessOption.PERSISTENT);
-  }
-
-  /**
    * Add user configuration to the existing cluster user configuration. Overwrites properties with
    * the same key
    * @param userConfig the user config key-value pairs to add

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java b/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
index f9af914..7bb214e 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
@@ -33,7 +33,6 @@ import org.apache.helix.ConfigChangeListener;
 import org.apache.helix.ControllerChangeListener;
 import org.apache.helix.CurrentStateChangeListener;
 import org.apache.helix.ExternalViewChangeListener;
-import org.apache.helix.HealthStateChangeListener;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.IdealStateChangeListener;
@@ -66,7 +65,6 @@ import org.apache.helix.controller.stages.ResourceValidationStage;
 import org.apache.helix.controller.stages.TaskAssignmentStage;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.ExternalView;
-import org.apache.helix.model.HealthStat;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.LiveInstance;
@@ -76,23 +74,21 @@ import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
 import org.apache.log4j.Logger;
 
 /**
- * Cluster Controllers main goal is to keep the cluster state as close as possible to
- * Ideal State. It does this by listening to changes in cluster state and scheduling new
- * tasks to get cluster state to best possible ideal state. Every instance of this class
- * can control can control only one cluster
- * Get all the partitions use IdealState, CurrentState and Messages <br>
+ * Cluster Controllers main goal is to keep the cluster state as close as possible to Ideal State.
+ * It does this by listening to changes in cluster state and scheduling new tasks to get cluster
+ * state to best possible ideal state. Every instance of this class can control can control only one
+ * cluster Get all the partitions use IdealState, CurrentState and Messages <br>
  * foreach partition <br>
  * 1. get the (instance,state) from IdealState, CurrentState and PendingMessages <br>
- * 2. compute best possible state (instance,state) pair. This needs previous step data and
- * state model constraints <br>
+ * 2. compute best possible state (instance,state) pair. This needs previous step data and state
+ * model constraints <br>
  * 3. compute the messages/tasks needed to move to 1 to 2 <br>
  * 4. select the messages that can be sent, needs messages and state model constraints <br>
  * 5. send messages
  */
 public class GenericHelixController implements ConfigChangeListener, IdealStateChangeListener,
     LiveInstanceChangeListener, MessageListener, CurrentStateChangeListener,
-    ExternalViewChangeListener, ControllerChangeListener, HealthStateChangeListener,
-    InstanceConfigChangeListener {
+    ExternalViewChangeListener, ControllerChangeListener, InstanceConfigChangeListener {
   private static final Logger logger = Logger.getLogger(GenericHelixController.class.getName());
   volatile boolean init = false;
   private final PipelineRegistry _registry;
@@ -109,15 +105,14 @@ public class GenericHelixController implements ConfigChangeListener, IdealStateC
   private final ClusterEventProcessor _eventThread;
 
   /**
-   * The _paused flag is checked by function handleEvent(), while if the flag is set
-   * handleEvent() will be no-op. Other event handling logic keeps the same when the flag
-   * is set.
+   * The _paused flag is checked by function handleEvent(), while if the flag is set handleEvent()
+   * will be no-op. Other event handling logic keeps the same when the flag is set.
    */
   private boolean _paused;
 
   /**
-   * The timer that can periodically run the rebalancing pipeline. The timer will start if there
-   * is one resource group has the config to use the timer.
+   * The timer that can periodically run the rebalancing pipeline. The timer will start if there is
+   * one resource group has the config to use the timer.
    */
   Timer _rebalanceTimer = null;
   int _timerPeriod = Integer.MAX_VALUE;
@@ -128,9 +123,9 @@ public class GenericHelixController implements ConfigChangeListener, IdealStateC
   private ClusterDataCache _cache;
 
   /**
-   * Default constructor that creates a default pipeline registry. This is sufficient in
-   * most cases, but if there is a some thing specific needed use another constructor
-   * where in you can pass a pipeline registry
+   * Default constructor that creates a default pipeline registry. This is sufficient in most cases,
+   * but if there is a some thing specific needed use another constructor where in you can pass a
+   * pipeline registry
    */
   public GenericHelixController() {
     this(createDefaultRegistry());
@@ -160,9 +155,8 @@ public class GenericHelixController implements ConfigChangeListener, IdealStateC
 
   // TODO who should stop this timer
   /**
-   * Starts the rebalancing timer with the specified period. Start the timer if necessary;
-   * If the period is smaller than the current period, cancel the current timer and use
-   * the new period.
+   * Starts the rebalancing timer with the specified period. Start the timer if necessary; If the
+   * period is smaller than the current period, cancel the current timer and use the new period.
    */
   void startRebalancingTimer(int period, HelixManager manager) {
     logger.info("Controller starting timer at period " + period);
@@ -227,7 +221,6 @@ public class GenericHelixController implements ConfigChangeListener, IdealStateC
       registry.register("resume", dataRefresh, rebalancePipeline, externalViewPipeline);
       registry
           .register("periodicalRebalance", dataRefresh, rebalancePipeline, externalViewPipeline);
-
       return registry;
     }
   }
@@ -245,8 +238,8 @@ public class GenericHelixController implements ConfigChangeListener, IdealStateC
   }
 
   /**
-   * lock-always: caller always needs to obtain an external lock before call, calls to
-   * handleEvent() should be serialized
+   * lock-always: caller always needs to obtain an external lock before call, calls to handleEvent()
+   * should be serialized
    * @param event
    */
   protected synchronized void handleEvent(ClusterEvent event) {
@@ -345,16 +338,6 @@ public class GenericHelixController implements ConfigChangeListener, IdealStateC
   }
 
   @Override
-  public void onHealthChange(String instanceName, List<HealthStat> reports,
-      NotificationContext changeContext) {
-    /**
-     * When there are more participant ( > 20, can be in hundreds), This callback can be
-     * called quite frequently as each participant reports health stat every minute. Thus
-     * we change the health check pipeline to run in a timer callback.
-     */
-  }
-
-  @Override
   public void onMessage(String instanceName, List<Message> messages,
       NotificationContext changeContext) {
     logger.info("START: GenericClusterController.onMessage()");
@@ -526,9 +509,9 @@ public class GenericHelixController implements ConfigChangeListener, IdealStateC
   }
 
   /**
-   * Go through the list of liveinstances in the cluster, and add currentstateChange
-   * listener and Message listeners to them if they are newly added. For current state
-   * change, the observation is tied to the session id of each live instance.
+   * Go through the list of liveinstances in the cluster, and add currentstateChange listener and
+   * Message listeners to them if they are newly added. For current state change, the observation is
+   * tied to the session id of each live instance.
    */
   protected void checkLiveInstancesObservation(List<LiveInstance> liveInstances,
       NotificationContext changeContext) {

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/controller/stages/HealthDataCache.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/HealthDataCache.java b/helix-core/src/main/java/org/apache/helix/controller/stages/HealthDataCache.java
deleted file mode 100644
index 6b29e2d..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/HealthDataCache.java
+++ /dev/null
@@ -1,95 +0,0 @@
-package org.apache.helix.controller.stages;
-
-/*
- * 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.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.model.AlertStatus;
-import org.apache.helix.model.Alerts;
-import org.apache.helix.model.HealthStat;
-import org.apache.helix.model.LiveInstance;
-import org.apache.helix.model.PersistentStats;
-
-public class HealthDataCache {
-  Map<String, LiveInstance> _liveInstanceMap;
-
-  Map<String, Map<String, HealthStat>> _healthStatMap;
-  HealthStat _globalStats; // DON'T THINK I WILL USE THIS ANYMORE
-  PersistentStats _persistentStats;
-  Alerts _alerts;
-  AlertStatus _alertStatus;
-
-  public HealthStat getGlobalStats() {
-    return _globalStats;
-  }
-
-  public PersistentStats getPersistentStats() {
-    return _persistentStats;
-  }
-
-  public Alerts getAlerts() {
-    return _alerts;
-  }
-
-  public AlertStatus getAlertStatus() {
-    return _alertStatus;
-  }
-
-  public Map<String, HealthStat> getHealthStats(String instanceName) {
-    Map<String, HealthStat> map = _healthStatMap.get(instanceName);
-    if (map != null) {
-      return map;
-    } else {
-      return Collections.emptyMap();
-    }
-  }
-
-  public Map<String, LiveInstance> getLiveInstances() {
-    return _liveInstanceMap;
-  }
-
-  public boolean refresh(HelixDataAccessor accessor) {
-    Builder keyBuilder = accessor.keyBuilder();
-    _liveInstanceMap = accessor.getChildValuesMap(keyBuilder.liveInstances());
-
-    Map<String, Map<String, HealthStat>> hsMap = new HashMap<String, Map<String, HealthStat>>();
-
-    for (String instanceName : _liveInstanceMap.keySet()) {
-      // xxx clearly getting znodes for the instance here...so get the
-      // timestamp!
-
-      Map<String, HealthStat> childValuesMap =
-          accessor.getChildValuesMap(keyBuilder.healthReports(instanceName));
-      hsMap.put(instanceName, childValuesMap);
-    }
-    _healthStatMap = Collections.unmodifiableMap(hsMap);
-    _persistentStats = accessor.getProperty(keyBuilder.persistantStat());
-    _alerts = accessor.getProperty(keyBuilder.alerts());
-    _alertStatus = accessor.getProperty(keyBuilder.alertStatus());
-
-    return true;
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/controller/stages/ReadHealthDataStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ReadHealthDataStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ReadHealthDataStage.java
deleted file mode 100644
index 859c1d0..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/ReadHealthDataStage.java
+++ /dev/null
@@ -1,51 +0,0 @@
-package org.apache.helix.controller.stages;
-
-/*
- * 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 org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixManager;
-import org.apache.helix.controller.pipeline.AbstractBaseStage;
-import org.apache.helix.controller.pipeline.StageException;
-
-public class ReadHealthDataStage extends AbstractBaseStage {
-  HealthDataCache _cache;
-
-  public ReadHealthDataStage() {
-    _cache = new HealthDataCache();
-  }
-
-  @Override
-  public void process(ClusterEvent event) throws Exception {
-    long startTime = System.currentTimeMillis();
-
-    HelixManager manager = event.getAttribute("helixmanager");
-    if (manager == null) {
-      throw new StageException("HelixManager attribute value is null");
-    }
-    // DataAccessor dataAccessor = manager.getDataAccessor();
-    HelixDataAccessor accessor = manager.getHelixDataAccessor();
-    _cache.refresh(accessor);
-
-    event.addAttribute("HealthDataCache", _cache);
-
-    long processLatency = System.currentTimeMillis() - startTime;
-    addLatencyToMonitor(event, processLatency);
-  }
-}


[04/17] [HELIX-395] Remove old Helix alert/stat modules

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleAlert.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleAlert.java
deleted file mode 100644
index 44937fe..0000000
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleAlert.java
+++ /dev/null
@@ -1,200 +0,0 @@
-package org.apache.helix.healthcheck;
-
-/*
- * 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.Map;
-import java.util.Set;
-
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixManager;
-import org.apache.helix.NotificationContext;
-import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.TestHelper;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.alerts.AlertValueAndStatus;
-import org.apache.helix.api.State;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
-import org.apache.helix.integration.ZkIntegrationTestBase;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.participant.MockEspressoHealthReportProvider;
-import org.apache.helix.mock.participant.MockTransition;
-import org.apache.helix.model.Message;
-import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier;
-import org.testng.Assert;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
-import org.testng.annotations.Test;
-
-public class TestSimpleAlert extends ZkIntegrationTestBase {
-  protected ClusterSetup _setupTool = null;
-  protected final String _alertStr =
-      "EXP(decay(1.0)(localhost_12918.RestQueryStats@DBName=TestDB0.latency))CMP(GREATER)CON(10)";
-  protected final String _alertStatusStr = _alertStr; // +" : (*)";
-  protected final String _dbName = "TestDB0";
-
-  @BeforeClass()
-  public void beforeClass() throws Exception {
-    _setupTool = new ClusterSetup(_gZkClient);
-  }
-
-  @AfterClass
-  public void afterClass() {
-  }
-
-  public class SimpleAlertTransition extends MockTransition {
-    int _alertValue;
-
-    public SimpleAlertTransition(int value) {
-      _alertValue = value;
-    }
-
-    @Override
-    public void doTransition(Message message, NotificationContext context) {
-      HelixManager manager = context.getManager();
-      HelixDataAccessor accessor = manager.getHelixDataAccessor();
-      State fromState = message.getTypedFromState();
-      State toState = message.getTypedToState();
-      String instance = message.getTgtName();
-
-      if (fromState.toString().equalsIgnoreCase("SLAVE")
-          && toState.toString().equalsIgnoreCase("MASTER")) {
-
-        // add a stat and report to ZK
-        // perhaps should keep reporter per instance...
-        ParticipantHealthReportCollectorImpl reporter =
-            new ParticipantHealthReportCollectorImpl(manager, instance);
-        MockEspressoHealthReportProvider provider = new MockEspressoHealthReportProvider();
-        reporter.addHealthReportProvider(provider);
-        String statName = "latency";
-        provider.setStat(_dbName, statName, "" + (0.1 + _alertValue));
-        reporter.transmitHealthReports();
-
-        /*
-         * for (int i = 0; i < 5; i++)
-         * {
-         * accessor.setProperty(PropertyType.HEALTHREPORT,
-         * new ZNRecord("mockAlerts" + i),
-         * instance,
-         * "mockAlerts");
-         * try
-         * {
-         * Thread.sleep(1000);
-         * }
-         * catch (InterruptedException e)
-         * {
-         * // TODO Auto-generated catch block
-         * e.printStackTrace();
-         * }
-         * }
-         */
-      }
-    }
-
-  }
-
-  @Test()
-  public void testSimpleAlert() throws Exception {
-    String clusterName = getShortClassName();
-    MockParticipantManager[] participants = new MockParticipantManager[5];
-
-    System.out.println("START TestSimpleAlert at " + new Date(System.currentTimeMillis()));
-
-    TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant start port
-        "localhost", // participant name prefix
-        "TestDB", // resource name prefix
-        1, // resources
-        10, // partitions per resource
-        5, // number of nodes //change back to 5!!!
-        3, // replicas //change back to 3!!!
-        "MasterSlave", true); // do rebalance
-
-    // enableHealthCheck(clusterName);
-
-    ClusterControllerManager controller =
-        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
-    controller.syncStart();
-    controller.startTimerTasks();
-
-    _setupTool.getClusterManagementTool().addAlert(clusterName, _alertStr);
-    // start participants
-    for (int i = 0; i < 5; i++) // !!!change back to 5
-    {
-      String instanceName = "localhost_" + (12918 + i);
-
-      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
-      participants[i].setTransition(new SimpleAlertTransition(15));
-      participants[i].syncStart();
-    }
-
-    boolean result =
-        ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(
-            ZK_ADDR, clusterName));
-    Assert.assertTrue(result);
-
-    // HealthAggregationTask is supposed to run by a timer every 30s
-    // To make sure HealthAggregationTask is run, we invoke it explicitly for this test
-    // new HealthStatsAggregator(cmResult._manager).aggregate();
-    new HealthStatsAggregator(controller).aggregate();
-    // sleep for a few seconds to give stats stage time to trigger
-    Thread.sleep(3000);
-
-    // other verifications go here
-    ZKHelixDataAccessor accessor =
-        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
-    Builder keyBuilder = accessor.keyBuilder();
-    // for (int i = 0; i < 1; i++) //change 1 back to 5
-    // {
-    // String instance = "localhost_" + (12918 + i);
-    String instance = "localhost_12918";
-    ZNRecord record = accessor.getProperty(keyBuilder.alertStatus()).getRecord();
-    Map<String, Map<String, String>> recMap = record.getMapFields();
-    Set<String> keySet = recMap.keySet();
-    Map<String, String> alertStatusMap = recMap.get(_alertStatusStr);
-    String val = alertStatusMap.get(AlertValueAndStatus.VALUE_NAME);
-    boolean fired = Boolean.parseBoolean(alertStatusMap.get(AlertValueAndStatus.FIRED_NAME));
-    Assert.assertEquals(Double.parseDouble(val), Double.parseDouble("15.1"));
-    Assert.assertTrue(fired);
-
-    // Verify Alert history from ZK
-    ZNRecord alertHistory = accessor.getProperty(keyBuilder.alertHistory()).getRecord();
-
-    String deltakey = (String) (alertHistory.getMapFields().keySet().toArray()[0]);
-    Map<String, String> delta = alertHistory.getMapField(deltakey);
-    Assert.assertTrue(delta.size() == 1);
-    Assert
-        .assertTrue(delta
-            .get(
-                "EXP(decay(1.0)(localhost_12918.RestQueryStats@DBName#TestDB0.latency))CMP(GREATER)CON(10)--(%)")
-            .equals("ON"));
-    // }
-
-    // clean up
-    controller.syncStop();
-    for (int i = 0; i < 5; i++) {
-      participants[i].syncStop();
-    }
-    System.out.println("END TestSimpleAlert at " + new Date(System.currentTimeMillis()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleWildcardAlert.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleWildcardAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleWildcardAlert.java
deleted file mode 100644
index 99c0608..0000000
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleWildcardAlert.java
+++ /dev/null
@@ -1,246 +0,0 @@
-package org.apache.helix.healthcheck;
-
-/*
- * 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.Map;
-
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixManager;
-import org.apache.helix.NotificationContext;
-import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.TestHelper;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.alerts.AlertValueAndStatus;
-import org.apache.helix.api.State;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
-import org.apache.helix.integration.ZkIntegrationTestBase;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.participant.MockEspressoHealthReportProvider;
-import org.apache.helix.mock.participant.MockTransition;
-import org.apache.helix.model.Message;
-import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier;
-import org.apache.log4j.Logger;
-import org.testng.Assert;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
-import org.testng.annotations.Test;
-
-public class TestSimpleWildcardAlert extends ZkIntegrationTestBase {
-  private static Logger LOG = Logger.getLogger(TestSimpleWildcardAlert.class);
-
-  protected ClusterSetup _setupTool = null;
-  protected final String _alertStr =
-      "EXP(decay(1.0)(localhost_12918.RestQueryStats@DBName=TestDB0.latency))CMP(GREATER)CON(10)";
-  protected final String _alertStatusStr = _alertStr; // +" : (*)";
-  protected final String _dbName = "TestDB0";
-
-  @BeforeClass()
-  public void beforeClass() throws Exception {
-
-    _setupTool = new ClusterSetup(_gZkClient);
-  }
-
-  @AfterClass
-  public void afterClass() {
-  }
-
-  public class SimpleAlertTransition extends MockTransition {
-    int _alertValue;
-
-    public SimpleAlertTransition(int value) {
-      _alertValue = value;
-    }
-
-    @Override
-    public void doTransition(Message message, NotificationContext context) {
-      HelixManager manager = context.getManager();
-      HelixDataAccessor accessor = manager.getHelixDataAccessor();
-      State fromState = message.getTypedFromState();
-      State toState = message.getTypedToState();
-      String instance = message.getTgtName();
-
-      if (fromState.toString().equalsIgnoreCase("SLAVE")
-          && toState.toString().equalsIgnoreCase("MASTER")) {
-
-        // add a stat and report to ZK
-        // perhaps should keep reporter per instance...
-        ParticipantHealthReportCollectorImpl reporter =
-            new ParticipantHealthReportCollectorImpl(manager, instance);
-        MockEspressoHealthReportProvider provider = new MockEspressoHealthReportProvider();
-        reporter.addHealthReportProvider(provider);
-        String statName = "latency";
-        provider.setStat(_dbName, statName, "" + (0.1 + _alertValue));
-        reporter.transmitHealthReports();
-
-        /*
-         * for (int i = 0; i < 5; i++)
-         * {
-         * accessor.setProperty(PropertyType.HEALTHREPORT,
-         * new ZNRecord("mockAlerts" + i),
-         * instance,
-         * "mockAlerts");
-         * try
-         * {
-         * Thread.sleep(1000);
-         * }
-         * catch (InterruptedException e)
-         * {
-         * // TODO Auto-generated catch block
-         * e.printStackTrace();
-         * }
-         * }
-         */
-      }
-    }
-
-  }
-
-  @Test()
-  public void testSimpleWildcardAlert() throws Exception {
-    String clusterName = getShortClassName();
-    MockParticipantManager[] participants = new MockParticipantManager[5];
-
-    System.out.println("START testSimpleWildcardAlert at " + new Date(System.currentTimeMillis()));
-
-    TestHelper.setupCluster(clusterName, ZK_ADDR, 12944, // participant start port
-        "localhost", // participant name prefix
-        "TestDB", // resource name prefix
-        1, // resources
-        10, // partitions per resource
-        5, // number of nodes //change back to 5!!!
-        3, // replicas //change back to 3!!!
-        "MasterSlave", true); // do rebalance
-
-    // enableHealthCheck(clusterName);
-
-    ClusterControllerManager controller =
-        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
-    controller.syncStart();
-    controller.stopTimerTasks();
-
-    String alertwildcard =
-        "EXP(decay(1.0)(localhost*.RestQueryStats@DBName=TestDB0.latency))CMP(GREATER)CON(10)";
-
-    _setupTool.getClusterManagementTool().addAlert(clusterName, alertwildcard);
-    // start participants
-    for (int i = 0; i < 5; i++) // !!!change back to 5
-    {
-      String instanceName = "localhost_" + (12944 + i);
-
-      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
-      participants[i].setTransition(new SimpleAlertTransition(i * 5));
-      participants[i].syncStart();
-    }
-
-    boolean result =
-        ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(
-            ZK_ADDR, clusterName));
-    Assert.assertTrue(result);
-
-    Thread.sleep(1000);
-    // HealthAggregationTask is supposed to run by a timer every 30s
-    // To make sure HealthAggregationTask is run, we invoke it explicitly for this test
-    new HealthStatsAggregator(controller).aggregate();
-    // sleep for a few seconds to give stats stage time to trigger
-    Thread.sleep(1000);
-
-    // other verifications go here
-    ZKHelixDataAccessor accessor =
-        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
-    Builder keyBuilder = accessor.keyBuilder();
-    ZNRecord record = accessor.getProperty(keyBuilder.alertStatus()).getRecord();
-    Map<String, Map<String, String>> recMap = record.getMapFields();
-    for (int i = 0; i < 2; i++) {
-      String alertString = "(localhost_" + (12944 + i) + ".RestQueryStats@DBName=TestDB0.latency)";
-      Map<String, String> alertStatusMap = recMap.get(alertwildcard + " : " + alertString);
-      String val = alertStatusMap.get(AlertValueAndStatus.VALUE_NAME);
-      boolean fired = Boolean.parseBoolean(alertStatusMap.get(AlertValueAndStatus.FIRED_NAME));
-      Assert.assertEquals(Double.parseDouble(val), (double) i * 5 + 0.1);
-      Assert.assertFalse(fired);
-    }
-    for (int i = 2; i < 5; i++) {
-      String alertString = "(localhost_" + (12944 + i) + ".RestQueryStats@DBName=TestDB0.latency)";
-      Map<String, String> alertStatusMap = recMap.get(alertwildcard + " : " + alertString);
-      String val = alertStatusMap.get(AlertValueAndStatus.VALUE_NAME);
-      boolean fired = Boolean.parseBoolean(alertStatusMap.get(AlertValueAndStatus.FIRED_NAME));
-      Assert.assertEquals(Double.parseDouble(val), (double) i * 5 + 0.1);
-      Assert.assertTrue(fired);
-    }
-    ZNRecord alertHistory = accessor.getProperty(keyBuilder.alertHistory()).getRecord();
-
-    String deltakey = (String) (alertHistory.getMapFields().keySet().toArray()[0]);
-    Map<String, String> delta = alertHistory.getMapField(deltakey);
-    Assert.assertEquals(delta.size(), 3);
-    for (int i = 2; i < 5; i++) {
-      String alertString =
-          "(localhost_" + (12944 + i) + ".RestQueryStats@DBName#TestDB0.latency)GREATER(10)";
-      Assert.assertTrue(delta.get(alertString).equals("ON"));
-    }
-
-    // Drop and add another alert
-    _setupTool.getClusterManagementTool().dropAlert(clusterName, alertwildcard);
-    alertwildcard =
-        "EXP(decay(1.0)(localhost*.RestQueryStats@DBName=TestDB0.latency))CMP(GREATER)CON(15)";
-    _setupTool.getClusterManagementTool().addAlert(clusterName, alertwildcard);
-    new HealthStatsAggregator(controller).aggregate();
-    Thread.sleep(1000);
-
-    record = accessor.getProperty(keyBuilder.alertStatus()).getRecord();
-    recMap = record.getMapFields();
-    for (int i = 0; i < 3; i++) {
-      String alertString = "(localhost_" + (12944 + i) + ".RestQueryStats@DBName=TestDB0.latency)";
-      Map<String, String> alertStatusMap = recMap.get(alertwildcard + " : " + alertString);
-      String val = alertStatusMap.get(AlertValueAndStatus.VALUE_NAME);
-      boolean fired = Boolean.parseBoolean(alertStatusMap.get(AlertValueAndStatus.FIRED_NAME));
-      Assert.assertEquals(Double.parseDouble(val), (double) i * 5 + 0.1);
-      Assert.assertFalse(fired);
-    }
-    for (int i = 3; i < 5; i++) {
-      String alertString = "(localhost_" + (12944 + i) + ".RestQueryStats@DBName=TestDB0.latency)";
-      Map<String, String> alertStatusMap = recMap.get(alertwildcard + " : " + alertString);
-      String val = alertStatusMap.get(AlertValueAndStatus.VALUE_NAME);
-      boolean fired = Boolean.parseBoolean(alertStatusMap.get(AlertValueAndStatus.FIRED_NAME));
-      Assert.assertEquals(Double.parseDouble(val), (double) i * 5 + 0.1);
-      Assert.assertTrue(fired);
-    }
-    alertHistory = accessor.getProperty(keyBuilder.alertHistory()).getRecord();
-
-    deltakey = (String) (alertHistory.getMapFields().keySet().toArray()[1]);
-    delta = alertHistory.getMapField(deltakey);
-    Assert.assertTrue(delta.size() == 2);
-    for (int i = 3; i < 5; i++) {
-      String alertString =
-          "(localhost_" + (12944 + i) + ".RestQueryStats@DBName#TestDB0.latency)GREATER(15)";
-      Assert.assertTrue(delta.get(alertString).equals("ON"));
-    }
-
-    // clean up
-    controller.syncStop();
-    for (int i = 0; i < 5; i++) {
-      participants[i].syncStop();
-    }
-    System.out.println("END testSimpleWildcardAlert at " + new Date(System.currentTimeMillis()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/test/java/org/apache/helix/healthcheck/TestStalenessAlert.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestStalenessAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestStalenessAlert.java
deleted file mode 100644
index 3227a65..0000000
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestStalenessAlert.java
+++ /dev/null
@@ -1,180 +0,0 @@
-package org.apache.helix.healthcheck;
-
-/*
- * 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.Map;
-import java.util.Set;
-
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixManager;
-import org.apache.helix.NotificationContext;
-import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.TestHelper;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.alerts.AlertValueAndStatus;
-import org.apache.helix.api.State;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
-import org.apache.helix.integration.ZkIntegrationTestBase;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.participant.MockEspressoHealthReportProvider;
-import org.apache.helix.mock.participant.MockTransition;
-import org.apache.helix.model.Message;
-import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier;
-import org.testng.Assert;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
-import org.testng.annotations.Test;
-
-public class TestStalenessAlert extends ZkIntegrationTestBase {
-  protected ClusterSetup _setupTool = null;
-  protected final String _alertStr = "EXP(decay(1)(localhost_*.reportingage))CMP(GREATER)CON(600)";
-  protected final String _alertStatusStr = _alertStr + " : (localhost_12918.reportingage)";
-  protected final String _dbName = "TestDB0";
-
-  @BeforeClass()
-  public void beforeClass() throws Exception {
-
-    _setupTool = new ClusterSetup(_gZkClient);
-  }
-
-  @AfterClass
-  public void afterClass() {
-  }
-
-  public class StalenessAlertTransition extends MockTransition {
-    @Override
-    public void doTransition(Message message, NotificationContext context) {
-      HelixManager manager = context.getManager();
-      HelixDataAccessor accessor = manager.getHelixDataAccessor();
-      State fromState = message.getTypedFromState();
-      State toState = message.getTypedToState();
-      String instance = message.getTgtName();
-
-      if (fromState.toString().equalsIgnoreCase("SLAVE")
-          && toState.toString().equalsIgnoreCase("MASTER")) {
-
-        // add a stat and report to ZK
-        // perhaps should keep reporter per instance...
-        ParticipantHealthReportCollectorImpl reporter =
-            new ParticipantHealthReportCollectorImpl(manager, instance);
-        MockEspressoHealthReportProvider provider = new MockEspressoHealthReportProvider();
-        reporter.addHealthReportProvider(provider);
-        String statName = "latency";
-        provider.setStat(_dbName, statName, "15");
-        reporter.transmitHealthReports();
-
-        /*
-         * for (int i = 0; i < 5; i++)
-         * {
-         * accessor.setProperty(PropertyType.HEALTHREPORT,
-         * new ZNRecord("mockAlerts" + i),
-         * instance,
-         * "mockAlerts");
-         * try
-         * {
-         * Thread.sleep(1000);
-         * }
-         * catch (InterruptedException e)
-         * {
-         * // TODO Auto-generated catch block
-         * e.printStackTrace();
-         * }
-         * }
-         */
-      }
-    }
-
-  }
-
-  @Test()
-  public void testStalenessAlert() throws Exception {
-    String clusterName = getShortClassName();
-    MockParticipantManager[] participants = new MockParticipantManager[5];
-
-    System.out.println("START TestStalenessAlert at " + new Date(System.currentTimeMillis()));
-
-    TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant start port
-        "localhost", // participant name prefix
-        "TestDB", // resource name prefix
-        1, // resources
-        10, // partitions per resource
-        5, // number of nodes //change back to 5!!!
-        3, // replicas //change back to 3!!!
-        "MasterSlave", true); // do rebalance
-    // enableHealthCheck(clusterName);
-
-    _setupTool.getClusterManagementTool().addAlert(clusterName, _alertStr);
-
-    ClusterControllerManager controller =
-        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
-    controller.syncStart();
-
-    // start participants
-    for (int i = 0; i < 5; i++) // !!!change back to 5
-    {
-      String instanceName = "localhost_" + (12918 + i);
-
-      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
-      participants[i].setTransition(new StalenessAlertTransition());
-      participants[i].syncStart();
-    }
-
-    boolean result =
-        ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(
-            ZK_ADDR, clusterName));
-    Assert.assertTrue(result);
-
-    // HealthAggregationTask is supposed to run by a timer every 30s
-    // To make sure HealthAggregationTask is run, we invoke it explicitly for this test
-    new HealthStatsAggregator(controller).aggregate();
-    // sleep for a few seconds to give stats stage time to trigger
-    Thread.sleep(3000);
-
-    // other verifications go here
-    ZKHelixDataAccessor accessor =
-        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
-    Builder keyBuilder = accessor.keyBuilder();
-    // for (int i = 0; i < 1; i++) //change 1 back to 5
-    // {
-    // String instance = "localhost_" + (12918 + i);
-    // String instance = "localhost_12918";
-    ZNRecord record = accessor.getProperty(keyBuilder.alertStatus()).getRecord();
-    Map<String, Map<String, String>> recMap = record.getMapFields();
-    Set<String> keySet = recMap.keySet();
-    Map<String, String> alertStatusMap = recMap.get(_alertStatusStr);
-    String val = alertStatusMap.get(AlertValueAndStatus.VALUE_NAME);
-    boolean fired = Boolean.parseBoolean(alertStatusMap.get(AlertValueAndStatus.FIRED_NAME));
-    // Assert.assertEquals(Double.parseDouble(val), Double.parseDouble("75.0"));
-    // Assert.assertFalse(fired);
-    // }
-
-    // clean up
-    controller.syncStop();
-    for (int i = 0; i < 5; i++) {
-      participants[i].syncStop();
-    }
-    System.out.println("END TestStalenessAlert at " + new Date(System.currentTimeMillis()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/test/java/org/apache/helix/healthcheck/TestWildcardAlert.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestWildcardAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestWildcardAlert.java
deleted file mode 100644
index 9677fcf..0000000
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestWildcardAlert.java
+++ /dev/null
@@ -1,296 +0,0 @@
-package org.apache.helix.healthcheck;
-
-/*
- * 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.io.IOException;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-
-import javax.management.AttributeNotFoundException;
-import javax.management.InstanceNotFoundException;
-import javax.management.IntrospectionException;
-import javax.management.MBeanAttributeInfo;
-import javax.management.MBeanException;
-import javax.management.MBeanInfo;
-import javax.management.MBeanServerConnection;
-import javax.management.MBeanServerNotification;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-import javax.management.ReflectionException;
-
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixManager;
-import org.apache.helix.NotificationContext;
-import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.TestHelper;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.alerts.AlertValueAndStatus;
-import org.apache.helix.api.State;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
-import org.apache.helix.integration.ZkIntegrationTestBase;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.participant.MockEspressoHealthReportProvider;
-import org.apache.helix.mock.participant.MockTransition;
-import org.apache.helix.model.Message;
-import org.apache.helix.monitoring.mbeans.ClusterAlertMBeanCollection;
-import org.apache.helix.monitoring.mbeans.ClusterMBeanObserver;
-import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier;
-import org.apache.log4j.Logger;
-import org.testng.Assert;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
-import org.testng.annotations.Test;
-
-public class TestWildcardAlert extends ZkIntegrationTestBase {
-  public static class TestClusterMBeanObserver extends ClusterMBeanObserver {
-    public Map<String, Map<String, Object>> _beanValueMap =
-        new ConcurrentHashMap<String, Map<String, Object>>();
-
-    public TestClusterMBeanObserver(String domain) throws InstanceNotFoundException, IOException,
-        MalformedObjectNameException, NullPointerException {
-      super(domain);
-    }
-
-    @Override
-    public void onMBeanRegistered(MBeanServerConnection server,
-        MBeanServerNotification mbsNotification) {
-      try {
-        MBeanInfo info = _server.getMBeanInfo(mbsNotification.getMBeanName());
-        MBeanAttributeInfo[] infos = info.getAttributes();
-        _beanValueMap.put(mbsNotification.getMBeanName().toString(),
-            new ConcurrentHashMap<String, Object>());
-        for (MBeanAttributeInfo infoItem : infos) {
-          Object val = _server.getAttribute(mbsNotification.getMBeanName(), infoItem.getName());
-          System.out.println("         " + infoItem.getName() + " : "
-              + _server.getAttribute(mbsNotification.getMBeanName(), infoItem.getName())
-              + " type : " + infoItem.getType());
-          _beanValueMap.get(mbsNotification.getMBeanName().toString()).put(infoItem.getName(), val);
-        }
-      } catch (Exception e) {
-        _logger.error("Error getting bean info, domain=" + _domain, e);
-      }
-    }
-
-    @Override
-    public void onMBeanUnRegistered(MBeanServerConnection server,
-        MBeanServerNotification mbsNotification) {
-      _beanValueMap.remove(mbsNotification.getMBeanName().toString());
-    }
-
-    public void refresh() throws MalformedObjectNameException, NullPointerException,
-        InstanceNotFoundException, IntrospectionException, ReflectionException, IOException,
-        AttributeNotFoundException, MBeanException {
-      for (String beanName : _beanValueMap.keySet()) {
-        ObjectName objName = new ObjectName(beanName);
-        MBeanInfo info = _server.getMBeanInfo(objName);
-        MBeanAttributeInfo[] infos = info.getAttributes();
-        _beanValueMap.put(objName.toString(), new HashMap<String, Object>());
-        for (MBeanAttributeInfo infoItem : infos) {
-          Object val = _server.getAttribute(objName, infoItem.getName());
-          System.out
-              .println("         " + infoItem.getName() + " : "
-                  + _server.getAttribute(objName, infoItem.getName()) + " type : "
-                  + infoItem.getType());
-          _beanValueMap.get(objName.toString()).put(infoItem.getName(), val);
-        }
-      }
-    }
-
-  }
-
-  private static final Logger _logger = Logger.getLogger(TestWildcardAlert.class);
-  protected ClusterSetup _setupTool = null;
-  protected final String _alertStr =
-      "EXP(decay(1)(localhost_*.RestQueryStats@DBName=TestDB0.latency)|EXPAND|SUMEACH)CMP(GREATER)CON(10)";
-  protected final String _alertStatusStr = _alertStr; // +" : (*)";
-  protected final String _dbName = "TestDB0";
-
-  @BeforeClass()
-  public void beforeClass() throws Exception {
-
-    _setupTool = new ClusterSetup(_gZkClient);
-  }
-
-  @AfterClass
-  public void afterClass() {
-  }
-
-  public class WildcardAlertTransition extends MockTransition {
-    @Override
-    public void doTransition(Message message, NotificationContext context) {
-      HelixManager manager = context.getManager();
-      HelixDataAccessor accessor = manager.getHelixDataAccessor();
-      State fromState = message.getTypedFromState();
-      State toState = message.getTypedToState();
-      String instance = message.getTgtName();
-
-      if (fromState.toString().equalsIgnoreCase("SLAVE")
-          && toState.toString().equalsIgnoreCase("MASTER")) {
-        // add a stat and report to ZK
-        // perhaps should keep reporter per instance...
-        ParticipantHealthReportCollectorImpl reporter =
-            new ParticipantHealthReportCollectorImpl(manager, instance);
-        MockEspressoHealthReportProvider provider = new MockEspressoHealthReportProvider();
-        reporter.addHealthReportProvider(provider);
-        String statName = "latency";
-        // using constant as timestamp so that when each partition does this transition,
-        // they do not advance timestamp, and no stats double-counted
-        String timestamp = "12345";
-        provider.setStat(_dbName, statName, "15", timestamp);
-
-        // sleep for random time and see about errors.
-        /*
-         * Random r = new Random();
-         * int x = r.nextInt(30000);
-         * try {
-         * Thread.sleep(x);
-         * } catch (InterruptedException e) {
-         * // TODO Auto-generated catch block
-         * e.printStackTrace();
-         * }
-         */
-
-        reporter.transmitHealthReports();
-
-        /*
-         * for (int i = 0; i < 5; i++)
-         * {
-         * accessor.setProperty(PropertyType.HEALTHREPORT,
-         * new ZNRecord("mockAlerts" + i),
-         * instance,
-         * "mockAlerts");
-         * try
-         * {
-         * Thread.sleep(1000);
-         * }
-         * catch (InterruptedException e)
-         * {
-         * // TODO Auto-generated catch block
-         * e.printStackTrace();
-         * }
-         * }
-         */
-      }
-    }
-
-  }
-
-  @Test()
-  public void testWildcardAlert() throws Exception {
-    String clusterName = getShortClassName();
-    MockParticipantManager[] participants = new MockParticipantManager[5];
-
-    System.out.println("START TestWildcardAlert at " + new Date(System.currentTimeMillis()));
-
-    TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant start
-                                                         // port
-        "localhost", // participant name prefix
-        "TestDB", // resource name prefix
-        1, // resources
-        10, // partitions per resource
-        5, // number of nodes //change back to 5!!!
-        3, // replicas //change back to 3!!!
-        "MasterSlave", true); // do rebalance
-
-    // enableHealthCheck(clusterName);
-
-    _setupTool.getClusterManagementTool().addAlert(clusterName, _alertStr);
-    // _setupTool.getClusterManagementTool().addAlert(clusterName, _alertStr2);
-
-    ClusterControllerManager controller =
-        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
-    controller.syncStart();
-    // start participants
-    for (int i = 0; i < 5; i++) // !!!change back to 5
-    {
-      String instanceName = "localhost_" + (12918 + i);
-
-      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
-      participants[i].setTransition(new WildcardAlertTransition());
-      participants[i].syncStart();
-    }
-
-    TestClusterMBeanObserver jmxMBeanObserver =
-        new TestClusterMBeanObserver(ClusterAlertMBeanCollection.DOMAIN_ALERT);
-
-    boolean result =
-        ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(
-            ZK_ADDR, clusterName));
-    Assert.assertTrue(result);
-    Thread.sleep(3000);
-    // HealthAggregationTask is supposed to run by a timer every 30s
-    // To make sure HealthAggregationTask is run, we invoke it explicitly for this test
-    new HealthStatsAggregator(controller).aggregate();
-
-    // sleep for a few seconds to give stats stage time to trigger and for bean to trigger
-    Thread.sleep(3000);
-
-    ZKHelixDataAccessor accessor =
-        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
-    Builder keyBuilder = accessor.keyBuilder();
-
-    // for (int i = 0; i < 1; i++) //change 1 back to 5
-    // {
-    // String instance = "localhost_" + (12918 + i);
-    // String instance = "localhost_12918";
-    ZNRecord record = accessor.getProperty(keyBuilder.alertStatus()).getRecord();
-    Map<String, Map<String, String>> recMap = record.getMapFields();
-    Set<String> keySet = recMap.keySet();
-    Map<String, String> alertStatusMap = recMap.get(_alertStatusStr);
-    String val = alertStatusMap.get(AlertValueAndStatus.VALUE_NAME);
-    boolean fired = Boolean.parseBoolean(alertStatusMap.get(AlertValueAndStatus.FIRED_NAME));
-    Assert.assertEquals(Double.parseDouble(val), Double.parseDouble("75.0"));
-    Assert.assertTrue(fired);
-
-    // Make sure that the jmxObserver has received all the jmx bean value that is corresponding
-    // to the alerts.
-    jmxMBeanObserver.refresh();
-    Assert.assertTrue(jmxMBeanObserver._beanValueMap.size() >= 1);
-
-    String beanName =
-        "HelixAlerts:alert=EXP(decay(1)(localhost_%.RestQueryStats@DBName#TestDB0.latency)|EXPAND|SUMEACH)CMP(GREATER)CON(10)--(%)";
-    Assert.assertTrue(jmxMBeanObserver._beanValueMap.containsKey(beanName));
-
-    Map<String, Object> beanValueMap = jmxMBeanObserver._beanValueMap.get(beanName);
-    Assert.assertEquals(beanValueMap.size(), 4);
-    Assert.assertEquals((beanValueMap.get("AlertFired")), new Integer(1));
-    Assert.assertEquals((beanValueMap.get("AlertValue")), new Double(75.0));
-    Assert
-        .assertEquals(
-            (String) (beanValueMap.get("SensorName")),
-            "EXP(decay(1)(localhost_%.RestQueryStats@DBName#TestDB0.latency)|EXPAND|SUMEACH)CMP(GREATER)CON(10)--(%)");
-    // }
-
-    // clean up
-    controller.syncStop();
-    for (int i = 0; i < 5; i++) {
-      participants[i].syncStop();
-    }
-
-    System.out.println("END TestWildcardAlert at " + new Date(System.currentTimeMillis()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java
index 83dc986..bcab38e 100644
--- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java
+++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java
@@ -106,7 +106,6 @@ public class TestZkClusterManager extends ZkUnitTestBase {
     AssertJUnit.assertEquals("node_1", record.getId());
 
     controller.getMessagingService();
-    controller.getHealthReportCollector();
     controller.getClusterManagmentTool();
 
     controller.handleNewSession();
@@ -247,8 +246,6 @@ public class TestZkClusterManager extends ZkUnitTestBase {
     AssertJUnit.assertTrue(admin.isConnected());
 
     HelixAdmin adminTool = admin.getClusterManagmentTool();
-    // ConfigScope scope = new ConfigScopeBuilder().forCluster(clusterName)
-    // .forResource("testResource").forPartition("testPartition").build();
     HelixConfigScope scope =
         new HelixConfigScopeBuilder(ConfigScopeProperty.PARTITION).forCluster(clusterName)
             .forResource("testResource").forPartition("testPartition").build();

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/test/java/org/apache/helix/mock/participant/MockEspressoHealthReportProvider.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/mock/participant/MockEspressoHealthReportProvider.java b/helix-core/src/test/java/org/apache/helix/mock/participant/MockEspressoHealthReportProvider.java
deleted file mode 100644
index 84ba081..0000000
--- a/helix-core/src/test/java/org/apache/helix/mock/participant/MockEspressoHealthReportProvider.java
+++ /dev/null
@@ -1,81 +0,0 @@
-package org.apache.helix.mock.participant;
-
-/*
- * 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.HashMap;
-import java.util.Map;
-
-import org.apache.helix.alerts.StatsHolder;
-import org.apache.helix.healthcheck.HealthReportProvider;
-
-public class MockEspressoHealthReportProvider extends HealthReportProvider {
-
-  private final String _reportName = "RestQueryStats";
-  private HashMap<String, Map<String, String>> _statMap;
-  private final String DB_NAME = "DBName";
-
-  public MockEspressoHealthReportProvider() {
-    super();
-    _statMap = new HashMap<String, Map<String, String>>();
-  }
-
-  public String buildMapKey(String dbName) {
-    return _reportName + "@" + DB_NAME + "=" + dbName;
-  }
-
-  public void setStat(String dbName, String statName, String statVal) {
-    String currTime = String.valueOf(System.currentTimeMillis());
-    setStat(dbName, statName, statVal, currTime);
-  }
-
-  /*
-   * This version takes a fixed timestamp to ease with testing
-   */
-  public void setStat(String dbName, String statName, String statVal, String timestamp) {
-    String key = buildMapKey(dbName);
-    Map<String, String> dbStatMap = _statMap.get(key);
-    if (dbStatMap == null) {
-      dbStatMap = new HashMap<String, String>();
-      _statMap.put(key, dbStatMap);
-    }
-    dbStatMap.put(statName, statVal);
-    dbStatMap.put(StatsHolder.TIMESTAMP_NAME, timestamp);
-  }
-
-  @Override
-  public Map<String, String> getRecentHealthReport() {
-    return null;
-  }
-
-  @Override
-  public Map<String, Map<String, String>> getRecentPartitionHealthReport() {
-    return _statMap;
-  }
-
-  @Override
-  public void resetStats() {
-    _statMap.clear();
-  }
-
-  public String getReportName() {
-    return _reportName;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/test/java/org/apache/helix/mock/participant/MockHealthReportParticipant.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/mock/participant/MockHealthReportParticipant.java b/helix-core/src/test/java/org/apache/helix/mock/participant/MockHealthReportParticipant.java
deleted file mode 100644
index 31811bb..0000000
--- a/helix-core/src/test/java/org/apache/helix/mock/participant/MockHealthReportParticipant.java
+++ /dev/null
@@ -1,248 +0,0 @@
-package org.apache.helix.mock.participant;
-
-/*
- * 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.HashMap;
-import java.util.Map;
-import java.util.Random;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.CommandLineParser;
-import org.apache.commons.cli.GnuParser;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.OptionBuilder;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-import org.apache.helix.HelixManager;
-import org.apache.helix.healthcheck.HealthReportProvider;
-import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.log4j.Logger;
-
-public class MockHealthReportParticipant {
-  private static final Logger LOG = Logger.getLogger(MockHealthReportParticipant.class);
-  public static final String zkServer = "zkSvr";
-  public static final String cluster = "cluster";
-  public static final String host = "host";
-  public static final String port = "port";
-  public static final String help = "help";
-
-  static class MockHealthReportProvider extends HealthReportProvider {
-    private final String _reportName = "MockRestQueryStats";
-    private final Map<String, Map<String, String>> _mockHealthReport;
-
-    public MockHealthReportProvider() {
-      _mockHealthReport = new HashMap<String, Map<String, String>>();
-
-      Map<String, String> reportMap = new HashMap<String, String>();
-      _mockHealthReport.put("MockRestQueryStats@DBName=BizProfile", reportMap);
-
-      reportMap.put("MeanMysqlLatency", "2.132700625");
-      reportMap.put("95PercentileLatencyLucene", "108.40825525");
-      reportMap.put("99PercentileLatencyMysql", "9.369827");
-      reportMap.put("99PercentileLatencyServer", "167.714208");
-      reportMap.put("95PercentileLatencyMysqlPool", "8.03621375");
-      reportMap.put("95PercentileLatencyServer", "164.68374265");
-      reportMap.put("MinLuceneLatency", "1.765908");
-      reportMap.put("MaxServerLatency", "167.714208");
-      reportMap.put("MeanLuceneLatency", "16.107599458333336");
-      reportMap.put("CollectorName", "RestQueryStats");
-      reportMap.put("MeanLucenePoolLatency", "8.120545333333332");
-      reportMap.put("99PercentileLatencyLucenePool", "65.930564");
-      reportMap.put("MinServerLatency", "0.425272");
-      reportMap.put("IndexStoreMismatchCount", "0");
-      reportMap.put("ErrorCount", "0");
-      reportMap.put("MeanMysqlPoolLatency", "1.0704102916666667");
-      reportMap.put("MinLucenePoolLatency", "0.008189");
-      reportMap.put("MinMysqlLatency", "0.709691");
-      reportMap.put("MaxMysqlPoolLatency", "8.606973");
-      reportMap.put("99PercentileLatencyMysqlPool", "8.606973");
-      reportMap.put("MinMysqlPoolLatency", "0.091883");
-      reportMap.put("MaxLucenePoolLatency", "65.930564");
-      reportMap.put("99PercentileLatencyLucene", "111.78799");
-      reportMap.put("MaxMysqlLatency", "9.369827");
-      reportMap.put("TimeStamp", "1332895048143");
-      reportMap.put("MeanConcurrencyLevel", "1.9");
-      reportMap.put("95PercentileLatencyMysql", "8.96594875");
-      reportMap.put("QueryStartCount", "0");
-      reportMap.put("95PercentileLatencyLucenePool", "63.518656500000006");
-      reportMap.put("MeanServerLatency", "39.5451532");
-      reportMap.put("MaxLuceneLatency", "111.78799");
-      reportMap.put("QuerySuccessCount", "0");
-    }
-
-    @Override
-    public Map<String, String> getRecentHealthReport() {
-      // TODO Auto-generated method stub
-      return null;
-    }
-
-    @Override
-    public void resetStats() {
-      // TODO Auto-generated method stub
-
-    }
-
-    @Override
-    public Map<String, Map<String, String>> getRecentPartitionHealthReport() {
-      // tweak: randomly change the last digit
-      for (String key1 : _mockHealthReport.keySet()) {
-        Map<String, String> reportMap = _mockHealthReport.get(key1);
-        for (String key2 : reportMap.keySet()) {
-          String value = reportMap.get(key2);
-          String lastDigit = "" + new Random().nextInt(10);
-          value = value.substring(0, value.length() - 1) + lastDigit;
-          reportMap.put(key2, value);
-        }
-      }
-
-      return _mockHealthReport;
-    }
-
-    @Override
-    public String getReportName() {
-      return _reportName;
-    }
-  }
-
-  static class MockHealthReportJob implements MockJobIntf {
-
-    @Override
-    public void doPreConnectJob(HelixManager manager) {
-      // TODO Auto-generated method stub
-
-    }
-
-    @Override
-    public void doPostConnectJob(HelixManager manager) {
-      // TODO Auto-generated method stub
-      manager.getHealthReportCollector().addHealthReportProvider(new MockHealthReportProvider());
-
-      // // set property store path for perf test
-      // final String setPath = "/TEST_PERF/set";
-      // final String updatePath = "/TEST_PERF/update";
-      // manager.getHelixPropertyStore().create(setPath, new ZNRecord(setPath),
-      // BaseDataAccessor.Option.PERSISTENT);
-      // manager.getHelixPropertyStore().set(updatePath, new ZNRecord(updatePath),
-      // BaseDataAccessor.Option.PERSISTENT);
-    }
-
-  }
-
-  // hack OptionBuilder is not thread safe
-  @SuppressWarnings("static-access")
-  synchronized private static Options constructCommandLineOptions() {
-    Option helpOption =
-        OptionBuilder.withLongOpt(help).withDescription("Prints command-line options info")
-            .create();
-
-    Option clusterOption =
-        OptionBuilder.withLongOpt(cluster).withDescription("Provide cluster name").create();
-    clusterOption.setArgs(1);
-    clusterOption.setRequired(true);
-    clusterOption.setArgName("Cluster name (Required)");
-
-    Option hostOption =
-        OptionBuilder.withLongOpt(host).withDescription("Provide host name").create();
-    hostOption.setArgs(1);
-    hostOption.setRequired(true);
-    hostOption.setArgName("Host name (Required)");
-
-    Option portOption =
-        OptionBuilder.withLongOpt(port).withDescription("Provide host port").create();
-    portOption.setArgs(1);
-    portOption.setRequired(true);
-    portOption.setArgName("Host port (Required)");
-
-    Option zkServerOption =
-        OptionBuilder.withLongOpt(zkServer).withDescription("Provide zookeeper address").create();
-    zkServerOption.setArgs(1);
-    zkServerOption.setRequired(true);
-    zkServerOption.setArgName("Zookeeper server address(Required)");
-
-    Options options = new Options();
-    options.addOption(helpOption);
-    options.addOption(clusterOption);
-    options.addOption(hostOption);
-    options.addOption(portOption);
-    options.addOption(zkServerOption);
-
-    return options;
-  }
-
-  public static void printUsage(Options cliOptions) {
-    HelpFormatter helpFormatter = new HelpFormatter();
-    helpFormatter.printHelp("java " + MockHealthReportParticipant.class.getName(), cliOptions);
-  }
-
-  public static CommandLine processCommandLineArgs(String[] cliArgs) throws Exception {
-    CommandLineParser cliParser = new GnuParser();
-    Options cliOptions = constructCommandLineOptions();
-
-    try {
-
-      return cliParser.parse(cliOptions, cliArgs);
-    } catch (ParseException pe) {
-      System.err.println("CommandLineClient: failed to parse command-line options: "
-          + pe.toString());
-      printUsage(cliOptions);
-      System.exit(1);
-    }
-    return null;
-  }
-
-  // NOT working for kill -9, working for kill -2/-15
-  static class MockHealthReportParticipantShutdownHook extends Thread {
-    final MockParticipantManager _participant;
-
-    MockHealthReportParticipantShutdownHook(MockParticipantManager participant) {
-      _participant = participant;
-    }
-
-    @Override
-    public void run() {
-      LOG.info("MockHealthReportParticipantShutdownHook invoked");
-      _participant.syncStop();
-    }
-  }
-
-  public static void main(String[] args) throws Exception {
-    CommandLine cmd = processCommandLineArgs(args);
-    String zkConnectStr = cmd.getOptionValue(zkServer);
-    String clusterName = cmd.getOptionValue(cluster);
-    String hostStr = cmd.getOptionValue(host);
-    String portStr = cmd.getOptionValue(port);
-
-    String instanceName = hostStr + "_" + portStr;
-
-    MockParticipantManager participant =
-        new MockParticipantManager(zkConnectStr, clusterName, instanceName);
-    // participant.setTransition(new StoreAccessDiffNodeTransition());
-    // participant.setTransition(new StoreAccessOneNodeTransition()));
-    // new MockHealthReportJob());
-    Runtime.getRuntime().addShutdownHook(new MockHealthReportParticipantShutdownHook(participant));
-
-    // Espresso_driver.py will consume this
-    System.out
-        .println("MockHealthReportParticipant process started, instanceName: " + instanceName);
-
-    participant.run();
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestClusterAlertItemMBeanCollection.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestClusterAlertItemMBeanCollection.java b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestClusterAlertItemMBeanCollection.java
deleted file mode 100644
index bb4f46d..0000000
--- a/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestClusterAlertItemMBeanCollection.java
+++ /dev/null
@@ -1,289 +0,0 @@
-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.io.IOException;
-import java.io.StringReader;
-import java.util.HashMap;
-import java.util.Map;
-
-import javax.management.AttributeNotFoundException;
-import javax.management.InstanceNotFoundException;
-import javax.management.IntrospectionException;
-import javax.management.MBeanException;
-import javax.management.MalformedObjectNameException;
-import javax.management.ReflectionException;
-
-import org.apache.helix.alerts.AlertValueAndStatus;
-import org.apache.helix.alerts.Tuple;
-import org.apache.helix.healthcheck.TestWildcardAlert.TestClusterMBeanObserver;
-import org.apache.log4j.Logger;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.testng.Assert;
-import org.testng.annotations.Test;
-
-public class TestClusterAlertItemMBeanCollection {
-  private static final Logger _logger = Logger.getLogger(TestClusterAlertItemMBeanCollection.class);
-
-  @Test
-  public void TestAlertReportingHistory() throws InstanceNotFoundException,
-      MalformedObjectNameException, NullPointerException, IOException, IntrospectionException,
-      AttributeNotFoundException, ReflectionException, MBeanException {
-    ClusterAlertMBeanCollection beanCollection = new ClusterAlertMBeanCollection();
-
-    String clusterName = "TestCluster";
-    String originAlert1 =
-        "EXP(decay(1.0)(esv4-app7*.RestQueryStats@DBName=BizProfile.MinServerLatency))CMP(GREATER)CON(10)";
-    Map<String, AlertValueAndStatus> alertResultMap1 = new HashMap<String, AlertValueAndStatus>();
-    int nAlerts1 = 5;
-
-    String originAlert2 =
-        "EXP(decay(1.0)(esv4-app9*.RestQueryStats@DBName=BizProfile.MaxServerLatency))CMP(GREATER)CON(10)";
-    Map<String, AlertValueAndStatus> alertResultMap2 = new HashMap<String, AlertValueAndStatus>();
-    int nAlerts2 = 3;
-
-    TestClusterMBeanObserver jmxMBeanObserver =
-        new TestClusterMBeanObserver(ClusterAlertMBeanCollection.DOMAIN_ALERT);
-
-    for (int i = 0; i < nAlerts1; i++) {
-      String alertName =
-          "esv4-app7" + i
-              + ".stg.linkedin.com_12918.RestQueryStats@DBName=BizProfile.MinServerLatency";
-      Tuple<String> value = new Tuple<String>();
-      value.add("22" + i);
-      AlertValueAndStatus valueAndStatus = new AlertValueAndStatus(value, true);
-      alertResultMap1.put(alertName, valueAndStatus);
-    }
-
-    for (int i = 0; i < nAlerts2; i++) {
-      String alertName =
-          "esv4-app9" + i
-              + ".stg.linkedin.com_12918.RestQueryStats@DBName=BizProfile.MaxServerLatency";
-      Tuple<String> value = new Tuple<String>();
-      value.add("22" + i);
-      AlertValueAndStatus valueAndStatus = new AlertValueAndStatus(value, true);
-      alertResultMap1.put(alertName, valueAndStatus);
-    }
-
-    beanCollection.setAlerts(originAlert1, alertResultMap1, clusterName);
-    beanCollection.setAlerts(originAlert2, alertResultMap2, clusterName);
-
-    beanCollection.refreshAlertDelta(clusterName);
-    String summaryKey = ClusterAlertMBeanCollection.ALERT_SUMMARY + "_" + clusterName;
-    jmxMBeanObserver.refresh();
-
-    // Get the history list
-    String beanName = "HelixAlerts:alert=" + summaryKey;
-    Map<String, Object> beanValueMap = jmxMBeanObserver._beanValueMap.get(beanName);
-    String history1 = (String) (beanValueMap.get("AlertFiredHistory"));
-
-    StringReader sr = new StringReader(history1);
-    ObjectMapper mapper = new ObjectMapper();
-
-    // check the history
-
-    Map<String, String> delta = beanCollection.getRecentAlertDelta();
-    Assert.assertEquals(delta.size(), nAlerts1 + nAlerts2);
-    for (int i = 0; i < nAlerts1; i++) {
-      String alertBeanName =
-          "(esv4-app7"
-              + i
-              + ".stg.linkedin.com_12918.RestQueryStats@DBName#BizProfile.MinServerLatency)GREATER(10)";
-      Assert.assertTrue(delta.get(alertBeanName).equals("ON"));
-    }
-
-    for (int i = 0; i < nAlerts2; i++) {
-      String alertBeanName =
-          "(esv4-app9"
-              + i
-              + ".stg.linkedin.com_12918.RestQueryStats@DBName#BizProfile.MaxServerLatency)GREATER(10)";
-      Assert.assertTrue(delta.get(alertBeanName).equals("ON"));
-    }
-
-    alertResultMap1 = new HashMap<String, AlertValueAndStatus>();
-    for (int i = 0; i < 3; i++) {
-      String alertName =
-          "esv4-app7" + i
-              + ".stg.linkedin.com_12918.RestQueryStats@DBName=BizProfile.MinServerLatency";
-      Tuple<String> value = new Tuple<String>();
-      value.add("22" + i);
-      AlertValueAndStatus valueAndStatus = new AlertValueAndStatus(value, true);
-      alertResultMap1.put(alertName, valueAndStatus);
-    }
-
-    for (int i = 3; i < 5; i++) {
-      String alertName =
-          "esv4-app7" + i
-              + ".stg.linkedin.com_12918.RestQueryStats@DBName=BizProfile.MinServerLatency";
-      Tuple<String> value = new Tuple<String>();
-      value.add("22" + i);
-      AlertValueAndStatus valueAndStatus = new AlertValueAndStatus(value, false);
-      alertResultMap1.put(alertName, valueAndStatus);
-    }
-
-    for (int i = 7; i < 9; i++) {
-      String alertName =
-          "esv4-app7" + i
-              + ".stg.linkedin.com_12918.RestQueryStats@DBName=BizProfile.MinServerLatency";
-      Tuple<String> value = new Tuple<String>();
-      value.add("22" + i);
-      AlertValueAndStatus valueAndStatus = new AlertValueAndStatus(value, true);
-      alertResultMap1.put(alertName, valueAndStatus);
-    }
-
-    for (int i = 0; i < 2; i++) {
-      String alertName =
-          "esv4-app9" + i
-              + ".stg.linkedin.com_12918.RestQueryStats@DBName=BizProfile.MaxServerLatency";
-      Tuple<String> value = new Tuple<String>();
-      value.add("22" + i);
-      AlertValueAndStatus valueAndStatus = new AlertValueAndStatus(value, false);
-      alertResultMap1.put(alertName, valueAndStatus);
-    }
-
-    for (int i = 2; i < 3; i++) {
-      String alertName =
-          "esv4-app9" + i
-              + ".stg.linkedin.com_12918.RestQueryStats@DBName=BizProfile.MaxServerLatency";
-      Tuple<String> value = new Tuple<String>();
-      value.add("22" + i);
-      AlertValueAndStatus valueAndStatus = new AlertValueAndStatus(value, true);
-      alertResultMap1.put(alertName, valueAndStatus);
-    }
-    for (int i = 7; i < 9; i++) {
-      String alertName =
-          "esv4-app9" + i
-              + ".stg.linkedin.com_12918.RestQueryStats@DBName=BizProfile.MaxServerLatency";
-      Tuple<String> value = new Tuple<String>();
-      value.add("22" + i);
-      AlertValueAndStatus valueAndStatus = new AlertValueAndStatus(value, true);
-      alertResultMap1.put(alertName, valueAndStatus);
-    }
-
-    beanCollection.setAlerts(originAlert1, alertResultMap1, clusterName);
-    beanCollection.refreshAlertDelta(clusterName);
-    jmxMBeanObserver.refresh();
-
-    beanValueMap = jmxMBeanObserver._beanValueMap.get(beanName);
-    history1 = (String) (beanValueMap.get("AlertFiredHistory"));
-
-    sr = new StringReader(history1);
-    mapper = new ObjectMapper();
-
-    // check the history
-    delta = beanCollection.getRecentAlertDelta();
-    Assert.assertEquals(delta.size(), 8);
-    for (int i = 3; i < 5; i++) {
-      String alertBeanName =
-          "(esv4-app7"
-              + i
-              + ".stg.linkedin.com_12918.RestQueryStats@DBName#BizProfile.MinServerLatency)GREATER(10)";
-      Assert.assertTrue(delta.get(alertBeanName).equals("OFF"));
-    }
-    for (int i = 7; i < 9; i++) {
-      String alertBeanName =
-          "(esv4-app7"
-              + i
-              + ".stg.linkedin.com_12918.RestQueryStats@DBName#BizProfile.MinServerLatency)GREATER(10)";
-      Assert.assertTrue(delta.get(alertBeanName).equals("ON"));
-    }
-
-    for (int i = 0; i < 2; i++) {
-      String alertBeanName =
-          "(esv4-app9"
-              + i
-              + ".stg.linkedin.com_12918.RestQueryStats@DBName#BizProfile.MaxServerLatency)GREATER(10)";
-      Assert.assertTrue(delta.get(alertBeanName).equals("OFF"));
-    }
-    for (int i = 7; i < 9; i++) {
-      String alertBeanName =
-          "(esv4-app9"
-              + i
-              + ".stg.linkedin.com_12918.RestQueryStats@DBName#BizProfile.MaxServerLatency)GREATER(10)";
-      Assert.assertTrue(delta.get(alertBeanName).equals("ON"));
-    }
-  }
-
-  @Test
-  public void TestAlertRefresh() throws InstanceNotFoundException, MalformedObjectNameException,
-      NullPointerException, IOException, IntrospectionException, AttributeNotFoundException,
-      ReflectionException, MBeanException, InterruptedException {
-    ClusterAlertMBeanCollection beanCollection = new ClusterAlertMBeanCollection();
-
-    String clusterName = "TestCluster";
-    String originAlert1 =
-        "EXP(decay(1.0)(esv4-app7*.RestQueryStats@DBName=BizProfile.MinServerLatency))CMP(GREATER)CON(10)";
-    Map<String, AlertValueAndStatus> alertResultMap1 = new HashMap<String, AlertValueAndStatus>();
-    int nAlerts1 = 5;
-
-    String originAlert2 =
-        "EXP(decay(1.0)(esv4-app9*.RestQueryStats@DBName=BizProfile.MaxServerLatency))CMP(GREATER)CON(10)";
-    Map<String, AlertValueAndStatus> alertResultMap2 = new HashMap<String, AlertValueAndStatus>();
-    int nAlerts2 = 3;
-
-    TestClusterMBeanObserver jmxMBeanObserver =
-        new TestClusterMBeanObserver(ClusterAlertMBeanCollection.DOMAIN_ALERT);
-
-    for (int i = 0; i < nAlerts1; i++) {
-      String alertName =
-          "esv4-app7" + i
-              + ".stg.linkedin.com_12918.RestQueryStats@DBName=BizProfile.MinServerLatency";
-      Tuple<String> value = new Tuple<String>();
-      value.add("22" + i);
-      AlertValueAndStatus valueAndStatus = new AlertValueAndStatus(value, true);
-      alertResultMap1.put(alertName, valueAndStatus);
-    }
-
-    for (int i = 0; i < nAlerts2; i++) {
-      String alertName =
-          "esv4-app9" + i
-              + ".stg.linkedin.com_12918.RestQueryStats@DBName=BizProfile.MaxServerLatency";
-      Tuple<String> value = new Tuple<String>();
-      value.add("22" + i);
-      AlertValueAndStatus valueAndStatus = new AlertValueAndStatus(value, true);
-      alertResultMap2.put(alertName, valueAndStatus);
-    }
-
-    beanCollection.setAlerts(originAlert1, alertResultMap1, clusterName);
-    beanCollection.setAlerts(originAlert2, alertResultMap2, clusterName);
-
-    beanCollection.refreshAlertDelta(clusterName);
-    String summaryKey = ClusterAlertMBeanCollection.ALERT_SUMMARY + "_" + clusterName;
-    jmxMBeanObserver.refresh();
-
-    Assert.assertEquals(jmxMBeanObserver._beanValueMap.size(), nAlerts2 + nAlerts1 + 1);
-
-    Thread.sleep(300);
-
-    beanCollection.setAlerts(originAlert1, alertResultMap1, clusterName);
-    beanCollection.checkMBeanFreshness(200);
-
-    for (int i = 0; i < 20; i++) {
-      Thread.sleep(500);
-
-      jmxMBeanObserver.refresh();
-
-      if (jmxMBeanObserver._beanValueMap.size() == nAlerts1 + 1) {
-        break;
-      }
-    }
-    Assert.assertEquals(jmxMBeanObserver._beanValueMap.size(), nAlerts1 + 1);
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/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 7d252c5..11cdd34 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
@@ -27,7 +27,6 @@ import org.apache.helix.ConfigChangeListener;
 import org.apache.helix.ControllerChangeListener;
 import org.apache.helix.CurrentStateChangeListener;
 import org.apache.helix.ExternalViewChangeListener;
-import org.apache.helix.HealthStateChangeListener;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
@@ -42,7 +41,6 @@ import org.apache.helix.PreConnectCallback;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.ScopedConfigChangeListener;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollector;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.manager.zk.ZkClient;
@@ -170,24 +168,11 @@ public class MockZKHelixManager implements HelixManager {
   }
 
   @Override
-  public ParticipantHealthReportCollector getHealthReportCollector() {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  @Override
   public InstanceType getInstanceType() {
     return _type;
   }
 
   @Override
-  public void addHealthStateChangeListener(HealthStateChangeListener listener, String instanceName)
-      throws Exception {
-    // TODO Auto-generated method stub
-
-  }
-
-  @Override
   public String getVersion() {
     // TODO Auto-generated method stub
     return UUID.randomUUID().toString();


[14/17] git commit: [HELIX-130] Add a zk copy tool, rb=22986

Posted by ka...@apache.org.
[HELIX-130] Add a zk copy tool, rb=22986


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

Branch: refs/heads/master
Commit: 7d04053d39c42c1425b0d780be088c2b1971652a
Parents: c24a730
Author: zzhang <zz...@apache.org>
Authored: Wed Jun 25 16:42:53 2014 -0700
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Fri Jul 11 12:54:55 2014 -0700

----------------------------------------------------------------------
 helix-core/pom.xml                              |   4 +
 .../java/org/apache/helix/tools/ZkCopy.java     | 193 +++++++++++++++++++
 .../java/org/apache/helix/tools/TestZkCopy.java |  67 +++++++
 3 files changed, 264 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/7d04053d/helix-core/pom.xml
----------------------------------------------------------------------
diff --git a/helix-core/pom.xml b/helix-core/pom.xml
index bacbb07..8bb9ed1 100644
--- a/helix-core/pom.xml
+++ b/helix-core/pom.xml
@@ -193,6 +193,10 @@ under the License.
               <name>zk-dumper</name>
             </program>
             <program>
+              <mainClass>org.apache.helix.tools.ZkCopy</mainClass>
+              <name>zkcopy</name>
+            </program>
+            <program>
               <mainClass>org.apache.helix.tools.ZKLogFormatter</mainClass>
               <name>zk-log-parser</name>
             </program>

http://git-wip-us.apache.org/repos/asf/helix/blob/7d04053d/helix-core/src/main/java/org/apache/helix/tools/ZkCopy.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/tools/ZkCopy.java b/helix-core/src/main/java/org/apache/helix/tools/ZkCopy.java
new file mode 100644
index 0000000..69369a5
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/tools/ZkCopy.java
@@ -0,0 +1,193 @@
+package org.apache.helix.tools;
+
+/*
+ * 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.net.URI;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.helix.manager.zk.ByteArraySerializer;
+import org.apache.helix.manager.zk.ZkClient;
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ * Tool for copying a zk/file path to another zk/file path
+ */
+public class ZkCopy {
+  enum ZkCopyScheme {
+    zk
+  }
+
+  private static Logger logger = Logger.getLogger(ZkCopy.class);
+  private static final String src = "src";
+  private static final String dst = "dst";
+
+  @SuppressWarnings("static-access")
+  private static Options constructCmdLineOpt() {
+    Option srcOpt =
+        OptionBuilder.withLongOpt(src).hasArgs(1).isRequired(true)
+            .withArgName("source-URI (e.g. zk://localhost:2181/src-path")
+            .withDescription("Provide source URI").create();
+
+    Option dstOpt =
+        OptionBuilder.withLongOpt(dst).hasArgs(1).isRequired(true)
+            .withArgName("destination-URI (e.g. zk://localhost:2181/dst-path")
+            .withDescription("Provide destination URI").create();
+
+    Options options = new Options();
+    options.addOption(srcOpt);
+    options.addOption(dstOpt);
+    return options;
+  }
+
+  private static void printUsage(Options cliOptions) {
+    HelpFormatter helpFormatter = new HelpFormatter();
+    helpFormatter.setWidth(1000);
+    helpFormatter.printHelp("java " + ZkCopy.class.getName(), cliOptions);
+  }
+
+  private static String concatenate(String path, String suffix) {
+    if (suffix == null || suffix.isEmpty()) {
+      return path;
+    }
+
+    if (path.endsWith("/") || suffix.startsWith("/")) {
+      return path + suffix;
+    } else {
+      return path + "/" + suffix;
+    }
+  }
+
+  private static void zkCopy(ZkClient srcClient, String srcPath, ZkClient dstClient, String dstPath) {
+    // Strip off tailing "/"
+    if (!srcPath.equals("/") && srcPath.endsWith("/")) {
+      srcPath = srcPath.substring(0, srcPath.length() - 1);
+    }
+
+    if (!dstPath.equals("/") && dstPath.endsWith("/")) {
+      dstPath = dstPath.substring(0, dstPath.length() - 1);
+    }
+
+    // Validate paths
+    PathUtils.validatePath(srcPath);
+    PathUtils.validatePath(dstPath);
+
+    if (srcPath.equals(dstPath)) {
+      logger.info("srcPath == dstPath. Skip copying");
+      return;
+    }
+
+    if (srcPath.startsWith(dstPath) || dstPath.startsWith(srcPath)) {
+      throw new IllegalArgumentException(
+          "srcPath/dstPath can't be prefix of dstPath/srcPath, was srcPath: " + srcPath
+              + ", dstPath: " + dstPath);
+    }
+
+    // Recursive copy using BFS
+    List<String> queue = new LinkedList<String>();
+    queue.add("");
+    while (!queue.isEmpty()) {
+      String path = queue.remove(0);
+      Stat stat = new Stat();
+      String fromPath = concatenate(srcPath, path);
+      Object data = srcClient.readDataAndStat(fromPath, stat, false);
+      if (stat.getEphemeralOwner() != 0) {
+        logger.info("Skip copying ephemeral znode: " + fromPath);
+        continue;
+      }
+      String toPath = concatenate(dstPath, path);
+      System.out.println("Copy " + fromPath + " to " + toPath);
+      dstClient.createPersistent(toPath, data);
+      List<String> children = srcClient.getChildren(fromPath);
+      if (children != null && children.size() > 0) {
+        for (String child : children) {
+          queue.add(concatenate(path, child));
+        }
+      }
+    }
+  }
+
+  public static void main(String[] args) throws Exception {
+    CommandLineParser cliParser = new GnuParser();
+    Options cliOptions = constructCmdLineOpt();
+    CommandLine cmd = null;
+
+    try {
+      cmd = cliParser.parse(cliOptions, args);
+    } catch (ParseException pe) {
+      System.err.println("CommandLineClient: failed to parse command-line options: "
+          + pe.toString());
+      printUsage(cliOptions);
+      System.exit(1);
+    }
+
+    URI srcUri = new URI(cmd.getOptionValue(src));
+    URI dstUri = new URI(cmd.getOptionValue(dst));
+
+    ZkCopyScheme srcScheme = ZkCopyScheme.valueOf(srcUri.getScheme());
+    ZkCopyScheme dstScheme = ZkCopyScheme.valueOf(dstUri.getScheme());
+
+    if (srcScheme == ZkCopyScheme.zk && dstScheme == ZkCopyScheme.zk) {
+      String srcZkAddr = srcUri.getAuthority();
+      String dstZkAddr = dstUri.getAuthority();
+
+      ZkClient srcClient = null;
+      ZkClient dstClient = null;
+      try {
+        if (srcZkAddr.equals(dstZkAddr)) {
+          srcClient =
+              dstClient =
+                  new ZkClient(srcZkAddr, ZkClient.DEFAULT_SESSION_TIMEOUT,
+                      ZkClient.DEFAULT_CONNECTION_TIMEOUT, new ByteArraySerializer());
+        } else {
+          srcClient =
+              new ZkClient(srcZkAddr, ZkClient.DEFAULT_SESSION_TIMEOUT,
+                  ZkClient.DEFAULT_CONNECTION_TIMEOUT, new ByteArraySerializer());
+          dstClient =
+              new ZkClient(dstZkAddr, ZkClient.DEFAULT_SESSION_TIMEOUT,
+                  ZkClient.DEFAULT_CONNECTION_TIMEOUT, new ByteArraySerializer());
+        }
+        String srcPath = srcUri.getPath();
+        String dstPath = dstUri.getPath();
+        zkCopy(srcClient, srcPath, dstClient, dstPath);
+      } finally {
+        if (srcClient != null) {
+          srcClient.close();
+        }
+        if (dstClient != null) {
+          dstClient.close();
+        }
+      }
+    } else {
+      System.err.println("Unsupported scheme. srcScheme: " + srcScheme + ", dstScheme: " + dstScheme);
+      System.exit(1);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/7d04053d/helix-core/src/test/java/org/apache/helix/tools/TestZkCopy.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/tools/TestZkCopy.java b/helix-core/src/test/java/org/apache/helix/tools/TestZkCopy.java
new file mode 100644
index 0000000..21a1ad5
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/tools/TestZkCopy.java
@@ -0,0 +1,67 @@
+package org.apache.helix.tools;
+
+/*
+ * 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 org.apache.helix.TestHelper;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.ZkUnitTestBase;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class TestZkCopy extends ZkUnitTestBase {
+
+  @Test
+  public void test() throws Exception {
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+
+    System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
+    String fromPath = "/" + clusterName + "/from";
+    _gZkClient.createPersistent(fromPath, true);
+    for (int i = 0; i < 5; i++) {
+      for (int j = 0; j < 5; j++) {
+        String path = String.format("%s/%d/%d", fromPath, i, j);
+        _gZkClient.createPersistent(path, true);
+        _gZkClient.writeData(path, new ZNRecord(String.format("%d/%d", i, j)));
+      }
+    }
+
+    // Copy
+    String toPath = "/" + clusterName + "/to";
+    ZkCopy.main(new String[]{"--src", "zk://" + ZK_ADDR + fromPath, "--dst", "zk://" + ZK_ADDR + toPath});
+
+    // Verify
+    Assert.assertTrue(_gZkClient.exists(toPath));
+    Assert.assertNull(_gZkClient.readData(toPath));
+    for (int i = 0; i < 5; i++) {
+      for (int j = 0; j < 5; j++) {
+        String path = String.format("%s/%d/%d", toPath, i, j);
+        Assert.assertTrue(_gZkClient.exists(path));
+        ZNRecord record = _gZkClient.readData(path);
+        Assert.assertEquals(String.format("%d/%d", i, j), record.getId());
+      }
+    }
+
+    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
+  }
+}


[12/17] git commit: [HELIX-463] Create useful message queue length jmx

Posted by ka...@apache.org.
[HELIX-463] Create useful message queue length jmx


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

Branch: refs/heads/master
Commit: 2ccaa6b434e48415cc56e6e9dbb663a5a289e960
Parents: 0ee1cdc
Author: Kanak Biscuitwala <ka...@apache.org>
Authored: Wed Jun 25 17:42:45 2014 -0700
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Fri Jul 11 12:49:15 2014 -0700

----------------------------------------------------------------------
 .../messaging/handling/HelixTaskExecutor.java   | 23 ++++-
 .../monitoring/mbeans/ClusterStatusMonitor.java | 47 +++-------
 .../mbeans/ClusterStatusMonitorMBean.java       |  6 +-
 .../monitoring/mbeans/MessageQueueMonitor.java  | 94 +++++++++++++++++---
 .../mbeans/MessageQueueMonitorMBean.java        | 11 +--
 .../TestClusterStatusMonitorLifecycle.java      | 26 +++---
 6 files changed, 128 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/2ccaa6b4/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
index b507755..c4d7a67 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
@@ -47,16 +47,15 @@ import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.api.id.SessionId;
-import org.apache.helix.model.ConfigScope;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.HelixConfigScope;
-import org.apache.helix.model.builder.HelixConfigScopeBuilder;
 import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.MessageState;
 import org.apache.helix.model.Message.MessageType;
-import org.apache.helix.model.builder.ConfigScopeBuilder;
+import org.apache.helix.model.builder.HelixConfigScopeBuilder;
 import org.apache.helix.monitoring.ParticipantMonitor;
+import org.apache.helix.monitoring.mbeans.MessageQueueMonitor;
 import org.apache.helix.participant.HelixStateMachineEngine;
 import org.apache.helix.util.StatusUpdateUtil;
 import org.apache.log4j.Logger;
@@ -104,6 +103,8 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
   private final ParticipantMonitor _monitor;
   public static final String MAX_THREADS = "maxThreads";
 
+  private MessageQueueMonitor _messageQueueMonitor;
+
   /**
    * Map of MsgType->MsgHandlerFactoryRegistryItem
    */
@@ -441,6 +442,11 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
 
   void reset() {
     LOG.info("Reset HelixTaskExecutor");
+
+    if (_messageQueueMonitor != null) {
+      _messageQueueMonitor.reset();
+    }
+
     for (String msgType : _hdlrFtyRegistry.keySet()) {
       // don't un-register factories, just shutdown all executors
       ExecutorService pool = _executorMap.remove(msgType);
@@ -466,6 +472,10 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
   void init() {
     LOG.info("Init HelixTaskExecutor");
 
+    if (_messageQueueMonitor != null) {
+      _messageQueueMonitor.init();
+    }
+
     // Re-init all existing factories
     for (String msgType : _hdlrFtyRegistry.keySet()) {
       MsgHandlerFactoryRegistryItem item = _hdlrFtyRegistry.get(msgType);
@@ -484,6 +494,12 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
   public void onMessage(String instanceName, List<Message> messages,
       NotificationContext changeContext) {
 
+    HelixManager manager = changeContext.getManager();
+    if (_messageQueueMonitor == null) {
+      _messageQueueMonitor =
+          new MessageQueueMonitor(manager.getClusterName(), manager.getInstanceName());
+    }
+
     // If FINALIZE notification comes, reset all handler factories
     // and terminate all the thread pools
     // TODO: see if we should have a separate notification call for resetting
@@ -505,7 +521,6 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
     // sort message by creation timestamp, so message created earlier is processed first
     Collections.sort(messages, Message.CREATE_TIME_COMPARATOR);
 
-    HelixManager manager = changeContext.getManager();
     HelixDataAccessor accessor = manager.getHelixDataAccessor();
     Builder keyBuilder = accessor.keyBuilder();
 

http://git-wip-us.apache.org/repos/asf/helix/blob/2ccaa6b4/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterStatusMonitor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterStatusMonitor.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterStatusMonitor.java
index 1cce342..9ea51b1 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterStatusMonitor.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterStatusMonitor.java
@@ -27,7 +27,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.TreeMap;
 import java.util.concurrent.ConcurrentHashMap;
 
 import javax.management.MBeanServer;
@@ -48,6 +47,7 @@ import org.apache.helix.model.ResourceAssignment;
 import org.apache.helix.model.StateModelDefinition;
 import org.apache.log4j.Logger;
 
+import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
 public class ClusterStatusMonitor implements ClusterStatusMonitorMBean {
@@ -60,6 +60,7 @@ public class ClusterStatusMonitor implements ClusterStatusMonitorMBean {
   static final String CLUSTER_DN_KEY = "cluster";
   static final String RESOURCE_DN_KEY = "resourceName";
   static final String INSTANCE_DN_KEY = "instanceName";
+  static final String MESSAGE_QUEUE_DN_KEY = "messageQueue";
 
   public static final String DEFAULT_TAG = "DEFAULT";
 
@@ -70,13 +71,11 @@ public class ClusterStatusMonitor implements ClusterStatusMonitorMBean {
   private Set<String> _instances = Collections.emptySet();
   private Set<String> _disabledInstances = Collections.emptySet();
   private Map<String, Set<String>> _disabledPartitions = Collections.emptyMap();
+  private Map<String, Long> _instanceMsgQueueSizes = Maps.newConcurrentMap();
 
   private final ConcurrentHashMap<String, ResourceMonitor> _resourceMbeanMap =
       new ConcurrentHashMap<String, ResourceMonitor>();
 
-  private final ConcurrentHashMap<String, MessageQueueMonitor> _instanceMsgQueueMbeanMap =
-      new ConcurrentHashMap<String, MessageQueueMonitor>();
-
   private final ConcurrentHashMap<String, InstanceMonitor> _instanceMbeanMap =
       new ConcurrentHashMap<String, InstanceMonitor>();
 
@@ -133,24 +132,21 @@ public class ClusterStatusMonitor implements ClusterStatusMonitorMBean {
   @Override
   public long getMaxMessageQueueSizeGauge() {
     long maxQueueSize = 0;
-    for (MessageQueueMonitor msgQueue : _instanceMsgQueueMbeanMap.values()) {
-      if (msgQueue.getMaxMessageQueueSize() > maxQueueSize) {
-        maxQueueSize = (long) msgQueue.getMaxMessageQueueSize();
+    for (Long queueSize : _instanceMsgQueueSizes.values()) {
+      if (queueSize > maxQueueSize) {
+        maxQueueSize = queueSize;
       }
     }
-
     return maxQueueSize;
   }
 
   @Override
-  public String getMessageQueueSizes() {
-    Map<String, Long> msgQueueSizes = new TreeMap<String, Long>();
-    for (String instance : _instanceMsgQueueMbeanMap.keySet()) {
-      MessageQueueMonitor msgQueue = _instanceMsgQueueMbeanMap.get(instance);
-      msgQueueSizes.put(instance, new Long((long) msgQueue.getMaxMessageQueueSize()));
+  public long getInstanceMessageQueueBacklog() {
+    long sum = 0;
+    for (Long queueSize : _instanceMsgQueueSizes.values()) {
+      sum += queueSize;
     }
-
-    return msgQueueSizes.toString();
+    return sum;
   }
 
   private void register(Object bean, ObjectName name) {
@@ -349,20 +345,8 @@ public class ClusterStatusMonitor implements ClusterStatusMonitorMBean {
     }
   }
 
-  public void addMessageQueueSize(String instanceName, int msgQueueSize) {
-    try {
-      if (!_instanceMsgQueueMbeanMap.containsKey(instanceName)) {
-        synchronized (this) {
-          if (!_instanceMsgQueueMbeanMap.containsKey(instanceName)) {
-            MessageQueueMonitor bean = new MessageQueueMonitor(_clusterName, instanceName);
-            _instanceMsgQueueMbeanMap.put(instanceName, bean);
-          }
-        }
-      }
-      _instanceMsgQueueMbeanMap.get(instanceName).addMessageQueueSize(msgQueueSize);
-    } catch (Exception e) {
-      LOG.error("Fail to add message queue size to mbean, instance: " + instanceName, e);
-    }
+  public void addMessageQueueSize(String instanceName, long msgQueueSize) {
+    _instanceMsgQueueSizes.put(instanceName, msgQueueSize);
   }
 
   public void reset() {
@@ -372,10 +356,7 @@ public class ClusterStatusMonitor implements ClusterStatusMonitorMBean {
 
       _resourceMbeanMap.clear();
 
-      for (MessageQueueMonitor bean : _instanceMsgQueueMbeanMap.values()) {
-        bean.reset();
-      }
-      _instanceMsgQueueMbeanMap.clear();
+      _instanceMsgQueueSizes.clear();
 
       unregisterInstances(_instanceMbeanMap.keySet());
       _instanceMbeanMap.clear();

http://git-wip-us.apache.org/repos/asf/helix/blob/2ccaa6b4/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterStatusMonitorMBean.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterStatusMonitorMBean.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterStatusMonitorMBean.java
index 9070aaa..9d15af9 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterStatusMonitorMBean.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterStatusMonitorMBean.java
@@ -32,15 +32,13 @@ public interface ClusterStatusMonitorMBean extends SensorNameProvider {
 
   /**
    * The max message queue size across all instances including controller
-   * will report to ingraph
    * @return
    */
   public long getMaxMessageQueueSizeGauge();
 
   /**
-   * Get all message queue sizes as a string
-   * will NOT report to ingraph
+   * The sum of all message queue sizes for instances in this cluster
    * @return
    */
-  public String getMessageQueueSizes();
+  public long getInstanceMessageQueueBacklog();
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/2ccaa6b4/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MessageQueueMonitor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MessageQueueMonitor.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MessageQueueMonitor.java
index 6b8b9e3..1ebf851 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MessageQueueMonitor.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MessageQueueMonitor.java
@@ -19,42 +19,108 @@ package org.apache.helix.monitoring.mbeans;
  * under the License.
  */
 
-import org.apache.helix.monitoring.StatCollector;
+import java.lang.management.ManagementFactory;
+
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+
 import org.apache.log4j.Logger;
 
 public class MessageQueueMonitor implements MessageQueueMonitorMBean {
   private static final Logger LOG = Logger.getLogger(MessageQueueMonitor.class);
 
-  private final StatCollector _messageQueueSizeStat;
   private final String _clusterName;
   private final String _instanceName;
+  private final MBeanServer _beanServer;
+  private long _messageQueueBacklog;
 
   public MessageQueueMonitor(String clusterName, String instanceName) {
     _clusterName = clusterName;
     _instanceName = instanceName;
-    _messageQueueSizeStat = new StatCollector();
+    _beanServer = ManagementFactory.getPlatformMBeanServer();
+    _messageQueueBacklog = 0;
   }
 
-  public void addMessageQueueSize(long size) {
-    _messageQueueSizeStat.addData(size);
+  /**
+   * Set the current backlog size for this instance
+   * @param size the message queue size
+   */
+  public void setMessageQueueBacklog(long size) {
+    _messageQueueBacklog = size;
   }
 
-  public void reset() {
-    _messageQueueSizeStat.reset();
+  @Override
+  public long getMessageQueueBacklog() {
+    return _messageQueueBacklog;
   }
 
-  @Override
-  public double getMaxMessageQueueSize() {
-    return _messageQueueSizeStat.getMax();
+  /**
+   * Register this bean with the server
+   */
+  public void init() {
+    try {
+      register(this, getObjectName(getBeanName()));
+    } catch (Exception e) {
+      LOG.error("Fail to register MessageQueueMonitor", e);
+    }
   }
 
-  @Override
-  public double getMeanMessageQueueSize() {
-    return _messageQueueSizeStat.getMean();
+  /**
+   * Remove this bean from the server
+   */
+  public void reset() {
+    _messageQueueBacklog = 0;
+    try {
+      unregister(getObjectName(getBeanName()));
+    } catch (Exception e) {
+      LOG.error("Fail to register MessageQueueMonitor", e);
+    }
   }
 
   @Override
   public String getSensorName() {
-    return ClusterStatusMonitor.MESSAGE_QUEUE_STATUS_KEY + "." + _clusterName + "." + _instanceName;
+    return ClusterStatusMonitor.MESSAGE_QUEUE_STATUS_KEY + "." + _clusterName;
+  }
+
+  private void register(Object bean, ObjectName name) {
+    try {
+      if (_beanServer.isRegistered(name)) {
+        _beanServer.unregisterMBean(name);
+      }
+    } catch (Exception e) {
+      // OK
+    }
+
+    try {
+      LOG.info("Register MBean: " + name);
+      _beanServer.registerMBean(bean, name);
+    } catch (Exception e) {
+      LOG.warn("Could not register MBean: " + name, e);
+    }
+  }
+
+  private void unregister(ObjectName name) {
+    try {
+      if (_beanServer.isRegistered(name)) {
+        LOG.info("Unregistering " + name.toString());
+        _beanServer.unregisterMBean(name);
+      }
+    } catch (Exception e) {
+      LOG.warn("Could not unregister MBean: " + name, e);
+    }
+  }
+
+  private String getClusterBeanName() {
+    return String.format("%s=%s", ClusterStatusMonitor.CLUSTER_DN_KEY, _clusterName);
+  }
+
+  private String getBeanName() {
+    return String.format("%s,%s=%s", getClusterBeanName(),
+        ClusterStatusMonitor.MESSAGE_QUEUE_DN_KEY, _instanceName);
+  }
+
+  public ObjectName getObjectName(String name) throws MalformedObjectNameException {
+    return new ObjectName(String.format("%s: %s", ClusterStatusMonitor.CLUSTER_STATUS_KEY, name));
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/2ccaa6b4/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MessageQueueMonitorMBean.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MessageQueueMonitorMBean.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MessageQueueMonitorMBean.java
index acf3824..14c0af5 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MessageQueueMonitorMBean.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MessageQueueMonitorMBean.java
@@ -23,15 +23,8 @@ import org.apache.helix.monitoring.SensorNameProvider;
 
 public interface MessageQueueMonitorMBean extends SensorNameProvider {
   /**
-   * Get the max message queue size
+   * Get the message queue size
    * @return
    */
-  public double getMaxMessageQueueSize();
-
-  /**
-   * Get the mean message queue size
-   * @return
-   */
-  public double getMeanMessageQueueSize();
-
+  public long getMessageQueueBacklog();
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/2ccaa6b4/helix-core/src/test/java/org/apache/helix/monitoring/TestClusterStatusMonitorLifecycle.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/monitoring/TestClusterStatusMonitorLifecycle.java b/helix-core/src/test/java/org/apache/helix/monitoring/TestClusterStatusMonitorLifecycle.java
index 0981a2e..b062282 100644
--- a/helix-core/src/test/java/org/apache/helix/monitoring/TestClusterStatusMonitorLifecycle.java
+++ b/helix-core/src/test/java/org/apache/helix/monitoring/TestClusterStatusMonitorLifecycle.java
@@ -150,10 +150,6 @@ public class TestClusterStatusMonitorLifecycle extends ZkIntegrationTestBase {
   public void afterClass() {
     System.out.println("Cleaning up...");
     for (int i = 0; i < 5; i++) {
-      boolean result =
-          ClusterStateVerifier
-              .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR,
-                  _controllerClusterName));
       _controllers[i].syncStop();
     }
 
@@ -202,10 +198,10 @@ public class TestClusterStatusMonitorLifecycle extends ZkIntegrationTestBase {
 
     // 1 participant goes away
     // No change in instance/resource mbean
-    // Unregister 1 per-instance resource mbean
+    // Unregister 1 per-instance resource mbean and message queue mbean
     Thread.sleep(1000);
-    Assert.assertTrue(nMbeansUnregistered == listener._nMbeansUnregistered - 1);
-    Assert.assertTrue(nMbeansRegistered == listener._nMbeansRegistered);
+    Assert.assertEquals(nMbeansUnregistered, listener._nMbeansUnregistered - 2);
+    Assert.assertEquals(nMbeansRegistered, listener._nMbeansRegistered);
 
     HelixDataAccessor accessor = _participants[n - 1].getHelixDataAccessor();
     String firstControllerName =
@@ -223,8 +219,8 @@ public class TestClusterStatusMonitorLifecycle extends ZkIntegrationTestBase {
     // 1 cluster status monitor, 1 resource monitor, 5 instances
     // Unregister 1+4+1 per-instance resource mbean
     // Register 4 per-instance resource mbean
-    Assert.assertTrue(nMbeansUnregistered == listener._nMbeansUnregistered - 13);
-    Assert.assertTrue(nMbeansRegistered == listener._nMbeansRegistered - 11);
+    Assert.assertEquals(nMbeansUnregistered, listener._nMbeansUnregistered - 16);
+    Assert.assertEquals(nMbeansRegistered, listener._nMbeansRegistered - 12);
 
     String instanceName = "localhost0_" + (12918 + 0);
     _participants[0] = new MockParticipantManager(ZK_ADDR, _firstClusterName, instanceName);
@@ -234,8 +230,8 @@ public class TestClusterStatusMonitorLifecycle extends ZkIntegrationTestBase {
     // No change in instance/resource mbean
     // Register 1 per-instance resource mbean
     Thread.sleep(1000);
-    Assert.assertTrue(nMbeansUnregistered == listener._nMbeansUnregistered - 13);
-    Assert.assertTrue(nMbeansRegistered == listener._nMbeansRegistered - 12);
+    Assert.assertEquals(nMbeansUnregistered, listener._nMbeansUnregistered - 16);
+    Assert.assertEquals(nMbeansRegistered, listener._nMbeansRegistered - 14);
 
     // Add a resource
     // Register 1 resource mbean
@@ -249,16 +245,16 @@ public class TestClusterStatusMonitorLifecycle extends ZkIntegrationTestBase {
         Integer.parseInt(idealState.getReplicas()));
 
     Thread.sleep(1000);
-    Assert.assertTrue(nMbeansUnregistered == listener._nMbeansUnregistered - 13);
-    Assert.assertTrue(nMbeansRegistered == listener._nMbeansRegistered - 18);
+    Assert.assertEquals(nMbeansUnregistered, listener._nMbeansUnregistered - 16);
+    Assert.assertEquals(nMbeansRegistered, listener._nMbeansRegistered - 20);
 
     // Remove a resource
     // No change in instance/resource mbean
     // Unregister 5 per-instance resource mbean
     setupTool.dropResourceFromCluster(_firstClusterName, "TestDB1");
     Thread.sleep(1000);
-    Assert.assertTrue(nMbeansUnregistered == listener._nMbeansUnregistered - 18);
-    Assert.assertTrue(nMbeansRegistered == listener._nMbeansRegistered - 18);
+    Assert.assertEquals(nMbeansUnregistered, listener._nMbeansUnregistered - 21);
+    Assert.assertEquals(nMbeansRegistered, listener._nMbeansRegistered - 20);
 
   }
 }


[06/17] [HELIX-395] Remove old Helix alert/stat modules

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/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 3bc1985..e3d9c7b 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
@@ -35,7 +35,6 @@ import org.apache.helix.ConfigChangeListener;
 import org.apache.helix.ControllerChangeListener;
 import org.apache.helix.CurrentStateChangeListener;
 import org.apache.helix.ExternalViewChangeListener;
-import org.apache.helix.HealthStateChangeListener;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixException;
@@ -57,11 +56,6 @@ import org.apache.helix.PropertyType;
 import org.apache.helix.ScopedConfigChangeListener;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.controller.GenericHelixController;
-import org.apache.helix.healthcheck.HealthStatsAggregationTask;
-import org.apache.helix.healthcheck.HealthStatsAggregator;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollector;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
-import org.apache.helix.healthcheck.ParticipantHealthReportTask;
 import org.apache.helix.messaging.DefaultMessagingService;
 import org.apache.helix.model.ConfigScope;
 import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
@@ -123,7 +117,6 @@ public class ZKHelixManager implements HelixManager, IZkStateListener {
    * participant fields
    */
   private final StateMachineEngine _stateMachineEngine;
-  private final ParticipantHealthReportCollectorImpl _participantHealthInfoCollector;
   private final List<HelixTimerTask> _timerTasks = new ArrayList<HelixTimerTask>();
 
   /**
@@ -220,34 +213,19 @@ public class ZKHelixManager implements HelixManager, IZkStateListener {
     switch (instanceType) {
     case PARTICIPANT:
       _stateMachineEngine = new HelixStateMachineEngine(this);
-      _participantHealthInfoCollector =
-          new ParticipantHealthReportCollectorImpl(this, _instanceName);
-
-      _timerTasks.add(new ParticipantHealthReportTask(_participantHealthInfoCollector));
-
       break;
     case CONTROLLER:
       _stateMachineEngine = null;
-      _participantHealthInfoCollector = null;
-      _controllerTimerTasks.add(new HealthStatsAggregationTask(new HealthStatsAggregator(this)));
       _controllerTimerTasks.add(new StatusDumpTask(_zkclient, this));
 
       break;
     case CONTROLLER_PARTICIPANT:
       _stateMachineEngine = new HelixStateMachineEngine(this);
-      _participantHealthInfoCollector =
-          new ParticipantHealthReportCollectorImpl(this, _instanceName);
-
-      _timerTasks.add(new ParticipantHealthReportTask(_participantHealthInfoCollector));
-
-      _controllerTimerTasks.add(new HealthStatsAggregationTask(new HealthStatsAggregator(this)));
       _controllerTimerTasks.add(new StatusDumpTask(_zkclient, this));
-
       break;
     case ADMINISTRATOR:
     case SPECTATOR:
       _stateMachineEngine = null;
-      _participantHealthInfoCollector = null;
       break;
     default:
       throw new IllegalArgumentException("unrecognized type: " + instanceType);
@@ -419,15 +397,6 @@ public class ZKHelixManager implements HelixManager, IZkStateListener {
   }
 
   @Override
-  public void addHealthStateChangeListener(HealthStateChangeListener listener, String instanceName)
-      throws Exception {
-    addListener(listener, new Builder(_clusterName).healthReports(instanceName), ChangeType.HEALTH,
-        new EventType[] {
-            EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated
-        });
-  }
-
-  @Override
   public void addExternalViewChangeListener(ExternalViewChangeListener listener) throws Exception {
     addListener(listener, new Builder(_clusterName).externalViews(), ChangeType.EXTERNAL_VIEW,
         new EventType[] {
@@ -625,8 +594,8 @@ public class ZKHelixManager implements HelixManager, IZkStateListener {
       String path = PropertyPathConfig.getPath(PropertyType.PROPERTYSTORE, _clusterName);
       String fallbackPath = String.format("/%s/%s", _clusterName, "HELIX_PROPERTYSTORE");
       _helixPropertyStore =
-          new AutoFallbackPropertyStore<ZNRecord>(new ZkBaseDataAccessor<ZNRecord>(_zkclient), path,
-              fallbackPath);
+          new AutoFallbackPropertyStore<ZNRecord>(new ZkBaseDataAccessor<ZNRecord>(_zkclient),
+              path, fallbackPath);
     }
 
     return _helixPropertyStore;
@@ -651,12 +620,6 @@ public class ZKHelixManager implements HelixManager, IZkStateListener {
   }
 
   @Override
-  public ParticipantHealthReportCollector getHealthReportCollector() {
-    checkConnected();
-    return _participantHealthInfoCollector;
-  }
-
-  @Override
   public InstanceType getInstanceType() {
     return _instanceType;
   }
@@ -880,11 +843,6 @@ public class ZKHelixManager implements HelixManager, IZkStateListener {
      * setup message listener
      */
     participantHelper.setupMsgHandler();
-
-    /**
-     * start health check timer task
-     */
-    participantHelper.createHealthCheckPath();
   }
 
   void handleNewSessionAsController() {

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCallbackHandler.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCallbackHandler.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCallbackHandler.java
index 8d39268..5961fe3 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCallbackHandler.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCallbackHandler.java
@@ -41,7 +41,6 @@ import org.apache.helix.ConfigChangeListener;
 import org.apache.helix.ControllerChangeListener;
 import org.apache.helix.CurrentStateChangeListener;
 import org.apache.helix.ExternalViewChangeListener;
-import org.apache.helix.HealthStateChangeListener;
 import org.apache.helix.HelixConnection;
 import org.apache.helix.HelixConstants.ChangeType;
 import org.apache.helix.HelixDataAccessor;
@@ -61,7 +60,6 @@ import org.apache.helix.ScopedConfigChangeListener;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.ExternalView;
-import org.apache.helix.model.HealthStat;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.LiveInstance;
@@ -230,15 +228,6 @@ public class ZkCallbackHandler implements IZkChildListener, IZkDataListener
         ControllerChangeListener controllerChangelistener = (ControllerChangeListener) _listener;
         subscribeForChanges(changeContext, _path, true, false);
         controllerChangelistener.onControllerChange(changeContext);
-      } else if (_changeType == ChangeType.HEALTH) {
-        HealthStateChangeListener healthStateChangeListener = (HealthStateChangeListener) _listener;
-        subscribeForChanges(changeContext, _path, true, true); // TODO: figure out
-        // settings here
-        String instanceName = PropertyPathConfig.getInstanceNameFromPath(_path);
-
-        List<HealthStat> healthReportList = _accessor.getChildValues(_propertyKey);
-
-        healthStateChangeListener.onHealthChange(instanceName, healthReportList, changeContext);
       }
 
       long end = System.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixConnection.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixConnection.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixConnection.java
index 8c9c376..ea2536d 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixConnection.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixConnection.java
@@ -38,7 +38,6 @@ import org.apache.helix.ConfigChangeListener;
 import org.apache.helix.ControllerChangeListener;
 import org.apache.helix.CurrentStateChangeListener;
 import org.apache.helix.ExternalViewChangeListener;
-import org.apache.helix.HealthStateChangeListener;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixAutoController;
 import org.apache.helix.HelixConnection;
@@ -380,16 +379,6 @@ public class ZkHelixConnection implements HelixConnection, IZkStateListener {
   }
 
   @Override
-  public void addHealthStateChangeListener(HelixRole role, HealthStateChangeListener listener,
-      ClusterId clusterId, ParticipantId participantId) {
-    addListener(role, listener,
-        new PropertyKey.Builder(clusterId.stringify()).healthReports(participantId.stringify()),
-        ChangeType.HEALTH, new EventType[] {
-            EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated
-        });
-  }
-
-  @Override
   public void addExternalViewChangeListener(HelixRole role, ExternalViewChangeListener listener,
       ClusterId clusterId) {
     addListener(role, listener, new PropertyKey.Builder(clusterId.stringify()).externalViews(),

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixController.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixController.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixController.java
index 51bb746..475b3cf 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixController.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixController.java
@@ -37,8 +37,6 @@ import org.apache.helix.api.id.ClusterId;
 import org.apache.helix.api.id.ControllerId;
 import org.apache.helix.api.id.Id;
 import org.apache.helix.controller.GenericHelixController;
-import org.apache.helix.healthcheck.HealthStatsAggregationTask;
-import org.apache.helix.healthcheck.HealthStatsAggregator;
 import org.apache.helix.messaging.DefaultMessagingService;
 import org.apache.helix.messaging.handling.MessageHandlerFactory;
 import org.apache.helix.model.LiveInstance;
@@ -74,7 +72,6 @@ public class ZkHelixController implements HelixController {
     _manager = new HelixConnectionAdaptor(this);
     _leaderElection = new ZkHelixLeaderElection(this, _pipeline);
 
-    _timerTasks.add(new HealthStatsAggregationTask(new HealthStatsAggregator(_manager)));
     _timerTasks.add(new StatusDumpTask(clusterId, _manager.getHelixDataAccessor()));
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixParticipant.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixParticipant.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixParticipant.java
index 0c6e772..d42b7b7 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixParticipant.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixParticipant.java
@@ -45,8 +45,6 @@ import org.apache.helix.api.config.ParticipantConfig;
 import org.apache.helix.api.id.ClusterId;
 import org.apache.helix.api.id.Id;
 import org.apache.helix.api.id.ParticipantId;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
-import org.apache.helix.healthcheck.ParticipantHealthReportTask;
 import org.apache.helix.messaging.DefaultMessagingService;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.HelixConfigScope;
@@ -76,7 +74,6 @@ public class ZkHelixParticipant implements HelixParticipant {
   final DefaultMessagingService _messagingService;
   final List<PreConnectCallback> _preConnectCallbacks;
   final List<HelixTimerTask> _timerTasks;
-  final ParticipantHealthReportCollectorImpl _participantHealthInfoCollector;
 
   /**
    * state-transition message handler factory for helix-participant
@@ -103,10 +100,6 @@ public class ZkHelixParticipant implements HelixParticipant {
     _stateMachineEngine = new HelixStateMachineEngine(manager);
     _preConnectCallbacks = new ArrayList<PreConnectCallback>();
     _timerTasks = new ArrayList<HelixTimerTask>();
-    _participantHealthInfoCollector =
-        new ParticipantHealthReportCollectorImpl(manager, participantId.stringify());
-
-    _timerTasks.add(new ParticipantHealthReportTask(_participantHealthInfoCollector));
 
   }
 
@@ -361,17 +354,6 @@ public class ZkHelixParticipant implements HelixParticipant {
     _messagingService.onConnected();
   }
 
-  /**
-   * create zk path for health check info
-   * TODO move it to cluster-setup
-   */
-  private void createHealthCheckPath() {
-    PropertyKey healthCheckInfoKey = _keyBuilder.healthReports(_participantId.stringify());
-    if (_accessor.createProperty(healthCheckInfoKey, null)) {
-      LOG.info("Created healthcheck info path: " + healthCheckInfoKey.getPath());
-    }
-  }
-
   void init() {
     /**
      * from here on, we are dealing with new session
@@ -403,7 +385,6 @@ public class ZkHelixParticipant implements HelixParticipant {
     /**
      * start health check timer task
      */
-    createHealthCheckPath();
     startTimerTasks();
 
     /**

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/model/HealthStat.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/HealthStat.java b/helix-core/src/main/java/org/apache/helix/model/HealthStat.java
deleted file mode 100644
index b884534..0000000
--- a/helix-core/src/main/java/org/apache/helix/model/HealthStat.java
+++ /dev/null
@@ -1,147 +0,0 @@
-package org.apache.helix.model;
-
-/*
- * 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.HashMap;
-import java.util.Map;
-
-import org.apache.helix.HelixProperty;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.alerts.ExpressionParser;
-import org.apache.helix.alerts.StatsHolder;
-import org.apache.helix.model.Message.Attributes;
-import org.apache.log4j.Logger;
-
-/**
- * Represents a set of properties that can be queried to determine the health of instances on a
- * Helix-managed cluster
- */
-public class HealthStat extends HelixProperty {
-  /**
-   * Queryable health statistic properties
-   */
-  public enum HealthStatProperty {
-    FIELDS
-  }
-
-  private static final Logger _logger = Logger.getLogger(HealthStat.class.getName());
-
-  /**
-   * Instantiate with an identifier
-   * @param id the name of these statistics
-   */
-  public HealthStat(String id) {
-    super(id);
-  }
-
-  /**
-   * Instantiate with a pre-populated record
-   * @param record a ZNRecord corresponding to health statistics
-   */
-  public HealthStat(ZNRecord record) {
-    super(record);
-    if (getCreateTimeStamp() == 0) {
-      _record.setLongField(Attributes.CREATE_TIMESTAMP.toString(), new Date().getTime());
-    }
-  }
-
-  /**
-   * Get when these statistics were last modified
-   * @return a UNIX timestamp
-   */
-  public long getLastModifiedTimeStamp() {
-    return _record.getModifiedTime();
-  }
-
-  /**
-   * Get when these statistics were created
-   * @return a UNIX timestamp
-   */
-  public long getCreateTimeStamp() {
-    return _record.getLongField(Attributes.CREATE_TIMESTAMP.toString(), 0L);
-  }
-
-  /**
-   * Get the value of a test field corresponding to a request count
-   * @return the number of requests
-   */
-  public String getTestField() {
-    return _record.getSimpleField("requestCountStat");
-  }
-
-  /**
-   * Set a group of heath statistics, grouped by the statistic
-   * @param healthFields a map of statistic name, the corresponding entity, and the value
-   */
-  public void setHealthFields(Map<String, Map<String, String>> healthFields) {
-    _record.setMapFields(healthFields);
-  }
-
-  /**
-   * Create a key based on a parent key, instance, and statistic
-   * @param instance the instance for which these statistics exist
-   * @param parentKey the originating key
-   * @param statName the statistic
-   * @return a unified key
-   */
-  public String buildCompositeKey(String instance, String parentKey, String statName) {
-    String delim = ExpressionParser.statFieldDelim;
-    return instance + delim + parentKey + delim + statName;
-  }
-
-  /**
-   * Get all the health statistics for a given instance
-   * @param instanceName the instance for which to get health statistics
-   * @return a map of (instance and statistic, value or timestamp, value) triples
-   */
-  public Map<String, Map<String, String>> getHealthFields(String instanceName) // ,
-                                                                               // String
-                                                                               // timestamp)
-  {
-    // XXX: need to do some conversion of input format to the format that stats
-    // computation wants
-    Map<String, Map<String, String>> currMapFields = _record.getMapFields();
-    Map<String, Map<String, String>> convertedMapFields =
-        new HashMap<String, Map<String, String>>();
-    for (String key : currMapFields.keySet()) {
-      Map<String, String> currMap = currMapFields.get(key);
-      String timestamp = _record.getStringField(StatsHolder.TIMESTAMP_NAME, "-1");
-      for (String subKey : currMap.keySet()) {
-        if (subKey.equals("StatsHolder.TIMESTAMP_NAME")) { // don't want to get timestamp again
-          continue;
-        }
-        String compositeKey = buildCompositeKey(instanceName, key, subKey);
-        String value = currMap.get(subKey);
-        Map<String, String> convertedMap = new HashMap<String, String>();
-        convertedMap.put(StatsHolder.VALUE_NAME, value);
-        convertedMap.put(StatsHolder.TIMESTAMP_NAME, timestamp);
-        convertedMapFields.put(compositeKey, convertedMap);
-      }
-    }
-    return convertedMapFields;
-  }
-
-  @Override
-  public boolean isValid() {
-    // TODO Auto-generated method stub
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterAlertItem.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterAlertItem.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterAlertItem.java
deleted file mode 100644
index 19634e0..0000000
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterAlertItem.java
+++ /dev/null
@@ -1,88 +0,0 @@
-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.Date;
-
-import org.apache.helix.alerts.AlertValueAndStatus;
-
-public class ClusterAlertItem implements ClusterAlertItemMBean {
-  String _alertItemName;
-  double _alertValue;
-  int _alertFired;
-  String _additionalInfo = "";
-  AlertValueAndStatus _valueAndStatus;
-  long _lastUpdateTime = 0;
-
-  public ClusterAlertItem(String name, AlertValueAndStatus valueAndStatus) {
-    _valueAndStatus = valueAndStatus;
-    _alertItemName = name;
-    refreshValues();
-  }
-
-  @Override
-  public String getSensorName() {
-    return _alertItemName;
-  }
-
-  @Override
-  public double getAlertValue() {
-    return _alertValue;
-  }
-
-  public void setValueMap(AlertValueAndStatus valueAndStatus) {
-    _valueAndStatus = valueAndStatus;
-    refreshValues();
-  }
-
-  void refreshValues() {
-    _lastUpdateTime = new Date().getTime();
-    if (_valueAndStatus.getValue().getElements().size() > 0) {
-      _alertValue = Double.parseDouble(_valueAndStatus.getValue().getElements().get(0));
-    } else {
-      _alertValue = 0;
-    }
-    _alertFired = _valueAndStatus.isFired() ? 1 : 0;
-  }
-
-  @Override
-  public int getAlertFired() {
-    return _alertFired;
-  }
-
-  public void setAdditionalInfo(String additionalInfo) {
-    _additionalInfo = additionalInfo;
-  }
-
-  @Override
-  public String getAdditionalInfo() {
-    return _additionalInfo;
-  }
-
-  public void reset() {
-    _alertFired = 0;
-    _additionalInfo = "";
-    _alertValue = 0;
-  }
-
-  public long getLastUpdateTime() {
-    return _lastUpdateTime;
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterAlertItemMBean.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterAlertItemMBean.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterAlertItemMBean.java
deleted file mode 100644
index 573391d..0000000
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterAlertItemMBean.java
+++ /dev/null
@@ -1,30 +0,0 @@
-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.
- */
-
-public interface ClusterAlertItemMBean {
-  String getSensorName();
-
-  double getAlertValue();
-
-  int getAlertFired();
-
-  String getAdditionalInfo();
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterAlertMBeanCollection.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterAlertMBeanCollection.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterAlertMBeanCollection.java
deleted file mode 100644
index 0d06ca3..0000000
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterAlertMBeanCollection.java
+++ /dev/null
@@ -1,275 +0,0 @@
-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.io.StringWriter;
-import java.lang.management.ManagementFactory;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-
-import javax.management.MBeanServer;
-import javax.management.ObjectName;
-
-import org.apache.helix.PropertyType;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.alerts.AlertParser;
-import org.apache.helix.alerts.AlertValueAndStatus;
-import org.apache.helix.alerts.Tuple;
-import org.apache.log4j.Logger;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.map.SerializationConfig;
-
-public class ClusterAlertMBeanCollection {
-  public static String DOMAIN_ALERT = "HelixAlerts";
-  public static String ALERT_SUMMARY = "AlertSummary";
-
-  private static final Logger _logger = Logger.getLogger(ClusterAlertMBeanCollection.class);
-  ConcurrentHashMap<String, ClusterAlertItem> _alertBeans =
-      new ConcurrentHashMap<String, ClusterAlertItem>();
-
-  Map<String, String> _recentAlertDelta;
-  ClusterAlertSummary _clusterAlertSummary;
-  ZNRecord _alertHistory = new ZNRecord(PropertyType.ALERT_HISTORY.toString());
-  Set<String> _previousFiredAlerts = new HashSet<String>();
-  // 5 min for mbean freshness threshold
-  public static final long ALERT_NOCHANGE_THRESHOLD = 5 * 60 * 1000;
-
-  final MBeanServer _beanServer;
-
-  public interface ClusterAlertSummaryMBean extends ClusterAlertItemMBean {
-    public String getAlertFiredHistory();
-  }
-
-  class ClusterAlertSummary extends ClusterAlertItem implements ClusterAlertSummaryMBean {
-    public ClusterAlertSummary(String name, AlertValueAndStatus valueAndStatus) {
-      super(name, valueAndStatus);
-    }
-
-    /**
-     * Returns the previous 100 alert mbean turn on / off history
-     */
-    @Override
-    public String getAlertFiredHistory() {
-      try {
-        ObjectMapper mapper = new ObjectMapper();
-        SerializationConfig serializationConfig = mapper.getSerializationConfig();
-        serializationConfig.set(SerializationConfig.Feature.INDENT_OUTPUT, true);
-        StringWriter sw = new StringWriter();
-        mapper.writeValue(sw, _alertHistory);
-        return sw.toString();
-      } catch (Exception e) {
-        _logger.warn("", e);
-        return "";
-      }
-    }
-  }
-
-  public ClusterAlertMBeanCollection() {
-    _beanServer = ManagementFactory.getPlatformMBeanServer();
-  }
-
-  public Collection<ClusterAlertItemMBean> getCurrentAlertMBeans() {
-    ArrayList<ClusterAlertItemMBean> beans = new ArrayList<ClusterAlertItemMBean>();
-    for (ClusterAlertItem item : _alertBeans.values()) {
-      beans.add(item);
-    }
-    return beans;
-  }
-
-  void onNewAlertMbeanAdded(ClusterAlertItemMBean bean) {
-    try {
-      _logger.info("alert bean " + bean.getSensorName() + " exposed to jmx");
-      System.out.println("alert bean " + bean.getSensorName() + " exposed to jmx");
-      ObjectName objectName = new ObjectName(DOMAIN_ALERT + ":alert=" + bean.getSensorName());
-      register(bean, objectName);
-    } catch (Exception e) {
-      _logger.error("", e);
-      e.printStackTrace();
-    }
-  }
-
-  public void setAlerts(String originAlert, Map<String, AlertValueAndStatus> alertResultMap,
-      String clusterName) {
-    if (alertResultMap == null) {
-      _logger.warn("null alertResultMap");
-      return;
-    }
-    for (String alertName : alertResultMap.keySet()) {
-      String beanName = "";
-      if (alertName.length() > 1) {
-        String comparator = AlertParser.getComponent(AlertParser.COMPARATOR_NAME, originAlert);
-        String constant = AlertParser.getComponent(AlertParser.CONSTANT_NAME, originAlert);
-        beanName = "(" + alertName + ")" + comparator + "(" + constant + ")";
-      } else {
-        beanName = originAlert + "--(" + alertName + ")";
-      }
-      // This is to make JMX happy; certain charaters cannot be in JMX bean name
-      beanName = beanName.replace('*', '%').replace('=', '#').replace(',', ';');
-      if (!_alertBeans.containsKey(beanName)) {
-        ClusterAlertItem item = new ClusterAlertItem(beanName, alertResultMap.get(alertName));
-        onNewAlertMbeanAdded(item);
-        _alertBeans.put(beanName, item);
-      } else {
-        _alertBeans.get(beanName).setValueMap(alertResultMap.get(alertName));
-      }
-    }
-    refreshSummayAlert(clusterName);
-  }
-
-  public void setAlertHistory(ZNRecord alertHistory) {
-    _alertHistory = alertHistory;
-  }
-
-  /**
-   * The summary alert is a combination of all alerts, if it is on, something is wrong on this
-   * cluster. The additional info contains all alert mbean names that has been fired.
-   */
-  void refreshSummayAlert(String clusterName) {
-    boolean fired = false;
-    String alertsFired = "";
-    String summaryKey = ALERT_SUMMARY + "_" + clusterName;
-    for (String key : _alertBeans.keySet()) {
-      if (!key.equals(summaryKey)) {
-        ClusterAlertItem item = _alertBeans.get(key);
-        fired = (item.getAlertFired() == 1) | fired;
-        if (item.getAlertFired() == 1) {
-          alertsFired += item._alertItemName;
-          alertsFired += ";";
-        }
-      }
-    }
-    Tuple<String> t = new Tuple<String>();
-    t.add("0");
-    AlertValueAndStatus summaryStatus = new AlertValueAndStatus(t, fired);
-    if (!_alertBeans.containsKey(summaryKey)) {
-      ClusterAlertSummary item = new ClusterAlertSummary(summaryKey, summaryStatus);
-      onNewAlertMbeanAdded(item);
-      item.setAdditionalInfo(alertsFired);
-      _alertBeans.put(summaryKey, item);
-      _clusterAlertSummary = item;
-    } else {
-      _alertBeans.get(summaryKey).setValueMap(summaryStatus);
-      _alertBeans.get(summaryKey).setAdditionalInfo(alertsFired);
-    }
-  }
-
-  void register(Object bean, ObjectName name) {
-    try {
-      _beanServer.unregisterMBean(name);
-    } catch (Exception e) {
-    }
-    try {
-      _beanServer.registerMBean(bean, name);
-    } catch (Exception e) {
-      _logger.error("Could not register MBean", e);
-    }
-  }
-
-  public void reset() {
-    for (String beanName : _alertBeans.keySet()) {
-      ClusterAlertItem item = _alertBeans.get(beanName);
-      item.reset();
-      try {
-        ObjectName objectName = new ObjectName(DOMAIN_ALERT + ":alert=" + item.getSensorName());
-        _beanServer.unregisterMBean(objectName);
-      } catch (Exception e) {
-        _logger.warn("", e);
-      }
-    }
-    _alertBeans.clear();
-  }
-
-  public void refreshAlertDelta(String clusterName) {
-    // Update the alert turn on/turn off history
-    String summaryKey = ALERT_SUMMARY + "_" + clusterName;
-    Set<String> currentFiredAlerts = new HashSet<String>();
-    for (String key : _alertBeans.keySet()) {
-      if (!key.equals(summaryKey)) {
-        ClusterAlertItem item = _alertBeans.get(key);
-        if (item.getAlertFired() == 1) {
-          currentFiredAlerts.add(item._alertItemName);
-        }
-      }
-    }
-
-    Map<String, String> onOffAlertsMap = new HashMap<String, String>();
-    for (String alertName : currentFiredAlerts) {
-      if (!_previousFiredAlerts.contains(alertName)) {
-        onOffAlertsMap.put(alertName, "ON");
-        _logger.info(alertName + " ON");
-        _previousFiredAlerts.add(alertName);
-      }
-    }
-    for (String cachedAlert : _previousFiredAlerts) {
-      if (!currentFiredAlerts.contains(cachedAlert)) {
-        onOffAlertsMap.put(cachedAlert, "OFF");
-        _logger.info(cachedAlert + " OFF");
-      }
-    }
-    for (String key : onOffAlertsMap.keySet()) {
-      if (onOffAlertsMap.get(key).equals("OFF")) {
-        _previousFiredAlerts.remove(key);
-      }
-    }
-    if (onOffAlertsMap.size() == 0) {
-      _logger.info("No MBean change");
-    }
-    _recentAlertDelta = onOffAlertsMap;
-
-    checkMBeanFreshness(ALERT_NOCHANGE_THRESHOLD);
-  }
-
-  public Map<String, String> getRecentAlertDelta() {
-    return _recentAlertDelta;
-  }
-
-  /**
-   * Remove mbeans that has not been changed for thresholdInMs MS
-   */
-  void checkMBeanFreshness(long thresholdInMs) {
-    long now = new Date().getTime();
-    Set<String> oldBeanNames = new HashSet<String>();
-    // Get mbean items that has not been updated for thresholdInMs
-    for (String beanName : _alertBeans.keySet()) {
-      ClusterAlertItem item = _alertBeans.get(beanName);
-      if (now - item.getLastUpdateTime() > thresholdInMs) {
-        oldBeanNames.add(beanName);
-        _logger.info("bean " + beanName + " has not been updated for " + thresholdInMs + " MS");
-      }
-    }
-    for (String beanName : oldBeanNames) {
-      ClusterAlertItem item = _alertBeans.get(beanName);
-      _alertBeans.remove(beanName);
-      try {
-        item.reset();
-        ObjectName objectName = new ObjectName(DOMAIN_ALERT + ":alert=" + item.getSensorName());
-        _beanServer.unregisterMBean(objectName);
-      } catch (Exception e) {
-        _logger.warn("", e);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/tools/ClusterSetup.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/tools/ClusterSetup.java b/helix-core/src/main/java/org/apache/helix/tools/ClusterSetup.java
index 4479f97..0a8a41d 100644
--- a/helix-core/src/main/java/org/apache/helix/tools/ClusterSetup.java
+++ b/helix-core/src/main/java/org/apache/helix/tools/ClusterSetup.java
@@ -119,12 +119,6 @@ public class ClusterSetup {
   // help
   public static final String help = "help";
 
-  // stats/alerts
-  public static final String addStat = "addStat";
-  public static final String addAlert = "addAlert";
-  public static final String dropStat = "dropStat";
-  public static final String dropAlert = "dropAlert";
-
   // get/set/remove configs
   public static final String getConfig = "getConfig";
   public static final String setConfig = "setConfig";
@@ -857,17 +851,6 @@ public class ClusterSetup {
     listStateModelOption.setRequired(false);
     listStateModelOption.setArgName("clusterName stateModelName");
 
-    Option addStatOption =
-        OptionBuilder.withLongOpt(addStat).withDescription("Add a persistent stat").create();
-    addStatOption.setArgs(2);
-    addStatOption.setRequired(false);
-    addStatOption.setArgName("clusterName statName");
-    Option addAlertOption =
-        OptionBuilder.withLongOpt(addAlert).withDescription("Add an alert").create();
-    addAlertOption.setArgs(2);
-    addAlertOption.setRequired(false);
-    addAlertOption.setArgName("clusterName alertName");
-
     Option addInstanceTagOption =
         OptionBuilder.withLongOpt(addInstanceTag).withDescription("Add a tag to instance").create();
     addInstanceTagOption.setArgs(3);
@@ -880,17 +863,6 @@ public class ClusterSetup {
     removeInstanceTagOption.setRequired(false);
     removeInstanceTagOption.setArgName("clusterName instanceName tag");
 
-    Option dropStatOption =
-        OptionBuilder.withLongOpt(dropStat).withDescription("Drop a persistent stat").create();
-    dropStatOption.setArgs(2);
-    dropStatOption.setRequired(false);
-    dropStatOption.setArgName("clusterName statName");
-    Option dropAlertOption =
-        OptionBuilder.withLongOpt(dropAlert).withDescription("Drop an alert").create();
-    dropAlertOption.setArgs(2);
-    dropAlertOption.setRequired(false);
-    dropAlertOption.setArgName("clusterName alertName");
-
     // TODO need deal with resource-names containing ","
     // set/get/remove configs options
     Option setConfOption =
@@ -976,10 +948,6 @@ public class ClusterSetup {
     group.addOption(addStateModelDefOption);
     group.addOption(listStateModelsOption);
     group.addOption(listStateModelOption);
-    group.addOption(addStatOption);
-    group.addOption(addAlertOption);
-    group.addOption(dropStatOption);
-    group.addOption(dropAlertOption);
     group.addOption(addResourcePropertyOption);
     group.addOption(removeResourcePropertyOption);
 
@@ -1355,26 +1323,6 @@ public class ClusterSetup {
 
       setupTool.addIdealState(clusterName, resourceName, idealStateFile);
       return 0;
-    } else if (cmd.hasOption(addStat)) {
-      String clusterName = cmd.getOptionValues(addStat)[0];
-      String statName = cmd.getOptionValues(addStat)[1];
-
-      setupTool.getClusterManagementTool().addStat(clusterName, statName);
-    } else if (cmd.hasOption(addAlert)) {
-      String clusterName = cmd.getOptionValues(addAlert)[0];
-      String alertName = cmd.getOptionValues(addAlert)[1];
-
-      setupTool.getClusterManagementTool().addAlert(clusterName, alertName);
-    } else if (cmd.hasOption(dropStat)) {
-      String clusterName = cmd.getOptionValues(dropStat)[0];
-      String statName = cmd.getOptionValues(dropStat)[1];
-
-      setupTool.getClusterManagementTool().dropStat(clusterName, statName);
-    } else if (cmd.hasOption(dropAlert)) {
-      String clusterName = cmd.getOptionValues(dropAlert)[0];
-      String alertName = cmd.getOptionValues(dropAlert)[1];
-
-      setupTool.getClusterManagementTool().dropAlert(clusterName, alertName);
     } else if (cmd.hasOption(dropResource)) {
       String clusterName = cmd.getOptionValues(dropResource)[0];
       String resourceName = cmd.getOptionValues(dropResource)[1];

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/tools/NewClusterSetup.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/tools/NewClusterSetup.java b/helix-core/src/main/java/org/apache/helix/tools/NewClusterSetup.java
index ea28c76..c8281ba 100644
--- a/helix-core/src/main/java/org/apache/helix/tools/NewClusterSetup.java
+++ b/helix-core/src/main/java/org/apache/helix/tools/NewClusterSetup.java
@@ -917,34 +917,6 @@ public class NewClusterSetup {
     accessor.resetParticipants(participantIds);
   }
 
-  void addStat(String[] optValues) {
-    String clusterName = optValues[0];
-    String statName = optValues[1];
-    ClusterAccessor accessor = clusterAccessor(clusterName);
-    accessor.addStat(statName);
-  }
-
-  void dropStat(String[] optValues) {
-    String clusterName = optValues[0];
-    String statName = optValues[1];
-    ClusterAccessor accessor = clusterAccessor(clusterName);
-    accessor.dropStat(statName);
-  }
-
-  void addAlert(String[] optValues) {
-    String clusterName = optValues[0];
-    String alertName = optValues[1];
-    ClusterAccessor accessor = clusterAccessor(clusterName);
-    accessor.addAlert(alertName);
-  }
-
-  void dropAlert(String[] optValues) {
-    String clusterName = optValues[0];
-    String alertName = optValues[1];
-    ClusterAccessor accessor = clusterAccessor(clusterName);
-    accessor.dropAlert(alertName);
-  }
-
   void expandResource(String[] optValues) {
     String clusterName = optValues[0];
     String resourceName = optValues[1];
@@ -1146,18 +1118,6 @@ public class NewClusterSetup {
         case resetResource:
           setup.resetResource(optValues);
           break;
-        case addStat:
-          setup.addStat(optValues);
-          break;
-        case addAlert:
-          setup.addAlert(optValues);
-          break;
-        case dropStat:
-          setup.dropStat(optValues);
-          break;
-        case dropAlert:
-          setup.dropAlert(optValues);
-          break;
         case getConfig:
           setup.getConfig(optValues);
           break;

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/util/HelixUtil.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/util/HelixUtil.java b/helix-core/src/main/java/org/apache/helix/util/HelixUtil.java
index f289211..48e29e5 100644
--- a/helix-core/src/main/java/org/apache/helix/util/HelixUtil.java
+++ b/helix-core/src/main/java/org/apache/helix/util/HelixUtil.java
@@ -115,22 +115,6 @@ public final class HelixUtil {
     return getInstancePropertyPath(clusterName, instanceName, PropertyType.STATUSUPDATES);
   }
 
-  public static String getHealthPath(String clusterName, String instanceName) {
-    return PropertyPathConfig.getPath(PropertyType.HEALTHREPORT, clusterName, instanceName);
-  }
-
-  public static String getPersistentStatsPath(String clusterName) {
-    return PropertyPathConfig.getPath(PropertyType.PERSISTENTSTATS, clusterName);
-  }
-
-  public static String getAlertsPath(String clusterName) {
-    return PropertyPathConfig.getPath(PropertyType.ALERTS, clusterName);
-  }
-
-  public static String getAlertStatusPath(String clusterName) {
-    return PropertyPathConfig.getPath(PropertyType.ALERT_STATUS, clusterName);
-  }
-
   public static String getInstanceNameFromPath(String path) {
     // path structure
     // /<cluster_name>/instances/<instance_name>/[currentStates/messages]

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/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 a89d27f..a3a6044 100644
--- a/helix-core/src/test/java/org/apache/helix/Mocks.java
+++ b/helix-core/src/test/java/org/apache/helix/Mocks.java
@@ -31,8 +31,6 @@ import org.I0Itec.zkclient.DataUpdater;
 import org.I0Itec.zkclient.IZkChildListener;
 import org.I0Itec.zkclient.IZkDataListener;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.healthcheck.HealthReportProvider;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollector;
 import org.apache.helix.messaging.AsyncCallback;
 import org.apache.helix.messaging.handling.HelixTaskExecutor;
 import org.apache.helix.messaging.handling.HelixTaskResult;
@@ -189,18 +187,6 @@ public class Mocks {
       return false;
     }
 
-    // @Override
-    // public boolean subscribe(String path, IZkListener listener) {
-    // // TODO Auto-generated method stub
-    // return false;
-    // }
-    //
-    // @Override
-    // public boolean unsubscribe(String path, IZkListener listener) {
-    // // TODO Auto-generated method stub
-    // return false;
-    // }
-
   }
 
   public static class MockStateModel extends StateModel {
@@ -371,12 +357,6 @@ public class Mocks {
     }
 
     @Override
-    public ParticipantHealthReportCollector getHealthReportCollector() {
-      // TODO Auto-generated method stub
-      return null;
-    }
-
-    @Override
     public InstanceType getInstanceType() {
       return InstanceType.PARTICIPANT;
     }
@@ -393,13 +373,6 @@ public class Mocks {
     }
 
     @Override
-    public void addHealthStateChangeListener(HealthStateChangeListener listener, String instanceName)
-        throws Exception {
-      // TODO Auto-generated method stub
-
-    }
-
-    @Override
     public StateMachineEngine getStateMachineEngine() {
       // TODO Auto-generated method stub
       return null;
@@ -426,7 +399,6 @@ public class Mocks {
     @Override
     public void stopTimerTasks() {
       // TODO Auto-generated method stub
-
     }
 
     @Override
@@ -437,7 +409,6 @@ public class Mocks {
     @Override
     public void addPreConnectCallback(PreConnectCallback callback) {
       // TODO Auto-generated method stub
-
     }
 
     @Override
@@ -450,20 +421,17 @@ public class Mocks {
     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
-
     }
 
     @Override
     public void setLiveInstanceInfoProvider(LiveInstanceInfoProvider liveInstanceInfoProvider) {
       // TODO Auto-generated method stub
-
     }
 
     @Override
@@ -475,13 +443,11 @@ public class Mocks {
     @Override
     public void addControllerMessageListener(MessageListener listener) {
       // TODO Auto-generated method stub
-
     }
 
   }
 
-  public static class MockAccessor implements HelixDataAccessor // DataAccessor
-  {
+  public static class MockAccessor implements HelixDataAccessor {
     private final String _clusterName;
     Map<String, ZNRecord> data = new HashMap<String, ZNRecord>();
     private final Builder _propertyKeyBuilder;
@@ -498,35 +464,14 @@ public class Mocks {
     Map<String, ZNRecord> map = new HashMap<String, ZNRecord>();
 
     @Override
-    // public boolean setProperty(PropertyType type, HelixProperty value,
-    // String... keys)
     public boolean setProperty(PropertyKey key, HelixProperty value) {
-      // return setProperty(type, value.getRecord(), keys);
       String path = key.getPath();
       data.put(path, value.getRecord());
       return true;
     }
 
-    // @Override
-    // public boolean setProperty(PropertyType type, ZNRecord value,
-    // String... keys)
-    // {
-    // String path = PropertyPathConfig.getPath(type, _clusterName, keys);
-    // data.put(path, value);
-    // return true;
-    // }
-
-    // @Override
-    // public boolean updateProperty(PropertyType type, HelixProperty value,
-    // String... keys)
-    // {
-    // return updateProperty(type, value.getRecord(), keys);
-    // }
-
     @Override
     public <T extends HelixProperty> boolean updateProperty(PropertyKey key, T value) {
-      // String path = PropertyPathConfig.getPath(type, _clusterName,
-      // keys);
       String path = key.getPath();
       PropertyType type = key.getType();
       if (type.updateOnlyOnExists) {
@@ -556,34 +501,15 @@ public class Mocks {
       return true;
     }
 
-    // @Override
-    // public <T extends HelixProperty> T getProperty(Class<T> clazz,
-    // PropertyType type,
-    // String... keys)
-    // {
-    // ZNRecord record = getProperty(type, keys);
-    // if (record == null)
-    // {
-    // return null;
-    // }
-    // return HelixProperty.convertToTypedInstance(clazz, record);
-    // }
-
     @SuppressWarnings("unchecked")
     @Override
-    public <T extends HelixProperty> T getProperty(PropertyKey key)
-    // public ZNRecord getProperty(PropertyType type, String... keys)
-    {
-      // String path = PropertyPathConfig.getPath(type, _clusterName,
-      // keys);
+    public <T extends HelixProperty> T getProperty(PropertyKey key) {
       String path = key.getPath();
       return (T) HelixProperty.convertToTypedInstance(key.getTypeClass(), data.get(path));
     }
 
     @Override
-    public boolean removeProperty(PropertyKey key)
-    // public boolean removeProperty(PropertyType type, String... keys)
-    {
+    public boolean removeProperty(PropertyKey key) {
       String path = key.getPath(); // PropertyPathConfig.getPath(type,
       // _clusterName, keys);
       data.remove(path);
@@ -591,12 +517,9 @@ public class Mocks {
     }
 
     @Override
-    public List<String> getChildNames(PropertyKey propertyKey)
-    // public List<String> getChildNames(PropertyType type, String... keys)
-    {
+    public List<String> getChildNames(PropertyKey propertyKey) {
       List<String> child = new ArrayList<String>();
-      String path = propertyKey.getPath(); // PropertyPathConfig.getPath(type,
-      // _clusterName, keys);
+      String path = propertyKey.getPath();
       for (String key : data.keySet()) {
         if (key.startsWith(path)) {
           String[] keySplit = key.split("\\/");
@@ -609,21 +532,9 @@ public class Mocks {
       return child;
     }
 
-    // @Override
-    // public <T extends HelixProperty> List<T> getChildValues(Class<T>
-    // clazz, PropertyType type,
-    // String... keys)
-    // {
-    // List<ZNRecord> list = getChildValues(type, keys);
-    // return HelixProperty.convertToTypedList(clazz, list);
-    // }
-
     @SuppressWarnings("unchecked")
     @Override
-    public <T extends HelixProperty> List<T> getChildValues(PropertyKey propertyKey)
-    // public List<ZNRecord> getChildValues(PropertyType type, String...
-    // keys)
-    {
+    public <T extends HelixProperty> List<T> getChildValues(PropertyKey propertyKey) {
       List<ZNRecord> childs = new ArrayList<ZNRecord>();
       String path = propertyKey.getPath(); // PropertyPathConfig.getPath(type,
       // _clusterName, keys);
@@ -646,11 +557,7 @@ public class Mocks {
     }
 
     @Override
-    public <T extends HelixProperty> Map<String, T> getChildValuesMap(PropertyKey key)
-    // public <T extends HelixProperty> Map<String, T>
-    // getChildValuesMap(Class<T> clazz,
-    // PropertyType type, String... keys)
-    {
+    public <T extends HelixProperty> Map<String, T> getChildValuesMap(PropertyKey key) {
       List<T> list = getChildValues(key);
       return HelixProperty.convertListToMap(list);
     }
@@ -704,22 +611,6 @@ public class Mocks {
     }
   }
 
-  public static class MockHealthReportProvider extends HealthReportProvider {
-
-    @Override
-    public Map<String, String> getRecentHealthReport() {
-      // TODO Auto-generated method stub
-      return null;
-    }
-
-    @Override
-    public void resetStats() {
-      // TODO Auto-generated method stub
-
-    }
-
-  }
-
   public static class MockClusterMessagingService implements ClusterMessagingService {
 
     @Override

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/test/java/org/apache/helix/TestParticipantHealthReportCollectorImpl.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/TestParticipantHealthReportCollectorImpl.java b/helix-core/src/test/java/org/apache/helix/TestParticipantHealthReportCollectorImpl.java
deleted file mode 100644
index fe4c688..0000000
--- a/helix-core/src/test/java/org/apache/helix/TestParticipantHealthReportCollectorImpl.java
+++ /dev/null
@@ -1,78 +0,0 @@
-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 org.apache.helix.Mocks.MockHealthReportProvider;
-import org.apache.helix.Mocks.MockManager;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
-import org.apache.helix.healthcheck.ParticipantHealthReportTask;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
-
-public class TestParticipantHealthReportCollectorImpl {
-
-  protected ParticipantHealthReportCollectorImpl _providerImpl;
-  protected ParticipantHealthReportTask _providerTask;
-  protected HelixManager _manager;
-  protected MockHealthReportProvider _mockProvider;
-
-  @BeforeMethod(groups = {
-    "unitTest"
-  })
-  public void setup() {
-    _providerImpl = new ParticipantHealthReportCollectorImpl(new MockManager(), "instance_123");
-    _providerTask = new ParticipantHealthReportTask(_providerImpl);
-    _mockProvider = new MockHealthReportProvider();
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testStart() throws Exception {
-    _providerTask.start();
-    _providerTask.start();
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testStop() throws Exception {
-    _providerTask.stop();
-    _providerTask.stop();
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testAddProvider() throws Exception {
-    _providerImpl.removeHealthReportProvider(_mockProvider);
-    _providerImpl.addHealthReportProvider(_mockProvider);
-    _providerImpl.addHealthReportProvider(_mockProvider);
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testRemoveProvider() throws Exception {
-    _providerImpl.addHealthReportProvider(_mockProvider);
-    _providerImpl.removeHealthReportProvider(_mockProvider);
-    _providerImpl.removeHealthReportProvider(_mockProvider);
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/test/java/org/apache/helix/TestPerfCounters.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/TestPerfCounters.java b/helix-core/src/test/java/org/apache/helix/TestPerfCounters.java
deleted file mode 100644
index bac6e7a..0000000
--- a/helix-core/src/test/java/org/apache/helix/TestPerfCounters.java
+++ /dev/null
@@ -1,70 +0,0 @@
-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 org.apache.helix.healthcheck.DefaultPerfCounters;
-import org.testng.AssertJUnit;
-import org.testng.annotations.BeforeTest;
-import org.testng.annotations.Test;
-
-@SuppressWarnings("deprecation")
-public class TestPerfCounters {
-
-  final String INSTANCE_NAME = "instance_123";
-  final long AVAILABLE_CPUS = 1;
-  final long FREE_PHYSICAL_MEMORY = 2;
-  final long FREE_JVM_MEMORY = 3;
-  final long TOTAL_JVM_MEMORY = 4;
-  final double AVERAGE_SYSTEM_LOAD = 5;
-
-  DefaultPerfCounters _perfCounters;
-
-  @BeforeTest()
-  public void setup() {
-    _perfCounters =
-        new DefaultPerfCounters(INSTANCE_NAME, AVAILABLE_CPUS, FREE_PHYSICAL_MEMORY,
-            FREE_JVM_MEMORY, TOTAL_JVM_MEMORY, AVERAGE_SYSTEM_LOAD);
-  }
-
-  @Test()
-  public void testGetAvailableCpus() {
-    AssertJUnit.assertEquals(AVAILABLE_CPUS, _perfCounters.getAvailableCpus());
-  }
-
-  @Test()
-  public void testGetAverageSystemLoad() {
-    AssertJUnit.assertEquals(AVERAGE_SYSTEM_LOAD, _perfCounters.getAverageSystemLoad());
-  }
-
-  @Test()
-  public void testGetTotalJvmMemory() {
-    AssertJUnit.assertEquals(TOTAL_JVM_MEMORY, _perfCounters.getTotalJvmMemory());
-  }
-
-  @Test()
-  public void testGetFreeJvmMemory() {
-    AssertJUnit.assertEquals(FREE_JVM_MEMORY, _perfCounters.getFreeJvmMemory());
-  }
-
-  @Test()
-  public void testGetFreePhysicalMemory() {
-    AssertJUnit.assertEquals(FREE_PHYSICAL_MEMORY, _perfCounters.getFreePhysicalMemory());
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/test/java/org/apache/helix/TestPerformanceHealthReportProvider.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/TestPerformanceHealthReportProvider.java b/helix-core/src/test/java/org/apache/helix/TestPerformanceHealthReportProvider.java
deleted file mode 100644
index b49575f..0000000
--- a/helix-core/src/test/java/org/apache/helix/TestPerformanceHealthReportProvider.java
+++ /dev/null
@@ -1,158 +0,0 @@
-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.Map;
-
-import org.apache.helix.Mocks.MockManager;
-import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.healthcheck.PerformanceHealthReportProvider;
-import org.apache.helix.model.HealthStat;
-import org.testng.AssertJUnit;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
-
-public class TestPerformanceHealthReportProvider {
-
-  protected static final String CLUSTER_NAME = "TestCluster";
-  protected final String STAT_NAME = "Stat_123";
-  protected final String PARTITION_NAME = "Partition_456";
-  protected final String FAKE_STAT_NAME = "Stat_ABC";
-  protected final String FAKE_PARTITION_NAME = "Partition_DEF";
-  protected final String STORED_STAT = "789";
-  protected final String INSTANCE_NAME = "instance:1";
-
-  PerformanceHealthReportProvider _healthProvider;
-  MockManager _helixManager;
-
-  public void incrementPartitionStat() throws Exception {
-    _helixManager = new MockManager(CLUSTER_NAME);
-    _healthProvider.incrementPartitionStat(STAT_NAME, PARTITION_NAME);
-  }
-
-  public void transmitReport() throws Exception {
-    _helixManager = new MockManager(CLUSTER_NAME);
-    Map<String, Map<String, String>> partitionReport =
-        _healthProvider.getRecentPartitionHealthReport();
-    ZNRecord record = new ZNRecord(_healthProvider.getReportName());
-    if (partitionReport != null) {
-      record.setMapFields(partitionReport);
-    }
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-
-    Builder keyBuilder = accessor.keyBuilder();
-    accessor.setProperty(keyBuilder.healthReport(INSTANCE_NAME, record.getId()), new HealthStat(
-        record));
-  }
-
-  @BeforeMethod()
-  public void setup() {
-    _healthProvider = new PerformanceHealthReportProvider();
-  }
-
-  @Test()
-  public void testGetRecentHealthReports() throws Exception {
-    _healthProvider.getRecentHealthReport();
-    _healthProvider.getRecentPartitionHealthReport();
-  }
-
-  @Test()
-  public void testIncrementPartitionStat() throws Exception {
-    // stat does not exist yet
-    _healthProvider.incrementPartitionStat(STAT_NAME, PARTITION_NAME);
-    transmitReport();
-    // stat does exist
-    _healthProvider.incrementPartitionStat(STAT_NAME, PARTITION_NAME);
-    transmitReport();
-    String retrievedStat = _healthProvider.getPartitionStat(STAT_NAME, PARTITION_NAME);
-    AssertJUnit.assertEquals(2.0, Double.parseDouble(retrievedStat));
-
-    // set to some other value
-    _healthProvider.submitPartitionStat(STAT_NAME, PARTITION_NAME, STORED_STAT);
-    transmitReport();
-    _healthProvider.incrementPartitionStat(STAT_NAME, PARTITION_NAME);
-    transmitReport();
-    retrievedStat = _healthProvider.getPartitionStat(STAT_NAME, PARTITION_NAME);
-    AssertJUnit
-        .assertEquals(Double.parseDouble(retrievedStat), Double.parseDouble(STORED_STAT) + 1);
-  }
-
-  @Test()
-  public void testSetGetPartitionStat() throws Exception {
-    _healthProvider.submitPartitionStat(STAT_NAME, PARTITION_NAME, STORED_STAT);
-    transmitReport();
-    String retrievedStat = _healthProvider.getPartitionStat(STAT_NAME, PARTITION_NAME);
-    // check on correct retrieval for real stat, real partition
-    AssertJUnit.assertEquals(STORED_STAT, retrievedStat);
-
-    // real stat, fake partition
-    retrievedStat = _healthProvider.getPartitionStat(STAT_NAME, FAKE_PARTITION_NAME);
-    AssertJUnit.assertNull(retrievedStat);
-
-    // fake stat, real partition
-    retrievedStat = _healthProvider.getPartitionStat(FAKE_STAT_NAME, PARTITION_NAME);
-    AssertJUnit.assertNull(retrievedStat);
-
-    // fake stat, fake partition
-    retrievedStat = _healthProvider.getPartitionStat(FAKE_STAT_NAME, FAKE_PARTITION_NAME);
-    AssertJUnit.assertNull(retrievedStat);
-  }
-
-  @Test()
-  public void testGetPartitionHealthReport() throws Exception {
-    // test empty map case
-    Map<String, Map<String, String>> resultMap = _healthProvider.getRecentPartitionHealthReport();
-    AssertJUnit.assertEquals(resultMap.size(), 0);
-
-    // test non-empty case
-    testSetGetPartitionStat();
-    resultMap = _healthProvider.getRecentPartitionHealthReport();
-    // check contains 1 stat
-    AssertJUnit.assertEquals(1, resultMap.size());
-    // check contains STAT_NAME STAT
-    AssertJUnit.assertTrue(resultMap.keySet().contains(STAT_NAME));
-    Map<String, String> statMap = resultMap.get(STAT_NAME);
-    // check statMap has size 1
-    AssertJUnit.assertEquals(1, statMap.size());
-    // check contains PARTITION_NAME
-    AssertJUnit.assertTrue(statMap.keySet().contains(PARTITION_NAME));
-    // check stored val
-    String statVal = statMap.get(PARTITION_NAME);
-    AssertJUnit.assertEquals(statVal, STORED_STAT);
-  }
-
-  @Test()
-  public void testPartitionStatReset() throws Exception {
-    incrementPartitionStat();
-    // ensure stat appears
-    String retrievedStat = _healthProvider.getPartitionStat(STAT_NAME, PARTITION_NAME);
-    AssertJUnit.assertEquals(1.0, Double.parseDouble(retrievedStat));
-    // reset partition stats
-    _healthProvider.resetStats();
-    transmitReport();
-    retrievedStat = _healthProvider.getPartitionStat(STAT_NAME, PARTITION_NAME);
-    AssertJUnit.assertEquals(null, retrievedStat);
-  }
-
-  @Test()
-  public void testGetReportName() throws Exception {
-    _healthProvider.getReportName();
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/test/java/org/apache/helix/alerts/TestAddAlerts.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/alerts/TestAddAlerts.java b/helix-core/src/test/java/org/apache/helix/alerts/TestAddAlerts.java
deleted file mode 100644
index 723557c..0000000
--- a/helix-core/src/test/java/org/apache/helix/alerts/TestAddAlerts.java
+++ /dev/null
@@ -1,114 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.Map;
-
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.Mocks.MockManager;
-import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.controller.stages.HealthDataCache;
-import org.testng.AssertJUnit;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
-
-public class TestAddAlerts {
-
-  protected static final String CLUSTER_NAME = "TestCluster";
-
-  MockManager _helixManager;
-  AlertsHolder _alertsHolder;
-
-  public final String EXP = AlertParser.EXPRESSION_NAME;
-  public final String CMP = AlertParser.COMPARATOR_NAME;
-  public final String CON = AlertParser.CONSTANT_NAME;
-
-  @BeforeMethod()
-  public void setup() {
-    _helixManager = new MockManager(CLUSTER_NAME);
-    _alertsHolder = new AlertsHolder(_helixManager, new HealthDataCache());
-  }
-
-  public boolean alertRecordContains(ZNRecord rec, String alertName) {
-    Map<String, Map<String, String>> alerts = rec.getMapFields();
-    return alerts.containsKey(alertName);
-  }
-
-  public int alertsSize(ZNRecord rec) {
-    Map<String, Map<String, String>> alerts = rec.getMapFields();
-    return alerts.size();
-  }
-
-  @Test()
-  public void testAddAlert() throws Exception {
-    String alert =
-        EXP + "(accumulate()(dbFoo.partition10.latency))" + CMP + "(GREATER)" + CON + "(10)";
-    _alertsHolder.addAlert(alert);
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ZNRecord rec = accessor.getProperty(keyBuilder.alerts()).getRecord();
-    System.out.println("alert: " + alert);
-    System.out.println("rec: " + rec.toString());
-    AssertJUnit.assertTrue(alertRecordContains(rec, alert));
-    AssertJUnit.assertEquals(1, alertsSize(rec));
-  }
-
-  @Test()
-  public void testAddTwoAlerts() throws Exception {
-    String alert1 =
-        EXP + "(accumulate()(dbFoo.partition10.latency))" + CMP + "(GREATER)" + CON + "(10)";
-    String alert2 =
-        EXP + "(accumulate()(dbFoo.partition10.latency))" + CMP + "(GREATER)" + CON + "(100)";
-    _alertsHolder.addAlert(alert1);
-    _alertsHolder.addAlert(alert2);
-
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ZNRecord rec = accessor.getProperty(keyBuilder.alerts()).getRecord();
-    // System.out.println("alert: "+alert1);
-    System.out.println("rec: " + rec.toString());
-    AssertJUnit.assertTrue(alertRecordContains(rec, alert1));
-    AssertJUnit.assertTrue(alertRecordContains(rec, alert2));
-    AssertJUnit.assertEquals(2, alertsSize(rec));
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testAddTwoWildcardAlert() throws Exception {
-    String alert1 =
-        EXP + "(accumulate()(dbFoo.partition*.put*))" + CMP + "(GREATER)" + CON + "(10)";
-    _alertsHolder.addAlert(alert1);
-
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ZNRecord rec = accessor.getProperty(keyBuilder.alerts()).getRecord();
-    // System.out.println("alert: "+alert1);
-    System.out.println("rec: " + rec.toString());
-    AssertJUnit.assertTrue(alertRecordContains(rec, alert1));
-    AssertJUnit.assertEquals(1, alertsSize(rec));
-  }
-
-  // add 2 wildcard alert here
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/test/java/org/apache/helix/alerts/TestAddPersistentStats.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/alerts/TestAddPersistentStats.java b/helix-core/src/test/java/org/apache/helix/alerts/TestAddPersistentStats.java
deleted file mode 100644
index 0510c7d..0000000
--- a/helix-core/src/test/java/org/apache/helix/alerts/TestAddPersistentStats.java
+++ /dev/null
@@ -1,210 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.Map;
-
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixException;
-import org.apache.helix.Mocks.MockManager;
-import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.controller.stages.HealthDataCache;
-import org.testng.AssertJUnit;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
-
-public class TestAddPersistentStats {
-
-  protected static final String CLUSTER_NAME = "TestCluster";
-
-  MockManager _helixManager;
-  StatsHolder _statsHolder;
-
-  @BeforeMethod(groups = {
-    "unitTest"
-  })
-  public void setup() {
-    _helixManager = new MockManager(CLUSTER_NAME);
-    _statsHolder = new StatsHolder(_helixManager, new HealthDataCache());
-  }
-
-  public boolean statRecordContains(ZNRecord rec, String statName) {
-    Map<String, Map<String, String>> stats = rec.getMapFields();
-    return stats.containsKey(statName);
-  }
-
-  public int statsSize(ZNRecord rec) {
-    Map<String, Map<String, String>> stats = rec.getMapFields();
-    return stats.size();
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testAddStat() throws Exception {
-    String stat = "window(5)(dbFoo.partition10.latency)";
-    _statsHolder.addStat(stat);
-    _statsHolder.persistStats();
-
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord();
-    System.out.println("rec: " + rec.toString());
-    AssertJUnit.assertTrue(statRecordContains(rec, stat));
-    AssertJUnit.assertEquals(1, statsSize(rec));
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testAddTwoStats() throws Exception {
-    String stat1 = "window(5)(dbFoo.partition10.latency)";
-    _statsHolder.addStat(stat1);
-    _statsHolder.persistStats();
-    String stat2 = "window(5)(dbFoo.partition11.latency)";
-    _statsHolder.addStat(stat2);
-    _statsHolder.persistStats();
-
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord();
-
-    System.out.println("rec: " + rec.toString());
-    AssertJUnit.assertTrue(statRecordContains(rec, stat1));
-    AssertJUnit.assertTrue(statRecordContains(rec, stat2));
-    AssertJUnit.assertEquals(2, statsSize(rec));
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testAddDuplicateStat() throws Exception {
-    String stat = "window(5)(dbFoo.partition10.latency)";
-    _statsHolder.addStat(stat);
-    _statsHolder.addStat(stat);
-    _statsHolder.persistStats();
-
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord();
-
-    System.out.println("rec: " + rec.toString());
-    AssertJUnit.assertTrue(statRecordContains(rec, stat));
-    AssertJUnit.assertEquals(1, statsSize(rec));
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testAddPairOfStats() throws Exception {
-    String exp = "accumulate()(dbFoo.partition10.latency, dbFoo.partition10.count)";
-    _statsHolder.addStat(exp);
-    _statsHolder.persistStats();
-
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord();
-    System.out.println("rec: " + rec.toString());
-    AssertJUnit.assertTrue(statRecordContains(rec, "accumulate()(dbFoo.partition10.latency)"));
-    AssertJUnit.assertTrue(statRecordContains(rec, "accumulate()(dbFoo.partition10.count)"));
-    AssertJUnit.assertEquals(2, statsSize(rec));
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testAddStatsWithOperators() throws Exception {
-    String exp =
-        "accumulate()(dbFoo.partition10.latency, dbFoo.partition10.count)|EACH|ACCUMULATE|DIVIDE";
-    _statsHolder.addStat(exp);
-    _statsHolder.persistStats();
-
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord();
-
-    System.out.println("rec: " + rec.toString());
-    AssertJUnit.assertTrue(statRecordContains(rec, "accumulate()(dbFoo.partition10.latency)"));
-    AssertJUnit.assertTrue(statRecordContains(rec, "accumulate()(dbFoo.partition10.count)"));
-    AssertJUnit.assertEquals(2, statsSize(rec));
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testAddNonExistentAggregator() throws Exception {
-    String exp = "fakeagg()(dbFoo.partition10.latency)";
-    boolean caughtException = false;
-    try {
-      _statsHolder.addStat(exp);
-    } catch (HelixException e) {
-      caughtException = true;
-    }
-    AssertJUnit.assertTrue(caughtException);
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testGoodAggregatorBadArgs() throws Exception {
-    String exp = "accumulate(10)(dbFoo.partition10.latency)";
-    boolean caughtException = false;
-    try {
-      _statsHolder.addStat(exp);
-    } catch (HelixException e) {
-      caughtException = true;
-    }
-    AssertJUnit.assertTrue(caughtException);
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testAddBadNestingStat1() throws Exception {
-    String exp = "window((5)(dbFoo.partition10.latency)";
-    boolean caughtException = false;
-    try {
-      _statsHolder.addStat(exp);
-    } catch (HelixException e) {
-      caughtException = true;
-    }
-    AssertJUnit.assertTrue(caughtException);
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testAddBadNestingStat2() throws Exception {
-    String exp = "window(5)(dbFoo.partition10.latency))";
-    boolean caughtException = false;
-    try {
-      _statsHolder.addStat(exp);
-    } catch (HelixException e) {
-      caughtException = true;
-    }
-    AssertJUnit.assertTrue(caughtException);
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/test/java/org/apache/helix/alerts/TestAlertValidation.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/alerts/TestAlertValidation.java b/helix-core/src/test/java/org/apache/helix/alerts/TestAlertValidation.java
deleted file mode 100644
index 3da2466..0000000
--- a/helix-core/src/test/java/org/apache/helix/alerts/TestAlertValidation.java
+++ /dev/null
@@ -1,166 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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 org.apache.helix.HelixException;
-import org.testng.AssertJUnit;
-import org.testng.annotations.Test;
-
-@Test
-public class TestAlertValidation {
-
-  public final String EXP = AlertParser.EXPRESSION_NAME;
-  public final String CMP = AlertParser.COMPARATOR_NAME;
-  public final String CON = AlertParser.CONSTANT_NAME;
-
-  @Test
-  public void testSimple() {
-    String alertName =
-        EXP + "(accumulate()(dbFoo.partition10.latency)) " + CMP + "(GREATER) " + CON + "(10)";
-    boolean caughtException = false;
-    try {
-      AlertParser.validateAlert(alertName);
-    } catch (HelixException e) {
-      caughtException = true;
-      e.printStackTrace();
-    }
-    AssertJUnit.assertFalse(caughtException);
-  }
-
-  @Test
-  public void testSingleInSingleOut() {
-    String alertName =
-        EXP + "(accumulate()(dbFoo.partition10.latency)|EXPAND) " + CMP + "(GREATER) " + CON
-            + "(10)";
-    boolean caughtException = false;
-    try {
-      AlertParser.validateAlert(alertName);
-    } catch (HelixException e) {
-      caughtException = true;
-      e.printStackTrace();
-    }
-    AssertJUnit.assertFalse(caughtException);
-  }
-
-  @Test
-  public void testDoubleInDoubleOut() {
-    String alertName =
-        EXP + "(accumulate()(dbFoo.partition10.latency, dbFoo.partition11.latency)|EXPAND) " + CMP
-            + "(GREATER) " + CON + "(10)";
-    boolean caughtException = false;
-    try {
-      AlertParser.validateAlert(alertName);
-    } catch (HelixException e) {
-      caughtException = true;
-      e.printStackTrace();
-    }
-    AssertJUnit.assertTrue(caughtException);
-  }
-
-  @Test
-  public void testTwoStageOps() {
-    String alertName =
-        EXP + "(accumulate()(dbFoo.partition*.latency, dbFoo.partition*.count)|EXPAND|DIVIDE) "
-            + CMP + "(GREATER) " + CON + "(10)";
-    boolean caughtException = false;
-    try {
-      AlertParser.validateAlert(alertName);
-    } catch (HelixException e) {
-      caughtException = true;
-      e.printStackTrace();
-    }
-    AssertJUnit.assertFalse(caughtException);
-  }
-
-  @Test
-  public void testTwoListsIntoOne() {
-    String alertName =
-        EXP + "(accumulate()(dbFoo.partition10.latency, dbFoo.partition11.count)|SUM) " + CMP
-            + "(GREATER) " + CON + "(10)";
-    boolean caughtException = false;
-    try {
-      AlertParser.validateAlert(alertName);
-    } catch (HelixException e) {
-      caughtException = true;
-      e.printStackTrace();
-    }
-    AssertJUnit.assertFalse(caughtException);
-  }
-
-  @Test
-  public void testSumEach() {
-    String alertName =
-        EXP
-            + "(accumulate()(dbFoo.partition*.latency, dbFoo.partition*.count)|EXPAND|SUMEACH|DIVIDE) "
-            + CMP + "(GREATER) " + CON + "(10)";
-    boolean caughtException = false;
-    try {
-      AlertParser.validateAlert(alertName);
-    } catch (HelixException e) {
-      caughtException = true;
-      e.printStackTrace();
-    }
-    AssertJUnit.assertFalse(caughtException);
-  }
-
-  @Test
-  public void testNeedTwoTuplesGetOne() {
-    String alertName =
-        EXP + "(accumulate()(dbFoo.partition*.latency)|EXPAND|DIVIDE) " + CMP + "(GREATER) " + CON
-            + "(10)";
-    boolean caughtException = false;
-    try {
-      AlertParser.validateAlert(alertName);
-    } catch (HelixException e) {
-      caughtException = true;
-      e.printStackTrace();
-    }
-    AssertJUnit.assertTrue(caughtException);
-  }
-
-  @Test
-  public void testExtraPipe() {
-    String alertName =
-        EXP + "(accumulate()(dbFoo.partition10.latency)|) " + CMP + "(GREATER) " + CON + "(10)";
-    boolean caughtException = false;
-    try {
-      AlertParser.validateAlert(alertName);
-    } catch (HelixException e) {
-      caughtException = true;
-      e.printStackTrace();
-    }
-    AssertJUnit.assertTrue(caughtException);
-  }
-
-  @Test
-  public void testAlertUnknownOp() {
-    String alertName =
-        EXP + "(accumulate()(dbFoo.partition10.latency)|BADOP) " + CMP + "(GREATER) " + CON
-            + "(10)";
-    boolean caughtException = false;
-    try {
-      AlertParser.validateAlert(alertName);
-    } catch (HelixException e) {
-      caughtException = true;
-      e.printStackTrace();
-    }
-    AssertJUnit.assertTrue(caughtException);
-  }
-}


[11/17] git commit: [HELIX-453] On session expiry/recovery, not all message types are re-registered, rb=22432

Posted by ka...@apache.org.
[HELIX-453] On session expiry/recovery, not all message types are re-registered, rb=22432


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

Branch: refs/heads/master
Commit: 0ee1cdc42d97068924c8d978e68ae78f606d44a5
Parents: 33c2f15
Author: zzhang <zz...@apache.org>
Authored: Tue Jun 10 13:03:28 2014 -0700
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Fri Jul 11 12:48:00 2014 -0700

----------------------------------------------------------------------
 .../messaging/handling/HelixTaskExecutor.java   | 121 +++++++++---
 .../helix/integration/TestZkSessionExpiry.java  | 188 +++++++++++++++++++
 recipes/helloworld-provisioning-yarn/pom.xml    |   5 +
 recipes/jobrunner-yarn/pom.xml                  |   5 +
 4 files changed, 295 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/0ee1cdc4/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
index e2af382..b507755 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
@@ -62,6 +62,37 @@ import org.apache.helix.util.StatusUpdateUtil;
 import org.apache.log4j.Logger;
 
 public class HelixTaskExecutor implements MessageListener, TaskExecutor {
+  /**
+   * Put together all registration information about a message handler factory
+   */
+  class MsgHandlerFactoryRegistryItem {
+    private final MessageHandlerFactory _factory;
+    private final int _threadPoolSize;
+
+    public MsgHandlerFactoryRegistryItem(MessageHandlerFactory factory, int threadPoolSize) {
+      if (factory == null) {
+        throw new NullPointerException("Message handler factory is null");
+      }
+
+      if (threadPoolSize <= 0) {
+        throw new IllegalArgumentException("Illegal thread pool size: " + threadPoolSize);
+      }
+
+      _factory = factory;
+      _threadPoolSize = threadPoolSize;
+    }
+
+    int threadPoolSize() {
+      return _threadPoolSize;
+    }
+
+    MessageHandlerFactory factory() {
+      return _factory;
+    }
+  }
+
+  private static Logger LOG = Logger.getLogger(HelixTaskExecutor.class);
+
   // TODO: we need to further design how to throttle this.
   // From storage point of view, only bootstrap case is expensive
   // and we need to throttle, which is mostly IO / network bounded.
@@ -73,21 +104,24 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
   private final ParticipantMonitor _monitor;
   public static final String MAX_THREADS = "maxThreads";
 
-  final ConcurrentHashMap<String, MessageHandlerFactory> _handlerFactoryMap =
-      new ConcurrentHashMap<String, MessageHandlerFactory>();
+  /**
+   * Map of MsgType->MsgHandlerFactoryRegistryItem
+   */
+  final ConcurrentHashMap<String, MsgHandlerFactoryRegistryItem> _hdlrFtyRegistry;
 
   final ConcurrentHashMap<String, ExecutorService> _executorMap;
 
-  private static Logger LOG = Logger.getLogger(HelixTaskExecutor.class);
-
-  Map<String, Integer> _resourceThreadpoolSizeMap = new ConcurrentHashMap<String, Integer>();
+  final Map<String, Integer> _resourceThreadpoolSizeMap;
 
   // timer for schedule timeout tasks
   final Timer _timer;
 
   public HelixTaskExecutor() {
     _taskMap = new ConcurrentHashMap<String, MessageTaskInfo>();
+
+    _hdlrFtyRegistry = new ConcurrentHashMap<String, MsgHandlerFactoryRegistryItem>();
     _executorMap = new ConcurrentHashMap<String, ExecutorService>();
+    _resourceThreadpoolSizeMap = new ConcurrentHashMap<String, Integer>();
 
     _lock = new Object();
     _statusUpdateUtil = new StatusUpdateUtil();
@@ -111,18 +145,24 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
           + factory.getMessageType());
     }
 
-    MessageHandlerFactory prevFactory = _handlerFactoryMap.putIfAbsent(type, factory);
-    if (prevFactory == null) {
-      if (!_executorMap.contains(type)) {
-        _executorMap.put(type, Executors.newFixedThreadPool(threadpoolSize));
-      } else {
-        LOG.error("Skip to create new thread pool for type: " + type);
+    MsgHandlerFactoryRegistryItem newItem =
+        new MsgHandlerFactoryRegistryItem(factory, threadpoolSize);
+    MsgHandlerFactoryRegistryItem prevItem = _hdlrFtyRegistry.putIfAbsent(type, newItem);
+    if (prevItem == null) {
+      ExecutorService newPool = Executors.newFixedThreadPool(threadpoolSize);
+      ExecutorService prevExecutor = _executorMap.putIfAbsent(type, newPool);
+      if (prevExecutor != null) {
+        LOG.warn("Skip creating a new thread pool for type: " + type + ", already existing pool: "
+            + prevExecutor + ", isShutdown: " + prevExecutor.isShutdown());
+        newPool = null;
       }
       LOG.info("Registered message handler factory for type: " + type + ", poolSize: "
           + threadpoolSize + ", factory: " + factory + ", pool: " + _executorMap.get(type));
     } else {
-      LOG.warn("Fail to register message handler factory for type: " + type + ", poolSize: "
-          + threadpoolSize + ", factory: " + factory);
+      LOG.info("Skip register message handler factory for type: " + type + ", poolSize: "
+          + threadpoolSize + ", factory: " + factory + ", already existing factory: "
+          + prevItem.factory());
+      newItem = null;
     }
   }
 
@@ -165,7 +205,7 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
   /**
    * Find the executor service for the message. A message can have a per-statemodelfactory
    * executor service, or per-message type executor service.
-   **/
+   */
   ExecutorService findExecutorServiceForMsg(Message message) {
     ExecutorService executorService = _executorMap.get(message.getMsgType());
     if (message.getMsgType().equals(MessageType.STATE_TRANSITION.toString())) {
@@ -380,29 +420,39 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
    * @param type
    */
   void unregisterMessageHandlerFactory(String type) {
+    MsgHandlerFactoryRegistryItem item = _hdlrFtyRegistry.remove(type);
     ExecutorService pool = _executorMap.remove(type);
-    MessageHandlerFactory handlerFty = _handlerFactoryMap.remove(type);
 
-    LOG.info("Unregistering message handler factory for type: " + type + ", factory: " + handlerFty
-        + ", pool: " + pool);
+    LOG.info("Unregistering message handler factory for type: " + type + ", factory: "
+        + item.factory() + ", pool: " + pool);
 
     if (pool != null) {
       shutdownAndAwaitTermination(pool);
     }
 
     // reset state-model
-    if (handlerFty != null) {
-      handlerFty.reset();
+    if (item != null) {
+      item.factory().reset();
     }
 
-    LOG.info("Unregistered message handler factory for type: " + type + ", factory: " + handlerFty
-        + ", pool: " + pool);
+    LOG.info("Unregistered message handler factory for type: " + type + ", factory: "
+        + item.factory() + ", pool: " + pool);
   }
 
   void reset() {
     LOG.info("Reset HelixTaskExecutor");
-    for (String msgType : _executorMap.keySet()) {
-      unregisterMessageHandlerFactory(msgType);
+    for (String msgType : _hdlrFtyRegistry.keySet()) {
+      // don't un-register factories, just shutdown all executors
+      ExecutorService pool = _executorMap.remove(msgType);
+      if (pool != null) {
+        LOG.info("Reset exectuor for msgType: " + msgType + ", pool: " + pool);
+        shutdownAndAwaitTermination(pool);
+      }
+
+      MsgHandlerFactoryRegistryItem item = _hdlrFtyRegistry.get(msgType);
+      if (item.factory() != null) {
+        item.factory().reset();
+      }
     }
 
     // Log all tasks that fail to terminate
@@ -411,12 +461,29 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
       LOG.warn("Task: " + taskId + " fails to terminate. Message: " + info._task.getMessage());
     }
     _taskMap.clear();
+  }
 
+  void init() {
+    LOG.info("Init HelixTaskExecutor");
+
+    // Re-init all existing factories
+    for (String msgType : _hdlrFtyRegistry.keySet()) {
+      MsgHandlerFactoryRegistryItem item = _hdlrFtyRegistry.get(msgType);
+      ExecutorService newPool = Executors.newFixedThreadPool(item.threadPoolSize());
+      ExecutorService prevPool = _executorMap.putIfAbsent(msgType, newPool);
+      if (prevPool != null) {
+        // Will happen if we register and call init
+        LOG.info("Skip init a new thread pool for type: " + msgType + ", already existing pool: "
+            + prevPool + ", isShutdown: " + prevPool.isShutdown());
+        newPool = null;
+      }
+    }
   }
 
   @Override
   public void onMessage(String instanceName, List<Message> messages,
       NotificationContext changeContext) {
+
     // If FINALIZE notification comes, reset all handler factories
     // and terminate all the thread pools
     // TODO: see if we should have a separate notification call for resetting
@@ -425,6 +492,11 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
       return;
     }
 
+    if (changeContext.getType() == Type.INIT) {
+      init();
+      // continue to process messages
+    }
+
     if (messages == null || messages.size() == 0) {
       LOG.info("No Messages to process");
       return;
@@ -568,7 +640,8 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
   public MessageHandler createMessageHandler(Message message, NotificationContext changeContext) {
     String msgType = message.getMsgType().toString();
 
-    MessageHandlerFactory handlerFactory = _handlerFactoryMap.get(msgType);
+    MsgHandlerFactoryRegistryItem item = _hdlrFtyRegistry.get(msgType);
+    MessageHandlerFactory handlerFactory = item.factory();
 
     // Fail to find a MessageHandlerFactory for the message
     // we will keep the message and the message will be handled when

http://git-wip-us.apache.org/repos/asf/helix/blob/0ee1cdc4/helix-core/src/test/java/org/apache/helix/integration/TestZkSessionExpiry.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestZkSessionExpiry.java b/helix-core/src/test/java/org/apache/helix/integration/TestZkSessionExpiry.java
new file mode 100644
index 0000000..c911283
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestZkSessionExpiry.java
@@ -0,0 +1,188 @@
+package org.apache.helix.integration;
+
+/*
+ * 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.HashSet;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixManager;
+import org.apache.helix.NotificationContext;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.TestHelper;
+import org.apache.helix.ZkTestHelper;
+import org.apache.helix.ZkUnitTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.messaging.handling.HelixTaskResult;
+import org.apache.helix.messaging.handling.MessageHandler;
+import org.apache.helix.messaging.handling.MessageHandlerFactory;
+import org.apache.helix.model.Message;
+import org.apache.helix.tools.ClusterStateVerifier;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class TestZkSessionExpiry extends ZkUnitTestBase {
+  final static String DUMMY_MSG_TYPE = "DUMMY";
+
+  static class DummyMessageHandler extends MessageHandler {
+    final Set<String> _handledMsgSet;
+
+    public DummyMessageHandler(Message message, NotificationContext context,
+        Set<String> handledMsgSet) {
+      super(message, context);
+      _handledMsgSet = handledMsgSet;
+    }
+
+    @Override
+    public HelixTaskResult handleMessage() throws InterruptedException {
+      _handledMsgSet.add(_message.getId());
+      HelixTaskResult ret = new HelixTaskResult();
+      ret.setSuccess(true);
+      return ret;
+    }
+
+    @Override
+    public void onError(Exception e, ErrorCode code, ErrorType type) {
+      // Do nothing
+    }
+
+  }
+
+  static class DummyMessageHandlerFactory implements MessageHandlerFactory {
+    final Set<String> _handledMsgSet;
+
+    public DummyMessageHandlerFactory(Set<String> handledMsgSet) {
+      _handledMsgSet = handledMsgSet;
+    }
+
+    @Override
+    public MessageHandler createHandler(Message message, NotificationContext context) {
+      return new DummyMessageHandler(message, context, _handledMsgSet);
+    }
+
+    @Override
+    public String getMessageType() {
+      return DUMMY_MSG_TYPE;
+    }
+
+    @Override
+    public void reset() {
+      // Do nothing
+    }
+
+  }
+
+  @Test
+  public void testMsgHdlrFtyReRegistration() throws Exception {
+    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
+        8, // partitions per resource
+        n, // number of nodes
+        2, // replicas
+        "MasterSlave", true); // do rebalance
+
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+    controller.syncStart();
+
+    // start participants
+    Set<String> handledMsgSet = new HashSet<String>();
+    MockParticipantManager[] participants = new MockParticipantManager[n];
+    for (int i = 0; i < n; i++) {
+      String instanceName = "localhost_" + (12918 + i);
+
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+      participants[i].getMessagingService().registerMessageHandlerFactory(DUMMY_MSG_TYPE,
+          new DummyMessageHandlerFactory(handledMsgSet));
+      participants[i].syncStart();
+    }
+
+    boolean result =
+        ClusterStateVerifier
+            .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR,
+                clusterName));
+    Assert.assertTrue(result);
+
+    // trigger dummy message handler
+    checkDummyMsgHandler(participants[0], handledMsgSet);
+
+    // expire localhost_12918
+    ZkTestHelper.expireSession(participants[0].getZkClient());
+    result =
+        ClusterStateVerifier
+            .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR,
+                clusterName));
+    Assert.assertTrue(result);
+
+    // trigger dummy message handler again
+    checkDummyMsgHandler(participants[0], handledMsgSet);
+
+    // clean up
+    controller.syncStop();
+    for (int i = 0; i < n; i++) {
+      participants[i].syncStop();
+    }
+
+    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
+  }
+
+  /**
+   * trigger dummy message handler and verify it's invoked
+   * @param manager
+   * @param handledMsgMap
+   * @throws Exception
+   */
+  private static void checkDummyMsgHandler(HelixManager manager,
+      final Set<String> handledMsgSet) throws Exception {
+
+    final Message aMsg = newMsg();
+    HelixDataAccessor accessor = manager.getHelixDataAccessor();
+    PropertyKey.Builder keyBuilder = accessor.keyBuilder();
+    accessor.setProperty(keyBuilder.message(manager.getInstanceName(), aMsg.getId()), aMsg);
+    boolean result = TestHelper.verify(new TestHelper.Verifier() {
+
+      @Override
+      public boolean verify() throws Exception {
+
+        return handledMsgSet.contains(aMsg.getId());
+      }
+    }, 5 * 1000);
+    Assert.assertTrue(result);
+  }
+
+  private static Message newMsg() {
+    Message msg = new Message(DUMMY_MSG_TYPE, UUID.randomUUID().toString());
+    msg.setTgtSessionId("*");
+    msg.setTgtName("localhost_12918");
+    return msg;
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/0ee1cdc4/recipes/helloworld-provisioning-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/recipes/helloworld-provisioning-yarn/pom.xml b/recipes/helloworld-provisioning-yarn/pom.xml
index bf5a89c..fe41e9c 100644
--- a/recipes/helloworld-provisioning-yarn/pom.xml
+++ b/recipes/helloworld-provisioning-yarn/pom.xml
@@ -54,6 +54,11 @@ under the License.
       <artifactId>helix-provisioning</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.yaml</groupId>
+      <artifactId>snakeyaml</artifactId>
+      <version>1.12</version>
+    </dependency>
+    <dependency>
       <groupId>log4j</groupId>
       <artifactId>log4j</artifactId>
       <exclusions>

http://git-wip-us.apache.org/repos/asf/helix/blob/0ee1cdc4/recipes/jobrunner-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/recipes/jobrunner-yarn/pom.xml b/recipes/jobrunner-yarn/pom.xml
index 434fd8d..c4d560f 100644
--- a/recipes/jobrunner-yarn/pom.xml
+++ b/recipes/jobrunner-yarn/pom.xml
@@ -54,6 +54,11 @@ under the License.
       <artifactId>helix-provisioning</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.yaml</groupId>
+      <artifactId>snakeyaml</artifactId>
+      <version>1.12</version>
+    </dependency>
+    <dependency>
       <groupId>log4j</groupId>
       <artifactId>log4j</artifactId>
       <exclusions>


[05/17] [HELIX-395] Remove old Helix alert/stat modules

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/test/java/org/apache/helix/alerts/TestArrivingParticipantStats.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/alerts/TestArrivingParticipantStats.java b/helix-core/src/test/java/org/apache/helix/alerts/TestArrivingParticipantStats.java
deleted file mode 100644
index 454d410..0000000
--- a/helix-core/src/test/java/org/apache/helix/alerts/TestArrivingParticipantStats.java
+++ /dev/null
@@ -1,496 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.HashMap;
-import java.util.Map;
-
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.Mocks.MockManager;
-import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.controller.stages.HealthDataCache;
-import org.testng.AssertJUnit;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
-
-public class TestArrivingParticipantStats {
-  protected static final String CLUSTER_NAME = "TestCluster";
-
-  MockManager _helixManager;
-  StatsHolder _statsHolder;
-
-  @BeforeMethod(groups = {
-    "unitTest"
-  })
-  public void setup() {
-    _helixManager = new MockManager(CLUSTER_NAME);
-    _statsHolder = new StatsHolder(_helixManager, new HealthDataCache());
-  }
-
-  public Map<String, String> getStatFields(String value, String timestamp) {
-    Map<String, String> statMap = new HashMap<String, String>();
-    statMap.put(StatsHolder.VALUE_NAME, value);
-    statMap.put(StatsHolder.TIMESTAMP_NAME, timestamp);
-    return statMap;
-  }
-
-  public boolean statRecordContains(ZNRecord rec, String statName) {
-    Map<String, Map<String, String>> stats = rec.getMapFields();
-    return stats.containsKey(statName);
-  }
-
-  public boolean statRecordHasValue(ZNRecord rec, String statName, String value) {
-    Map<String, Map<String, String>> stats = rec.getMapFields();
-    Map<String, String> statFields = stats.get(statName);
-    return (statFields.get(StatsHolder.VALUE_NAME).equals(value));
-  }
-
-  public boolean statRecordHasTimestamp(ZNRecord rec, String statName, String timestamp) {
-    Map<String, Map<String, String>> stats = rec.getMapFields();
-    Map<String, String> statFields = stats.get(statName);
-    return (statFields.get(StatsHolder.TIMESTAMP_NAME).equals(timestamp));
-  }
-
-  // Exact matching persistent stat, but has no values yet
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testAddFirstParticipantStat() throws Exception {
-    // add a persistent stat
-    String persistentStat = "accumulate()(dbFoo.partition10.latency)";
-    _statsHolder.addStat(persistentStat);
-
-    // generate incoming stat
-    String incomingStatName = "dbFoo.partition10.latency";
-    Map<String, String> statFields = getStatFields("0", "0");
-    _statsHolder.applyStat(incomingStatName, statFields);
-    _statsHolder.persistStats();
-
-    // check persistent stats
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord();
-
-    System.out.println("rec: " + rec.toString());
-    AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "0.0"));
-    AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "0.0"));
-  }
-
-  // Exact matching persistent stat, but has no values yet
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testAddRepeatParticipantStat() throws Exception {
-    // add a persistent stat
-    String persistentStat = "accumulate()(dbFoo.partition10.latency)";
-    _statsHolder.addStat(persistentStat);
-
-    // generate incoming stat
-    String incomingStatName = "dbFoo.partition10.latency";
-    // apply stat once and then again
-    Map<String, String> statFields = getStatFields("0", "0");
-    _statsHolder.applyStat(incomingStatName, statFields);
-    statFields = getStatFields("1", "10");
-    _statsHolder.applyStat(incomingStatName, statFields);
-    _statsHolder.persistStats();
-
-    // check persistent stats
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord();
-
-    System.out.println("rec: " + rec.toString());
-    AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "1.0"));
-    AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "10.0"));
-  }
-
-  // test to ensure backdated stats not applied
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testBackdatedParticipantStat() throws Exception {
-    // add a persistent stat
-    String persistentStat = "accumulate()(dbFoo.partition10.latency)";
-    _statsHolder.addStat(persistentStat);
-
-    // generate incoming stat
-    String incomingStatName = "dbFoo.partition10.latency";
-    // apply stat once and then again
-    Map<String, String> statFields = getStatFields("0", "0");
-    _statsHolder.applyStat(incomingStatName, statFields);
-    statFields = getStatFields("1", "10");
-    _statsHolder.applyStat(incomingStatName, statFields);
-    statFields = getStatFields("5", "15");
-    _statsHolder.applyStat(incomingStatName, statFields);
-    statFields = getStatFields("1", "10");
-    _statsHolder.applyStat(incomingStatName, statFields);
-    _statsHolder.persistStats();
-
-    // check persistent stats
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord();
-
-    System.out.println("rec: " + rec.toString());
-    AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "6.0"));
-    AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "15.0"));
-  }
-
-  // Exact matching persistent stat, but has no values yet
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testAddFirstParticipantStatToWildCard() throws Exception {
-    // add a persistent stat
-    String persistentWildcardStat = "accumulate()(dbFoo.partition*.latency)";
-    _statsHolder.addStat(persistentWildcardStat);
-
-    // generate incoming stat
-    String incomingStatName = "dbFoo.partition10.latency";
-    Map<String, String> statFields = getStatFields("0", "0");
-    _statsHolder.applyStat(incomingStatName, statFields);
-    _statsHolder.persistStats();
-
-    // check persistent stats
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord();
-
-    System.out.println("rec: " + rec.toString());
-    String persistentStat = "accumulate()(dbFoo.partition10.latency)";
-    AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "0.0"));
-    AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "0.0"));
-  }
-
-  // test to add 2nd report to same stat
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testAddSecondParticipantStatToWildCard() throws Exception {
-    // add a persistent stat
-    String persistentWildcardStat = "accumulate()(dbFoo.partition*.latency)";
-    _statsHolder.addStat(persistentWildcardStat);
-
-    // generate incoming stat
-    String incomingStatName = "dbFoo.partition10.latency";
-    Map<String, String> statFields = getStatFields("1", "0");
-    _statsHolder.applyStat(incomingStatName, statFields);
-    statFields = getStatFields("1", "10");
-    _statsHolder.applyStat(incomingStatName, statFields);
-    _statsHolder.persistStats();
-
-    // check persistent stats
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord();
-
-    System.out.println("rec: " + rec.toString());
-    String persistentStat = "accumulate()(dbFoo.partition10.latency)";
-    AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "2.0"));
-    AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "10.0"));
-  }
-
-  // Exact matching persistent stat, but has no values yet
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testAddParticipantStatToDoubleWildCard() throws Exception {
-    // add a persistent stat
-    String persistentWildcardStat = "accumulate()(db*.partition*.latency)";
-    _statsHolder.addStat(persistentWildcardStat);
-
-    // generate incoming stat
-    String incomingStatName = "dbFoo.partition10.latency";
-    Map<String, String> statFields = getStatFields("0", "0");
-    _statsHolder.applyStat(incomingStatName, statFields);
-    _statsHolder.persistStats();
-
-    // check persistent stats
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord();
-
-    System.out.println("rec: " + rec.toString());
-    String persistentStat = "accumulate()(dbFoo.partition10.latency)";
-    AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "0.0"));
-    AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "0.0"));
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testAddWildcardInFirstStatToken() throws Exception {
-    String persistentWildcardStat = "accumulate()(instance*.reportingage)";
-    _statsHolder.addStat(persistentWildcardStat);
-
-    // generate incoming stat
-    String incomingStatName = "instance10.reportingage";
-    Map<String, String> statFields = getStatFields("1", "10");
-    _statsHolder.applyStat(incomingStatName, statFields);
-    _statsHolder.persistStats();
-
-    // check persistent stats
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord();
-
-    System.out.println("rec: " + rec.toString());
-    String persistentStat = "accumulate()(instance10.reportingage)";
-    AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "1.0"));
-    AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "10.0"));
-
-  }
-
-  // test to add report to same wildcard stat, different actual stat
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testAddTwoDistinctParticipantStatsToSameWildCard() throws Exception {
-    // add a persistent stat
-    String persistentWildcardStat = "accumulate()(dbFoo.partition*.latency)";
-    _statsHolder.addStat(persistentWildcardStat);
-
-    // generate incoming stat
-    String incomingStatName = "dbFoo.partition10.latency";
-    Map<String, String> statFields = getStatFields("1", "10");
-    _statsHolder.applyStat(incomingStatName, statFields);
-    incomingStatName = "dbFoo.partition11.latency";
-    statFields = getStatFields("5", "10");
-    _statsHolder.applyStat(incomingStatName, statFields);
-    _statsHolder.persistStats();
-
-    // check persistent stats
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord();
-
-    System.out.println("rec: " + rec.toString());
-    String persistentStat = "accumulate()(dbFoo.partition10.latency)";
-    AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "1.0"));
-    AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "10.0"));
-    persistentStat = "accumulate()(dbFoo.partition11.latency)";
-    AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "5.0"));
-    AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "10.0"));
-  }
-
-  // Exact matching persistent stat, but has no values yet
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testWindowStat() throws Exception {
-    // add a persistent stat
-    String persistentWildcardStat = "window(3)(dbFoo.partition*.latency)";
-    _statsHolder.addStat(persistentWildcardStat);
-
-    // generate incoming stat
-    String incomingStatName = "dbFoo.partition10.latency";
-    Map<String, String> statFields = getStatFields("0", "0");
-    _statsHolder.applyStat(incomingStatName, statFields);
-    _statsHolder.persistStats();
-
-    // check persistent stats
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord();
-
-    System.out.println("rec: " + rec.toString());
-    String persistentStat = "window(3)(dbFoo.partition10.latency)";
-    AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "0.0"));
-    AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "0.0"));
-
-    // add 2nd stat
-    statFields = getStatFields("10", "1");
-    _statsHolder.applyStat(incomingStatName, statFields);
-    _statsHolder.persistStats();
-
-    rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord();
-
-    System.out.println("rec: " + rec.toString());
-    AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "0.0,10.0"));
-    AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "0.0,1.0"));
-
-    // add 3rd stat
-    statFields = getStatFields("20", "2");
-    _statsHolder.applyStat(incomingStatName, statFields);
-    _statsHolder.persistStats();
-
-    rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord();
-
-    System.out.println("rec: " + rec.toString());
-    AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "0.0,10.0,20.0"));
-    AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "0.0,1.0,2.0"));
-
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testWindowStatExpiration() throws Exception {
-    String persistentWildcardStat = "window(3)(dbFoo.partition*.latency)";
-    String persistentStat = "window(3)(dbFoo.partition10.latency)";
-    // init with 3 elements
-    testWindowStat();
-
-    String incomingStatName = "dbFoo.partition10.latency";
-    Map<String, String> statFields = getStatFields("30", "3");
-    _statsHolder.applyStat(incomingStatName, statFields);
-    _statsHolder.persistStats();
-
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord();
-
-    System.out.println("rec: " + rec.toString());
-    AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "10.0,20.0,30.0"));
-    AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "1.0,2.0,3.0"));
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testWindowStatStale() throws Exception {
-    String persistentWildcardStat = "window(3)(dbFoo.partition*.latency)";
-    String persistentStat = "window(3)(dbFoo.partition10.latency)";
-    // init with 3 elements
-    testWindowStat();
-
-    String incomingStatName = "dbFoo.partition10.latency";
-    Map<String, String> statFields = getStatFields("10", "1");
-    _statsHolder.applyStat(incomingStatName, statFields);
-    _statsHolder.persistStats();
-
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord();
-
-    System.out.println("rec: " + rec.toString());
-    AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "0.0,10.0,20.0"));
-    AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "0.0,1.0,2.0"));
-  }
-
-  // test that has 2 agg stats for same raw stat
-  // Exact matching persistent stat, but has no values yet
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testAddStatForTwoAggTypes() throws Exception {
-    // add a persistent stat
-    String persistentStatOne = "accumulate()(dbFoo.partition10.latency)";
-    String persistentStatTwo = "window(3)(dbFoo.partition10.latency)";
-    _statsHolder.addStat(persistentStatOne);
-    _statsHolder.persistStats();
-    _statsHolder.addStat(persistentStatTwo);
-    _statsHolder.persistStats();
-
-    // generate incoming stat
-    String incomingStatName = "dbFoo.partition10.latency";
-    Map<String, String> statFields = getStatFields("0", "0");
-    _statsHolder.applyStat(incomingStatName, statFields);
-    _statsHolder.persistStats();
-
-    // check persistent stats
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord();
-
-    System.out.println("rec: " + rec.toString());
-    AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStatOne, "0.0"));
-    AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStatOne, "0.0"));
-    AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStatTwo, "0.0"));
-    AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStatTwo, "0.0"));
-  }
-
-  // test merging 2 window stats, new is applied
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testMergeTwoWindowsYesMerge() throws Exception {
-    String persistentWildcardStat = "window(3)(dbFoo.partition*.latency)";
-    String persistentStat = "window(3)(dbFoo.partition10.latency)";
-    String incomingStatName = "dbFoo.partition10.latency";
-    // init with 3 elements
-    testWindowStat();
-
-    // create a two tuples, value and time
-    Tuple<String> valTuple = new Tuple<String>();
-    Tuple<String> timeTuple = new Tuple<String>();
-    valTuple.add("30.0");
-    valTuple.add("40.0");
-    timeTuple.add("3.0");
-    timeTuple.add("4.0");
-    Map<String, String> statFields = getStatFields(valTuple.toString(), timeTuple.toString());
-    _statsHolder.applyStat(incomingStatName, statFields);
-    _statsHolder.persistStats();
-
-    // check persistent stats
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord();
-    System.out.println("rec: " + rec.toString());
-    AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "20.0,30.0,40.0"));
-    AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "2.0,3.0,4.0"));
-  }
-
-  // test merging 2 window stats, new is ignored
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testMergeTwoWindowsNoMerge() throws Exception {
-    String persistentWildcardStat = "window(3)(dbFoo.partition*.latency)";
-    String persistentStat = "window(3)(dbFoo.partition10.latency)";
-    String incomingStatName = "dbFoo.partition10.latency";
-    // init with 3 elements
-    testWindowStat();
-
-    // create a two tuples, value and time
-    Tuple<String> valTuple = new Tuple<String>();
-    Tuple<String> timeTuple = new Tuple<String>();
-    valTuple.add("0.0");
-    valTuple.add("40.0");
-    timeTuple.add("0.0");
-    timeTuple.add("4.0");
-    Map<String, String> statFields = getStatFields(valTuple.toString(), timeTuple.toString());
-    _statsHolder.applyStat(incomingStatName, statFields);
-    _statsHolder.persistStats();
-
-    // check persistent stats
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ZNRecord rec = accessor.getProperty(keyBuilder.persistantStat()).getRecord();
-    System.out.println("rec: " + rec.toString());
-    AssertJUnit.assertTrue(statRecordHasValue(rec, persistentStat, "0.0,10.0,20.0"));
-    AssertJUnit.assertTrue(statRecordHasTimestamp(rec, persistentStat, "0.0,1.0,2.0"));
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/test/java/org/apache/helix/alerts/TestBaseStatsValidation.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/alerts/TestBaseStatsValidation.java b/helix-core/src/test/java/org/apache/helix/alerts/TestBaseStatsValidation.java
deleted file mode 100644
index d9cc86f..0000000
--- a/helix-core/src/test/java/org/apache/helix/alerts/TestBaseStatsValidation.java
+++ /dev/null
@@ -1,169 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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 org.apache.helix.HelixException;
-import org.testng.AssertJUnit;
-import org.testng.annotations.Test;
-
-@Test
-public class TestBaseStatsValidation {
-
-  @Test
-  public void testParseSingletonExpression() {
-    String[] actual = null;
-
-    String statName = "window(5)(dbFoo.partition10.latency)";
-    try {
-      actual = ExpressionParser.getBaseStats(statName);
-    } catch (HelixException e) {
-      e.printStackTrace();
-    }
-    AssertJUnit.assertEquals(statName, actual[0]);
-  }
-
-  @Test
-  public void testExtraParen() {
-    String[] actual = null;
-
-    String statName = "window(5)(dbFoo.partition10.latency)()";
-    boolean caughtException = false;
-    try {
-      actual = ExpressionParser.getBaseStats(statName);
-    } catch (HelixException e) {
-      caughtException = true;
-      // e.printStackTrace();
-    }
-    AssertJUnit.assertEquals(true, caughtException);
-  }
-
-  @Test
-  public void testParseSingletonWildcardExpression() {
-    String[] actual = null;
-
-    String statName = "accumulate()(dbFoo.partition*.latency)";
-    try {
-      actual = ExpressionParser.getBaseStats(statName);
-    } catch (HelixException e) {
-      e.printStackTrace();
-    }
-    AssertJUnit.assertEquals(statName, actual[0]);
-  }
-
-  @Test
-  public void testParsePairOfExpressions() {
-    String[] actual = null;
-
-    String expression = "accumulate()(dbFoo.partition10.latency, dbFoo.partition10.count)";
-    try {
-      actual = ExpressionParser.getBaseStats(expression);
-    } catch (HelixException e) {
-      e.printStackTrace();
-    }
-    AssertJUnit.assertEquals("accumulate()(dbFoo.partition10.latency)", actual[0]);
-    AssertJUnit.assertEquals("accumulate()(dbFoo.partition10.count)", actual[1]);
-  }
-
-  /*
-   * SUM is not to be persisted, so pull out the pieces
-   */
-  @Test
-  public void testSUMExpression() {
-    String[] actual = null;
-
-    String expression = "accumulate()(dbFoo.partition*.latency)|SUM";
-    try {
-      actual = ExpressionParser.getBaseStats(expression);
-    } catch (HelixException e) {
-      e.printStackTrace();
-    }
-    AssertJUnit.assertEquals("accumulate()(dbFoo.partition*.latency)", actual[0]);
-  }
-
-  @Test
-  public void testSumPairExpression() {
-    String[] actual = null;
-
-    String expression = "window(5)(dbFoo.partition10.latency, dbFoo.partition11.latency)|SUM";
-    try {
-      actual = ExpressionParser.getBaseStats(expression);
-    } catch (HelixException e) {
-      e.printStackTrace();
-    }
-    AssertJUnit.assertEquals("window(5)(dbFoo.partition10.latency)", actual[0]);
-    AssertJUnit.assertEquals("window(5)(dbFoo.partition11.latency)", actual[1]);
-  }
-
-  @Test
-  public void testEachPairExpression() {
-    String[] actual = null;
-
-    String expression = "accumulate()(dbFoo.partition*.latency, dbFoo.partition*.count)|EACH";
-    try {
-      actual = ExpressionParser.getBaseStats(expression);
-    } catch (HelixException e) {
-      e.printStackTrace();
-    }
-    AssertJUnit.assertEquals("accumulate()(dbFoo.partition*.latency)", actual[0]);
-    AssertJUnit.assertEquals("accumulate()(dbFoo.partition*.count)", actual[1]);
-  }
-
-  @Test
-  public void testAccumulateExpression() {
-    String[] actual = null;
-
-    String expression = "accumulate()(dbFoo.partition10.latency)|ACCUMULATE";
-    try {
-      actual = ExpressionParser.getBaseStats(expression);
-    } catch (HelixException e) {
-      e.printStackTrace();
-    }
-    AssertJUnit.assertEquals("accumulate()(dbFoo.partition10.latency)", actual[0]);
-  }
-
-  @Test
-  public void testAccumulateEachExpression() {
-    String[] actual = null;
-
-    String expression = "window(5)(dbFoo.partition*.latency)|EACH|ACCUMULATE";
-    try {
-      actual = ExpressionParser.getBaseStats(expression);
-    } catch (HelixException e) {
-      e.printStackTrace();
-    }
-    AssertJUnit.assertEquals("window(5)(dbFoo.partition*.latency)", actual[0]);
-  }
-
-  @Test
-  public void testAccumulateEachPairExpression() {
-    String[] actual = null;
-
-    String expression =
-        "accumulate()(dbFoo.partition*.latency, dbFoo.partition*.count)|EACH|ACCUMULATE|DIVIDE";
-    try {
-      actual = ExpressionParser.getBaseStats(expression);
-    } catch (HelixException e) {
-      e.printStackTrace();
-    }
-    AssertJUnit.assertEquals("accumulate()(dbFoo.partition*.latency)", actual[0]);
-    AssertJUnit.assertEquals("accumulate()(dbFoo.partition*.count)", actual[1]);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/test/java/org/apache/helix/alerts/TestEvaluateAlerts.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/alerts/TestEvaluateAlerts.java b/helix-core/src/test/java/org/apache/helix/alerts/TestEvaluateAlerts.java
deleted file mode 100644
index 7089146..0000000
--- a/helix-core/src/test/java/org/apache/helix/alerts/TestEvaluateAlerts.java
+++ /dev/null
@@ -1,388 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.HashMap;
-import java.util.Map;
-
-import org.apache.helix.HelixException;
-import org.apache.helix.Mocks.MockManager;
-import org.apache.helix.controller.stages.HealthDataCache;
-import org.testng.AssertJUnit;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
-
-public class TestEvaluateAlerts {
-  protected static final String CLUSTER_NAME = "TestCluster";
-
-  MockManager _helixManager;
-  AlertsHolder _alertsHolder;
-  StatsHolder _statsHolder;
-
-  public final String EXP = AlertParser.EXPRESSION_NAME;
-  public final String CMP = AlertParser.COMPARATOR_NAME;
-  public final String CON = AlertParser.CONSTANT_NAME;
-
-  @BeforeMethod(groups = {
-    "unitTest"
-  })
-  public void setup() {
-    HealthDataCache cache = new HealthDataCache();
-    _helixManager = new MockManager(CLUSTER_NAME);
-    _alertsHolder = new AlertsHolder(_helixManager, cache);
-    _statsHolder = _alertsHolder._statsHolder;
-  }
-
-  public Map<String, String> getStatFields(String value, String timestamp) {
-    Map<String, String> statMap = new HashMap<String, String>();
-    statMap.put(StatsHolder.VALUE_NAME, value);
-    statMap.put(StatsHolder.TIMESTAMP_NAME, timestamp);
-    return statMap;
-  }
-
-  public String getSimpleStat() throws HelixException {
-    String stat = "accumulate()(dbFoo.partition10.latency)";
-    // _statsHolder.addStat(stat);
-    return stat;
-  }
-
-  public String addPairOfStats() throws HelixException {
-    String stat = "accumulate()(dbFoo.partition10.latency, dbFoo.partition11.latency)";
-    _statsHolder.addStat(stat);
-    _statsHolder.persistStats();
-    return stat;
-  }
-
-  public String getWildcardStat() throws HelixException {
-    String stat = "accumulate()(dbFoo.partition*.latency)";
-    // _statsHolder.addStat(stat);
-    return stat;
-  }
-
-  public String addSimpleAlert() throws HelixException {
-    String alert =
-        EXP + "(accumulate()(dbFoo.partition10.latency))" + CMP + "(GREATER)" + CON + "(100)";
-    _alertsHolder.addAlert(alert);
-    return alert;
-  }
-
-  public String addWildcardAlert() throws HelixException {
-    String alert =
-        EXP + "(accumulate()(dbFoo.partition*.latency))" + CMP + "(GREATER)" + CON + "(100)";
-    _alertsHolder.addAlert(alert);
-    return alert;
-  }
-
-  public String addTwoWildcardAlert() throws HelixException {
-    String alert =
-        EXP + "(accumulate()(dbFoo.partition*.put*))" + CMP + "(GREATER)" + CON + "(100)";
-    _alertsHolder.addAlert(alert);
-    return alert;
-  }
-
-  public String addExpandWildcardAlert() throws HelixException {
-    String alert =
-        EXP + "(accumulate()(dbFoo.partition*.latency)|EXPAND)" + CMP + "(GREATER)" + CON + "(100)";
-    _alertsHolder.addAlert(alert);
-    return alert;
-  }
-
-  public String addExpandSumAlert() throws HelixException {
-    String alert =
-        EXP + "(accumulate()(dbFoo.partition10.latency,dbFoo.partition11.latency)|EXPAND|SUM)"
-            + CMP + "(GREATER)" + CON + "(100)";
-    _alertsHolder.addAlert(alert);
-    return alert;
-  }
-
-  public String addExpandSumWildcardAlert() throws HelixException {
-    String alert =
-        EXP + "(accumulate()(dbFoo.partition*.success,dbFoo.partition*.failure)|EXPAND|SUM)" + CMP
-            + "(GREATER)" + CON + "(100)";
-    _alertsHolder.addAlert(alert);
-    return alert;
-  }
-
-  public String addExpandSumEachWildcardAlert() throws HelixException {
-    String alert =
-        EXP + "(accumulate()(dbFoo.partition*.success,dbFoo.partition*.failure)|EXPAND|SUMEACH)"
-            + CMP + "(GREATER)" + CON + "(100)";
-    _alertsHolder.addAlert(alert);
-    return alert;
-  }
-
-  public String addExpandSumEachSumWildcardAlert() throws HelixException {
-    String alert =
-        EXP
-            + "(accumulate()(dbFoo.partition*.success,dbFoo.partition*.failure)|EXPAND|SUMEACH|SUM)"
-            + CMP + "(GREATER)" + CON + "(100)";
-    _alertsHolder.addAlert(alert);
-    return alert;
-  }
-
-  public String addArrivingSimpleStat() throws HelixException {
-    _statsHolder.refreshStats();
-    String incomingStatName = "dbFoo.partition10.latency";
-    Map<String, String> statFields = getStatFields("110", "0");
-    _statsHolder.applyStat(incomingStatName, statFields);
-    _statsHolder.persistStats();
-    return incomingStatName;
-  }
-
-  public String addArrivingPairOfStats() throws HelixException {
-    _statsHolder.refreshStats();
-    String incomingStatName1 = "dbFoo.partition10.latency";
-    String incomingStatName2 = "dbFoo.partition11.latency";
-    Map<String, String> statFields = getStatFields("50", "0");
-    _statsHolder.applyStat(incomingStatName1, statFields);
-    statFields = getStatFields("51", "0");
-    _statsHolder.applyStat(incomingStatName2, statFields);
-    _statsHolder.persistStats();
-    return null;
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testSimpleAlertFires() {
-    String alert = addSimpleAlert();
-    String stat = AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert);
-    _statsHolder.refreshStats(); // need to refresh since not triggered by stats aggregation stage
-    addArrivingSimpleStat();
-    Map<String, Map<String, AlertValueAndStatus>> alertResult =
-        AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList());
-    boolean alertFired = alertResult.get(alert).get(AlertProcessor.noWildcardAlertKey).isFired();
-    AssertJUnit.assertTrue(alertFired);
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testSimpleAlertNoStatArrivesFires() {
-    String alert = addSimpleAlert();
-    String stat = AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert);
-    Map<String, Map<String, AlertValueAndStatus>> alertResult =
-        AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList());
-    AssertJUnit.assertEquals(null, alertResult.get(AlertProcessor.noWildcardAlertKey));
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testWildcardAlertFires() {
-    String alert = addWildcardAlert();
-    String stat = AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert);
-    String incomingStatName = addArrivingSimpleStat();
-
-    Map<String, Map<String, AlertValueAndStatus>> alertResult =
-        AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList());
-    String wildcardBinding = incomingStatName;
-    boolean alertFired = alertResult.get(alert).get(wildcardBinding).isFired();
-    AssertJUnit.assertTrue(alertFired);
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testExpandOperatorWildcardAlertFires() {
-    String alert = addExpandWildcardAlert();
-    String stat = AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert);
-    String incomingStatName = addArrivingSimpleStat();
-    Map<String, Map<String, AlertValueAndStatus>> alertResult =
-        AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList());
-    String wildcardBinding = incomingStatName;
-    boolean alertFired = alertResult.get(alert).get(wildcardBinding).isFired();
-    AssertJUnit.assertTrue(alertFired);
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testExpandSumOperatorAlertFires() {
-    String alert = addExpandSumAlert();
-    String stat = AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert);
-    addArrivingPairOfStats();
-    Map<String, Map<String, AlertValueAndStatus>> alertResult =
-        AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList());
-    boolean alertFired = alertResult.get(alert).get(AlertProcessor.noWildcardAlertKey).isFired();
-    AssertJUnit.assertTrue(alertFired);
-  }
-
-  /**
-   * We need to re-decide how to support the feature to specify more than one stats in
-   * an alert.
-   * Probabaly instead of
-   * "(dbFoo.partition*.success,dbFoo.partition*.failure)", use the form
-   * "(dbFoo.partition*.(success, failure))" as it seems that the stat source is always the
-   * same.
-   * 
-   //@Test (groups = {"unitTest"})
-   * public void testExpandSumOperatorWildcardAlert()
-   * {
-   * String alert = addExpandSumWildcardAlert();
-   * String stat = AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert);
-   * String part10SuccStat = "dbFoo.partition10.success";
-   * String part10FailStat = "dbFoo.partition10.failure";
-   * String part11SuccStat = "dbFoo.partition11.success";
-   * String part11FailStat = "dbFoo.partition11.failure";
-   * Map<String, String> statFields = getStatFields("50","0");
-   * _statsHolder.applyStat(part10SuccStat, statFields);
-   * statFields = getStatFields("51","0");
-   * _statsHolder.applyStat(part10FailStat, statFields);
-   * statFields = getStatFields("50","0");
-   * _statsHolder.applyStat(part11SuccStat, statFields);
-   * statFields = getStatFields("49","0");
-   * _statsHolder.applyStat(part11FailStat, statFields);
-   * Map<String, Map<String, AlertValueAndStatus>> alertResult =
-   * AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList());
-   * boolean alertFired = alertResult.get(alert).get("10").isFired(); //10 should fire
-   * AssertJUnit.assertTrue(alertFired);
-   * alertFired = alertResult.get(alert).get("11").isFired(); //11 should not fire
-   * AssertJUnit.assertFalse(alertFired);
-   * }
-   * //@Test (groups = {"unitTest"})
-   * public void testExpandSumEachSumOperatorWildcardAlert()
-   * {
-   * String alert = addExpandSumEachSumWildcardAlert();
-   * String stat = AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert);
-   * String part10SuccStat = "dbFoo.partition10.success";
-   * String part10FailStat = "dbFoo.partition10.failure";
-   * String part11SuccStat = "dbFoo.partition11.success";
-   * String part11FailStat = "dbFoo.partition11.failure";
-   * Map<String, String> statFields = getStatFields("50","0");
-   * _statsHolder.applyStat(part10SuccStat, statFields);
-   * statFields = getStatFields("51","0");
-   * _statsHolder.applyStat(part10FailStat, statFields);
-   * statFields = getStatFields("50","0");
-   * _statsHolder.applyStat(part11SuccStat, statFields);
-   * statFields = getStatFields("49","0");
-   * _statsHolder.applyStat(part11FailStat, statFields);
-   * Map<String, Map<String, AlertValueAndStatus>> alertResult =
-   * AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList());
-   * boolean alertFired = alertResult.get(alert).get(_statsHolder.getStatsList().get(0)).isFired();
-   * //10 should fire
-   * AssertJUnit.assertTrue(alertFired);
-   * }
-   * //@Test (groups = {"unitTest"})
-   * public void testTwoAlerts()
-   * {
-   * //alert 1
-   * String alert1 = addSimpleAlert();
-   * String stat = AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert1);
-   * addArrivingSimpleStat();
-   * //alert 2
-   * String alert2 = addExpandSumWildcardAlert();
-   * stat = AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert2);
-   * String part10SuccStat = "dbFoo.partition10.success";
-   * String part10FailStat = "dbFoo.partition10.failure";
-   * String part11SuccStat = "dbFoo.partition11.success";
-   * String part11FailStat = "dbFoo.partition11.failure";
-   * Map<String, String> statFields = getStatFields("50","0");
-   * _statsHolder.applyStat(part10SuccStat, statFields);
-   * statFields = getStatFields("51","0");
-   * _statsHolder.applyStat(part10FailStat, statFields);
-   * statFields = getStatFields("50","0");
-   * _statsHolder.applyStat(part11SuccStat, statFields);
-   * statFields = getStatFields("49","0");
-   * _statsHolder.applyStat(part11FailStat, statFields);
-   * Map<String, Map<String, AlertValueAndStatus>> alertResult =
-   * AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList());
-   * //alert 1 check
-   * boolean alertFired = alertResult.get(alert1).get(AlertProcessor.noWildcardAlertKey).isFired();
-   * AssertJUnit.assertTrue(alertFired);
-   * //alert 2 check
-   * alertFired = alertResult.get(alert2).get("10").isFired(); //10 should fire
-   * AssertJUnit.assertTrue(alertFired);
-   * alertFired = alertResult.get(alert2).get("11").isFired(); //11 should not fire
-   * AssertJUnit.assertFalse(alertFired);
-   * }
-   */
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testAddWildcardInFirstStatToken() throws Exception {
-    String alert = "EXP(decay(1)(instance*.reportingage))CMP(GREATER)CON(300)";
-    _alertsHolder.addAlert(alert);
-    _statsHolder.persistStats();
-
-    _statsHolder.refreshStats();
-    // generate incoming stat
-    String incomingStatName = "instance10.reportingage";
-    Map<String, String> statFields = getStatFields("301", "10");
-    _statsHolder.refreshStats();
-
-    _statsHolder.applyStat(incomingStatName, statFields);
-    _statsHolder.persistStats();
-
-    Map<String, Map<String, AlertValueAndStatus>> alertResult =
-        AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList());
-    String wildcardBinding = incomingStatName;
-    boolean alertFired = alertResult.get(alert).get(wildcardBinding).isFired();
-    AssertJUnit.assertTrue(alertFired);
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testTwoWildcardAlertFires() {
-    // error is with * and )
-    String alert = addTwoWildcardAlert();
-    String stat = AlertParser.getComponent(AlertParser.EXPRESSION_NAME, alert);
-    String incomingStatName = "dbFoo.partition10.putCount";
-    Map<String, String> statFields = getStatFields("110", "0");
-    _statsHolder.refreshStats();
-    _statsHolder.applyStat(incomingStatName, statFields);
-    _statsHolder.persistStats();
-    Map<String, Map<String, AlertValueAndStatus>> alertResult =
-        AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList());
-    String wildcardBinding = incomingStatName; // XXX: this is not going to work...need "Count" in
-                                               // here too.
-    boolean alertFired = alertResult.get(alert).get(wildcardBinding).isFired();
-    AssertJUnit.assertTrue(alertFired);
-  }
-
-  /*
-   * only supporting wildcards at end of components right now
-   * @Test (groups = {"unitTest"})
-   * public void testTwoWildcardsNotAtEndFires()
-   * {
-   * String alert = EXP + "(accumulate()(dbFoo.partition*.*Count))"
-   * + CMP + "(GREATER)" + CON + "(100)";
-   * _alertsHolder.addAlert(alert);
-   * String incomingStatName = "dbFoo.partition10.putCount";
-   * Map<String, String> statFields = getStatFields("110","0");
-   * _statsHolder.applyStat(incomingStatName, statFields);
-   * Map<String, Map<String, AlertValueAndStatus>> alertResult =
-   * AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList());
-   * String wildcardBinding = "10,put"; //XXX: this is not going to work...need "Count" in here too.
-   * boolean alertFired = alertResult.get(alert).get(wildcardBinding).isFired();
-   * AssertJUnit.assertTrue(alertFired);
-   * }
-   */
-
-  // test using sumall
-  // test using rows where some tuples are null (no stat sent)
-  // test with window tuples where some windows are different lengths
-  // anything else, look around at the code
-
-  // next: review all older tests
-  // next: actually write the fired alerts to ZK
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/test/java/org/apache/helix/alerts/TestOperators.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/alerts/TestOperators.java b/helix-core/src/test/java/org/apache/helix/alerts/TestOperators.java
deleted file mode 100644
index 4e141dc..0000000
--- a/helix-core/src/test/java/org/apache/helix/alerts/TestOperators.java
+++ /dev/null
@@ -1,325 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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.Iterator;
-import java.util.List;
-
-import org.testng.AssertJUnit;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
-
-public class TestOperators {
-
-  SumOperator _sumOp;
-  SumEachOperator _sumEachOp;
-
-  @BeforeMethod(groups = {
-    "unitTest"
-  })
-  public void setup() {
-    _sumOp = new SumOperator();
-    _sumEachOp = new SumEachOperator();
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testTwoNulls() {
-    Tuple<String> tup1 = null;
-    Tuple<String> tup2 = null;
-    List<Tuple<String>> tup1List = new ArrayList<Tuple<String>>();
-    List<Tuple<String>> tup2List = new ArrayList<Tuple<String>>();
-    tup1List.add(tup1);
-    tup2List.add(tup2);
-    List<Iterator<Tuple<String>>> tupsList = new ArrayList<Iterator<Tuple<String>>>();
-    tupsList.add(tup1List.iterator());
-    tupsList.add(tup2List.iterator());
-    List<Iterator<Tuple<String>>> result = _sumOp.execute(tupsList);
-    AssertJUnit.assertEquals(1, result.size()); // should be just 1 iter
-    Iterator<Tuple<String>> resultIter = result.get(0);
-    AssertJUnit.assertTrue(resultIter.hasNext());
-    Tuple<String> resultTup = resultIter.next();
-    AssertJUnit.assertEquals(null, resultTup);
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testOneNullLeft() {
-    Tuple<String> tup1 = null;
-    Tuple<String> tup2 = new Tuple<String>();
-    tup2.add("1.0");
-    List<Tuple<String>> tup1List = new ArrayList<Tuple<String>>();
-    List<Tuple<String>> tup2List = new ArrayList<Tuple<String>>();
-    tup1List.add(tup1);
-    tup2List.add(tup2);
-    List<Iterator<Tuple<String>>> tupsList = new ArrayList<Iterator<Tuple<String>>>();
-    tupsList.add(tup1List.iterator());
-    tupsList.add(tup2List.iterator());
-    List<Iterator<Tuple<String>>> result = _sumOp.execute(tupsList);
-    AssertJUnit.assertEquals(1, result.size()); // should be just 1 iter
-    Iterator<Tuple<String>> resultIter = result.get(0);
-    AssertJUnit.assertTrue(resultIter.hasNext());
-    Tuple<String> resultTup = resultIter.next();
-    AssertJUnit.assertEquals("1.0", resultTup.toString());
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testOneNullRight() {
-    Tuple<String> tup1 = new Tuple<String>();
-    Tuple<String> tup2 = null;
-    tup1.add("1.0");
-    List<Tuple<String>> tup1List = new ArrayList<Tuple<String>>();
-    List<Tuple<String>> tup2List = new ArrayList<Tuple<String>>();
-    tup1List.add(tup1);
-    tup2List.add(tup2);
-    List<Iterator<Tuple<String>>> tupsList = new ArrayList<Iterator<Tuple<String>>>();
-    tupsList.add(tup1List.iterator());
-    tupsList.add(tup2List.iterator());
-    List<Iterator<Tuple<String>>> result = _sumOp.execute(tupsList);
-    AssertJUnit.assertEquals(1, result.size()); // should be just 1 iter
-    Iterator<Tuple<String>> resultIter = result.get(0);
-    AssertJUnit.assertTrue(resultIter.hasNext());
-    Tuple<String> resultTup = resultIter.next();
-    AssertJUnit.assertEquals("1.0", resultTup.toString());
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testTwoSingeltons() {
-    Tuple<String> tup1 = new Tuple<String>();
-    Tuple<String> tup2 = new Tuple<String>();
-    tup1.add("1.0");
-    tup2.add("2.0");
-    List<Tuple<String>> tup1List = new ArrayList<Tuple<String>>();
-    List<Tuple<String>> tup2List = new ArrayList<Tuple<String>>();
-    tup1List.add(tup1);
-    tup2List.add(tup2);
-    List<Iterator<Tuple<String>>> tupsList = new ArrayList<Iterator<Tuple<String>>>();
-    tupsList.add(tup1List.iterator());
-    tupsList.add(tup2List.iterator());
-    List<Iterator<Tuple<String>>> result = _sumOp.execute(tupsList);
-    AssertJUnit.assertEquals(1, result.size()); // should be just 1 iter
-    Iterator<Tuple<String>> resultIter = result.get(0);
-    AssertJUnit.assertTrue(resultIter.hasNext());
-    Tuple<String> resultTup = resultIter.next();
-    AssertJUnit.assertEquals("3.0", resultTup.toString());
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testThreeSingeltons() {
-    Tuple<String> tup1 = new Tuple<String>();
-    Tuple<String> tup2 = new Tuple<String>();
-    Tuple<String> tup3 = new Tuple<String>();
-    tup1.add("1.0");
-    tup2.add("2.0");
-    tup3.add("3.0");
-    List<Tuple<String>> tup1List = new ArrayList<Tuple<String>>();
-    List<Tuple<String>> tup2List = new ArrayList<Tuple<String>>();
-    List<Tuple<String>> tup3List = new ArrayList<Tuple<String>>();
-    tup1List.add(tup1);
-    tup2List.add(tup2);
-    tup3List.add(tup3);
-    List<Iterator<Tuple<String>>> tupsList = new ArrayList<Iterator<Tuple<String>>>();
-    tupsList.add(tup1List.iterator());
-    tupsList.add(tup2List.iterator());
-    tupsList.add(tup3List.iterator());
-    List<Iterator<Tuple<String>>> result = _sumOp.execute(tupsList);
-    AssertJUnit.assertEquals(1, result.size()); // should be just 1 iter
-    Iterator<Tuple<String>> resultIter = result.get(0);
-    AssertJUnit.assertTrue(resultIter.hasNext());
-    Tuple<String> resultTup = resultIter.next();
-    AssertJUnit.assertEquals("6.0", resultTup.toString());
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testThreeTriples() {
-    Tuple<String> tup1 = new Tuple<String>();
-    Tuple<String> tup2 = new Tuple<String>();
-    Tuple<String> tup3 = new Tuple<String>();
-    tup1.add("1.0");
-    tup1.add("2.0");
-    tup1.add("3.0");
-    tup2.add("4.0");
-    tup2.add("5.0");
-    tup2.add("6.0");
-    tup3.add("7.0");
-    tup3.add("8.0");
-    tup3.add("9.0");
-    List<Tuple<String>> tup1List = new ArrayList<Tuple<String>>();
-    List<Tuple<String>> tup2List = new ArrayList<Tuple<String>>();
-    List<Tuple<String>> tup3List = new ArrayList<Tuple<String>>();
-    tup1List.add(tup1);
-    tup2List.add(tup2);
-    tup3List.add(tup3);
-    List<Iterator<Tuple<String>>> tupsList = new ArrayList<Iterator<Tuple<String>>>();
-    tupsList.add(tup1List.iterator());
-    tupsList.add(tup2List.iterator());
-    tupsList.add(tup3List.iterator());
-    List<Iterator<Tuple<String>>> result = _sumOp.execute(tupsList);
-    AssertJUnit.assertEquals(1, result.size()); // should be just 1 iter
-    Iterator<Tuple<String>> resultIter = result.get(0);
-    AssertJUnit.assertTrue(resultIter.hasNext());
-    Tuple<String> resultTup = resultIter.next();
-    AssertJUnit.assertEquals("12.0,15.0,18.0", resultTup.toString());
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testThreeTriplesOneMissing() {
-    Tuple<String> tup1 = new Tuple<String>();
-    Tuple<String> tup2 = new Tuple<String>();
-    Tuple<String> tup3 = new Tuple<String>();
-    tup1.add("1.0");
-    tup1.add("2.0");
-    tup1.add("3.0");
-    tup2.add("5.0");
-    tup2.add("6.0");
-    tup3.add("7.0");
-    tup3.add("8.0");
-    tup3.add("9.0");
-    List<Tuple<String>> tup1List = new ArrayList<Tuple<String>>();
-    List<Tuple<String>> tup2List = new ArrayList<Tuple<String>>();
-    List<Tuple<String>> tup3List = new ArrayList<Tuple<String>>();
-    tup1List.add(tup1);
-    tup2List.add(tup2);
-    tup3List.add(tup3);
-    List<Iterator<Tuple<String>>> tupsList = new ArrayList<Iterator<Tuple<String>>>();
-    tupsList.add(tup1List.iterator());
-    tupsList.add(tup2List.iterator());
-    tupsList.add(tup3List.iterator());
-    List<Iterator<Tuple<String>>> result = _sumOp.execute(tupsList);
-    AssertJUnit.assertEquals(1, result.size()); // should be just 1 iter
-    Iterator<Tuple<String>> resultIter = result.get(0);
-    AssertJUnit.assertTrue(resultIter.hasNext());
-    Tuple<String> resultTup = resultIter.next();
-    // tuple 2 missing 1 entry, other 2 get bumped to right
-    AssertJUnit.assertEquals("8.0,15.0,18.0", resultTup.toString());
-  }
-
-  // test multiple rows
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testThreeTriplesOneMissingTwoRows() {
-    Tuple<String> tup1Dot1 = new Tuple<String>();
-    Tuple<String> tup2Dot1 = new Tuple<String>();
-    Tuple<String> tup3Dot1 = new Tuple<String>();
-    Tuple<String> tup1Dot2 = new Tuple<String>();
-    Tuple<String> tup2Dot2 = new Tuple<String>();
-    Tuple<String> tup3Dot2 = new Tuple<String>();
-    tup1Dot1.add("1.0");
-    tup1Dot1.add("2.0");
-    tup1Dot1.add("3.0");
-    tup2Dot1.add("5.0");
-    tup2Dot1.add("6.0");
-    tup3Dot1.add("7.0");
-    tup3Dot1.add("8.0");
-    tup3Dot1.add("9.0");
-    tup1Dot2.add("10.0");
-    tup1Dot2.add("11.0");
-    tup1Dot2.add("12.0");
-    tup2Dot2.add("13.0");
-    tup2Dot2.add("14.0");
-    tup2Dot2.add("15.0");
-    tup3Dot2.add("16.0");
-    tup3Dot2.add("17.0");
-    tup3Dot2.add("18.0");
-    List<Tuple<String>> tup1List = new ArrayList<Tuple<String>>();
-    List<Tuple<String>> tup2List = new ArrayList<Tuple<String>>();
-    List<Tuple<String>> tup3List = new ArrayList<Tuple<String>>();
-    tup1List.add(tup1Dot1);
-    tup2List.add(tup2Dot1);
-    tup3List.add(tup3Dot1);
-    tup1List.add(tup1Dot2);
-    tup2List.add(tup2Dot2);
-    tup3List.add(tup3Dot2);
-    List<Iterator<Tuple<String>>> tupsList = new ArrayList<Iterator<Tuple<String>>>();
-    tupsList.add(tup1List.iterator());
-    tupsList.add(tup2List.iterator());
-    tupsList.add(tup3List.iterator());
-    List<Iterator<Tuple<String>>> result = _sumOp.execute(tupsList);
-    AssertJUnit.assertEquals(1, result.size()); // should be just 1 iter
-    Iterator<Tuple<String>> resultIter = result.get(0);
-    AssertJUnit.assertTrue(resultIter.hasNext());
-    Tuple<String> resultTup1 = resultIter.next();
-    // tuple 2 missing 1 entry, other 2 get bumped to right
-    AssertJUnit.assertEquals("8.0,15.0,18.0", resultTup1.toString());
-    AssertJUnit.assertTrue(resultIter.hasNext());
-    Tuple<String> resultTup2 = resultIter.next();
-    AssertJUnit.assertEquals("39.0,42.0,45.0", resultTup2.toString());
-  }
-
-  @Test(groups = {
-    "unitTest"
-  })
-  public void testSumAll() {
-    Tuple<String> tup1 = new Tuple<String>();
-    Tuple<String> tup2 = new Tuple<String>();
-    Tuple<String> tup3 = new Tuple<String>();
-    Tuple<String> tup4 = new Tuple<String>();
-    Tuple<String> tup5 = new Tuple<String>();
-    Tuple<String> tup6 = new Tuple<String>();
-    tup1.add("1.0");
-    tup2.add("2.0");
-    tup3.add("3.0");
-    tup4.add("4.0");
-    tup5.add("5.0");
-    tup6.add("6.0");
-    List<Tuple<String>> list1 = new ArrayList<Tuple<String>>();
-    List<Tuple<String>> list2 = new ArrayList<Tuple<String>>();
-    List<Tuple<String>> list3 = new ArrayList<Tuple<String>>();
-    list1.add(tup1);
-    list1.add(tup4);
-    list2.add(tup2);
-    list2.add(tup5);
-    list3.add(tup3);
-    list3.add(tup6);
-
-    List<Iterator<Tuple<String>>> tupsList = new ArrayList<Iterator<Tuple<String>>>();
-    tupsList.add(list1.iterator());
-    tupsList.add(list2.iterator());
-    tupsList.add(list3.iterator());
-    List<Iterator<Tuple<String>>> result = _sumEachOp.execute(tupsList);
-    AssertJUnit.assertEquals(3, result.size()); // should be just 1 iter
-    Iterator<Tuple<String>> resultIter = result.get(0);
-    AssertJUnit.assertTrue(resultIter.hasNext());
-    Tuple<String> resultTup1 = resultIter.next();
-    AssertJUnit.assertEquals("5.0", resultTup1.toString());
-    resultIter = result.get(1);
-    AssertJUnit.assertTrue(resultIter.hasNext());
-    resultTup1 = resultIter.next();
-    AssertJUnit.assertEquals("7.0", resultTup1.toString());
-    resultIter = result.get(2);
-    AssertJUnit.assertTrue(resultIter.hasNext());
-    resultTup1 = resultIter.next();
-    AssertJUnit.assertEquals("9.0", resultTup1.toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/test/java/org/apache/helix/alerts/TestStatsMatch.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/alerts/TestStatsMatch.java b/helix-core/src/test/java/org/apache/helix/alerts/TestStatsMatch.java
deleted file mode 100644
index 81a2cd4..0000000
--- a/helix-core/src/test/java/org/apache/helix/alerts/TestStatsMatch.java
+++ /dev/null
@@ -1,103 +0,0 @@
-package org.apache.helix.alerts;
-
-/*
- * 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 org.testng.AssertJUnit;
-import org.testng.annotations.Test;
-
-@Test
-public class TestStatsMatch {
-
-  @Test
-  public void testExactMatch() {
-
-    String persistedStatName = "window(5)(dbFoo.partition10.latency)";
-    String incomingStatName = "dbFoo.partition10.latency";
-    AssertJUnit.assertTrue(ExpressionParser.isIncomingStatExactMatch(persistedStatName,
-        incomingStatName));
-  }
-
-  @Test
-  public void testSingleWildcardMatch() {
-
-    String persistedStatName = "window(5)(dbFoo.partition*.latency)";
-    String incomingStatName = "dbFoo.partition10.latency";
-    AssertJUnit.assertTrue(ExpressionParser.isIncomingStatWildcardMatch(persistedStatName,
-        incomingStatName));
-  }
-
-  @Test
-  public void testDoubleWildcardMatch() {
-
-    String persistedStatName = "window(5)(db*.partition*.latency)";
-    String incomingStatName = "dbFoo.partition10.latency";
-    AssertJUnit.assertTrue(ExpressionParser.isIncomingStatWildcardMatch(persistedStatName,
-        incomingStatName));
-  }
-
-  @Test
-  public void testWildcardMatchNoWildcard() {
-
-    String persistedStatName = "window(5)(dbFoo.partition10.latency)";
-    String incomingStatName = "dbFoo.partition10.latency";
-    AssertJUnit.assertFalse(ExpressionParser.isIncomingStatWildcardMatch(persistedStatName,
-        incomingStatName));
-  }
-
-  @Test
-  public void testWildcardMatchTooManyFields() {
-
-    String persistedStatName = "window(5)(dbFoo.partition*.latency)";
-    String incomingStatName = "dbFoo.tableBar.partition10.latency";
-    AssertJUnit.assertFalse(ExpressionParser.isIncomingStatWildcardMatch(persistedStatName,
-        incomingStatName));
-  }
-
-  @Test
-  public void testWildcardMatchTooFewFields() {
-
-    String persistedStatName = "window(5)(dbFoo.partition*.latency)";
-    String incomingStatName = "dbFoo.latency";
-    AssertJUnit.assertFalse(ExpressionParser.isIncomingStatWildcardMatch(persistedStatName,
-        incomingStatName));
-  }
-
-  @Test
-  public void testBadWildcardRepeated() {
-
-    String persistedStatName = "window(5)(dbFoo.partition**4.latency)";
-    String incomingStatName = "dbFoo.partition10.latency";
-    boolean match =
-        ExpressionParser.isIncomingStatWildcardMatch(persistedStatName, incomingStatName);
-
-    AssertJUnit.assertFalse(match);
-  }
-
-  @Test
-  public void testBadWildcardNotAtEnd() {
-
-    String persistedStatName = "window(5)(dbFoo.*partition.latency)";
-    String incomingStatName = "dbFoo.partition10.latency";
-    boolean match =
-        ExpressionParser.isIncomingStatWildcardMatch(persistedStatName, incomingStatName);
-
-    AssertJUnit.assertFalse(match);
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/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 fc9b7d5..73ba122 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
@@ -25,7 +25,6 @@ import org.apache.helix.ConfigChangeListener;
 import org.apache.helix.ControllerChangeListener;
 import org.apache.helix.CurrentStateChangeListener;
 import org.apache.helix.ExternalViewChangeListener;
-import org.apache.helix.HealthStateChangeListener;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
@@ -40,7 +39,6 @@ import org.apache.helix.PreConnectCallback;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.ScopedConfigChangeListener;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollector;
 import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
@@ -158,12 +156,6 @@ public class DummyClusterManager implements HelixManager {
   }
 
   @Override
-  public ParticipantHealthReportCollector getHealthReportCollector() {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  @Override
   public InstanceType getInstanceType() {
     // TODO Auto-generated method stub
     return null;
@@ -176,13 +168,6 @@ public class DummyClusterManager implements HelixManager {
   }
 
   @Override
-  public void addHealthStateChangeListener(HealthStateChangeListener listener, String instanceName)
-      throws Exception {
-    // TODO Auto-generated method stub
-
-  }
-
-  @Override
   public StateMachineEngine getStateMachineEngine() {
     // TODO Auto-generated method stub
     return null;

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/test/java/org/apache/helix/controller/stages/TestParseInfoFromAlert.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestParseInfoFromAlert.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestParseInfoFromAlert.java
deleted file mode 100644
index 2dbf5f6..0000000
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestParseInfoFromAlert.java
+++ /dev/null
@@ -1,56 +0,0 @@
-package org.apache.helix.controller.stages;
-
-/*
- * 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 org.apache.helix.HelixManager;
-import org.apache.helix.integration.ZkStandAloneCMTestBase;
-import org.testng.Assert;
-import org.testng.annotations.Test;
-
-public class TestParseInfoFromAlert extends ZkStandAloneCMTestBase {
-  @Test
-  public void TestParse() {
-
-    HelixManager manager = _controller;
-
-    String instanceName =
-        StatsAggregationStage.parseInstanceName("localhost_12918.TestStat@DB=123.latency", manager);
-    Assert.assertTrue(instanceName.equals("localhost_12918"));
-
-    instanceName =
-        StatsAggregationStage.parseInstanceName("localhost_12955.TestStat@DB=123.latency", manager);
-    Assert.assertTrue(instanceName == null);
-
-    instanceName =
-        StatsAggregationStage.parseInstanceName("localhost_12922.TestStat@DB=123.latency", manager);
-    Assert.assertTrue(instanceName.equals("localhost_12922"));
-
-    String resourceName =
-        StatsAggregationStage.parseResourceName("localhost_12918.TestStat@DB=TestDB.latency",
-            manager);
-    Assert.assertTrue(resourceName.equals("TestDB"));
-
-    String partitionName =
-        StatsAggregationStage.parsePartitionName(
-            "localhost_12918.TestStat@DB=TestDB;Partition=TestDB_22.latency", manager);
-
-    Assert.assertTrue(partitionName.equals("TestDB_22"));
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/test/java/org/apache/helix/healthcheck/TestAddDropAlert.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestAddDropAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestAddDropAlert.java
deleted file mode 100644
index cadbdc7..0000000
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestAddDropAlert.java
+++ /dev/null
@@ -1,172 +0,0 @@
-package org.apache.helix.healthcheck;
-
-/*
- * 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.Map;
-import java.util.Set;
-
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixManager;
-import org.apache.helix.NotificationContext;
-import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.TestHelper;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.api.State;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
-import org.apache.helix.integration.ZkIntegrationTestBase;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.participant.MockEspressoHealthReportProvider;
-import org.apache.helix.mock.participant.MockTransition;
-import org.apache.helix.model.Message;
-import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier;
-import org.testng.Assert;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
-import org.testng.annotations.Test;
-
-public class TestAddDropAlert extends ZkIntegrationTestBase {
-  protected ClusterSetup _setupTool = null;
-  protected final String _alertStr =
-      "EXP(accumulate()(localhost_12918.RestQueryStats@DBName=TestDB0.latency))CMP(GREATER)CON(10)";
-  protected final String _alertStatusStr = _alertStr; // +" : (*)";
-  protected final String _dbName = "TestDB0";
-
-  @BeforeClass()
-  public void beforeClass() throws Exception {
-    _setupTool = new ClusterSetup(_gZkClient);
-  }
-
-  @AfterClass
-  public void afterClass() {
-  }
-
-  public class AddDropAlertTransition extends MockTransition {
-    @Override
-    public void doTransition(Message message, NotificationContext context) {
-      HelixManager manager = context.getManager();
-      HelixDataAccessor accessor = manager.getHelixDataAccessor();
-      State fromState = message.getTypedFromState();
-      State toState = message.getTypedToState();
-      String instance = message.getTgtName();
-
-      if (fromState.toString().equalsIgnoreCase("SLAVE")
-          && toState.toString().equalsIgnoreCase("MASTER")) {
-
-        // add a stat and report to ZK
-        // perhaps should keep reporter per instance...
-        ParticipantHealthReportCollectorImpl reporter =
-            new ParticipantHealthReportCollectorImpl(manager, instance);
-        MockEspressoHealthReportProvider provider = new MockEspressoHealthReportProvider();
-        reporter.addHealthReportProvider(provider);
-        String statName = "latency";
-        provider.setStat(_dbName, statName, "15");
-        reporter.transmitHealthReports();
-
-        // sleep long enough for first set of alerts to report and alert to get deleted
-        // then change reported data
-        try {
-          Thread.sleep(10000);
-        } catch (InterruptedException e) {
-          System.err.println("Error sleeping");
-        }
-        provider.setStat(_dbName, statName, "1");
-        reporter.transmitHealthReports();
-
-        /*
-         * for (int i = 0; i < 5; i++) { accessor.setProperty(PropertyType.HEALTHREPORT,
-         * new ZNRecord("mockAlerts" + i), instance, "mockAlerts"); try {
-         * Thread.sleep(1000); } catch (InterruptedException e) { // TODO Auto-generated
-         * catch block e.printStackTrace(); } }
-         */
-      }
-    }
-  }
-
-  @Test()
-  public void testAddDropAlert() throws Exception {
-    String clusterName = getShortClassName();
-    MockParticipantManager[] participants = new MockParticipantManager[5];
-
-    System.out.println("START TestAddDropAlert at " + new Date(System.currentTimeMillis()));
-
-    TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant start port
-        "localhost", // participant name prefix
-        "TestDB", // resource name prefix
-        1, // resources
-        10, // partitions per resource group
-        5, // number of nodes //change back to 5!!!
-        1, // replicas //change back to 3!!!
-        "MasterSlave", true); // do rebalance
-    // enableHealthCheck(clusterName);
-
-    _setupTool.getClusterManagementTool().addAlert(clusterName, _alertStr);
-
-    ClusterControllerManager controller =
-        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
-    controller.syncStart();
-
-    // start participants
-    for (int i = 0; i < 5; i++) // !!!change back to 5
-    {
-      String instanceName = "localhost_" + (12918 + i);
-
-      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
-      participants[i].setTransition(new AddDropAlertTransition());
-      participants[i].syncStart();
-    }
-
-    boolean result =
-        ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(
-            ZK_ADDR, clusterName));
-    Assert.assertTrue(result);
-
-    // drop alert soon after adding, but leave enough time for alert to fire once
-    // Thread.sleep(3000);
-    ZKHelixDataAccessor accessor =
-        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
-    Builder keyBuilder = accessor.keyBuilder();
-
-    new HealthStatsAggregator(controller).aggregate();
-    String instance = "localhost_12918";
-    ZNRecord record = accessor.getProperty(keyBuilder.alertStatus()).getRecord();
-    Map<String, Map<String, String>> recMap = record.getMapFields();
-    Set<String> keySet = recMap.keySet();
-    Assert.assertTrue(keySet.size() > 0);
-
-    _setupTool.getClusterManagementTool().dropAlert(clusterName, _alertStr);
-    new HealthStatsAggregator(controller).aggregate();
-    // other verifications go here
-    // for (int i = 0; i < 1; i++) //change 1 back to 5
-    // {
-    // String instance = "localhost_" + (12918 + i);
-    record = accessor.getProperty(keyBuilder.alertStatus()).getRecord();
-    recMap = record.getMapFields();
-    keySet = recMap.keySet();
-    Assert.assertEquals(keySet.size(), 0);
-    // }
-
-    System.out.println("END TestAddDropAlert at " + new Date(System.currentTimeMillis()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/test/java/org/apache/helix/healthcheck/TestDummyAlerts.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestDummyAlerts.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestDummyAlerts.java
deleted file mode 100644
index 29f4893..0000000
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestDummyAlerts.java
+++ /dev/null
@@ -1,148 +0,0 @@
-package org.apache.helix.healthcheck;
-
-/*
- * 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.concurrent.atomic.AtomicBoolean;
-
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixManager;
-import org.apache.helix.NotificationContext;
-import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.TestHelper;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.integration.ZkIntegrationTestBase;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.participant.MockTransition;
-import org.apache.helix.model.HealthStat;
-import org.apache.helix.model.Message;
-import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.MasterNbInExtViewVerifier;
-import org.testng.Assert;
-import org.testng.annotations.Test;
-
-public class TestDummyAlerts extends ZkIntegrationTestBase {
-  public class DummyAlertsTransition extends MockTransition {
-    private final AtomicBoolean _done = new AtomicBoolean(false);
-
-    @Override
-    public void doTransition(Message message, NotificationContext context) {
-      HelixManager manager = context.getManager();
-      HelixDataAccessor accessor = manager.getHelixDataAccessor();
-      Builder keyBuilder = accessor.keyBuilder();
-
-      String instance = message.getTgtName();
-      if (_done.getAndSet(true) == false) {
-        for (int i = 0; i < 5; i++) {
-          // System.out.println(instance + " sets healthReport: " + "mockAlerts" + i);
-          accessor.setProperty(keyBuilder.healthReport(instance, "mockAlerts"), new HealthStat(
-              new ZNRecord("mockAlerts" + i)));
-        }
-      }
-    }
-
-  }
-
-  @Test()
-  public void testDummyAlerts() throws Exception {
-    // Logger.getRootLogger().setLevel(Level.INFO);
-    String className = TestHelper.getTestClassName();
-    String methodName = TestHelper.getTestMethodName();
-    String clusterName = className + "_" + methodName;
-    final int n = 5;
-
-    MockParticipantManager[] participants = new MockParticipantManager[n];
-
-    System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
-
-    TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant start
-                                                         // port
-        "localhost", // participant name prefix
-        "TestDB", // resource name prefix
-        1, // resources
-        10, // partitions per resource
-        n, // number of nodes
-        3, // replicas
-        "MasterSlave", true); // do rebalance
-
-    ClusterSetup setupTool = new ClusterSetup(ZK_ADDR);
-    enableHealthCheck(clusterName);
-    setupTool
-        .getClusterManagementTool()
-        .addAlert(clusterName,
-            "EXP(decay(1.0)(*.defaultPerfCounters@defaultPerfCounters.availableCPUs))CMP(GREATER)CON(2)");
-
-    // start controller
-    ClusterControllerManager controller =
-        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
-    controller.syncStart();
-
-    // start participants
-    for (int i = 0; i < n; i++) {
-      String instanceName = "localhost_" + (12918 + i);
-
-      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
-      participants[i].setTransition(new DummyAlertsTransition());
-      participants[i].syncStart();
-    }
-
-    boolean result =
-        ClusterStateVerifier
-            .verifyByZkCallback(new MasterNbInExtViewVerifier(ZK_ADDR, clusterName));
-    Assert.assertTrue(result);
-
-    result =
-        ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR,
-            clusterName));
-    Assert.assertTrue(result);
-
-    // other verifications go here
-    ZKHelixDataAccessor accessor =
-        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
-    Builder keyBuilder = accessor.keyBuilder();
-
-    for (int i = 0; i < n; i++) {
-      String instance = "localhost_" + (12918 + i);
-      ZNRecord record = null;
-      for (int j = 0; j < 10; j++) {
-        record = accessor.getProperty(keyBuilder.healthReport(instance, "mockAlerts")).getRecord();
-        if (record.getId().equals("mockAlerts4")) {
-          break;
-        } else {
-          Thread.sleep(500);
-        }
-      }
-      Assert.assertEquals(record.getId(), "mockAlerts4");
-    }
-
-    // clean up
-    controller.syncStop();
-    for (int i = 0; i < 5; i++) {
-      participants[i].syncStop();
-    }
-
-    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/test/java/org/apache/helix/healthcheck/TestExpandAlert.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestExpandAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestExpandAlert.java
deleted file mode 100644
index a902158..0000000
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestExpandAlert.java
+++ /dev/null
@@ -1,186 +0,0 @@
-package org.apache.helix.healthcheck;
-
-/*
- * 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.Map;
-import java.util.Set;
-
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixManager;
-import org.apache.helix.NotificationContext;
-import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.TestHelper;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.alerts.AlertValueAndStatus;
-import org.apache.helix.api.State;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
-import org.apache.helix.integration.ZkIntegrationTestBase;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.participant.MockEspressoHealthReportProvider;
-import org.apache.helix.mock.participant.MockTransition;
-import org.apache.helix.model.Message;
-import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier;
-import org.testng.Assert;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
-import org.testng.annotations.Test;
-
-public class TestExpandAlert extends ZkIntegrationTestBase {
-  protected ClusterSetup _setupTool = null;
-  protected final String _alertStr =
-      "EXP(decay(1.0)(localhost_*.RestQueryStats@DBName=TestDB0.latency))CMP(GREATER)CON(16)";
-  protected final String _alertStatusStr = _alertStr
-      + " : (localhost_12918.RestQueryStats@DBName=TestDB0.latency)";
-  protected final String _dbName = "TestDB0";
-
-  @BeforeClass()
-  public void beforeClass() throws Exception {
-
-    _setupTool = new ClusterSetup(_gZkClient);
-  }
-
-  @AfterClass
-  public void afterClass() {
-  }
-
-  public class ExpandAlertTransition extends MockTransition {
-    @Override
-    public void doTransition(Message message, NotificationContext context) {
-      HelixManager manager = context.getManager();
-      HelixDataAccessor accessor = manager.getHelixDataAccessor();
-      State fromState = message.getTypedFromState();
-      State toState = message.getTypedToState();
-      String instance = message.getTgtName();
-
-      if (fromState.toString().equalsIgnoreCase("SLAVE")
-          && toState.toString().equalsIgnoreCase("MASTER")) {
-
-        // add a stat and report to ZK
-        // perhaps should keep reporter per instance...
-        ParticipantHealthReportCollectorImpl reporter =
-            new ParticipantHealthReportCollectorImpl(manager, instance);
-        MockEspressoHealthReportProvider provider = new MockEspressoHealthReportProvider();
-        reporter.addHealthReportProvider(provider);
-        String statName = "latency";
-        provider.setStat(_dbName, statName, "15");
-        reporter.transmitHealthReports();
-
-        /*
-         * for (int i = 0; i < 5; i++)
-         * {
-         * accessor.setProperty(PropertyType.HEALTHREPORT,
-         * new ZNRecord("mockAlerts" + i),
-         * instance,
-         * "mockAlerts");
-         * try
-         * {
-         * Thread.sleep(1000);
-         * }
-         * catch (InterruptedException e)
-         * {
-         * // TODO Auto-generated catch block
-         * e.printStackTrace();
-         * }
-         * }
-         */
-      }
-    }
-
-  }
-
-  @Test()
-  public void testExpandAlert() throws Exception {
-    String clusterName = getShortClassName();
-    MockParticipantManager[] participants = new MockParticipantManager[5];
-
-    System.out.println("START TestExpandAlert at " + new Date(System.currentTimeMillis()));
-
-    TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant start port
-        "localhost", // participant name prefix
-        "TestDB", // resource name prefix
-        1, // resources
-        10, // partitions per resource
-        5, // number of nodes //change back to 5!!!
-        3, // replicas //change back to 3!!!
-        "MasterSlave", true); // do rebalance
-    // enableHealthCheck(clusterName);
-
-    _setupTool.getClusterManagementTool().addAlert(clusterName, _alertStr);
-
-    ClusterControllerManager controller =
-        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
-    controller.syncStart();
-
-    // start participants
-    for (int i = 0; i < 5; i++) // !!!change back to 5
-    {
-      String instanceName = "localhost_" + (12918 + i);
-
-      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
-      participants[i].setTransition(new ExpandAlertTransition());
-      participants[i].syncStart();
-    }
-
-    boolean result =
-        ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(
-            ZK_ADDR, clusterName));
-    Assert.assertTrue(result);
-
-    Thread.sleep(1000);
-    // HealthAggregationTask is supposed to run by a timer every 30s
-    // To make sure HealthAggregationTask is run, we invoke it explicitly for this test
-    // new HealthStatsAggregator(cmResult._manager).aggregate();
-    new HealthStatsAggregator(controller).aggregate();
-    // sleep for a few seconds to give stats stage time to trigger
-    Thread.sleep(3000);
-
-    // other verifications go here
-    ZKHelixDataAccessor accessor =
-        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
-    Builder keyBuilder = accessor.keyBuilder();
-
-    // for (int i = 0; i < 1; i++) //change 1 back to 5
-    // {
-    // String instance = "localhost_" + (12918 + i);
-    // String instance = "localhost_12918";
-    ZNRecord record = accessor.getProperty(keyBuilder.alertStatus()).getRecord();
-    Map<String, Map<String, String>> recMap = record.getMapFields();
-    Set<String> keySet = recMap.keySet();
-    Map<String, String> alertStatusMap = recMap.get(_alertStatusStr);
-    String val = alertStatusMap.get(AlertValueAndStatus.VALUE_NAME);
-    boolean fired = Boolean.parseBoolean(alertStatusMap.get(AlertValueAndStatus.FIRED_NAME));
-    Assert.assertEquals(Double.parseDouble(val), Double.parseDouble("15.0"));
-    Assert.assertFalse(fired);
-    // }
-
-    // clean up
-    controller.syncStop();
-    for (int i = 0; i < 5; i++) {
-      participants[i].syncStop();
-
-    }
-    System.out.println("END TestExpandAlert at " + new Date(System.currentTimeMillis()));
-  }
-}


[07/17] [HELIX-395] Remove old Helix alert/stat modules

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/controller/stages/StatsAggregationStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/StatsAggregationStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/StatsAggregationStage.java
deleted file mode 100644
index c48f156..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/StatsAggregationStage.java
+++ /dev/null
@@ -1,399 +0,0 @@
-package org.apache.helix.controller.stages;
-
-/*
- * 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.text.SimpleDateFormat;
-import java.util.Arrays;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixManager;
-import org.apache.helix.HelixProperty;
-import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.PropertyType;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.alerts.AlertParser;
-import org.apache.helix.alerts.AlertProcessor;
-import org.apache.helix.alerts.AlertValueAndStatus;
-import org.apache.helix.alerts.AlertsHolder;
-import org.apache.helix.alerts.ExpressionParser;
-import org.apache.helix.alerts.StatsHolder;
-import org.apache.helix.alerts.Tuple;
-import org.apache.helix.controller.pipeline.AbstractBaseStage;
-import org.apache.helix.controller.pipeline.StageContext;
-import org.apache.helix.controller.pipeline.StageException;
-import org.apache.helix.healthcheck.StatHealthReportProvider;
-import org.apache.helix.manager.zk.DefaultParticipantErrorMessageHandlerFactory.ActionOnError;
-import org.apache.helix.model.AlertHistory;
-import org.apache.helix.model.HealthStat;
-import org.apache.helix.model.IdealState;
-import org.apache.helix.model.LiveInstance;
-import org.apache.helix.model.PersistentStats;
-import org.apache.helix.monitoring.mbeans.ClusterAlertMBeanCollection;
-import org.apache.log4j.Logger;
-
-/**
- * For each LiveInstances select currentState and message whose sessionId matches
- * sessionId from LiveInstance Get Partition,State for all the resources computed in
- * previous State [ResourceComputationStage]
- */
-public class StatsAggregationStage extends AbstractBaseStage {
-
-  public static final int ALERT_HISTORY_SIZE = 30;
-
-  private static final Logger logger = Logger.getLogger(StatsAggregationStage.class.getName());
-
-  StatsHolder _statsHolder = null;
-  AlertsHolder _alertsHolder = null;
-  Map<String, Map<String, AlertValueAndStatus>> _alertStatus;
-  Map<String, Tuple<String>> _statStatus;
-  ClusterAlertMBeanCollection _alertBeanCollection = new ClusterAlertMBeanCollection();
-  Map<String, String> _alertActionTaken = new HashMap<String, String>();
-
-  public final String PARTICIPANT_STAT_REPORT_NAME = StatHealthReportProvider.REPORT_NAME;
-  public final String ESPRESSO_STAT_REPORT_NAME = "RestQueryStats";
-  public final String REPORT_NAME = "AggStats";
-  // public final String DEFAULT_AGG_TYPE = "decay";
-  // public final String DEFAULT_DECAY_PARAM = "0.1";
-  // public final String DEFAULT_AGG_TYPE = "window";
-  // public final String DEFAULT_DECAY_PARAM = "5";
-
-  public StatHealthReportProvider _aggStatsProvider;
-
-  // public AggregationType _defaultAggType;
-
-  public Map<String, Map<String, AlertValueAndStatus>> getAlertStatus() {
-    return _alertStatus;
-  }
-
-  public Map<String, Tuple<String>> getStatStatus() {
-    return _statStatus;
-  }
-
-  public void persistAggStats(HelixManager manager) {
-    Map<String, String> report = _aggStatsProvider.getRecentHealthReport();
-    Map<String, Map<String, String>> partitionReport =
-        _aggStatsProvider.getRecentPartitionHealthReport();
-    ZNRecord record = new ZNRecord(_aggStatsProvider.getReportName());
-    if (report != null) {
-      record.setSimpleFields(report);
-    }
-    if (partitionReport != null) {
-      record.setMapFields(partitionReport);
-    }
-
-    // DataAccessor accessor = manager.getDataAccessor();
-    HelixDataAccessor accessor = manager.getHelixDataAccessor();
-    // boolean retVal = accessor.setProperty(PropertyType.PERSISTENTSTATS, record);
-    Builder keyBuilder = accessor.keyBuilder();
-    boolean retVal = accessor.setProperty(keyBuilder.persistantStat(), new PersistentStats(record));
-    if (retVal == false) {
-      logger.error("attempt to persist derived stats failed");
-    }
-  }
-
-  @Override
-  public void init(StageContext context) {
-  }
-
-  public String getAgeStatName(String instance) {
-    return instance + ExpressionParser.statFieldDelim + "reportingage";
-  }
-
-  // currTime in seconds
-  public void reportAgeStat(LiveInstance instance, long modifiedTime, long currTime) {
-    String statName = getAgeStatName(instance.getInstanceName());
-    long age = (currTime - modifiedTime) / 1000; // XXX: ensure this is in
-                                                 // seconds
-    Map<String, String> ageStatMap = new HashMap<String, String>();
-    ageStatMap.put(StatsHolder.TIMESTAMP_NAME, String.valueOf(currTime));
-    ageStatMap.put(StatsHolder.VALUE_NAME, String.valueOf(age));
-    // note that applyStat will only work if alert already added
-    _statsHolder.applyStat(statName, ageStatMap);
-  }
-
-  @Override
-  public void process(ClusterEvent event) throws Exception {
-    long startTime = System.currentTimeMillis();
-    // String aggTypeName =
-    // DEFAULT_AGG_TYPE+AggregationType.DELIM+DEFAULT_DECAY_PARAM;
-    // _defaultAggType = AggregationTypeFactory.getAggregationType(aggTypeName);
-
-    HelixManager manager = event.getAttribute("helixmanager");
-    HealthDataCache cache = event.getAttribute("HealthDataCache");
-
-    if (manager == null || cache == null) {
-      throw new StageException("helixmanager|HealthDataCache attribute value is null");
-    }
-    if (_alertsHolder == null) {
-      _statsHolder = new StatsHolder(manager, cache);
-      _alertsHolder = new AlertsHolder(manager, cache, _statsHolder);
-    } else {
-      _statsHolder.updateCache(cache);
-      _alertsHolder.updateCache(cache);
-    }
-    if (_statsHolder.getStatsList().size() == 0) {
-      if (logger.isTraceEnabled()) {
-        logger.trace("stat holder is empty");
-      }
-      return;
-    }
-
-    // init agg stats from cache
-    // initAggStats(cache);
-
-    Map<String, LiveInstance> liveInstances = cache.getLiveInstances();
-
-    long currTime = System.currentTimeMillis();
-    // for each live node, read node's stats
-    long readInstancesStart = System.currentTimeMillis();
-    for (LiveInstance instance : liveInstances.values()) {
-      String instanceName = instance.getInstanceName();
-      logger.debug("instanceName: " + instanceName);
-      // XXX: now have map of HealthStats, so no need to traverse them...verify
-      // correctness
-      Map<String, HealthStat> stats;
-      stats = cache.getHealthStats(instanceName);
-      // find participants stats
-      long modTime = -1;
-      // TODO: get healthreport child node modified time and reportAgeStat based on that
-      boolean reportedAge = false;
-      for (HealthStat participantStat : stats.values()) {
-        if (participantStat != null && !reportedAge) {
-          // generate and report stats for how old this node's report is
-          modTime = participantStat.getLastModifiedTimeStamp();
-          reportAgeStat(instance, modTime, currTime);
-          reportedAge = true;
-        }
-        // System.out.println(modTime);
-        // XXX: need to convert participantStat to a better format
-        // need to get instanceName in here
-
-        if (participantStat != null) {
-          // String timestamp = String.valueOf(instance.getModifiedTime()); WANT
-          // REPORT LEVEL TS
-          Map<String, Map<String, String>> statMap = participantStat.getHealthFields(instanceName);
-          for (String key : statMap.keySet()) {
-            _statsHolder.applyStat(key, statMap.get(key));
-          }
-        }
-      }
-    }
-    // Call _statsHolder.persistStats() once per pipeline. This will
-    // write the updated persisted stats into zookeeper
-    _statsHolder.persistStats();
-    logger.info("Done processing stats: " + (System.currentTimeMillis() - readInstancesStart));
-    // populate _statStatus
-    _statStatus = _statsHolder.getStatsMap();
-
-    for (String statKey : _statStatus.keySet()) {
-      logger.debug("Stat key, value: " + statKey + ": " + _statStatus.get(statKey));
-    }
-
-    long alertExecuteStartTime = System.currentTimeMillis();
-    // execute alerts, populate _alertStatus
-    _alertStatus =
-        AlertProcessor.executeAllAlerts(_alertsHolder.getAlertList(), _statsHolder.getStatsList());
-    logger.info("done executing alerts: " + (System.currentTimeMillis() - alertExecuteStartTime));
-    for (String originAlertName : _alertStatus.keySet()) {
-      _alertBeanCollection.setAlerts(originAlertName, _alertStatus.get(originAlertName),
-          manager.getClusterName());
-    }
-
-    executeAlertActions(manager);
-    // Write alert fire history to zookeeper
-    updateAlertHistory(manager);
-    long writeAlertStartTime = System.currentTimeMillis();
-    // write out alert status (to zk)
-    _alertsHolder.addAlertStatusSet(_alertStatus);
-    logger.info("done writing alerts: " + (System.currentTimeMillis() - writeAlertStartTime));
-
-    // TODO: access the 2 status variables from somewhere to populate graphs
-
-    long logAlertStartTime = System.currentTimeMillis();
-    // logging alert status
-    for (String alertOuterKey : _alertStatus.keySet()) {
-      logger.debug("Alert Outer Key: " + alertOuterKey);
-      Map<String, AlertValueAndStatus> alertInnerMap = _alertStatus.get(alertOuterKey);
-      if (alertInnerMap == null) {
-        logger.debug(alertOuterKey + " has no alerts to report.");
-        continue;
-      }
-      for (String alertInnerKey : alertInnerMap.keySet()) {
-        logger.debug("  " + alertInnerKey + " value: "
-            + alertInnerMap.get(alertInnerKey).getValue() + ", status: "
-            + alertInnerMap.get(alertInnerKey).isFired());
-      }
-    }
-
-    logger.info("done logging alerts: " + (System.currentTimeMillis() - logAlertStartTime));
-
-    long processLatency = System.currentTimeMillis() - startTime;
-    addLatencyToMonitor(event, processLatency);
-    logger.info("process end: " + processLatency);
-  }
-
-  /**
-   * Go through the _alertStatus, and call executeAlertAction for those actual alerts that
-   * has been fired
-   */
-
-  void executeAlertActions(HelixManager manager) {
-    _alertActionTaken.clear();
-    // Go through the original alert strings
-    for (String originAlertName : _alertStatus.keySet()) {
-      Map<String, String> alertFields = _alertsHolder.getAlertsMap().get(originAlertName);
-      if (alertFields != null && alertFields.containsKey(AlertParser.ACTION_NAME)) {
-        String actionValue = alertFields.get(AlertParser.ACTION_NAME);
-        Map<String, AlertValueAndStatus> alertResultMap = _alertStatus.get(originAlertName);
-        if (alertResultMap == null) {
-          logger.info("Alert " + originAlertName + " does not have alert status map");
-          continue;
-        }
-        // For each original alert, iterate all actual alerts that it expands into
-        for (String actualStatName : alertResultMap.keySet()) {
-          // if the actual alert is fired, execute the action
-          if (alertResultMap.get(actualStatName).isFired()) {
-            logger.warn("Alert " + originAlertName + " action " + actionValue + " is triggered by "
-                + actualStatName);
-            _alertActionTaken.put(actualStatName, actionValue);
-            // move functionalities into a seperate class
-            executeAlertAction(actualStatName, actionValue, manager);
-          }
-        }
-      }
-    }
-  }
-
-  /**
-   * Execute the action if an alert is fired, and the alert has an action associated with it.
-   * NOTE: consider unify this with DefaultParticipantErrorMessageHandler.handleMessage()
-   */
-  void executeAlertAction(String actualStatName, String actionValue, HelixManager manager) {
-    if (actionValue.equals(ActionOnError.DISABLE_INSTANCE.toString())) {
-      String instanceName = parseInstanceName(actualStatName, manager);
-      if (instanceName != null) {
-        logger.info("Disabling instance " + instanceName);
-        manager.getClusterManagmentTool().enableInstance(manager.getClusterName(), instanceName,
-            false);
-      }
-    } else if (actionValue.equals(ActionOnError.DISABLE_PARTITION.toString())) {
-      String instanceName = parseInstanceName(actualStatName, manager);
-      String resourceName = parseResourceName(actualStatName, manager);
-      String partitionName = parsePartitionName(actualStatName, manager);
-      if (instanceName != null && resourceName != null && partitionName != null) {
-        logger.info("Disabling partition " + partitionName + " instanceName " + instanceName);
-        manager.getClusterManagmentTool().enablePartition(false, manager.getClusterName(),
-            instanceName, resourceName, Arrays.asList(partitionName));
-      }
-    } else if (actionValue.equals(ActionOnError.DISABLE_RESOURCE.toString())) {
-      String instanceName = parseInstanceName(actualStatName, manager);
-      String resourceName = parseResourceName(actualStatName, manager);
-      logger.info("Disabling resource " + resourceName + " instanceName " + instanceName
-          + " not implemented");
-
-    }
-  }
-
-  public static String parseResourceName(String actualStatName, HelixManager manager) {
-    HelixDataAccessor accessor = manager.getHelixDataAccessor();
-    Builder kb = accessor.keyBuilder();
-    List<IdealState> idealStates = accessor.getChildValues(kb.idealStates());
-    for (IdealState idealState : idealStates) {
-      String resourceName = idealState.getResourceId().stringify();
-      if (actualStatName.contains("=" + resourceName + ".")
-          || actualStatName.contains("=" + resourceName + ";")) {
-        return resourceName;
-      }
-    }
-    return null;
-  }
-
-  public static String parsePartitionName(String actualStatName, HelixManager manager) {
-    String resourceName = parseResourceName(actualStatName, manager);
-    if (resourceName != null) {
-      String partitionKey = "=" + resourceName + "_";
-      if (actualStatName.contains(partitionKey)) {
-        int pos = actualStatName.indexOf(partitionKey);
-        int nextDotPos = actualStatName.indexOf('.', pos + partitionKey.length());
-        int nextCommaPos = actualStatName.indexOf(';', pos + partitionKey.length());
-        if (nextCommaPos > 0 && nextCommaPos < nextDotPos) {
-          nextDotPos = nextCommaPos;
-        }
-
-        String partitionName = actualStatName.substring(pos + 1, nextDotPos);
-        return partitionName;
-      }
-    }
-    return null;
-  }
-
-  public static String parseInstanceName(String actualStatName, HelixManager manager) {
-    HelixDataAccessor accessor = manager.getHelixDataAccessor();
-    Builder kb = accessor.keyBuilder();
-    List<LiveInstance> liveInstances = accessor.getChildValues(kb.liveInstances());
-    for (LiveInstance instance : liveInstances) {
-      String instanceName = instance.getInstanceName();
-      if (actualStatName.startsWith(instanceName)) {
-        return instanceName;
-      }
-    }
-    return null;
-  }
-
-  void updateAlertHistory(HelixManager manager) {
-    // Write alert fire history to zookeeper
-    _alertBeanCollection.refreshAlertDelta(manager.getClusterName());
-    Map<String, String> delta = _alertBeanCollection.getRecentAlertDelta();
-    // Update history only when some beans has changed
-    if (delta.size() > 0) {
-      delta.putAll(_alertActionTaken);
-      SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd-hh:mm:ss:SSS");
-      String date = dateFormat.format(new Date());
-
-      HelixDataAccessor accessor = manager.getHelixDataAccessor();
-      Builder keyBuilder = accessor.keyBuilder();
-
-      HelixProperty property = accessor.getProperty(keyBuilder.alertHistory());
-      ZNRecord alertFiredHistory;
-      if (property == null) {
-        alertFiredHistory = new ZNRecord(PropertyType.ALERT_HISTORY.toString());
-      } else {
-        alertFiredHistory = property.getRecord();
-      }
-      while (alertFiredHistory.getMapFields().size() >= ALERT_HISTORY_SIZE) {
-        // ZNRecord uses TreeMap which is sorted ascending internally
-        String firstKey = (String) (alertFiredHistory.getMapFields().keySet().toArray()[0]);
-        alertFiredHistory.getMapFields().remove(firstKey);
-      }
-      alertFiredHistory.setMapField(date, delta);
-      // manager.getDataAccessor().setProperty(PropertyType.ALERT_HISTORY, alertFiredHistory);
-      accessor.setProperty(keyBuilder.alertHistory(), new AlertHistory(alertFiredHistory));
-      _alertBeanCollection.setAlertHistory(alertFiredHistory);
-    }
-  }
-
-  public ClusterAlertMBeanCollection getClusterAlertMBeanCollection() {
-    return _alertBeanCollection;
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/healthcheck/AccumulateAggregationType.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/AccumulateAggregationType.java b/helix-core/src/main/java/org/apache/helix/healthcheck/AccumulateAggregationType.java
deleted file mode 100644
index a3c443f..0000000
--- a/helix-core/src/main/java/org/apache/helix/healthcheck/AccumulateAggregationType.java
+++ /dev/null
@@ -1,41 +0,0 @@
-package org.apache.helix.healthcheck;
-
-/*
- * 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 org.apache.log4j.Logger;
-
-public class AccumulateAggregationType implements AggregationType {
-
-  private static final Logger logger = Logger.getLogger(AccumulateAggregationType.class);
-
-  public final static String TYPE_NAME = "accumulate";
-
-  @Override
-  public String getName() {
-    return TYPE_NAME;
-  }
-
-  @Override
-  public String merge(String iv, String ev, long prevTimestamp) {
-    double inVal = Double.parseDouble(iv);
-    double existingVal = Double.parseDouble(ev);
-    return String.valueOf(inVal + existingVal);
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/healthcheck/AggregationType.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/AggregationType.java b/helix-core/src/main/java/org/apache/helix/healthcheck/AggregationType.java
deleted file mode 100644
index 29f5921..0000000
--- a/helix-core/src/main/java/org/apache/helix/healthcheck/AggregationType.java
+++ /dev/null
@@ -1,31 +0,0 @@
-package org.apache.helix.healthcheck;
-
-/*
- * 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.
- */
-
-public interface AggregationType {
-
-  // public abstract <T extends Object> T merge(T iv, T ev);
-
-  public final static String DELIM = "#";
-
-  public abstract String merge(String incomingVal, String existingVal, long prevTimestamp);
-
-  public abstract String getName();
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/healthcheck/AggregationTypeFactory.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/AggregationTypeFactory.java b/helix-core/src/main/java/org/apache/helix/healthcheck/AggregationTypeFactory.java
deleted file mode 100644
index d946641..0000000
--- a/helix-core/src/main/java/org/apache/helix/healthcheck/AggregationTypeFactory.java
+++ /dev/null
@@ -1,69 +0,0 @@
-package org.apache.helix.healthcheck;
-
-/*
- * 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.StringTokenizer;
-
-import org.apache.log4j.Logger;
-
-public class AggregationTypeFactory {
-  private static final Logger logger = Logger.getLogger(AggregationTypeFactory.class);
-
-  public AggregationTypeFactory() {
-  }
-
-  // TODO: modify this function so that it takes a single string, but can parse
-  // apart params from type
-  public static AggregationType getAggregationType(String input) {
-    if (input == null) {
-      logger.error("AggregationType name is null");
-      return null;
-    }
-    StringTokenizer tok = new StringTokenizer(input, AggregationType.DELIM);
-    String type = tok.nextToken();
-    int numParams = tok.countTokens();
-    String[] params = new String[numParams];
-    for (int i = 0; i < numParams; i++) {
-      if (!tok.hasMoreTokens()) {
-        logger.error("Trying to parse non-existent params");
-        return null;
-      }
-      params[i] = tok.nextToken();
-    }
-
-    if (type.equals(AccumulateAggregationType.TYPE_NAME)) {
-      return new AccumulateAggregationType();
-    } else if (type.equals(DecayAggregationType.TYPE_NAME)) {
-      if (params.length < 1) {
-        logger.error("DecayAggregationType must contain <decay weight> parameter");
-        return null;
-      }
-      return new DecayAggregationType(Double.parseDouble(params[0]));
-    } else if (type.equals(WindowAggregationType.TYPE_NAME)) {
-      if (params.length < 1) {
-        logger.error("WindowAggregationType must contain <window size> parameter");
-      }
-      return new WindowAggregationType(Integer.parseInt(params[0]));
-    } else {
-      logger.error("Unknown AggregationType " + type);
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/healthcheck/DecayAggregationType.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/DecayAggregationType.java b/helix-core/src/main/java/org/apache/helix/healthcheck/DecayAggregationType.java
deleted file mode 100644
index 2409b84..0000000
--- a/helix-core/src/main/java/org/apache/helix/healthcheck/DecayAggregationType.java
+++ /dev/null
@@ -1,57 +0,0 @@
-package org.apache.helix.healthcheck;
-
-/*
- * 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 org.apache.log4j.Logger;
-
-public class DecayAggregationType implements AggregationType {
-
-  private static final Logger logger = Logger.getLogger(DecayAggregationType.class);
-
-  public final static String TYPE_NAME = "decay";
-
-  double _decayFactor = 0.1;
-
-  public DecayAggregationType(double df) {
-    super();
-    _decayFactor = df;
-  }
-
-  @Override
-  public String getName() {
-    StringBuilder sb = new StringBuilder();
-    sb.append(TYPE_NAME);
-    sb.append(DELIM);
-    sb.append(_decayFactor);
-    return sb.toString();
-  }
-
-  @Override
-  public String merge(String iv, String ev, long prevTimestamp) {
-    double incomingVal = Double.parseDouble(iv);
-    double existingVal = Double.parseDouble(ev);
-    long currTimestamp = System.currentTimeMillis();
-    double minutesOld = (currTimestamp - prevTimestamp) / 60000.0;
-    // come up with decay coeff for old value. More time passed, the more it
-    // decays
-    double oldDecayCoeff = Math.pow((1 - _decayFactor), minutesOld);
-    return String.valueOf((oldDecayCoeff * existingVal + (1 - oldDecayCoeff) * incomingVal));
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/healthcheck/DefaultHealthReportProvider.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/DefaultHealthReportProvider.java b/helix-core/src/main/java/org/apache/helix/healthcheck/DefaultHealthReportProvider.java
deleted file mode 100644
index 619667c..0000000
--- a/helix-core/src/main/java/org/apache/helix/healthcheck/DefaultHealthReportProvider.java
+++ /dev/null
@@ -1,86 +0,0 @@
-package org.apache.helix.healthcheck;
-
-/*
- * 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.lang.management.ManagementFactory;
-import java.lang.management.OperatingSystemMXBean;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.TreeMap;
-
-import org.apache.log4j.Logger;
-
-class DefaultHealthReportProvider extends HealthReportProvider {
-  private static final Logger _logger = Logger.getLogger(DefaultHealthReportProvider.class);
-
-  public final static String _availableCPUs = "availableCPUs";
-  public final static String _freePhysicalMemory = "freePhysicalMemory";
-  public final static String _totalJvmMemory = "totalJvmMemory";
-  public final static String _freeJvmMemory = "freeJvmMemory";
-  public final static String _averageSystemLoad = "averageSystemLoad";
-
-  public DefaultHealthReportProvider() {
-  }
-
-  @Override
-  public Map<String, String> getRecentHealthReport() {
-    OperatingSystemMXBean osMxBean = ManagementFactory.getOperatingSystemMXBean();
-    long freeJvmMemory = Runtime.getRuntime().freeMemory();
-    long totalJvmMemory = Runtime.getRuntime().totalMemory();
-    int availableCPUs = osMxBean.getAvailableProcessors();
-    double avgSystemLoad = osMxBean.getSystemLoadAverage();
-    long freePhysicalMemory = Long.MAX_VALUE;
-
-    try {
-      // if( osMxBean instanceof com.sun.management.OperatingSystemMXBean)
-      // {
-      // com.sun.management.OperatingSystemMXBean sunOsMxBean
-      // = (com.sun.management.OperatingSystemMXBean) osMxBean;
-      // freePhysicalMemory = sunOsMxBean.getFreePhysicalMemorySize();
-      // }
-    } catch (Throwable t) {
-      _logger.error(t);
-    }
-
-    Map<String, String> result = new TreeMap<String, String>();
-
-    result.put(_availableCPUs, "" + availableCPUs);
-    result.put(_freePhysicalMemory, "" + freePhysicalMemory);
-    result.put(_freeJvmMemory, "" + freeJvmMemory);
-    result.put(_totalJvmMemory, "" + totalJvmMemory);
-    result.put(_averageSystemLoad, "" + avgSystemLoad);
-
-    return result;
-  }
-
-  @Override
-  public Map<String, Map<String, String>> getRecentPartitionHealthReport() {
-    Map<String, Map<String, String>> result = new HashMap<String, Map<String, String>>();
-
-    result.put(getReportName(), getRecentHealthReport());
-    return result;
-  }
-
-  @Override
-  public void resetStats() {
-    // TODO Auto-generated method stub
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/healthcheck/DefaultPerfCounters.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/DefaultPerfCounters.java b/helix-core/src/main/java/org/apache/helix/healthcheck/DefaultPerfCounters.java
deleted file mode 100644
index e1afd5c..0000000
--- a/helix-core/src/main/java/org/apache/helix/healthcheck/DefaultPerfCounters.java
+++ /dev/null
@@ -1,95 +0,0 @@
-package org.apache.helix.healthcheck;
-
-/*
- * 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 org.apache.helix.ZNRecord;
-import org.apache.log4j.Logger;
-
-@Deprecated
-public class DefaultPerfCounters extends ZNRecord {
-  private static final Logger _logger = Logger.getLogger(DefaultPerfCounters.class);
-
-  public final static String _availableCPUs = "availableCPUs";
-  public final static String _freePhysicalMemory = "freePhysicalMemory";
-  public final static String _totalJvmMemory = "totalJvmMemory";
-  public final static String _freeJvmMemory = "freeJvmMemory";
-  public final static String _averageSystemLoad = "averageSystemLoad";
-
-  public DefaultPerfCounters(String instanceName, long availableCPUs, long freePhysicalMemory,
-      long freeJvmMemory, long totalJvmMemory, double averageSystemLoad) {
-    super("DefaultPerfCounters");
-    setSimpleField("instanceName", instanceName);
-    setSimpleField("createTime", new Date().toString());
-
-    setSimpleField(_availableCPUs, "" + availableCPUs);
-    setSimpleField(_freePhysicalMemory, "" + freePhysicalMemory);
-    setSimpleField(_freeJvmMemory, "" + freeJvmMemory);
-    setSimpleField(_totalJvmMemory, "" + totalJvmMemory);
-    setSimpleField(_averageSystemLoad, "" + averageSystemLoad);
-  }
-
-  public long getAvailableCpus() {
-    return getSimpleLongVal(_availableCPUs);
-  }
-
-  public double getAverageSystemLoad() {
-    return getSimpleDoubleVal(_averageSystemLoad);
-  }
-
-  public long getTotalJvmMemory() {
-    return getSimpleLongVal(_totalJvmMemory);
-  }
-
-  public long getFreeJvmMemory() {
-    return getSimpleLongVal(_freeJvmMemory);
-  }
-
-  public long getFreePhysicalMemory() {
-    return getSimpleLongVal(_freePhysicalMemory);
-  }
-
-  long getSimpleLongVal(String key) {
-    String strVal = getSimpleField(key);
-    if (strVal == null) {
-      return 0;
-    }
-    try {
-      return Long.parseLong(strVal);
-    } catch (Exception e) {
-      _logger.warn(e);
-      return 0;
-    }
-  }
-
-  double getSimpleDoubleVal(String key) {
-    String strVal = getSimpleField(key);
-    if (strVal == null) {
-      return 0;
-    }
-    try {
-      return Double.parseDouble(strVal);
-    } catch (Exception e) {
-      _logger.warn(e);
-      return 0;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/healthcheck/HealthReportProvider.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/HealthReportProvider.java b/helix-core/src/main/java/org/apache/helix/healthcheck/HealthReportProvider.java
deleted file mode 100644
index f7afd04..0000000
--- a/helix-core/src/main/java/org/apache/helix/healthcheck/HealthReportProvider.java
+++ /dev/null
@@ -1,39 +0,0 @@
-package org.apache.helix.healthcheck;
-
-/*
- * 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.Map;
-
-public abstract class HealthReportProvider {
-  public static final String _defaultPerfCounters = "defaultPerfCounters";
-
-  public abstract Map<String, String> getRecentHealthReport();
-
-  public Map<String, Map<String, String>> getRecentPartitionHealthReport() {
-    return null;
-  }
-
-  public abstract void resetStats();
-
-  public String getReportName() {
-    return _defaultPerfCounters;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/healthcheck/HealthStatsAggregationTask.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/HealthStatsAggregationTask.java b/helix-core/src/main/java/org/apache/helix/healthcheck/HealthStatsAggregationTask.java
deleted file mode 100644
index 05ffaef..0000000
--- a/helix-core/src/main/java/org/apache/helix/healthcheck/HealthStatsAggregationTask.java
+++ /dev/null
@@ -1,89 +0,0 @@
-package org.apache.helix.healthcheck;
-
-/*
- * 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.Random;
-import java.util.Timer;
-import java.util.TimerTask;
-
-import org.apache.helix.HelixTimerTask;
-import org.apache.log4j.Logger;
-
-public class HealthStatsAggregationTask extends HelixTimerTask {
-  private static final Logger LOG = Logger.getLogger(HealthStatsAggregationTask.class);
-  public final static int DEFAULT_HEALTH_CHECK_LATENCY = 30 * 1000;
-
-  final HealthStatsAggregator _healthStatsAggregator;
-
-  class HealthStatsAggregationTaskTimer extends TimerTask {
-
-    @Override
-    public void run() {
-      _healthStatsAggregator.aggregate();
-    }
-
-  }
-
-  private Timer _timer;
-  private final int _delay;
-  private final int _period;
-
-  public HealthStatsAggregationTask(HealthStatsAggregator healthStatsAggregator, int delay,
-      int period) {
-    _healthStatsAggregator = healthStatsAggregator;
-
-    _delay = delay;
-    _period = period;
-  }
-
-  public HealthStatsAggregationTask(HealthStatsAggregator healthStatsAggregator) {
-    this(healthStatsAggregator, DEFAULT_HEALTH_CHECK_LATENCY, DEFAULT_HEALTH_CHECK_LATENCY);
-  }
-
-  @Override
-  public void start() {
-
-    if (_timer == null) {
-      LOG.info("START HealthStatsAggregationTimerTask");
-
-      // Remove all the previous health check values, if any
-      _healthStatsAggregator.init();
-
-      _timer = new Timer("HealthStatsAggregationTimerTask", true);
-      _timer.scheduleAtFixedRate(new HealthStatsAggregationTaskTimer(),
-          new Random().nextInt(_delay), _period);
-    } else {
-      LOG.warn("HealthStatsAggregationTimerTask already started");
-    }
-  }
-
-  @Override
-  public synchronized void stop() {
-    if (_timer != null) {
-      LOG.info("Stop HealthStatsAggregationTimerTask");
-      _timer.cancel();
-      _healthStatsAggregator.reset();
-      _timer = null;
-    } else {
-      LOG.warn("HealthStatsAggregationTimerTask already stopped");
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/healthcheck/HealthStatsAggregator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/HealthStatsAggregator.java b/helix-core/src/main/java/org/apache/helix/healthcheck/HealthStatsAggregator.java
deleted file mode 100644
index bc95e6d..0000000
--- a/helix-core/src/main/java/org/apache/helix/healthcheck/HealthStatsAggregator.java
+++ /dev/null
@@ -1,141 +0,0 @@
-package org.apache.helix.healthcheck;
-
-/*
- * 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.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.helix.ConfigAccessor;
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixManager;
-import org.apache.helix.controller.pipeline.Pipeline;
-import org.apache.helix.controller.pipeline.Stage;
-import org.apache.helix.controller.stages.ClusterEvent;
-import org.apache.helix.controller.stages.ReadHealthDataStage;
-import org.apache.helix.controller.stages.StatsAggregationStage;
-import org.apache.helix.model.ConfigScope;
-import org.apache.helix.model.builder.ConfigScopeBuilder;
-import org.apache.helix.monitoring.mbeans.ClusterAlertMBeanCollection;
-import org.apache.helix.monitoring.mbeans.HelixStageLatencyMonitor;
-import org.apache.log4j.Logger;
-
-public class HealthStatsAggregator {
-  private static final Logger LOG = Logger.getLogger(HealthStatsAggregator.class);
-
-  public final static int DEFAULT_HEALTH_CHECK_LATENCY = 30 * 1000;
-
-  private final HelixManager _manager;
-  private final Pipeline _healthStatsAggregationPipeline;
-  private final ClusterAlertMBeanCollection _alertItemCollection;
-  private final Map<String, HelixStageLatencyMonitor> _stageLatencyMonitorMap =
-      new HashMap<String, HelixStageLatencyMonitor>();
-
-  public HealthStatsAggregator(HelixManager manager) {
-    _manager = manager;
-
-    // health stats pipeline
-    _healthStatsAggregationPipeline = new Pipeline();
-    _healthStatsAggregationPipeline.addStage(new ReadHealthDataStage());
-    StatsAggregationStage statAggregationStage = new StatsAggregationStage();
-    _healthStatsAggregationPipeline.addStage(statAggregationStage);
-    _alertItemCollection = statAggregationStage.getClusterAlertMBeanCollection();
-
-    registerStageLatencyMonitor(_healthStatsAggregationPipeline);
-  }
-
-  private void registerStageLatencyMonitor(Pipeline pipeline) {
-    for (Stage stage : pipeline.getStages()) {
-      String stgName = stage.getStageName();
-      if (!_stageLatencyMonitorMap.containsKey(stgName)) {
-        try {
-          _stageLatencyMonitorMap.put(stage.getStageName(),
-              new HelixStageLatencyMonitor(_manager.getClusterName(), stgName));
-        } catch (Exception e) {
-          LOG.error("Couldn't create StageLatencyMonitor mbean for stage: " + stgName, e);
-        }
-      } else {
-        LOG.error("StageLatencyMonitor for stage: " + stgName + " already exists. Skip register it");
-      }
-    }
-  }
-
-  public synchronized void aggregate() {
-    if (!isEnabled()) {
-      LOG.info("HealthAggregationTask is disabled.");
-      return;
-    }
-
-    if (!_manager.isLeader()) {
-      LOG.error("Cluster manager: " + _manager.getInstanceName()
-          + " is not leader. Pipeline will not be invoked");
-      return;
-    }
-
-    try {
-      ClusterEvent event = new ClusterEvent("healthChange");
-      event.addAttribute("helixmanager", _manager);
-      event.addAttribute("HelixStageLatencyMonitorMap", _stageLatencyMonitorMap);
-
-      _healthStatsAggregationPipeline.handle(event);
-      _healthStatsAggregationPipeline.finish();
-    } catch (Exception e) {
-      LOG.error("Exception while executing pipeline: " + _healthStatsAggregationPipeline, e);
-    }
-  }
-
-  private boolean isEnabled() {
-    ConfigAccessor configAccessor = _manager.getConfigAccessor();
-    boolean enabled = true;
-    if (configAccessor != null) {
-      // zk-based cluster manager
-      ConfigScope scope = new ConfigScopeBuilder().forCluster(_manager.getClusterName()).build();
-      String isEnabled = configAccessor.get(scope, "healthChange.enabled");
-      if (isEnabled != null) {
-        enabled = new Boolean(isEnabled);
-      }
-    } else {
-      LOG.debug("File-based cluster manager doesn't support disable healthChange");
-    }
-    return enabled;
-  }
-
-  public void init() {
-    // Remove all the previous health check values, if any
-    HelixDataAccessor accessor = _manager.getHelixDataAccessor();
-    List<String> existingHealthRecordNames =
-        accessor.getChildNames(accessor.keyBuilder().healthReports(_manager.getInstanceName()));
-    for (String healthReportName : existingHealthRecordNames) {
-      LOG.info("Removing old healthrecord " + healthReportName);
-      accessor.removeProperty(accessor.keyBuilder().healthReport(_manager.getInstanceName(),
-          healthReportName));
-    }
-
-  }
-
-  public void reset() {
-    _alertItemCollection.reset();
-
-    for (HelixStageLatencyMonitor stgLatencyMonitor : _stageLatencyMonitorMap.values()) {
-      stgLatencyMonitor.reset();
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportCollector.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportCollector.java b/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportCollector.java
deleted file mode 100644
index 266ed8b..0000000
--- a/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportCollector.java
+++ /dev/null
@@ -1,32 +0,0 @@
-package org.apache.helix.healthcheck;
-
-/*
- * 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 org.apache.helix.ZNRecord;
-
-public interface ParticipantHealthReportCollector {
-  public abstract void addHealthReportProvider(HealthReportProvider provider);
-
-  public abstract void removeHealthReportProvider(HealthReportProvider provider);
-
-  public abstract void reportHealthReportMessage(ZNRecord healthReport);
-
-  public abstract void transmitHealthReports();
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportCollectorImpl.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportCollectorImpl.java b/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportCollectorImpl.java
deleted file mode 100644
index 9023641..0000000
--- a/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportCollectorImpl.java
+++ /dev/null
@@ -1,115 +0,0 @@
-package org.apache.helix.healthcheck;
-
-/*
- * 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.LinkedList;
-import java.util.Map;
-
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixManager;
-import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.alerts.StatsHolder;
-import org.apache.helix.model.HealthStat;
-import org.apache.log4j.Logger;
-
-public class ParticipantHealthReportCollectorImpl implements ParticipantHealthReportCollector {
-  private final LinkedList<HealthReportProvider> _healthReportProviderList =
-      new LinkedList<HealthReportProvider>();
-  private static final Logger _logger = Logger
-      .getLogger(ParticipantHealthReportCollectorImpl.class);
-  private final HelixManager _helixManager;
-  String _instanceName;
-
-  public ParticipantHealthReportCollectorImpl(HelixManager helixManager, String instanceName) {
-    _helixManager = helixManager;
-    _instanceName = instanceName;
-    addDefaultHealthCheckInfoProvider();
-  }
-
-  private void addDefaultHealthCheckInfoProvider() {
-    addHealthReportProvider(new DefaultHealthReportProvider());
-  }
-
-  @Override
-  public void addHealthReportProvider(HealthReportProvider provider) {
-    try {
-      synchronized (_healthReportProviderList) {
-        if (!_healthReportProviderList.contains(provider)) {
-          _healthReportProviderList.add(provider);
-        } else {
-          _logger.warn("Skipping a duplicated HealthCheckInfoProvider");
-        }
-      }
-    } catch (Exception e) {
-      _logger.error(e);
-    }
-  }
-
-  @Override
-  public void removeHealthReportProvider(HealthReportProvider provider) {
-    synchronized (_healthReportProviderList) {
-      if (_healthReportProviderList.contains(provider)) {
-        _healthReportProviderList.remove(provider);
-      } else {
-        _logger.warn("Skip removing a non-exist HealthCheckInfoProvider");
-      }
-    }
-  }
-
-  @Override
-  public void reportHealthReportMessage(ZNRecord healthCheckInfoUpdate) {
-    HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-    accessor.setProperty(keyBuilder.healthReport(_instanceName, healthCheckInfoUpdate.getId()),
-        new HealthStat(healthCheckInfoUpdate));
-
-  }
-
-  @Override
-  public synchronized void transmitHealthReports() {
-    synchronized (_healthReportProviderList) {
-      for (HealthReportProvider provider : _healthReportProviderList) {
-        try {
-          Map<String, String> report = provider.getRecentHealthReport();
-          Map<String, Map<String, String>> partitionReport =
-              provider.getRecentPartitionHealthReport();
-          ZNRecord record = new ZNRecord(provider.getReportName());
-          if (report != null) {
-            record.setSimpleFields(report);
-          }
-          if (partitionReport != null) {
-            record.setMapFields(partitionReport);
-          }
-          record.setSimpleField(StatsHolder.TIMESTAMP_NAME, "" + System.currentTimeMillis());
-
-          HelixDataAccessor accessor = _helixManager.getHelixDataAccessor();
-          Builder keyBuilder = accessor.keyBuilder();
-          accessor.setProperty(keyBuilder.healthReport(_instanceName, record.getId()),
-              new HealthStat(record));
-
-          provider.resetStats();
-        } catch (Exception e) {
-          _logger.error("fail to transmit health report", e);
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportTask.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportTask.java b/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportTask.java
deleted file mode 100644
index 59d74c7..0000000
--- a/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportTask.java
+++ /dev/null
@@ -1,71 +0,0 @@
-package org.apache.helix.healthcheck;
-
-/*
- * 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.Random;
-import java.util.Timer;
-import java.util.TimerTask;
-
-import org.apache.helix.HelixTimerTask;
-import org.apache.log4j.Logger;
-
-public class ParticipantHealthReportTask extends HelixTimerTask {
-  private static final Logger LOG = Logger.getLogger(ParticipantHealthReportTask.class);
-  public final static int DEFAULT_REPORT_LATENCY = 60 * 1000;
-
-  Timer _timer;
-  final ParticipantHealthReportCollectorImpl _healthReportCollector;
-
-  class ParticipantHealthReportTimerTask extends TimerTask {
-
-    @Override
-    public void run() {
-      _healthReportCollector.transmitHealthReports();
-    }
-  }
-
-  public ParticipantHealthReportTask(ParticipantHealthReportCollectorImpl healthReportCollector) {
-    _healthReportCollector = healthReportCollector;
-  }
-
-  @Override
-  public void start() {
-    if (_timer == null) {
-      LOG.info("Start HealthCheckInfoReportingTask");
-      _timer = new Timer("ParticipantHealthReportTimerTask", true);
-      _timer.scheduleAtFixedRate(new ParticipantHealthReportTimerTask(),
-          new Random().nextInt(DEFAULT_REPORT_LATENCY), DEFAULT_REPORT_LATENCY);
-    } else {
-      LOG.warn("ParticipantHealthReportTimerTask already started");
-    }
-  }
-
-  @Override
-  public void stop() {
-    if (_timer != null) {
-      LOG.info("Stop ParticipantHealthReportTimerTask");
-      _timer.cancel();
-      _timer = null;
-    } else {
-      LOG.warn("ParticipantHealthReportTimerTask already stopped");
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/healthcheck/PerformanceHealthReportProvider.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/PerformanceHealthReportProvider.java b/helix-core/src/main/java/org/apache/helix/healthcheck/PerformanceHealthReportProvider.java
deleted file mode 100644
index 6bc33d3..0000000
--- a/helix-core/src/main/java/org/apache/helix/healthcheck/PerformanceHealthReportProvider.java
+++ /dev/null
@@ -1,138 +0,0 @@
-package org.apache.helix.healthcheck;
-
-/*
- * 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.HashMap;
-import java.util.Map;
-import java.util.TreeMap;
-
-import org.apache.log4j.Logger;
-
-public class PerformanceHealthReportProvider extends HealthReportProvider {
-
-  private static final Logger _logger = Logger.getLogger(PerformanceHealthReportProvider.class);
-
-  public final static String _testStat = "testStat";
-  public final static String _readLatencyStat = "readLatencyStat";
-  public final static String _requestCountStat = "requestCountStat";
-  public final static String _partitionRequestCountStat = "partitionRequestCountStat";
-
-  public static final String _performanceCounters = "performanceCounters";
-
-  public int readLatencyCount = 0;
-  public double readLatencySum = 0;
-
-  public int requestCount = 0;
-
-  // private final Map<String, String> _partitionCountsMap = new HashMap<String,
-  // String>();
-
-  private final Map<String, HashMap<String, String>> _partitionStatMaps =
-      new HashMap<String, HashMap<String, String>>();
-
-  public PerformanceHealthReportProvider() {
-  }
-
-  @Override
-  public Map<String, String> getRecentHealthReport() {
-    long testStat = 10;
-
-    Map<String, String> result = new TreeMap<String, String>();
-
-    result.put(_testStat, "" + testStat);
-    result.put(_readLatencyStat, "" + readLatencySum / readLatencyCount);
-    result.put(_requestCountStat, "" + requestCount);
-
-    return result;
-  }
-
-  @Override
-  public Map<String, Map<String, String>> getRecentPartitionHealthReport() {
-    Map<String, Map<String, String>> result = new TreeMap<String, Map<String, String>>();
-    for (String statName : _partitionStatMaps.keySet()) {
-      result.put(statName, _partitionStatMaps.get(statName));
-    }
-    return result;
-  }
-
-  HashMap<String, String> getStatMap(String statName, boolean createIfMissing) {
-    // check if map for this stat exists. if not, create it
-    HashMap<String, String> statMap;
-    if (!_partitionStatMaps.containsKey(statName)) {
-      if (!createIfMissing) {
-        return null;
-      }
-      statMap = new HashMap<String, String>();
-      _partitionStatMaps.put(statName, statMap);
-    } else {
-      statMap = _partitionStatMaps.get(statName);
-    }
-    return statMap;
-  }
-
-  // TODO:
-  // Currently participant is source of truth and updates ZK. We want ZK to be
-  // source of truth.
-  // Revise this approach the participant sends deltas of stats to controller
-  // (ZK?) and have controller do aggregation
-  // and update ZK. Make sure to wipe the participant between uploads.
-  String getPartitionStat(HashMap<String, String> partitionMap, String partitionName) {
-    return partitionMap.get(partitionName);
-  }
-
-  void setPartitionStat(HashMap<String, String> partitionMap, String partitionName, String value) {
-    partitionMap.put(partitionName, value);
-  }
-
-  public void incrementPartitionStat(String statName, String partitionName) {
-    HashMap<String, String> statMap = getStatMap(statName, true);
-    String currValStr = getPartitionStat(statMap, partitionName);
-    double currVal;
-    if (currValStr == null) {
-      currVal = 1.0;
-    } else {
-      currVal = Double.parseDouble(getPartitionStat(statMap, partitionName));
-      currVal++;
-    }
-    setPartitionStat(statMap, partitionName, String.valueOf(currVal));
-  }
-
-  public void submitPartitionStat(String statName, String partitionName, String value) {
-    HashMap<String, String> statMap = getStatMap(statName, true);
-    setPartitionStat(statMap, partitionName, value);
-  }
-
-  public String getPartitionStat(String statName, String partitionName) {
-    HashMap<String, String> statMap = getStatMap(statName, false);
-    if (statMap == null) {
-      return null;
-    } else {
-      return statMap.get(partitionName);
-    }
-  }
-
-  public void resetStats() {
-    _partitionStatMaps.clear();
-  }
-
-  public String getReportName() {
-    return _performanceCounters;
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/healthcheck/Stat.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/Stat.java b/helix-core/src/main/java/org/apache/helix/healthcheck/Stat.java
deleted file mode 100644
index e675792..0000000
--- a/helix-core/src/main/java/org/apache/helix/healthcheck/Stat.java
+++ /dev/null
@@ -1,125 +0,0 @@
-package org.apache.helix.healthcheck;
-
-/*
- * 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.Map;
-
-import org.apache.log4j.Logger;
-
-public class Stat {
-
-  private static final Logger _logger = Logger.getLogger(Stat.class);
-
-  public final static String OP_TYPE = "HTTP_OP";
-  public final static String MEASUREMENT_TYPE = "MEASUREMENT";
-  public final static String RESOURCE_NAME = "RESOURCE_NAME";
-  public final static String PARTITION_NAME = "PARTITION_NAME";
-  public final static String NODE_NAME = "NODE_NAME";
-  public final static String TIMESTAMP = "TIMESTAMP";
-  public final static String RETURN_STATUS = "RETURN_STATUS";
-  public final static String METRIC_NAME = "METRIC_NAME";
-  public final static String AGG_TYPE = "AGG_TYPE";
-
-  public String _opType;
-  public String _measurementType;
-  public String _resourceName;
-  public String _partitionName;
-  public String _nodeName;
-  public String _returnStatus;
-  public String _metricName;
-  public String _aggTypeName;
-  public String _timestamp;
-
-  public Stat(String opType, String measurementType, String resourceName, String partitionName,
-      String nodeName) {
-    // this(opType, measurementType, resourceName, partitionName, nodeName,
-    // null, null, null);
-    this(opType, measurementType, resourceName, partitionName, nodeName, null, null, null);
-  }
-
-  public Stat(String opType, String measurementType, String resourceName, String partitionName,
-      String nodeName, String returnStatus, String metricName, AggregationType aggType) {
-    this._opType = opType;
-    this._measurementType = measurementType;
-    this._resourceName = resourceName;
-    this._partitionName = partitionName;
-    this._nodeName = nodeName;
-    this._returnStatus = returnStatus;
-    this._metricName = metricName;
-    this._aggTypeName = null;
-    if (aggType != null) {
-      this._aggTypeName = aggType.getName();
-    }
-
-    _timestamp = String.valueOf(System.currentTimeMillis());
-  }
-
-  public Stat(Map<String, String> in) {
-    _opType = in.get(OP_TYPE);
-    _measurementType = in.get(MEASUREMENT_TYPE);
-    _resourceName = in.get(RESOURCE_NAME);
-    _partitionName = in.get(PARTITION_NAME);
-    _nodeName = in.get(NODE_NAME);
-    _timestamp = String.valueOf(System.currentTimeMillis());
-  }
-
-  public void setAggType(AggregationType aggType) {
-    this._aggTypeName = aggType.getName();
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (!(obj instanceof Stat)) {
-      return false;
-    }
-    Stat other = (Stat) obj;
-    if (!_partitionName.equals(other._partitionName)) {
-      return false;
-    }
-    if (!_opType.equals(other._opType)) {
-      return false;
-    }
-    if (!_measurementType.equals(other._measurementType)) {
-      return false;
-    }
-    if (!_resourceName.equals(other._resourceName)) {
-      return false;
-    }
-    if (!_nodeName.equals(other._nodeName)) {
-      return false;
-    }
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    return (_partitionName + _opType + _measurementType + _resourceName + _nodeName).hashCode();
-  }
-
-  public void addAlert(long value) {
-    // TODO Auto-generated method stub
-
-  }
-
-  public String toString() {
-    return _nodeName + "." + _resourceName + "." + _partitionName + "." + _opType + "."
-        + _measurementType + "." + _returnStatus + "." + _metricName + "." + _aggTypeName;
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/healthcheck/StatHealthReportProvider.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/StatHealthReportProvider.java b/helix-core/src/main/java/org/apache/helix/healthcheck/StatHealthReportProvider.java
deleted file mode 100644
index 82b2d31..0000000
--- a/helix-core/src/main/java/org/apache/helix/healthcheck/StatHealthReportProvider.java
+++ /dev/null
@@ -1,159 +0,0 @@
-package org.apache.helix.healthcheck;
-
-/*
- * 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.HashMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.log4j.Logger;
-
-public class StatHealthReportProvider extends HealthReportProvider {
-
-  private static final Logger _logger = Logger.getLogger(StatHealthReportProvider.class);
-
-  /*
-   * public final static String _testStat = "testStat"; public final static
-   * String _readLatencyStat = "readLatencyStat"; public final static String
-   * _requestCountStat = "requestCountStat"; public final static String
-   * _partitionRequestCountStat = "partitionRequestCountStat";
-   */
-
-  public static final String REPORT_NAME = "ParticipantStats";
-  public String _reportName = REPORT_NAME;
-
-  public static final String STAT_VALUE = "value";
-  public static final String TIMESTAMP = "timestamp";
-
-  public int readLatencyCount = 0;
-  public double readLatencySum = 0;
-
-  public int requestCount = 0;
-
-  // private final Map<String, String> _partitionCountsMap = new HashMap<String,
-  // String>();
-
-  // private final Map<String, HashMap<String,String>> _partitionStatMaps = new
-  // HashMap<String, HashMap<String,String>>();
-  private final ConcurrentHashMap<String, String> _statsToValues =
-      new ConcurrentHashMap<String, String>();
-  private final ConcurrentHashMap<String, String> _statsToTimestamps =
-      new ConcurrentHashMap<String, String>();
-
-  public StatHealthReportProvider() {
-  }
-
-  @Override
-  public Map<String, String> getRecentHealthReport() {
-    return null;
-  }
-
-  // TODO: function is misnamed, but return type is what I want
-  @Override
-  public Map<String, Map<String, String>> getRecentPartitionHealthReport() {
-    Map<String, Map<String, String>> result = new HashMap<String, Map<String, String>>();
-    for (String stat : _statsToValues.keySet()) {
-      Map<String, String> currStat = new HashMap<String, String>();
-      /*
-       * currStat.put(Stat.OP_TYPE, stat._opType);
-       * currStat.put(Stat.MEASUREMENT_TYPE, stat._measurementType);
-       * currStat.put(Stat.NODE_NAME, stat._nodeName);
-       * currStat.put(Stat.PARTITION_NAME, stat._partitionName);
-       * currStat.put(Stat.RESOURCE_NAME, stat._resourceName);
-       * currStat.put(Stat.RETURN_STATUS, stat._returnStatus);
-       * currStat.put(Stat.METRIC_NAME, stat._metricName);
-       * currStat.put(Stat.AGG_TYPE, stat._aggTypeName);
-       */
-      currStat.put(TIMESTAMP, _statsToTimestamps.get(stat));
-      currStat.put(STAT_VALUE, _statsToValues.get(stat));
-      result.put(stat, currStat);
-    }
-    return result;
-  }
-
-  public boolean contains(Stat inStat) {
-    return _statsToValues.containsKey(inStat);
-  }
-
-  public Set<String> keySet() {
-    return _statsToValues.keySet();
-  }
-
-  public String getStatValue(Stat inStat) {
-    return _statsToValues.get(inStat);
-  }
-
-  public long getStatTimestamp(Stat inStat) {
-    return Long.parseLong(_statsToTimestamps.get(inStat));
-  }
-
-  /*
-   * public String getStatValue(String opType, String measurementType, String
-   * resourceName, String partitionName, String nodeName, boolean
-   * createIfMissing) { Stat rs = new Stat(opType, measurementType,
-   * resourceName, partitionName, nodeName); String val =
-   * _statsToValues.get(rs); if (val == null && createIfMissing) { val = "0";
-   * _statsToValues.put(rs, val); } return val; }
-   */
-
-  public void writeStat(String statName, String val, String timestamp) {
-    _statsToValues.put(statName, val);
-    _statsToTimestamps.put(statName, timestamp);
-  }
-
-  /*
-   * public void setStat(Stat es, String val, String timestamp) { writeStat(es,
-   * val, timestamp); }
-   * public void setStat(String opType, String measurementType, String
-   * resourceName, String partitionName, String nodeName, double val, String
-   * timestamp) { Stat rs = new Stat(opType, measurementType, resourceName,
-   * partitionName, nodeName); writeStat(rs, String.valueOf(val), timestamp); }
-   */
-
-  public void incrementStat(String statName, String timestamp) {
-    // Stat rs = new Stat(opType, measurementType, resourceName, partitionName,
-    // nodeName);
-    String val = _statsToValues.get(statName);
-    if (val == null) {
-      val = "0";
-    } else {
-      val = String.valueOf(Double.parseDouble(val) + 1);
-    }
-    writeStat(statName, val, timestamp);
-  }
-
-  public int size() {
-    return _statsToValues.size();
-  }
-
-  public void resetStats() {
-    _statsToValues.clear();
-    _statsToTimestamps.clear();
-  }
-
-  public void setReportName(String name) {
-    _reportName = name;
-  }
-
-  public String getReportName() {
-    return _reportName;
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/healthcheck/WindowAggregationType.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/WindowAggregationType.java b/helix-core/src/main/java/org/apache/helix/healthcheck/WindowAggregationType.java
deleted file mode 100644
index 77161af..0000000
--- a/helix-core/src/main/java/org/apache/helix/healthcheck/WindowAggregationType.java
+++ /dev/null
@@ -1,67 +0,0 @@
-package org.apache.helix.healthcheck;
-
-/*
- * 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 org.apache.log4j.Logger;
-
-public class WindowAggregationType implements AggregationType {
-
-  private static final Logger logger = Logger.getLogger(WindowAggregationType.class);
-
-  public final String WINDOW_DELIM = "#";
-
-  public final static String TYPE_NAME = "window";
-
-  int _windowSize = 1;
-
-  public WindowAggregationType(int ws) {
-    super();
-    _windowSize = ws;
-  }
-
-  @Override
-  public String getName() {
-    StringBuilder sb = new StringBuilder();
-    sb.append(TYPE_NAME);
-    sb.append(DELIM);
-    sb.append(_windowSize);
-    return sb.toString();
-  }
-
-  @Override
-  public String merge(String incomingVal, String existingVal, long prevTimestamp) {
-    String[] windowVals;
-    if (existingVal == null) {
-      return incomingVal;
-    } else {
-      windowVals = existingVal.split(WINDOW_DELIM);
-      int currLength = windowVals.length;
-      // window not full
-      if (currLength < _windowSize) {
-        return existingVal + WINDOW_DELIM + incomingVal;
-      }
-      // evict oldest
-      else {
-        int firstDelim = existingVal.indexOf(WINDOW_DELIM);
-        return existingVal.substring(firstDelim + 1) + WINDOW_DELIM + incomingVal;
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/healthcheck/package-info.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/package-info.java b/helix-core/src/main/java/org/apache/helix/healthcheck/package-info.java
deleted file mode 100644
index f584b5b..0000000
--- a/helix-core/src/main/java/org/apache/helix/healthcheck/package-info.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * 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.
- */
-/**
- * Helix health check classes
- * 
- */
-package org.apache.helix.healthcheck;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/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 b844926..65fe2f9 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
@@ -41,7 +41,6 @@ import org.apache.helix.ConfigChangeListener;
 import org.apache.helix.ControllerChangeListener;
 import org.apache.helix.CurrentStateChangeListener;
 import org.apache.helix.ExternalViewChangeListener;
-import org.apache.helix.HealthStateChangeListener;
 import org.apache.helix.HelixConstants.ChangeType;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixException;
@@ -59,7 +58,6 @@ import org.apache.helix.ScopedConfigChangeListener;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.ExternalView;
-import org.apache.helix.model.HealthStat;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.LiveInstance;
@@ -209,15 +207,6 @@ public class CallbackHandler implements IZkChildListener, IZkDataListener
         ControllerChangeListener controllerChangelistener = (ControllerChangeListener) _listener;
         subscribeForChanges(changeContext, _path, true, false);
         controllerChangelistener.onControllerChange(changeContext);
-      } else if (_changeType == ChangeType.HEALTH) {
-        HealthStateChangeListener healthStateChangeListener = (HealthStateChangeListener) _listener;
-        subscribeForChanges(changeContext, _path, true, true); // TODO: figure out
-        // settings here
-        String instanceName = PropertyPathConfig.getInstanceNameFromPath(_path);
-
-        List<HealthStat> healthReportList = _accessor.getChildValues(_propertyKey);
-
-        healthStateChangeListener.onHealthChange(instanceName, healthReportList, changeContext);
       }
 
       long end = System.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/manager/zk/HelixConnectionAdaptor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/HelixConnectionAdaptor.java b/helix-core/src/main/java/org/apache/helix/manager/zk/HelixConnectionAdaptor.java
index e13c127..ef17715 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/HelixConnectionAdaptor.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/HelixConnectionAdaptor.java
@@ -25,7 +25,6 @@ import org.apache.helix.ConfigChangeListener;
 import org.apache.helix.ControllerChangeListener;
 import org.apache.helix.CurrentStateChangeListener;
 import org.apache.helix.ExternalViewChangeListener;
-import org.apache.helix.HealthStateChangeListener;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixAutoController;
 import org.apache.helix.HelixConnection;
@@ -49,7 +48,6 @@ import org.apache.helix.api.id.ClusterId;
 import org.apache.helix.api.id.Id;
 import org.apache.helix.api.id.ParticipantId;
 import org.apache.helix.api.id.SessionId;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollector;
 import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
@@ -136,13 +134,6 @@ public class HelixConnectionAdaptor implements HelixManager {
   }
 
   @Override
-  public void addHealthStateChangeListener(HealthStateChangeListener listener, String instanceName)
-      throws Exception {
-    _connection.addHealthStateChangeListener(_role, listener, _clusterId,
-        ParticipantId.from(instanceName));
-  }
-
-  @Override
   public void addExternalViewChangeListener(ExternalViewChangeListener listener) throws Exception {
     _connection.addExternalViewChangeListener(_role, listener, _clusterId);
   }
@@ -203,11 +194,6 @@ public class HelixConnectionAdaptor implements HelixManager {
   }
 
   @Override
-  public ParticipantHealthReportCollector getHealthReportCollector() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
   public InstanceType getInstanceType() {
     return _instanceType;
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManagerHelper.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManagerHelper.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManagerHelper.java
index 925c52f..c1d856d 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManagerHelper.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManagerHelper.java
@@ -278,15 +278,4 @@ public class ParticipantManagerHelper {
 
   }
 
-  /**
-   * create zk path for health check info
-   * TODO move it to cluster-setup
-   */
-  public void createHealthCheckPath() {
-    String healthCheckInfoPath = _dataAccessor.keyBuilder().healthReports(_instanceName).getPath();
-    if (!_zkclient.exists(healthCheckInfoPath)) {
-      _zkclient.createPersistent(healthCheckInfoPath, true);
-      LOG.info("Created healthcheck info path " + healthCheckInfoPath);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/38b43965/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
index da0c80c..dee343f 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
@@ -52,8 +52,6 @@ import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.PropertyPathConfig;
 import org.apache.helix.PropertyType;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.alerts.AlertsHolder;
-import org.apache.helix.alerts.StatsHolder;
 import org.apache.helix.api.State;
 import org.apache.helix.api.id.ConstraintId;
 import org.apache.helix.api.id.MessageId;
@@ -63,7 +61,6 @@ import org.apache.helix.api.id.SessionId;
 import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.api.id.StateModelFactoryId;
 import org.apache.helix.controller.strategy.DefaultTwoStateStrategy;
-import org.apache.helix.model.Alerts;
 import org.apache.helix.model.ClusterConstraints;
 import org.apache.helix.model.ClusterConstraints.ConstraintType;
 import org.apache.helix.model.ConstraintItem;
@@ -80,7 +77,6 @@ import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.MessageState;
 import org.apache.helix.model.Message.MessageType;
 import org.apache.helix.model.PauseSignal;
-import org.apache.helix.model.PersistentStats;
 import org.apache.helix.model.StateModelDefinition;
 import org.apache.helix.util.HelixUtil;
 import org.apache.helix.util.RebalanceUtil;
@@ -749,79 +745,6 @@ public class ZKHelixAdmin implements HelixAdmin {
   }
 
   @Override
-  public void addStat(String clusterName, final String statName) {
-    if (!ZKUtil.isClusterSetup(clusterName, _zkClient)) {
-      throw new HelixException("cluster " + clusterName + " is not setup yet");
-    }
-
-    String persistentStatsPath =
-        PropertyPathConfig.getPath(PropertyType.PERSISTENTSTATS, clusterName);
-    ZkBaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<ZNRecord>(_zkClient);
-
-    baseAccessor.update(persistentStatsPath, new DataUpdater<ZNRecord>() {
-
-      @Override
-      public ZNRecord update(ZNRecord statsRec) {
-        if (statsRec == null) {
-          // TODO: fix naming of this record, if it matters
-          statsRec = new ZNRecord(PersistentStats.nodeName);
-        }
-
-        Map<String, Map<String, String>> currStatMap = statsRec.getMapFields();
-        Map<String, Map<String, String>> newStatMap = StatsHolder.parseStat(statName);
-        for (String newStat : newStatMap.keySet()) {
-          if (!currStatMap.containsKey(newStat)) {
-            currStatMap.put(newStat, newStatMap.get(newStat));
-          }
-        }
-        statsRec.setMapFields(currStatMap);
-
-        return statsRec;
-      }
-    }, AccessOption.PERSISTENT);
-  }
-
-  @Override
-  public void addAlert(final String clusterName, final String alertName) {
-    if (!ZKUtil.isClusterSetup(clusterName, _zkClient)) {
-      throw new HelixException("cluster " + clusterName + " is not setup yet");
-    }
-
-    ZkBaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<ZNRecord>(_zkClient);
-
-    String alertsPath = PropertyPathConfig.getPath(PropertyType.ALERTS, clusterName);
-
-    baseAccessor.update(alertsPath, new DataUpdater<ZNRecord>() {
-
-      @Override
-      public ZNRecord update(ZNRecord alertsRec) {
-        if (alertsRec == null) {
-          // TODO: fix naming of this record, if it matters
-          alertsRec = new ZNRecord(Alerts.nodeName);
-
-        }
-
-        Map<String, Map<String, String>> currAlertMap = alertsRec.getMapFields();
-        StringBuilder newStatName = new StringBuilder();
-        Map<String, String> newAlertMap = new HashMap<String, String>();
-
-        // use AlertsHolder to get map of new stats and map for this alert
-        AlertsHolder.parseAlert(alertName, newStatName, newAlertMap);
-
-        // add stat
-        addStat(clusterName, newStatName.toString());
-
-        // add alert
-        currAlertMap.put(alertName, newAlertMap);
-
-        alertsRec.setMapFields(currAlertMap);
-
-        return alertsRec;
-      }
-    }, AccessOption.PERSISTENT);
-  }
-
-  @Override
   public void dropCluster(String clusterName) {
     logger.info("Deleting cluster " + clusterName);
     HelixDataAccessor accessor =
@@ -842,70 +765,6 @@ public class ZKHelixAdmin implements HelixAdmin {
   }
 
   @Override
-  public void dropStat(String clusterName, final String statName) {
-    if (!ZKUtil.isClusterSetup(clusterName, _zkClient)) {
-      throw new HelixException("cluster " + clusterName + " is not setup yet");
-    }
-
-    String persistentStatsPath =
-        PropertyPathConfig.getPath(PropertyType.PERSISTENTSTATS, clusterName);
-    ZkBaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<ZNRecord>(_zkClient);
-
-    baseAccessor.update(persistentStatsPath, new DataUpdater<ZNRecord>() {
-
-      @Override
-      public ZNRecord update(ZNRecord statsRec) {
-        if (statsRec == null) {
-          throw new HelixException("No stats record in ZK, nothing to drop");
-        }
-
-        Map<String, Map<String, String>> currStatMap = statsRec.getMapFields();
-        Map<String, Map<String, String>> newStatMap = StatsHolder.parseStat(statName);
-
-        // delete each stat from stat map
-        for (String newStat : newStatMap.keySet()) {
-          if (currStatMap.containsKey(newStat)) {
-            currStatMap.remove(newStat);
-          }
-        }
-        statsRec.setMapFields(currStatMap);
-
-        return statsRec;
-      }
-    }, AccessOption.PERSISTENT);
-  }
-
-  @Override
-  public void dropAlert(String clusterName, final String alertName) {
-    if (!ZKUtil.isClusterSetup(clusterName, _zkClient)) {
-      throw new HelixException("cluster " + clusterName + " is not setup yet");
-    }
-
-    String alertsPath = PropertyPathConfig.getPath(PropertyType.ALERTS, clusterName);
-
-    ZkBaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<ZNRecord>(_zkClient);
-
-    if (!baseAccessor.exists(alertsPath, 0)) {
-      throw new HelixException("No alerts node in ZK, nothing to drop");
-    }
-
-    baseAccessor.update(alertsPath, new DataUpdater<ZNRecord>() {
-      @Override
-      public ZNRecord update(ZNRecord alertsRec) {
-        if (alertsRec == null) {
-          throw new HelixException("No alerts record in ZK, nothing to drop");
-        }
-
-        Map<String, Map<String, String>> currAlertMap = alertsRec.getMapFields();
-        currAlertMap.remove(alertName);
-        alertsRec.setMapFields(currAlertMap);
-
-        return alertsRec;
-      }
-    }, AccessOption.PERSISTENT);
-  }
-
-  @Override
   public void addClusterToGrandCluster(String clusterName, String grandCluster) {
     if (!ZKUtil.isClusterSetup(grandCluster, _zkClient)) {
       throw new HelixException("Grand cluster " + grandCluster + " is not setup yet");


[10/17] git commit: [HELIX-445] NPE in ZkPathDataDumpTask, rb=21504

Posted by ka...@apache.org.
[HELIX-445] NPE in ZkPathDataDumpTask, rb=21504


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

Branch: refs/heads/master
Commit: 33c2f15ca825e2d6bd5bc2b07093b3eaebbb84f3
Parents: 38b4396
Author: zzhang <zz...@apache.org>
Authored: Wed May 21 18:26:40 2014 -0700
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Fri Jul 11 12:43:56 2014 -0700

----------------------------------------------------------------------
 helix-core/pom.xml                              |   5 +
 .../main/java/org/apache/helix/PropertyKey.java |  35 +++-
 .../apache/helix/manager/zk/ZKHelixManager.java |  12 +-
 .../helix/monitoring/ZKPathDataDumpTask.java    | 173 ++++++++++---------
 .../integration/TestHelixCustomCodeRunner.java  |  44 ++---
 .../helix/integration/TestSchedulerMessage.java |   2 +-
 .../manager/MockParticipantManager.java         |   1 -
 .../helix/mock/participant/MockJobIntf.java     |  28 ---
 .../monitoring/TestParticipantMonitor.java      |   6 +-
 .../helix/monitoring/TestStatCollector.java     |   6 +-
 .../monitoring/TestZKPathDataDumpTask.java      | 113 ++++++++++++
 pom.xml                                         |   6 +-
 12 files changed, 262 insertions(+), 169 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/33c2f15c/helix-core/pom.xml
----------------------------------------------------------------------
diff --git a/helix-core/pom.xml b/helix-core/pom.xml
index 0d06914..bacbb07 100644
--- a/helix-core/pom.xml
+++ b/helix-core/pom.xml
@@ -125,6 +125,11 @@ under the License.
       </exclusions>
     </dependency>
     <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-math</artifactId>
       <version>2.1</version>

http://git-wip-us.apache.org/repos/asf/helix/blob/33c2f15c/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 fb3e1d1..cbb4e7b 100644
--- a/helix-core/src/main/java/org/apache/helix/PropertyKey.java
+++ b/helix-core/src/main/java/org/apache/helix/PropertyKey.java
@@ -481,6 +481,15 @@ public class PropertyKey {
     }
 
     /**
+     * Get a property key associated with {@link StatusUpdate} of an instance
+     * @param instanceName
+     * @return {@link PropertyKey}
+     */
+    public PropertyKey stateTransitionStatus(String instanceName) {
+      return new PropertyKey(STATUSUPDATES, StatusUpdate.class, _clusterName, instanceName);
+    }
+
+    /**
      * Used to get status update for a NON STATE TRANSITION type
      * @param instanceName
      * @param sessionId
@@ -533,6 +542,16 @@ public class PropertyKey {
     }
 
     /**
+     * Get a property key associated with {@link Error} of an instance, session, and
+     * resource
+     * @param instanceName
+     * @return {@link PropertyKey}
+     */
+    public PropertyKey stateTransitionErrors(String instanceName) {
+      return new PropertyKey(ERRORS, Error.class, _clusterName, instanceName);
+    }
+
+    /**
      * Used to get status update for a NON STATE TRANSITION type
      * @param instanceName
      * @param sessionId
@@ -587,14 +606,6 @@ public class PropertyKey {
     }
 
     /**
-     * Get the root of all controller status updates
-     * @return {@link PropertyKey}
-     */
-    public PropertyKey controllerTaskStatuses() {
-      return new PropertyKey(STATUSUPDATES_CONTROLLER, StatusUpdate.class, _clusterName);
-    }
-
-    /**
      * Get a property key associated with {@link StatusUpdate} of controller status updates
      * @param subPath
      * @return {@link PropertyKey}
@@ -615,6 +626,14 @@ public class PropertyKey {
     }
 
     /**
+     * Get a property key associated with {@link StatusUpdate} of controller status updates
+     * @return {@link PropertyKey}
+     */
+    public PropertyKey controllerTaskStatuses() {
+      return new PropertyKey(STATUSUPDATES_CONTROLLER, StatusUpdate.class, _clusterName);
+    }
+
+    /**
      * Get a property key associated with all {@link Message}s for the controller
      * @return {@link PropertyKey}
      */

http://git-wip-us.apache.org/repos/asf/helix/blob/33c2f15c/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 e3d9c7b..9aa8d6b 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
@@ -131,11 +131,9 @@ public class ZKHelixManager implements HelixManager, IZkStateListener {
    */
   static class StatusDumpTask extends HelixTimerTask {
     Timer _timer = null;
-    final ZkClient zkclient;
     final HelixManager helixController;
 
-    public StatusDumpTask(ZkClient zkclient, HelixManager helixController) {
-      this.zkclient = zkclient;
+    public StatusDumpTask(HelixManager helixController) {
       this.helixController = helixController;
     }
 
@@ -148,8 +146,8 @@ public class ZKHelixManager implements HelixManager, IZkStateListener {
       if (_timer == null) {
         LOG.info("Start StatusDumpTask");
         _timer = new Timer("StatusDumpTimerTask", true);
-        _timer.scheduleAtFixedRate(new ZKPathDataDumpTask(helixController, zkclient,
-            timeThresholdNoChange), initialDelay, period);
+        _timer.scheduleAtFixedRate(new ZKPathDataDumpTask(helixController, timeThresholdNoChange),
+            initialDelay, period);
       }
     }
 
@@ -216,12 +214,12 @@ public class ZKHelixManager implements HelixManager, IZkStateListener {
       break;
     case CONTROLLER:
       _stateMachineEngine = null;
-      _controllerTimerTasks.add(new StatusDumpTask(_zkclient, this));
+      _controllerTimerTasks.add(new StatusDumpTask(this));
 
       break;
     case CONTROLLER_PARTICIPANT:
       _stateMachineEngine = new HelixStateMachineEngine(this);
-      _controllerTimerTasks.add(new StatusDumpTask(_zkclient, this));
+      _controllerTimerTasks.add(new StatusDumpTask(this));
       break;
     case ADMINISTRATOR:
     case SPECTATOR:

http://git-wip-us.apache.org/repos/asf/helix/blob/33c2f15c/helix-core/src/main/java/org/apache/helix/monitoring/ZKPathDataDumpTask.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/ZKPathDataDumpTask.java b/helix-core/src/main/java/org/apache/helix/monitoring/ZKPathDataDumpTask.java
index 226b250..2ab1be3 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/ZKPathDataDumpTask.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/ZKPathDataDumpTask.java
@@ -19,36 +19,35 @@ package org.apache.helix.monitoring;
  * under the License.
  */
 
-import java.io.StringWriter;
-import java.util.Date;
 import java.util.List;
 import java.util.TimerTask;
 
+import org.apache.helix.BaseDataAccessor;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.PropertyType;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.manager.zk.ZNRecordSerializer;
-import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.util.HelixUtil;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.data.Stat;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.map.SerializationConfig;
+
+import com.google.common.collect.Lists;
 
 public class ZKPathDataDumpTask extends TimerTask {
-  static Logger logger = Logger.getLogger(ZKPathDataDumpTask.class);
+  static Logger LOG = Logger.getLogger(ZKPathDataDumpTask.class);
 
   private final int _thresholdNoChangeInMs;
   private final HelixManager _manager;
-  private final ZkClient _zkClient;
+  private final ZNRecordSerializer _jsonSerializer;
+
+  public ZKPathDataDumpTask(HelixManager manager, int thresholdNoChangeInMs) {
+    LOG.info("Init ZKPathDataDumpTask for cluster: " + manager.getClusterName()
+        + ", thresholdNoChangeInMs: " + thresholdNoChangeInMs);
 
-  public ZKPathDataDumpTask(HelixManager manager, ZkClient zkClient, int thresholdNoChangeInMs) {
     _manager = manager;
-    _zkClient = zkClient;
-    logger.info("Scanning cluster statusUpdate " + manager.getClusterName()
-        + " thresholdNoChangeInMs: " + thresholdNoChangeInMs);
+    _jsonSerializer = new ZNRecordSerializer();
     _thresholdNoChangeInMs = thresholdNoChangeInMs;
   }
 
@@ -59,88 +58,96 @@ public class ZKPathDataDumpTask extends TimerTask {
     // We need to think if we should create per-instance log files that contains
     // per-instance statusUpdates
     // and errors
-    logger.info("Scanning status updates ...");
-    try {
-      HelixDataAccessor accessor = _manager.getHelixDataAccessor();
-      Builder keyBuilder = accessor.keyBuilder();
-
-      List<String> instances = accessor.getChildNames(keyBuilder.instanceConfigs());
-      for (String instanceName : instances) {
-        scanPath(HelixUtil.getInstancePropertyPath(_manager.getClusterName(), instanceName,
-            PropertyType.STATUSUPDATES), _thresholdNoChangeInMs);
-        scanPath(HelixUtil.getInstancePropertyPath(_manager.getClusterName(), instanceName,
-            PropertyType.ERRORS), _thresholdNoChangeInMs * 3);
-      }
-      scanPath(HelixUtil.getControllerPropertyPath(_manager.getClusterName(),
-          PropertyType.STATUSUPDATES_CONTROLLER), _thresholdNoChangeInMs);
+    LOG.info("Scan statusUpdates and errors for cluster: " + _manager.getClusterName()
+        + ", by controller: " + _manager);
+    HelixDataAccessor accessor = _manager.getHelixDataAccessor();
+    Builder keyBuilder = accessor.keyBuilder();
+    BaseDataAccessor<ZNRecord> baseAccessor = accessor.getBaseDataAccessor();
+
+    List<String> instances = accessor.getChildNames(keyBuilder.instanceConfigs());
+    for (String instance : instances) {
+      // dump participant status updates
+      String statusUpdatePath =
+          HelixUtil.getInstancePropertyPath(_manager.getClusterName(), instance,
+              PropertyType.STATUSUPDATES);
+      dump(baseAccessor, statusUpdatePath, _thresholdNoChangeInMs);
 
-      scanPath(HelixUtil.getControllerPropertyPath(_manager.getClusterName(),
-          PropertyType.ERRORS_CONTROLLER), _thresholdNoChangeInMs * 3);
-    } catch (Exception e) {
-      logger.error(e);
+      // dump participant errors
+      String errorPath =
+          HelixUtil.getInstancePropertyPath(_manager.getClusterName(), instance,
+              PropertyType.ERRORS);
+      dump(baseAccessor, errorPath, _thresholdNoChangeInMs * 3);
     }
+    // dump controller status updates
+    String controllerStatusUpdatePath =
+        HelixUtil.getControllerPropertyPath(_manager.getClusterName(),
+            PropertyType.STATUSUPDATES_CONTROLLER);
+    dump(baseAccessor, controllerStatusUpdatePath, _thresholdNoChangeInMs);
+
+    // dump controller errors
+    String controllerErrorPath =
+        HelixUtil.getControllerPropertyPath(_manager.getClusterName(),
+            PropertyType.ERRORS_CONTROLLER);
+    dump(baseAccessor, controllerErrorPath, _thresholdNoChangeInMs);
   }
 
-  void scanPath(String path, int thresholdNoChangeInMs) {
-    logger.info("Scanning path " + path);
-    List<String> subPaths = _zkClient.getChildren(path);
-    for (String subPath : subPaths) {
-      try {
-        String nextPath = path + "/" + subPath;
-        List<String> subSubPaths = _zkClient.getChildren(nextPath);
-        for (String subsubPath : subSubPaths) {
-          try {
-            checkAndDump(nextPath + "/" + subsubPath, thresholdNoChangeInMs);
-          } catch (Exception e) {
-            logger.error(e);
-          }
-        }
-      } catch (Exception e) {
-        logger.error(e);
+  /**
+   * Find paths of all leaf nodes under an ancestor path (exclusive)
+   * @param accessor
+   * @param ancestorPath
+   * @return a list of paths
+   */
+  static List<String> scanPath(BaseDataAccessor<ZNRecord> accessor, String ancestorPath) {
+    List<String> queue = Lists.newLinkedList();
+    queue.add(ancestorPath);
+
+    // BFS
+    List<String> leafPaths = Lists.newArrayList();
+    while (!queue.isEmpty()) {
+      String path = queue.remove(0);
+      List<String> childNames = accessor.getChildNames(path, 0);
+      if (childNames == null) {
+        // path doesn't exist
+        continue;
+      }
+      if (childNames.isEmpty() && !path.equals(ancestorPath)) {
+        // leaf node, excluding ancestorPath
+        leafPaths.add(path);
+      }
+      for (String childName : childNames) {
+        String subPath = String.format("%s/%s", path, childName);
+        queue.add(subPath);
       }
     }
+    return leafPaths;
   }
 
-  void checkAndDump(String path, int thresholdNoChangeInMs) {
-    List<String> subPaths = _zkClient.getChildren(path);
-    if (subPaths.size() == 0) {
-      subPaths.add("");
+  void dump(BaseDataAccessor<ZNRecord> accessor, String ancestorPath, int threshold) {
+    List<String> leafPaths = scanPath(accessor, ancestorPath);
+    if (leafPaths.isEmpty()) {
+      return;
+    }
+
+    Stat[] stats = accessor.getStats(leafPaths, 0);
+    List<String> dumpPaths = Lists.newArrayList();
+    long now = System.currentTimeMillis();
+    for (int i = 0; i < stats.length; i++) {
+      Stat stat = stats[i];
+      if ((now - stat.getMtime()) > threshold) {
+        dumpPaths.add(leafPaths.get(i));
+      }
     }
-    for (String subPath : subPaths) {
-      String fullPath = subPath.length() > 0 ? path + "/" + subPath : path;
-      Stat pathStat = _zkClient.getStat(fullPath);
-
-      long lastModifiedTimeInMs = pathStat.getMtime();
-      long nowInMs = new Date().getTime();
-      // logger.info(nowInMs + " " + lastModifiedTimeInMs + " " + fullPath);
-
-      // Check the last modified time
-      if (nowInMs > lastModifiedTimeInMs) {
-        long timeDiff = nowInMs - lastModifiedTimeInMs;
-        if (timeDiff > thresholdNoChangeInMs) {
-          logger.info("Dumping status update path " + fullPath + " " + timeDiff + "MS has passed");
-          _zkClient.setZkSerializer(new ZNRecordSerializer());
-          ZNRecord record = _zkClient.readData(fullPath);
-
-          // dump the node content into log file
-          ObjectMapper mapper = new ObjectMapper();
-          SerializationConfig serializationConfig = mapper.getSerializationConfig();
-          serializationConfig.set(SerializationConfig.Feature.INDENT_OUTPUT, true);
-
-          StringWriter sw = new StringWriter();
-          try {
-            mapper.writeValue(sw, record);
-            logger.info(sw.toString());
-          } catch (Exception e) {
-            logger
-                .warn(
-                    "Exception during serialization in ZKPathDataDumpTask.checkAndDump. This can mostly be ignored",
-                    e);
-          }
-          // Delete the leaf data
-          _zkClient.deleteRecursive(fullPath);
-        }
+
+    // dump
+    LOG.info("Dump statusUpdates and errors records for pahts: " + dumpPaths);
+    List<ZNRecord> dumpRecords = accessor.get(dumpPaths, null, 0);
+    for (ZNRecord record : dumpRecords) {
+      if (record != null) {
+        LOG.info(new String(_jsonSerializer.serialize(record)));
       }
     }
+
+    // clean up
+    accessor.remove(dumpPaths, 0);
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/33c2f15c/helix-core/src/test/java/org/apache/helix/integration/TestHelixCustomCodeRunner.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestHelixCustomCodeRunner.java b/helix-core/src/test/java/org/apache/helix/integration/TestHelixCustomCodeRunner.java
index c7a1700..9423326 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestHelixCustomCodeRunner.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestHelixCustomCodeRunner.java
@@ -22,9 +22,9 @@ package org.apache.helix.integration;
 import java.util.Date;
 
 import org.apache.helix.HelixConstants.ChangeType;
+import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.NotificationContext;
-import org.apache.helix.NotificationContext.Type;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
@@ -32,7 +32,6 @@ import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.participant.MockJobIntf;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.participant.CustomCodeCallbackHandler;
 import org.apache.helix.participant.HelixCustomCodeRunner;
@@ -54,38 +53,26 @@ public class TestHelixCustomCodeRunner extends ZkIntegrationTestBase {
 
     @Override
     public void onCallback(NotificationContext context) {
-      HelixManager manager = context.getManager();
-      Type type = context.getType();
       _isCallbackInvoked = true;
       // System.out.println(type + ": TestCallback invoked on " + manager.getInstanceName());
     }
 
   }
 
-  class MockJob implements MockJobIntf {
-    @Override
-    public void doPreConnectJob(HelixManager manager) {
-      try {
-        // delay the start of the 1st participant
-        // so there will be a leadership transfer from localhost_12919 to 12918
-        if (manager.getInstanceName().equals("localhost_12918")) {
-          Thread.sleep(2000);
-        }
-
-        HelixCustomCodeRunner customCodeRunner = new HelixCustomCodeRunner(manager, ZK_ADDR);
-        customCodeRunner.invoke(_callback).on(ChangeType.LIVE_INSTANCE)
-            .usingLeaderStandbyModel("TestParticLeader").start();
-      } catch (Exception e) {
-        LOG.error("Exception do pre-connect job", e);
+  private void registerCustomCodeRunner(HelixManager manager) {
+    try {
+      // delay the start of the 1st participant
+      // so there will be a leadership transfer from localhost_12919 to 12918
+      if (manager.getInstanceName().equals("localhost_12918")) {
+        Thread.sleep(2000);
       }
-    }
-
-    @Override
-    public void doPostConnectJob(HelixManager manager) {
-      // TODO Auto-generated method stub
 
+      HelixCustomCodeRunner customCodeRunner = new HelixCustomCodeRunner(manager, ZK_ADDR);
+      customCodeRunner.invoke(_callback).on(ChangeType.LIVE_INSTANCE)
+          .usingLeaderStandbyModel("TestParticLeader").start();
+    } catch (Exception e) {
+      LOG.error("Exception do pre-connect job", e);
     }
-
   }
 
   @Test
@@ -109,10 +96,9 @@ public class TestHelixCustomCodeRunner extends ZkIntegrationTestBase {
     for (int i = 0; i < _nodeNb; i++) {
       String instanceName = "localhost_" + (_startPort + i);
 
-      MockJob job = new MockJob();
       participants[i] = new MockParticipantManager(ZK_ADDR, _clusterName, instanceName);
 
-      job.doPreConnectJob(participants[i]);
+      registerCustomCodeRunner(participants[i]);
       participants[i].syncStart();
     }
     boolean result =
@@ -125,9 +111,7 @@ public class TestHelixCustomCodeRunner extends ZkIntegrationTestBase {
     _callback._isCallbackInvoked = false;
 
     // add a new live instance
-    // ZkClient zkClient = new ZkClient(ZK_ADDR);
-    // zkClient.setZkSerializer(new ZNRecordSerializer());
-    ZKHelixDataAccessor accessor =
+    HelixDataAccessor accessor =
         new ZKHelixDataAccessor(_clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
     Builder keyBuilder = accessor.keyBuilder();
 

http://git-wip-us.apache.org/repos/asf/helix/blob/33c2f15c/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMessage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMessage.java b/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMessage.java
index e6117d6..623db80 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMessage.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMessage.java
@@ -397,7 +397,7 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBase {
       }
     }
     Thread.sleep(3000);
-    ZKPathDataDumpTask dumpTask = new ZKPathDataDumpTask(manager, _gZkClient, 0);
+    ZKPathDataDumpTask dumpTask = new ZKPathDataDumpTask(manager, 0);
     dumpTask.run();
 
     subPaths = _gZkClient.getChildren(controllerStatusPath);

http://git-wip-us.apache.org/repos/asf/helix/blob/33c2f15c/helix-core/src/test/java/org/apache/helix/integration/manager/MockParticipantManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/MockParticipantManager.java b/helix-core/src/test/java/org/apache/helix/integration/manager/MockParticipantManager.java
index 34efe34..917be17 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/manager/MockParticipantManager.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/manager/MockParticipantManager.java
@@ -28,7 +28,6 @@ import org.apache.helix.manager.zk.ZKHelixManager;
 import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.mock.participant.DummyProcess.DummyLeaderStandbyStateModelFactory;
 import org.apache.helix.mock.participant.DummyProcess.DummyOnlineOfflineStateModelFactory;
-import org.apache.helix.mock.participant.MockJobIntf;
 import org.apache.helix.mock.participant.MockMSModelFactory;
 import org.apache.helix.mock.participant.MockSchemataModelFactory;
 import org.apache.helix.mock.participant.MockTransition;

http://git-wip-us.apache.org/repos/asf/helix/blob/33c2f15c/helix-core/src/test/java/org/apache/helix/mock/participant/MockJobIntf.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/mock/participant/MockJobIntf.java b/helix-core/src/test/java/org/apache/helix/mock/participant/MockJobIntf.java
deleted file mode 100644
index 4b637a6..0000000
--- a/helix-core/src/test/java/org/apache/helix/mock/participant/MockJobIntf.java
+++ /dev/null
@@ -1,28 +0,0 @@
-package org.apache.helix.mock.participant;
-
-/*
- * 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 org.apache.helix.HelixManager;
-
-public interface MockJobIntf {
-  public void doPreConnectJob(HelixManager manager);
-
-  public void doPostConnectJob(HelixManager manager);
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/33c2f15c/helix-core/src/test/java/org/apache/helix/monitoring/TestParticipantMonitor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/monitoring/TestParticipantMonitor.java b/helix-core/src/test/java/org/apache/helix/monitoring/TestParticipantMonitor.java
index 1dc0e2d..15987cd 100644
--- a/helix-core/src/test/java/org/apache/helix/monitoring/TestParticipantMonitor.java
+++ b/helix-core/src/test/java/org/apache/helix/monitoring/TestParticipantMonitor.java
@@ -93,10 +93,8 @@ public class TestParticipantMonitor {
     }
   }
 
-  @Test(groups = {
-    "unitTest"
-  })
-  public void TestReportData() throws InstanceNotFoundException, MalformedObjectNameException,
+  @Test()
+  public void testReportData() throws InstanceNotFoundException, MalformedObjectNameException,
       NullPointerException, IOException, InterruptedException {
     System.out.println("START TestParticipantMonitor");
     ParticipantMonitor monitor = new ParticipantMonitor();

http://git-wip-us.apache.org/repos/asf/helix/blob/33c2f15c/helix-core/src/test/java/org/apache/helix/monitoring/TestStatCollector.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/monitoring/TestStatCollector.java b/helix-core/src/test/java/org/apache/helix/monitoring/TestStatCollector.java
index 5d38db1..caa0042 100644
--- a/helix-core/src/test/java/org/apache/helix/monitoring/TestStatCollector.java
+++ b/helix-core/src/test/java/org/apache/helix/monitoring/TestStatCollector.java
@@ -23,10 +23,8 @@ import org.testng.AssertJUnit;
 import org.testng.annotations.Test;
 
 public class TestStatCollector {
-  @Test(groups = {
-    "unitTest"
-  })
-  public void TestCollectData() {
+  @Test()
+  public void testCollectData() {
     StatCollector collector = new StatCollector();
 
     int nPoints = 100;

http://git-wip-us.apache.org/repos/asf/helix/blob/33c2f15c/helix-core/src/test/java/org/apache/helix/monitoring/TestZKPathDataDumpTask.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/monitoring/TestZKPathDataDumpTask.java b/helix-core/src/test/java/org/apache/helix/monitoring/TestZKPathDataDumpTask.java
new file mode 100644
index 0000000..a3d8ae3
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/monitoring/TestZKPathDataDumpTask.java
@@ -0,0 +1,113 @@
+package org.apache.helix.monitoring;
+
+/*
+ * 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 org.apache.helix.BaseDataAccessor;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixManager;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.TestHelper;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.ZkUnitTestBase;
+import org.apache.helix.manager.zk.ZKHelixDataAccessor;
+import org.apache.helix.manager.zk.ZkBaseDataAccessor;
+import org.apache.helix.model.StatusUpdate;
+import org.apache.helix.model.Error;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class TestZKPathDataDumpTask extends ZkUnitTestBase {
+
+  @Test
+  public void test() throws Exception {
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+    int n = 1;
+
+    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
+        2, // partitions per resource
+        n, // number of nodes
+        1, // replicas
+        "MasterSlave", true); // do rebalance
+
+    HelixDataAccessor accessor =
+        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
+    PropertyKey.Builder keyBuilder = accessor.keyBuilder();
+    BaseDataAccessor<ZNRecord> baseAccessor = accessor.getBaseDataAccessor();
+
+    HelixManager manager = mock(HelixManager.class);
+    when(manager.getHelixDataAccessor()).thenReturn(accessor);
+    when(manager.getClusterName()).thenReturn(clusterName);
+
+    // run dump task without statusUpdates and errors, should not remove any existing statusUpdate/error paths
+    ZKPathDataDumpTask task = new ZKPathDataDumpTask(manager, 0);
+    task.run();
+    PropertyKey controllerStatusUpdateKey = keyBuilder.controllerTaskStatuses();
+    Assert.assertTrue(baseAccessor.exists(controllerStatusUpdateKey.getPath(), 0));
+    PropertyKey controllerErrorKey = keyBuilder.controllerTaskErrors();
+    Assert.assertTrue(baseAccessor.exists(controllerErrorKey.getPath(), 0));
+    PropertyKey statusUpdateKey = keyBuilder.stateTransitionStatus("localhost_12918");
+    Assert.assertTrue(baseAccessor.exists(statusUpdateKey.getPath(), 0));
+    PropertyKey errorKey = keyBuilder.stateTransitionErrors("localhost_12918");
+
+    // add participant status updates and errors
+    statusUpdateKey =
+        keyBuilder.stateTransitionStatus("localhost_12918", "session_0", "TestDB0", "TestDB0_0");
+    accessor.setProperty(statusUpdateKey, new StatusUpdate(new ZNRecord("statusUpdate")));
+    errorKey =
+        keyBuilder.stateTransitionError("localhost_12918", "session_0", "TestDB0", "TestDB0_0");
+    accessor.setProperty(errorKey, new Error(new ZNRecord("error")));
+
+    // add controller status updates and errors
+    controllerStatusUpdateKey = keyBuilder.controllerTaskStatus("session_0", "TestDB");
+    accessor.setProperty(controllerStatusUpdateKey, new StatusUpdate(new ZNRecord("controllerStatusUpdate")));
+    controllerErrorKey = keyBuilder.controllerTaskError("TestDB_error");
+    accessor.setProperty(controllerErrorKey, new Error(new ZNRecord("controllerError")));
+
+    // run dump task, should remove existing statusUpdate/error paths
+    task.run();
+    Assert.assertFalse(baseAccessor.exists(controllerStatusUpdateKey.getPath(), 0));
+    Assert.assertFalse(baseAccessor.exists(controllerErrorKey.getPath(), 0));
+    Assert.assertFalse(baseAccessor.exists(statusUpdateKey.getPath(), 0));
+    Assert.assertFalse(baseAccessor.exists(errorKey.getPath(), 0));
+
+    controllerStatusUpdateKey = keyBuilder.controllerTaskStatuses();
+    Assert.assertTrue(baseAccessor.exists(controllerStatusUpdateKey.getPath(), 0));
+    controllerErrorKey = keyBuilder.controllerTaskErrors();
+    Assert.assertTrue(baseAccessor.exists(controllerErrorKey.getPath(), 0));
+    statusUpdateKey = keyBuilder.stateTransitionStatus("localhost_12918");
+    Assert.assertTrue(baseAccessor.exists(statusUpdateKey.getPath(), 0));
+    errorKey = keyBuilder.stateTransitionErrors("localhost_12918");
+
+    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/33c2f15c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 8b45e0a..dc5cc88 100644
--- a/pom.xml
+++ b/pom.xml
@@ -370,9 +370,9 @@ under the License.
         <version>6.0.1</version>
       </dependency>
       <dependency>
-        <groupId>org.yaml</groupId>
-        <artifactId>snakeyaml</artifactId>
-        <version>1.12</version>
+        <groupId>org.mockito</groupId>
+        <artifactId>mockito-all</artifactId>
+        <version>1.9.5</version>
       </dependency>
     </dependencies>
   </dependencyManagement>


[15/17] git commit: [HELIX-465] ZkCopy skips paths already exist in destination namespace, rb=23184

Posted by ka...@apache.org.
[HELIX-465] ZkCopy skips paths already exist in destination namespace, rb=23184


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

Branch: refs/heads/master
Commit: 506dc298df4ba8d569c3339d3f2ae9d1ca8895b3
Parents: 7d04053
Author: zzhang <zz...@uci.edu>
Authored: Mon Jun 30 17:27:36 2014 -0700
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Fri Jul 11 12:55:11 2014 -0700

----------------------------------------------------------------------
 .../java/org/apache/helix/tools/ZkCopy.java     | 13 ++++--
 .../java/org/apache/helix/tools/TestZkCopy.java | 46 ++++++++++++++++++++
 2 files changed, 56 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/506dc298/helix-core/src/main/java/org/apache/helix/tools/ZkCopy.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/tools/ZkCopy.java b/helix-core/src/main/java/org/apache/helix/tools/ZkCopy.java
index 69369a5..3991c09 100644
--- a/helix-core/src/main/java/org/apache/helix/tools/ZkCopy.java
+++ b/helix-core/src/main/java/org/apache/helix/tools/ZkCopy.java
@@ -23,6 +23,7 @@ import java.net.URI;
 import java.util.LinkedList;
 import java.util.List;
 
+import org.I0Itec.zkclient.exception.ZkNodeExistsException;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.GnuParser;
@@ -119,12 +120,18 @@ public class ZkCopy {
       String fromPath = concatenate(srcPath, path);
       Object data = srcClient.readDataAndStat(fromPath, stat, false);
       if (stat.getEphemeralOwner() != 0) {
-        logger.info("Skip copying ephemeral znode: " + fromPath);
+        logger.warn("Skip copying ephemeral znode: " + fromPath);
         continue;
       }
       String toPath = concatenate(dstPath, path);
-      System.out.println("Copy " + fromPath + " to " + toPath);
-      dstClient.createPersistent(toPath, data);
+      try {
+        dstClient.createPersistent(toPath, data);
+        System.out.println("Copy " + fromPath + " to " + toPath);
+
+      } catch (ZkNodeExistsException e) {
+        logger.warn("Skip copying znode: " + fromPath + ", " + toPath + " already exists");
+      }
+
       List<String> children = srcClient.getChildren(fromPath);
       if (children != null && children.size() > 0) {
         for (String child : children) {

http://git-wip-us.apache.org/repos/asf/helix/blob/506dc298/helix-core/src/test/java/org/apache/helix/tools/TestZkCopy.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/tools/TestZkCopy.java b/helix-core/src/test/java/org/apache/helix/tools/TestZkCopy.java
index 21a1ad5..ee3d2bd 100644
--- a/helix-core/src/test/java/org/apache/helix/tools/TestZkCopy.java
+++ b/helix-core/src/test/java/org/apache/helix/tools/TestZkCopy.java
@@ -21,9 +21,11 @@ package org.apache.helix.tools;
 
 import java.util.Date;
 
+import org.apache.helix.InstanceType;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkUnitTestBase;
+import org.apache.helix.manager.zk.ZKUtil;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
@@ -64,4 +66,48 @@ public class TestZkCopy extends ZkUnitTestBase {
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
+
+  @Test
+  public void testSkipCopyExistZnode() throws Exception {
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String testName = className + "_" + methodName;
+
+    System.out.println("START " + testName + " at " + new Date(System.currentTimeMillis()));
+    String srcClusterName = testName + "_src";
+    String dstClusterName = testName + "_dst";
+    int n = 5;
+
+    TestHelper.setupCluster(srcClusterName, 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
+
+    TestHelper.setupEmptyCluster(_gZkClient, dstClusterName);
+
+    String fromPath = String.format("/%s/INSTANCES", srcClusterName);
+    String toPath = String.format("/%s/INSTANCES", dstClusterName);
+    ZkCopy.main(new String[] {
+        "--src", "zk://" + ZK_ADDR + fromPath, "--dst", "zk://" + ZK_ADDR + toPath
+    });
+
+    fromPath = String.format("/%s/CONFIGS/PARTICIPANT", srcClusterName);
+    toPath = String.format("/%s/CONFIGS/PARTICIPANT", dstClusterName);
+    ZkCopy.main(new String[] {
+        "--src", "zk://" + ZK_ADDR + fromPath, "--dst", "zk://" + ZK_ADDR + toPath
+    });
+
+    for (int i = 0; i < n; i++) {
+      String instanceName = "localhost_" + (12918 + i);
+      boolean ret =
+          ZKUtil
+              .isInstanceSetup(_gZkClient, dstClusterName, instanceName, InstanceType.PARTICIPANT);
+      Assert.assertTrue(ret);
+    }
+    System.out.println("END " + testName + " at " + new Date(System.currentTimeMillis()));
+  }
 }


[16/17] git commit: [HELIX-466] Speed up zkcopy by using asyn read/write, rb=23241

Posted by ka...@apache.org.
[HELIX-466] Speed up zkcopy by using asyn read/write, rb=23241


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

Branch: refs/heads/master
Commit: 28a8f20d8f88781e9798d4cfc6439246977716a0
Parents: 506dc29
Author: zzhang <zz...@uci.edu>
Authored: Wed Jul 2 14:58:10 2014 -0700
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Fri Jul 11 12:55:24 2014 -0700

----------------------------------------------------------------------
 .../java/org/apache/helix/tools/ZkCopy.java     | 110 ++++++++++++++-----
 1 file changed, 81 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/28a8f20d/helix-core/src/main/java/org/apache/helix/tools/ZkCopy.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/tools/ZkCopy.java b/helix-core/src/main/java/org/apache/helix/tools/ZkCopy.java
index 3991c09..dc6d405 100644
--- a/helix-core/src/main/java/org/apache/helix/tools/ZkCopy.java
+++ b/helix-core/src/main/java/org/apache/helix/tools/ZkCopy.java
@@ -20,10 +20,11 @@ package org.apache.helix.tools;
  */
 
 import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.LinkedList;
 import java.util.List;
 
-import org.I0Itec.zkclient.exception.ZkNodeExistsException;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.GnuParser;
@@ -32,7 +33,10 @@ import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
+import org.apache.helix.AccessOption;
+import org.apache.helix.BaseDataAccessor;
 import org.apache.helix.manager.zk.ByteArraySerializer;
+import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.manager.zk.ZkClient;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.common.PathUtils;
@@ -86,57 +90,105 @@ public class ZkCopy {
     }
   }
 
-  private static void zkCopy(ZkClient srcClient, String srcPath, ZkClient dstClient, String dstPath) {
+  /**
+   * Copy a list of paths from src to dst
+   * @param srcClient
+   * @param srcRootPath
+   * @param dstClient
+   * @param dstRootPath
+   * @param paths
+   */
+  private static void copy(ZkClient srcClient, String srcRootPath, ZkClient dstClient,
+      String dstRootPath, List<String> paths) {
+    BaseDataAccessor<Object> srcAccessor = new ZkBaseDataAccessor<Object>(srcClient);
+    List<String> readPaths = new ArrayList<String>();
+    for (String path : paths) {
+      readPaths.add(concatenate(srcRootPath, path));
+    }
+    List<Stat> stats = new ArrayList<Stat>();
+    List<Object> readData = srcAccessor.get(readPaths, stats, 0);
+
+    List<String> writePaths = new ArrayList<String>();
+    List<Object> writeData = new ArrayList<Object>();
+    for (int i = 0; i < paths.size(); i++) {
+      if (stats.get(i).getEphemeralOwner() != 0) {
+        logger.warn("Skip copying ephemeral znode: " + readPaths.get(i));
+        continue;
+      }
+
+      writePaths.add(concatenate(dstRootPath, paths.get(i)));
+      writeData.add(readData.get(i));
+    }
+
+    if (writePaths.size() > 0) {
+      BaseDataAccessor<Object> dstAccessor = new ZkBaseDataAccessor<Object>(dstClient);
+      boolean[] success =
+          dstAccessor.createChildren(writePaths, writeData, AccessOption.PERSISTENT);
+      List<String> successPaths = new ArrayList<String>();
+      List<String> failPaths = new ArrayList<String>();
+      for (int i = 0; i < success.length; i++) {
+        if (success[i]) {
+          successPaths.add(writePaths.get(i));
+        } else {
+          failPaths.add(writePaths.get(i));
+        }
+      }
+
+      // Print
+      if (!successPaths.isEmpty()) {
+        System.out.println("Copy " + successPaths);
+      }
+
+      if (!failPaths.isEmpty()) {
+        System.out.println("Skip " + failPaths);
+      }
+    }
+  }
+
+  private static void zkCopy(ZkClient srcClient, String srcRootPath, ZkClient dstClient, String dstRootPath) {
     // Strip off tailing "/"
-    if (!srcPath.equals("/") && srcPath.endsWith("/")) {
-      srcPath = srcPath.substring(0, srcPath.length() - 1);
+    if (!srcRootPath.equals("/") && srcRootPath.endsWith("/")) {
+      srcRootPath = srcRootPath.substring(0, srcRootPath.length() - 1);
     }
 
-    if (!dstPath.equals("/") && dstPath.endsWith("/")) {
-      dstPath = dstPath.substring(0, dstPath.length() - 1);
+    if (!dstRootPath.equals("/") && dstRootPath.endsWith("/")) {
+      dstRootPath = dstRootPath.substring(0, dstRootPath.length() - 1);
     }
 
     // Validate paths
-    PathUtils.validatePath(srcPath);
-    PathUtils.validatePath(dstPath);
+    PathUtils.validatePath(srcRootPath);
+    PathUtils.validatePath(dstRootPath);
 
-    if (srcPath.equals(dstPath)) {
+    if (srcRootPath.equals(dstRootPath)) {
       logger.info("srcPath == dstPath. Skip copying");
       return;
     }
 
-    if (srcPath.startsWith(dstPath) || dstPath.startsWith(srcPath)) {
+    if (srcRootPath.startsWith(dstRootPath) || dstRootPath.startsWith(srcRootPath)) {
       throw new IllegalArgumentException(
-          "srcPath/dstPath can't be prefix of dstPath/srcPath, was srcPath: " + srcPath
-              + ", dstPath: " + dstPath);
+          "srcPath/dstPath can't be prefix of dstPath/srcPath, was srcPath: " + srcRootPath
+              + ", dstPath: " + dstRootPath);
     }
 
     // Recursive copy using BFS
     List<String> queue = new LinkedList<String>();
-    queue.add("");
+    String root = "";
+    copy(srcClient, srcRootPath, dstClient, dstRootPath, Arrays.asList(root));
+
+    queue.add(root);
     while (!queue.isEmpty()) {
       String path = queue.remove(0);
-      Stat stat = new Stat();
-      String fromPath = concatenate(srcPath, path);
-      Object data = srcClient.readDataAndStat(fromPath, stat, false);
-      if (stat.getEphemeralOwner() != 0) {
-        logger.warn("Skip copying ephemeral znode: " + fromPath);
-        continue;
-      }
-      String toPath = concatenate(dstPath, path);
-      try {
-        dstClient.createPersistent(toPath, data);
-        System.out.println("Copy " + fromPath + " to " + toPath);
-
-      } catch (ZkNodeExistsException e) {
-        logger.warn("Skip copying znode: " + fromPath + ", " + toPath + " already exists");
-      }
+      String fromPath = concatenate(srcRootPath, path);
 
       List<String> children = srcClient.getChildren(fromPath);
+      List<String> paths = new ArrayList<String>();
       if (children != null && children.size() > 0) {
         for (String child : children) {
-          queue.add(concatenate(path, child));
+          String childPath = concatenate(path, child);
+          paths.add(childPath);
         }
+        copy(srcClient, srcRootPath, dstClient, dstRootPath, paths);
+        queue.addAll(paths);
       }
     }
   }


[13/17] git commit: [HELIX-463] Create useful message queue length jmx, update count

Posted by ka...@apache.org.
[HELIX-463] Create useful message queue length jmx, update count


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

Branch: refs/heads/master
Commit: c24a7301297382823f3063660d77e5af31f6f95e
Parents: 2ccaa6b
Author: Kanak Biscuitwala <ka...@apache.org>
Authored: Thu Jun 26 11:12:49 2014 -0700
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Fri Jul 11 12:49:51 2014 -0700

----------------------------------------------------------------------
 .../org/apache/helix/messaging/handling/HelixTaskExecutor.java    | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/c24a7301/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
index c4d7a67..0ac3b58 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
@@ -518,6 +518,9 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
       return;
     }
 
+    // Update message count
+    _messageQueueMonitor.setMessageQueueBacklog(messages.size());
+
     // sort message by creation timestamp, so message created earlier is processed first
     Collections.sort(messages, Message.CREATE_TIME_COMPARATOR);