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 2013/10/07 21:53:47 UTC

git commit: [HELIX-18] Unify ClusterSetup and HelixAdmin

Updated Branches:
  refs/heads/helix-logical-model aa1bd8524 -> 48a5f48f3


[HELIX-18] Unify ClusterSetup and HelixAdmin


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

Branch: refs/heads/helix-logical-model
Commit: 48a5f48f319dcd5d051f9d473dc33954f56fe498
Parents: aa1bd85
Author: Kanak Biscuitwala <ka...@apache.org>
Authored: Mon Oct 7 12:53:12 2013 -0700
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Mon Oct 7 12:53:12 2013 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/helix/api/Cluster.java |  27 ++-
 .../helix/api/accessor/ClusterAccessor.java     | 162 +++++++++++++++-
 .../apache/helix/api/config/ClusterConfig.java  | 187 ++++++++++++++++++-
 .../org/apache/helix/tools/NewClusterSetup.java |  36 +++-
 4 files changed, 399 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/48a5f48f/helix-core/src/main/java/org/apache/helix/api/Cluster.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/Cluster.java b/helix-core/src/main/java/org/apache/helix/api/Cluster.java
index 04c87d4..fdeb879 100644
--- a/helix-core/src/main/java/org/apache/helix/api/Cluster.java
+++ b/helix-core/src/main/java/org/apache/helix/api/Cluster.java
@@ -32,8 +32,10 @@ import org.apache.helix.api.id.ParticipantId;
 import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.api.id.SpectatorId;
 import org.apache.helix.api.id.StateModelDefId;
+import org.apache.helix.model.Alerts;
 import org.apache.helix.model.ClusterConstraints;
 import org.apache.helix.model.ClusterConstraints.ConstraintType;
+import org.apache.helix.model.PersistentStats;
 import org.apache.helix.model.StateModelDefinition;
 import org.apache.helix.model.Transition;
 
