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:58:01 UTC

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

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);
-  }
-}