@@ -84,6 +86,8 @@ public class Cluster {
    * @param leaderId
    * @param constraintMap
    * @param stateModelMap
+   * @param stats
+   * @param alerts
    * @param userConfig
    * @param isPaused
    * @param autoJoinAllowed
@@ -91,8 +95,8 @@ public class Cluster {
   public Cluster(ClusterId id, Map<ResourceId, Resource> resourceMap,
       Map<ParticipantId, Participant> participantMap, Map<ControllerId, Controller> controllerMap,
       ControllerId leaderId, Map<ConstraintType, ClusterConstraints> constraintMap,
-      Map<StateModelDefId, StateModelDefinition> stateModelMap, UserConfig userConfig,
-      boolean isPaused, boolean autoJoinAllowed) {
+      Map<StateModelDefId, StateModelDefinition> stateModelMap, PersistentStats stats,
+      Alerts alerts, UserConfig userConfig, boolean isPaused, boolean autoJoinAllowed) {
 
     // build the config
     // Guava's transform and "copy" functions really return views so the maps all reflect each other
@@ -114,7 +118,8 @@ public class Cluster {
         new ClusterConfig.Builder(id).addResources(resourceConfigMap.values())
             .addParticipants(participantConfigMap.values()).addConstraints(constraintMap.values())
             .addStateModelDefinitions(stateModelMap.values()).pausedStatus(isPaused)
-            .userConfig(userConfig).autoJoin(autoJoinAllowed).build();
+            .userConfig(userConfig).autoJoin(autoJoinAllowed).addStats(stats).addAlerts(alerts)
+            .build();
 
     _resourceMap = ImmutableMap.copyOf(resourceMap);
 
@@ -219,6 +224,22 @@ public class Cluster {
   }
 
   /**
+   * Get all the persisted stats for the cluster
+   * @return PersistentStats instance
+   */
+  public PersistentStats getStats() {
+    return _config.getStats();
+  }
+
+  /**
+   * Get all the persisted alerts for the cluster
+   * @return Alerts instance
+   */
+  public Alerts getAlerts() {
+    return _config.getAlerts();
+  }
+
+  /**
    * Get user-specified configuration properties of this cluster
    * @return UserConfig properties
    */

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/48a5f48f/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 6061c47..8780115 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
@@ -25,10 +25,15 @@ 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;
@@ -48,6 +53,7 @@ import org.apache.helix.api.id.SessionId;
 import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.controller.rebalancer.context.RebalancerConfig;
 import org.apache.helix.controller.rebalancer.context.RebalancerContext;
+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;
@@ -58,6 +64,7 @@ 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.ResourceAssignment;
 import org.apache.helix.model.ResourceConfiguration;
 import org.apache.helix.model.StateModelDefinition;
@@ -111,6 +118,9 @@ public class ClusterAccessor {
     if (cluster.autoJoinAllowed()) {
       clusterConfig.setAutoJoinAllowed(cluster.autoJoinAllowed());
     }
+    if (cluster.getStats() != null && !cluster.getStats().getMapFields().isEmpty()) {
+      _accessor.createProperty(_keyBuilder.persistantStat(), cluster.getStats());
+    }
     _accessor.createProperty(_keyBuilder.clusterConfig(), clusterConfig);
     if (cluster.isPaused()) {
       pauseCluster();
@@ -151,6 +161,16 @@ public class ClusterAccessor {
       ClusterConstraints constraint = constraints.get(type);
       _accessor.setProperty(_keyBuilder.constraint(type.toString()), constraint);
     }
+    if (config.getStats() == null || config.getStats().getMapFields().isEmpty()) {
+      _accessor.removeProperty(_keyBuilder.persistantStat());
+    } else {
+      _accessor.setProperty(_keyBuilder.persistantStat(), config.getStats());
+    }
+    if (config.getAlerts() == null || config.getAlerts().getMapFields().isEmpty()) {
+      _accessor.removeProperty(_keyBuilder.alerts());
+    } else {
+      _accessor.setProperty(_keyBuilder.alerts(), config.getAlerts());
+    }
     return true;
   }
 
@@ -229,9 +249,15 @@ public class ClusterAccessor {
     // read the state model definitions
     Map<StateModelDefId, StateModelDefinition> stateModelMap = readStateModelDefinitions();
 
+    // read the stats
+    PersistentStats stats = _accessor.getProperty(_keyBuilder.persistantStat());
+
+    // read the alerts
+    Alerts alerts = _accessor.getProperty(_keyBuilder.alerts());
+
     // create the cluster snapshot object
     return new Cluster(_clusterId, resourceMap, participantMap, controllerMap, leaderId,
-        clusterConstraintMap, stateModelMap, userConfig, isPaused, autoJoinAllowed);
+        clusterConstraintMap, stateModelMap, stats, alerts, userConfig, isPaused, autoJoinAllowed);
   }
 
   /**
@@ -404,6 +430,140 @@ 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());
+  }
+
+  /**
+   * 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()) {
+      throw new HelixException("cluster " + _clusterId + " is not setup yet");
+    }
+
+    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()) {
+      throw new HelixException("cluster " + _clusterId + " is not setup yet");
+    }
+
+    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()) {
+      throw new HelixException("cluster " + _clusterId + " is not setup yet");
+    }
+
+    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()) {
+      throw new HelixException("cluster " + _clusterId + " is not setup yet");
+    }
+
+    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/incubator-helix/blob/48a5f48f/helix-core/src/main/java/org/apache/helix/api/config/ClusterConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/config/ClusterConfig.java b/helix-core/src/main/java/org/apache/helix/api/config/ClusterConfig.java
index 1cc09e3..ed9750a 100644
--- a/helix-core/src/main/java/org/apache/helix/api/config/ClusterConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/api/config/ClusterConfig.java
@@ -5,6 +5,8 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.helix.alerts.AlertsHolder;
+import org.apache.helix.alerts.StatsHolder;
 import org.apache.helix.api.Scope;
 import org.apache.helix.api.State;
 import org.apache.helix.api.id.ClusterId;
@@ -12,12 +14,14 @@ import org.apache.helix.api.id.ConstraintId;
 import org.apache.helix.api.id.ParticipantId;
 import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.api.id.StateModelDefId;
+import org.apache.helix.model.Alerts;
 import org.apache.helix.model.ClusterConstraints;
 import org.apache.helix.model.ClusterConstraints.ConstraintAttribute;
 import org.apache.helix.model.ClusterConstraints.ConstraintType;
 import org.apache.helix.model.ClusterConstraints.ConstraintValue;
 import org.apache.helix.model.ConstraintItem;
 import org.apache.helix.model.Message.MessageType;
+import org.apache.helix.model.PersistentStats;
 import org.apache.helix.model.StateModelDefinition;
 import org.apache.helix.model.Transition;
 import org.apache.helix.model.builder.ConstraintItemBuilder;
@@ -57,6 +61,8 @@ public class ClusterConfig {
   private final Map<ParticipantId, ParticipantConfig> _participantMap;
   private final Map<ConstraintType, ClusterConstraints> _constraintMap;
   private final Map<StateModelDefId, StateModelDefinition> _stateModelMap;
+  private final PersistentStats _stats;
+  private final Alerts _alerts;
   private final UserConfig _userConfig;
   private final boolean _isPaused;
   private final boolean _autoJoin;
@@ -68,6 +74,8 @@ public class ClusterConfig {
    * @param participantMap map of participant id to participant config
    * @param constraintMap map of constraint type to all constraints of that type
    * @param stateModelMap map of state model id to state model definition
+   * @param stats statistics to watch on the cluster
+   * @param alerts alerts that the cluster can trigger
    * @param userConfig user-defined cluster properties
    * @param isPaused true if paused, false if active
    * @param allowAutoJoin true if participants can join automatically, false otherwise
@@ -75,13 +83,15 @@ public class ClusterConfig {
   private ClusterConfig(ClusterId id, Map<ResourceId, ResourceConfig> resourceMap,
       Map<ParticipantId, ParticipantConfig> participantMap,
       Map<ConstraintType, ClusterConstraints> constraintMap,
-      Map<StateModelDefId, StateModelDefinition> stateModelMap, UserConfig userConfig,
-      boolean isPaused, boolean allowAutoJoin) {
+      Map<StateModelDefId, StateModelDefinition> stateModelMap, PersistentStats stats,
+      Alerts alerts, UserConfig userConfig, boolean isPaused, boolean allowAutoJoin) {
     _id = id;
     _resourceMap = ImmutableMap.copyOf(resourceMap);
     _participantMap = ImmutableMap.copyOf(participantMap);
     _constraintMap = ImmutableMap.copyOf(constraintMap);
     _stateModelMap = ImmutableMap.copyOf(stateModelMap);
+    _stats = stats;
+    _alerts = alerts;
     _userConfig = userConfig;
     _isPaused = isPaused;
     _autoJoin = allowAutoJoin;
@@ -227,6 +237,22 @@ public class ClusterConfig {
   }
 
   /**
+   * Get all the statistics persisted on the cluster
+   * @return PersistentStats instance
+   */
+  public PersistentStats getStats() {
+    return _stats;
+  }
+
+  /**
+   * Get all the alerts persisted on the cluster
+   * @return Alerts instance
+   */
+  public Alerts getAlerts() {
+    return _alerts;
+  }
+
+  /**
    * Get user-specified configuration properties of this cluster
    * @return UserConfig properties
    */
@@ -261,6 +287,8 @@ public class ClusterConfig {
 
     private Set<Fields> _updateFields;
     private Map<ConstraintType, Set<ConstraintId>> _removedConstraints;
+    private PersistentStats _removedStats;
+    private Alerts _removedAlerts;
     private Builder _builder;
 
     /**
@@ -274,6 +302,8 @@ public class ClusterConfig {
         Set<ConstraintId> constraints = Sets.newHashSet();
         _removedConstraints.put(type, constraints);
       }
+      _removedStats = new PersistentStats(PersistentStats.nodeName);
+      _removedAlerts = new Alerts(Alerts.nodeName);
       _builder = new Builder(clusterId);
     }
 
@@ -401,6 +431,57 @@ public class ClusterConfig {
     }
 
     /**
+     * Add a statistic specification to the cluster. Existing specifications will not be overwritten
+     * @param stat string specifying the stat specification
+     * @return Delta
+     */
+    public Delta addStat(String stat) {
+      _builder.addStat(stat);
+      return this;
+    }
+
+    /**
+     * Add an alert specification for the cluster. Existing specifications will not be overwritten
+     * @param alert string specifying the alert specification
+     * @return Delta
+     */
+    public Delta addAlert(String alert) {
+      _builder.addAlert(alert);
+      return this;
+    }
+
+    /**
+     * Remove a statistic specification from the cluster
+     * @param stat statistic specification
+     * @return Delta
+     */
+    public Delta removeStat(String stat) {
+      Map<String, Map<String, String>> parsedStat = StatsHolder.parseStat(stat);
+      Map<String, Map<String, String>> currentStats = _removedStats.getMapFields();
+      for (String statName : parsedStat.keySet()) {
+        currentStats.put(statName, parsedStat.get(statName));
+      }
+      return this;
+    }
+
+    /**
+     * Remove an alert specification for the cluster
+     * @param alert alert specification
+     * @return Delta
+     */
+    public Delta removeAlert(String alert) {
+      Map<String, Map<String, String>> currAlertMap = _removedAlerts.getMapFields();
+      if (!currAlertMap.containsKey(alert)) {
+        Map<String, String> parsedAlert = Maps.newHashMap();
+        StringBuilder statsName = new StringBuilder();
+        AlertsHolder.parseAlert(alert, statsName, parsedAlert);
+        removeStat(statsName.toString());
+        currAlertMap.put(alert, parsedAlert);
+      }
+      return this;
+    }
+
+    /**
      * Create a ClusterConfig that is the combination of an existing ClusterConfig and this delta
      * @param orig the original ClusterConfig
      * @return updated ClusterConfig
@@ -413,7 +494,8 @@ public class ClusterConfig {
               .addParticipants(orig.getParticipantMap().values())
               .addStateModelDefinitions(orig.getStateModelMap().values())
               .userConfig(orig.getUserConfig()).pausedStatus(orig.isPaused())
-              .autoJoin(orig.autoJoinAllowed());
+              .autoJoin(orig.autoJoinAllowed()).addStats(orig.getStats())
+              .addAlerts(orig.getAlerts());
       for (Fields field : _updateFields) {
         switch (field) {
         case USER_CONFIG:
@@ -446,7 +528,30 @@ public class ClusterConfig {
         }
         builder.addConstraint(constraints);
       }
-      return builder.build();
+
+      // add stats and alerts
+      builder.addStats(deltaConfig.getStats());
+      builder.addAlerts(deltaConfig.getAlerts());
+
+      // get the result
+      ClusterConfig result = builder.build();
+
+      // remove stats
+      PersistentStats stats = result.getStats();
+      for (String removedStat : _removedStats.getMapFields().keySet()) {
+        if (stats.getMapFields().containsKey(removedStat)) {
+          stats.getMapFields().remove(removedStat);
+        }
+      }
+
+      // remove alerts
+      Alerts alerts = result.getAlerts();
+      for (String removedAlert : _removedAlerts.getMapFields().keySet()) {
+        if (alerts.getMapFields().containsKey(removedAlert)) {
+          alerts.getMapFields().remove(removedAlert);
+        }
+      }
+      return result;
     }
   }
 
@@ -460,6 +565,8 @@ public class ClusterConfig {
     private final Map<ConstraintType, ClusterConstraints> _constraintMap;
     private final Map<StateModelDefId, StateModelDefinition> _stateModelMap;
     private UserConfig _userConfig;
+    private PersistentStats _stats;
+    private Alerts _alerts;
     private boolean _isPaused;
     private boolean _autoJoin;
 
@@ -476,6 +583,8 @@ public class ClusterConfig {
       _isPaused = false;
       _autoJoin = false;
       _userConfig = new UserConfig(Scope.cluster(id));
+      _stats = new PersistentStats(PersistentStats.nodeName);
+      _alerts = new Alerts(Alerts.nodeName);
     }
 
     /**
@@ -680,6 +789,74 @@ public class ClusterConfig {
     }
 
     /**
+     * Add a statistic specification to the cluster. Existing specifications will not be overwritten
+     * @param stat String specifying the stat specification
+     * @return Builder
+     */
+    public Builder addStat(String stat) {
+      Map<String, Map<String, String>> parsedStat = StatsHolder.parseStat(stat);
+      Map<String, Map<String, String>> currentStats = _stats.getMapFields();
+      for (String statName : parsedStat.keySet()) {
+        if (!currentStats.containsKey(statName)) {
+          currentStats.put(statName, parsedStat.get(statName));
+        }
+      }
+      return this;
+    }
+
+    /**
+     * Add statistic specifications to the cluster. Existing specifications will not be overwritten
+     * @param stats PersistentStats specifying the stat specification
+     * @return Builder
+     */
+    public Builder addStats(PersistentStats stats) {
+      if (stats == null) {
+        return this;
+      }
+      Map<String, Map<String, String>> parsedStat = stats.getMapFields();
+      Map<String, Map<String, String>> currentStats = _stats.getMapFields();
+      for (String statName : parsedStat.keySet()) {
+        if (!currentStats.containsKey(statName)) {
+          currentStats.put(statName, parsedStat.get(statName));
+        }
+      }
+      return this;
+    }
+
+    /**
+     * Add alert specifications to the cluster. Existing specifications will not be overwritten
+     * @param alert string representing alert specifications
+     * @return Builder
+     */
+    public Builder addAlert(String alert) {
+      Map<String, Map<String, String>> currAlertMap = _alerts.getMapFields();
+      if (!currAlertMap.containsKey(alert)) {
+        Map<String, String> parsedAlert = Maps.newHashMap();
+        StringBuilder statsName = new StringBuilder();
+        AlertsHolder.parseAlert(alert, statsName, parsedAlert);
+        addStat(statsName.toString());
+        currAlertMap.put(alert, parsedAlert);
+      }
+      return this;
+    }
+
+    /**
+     * Add alert specifications to the cluster. Existing specifications will not be overwritten
+     * @param alerts Alerts instance
+     * @return Builder
+     */
+    public Builder addAlerts(Alerts alerts) {
+      if (alerts == null) {
+        return this;
+      }
+      Map<String, Map<String, String>> alertMap = alerts.getMapFields();
+      for (String alert : alertMap.keySet()) {
+        addAlert(alert);
+      }
+      return this;
+    }
+
+    /**
      * Set the paused status of the cluster
      * @param isPaused true if paused, false otherwise
      * @return Builder
@@ -715,7 +892,7 @@ public class ClusterConfig {
      */
     public ClusterConfig build() {
       return new ClusterConfig(_id, _resourceMap, _participantMap, _constraintMap, _stateModelMap,
-          _userConfig, _isPaused, _autoJoin);
+          _stats, _alerts, _userConfig, _isPaused, _autoJoin);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/48a5f48f/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 4c7fc66..85330be 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
@@ -891,6 +891,34 @@ 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];
@@ -1092,16 +1120,16 @@ public class NewClusterSetup {
           setup.resetResource(optValues);
           break;
         case addStat:
-          // TODO impl ClusterAccessor.addStat()
+          setup.addStat(optValues);
           break;
         case addAlert:
-          // TODO impl ClusterAccessor#addAlert()
+          setup.addAlert(optValues);
           break;
         case dropStat:
-          // TODO impl ClusterAccessor.dropStat()
+          setup.dropStat(optValues);
           break;
         case dropAlert:
-          // TODO impl ClusterAccessor#dropAlert()
+          setup.dropAlert(optValues);
           break;
         case getConfig:
           setup.getConfig(optValues);