You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by jo...@apache.org on 2014/08/13 22:01:16 UTC

[6/6] git commit: AMBARI-6853 - Alerts: Calculate Hash Based On Alert Definitions (jonathanhurley)

AMBARI-6853 - Alerts: Calculate Hash Based On Alert Definitions (jonathanhurley)


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

Branch: refs/heads/branch-alerts-dev
Commit: 84b988bcb4a6d127a1bb0a284e34f157ccf78fce
Parents: 1b72f6d
Author: Jonathan Hurley <jh...@hortonworks.com>
Authored: Wed Aug 13 15:39:35 2014 -0400
Committer: Jonathan Hurley <jh...@hortonworks.com>
Committed: Wed Aug 13 16:00:08 2014 -0400

----------------------------------------------------------------------
 .../ambari/server/agent/HeartBeatHandler.java   |  93 ++++---
 .../ambari/server/agent/HeartBeatResponse.java  |  54 ++--
 .../AlertDefinitionResourceProvider.java        | 139 +++++-----
 .../server/orm/dao/AlertDefinitionDAO.java      | 129 ++++++++-
 .../orm/entities/AlertDefinitionEntity.java     |   9 +-
 .../server/state/alert/AlertDefinitionHash.java | 272 +++++++++++++++++++
 .../src/main/resources/properties.json          |   3 +-
 .../AlertDefinitionResourceProviderTest.java    | 134 ++++-----
 .../server/orm/dao/AlertDefinitionDAOTest.java  | 117 +++++++-
 .../state/alerts/AlertDefinitionHashTest.java   | 224 +++++++++++++++
 10 files changed, 969 insertions(+), 205 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/84b988bc/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java b/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java
index fa633c1..8a818a6 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java
@@ -17,20 +17,16 @@
  */
 package org.apache.ambari.server.agent;
 
-import com.google.gson.Gson;
-import com.google.inject.Inject;
-import com.google.inject.Injector;
-import com.google.inject.Singleton;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.regex.Pattern;
+
 import org.apache.ambari.server.AmbariException;
 import org.apache.ambari.server.HostNotFoundException;
 import org.apache.ambari.server.RoleCommand;
@@ -46,7 +42,6 @@ import org.apache.ambari.server.controller.MaintenanceStateHelper;
 import org.apache.ambari.server.metadata.ActionMetadata;
 import org.apache.ambari.server.state.AgentVersion;
 import org.apache.ambari.server.state.Alert;
-import org.apache.ambari.server.state.AlertState;
 import org.apache.ambari.server.state.Cluster;
 import org.apache.ambari.server.state.Clusters;
 import org.apache.ambari.server.state.ComponentInfo;
@@ -63,6 +58,7 @@ import org.apache.ambari.server.state.ServiceInfo;
 import org.apache.ambari.server.state.StackId;
 import org.apache.ambari.server.state.StackInfo;
 import org.apache.ambari.server.state.State;
+import org.apache.ambari.server.state.alert.AlertDefinitionHash;
 import org.apache.ambari.server.state.fsm.InvalidStateTransitionException;
 import org.apache.ambari.server.state.host.HostHealthyHeartbeatEvent;
 import org.apache.ambari.server.state.host.HostRegistrationRequestEvent;
@@ -78,6 +74,11 @@ import org.apache.ambari.server.utils.VersionUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
+import com.google.gson.Gson;
+import com.google.inject.Inject;
+import com.google.inject.Injector;
+import com.google.inject.Singleton;
+
 
 /**
  * This class handles the heartbeats coming from the agent, passes on the information
@@ -90,29 +91,40 @@ public class HeartBeatHandler {
   private final Clusters clusterFsm;
   private final ActionQueue actionQueue;
   private final ActionManager actionManager;
+  private HeartbeatMonitor heartbeatMonitor;
+
   @Inject
-  Injector injector;
+  private Injector injector;
+
   @Inject
-  Configuration config;
+  private Configuration config;
+
   @Inject
-  AmbariMetaInfo ambariMetaInfo;
+  private AmbariMetaInfo ambariMetaInfo;
+
   @Inject
-  ActionMetadata actionMetadata;
-  private HeartbeatMonitor heartbeatMonitor;
+  private ActionMetadata actionMetadata;
+
   @Inject
   private Gson gson;
+
   @Inject
-  ConfigHelper configHelper;
+  private ConfigHelper configHelper;
+
+  @Inject
+  private AlertDefinitionHash alertDefinitionHash;
+
   private Map<String, Long> hostResponseIds = new ConcurrentHashMap<String, Long>();
+
   private Map<String, HeartBeatResponse> hostResponses = new ConcurrentHashMap<String, HeartBeatResponse>();
 
   @Inject
   public HeartBeatHandler(Clusters fsm, ActionQueue aq, ActionManager am,
                           Injector injector) {
-    this.clusterFsm = fsm;
-    this.actionQueue = aq;
-    this.actionManager = am;
-    this.heartbeatMonitor = new HeartbeatMonitor(fsm, aq, am, 60000, injector);
+    clusterFsm = fsm;
+    actionQueue = aq;
+    actionManager = am;
+    heartbeatMonitor = new HeartbeatMonitor(fsm, aq, am, 60000, injector);
     injector.injectMembers(this);
   }
 
@@ -130,14 +142,17 @@ public class HeartBeatHandler {
     if(heartbeat.getAgentEnv() != null && heartbeat.getAgentEnv().getHostHealth() != null) {
       heartbeat.getAgentEnv().getHostHealth().setServerTimeStampAtReporting(now);
     }
+
     String hostname = heartbeat.getHostname();
     Long currentResponseId = hostResponseIds.get(hostname);
     HeartBeatResponse response;
+
     if (currentResponseId == null) {
       //Server restarted, or unknown host.
       LOG.error("CurrentResponseId unknown for " + hostname + " - send register command");
       return createRegisterCommand();
     }
+
     LOG.debug("Received heartbeat from host"
         + ", hostname=" + hostname
         + ", currentResponseId=" + currentResponseId
@@ -195,18 +210,23 @@ public class HeartBeatHandler {
 
     // Examine heartbeart for component live status reports
     processStatusReports(heartbeat, hostname, clusterFsm);
-    
+
     // Calculate host status
     // NOTE: This step must be after processing command/status reports
     processHostStatus(heartbeat, hostname);
-    
+
     calculateHostAlerts(heartbeat, hostname);
 
     // Send commands if node is active
     if (hostObject.getState().equals(HostState.HEALTHY)) {
       sendCommands(hostname, response);
       annotateResponse(hostname, response);
-    }    
+    }
+
+    // send the alert definition hash for this host
+    Map<String, String> alertDefinitionHashes = alertDefinitionHash.getHashes(hostname);
+    response.setAlertDefinitionHash(alertDefinitionHashes);
+
     return response;
   }
 
@@ -218,7 +238,7 @@ public class HeartBeatHandler {
         }
       }
   }
-   
+
   protected void processHostStatus(HeartBeat heartbeat, String hostname) throws AmbariException {
 
     Host host = clusterFsm.getHost(hostname);
@@ -268,7 +288,7 @@ public class HeartBeatHandler {
         StackId stackId;
         Cluster cluster = clusterFsm.getCluster(clusterName);
         stackId = cluster.getDesiredStackVersion();
-        
+
         MaintenanceStateHelper psh = injector.getInstance(MaintenanceStateHelper.class);
 
         List<ServiceComponentHost> scHosts = cluster.getServiceComponentHosts(heartbeat.getHostname());
@@ -347,7 +367,7 @@ public class HeartBeatHandler {
          "STOP".equals(report.getCustomCommand())))) {
         continue;
       }
-      
+
       Cluster cl = clusterFsm.getCluster(report.getClusterName());
       String service = report.getServiceName();
       if (service == null || service.isEmpty()) {
@@ -439,7 +459,7 @@ public class HeartBeatHandler {
         if (status.getClusterName().equals(cl.getClusterName())) {
           try {
             Service svc = cl.getService(status.getServiceName());
-            
+
             String componentName = status.getComponentName();
             if (svc.getServiceComponents().containsKey(componentName)) {
               ServiceComponent svcComp = svc.getServiceComponent(
@@ -470,7 +490,7 @@ public class HeartBeatHandler {
               if (null != status.getConfigTags()) {
                 scHost.updateActualConfigs(status.getConfigTags());
               }
-              
+
               Map<String, Object> extra = status.getExtra();
               if (null != extra && !extra.isEmpty()) {
                 try {
@@ -479,7 +499,7 @@ public class HeartBeatHandler {
                     List<Map<String, String>> list = (List<Map<String, String>>) extra.get("processes");
                     scHost.setProcesses(list);
                   }
-                  
+
                 } catch (Exception e) {
                   LOG.error("Could not access extra JSON for " +
                       scHost.getServiceComponentName() + " from " +
@@ -487,7 +507,7 @@ public class HeartBeatHandler {
                       " (" + e.getMessage() + ")");
                 }
               }
-              
+
               if (null != status.getAlerts()) {
                 List<Alert> clusterAlerts = new ArrayList<Alert>();
                 for (AgentAlert aa : status.getAlerts()) {
@@ -496,14 +516,15 @@ public class HeartBeatHandler {
                       scHost.getHostName(), aa.getState());
                   alert.setLabel(aa.getLabel());
                   alert.setText(aa.getText());
-                  
+
                   clusterAlerts.add(alert);
                 }
-                
-               if (0 != clusterAlerts.size())
-                 cl.addAlerts(clusterAlerts);
+
+               if (0 != clusterAlerts.size()) {
+                cl.addAlerts(clusterAlerts);
               }
-              
+              }
+
 
             } else {
               // TODO: What should be done otherwise?
@@ -563,7 +584,7 @@ public class HeartBeatHandler {
           throw new AmbariException("Could not get jaxb string for command", e);
         }
         switch (ac.getCommandType()) {
-          case BACKGROUND_EXECUTION_COMMAND: 
+          case BACKGROUND_EXECUTION_COMMAND:
           case EXECUTION_COMMAND: {
             response.addExecutionCommand((ExecutionCommand) ac);
             break;
@@ -699,7 +720,7 @@ public class HeartBeatHandler {
    * @throws org.apache.ambari.server.AmbariException
    */
   private void annotateResponse(String hostname, HeartBeatResponse response) throws AmbariException {
-    for (Cluster cl : this.clusterFsm.getClustersForHost(hostname)) {
+    for (Cluster cl : clusterFsm.getClustersForHost(hostname)) {
       List<ServiceComponentHost> scHosts = cl.getServiceComponentHosts(hostname);
       if (scHosts != null && scHosts.size() > 0) {
         response.setHasMappedComponents(true);
@@ -718,19 +739,19 @@ public class HeartBeatHandler {
       throws AmbariException {
     ComponentsResponse response = new ComponentsResponse();
 
-    Cluster cluster = this.clusterFsm.getCluster(clusterName);
+    Cluster cluster = clusterFsm.getCluster(clusterName);
     StackId stackId = cluster.getCurrentStackVersion();
     if (stackId == null) {
       throw new AmbariException("Cannot provide stack components map. " +
         "Stack hasn't been selected yet.");
     }
-    StackInfo stack = this.ambariMetaInfo.getStackInfo(stackId.getStackName(),
+    StackInfo stack = ambariMetaInfo.getStackInfo(stackId.getStackName(),
         stackId.getStackVersion());
 
     response.setClusterName(clusterName);
     response.setStackName(stackId.getStackName());
     response.setStackVersion(stackId.getStackVersion());
-    response.setComponents(this.getComponentsMap(stack));
+    response.setComponents(getComponentsMap(stack));
 
     return response;
   }

http://git-wip-us.apache.org/repos/asf/ambari/blob/84b988bc/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatResponse.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatResponse.java b/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatResponse.java
index 1670beb..24bd8a2 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatResponse.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatResponse.java
@@ -20,26 +20,34 @@ package org.apache.ambari.server.agent;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 import org.codehaus.jackson.annotate.JsonProperty;
 
 /**
- *
  * Controller to Agent response data model.
- *
  */
 public class HeartBeatResponse {
 
   private long responseId;
- 
-  List<ExecutionCommand> executionCommands = new ArrayList<ExecutionCommand>();
-  List<StatusCommand> statusCommands = new ArrayList<StatusCommand>();
-  List<CancelCommand> cancelCommands = new ArrayList<CancelCommand>();
 
-  RegistrationCommand registrationCommand;
+  private List<ExecutionCommand> executionCommands = new ArrayList<ExecutionCommand>();
+  private List<StatusCommand> statusCommands = new ArrayList<StatusCommand>();
+  private List<CancelCommand> cancelCommands = new ArrayList<CancelCommand>();
+
+  private RegistrationCommand registrationCommand;
+
+  private boolean restartAgent = false;
+  private boolean hasMappedComponents = false;
 
-  boolean restartAgent = false;
-  boolean hasMappedComponents = false;
+  /**
+   * A mapping between cluster name and the alert defintion hash for that
+   * cluster. The alert definition hash for a cluster is a hashed value of all
+   * of the UUIDs for each alert definition that the agent host should be
+   * scheduling. If any of the alert definitions change, their UUID will change
+   * which will cause this hash value to change.
+   */
+  private Map<String, String> alertDefinitionHashes = null;
 
   @JsonProperty("responseId")
   public long getResponseId() {
@@ -111,6 +119,16 @@ public class HeartBeatResponse {
     this.hasMappedComponents = hasMappedComponents;
   }
 
+  @JsonProperty("alertDefinitionHashes")
+  public Map<String, String> getAlertDefinitionHash() {
+    return alertDefinitionHashes;
+  }
+
+  @JsonProperty("alertDefinitionHashes")
+  public void setAlertDefinitionHash(Map<String, String> alertDefinitionHashes) {
+    this.alertDefinitionHashes = alertDefinitionHashes;
+  }
+
   public void addExecutionCommand(ExecutionCommand execCmd) {
     executionCommands.add(execCmd);
   }
@@ -125,13 +143,15 @@ public class HeartBeatResponse {
 
   @Override
   public String toString() {
-    return "HeartBeatResponse{" +
-            "responseId=" + responseId +
-            ", executionCommands=" + executionCommands +
-            ", statusCommands=" + statusCommands +
-            ", cancelCommands=" + cancelCommands +
-            ", registrationCommand=" + registrationCommand +
-            ", restartAgent=" + restartAgent +
-            '}';
+    StringBuilder buffer = new StringBuilder("HeartBeatResponse{");
+    buffer.append("responseId=").append(responseId);
+    buffer.append(", executionCommands=").append(executionCommands);
+    buffer.append(", statusCommands=").append(statusCommands);
+    buffer.append(", cancelCommands=").append(cancelCommands);
+    buffer.append(", registrationCommand=").append(registrationCommand);
+    buffer.append(", restartAgent=").append(restartAgent);
+    buffer.append(", alertDefinitionHashes=").append(alertDefinitionHashes);
+    buffer.append('}');
+    return buffer.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/84b988bc/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/AlertDefinitionResourceProvider.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/AlertDefinitionResourceProvider.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/AlertDefinitionResourceProvider.java
index 6f00c27..fd8210b 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/AlertDefinitionResourceProvider.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/AlertDefinitionResourceProvider.java
@@ -64,13 +64,14 @@ public class AlertDefinitionResourceProvider extends AbstractControllerResourceP
   protected static final String ALERT_DEF_COMPONENT_NAME = "AlertDefinition/component_name";
   protected static final String ALERT_DEF_ENABLED = "AlertDefinition/enabled";
   protected static final String ALERT_DEF_SCOPE = "AlertDefinition/scope";
-  
+  protected static final String ALERT_DEF_UUID = "AlertDefinition/uuid";
+
   private static Set<String> pkPropertyIds = new HashSet<String>(
       Arrays.asList(ALERT_DEF_ID, ALERT_DEF_NAME));
   private static AlertDefinitionDAO alertDefinitionDAO = null;
-  
+
   private static Gson gson = new Gson();
-  
+
   /**
    * @param instance
    */
@@ -78,13 +79,13 @@ public class AlertDefinitionResourceProvider extends AbstractControllerResourceP
   public static void init(AlertDefinitionDAO instance) {
     alertDefinitionDAO = instance;
   }
-  
+
   AlertDefinitionResourceProvider(Set<String> propertyIds,
       Map<Resource.Type, String> keyPropertyIds,
       AmbariManagementController managementController) {
     super(propertyIds, keyPropertyIds, managementController);
   }
-  
+
   @Override
   protected Set<String> getPKPropertyIds() {
     return pkPropertyIds;
@@ -103,49 +104,55 @@ public class AlertDefinitionResourceProvider extends AbstractControllerResourceP
       }
     });
     notifyCreate(Resource.Type.AlertDefinition, request);
-    
+
     return getRequestStatus(null);
   }
-  
+
   private void createAlertDefinitions(Set<Map<String, Object>> requestMaps)
     throws AmbariException {
     List<AlertDefinitionEntity> entities = new ArrayList<AlertDefinitionEntity>();
-    
+
     for (Map<String, Object> requestMap : requestMaps) {
       entities.add(toCreateEntity(requestMap));
     }
 
     // !!! TODO multi-create in a transaction
-    for (AlertDefinitionEntity entity : entities)
+    for (AlertDefinitionEntity entity : entities) {
       alertDefinitionDAO.create(entity);
+    }
   }
-  
+
   private AlertDefinitionEntity toCreateEntity(Map<String, Object> requestMap)
     throws AmbariException {
 
     String clusterName = (String) requestMap.get(ALERT_DEF_CLUSTER_NAME);
-    
-    if (null == clusterName || clusterName.isEmpty())
+
+    if (null == clusterName || clusterName.isEmpty()) {
       throw new IllegalArgumentException("Invalid argument, cluster name is required");
-    
-    if (!requestMap.containsKey(ALERT_DEF_INTERVAL))
+    }
+
+    if (!requestMap.containsKey(ALERT_DEF_INTERVAL)) {
       throw new IllegalArgumentException("Check interval must be specified");
-    
+    }
+
     Integer interval = Integer.valueOf((String) requestMap.get(ALERT_DEF_INTERVAL));
 
-    if (!requestMap.containsKey(ALERT_DEF_NAME))
+    if (!requestMap.containsKey(ALERT_DEF_NAME)) {
       throw new IllegalArgumentException("Definition name must be specified");
-    
-    if (!requestMap.containsKey(ALERT_DEF_SERVICE_NAME))
+    }
+
+    if (!requestMap.containsKey(ALERT_DEF_SERVICE_NAME)) {
       throw new IllegalArgumentException("Service name must be specified");
-    
-    if (!requestMap.containsKey(ALERT_DEF_SOURCE_TYPE))
+    }
+
+    if (!requestMap.containsKey(ALERT_DEF_SOURCE_TYPE)) {
       throw new IllegalArgumentException(String.format(
           "Source type must be specified and one of %s", EnumSet.allOf(
               SourceType.class)));
+    }
 
     JsonObject jsonObj = new JsonObject();
-    
+
     for (Entry<String, Object> entry : requestMap.entrySet()) {
       String propCat = PropertyHelper.getPropertyCategory(entry.getKey());
       String propName = PropertyHelper.getPropertyName(entry.getKey());
@@ -155,11 +162,12 @@ public class AlertDefinitionResourceProvider extends AbstractControllerResourceP
       }
     }
 
-    if (0 == jsonObj.entrySet().size())
+    if (0 == jsonObj.entrySet().size()) {
       throw new IllegalArgumentException("Source must be specified");
-    
+    }
+
     Cluster cluster = getManagementController().getClusters().getCluster(clusterName);
-    
+
     AlertDefinitionEntity entity = new AlertDefinitionEntity();
     entity.setClusterId(Long.valueOf(cluster.getClusterId()));
     entity.setComponentName((String) requestMap.get(ALERT_DEF_COMPONENT_NAME));
@@ -167,14 +175,14 @@ public class AlertDefinitionResourceProvider extends AbstractControllerResourceP
 
     boolean enabled = requestMap.containsKey(ALERT_DEF_ENABLED) ?
         Boolean.parseBoolean((String)requestMap.get(ALERT_DEF_ENABLED)) : true;
-    
+
     entity.setEnabled(enabled);
     entity.setHash(UUID.randomUUID().toString());
     entity.setScheduleInterval(interval);
     entity.setServiceName((String) requestMap.get(ALERT_DEF_SERVICE_NAME));
     entity.setSourceType((String) requestMap.get(ALERT_DEF_SOURCE_TYPE));
     entity.setSource(jsonObj.toString());
-    
+
     Scope scope = null;
     String desiredScope = (String) requestMap.get(ALERT_DEF_SCOPE);
     if (null != desiredScope && desiredScope.length() > 0) {
@@ -190,17 +198,18 @@ public class AlertDefinitionResourceProvider extends AbstractControllerResourceP
   public Set<Resource> getResources(Request request, Predicate predicate)
       throws SystemException, UnsupportedPropertyException,
       NoSuchResourceException, NoSuchParentResourceException {
-    
+
     Set<String> requestPropertyIds = getRequestPropertyIds(request, predicate);
-    
+
     Set<Resource> results = new HashSet<Resource>();
-    
+
     for (Map<String, Object> propertyMap : getPropertyMaps(predicate)) {
       String clusterName = (String) propertyMap.get(ALERT_DEF_CLUSTER_NAME);
-      
-      if (null == clusterName || clusterName.isEmpty())
+
+      if (null == clusterName || clusterName.isEmpty()) {
         throw new IllegalArgumentException("Invalid argument, cluster name is required");
-      
+      }
+
       String id = (String) propertyMap.get(ALERT_DEF_ID);
       if (null != id) {
         AlertDefinitionEntity entity = alertDefinitionDAO.findById(Long.parseLong(id));
@@ -208,14 +217,14 @@ public class AlertDefinitionResourceProvider extends AbstractControllerResourceP
           results.add(toResource(false, clusterName, entity, requestPropertyIds));
         }
       } else {
-        
+
         Cluster cluster = null;
         try {
           cluster = getManagementController().getClusters().getCluster(clusterName);
         } catch (AmbariException e) {
           throw new NoSuchResourceException("Parent Cluster resource doesn't exist", e);
         }
-        
+
         List<AlertDefinitionEntity> entities = alertDefinitionDAO.findAll(
             cluster.getClusterId());
 
@@ -224,7 +233,7 @@ public class AlertDefinitionResourceProvider extends AbstractControllerResourceP
         }
       }
     }
-    
+
     return results;
   }
 
@@ -236,40 +245,44 @@ public class AlertDefinitionResourceProvider extends AbstractControllerResourceP
     for (Map<String, Object> requestPropMap : request.getProperties()) {
       for (Map<String, Object> propertyMap : getPropertyMaps(requestPropMap, predicate)) {
         Long id = (Long) propertyMap.get(ALERT_DEF_ID);
-        
+
         AlertDefinitionEntity entity = alertDefinitionDAO.findById(id.longValue());
-        if (null == entity)
+        if (null == entity) {
           continue;
+        }
 
-        if (propertyMap.containsKey(ALERT_DEF_NAME))
+        if (propertyMap.containsKey(ALERT_DEF_NAME)) {
           entity.setDefinitionName((String) propertyMap.get(ALERT_DEF_NAME));
-        
+        }
+
         if (propertyMap.containsKey(ALERT_DEF_ENABLED)) {
           entity.setEnabled(Boolean.parseBoolean(
               (String) propertyMap.get(ALERT_DEF_ENABLED)));
         }
-        
+
         if (propertyMap.containsKey(ALERT_DEF_INTERVAL)) {
           entity.setScheduleInterval(Integer.valueOf(
               (String) propertyMap.get(ALERT_DEF_INTERVAL)));
         }
-                
+
         if (propertyMap.containsKey(ALERT_DEF_SCOPE)){
           Scope scope = null;
           String desiredScope = (String) propertyMap.get(ALERT_DEF_SCOPE);
-              
-          if (null != desiredScope && desiredScope.length() > 0)
+
+          if (null != desiredScope && desiredScope.length() > 0) {
             scope = Scope.valueOf((desiredScope));
-         
+          }
+
           entity.setScope(scope);
         }
-        
 
-        if (propertyMap.containsKey(ALERT_DEF_SOURCE_TYPE))
+
+        if (propertyMap.containsKey(ALERT_DEF_SOURCE_TYPE)) {
           entity.setSourceType((String) propertyMap.get(ALERT_DEF_SOURCE_TYPE));
-          
+        }
+
         JsonObject jsonObj = new JsonObject();
-        
+
         for (Entry<String, Object> entry : propertyMap.entrySet()) {
           String propCat = PropertyHelper.getPropertyCategory(entry.getKey());
           String propName = PropertyHelper.getPropertyName(entry.getKey());
@@ -278,16 +291,16 @@ public class AlertDefinitionResourceProvider extends AbstractControllerResourceP
             jsonObj.addProperty(propName, entry.getValue().toString());
           }
         }
-        
+
         entity.setHash(UUID.randomUUID().toString());
-        
+
         alertDefinitionDAO.merge(entity);
       }
     }
-    
+
     notifyUpdate(Resource.Type.AlertDefinition, request, predicate);
 
-    return getRequestStatus(null);    
+    return getRequestStatus(null);
   }
 
   @Override
@@ -297,7 +310,7 @@ public class AlertDefinitionResourceProvider extends AbstractControllerResourceP
 
     Set<Resource> resources = getResources(
         new RequestImpl(null, null, null, null), predicate);
-    
+
     Set<Long> definitionIds = new HashSet<Long>();
 
     for (final Resource resource : resources) {
@@ -307,7 +320,7 @@ public class AlertDefinitionResourceProvider extends AbstractControllerResourceP
     for (Long definitionId : definitionIds) {
 
       LOG.info("Deleting alert definition {}", definitionId);
-      
+
       final AlertDefinitionEntity ad = alertDefinitionDAO.findById(definitionId.longValue());
 
       modifyResources(new Command<Void>() {
@@ -323,11 +336,11 @@ public class AlertDefinitionResourceProvider extends AbstractControllerResourceP
     return getRequestStatus(null);
 
   }
-  
+
   private Resource toResource(boolean isCollection, String clusterName,
       AlertDefinitionEntity entity, Set<String> requestedIds) {
     Resource resource = new ResourceImpl(Resource.Type.AlertDefinition);
-    
+
     setResourceProperty(resource, ALERT_DEF_CLUSTER_NAME, clusterName, requestedIds);
     setResourceProperty(resource, ALERT_DEF_ID, entity.getDefinitionId(), requestedIds);
     setResourceProperty(resource, ALERT_DEF_NAME, entity.getDefinitionName(), requestedIds);
@@ -337,22 +350,24 @@ public class AlertDefinitionResourceProvider extends AbstractControllerResourceP
     setResourceProperty(resource, ALERT_DEF_ENABLED, Boolean.valueOf(entity.getEnabled()), requestedIds);
     setResourceProperty(resource, ALERT_DEF_SCOPE, entity.getScope(), requestedIds);
     setResourceProperty(resource, ALERT_DEF_SOURCE_TYPE, entity.getSourceType(), requestedIds);
-    
+    setResourceProperty(resource, ALERT_DEF_UUID, entity.getHash(),
+        requestedIds);
+
     if (!isCollection && null != resource.getPropertyValue(ALERT_DEF_SOURCE_TYPE)) {
-      
+
       try {
         Map<String, String> map = gson.<Map<String, String>>fromJson(entity.getSource(), Map.class);
-        
+
         for (Entry<String, String> entry : map.entrySet()) {
           String subProp = PropertyHelper.getPropertyId(ALERT_DEF_SOURCE, entry.getKey());
-          resource.setProperty(subProp, entry.getValue());  
+          resource.setProperty(subProp, entry.getValue());
         }
       } catch (Exception e) {
         LOG.error("Could not coerce alert JSON into a type");
       }
     }
-    
+
     return resource;
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/84b988bc/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/AlertDefinitionDAO.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/AlertDefinitionDAO.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/AlertDefinitionDAO.java
index 05881e4..db5c63f 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/AlertDefinitionDAO.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/AlertDefinitionDAO.java
@@ -17,12 +17,16 @@
  */
 package org.apache.ambari.server.orm.dao;
 
+import java.util.Collections;
 import java.util.List;
+import java.util.Set;
 
 import javax.persistence.EntityManager;
 import javax.persistence.TypedQuery;
 
+import org.apache.ambari.server.controller.RootServiceResponseFactory;
 import org.apache.ambari.server.orm.entities.AlertDefinitionEntity;
+import org.apache.ambari.server.state.alert.Scope;
 
 import com.google.inject.Inject;
 import com.google.inject.Provider;
@@ -61,7 +65,7 @@ public class AlertDefinitionDAO {
 
   /**
    * Gets an alert definition with the specified ID.
-   * 
+   *
    * @param definitionId
    *          the ID of the definition to retrieve.
    * @return the alert definition or {@code null} if none exists.
@@ -74,7 +78,7 @@ public class AlertDefinitionDAO {
   /**
    * Gets an alert definition with the specified name. Alert definition names
    * are unique within a cluster.
-   * 
+   *
    * @param clusterId
    *          the ID of the cluster.
    * @param definitionName
@@ -93,7 +97,7 @@ public class AlertDefinitionDAO {
 
   /**
    * Gets all alert definitions stored in the database.
-   * 
+   *
    * @return all alert definitions or an empty list if none exist (never
    *         {@code null}).
    */
@@ -106,7 +110,7 @@ public class AlertDefinitionDAO {
 
   /**
    * Gets all alert definitions stored in the database.
-   * 
+   *
    * @return all alert definitions or empty list if none exist (never
    *         {@code null}).
    */
@@ -120,8 +124,114 @@ public class AlertDefinitionDAO {
   }
 
   /**
+   * Gets all alert definitions for the given service in the specified cluster.
+   *
+   * @param clusterId
+   *          the ID of the cluster.
+   * @param serviceName
+   *          the name of the service.
+   *
+   * @return all alert definitions for the service or empty list if none exist
+   *         (never {@code null}).
+   */
+  public List<AlertDefinitionEntity> findByService(long clusterId,
+      String serviceName) {
+    TypedQuery<AlertDefinitionEntity> query = entityManagerProvider.get().createNamedQuery(
+        "AlertDefinitionEntity.findByService", AlertDefinitionEntity.class);
+
+    query.setParameter("clusterId", clusterId);
+    query.setParameter("serviceName", serviceName);
+
+    return daoUtils.selectList(query);
+  }
+
+  /**
+   * Gets all alert definitions for the specified services that do not have a
+   * component. These definitions are assumed to be run on the master hosts.
+   *
+   * @param clusterId
+   *          the ID of the cluster.
+   * @param services
+   *          the services to match on.
+   *
+   * @return all alert definitions for the services or empty list if none exist
+   *         (never {@code null}).
+   */
+  public List<AlertDefinitionEntity> findByServiceMaster(long clusterId,
+      Set<String> services) {
+    if (null == services || services.size() == 0) {
+      return Collections.emptyList();
+    }
+
+    TypedQuery<AlertDefinitionEntity> query = entityManagerProvider.get().createNamedQuery(
+        "AlertDefinitionEntity.findByServiceMaster",
+        AlertDefinitionEntity.class);
+
+    query.setParameter("clusterId", clusterId);
+    query.setParameter("services", services);
+    query.setParameter("scope", Scope.SERVICE);
+
+    return daoUtils.selectList(query);
+  }
+
+  /**
+   * Gets all alert definitions that are not bound to a particular service. An
+   * example of this type of definition is a host capacity alert.
+   *
+   * @param clusterId
+   *          the ID of the cluster.
+   * @param serviceName
+   *          the name of the service (not {@code null}).
+   * @param componentName
+   *          the name of the service component (not {@code null}).
+   * @return all alert definitions that are not bound to a service or an empty
+   *         list (never {@code null}).
+   */
+  public List<AlertDefinitionEntity> findByServiceComponent(long clusterId,
+      String serviceName, String componentName) {
+    if (null == serviceName || null == componentName) {
+      return Collections.emptyList();
+    }
+
+    TypedQuery<AlertDefinitionEntity> query = entityManagerProvider.get().createNamedQuery(
+        "AlertDefinitionEntity.findByServiceAndComponent",
+        AlertDefinitionEntity.class);
+
+    query.setParameter("clusterId", clusterId);
+    query.setParameter("serviceName", serviceName);
+    query.setParameter("componentName", componentName);
+
+    return daoUtils.selectList(query);
+  }
+
+  /**
+   * Gets all alert definitions that are not bound to a particular service. An
+   * example of this type of definition is a host capacity alert.
+   *
+   * @param clusterId
+   *          the ID of the cluster.
+   * @return all alert definitions that are not bound to a service or an empty
+   *         list (never {@code null}).
+   */
+  public List<AlertDefinitionEntity> findAgentScoped(long clusterId) {
+    TypedQuery<AlertDefinitionEntity> query = entityManagerProvider.get().createNamedQuery(
+        "AlertDefinitionEntity.findByServiceAndComponent",
+        AlertDefinitionEntity.class);
+
+    query.setParameter("clusterId", clusterId);
+
+    query.setParameter("serviceName",
+        RootServiceResponseFactory.Services.AMBARI.name());
+
+    query.setParameter("componentName",
+        RootServiceResponseFactory.Components.AMBARI_AGENT.name());
+
+    return daoUtils.selectList(query);
+  }
+
+  /**
    * Persists a new alert definition.
-   * 
+   *
    * @param alertDefinition
    *          the definition to persist (not {@code null}).
    */
@@ -132,7 +242,7 @@ public class AlertDefinitionDAO {
 
   /**
    * Refresh the state of the alert definition from the database.
-   * 
+   *
    * @param alertDefinition
    *          the definition to refresh (not {@code null}).
    */
@@ -144,7 +254,7 @@ public class AlertDefinitionDAO {
   /**
    * Merge the speicified alert definition with the existing definition in the
    * database.
-   * 
+   *
    * @param alertDefinition
    *          the definition to merge (not {@code null}).
    * @return the updated definition with merged content (never {@code null}).
@@ -157,7 +267,7 @@ public class AlertDefinitionDAO {
   /**
    * Removes the specified alert definition and all related history and
    * associations from the database.
-   * 
+   *
    * @param alertDefinition
    *          the definition to remove.
    */
@@ -170,7 +280,8 @@ public class AlertDefinitionDAO {
     EntityManager entityManager = entityManagerProvider.get();
 
     alertDefinition = findById(alertDefinition.getDefinitionId());
-    if (null != alertDefinition)
+    if (null != alertDefinition) {
       entityManager.remove(alertDefinition);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/84b988bc/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/AlertDefinitionEntity.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/AlertDefinitionEntity.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/AlertDefinitionEntity.java
index de30921..66a3af0 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/AlertDefinitionEntity.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/AlertDefinitionEntity.java
@@ -50,9 +50,12 @@ import org.apache.ambari.server.state.alert.Scope;
     "cluster_id", "definition_name" }))
 @TableGenerator(name = "alert_definition_id_generator", table = "ambari_sequences", pkColumnName = "sequence_name", valueColumnName = "value", pkColumnValue = "alert_definition_id_seq", initialValue = 0, allocationSize = 1)
 @NamedQueries({
-    @NamedQuery(name = "AlertDefinitionEntity.findAll", query = "SELECT alertDefinition FROM AlertDefinitionEntity alertDefinition"),
-    @NamedQuery(name = "AlertDefinitionEntity.findAllInCluster", query = "SELECT alertDefinition FROM AlertDefinitionEntity alertDefinition WHERE alertDefinition.clusterId = :clusterId"),
-    @NamedQuery(name = "AlertDefinitionEntity.findByName", query = "SELECT alertDefinition FROM AlertDefinitionEntity alertDefinition WHERE alertDefinition.definitionName = :definitionName AND alertDefinition.clusterId = :clusterId"), })
+    @NamedQuery(name = "AlertDefinitionEntity.findAll", query = "SELECT ad FROM AlertDefinitionEntity ad"),
+    @NamedQuery(name = "AlertDefinitionEntity.findAllInCluster", query = "SELECT ad FROM AlertDefinitionEntity ad WHERE ad.clusterId = :clusterId"),
+    @NamedQuery(name = "AlertDefinitionEntity.findByName", query = "SELECT ad FROM AlertDefinitionEntity ad WHERE ad.definitionName = :definitionName AND ad.clusterId = :clusterId"),
+    @NamedQuery(name = "AlertDefinitionEntity.findByService", query = "SELECT ad FROM AlertDefinitionEntity ad WHERE ad.serviceName = :serviceName AND ad.clusterId = :clusterId"),
+    @NamedQuery(name = "AlertDefinitionEntity.findByServiceAndComponent", query = "SELECT ad FROM AlertDefinitionEntity ad WHERE ad.serviceName = :serviceName AND ad.componentName = :componentName AND ad.clusterId = :clusterId"),
+    @NamedQuery(name = "AlertDefinitionEntity.findByServiceMaster", query = "SELECT ad FROM AlertDefinitionEntity ad WHERE ad.serviceName IN :services AND ad.scope = :scope AND ad.clusterId = :clusterId") })
 public class AlertDefinitionEntity {
 
   @Id

http://git-wip-us.apache.org/repos/asf/ambari/blob/84b988bc/ambari-server/src/main/java/org/apache/ambari/server/state/alert/AlertDefinitionHash.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/alert/AlertDefinitionHash.java b/ambari-server/src/main/java/org/apache/ambari/server/state/alert/AlertDefinitionHash.java
new file mode 100644
index 0000000..1f31c35
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/alert/AlertDefinitionHash.java
@@ -0,0 +1,272 @@
+/**
+ * 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.
+ */
+package org.apache.ambari.server.state.alert;
+
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.ambari.server.AmbariException;
+import org.apache.ambari.server.orm.dao.AlertDefinitionDAO;
+import org.apache.ambari.server.orm.entities.AlertDefinitionEntity;
+import org.apache.ambari.server.state.Cluster;
+import org.apache.ambari.server.state.Clusters;
+import org.apache.ambari.server.state.Service;
+import org.apache.ambari.server.state.ServiceComponent;
+import org.apache.ambari.server.state.ServiceComponentHost;
+import org.apache.commons.codec.binary.Hex;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+
+/**
+ * The {@link AlertDefinitionHash} class is used to generate an MD5 hash for a
+ * list of {@link AlertDefinitionEntity}s. It is used in order to represent the
+ * state of a group of definitions by using
+ * {@link AlertDefinitionEntity#getHash()}
+ */
+@Singleton
+public class AlertDefinitionHash {
+
+  /**
+   * Logger.
+   */
+  private final static Logger LOG = LoggerFactory.getLogger(AlertDefinitionHash.class);
+
+  /**
+   * The hash returned when there are no definitions to hash.
+   */
+  public static String NULL_MD5_HASH = "37a6259cc0c1dae299a7866489dff0bd";
+
+  /**
+   * DAO for retrieving {@link AlertDefinitionEntity} instances.
+   */
+  @Inject
+  private AlertDefinitionDAO m_definitionDao;
+
+  /**
+   * All clusters.
+   */
+  @Inject
+  private Clusters m_clusters;
+
+  /**
+   * The hashes for all hosts.
+   */
+  private Map<String, String> m_hashes = new ConcurrentHashMap<String, String>();
+
+  /**
+   * Gets a unique hash value reprssenting all of the alert definitions that
+   * should be scheduled to run on a given host.
+   * <p/>
+   * This will not include alert definitions where the type is defined as
+   * {@link SourceType#AGGREGATE} since aggregate definitions are not scheduled
+   * to run on agent hosts.
+   * <p/>
+   * Hash values from this method are cached.
+   *
+   * @param clusterName
+   *          the cluster name (not {@code null}).
+   * @param hostName
+   *          the host name (not {@code null}).
+   * @return the unique hash or {@value #NULL_MD5_HASH} if none.
+   */
+  public String getHash(String clusterName, String hostName) {
+    String hash = m_hashes.get(hostName);
+    if (null != hash) {
+      return hash;
+    }
+
+    hash = hash(clusterName, hostName);
+    m_hashes.put(hostName, hash);
+
+    return hash;
+  }
+
+  /**
+   * Gets a mapping between cluster and alert definition hashes for all of the
+   * clusters that the given host belongs to.
+   *
+   * @param hostName
+   *          the host name (not {@code null}).
+   * @return a mapping between cluster and alert definition hash or an empty map
+   *         (never @code null).
+   * @see #getHash(String, String)
+   * @throws AmbariException
+   */
+  public Map<String, String> getHashes(String hostName)
+      throws AmbariException {
+    Set<Cluster> clusters = m_clusters.getClustersForHost(hostName);
+    if (null == clusters || clusters.size() == 0) {
+      return Collections.emptyMap();
+    }
+
+    Map<String, String> hashes = new HashMap<String, String>();
+    for (Cluster cluster : clusters) {
+      String clusterName = cluster.getClusterName();
+      String hash = getHash(clusterName, hostName);
+      hashes.put(clusterName, hash);
+    }
+
+    return hashes;
+  }
+
+  /**
+   * Gets the alert definitions for the specified host. This will include the
+   * following types of alert definitions:
+   * <ul>
+   * <li>Service/Component alerts</li>
+   * <li>Service alerts where the host is a MASTER</li>
+   * <li>Host alerts that are not bound to a service</li>
+   * </ul>
+   *
+   * @param clusterName
+   *          the cluster name (not {@code null}).
+   * @param hostName
+   *          the host name (not {@code null}).
+   * @return the alert definitions for the host, or an empty set (never
+   *         {@code null}).
+   */
+  public Set<AlertDefinitionEntity> getAlertDefinitions(String clusterName,
+      String hostName) {
+    Set<AlertDefinitionEntity> definitions = new HashSet<AlertDefinitionEntity>();
+
+    try {
+      Cluster cluster = m_clusters.getCluster(clusterName);
+      if (null == cluster) {
+        LOG.warn("Unable to get alert definitions for the missing cluster {}",
+            clusterName);
+
+        return Collections.emptySet();
+      }
+
+      long clusterId = cluster.getClusterId();
+      List<ServiceComponentHost> serviceComponents = cluster.getServiceComponentHosts(hostName);
+      if (null == serviceComponents || serviceComponents.size() == 0) {
+        LOG.warn(
+            "Unable to get alert definitions for {} since there are no service components defined",
+            hostName);
+
+        return Collections.emptySet();
+      }
+
+      for (ServiceComponentHost serviceComponent : serviceComponents) {
+        String serviceName = serviceComponent.getServiceName();
+        String componentName = serviceComponent.getServiceComponentName();
+
+        // add all alerts for this service/component pair
+        definitions.addAll(m_definitionDao.findByServiceComponent(
+            clusterId, serviceName, componentName));
+      }
+
+      // for every service, get the master components and see if the host
+      // is a master
+      Set<String> services = new HashSet<String>();
+      for (Entry<String, Service> entry : cluster.getServices().entrySet()) {
+        Service service = entry.getValue();
+        Map<String, ServiceComponent> components = service.getServiceComponents();
+        for (Entry<String, ServiceComponent> component : components.entrySet()) {
+          if (component.getValue().isMasterComponent()) {
+            Map<String, ServiceComponentHost> hosts = component.getValue().getServiceComponentHosts();
+
+            if( hosts.containsKey( hostName ) ){
+              services.add(service.getName());
+            }
+          }
+        }
+      }
+
+      // add all service scoped alerts
+      if( services.size() > 0 ){
+        definitions.addAll(m_definitionDao.findByServiceMaster(clusterId,
+            services));
+      }
+
+      // add any alerts not bound to a service (host level alerts)
+      definitions.addAll(m_definitionDao.findAgentScoped(clusterId));
+    } catch (AmbariException ambariException) {
+      LOG.error("Unable to get alert definitions", ambariException);
+      return Collections.emptySet();
+    }
+
+    return definitions;
+  }
+
+  /**
+   * Calculates a unique hash value representing all of the alert definitions
+   * that should be scheduled to run on a given host. Alerts of type
+   * {@link SourceType#AGGREGATE} are not included in the hash since they are
+   * not run on the agents.
+   *
+   * @param clusterName
+   *          the cluster name (not {@code null}).
+   * @param hostName
+   *          the host name (not {@code null}).
+   * @return the unique hash or {@value #NULL_MD5_HASH} if none.
+   */
+  private String hash(String clusterName, String hostName) {
+    Set<AlertDefinitionEntity> definitions = getAlertDefinitions(clusterName,
+        hostName);
+
+    // no definitions found for this host, don't bother hashing
+    if( null == definitions || definitions.size() == 0 ) {
+      return NULL_MD5_HASH;
+    }
+
+    // strip out all AGGREGATE types
+    Iterator<AlertDefinitionEntity> iterator = definitions.iterator();
+    while (iterator.hasNext()) {
+      if (SourceType.AGGREGATE.equals(iterator.next().getSourceType())) {
+        iterator.remove();
+      }
+    }
+
+    // build the UUIDs
+    List<String> uuids = new ArrayList<String>(definitions.size());
+    for (AlertDefinitionEntity definition : definitions) {
+      uuids.add(definition.getHash());
+    }
+
+    // sort the UUIDs so that the digest is created with bytes in the same order
+    Collections.sort(uuids);
+
+    try {
+      MessageDigest digest = MessageDigest.getInstance("MD5");
+      for (String uuid : uuids) {
+        digest.update(uuid.getBytes());
+      }
+
+      byte[] hashBytes = digest.digest();
+      return Hex.encodeHexString(hashBytes);
+    } catch (NoSuchAlgorithmException nsae) {
+      LOG.warn("Unable to calculate MD5 alert definition hash", nsae);
+      return NULL_MD5_HASH;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/84b988bc/ambari-server/src/main/resources/properties.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/properties.json b/ambari-server/src/main/resources/properties.json
index e515b91..e80c0e6 100644
--- a/ambari-server/src/main/resources/properties.json
+++ b/ambari-server/src/main/resources/properties.json
@@ -420,7 +420,8 @@
       "AlertDefinition/interval",
       "AlertDefinition/enabled",
       "AlertDefinition/scope",
-      "AlertDefinition/source"
+      "AlertDefinition/source",
+      "AlertDefinition/uuid"
     ],      
     "Controller":[
         "Controllers/name",

http://git-wip-us.apache.org/repos/asf/ambari/blob/84b988bc/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/AlertDefinitionResourceProviderTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/AlertDefinitionResourceProviderTest.java b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/AlertDefinitionResourceProviderTest.java
index fc57389..c6e26e8 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/AlertDefinitionResourceProviderTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/AlertDefinitionResourceProviderTest.java
@@ -34,6 +34,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.UUID;
 
 import org.apache.ambari.server.controller.AmbariManagementController;
 import org.apache.ambari.server.controller.spi.Predicate;
@@ -56,59 +57,65 @@ import org.junit.Test;
 public class AlertDefinitionResourceProviderTest {
 
   AlertDefinitionDAO dao = null;
-  
+
+  private static String DEFINITION_UUID = UUID.randomUUID().toString();
+
   @Before
   public void before() {
     dao = createStrictMock(AlertDefinitionDAO.class);
-    
+
     AlertDefinitionResourceProvider.init(dao);
   }
-  
+
   @Test
   public void testGetResourcesNoPredicate() throws Exception {
     AlertDefinitionResourceProvider provider = createProvider(null);
-    
+
     Request request = PropertyHelper.getReadRequest("AlertDefinition/cluster_name",
         "AlertDefinition/id");
-    
+
     Set<Resource> results = provider.getResources(request, null);
-    
+
     assertEquals(0, results.size());
-  }  
+  }
 
   @Test
   public void testGetResourcesClusterPredicate() throws Exception {
     Request request = PropertyHelper.getReadRequest(
         AlertDefinitionResourceProvider.ALERT_DEF_CLUSTER_NAME,
         AlertDefinitionResourceProvider.ALERT_DEF_ID,
-        AlertDefinitionResourceProvider.ALERT_DEF_NAME);
-    
+        AlertDefinitionResourceProvider.ALERT_DEF_NAME,
+        AlertDefinitionResourceProvider.ALERT_DEF_UUID);
+
     AmbariManagementController amc = createMock(AmbariManagementController.class);
     Clusters clusters = createMock(Clusters.class);
     Cluster cluster = createMock(Cluster.class);
     expect(amc.getClusters()).andReturn(clusters).atLeastOnce();
     expect(clusters.getCluster((String) anyObject())).andReturn(cluster).atLeastOnce();
     expect(cluster.getClusterId()).andReturn(Long.valueOf(1)).anyTimes();
-    
+
     Predicate predicate = new PredicateBuilder().property(
-        AlertDefinitionResourceProvider.ALERT_DEF_CLUSTER_NAME).equals("c1").toPredicate();    
-    
+        AlertDefinitionResourceProvider.ALERT_DEF_CLUSTER_NAME).equals("c1").toPredicate();
+
     expect(dao.findAll(1L)).andReturn(getMockEntities());
 
     replay(amc, clusters, cluster, dao);
-    
-    AlertDefinitionResourceProvider provider = createProvider(amc);    
+
+    AlertDefinitionResourceProvider provider = createProvider(amc);
     Set<Resource> results = provider.getResources(request, predicate);
-    
+
     assertEquals(1, results.size());
-    
+
     Resource r = results.iterator().next();
-    
+
     Assert.assertEquals("my_def", r.getPropertyValue(AlertDefinitionResourceProvider.ALERT_DEF_NAME));
-    
+
+    Assert.assertEquals(DEFINITION_UUID,
+        r.getPropertyValue(AlertDefinitionResourceProvider.ALERT_DEF_UUID));
+
     verify(amc, clusters, cluster, dao);
   }
-  
+
   @Test
   public void testGetSingleResource() throws Exception {
     Request request = PropertyHelper.getReadRequest(
@@ -116,29 +123,29 @@ public class AlertDefinitionResourceProviderTest {
         AlertDefinitionResourceProvider.ALERT_DEF_ID,
         AlertDefinitionResourceProvider.ALERT_DEF_NAME,
         AlertDefinitionResourceProvider.ALERT_DEF_SOURCE_TYPE);
-    
+
     AmbariManagementController amc = createMock(AmbariManagementController.class);
     Clusters clusters = createMock(Clusters.class);
     Cluster cluster = createMock(Cluster.class);
     expect(amc.getClusters()).andReturn(clusters).atLeastOnce();
     expect(clusters.getCluster((String) anyObject())).andReturn(cluster).atLeastOnce();
     expect(cluster.getClusterId()).andReturn(Long.valueOf(1)).anyTimes();
-    
+
     Predicate predicate = new PredicateBuilder().property(
         AlertDefinitionResourceProvider.ALERT_DEF_CLUSTER_NAME).equals("c1")
-          .and().property(AlertDefinitionResourceProvider.ALERT_DEF_ID).equals("1").toPredicate();    
-    
+          .and().property(AlertDefinitionResourceProvider.ALERT_DEF_ID).equals("1").toPredicate();
+
     expect(dao.findById(1L)).andReturn(getMockEntities().get(0));
 
     replay(amc, clusters, cluster, dao);
-    
-    AlertDefinitionResourceProvider provider = createProvider(amc);    
+
+    AlertDefinitionResourceProvider provider = createProvider(amc);
     Set<Resource> results = provider.getResources(request, predicate);
-    
+
     assertEquals(1, results.size());
-    
+
     Resource r = results.iterator().next();
-    
+
     Assert.assertEquals("my_def", r.getPropertyValue(AlertDefinitionResourceProvider.ALERT_DEF_NAME));
     Assert.assertEquals("metric", r.getPropertyValue(AlertDefinitionResourceProvider.ALERT_DEF_SOURCE_TYPE));
     Assert.assertNotNull(r.getPropertyValue("AlertDefinition/source/type"));
@@ -156,24 +163,24 @@ public class AlertDefinitionResourceProviderTest {
     Capture<AlertDefinitionEntity> entityCapture = new Capture<AlertDefinitionEntity>();
     dao.create(capture(entityCapture));
     expectLastCall();
-    
+
     replay(amc, clusters, cluster, dao);
-    
+
     AlertDefinitionResourceProvider provider = createProvider(amc);
-    
+
     Map<String, Object> requestProps = new HashMap<String, Object>();
     requestProps.put(AlertDefinitionResourceProvider.ALERT_DEF_CLUSTER_NAME, "c1");
     requestProps.put(AlertDefinitionResourceProvider.ALERT_DEF_INTERVAL, "1");
     requestProps.put(AlertDefinitionResourceProvider.ALERT_DEF_NAME, "my_def");
     requestProps.put(AlertDefinitionResourceProvider.ALERT_DEF_SERVICE_NAME, "HDFS");
     requestProps.put(AlertDefinitionResourceProvider.ALERT_DEF_SOURCE_TYPE, "METRIC");
-    
+
     Request request = PropertyHelper.getCreateRequest(Collections.singleton(requestProps), null);
 
     provider.createResources(request);
-    
+
     Assert.assertTrue(entityCapture.hasCaptured());
-    AlertDefinitionEntity entity = entityCapture.getValue(); 
+    AlertDefinitionEntity entity = entityCapture.getValue();
     Assert.assertNotNull(entity);
 
     Assert.assertEquals(Long.valueOf(1), entity.getClusterId());
@@ -186,11 +193,11 @@ public class AlertDefinitionResourceProviderTest {
     Assert.assertEquals("HDFS", entity.getServiceName());
     Assert.assertNotNull(entity.getSource());
     Assert.assertEquals("METRIC", entity.getSourceType());
-    
+
     verify(amc, clusters, cluster, dao);
 
   }
-  
+
   @Test
   public void testUpdateResources() throws Exception {
     AmbariManagementController amc = createMock(AmbariManagementController.class);
@@ -203,40 +210,40 @@ public class AlertDefinitionResourceProviderTest {
     Capture<AlertDefinitionEntity> entityCapture = new Capture<AlertDefinitionEntity>();
     dao.create(capture(entityCapture));
     expectLastCall();
-    
+
     replay(amc, clusters, cluster, dao);
-    
+
     Map<String, Object> requestProps = new HashMap<String, Object>();
     requestProps.put(AlertDefinitionResourceProvider.ALERT_DEF_CLUSTER_NAME, "c1");
     requestProps.put(AlertDefinitionResourceProvider.ALERT_DEF_INTERVAL, "1");
     requestProps.put(AlertDefinitionResourceProvider.ALERT_DEF_NAME, "my_def");
     requestProps.put(AlertDefinitionResourceProvider.ALERT_DEF_SERVICE_NAME, "HDFS");
     requestProps.put(AlertDefinitionResourceProvider.ALERT_DEF_SOURCE_TYPE, "METRIC");
-    
+
     Request request = PropertyHelper.getCreateRequest(Collections.singleton(requestProps), null);
 
     AlertDefinitionResourceProvider provider = createProvider(amc);
-    
+
     provider.createResources(request);
-    
+
     Assert.assertTrue(entityCapture.hasCaptured());
-    AlertDefinitionEntity entity = entityCapture.getValue(); 
+    AlertDefinitionEntity entity = entityCapture.getValue();
     Assert.assertNotNull(entity);
-    
+
     Predicate p = new PredicateBuilder().property(
         AlertDefinitionResourceProvider.ALERT_DEF_ID).equals("1").and().property(
             AlertDefinitionResourceProvider.ALERT_DEF_CLUSTER_NAME).equals("c1").toPredicate();
     // everything is mocked, there is no DB
     entity.setDefinitionId(Long.valueOf(1));
-    
+
     String oldName = entity.getDefinitionName();
     String oldHash = entity.getHash();
-    
+
     resetToStrict(dao);
     expect(dao.findById(1L)).andReturn(entity).anyTimes();
     expect(dao.merge((AlertDefinitionEntity) anyObject())).andReturn(entity).anyTimes();
     replay(dao);
-    
+
     requestProps = new HashMap<String, Object>();
     requestProps.put(AlertDefinitionResourceProvider.ALERT_DEF_CLUSTER_NAME, "c1");
     requestProps.put(AlertDefinitionResourceProvider.ALERT_DEF_INTERVAL, "1");
@@ -244,15 +251,15 @@ public class AlertDefinitionResourceProviderTest {
     requestProps.put(AlertDefinitionResourceProvider.ALERT_DEF_SERVICE_NAME, "HDFS");
     requestProps.put(AlertDefinitionResourceProvider.ALERT_DEF_SOURCE_TYPE, "METRIC");
     request = PropertyHelper.getUpdateRequest(requestProps, null);
-    
+
     provider.updateResources(request, p);
 
     Assert.assertFalse(oldHash.equals(entity.getHash()));
     Assert.assertFalse(oldName.equals(entity.getDefinitionName()));
-    
+
     verify(amc, clusters, cluster, dao);
   }
-  
+
   @Test
   public void testDeleteResources() throws Exception {
     AmbariManagementController amc = createMock(AmbariManagementController.class);
@@ -265,9 +272,9 @@ public class AlertDefinitionResourceProviderTest {
     Capture<AlertDefinitionEntity> entityCapture = new Capture<AlertDefinitionEntity>();
     dao.create(capture(entityCapture));
     expectLastCall();
-    
+
     replay(amc, clusters, cluster, dao);
-    
+
     AlertDefinitionResourceProvider provider = createProvider(amc);
 
     Map<String, Object> requestProps = new HashMap<String, Object>();
@@ -276,43 +283,43 @@ public class AlertDefinitionResourceProviderTest {
     requestProps.put(AlertDefinitionResourceProvider.ALERT_DEF_NAME, "my_def");
     requestProps.put(AlertDefinitionResourceProvider.ALERT_DEF_SERVICE_NAME, "HDFS");
     requestProps.put(AlertDefinitionResourceProvider.ALERT_DEF_SOURCE_TYPE, "METRIC");
-    
+
     Request request = PropertyHelper.getCreateRequest(Collections.singleton(requestProps), null);
 
     provider.createResources(request);
 
     Assert.assertTrue(entityCapture.hasCaptured());
-    AlertDefinitionEntity entity = entityCapture.getValue(); 
+    AlertDefinitionEntity entity = entityCapture.getValue();
     Assert.assertNotNull(entity);
-    
+
     Predicate p = new PredicateBuilder().property(
         AlertDefinitionResourceProvider.ALERT_DEF_ID).equals("1").and().property(
             AlertDefinitionResourceProvider.ALERT_DEF_CLUSTER_NAME).equals("c1").toPredicate();
     // everything is mocked, there is no DB
     entity.setDefinitionId(Long.valueOf(1));
-    
+
     resetToStrict(dao);
     expect(dao.findById(1L)).andReturn(entity).anyTimes();
     dao.remove(capture(entityCapture));
     expectLastCall();
     replay(dao);
-    
+
     provider.deleteResources(p);
-    
+
     AlertDefinitionEntity entity1 = entityCapture.getValue();
     Assert.assertEquals(Long.valueOf(1), entity1.getDefinitionId());
-    
+
     verify(amc, clusters, cluster, dao);
-    
+
   }
-  
+
   private AlertDefinitionResourceProvider createProvider(AmbariManagementController amc) {
     return new AlertDefinitionResourceProvider(
         PropertyHelper.getPropertyIds(Resource.Type.AlertDefinition),
         PropertyHelper.getKeyPropertyIds(Resource.Type.AlertDefinition),
         amc);
   }
-  
+
   private List<AlertDefinitionEntity> getMockEntities() {
     AlertDefinitionEntity entity = new AlertDefinitionEntity();
     entity.setClusterId(Long.valueOf(1L));
@@ -320,13 +327,12 @@ public class AlertDefinitionResourceProviderTest {
     entity.setDefinitionId(Long.valueOf(1L));
     entity.setDefinitionName("my_def");
     entity.setEnabled(true);
-    entity.setHash("tmphash");
+    entity.setHash(DEFINITION_UUID);
     entity.setScheduleInterval(Integer.valueOf(2));
     entity.setServiceName(null);
     entity.setSourceType("metric");
     entity.setSource("{'jmx': 'beanName/attributeName', 'host': '{{aa:123445}}'}");
-    
+
     return Arrays.asList(entity);
   }
-  
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/84b988bc/ambari-server/src/test/java/org/apache/ambari/server/orm/dao/AlertDefinitionDAOTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/orm/dao/AlertDefinitionDAOTest.java b/ambari-server/src/test/java/org/apache/ambari/server/orm/dao/AlertDefinitionDAOTest.java
index f2ddcd7..d621a9b 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/orm/dao/AlertDefinitionDAOTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/orm/dao/AlertDefinitionDAOTest.java
@@ -28,6 +28,7 @@ import java.util.List;
 import java.util.TimeZone;
 import java.util.UUID;
 
+import org.apache.ambari.server.controller.RootServiceResponseFactory;
 import org.apache.ambari.server.orm.GuiceJpaInitializer;
 import org.apache.ambari.server.orm.InMemoryDefaultTestModule;
 import org.apache.ambari.server.orm.OrmTestHelper;
@@ -41,7 +42,6 @@ import org.apache.ambari.server.state.MaintenanceState;
 import org.apache.ambari.server.state.NotificationState;
 import org.apache.ambari.server.state.alert.Scope;
 import org.junit.After;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -65,7 +65,7 @@ public class AlertDefinitionDAOTest {
   OrmTestHelper helper;
 
   /**
-   * 
+   *
    */
   @Before
   public void setup() {
@@ -78,7 +78,9 @@ public class AlertDefinitionDAOTest {
     helper = injector.getInstance(OrmTestHelper.class);
     clusterId = helper.createCluster();
 
-    for (int i = 0; i < 8; i++) {
+    // create 8 HDFS alerts
+    int i = 0;
+    for (; i < 8; i++) {
       AlertDefinitionEntity definition = new AlertDefinitionEntity();
       definition.setDefinitionName("Alert Definition " + i);
       definition.setServiceName("HDFS");
@@ -91,6 +93,58 @@ public class AlertDefinitionDAOTest {
       definition.setSourceType("SCRIPT");
       dao.create(definition);
     }
+
+    // create 2 HDFS with components
+    for (; i < 10; i++) {
+      AlertDefinitionEntity definition = new AlertDefinitionEntity();
+      definition.setDefinitionName("Alert Definition " + i);
+      definition.setServiceName("HDFS");
+
+      if (i == 9) {
+        definition.setComponentName("NAMENODE");
+      } else {
+        definition.setComponentName("DATANODE");
+      }
+
+      definition.setClusterId(clusterId);
+      definition.setHash(UUID.randomUUID().toString());
+      definition.setScheduleInterval(60);
+      definition.setScope(Scope.SERVICE);
+      definition.setSource("Source " + i);
+      definition.setSourceType("SCRIPT");
+      dao.create(definition);
+    }
+
+    // create 2 host scoped
+    for (; i < 12; i++) {
+      AlertDefinitionEntity definition = new AlertDefinitionEntity();
+      definition.setDefinitionName("Alert Definition " + i);
+      definition.setServiceName("OOZIE");
+      definition.setComponentName("OOZIE_SERVER");
+      definition.setClusterId(clusterId);
+      definition.setHash(UUID.randomUUID().toString());
+      definition.setScheduleInterval(60);
+      definition.setScope(Scope.HOST);
+      definition.setSource("Source " + i);
+      definition.setSourceType("SCRIPT");
+      dao.create(definition);
+    }
+
+    // create 3 agent alerts
+    for (; i < 15; i++) {
+      AlertDefinitionEntity definition = new AlertDefinitionEntity();
+      definition.setDefinitionName("Alert Definition " + i);
+      definition.setServiceName(RootServiceResponseFactory.Services.AMBARI.name());
+      definition.setComponentName(RootServiceResponseFactory.Components.AMBARI_AGENT.name());
+      definition.setClusterId(clusterId);
+      definition.setHash(UUID.randomUUID().toString());
+      definition.setScheduleInterval(60);
+      definition.setScope(Scope.HOST);
+      definition.setSource("Source " + i);
+      definition.setSourceType("SCRIPT");
+      dao.create(definition);
+    }
+
   }
 
   @After
@@ -100,40 +154,77 @@ public class AlertDefinitionDAOTest {
   }
 
   /**
-   * 
+   *
    */
   @Test
   public void testFindByName() {
     List<AlertDefinitionEntity> definitions = dao.findAll();
-    Assert.assertNotNull(definitions);
+    assertNotNull(definitions);
     AlertDefinitionEntity definition = definitions.get(2);
     AlertDefinitionEntity retrieved = dao.findByName(
         definition.getClusterId(), definition.getDefinitionName());
 
-    Assert.assertEquals(definition, retrieved);
+    assertEquals(definition, retrieved);
   }
 
   /**
-   * 
+   *
    */
   @Test
   public void testFindAll() {
     List<AlertDefinitionEntity> definitions = dao.findAll();
-    Assert.assertNotNull(definitions);
-    Assert.assertEquals(8, definitions.size());
+    assertNotNull(definitions);
+    assertEquals(15, definitions.size());
   }
 
   /**
-   * 
+   *
    */
   @Test
-  public void findById() {
+  public void testFindById() {
     List<AlertDefinitionEntity> definitions = dao.findAll();
-    Assert.assertNotNull(definitions);
+    assertNotNull(definitions);
     AlertDefinitionEntity definition = definitions.get(2);
     AlertDefinitionEntity retrieved = dao.findById(definition.getDefinitionId());
+    assertEquals(definition, retrieved);
+  }
+
+  /**
+   *
+   */
+  @Test
+  public void testFindByService() {
+    List<AlertDefinitionEntity> definitions = dao.findByService(clusterId,
+        "HDFS");
+
+    assertNotNull(definitions);
+    assertEquals(10, definitions.size());
+
+    definitions = dao.findByService(clusterId, "YARN");
+    assertNotNull(definitions);
+    assertEquals(0, definitions.size());
+  }
 
-    Assert.assertEquals(definition, retrieved);
+  /**
+   *
+   */
+  @Test
+  public void testFindByServiceComponent() {
+    List<AlertDefinitionEntity> definitions = dao.findByServiceComponent(
+        clusterId, "OOZIE", "OOZIE_SERVER");
+
+    assertNotNull(definitions);
+    assertEquals(2, definitions.size());
+  }
+
+  /**
+   *
+   */
+  @Test
+  public void testFindAgentScoped() {
+    List<AlertDefinitionEntity> definitions = dao.findAgentScoped(clusterId);
+    assertNotNull(definitions);
+    assertEquals(3, definitions.size());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/ambari/blob/84b988bc/ambari-server/src/test/java/org/apache/ambari/server/state/alerts/AlertDefinitionHashTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/state/alerts/AlertDefinitionHashTest.java b/ambari-server/src/test/java/org/apache/ambari/server/state/alerts/AlertDefinitionHashTest.java
new file mode 100644
index 0000000..937417a
--- /dev/null
+++ b/ambari-server/src/test/java/org/apache/ambari/server/state/alerts/AlertDefinitionHashTest.java
@@ -0,0 +1,224 @@
+/**
+ * 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.
+ */
+package org.apache.ambari.server.state.alerts;
+
+
+import static org.easymock.EasyMock.anyObject;
+import static org.easymock.EasyMock.expect;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+
+import junit.framework.TestCase;
+
+import org.apache.ambari.server.orm.InMemoryDefaultTestModule;
+import org.apache.ambari.server.orm.dao.AlertDefinitionDAO;
+import org.apache.ambari.server.orm.entities.AlertDefinitionEntity;
+import org.apache.ambari.server.state.Cluster;
+import org.apache.ambari.server.state.Clusters;
+import org.apache.ambari.server.state.Service;
+import org.apache.ambari.server.state.ServiceComponent;
+import org.apache.ambari.server.state.ServiceComponentHost;
+import org.apache.ambari.server.state.alert.AlertDefinitionHash;
+import org.apache.ambari.server.state.alert.Scope;
+import org.easymock.EasyMock;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.inject.Binder;
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+import com.google.inject.Module;
+import com.google.inject.util.Modules;
+
+/**
+ * Tests for {@link AlertDefinitionHash}.
+ */
+public class AlertDefinitionHashTest extends TestCase {
+
+  private AlertDefinitionHash m_hash;
+  private Clusters m_mockClusters;
+  private Cluster m_mockCluster;
+  private AlertDefinitionDAO m_mockDao;
+  private Injector m_injector;
+
+  private static final String CLUSTERNAME = "cluster1";
+  private static final String HOSTNAME = "c6401.ambari.apache.org";
+
+  /**
+   *
+   */
+  @Override
+  @Before
+  protected void setUp() throws Exception {
+    super.setUp();
+
+    m_injector = Guice.createInjector(Modules.override(
+        new InMemoryDefaultTestModule()).with(new MockModule()));
+
+    m_mockClusters = m_injector.getInstance(Clusters.class);
+    m_mockCluster = m_injector.getInstance(Cluster.class);
+    m_mockDao = m_injector.getInstance(AlertDefinitionDAO.class);
+
+    // add HDFS/NN
+    List<ServiceComponentHost> serviceComponentHosts = new ArrayList<ServiceComponentHost>();
+    ServiceComponentHost sch = EasyMock.createNiceMock(ServiceComponentHost.class);
+    expect(sch.getServiceName()).andReturn("HDFS").anyTimes();
+    expect(sch.getServiceComponentName()).andReturn("NAMENODE").anyTimes();
+    expect(sch.getHostName()).andReturn(HOSTNAME).anyTimes();
+    EasyMock.replay(sch);
+    serviceComponentHosts.add(sch);
+
+    // add HDFS/DN
+    sch = EasyMock.createNiceMock(ServiceComponentHost.class);
+    expect(sch.getServiceName()).andReturn("HDFS").anyTimes();
+    expect(sch.getServiceComponentName()).andReturn("DATANODE").anyTimes();
+    expect(sch.getHostName()).andReturn(HOSTNAME).anyTimes();
+    EasyMock.replay(sch);
+    serviceComponentHosts.add(sch);
+
+    Map<String, ServiceComponentHost> mapComponentHosts = new HashMap<String, ServiceComponentHost>();
+    ServiceComponentHost host = EasyMock.createNiceMock(ServiceComponentHost.class);
+    expect(host.getHostName()).andReturn(HOSTNAME).anyTimes();
+    mapComponentHosts.put(HOSTNAME, host);
+
+    Map<String, ServiceComponent> serviceComponents = new HashMap<String, ServiceComponent>();
+    ServiceComponent namenode = EasyMock.createNiceMock(ServiceComponent.class);
+    expect(namenode.getServiceComponentHosts()).andReturn(mapComponentHosts).anyTimes();
+    expect(namenode.isMasterComponent()).andReturn(true).anyTimes();
+    serviceComponents.put("NAMENODE", namenode);
+
+    // create HDFS for the cluster
+    Map<String, Service> services = new HashMap<String, Service>();
+    String hdfsName = "HDFS";
+    Service hdfs = EasyMock.createNiceMock(Service.class);
+    expect(hdfs.getName()).andReturn("HDFS").anyTimes();
+    expect(hdfs.getServiceComponents()).andReturn(serviceComponents).anyTimes();
+    services.put(hdfsName, hdfs);
+
+    // replay
+    EasyMock.replay(hdfs, host, namenode);
+
+    // Clusters mock
+    expect(m_mockClusters.getCluster((String) anyObject())).andReturn(
+        m_mockCluster).atLeastOnce();
+
+    // cluster mock
+    expect(m_mockCluster.getClusterId()).andReturn(Long.valueOf(1)).anyTimes();
+    expect(m_mockCluster.getClusterName()).andReturn(CLUSTERNAME).anyTimes();
+    expect(m_mockCluster.getServices()).andReturn(services).anyTimes();
+    expect(
+        m_mockCluster.getServiceComponentHosts(EasyMock.anyObject(String.class))).andReturn(
+        serviceComponentHosts).anyTimes();
+
+    AlertDefinitionEntity hdfsService = new AlertDefinitionEntity();
+    hdfsService.setDefinitionId(1L);
+    hdfsService.setClusterId(1L);
+    hdfsService.setHash(UUID.randomUUID().toString());
+    hdfsService.setServiceName("HDFS");
+    hdfsService.setComponentName("NAMENODE");
+    hdfsService.setScope(Scope.SERVICE);
+
+    AlertDefinitionEntity hdfsHost = new AlertDefinitionEntity();
+    hdfsHost.setDefinitionId(2L);
+    hdfsHost.setClusterId(1L);
+    hdfsHost.setHash(UUID.randomUUID().toString());
+    hdfsHost.setServiceName("HDFS");
+    hdfsHost.setComponentName("DATANODE");
+    hdfsHost.setScope(Scope.HOST);
+
+    AlertDefinitionEntity agentScoped = new AlertDefinitionEntity();
+    agentScoped.setDefinitionId(3L);
+    agentScoped.setClusterId(1L);
+    agentScoped.setHash(UUID.randomUUID().toString());
+    agentScoped.setServiceName("AMBARI");
+    agentScoped.setComponentName("AMBARI_AGENT");
+    agentScoped.setScope(Scope.HOST);
+
+    EasyMock.expect(
+        m_mockDao.findByServiceMaster(EasyMock.anyInt(),
+            (Set<String>) EasyMock.anyObject())).andReturn(
+        Collections.singletonList(hdfsService)).anyTimes();
+
+    EasyMock.expect(
+        m_mockDao.findByServiceComponent(EasyMock.anyInt(),
+            EasyMock.anyObject(String.class), EasyMock.anyObject(String.class))).andReturn(
+        Collections.singletonList(hdfsHost)).anyTimes();
+
+    EasyMock.expect(m_mockDao.findAgentScoped(EasyMock.anyInt())).andReturn(
+        Collections.singletonList(agentScoped)).anyTimes();
+
+    EasyMock.replay(m_mockClusters, m_mockCluster, m_mockDao);
+    m_hash = m_injector.getInstance(AlertDefinitionHash.class);
+  }
+
+  /**
+   *
+   */
+  @Override
+  @After
+  protected void tearDown() throws Exception {
+    super.tearDown();
+  }
+
+  /**
+   * Test method for {@link org.apache.ambari.server.state.alert.AlertDefinitionHash#getHash(java.lang.String, java.lang.String)}.
+   */
+  @Test
+  public void testGetHash() {
+    String hash = m_hash.getHash(CLUSTERNAME, HOSTNAME);
+    assertNotNull(hash);
+    assertNotSame(AlertDefinitionHash.NULL_MD5_HASH, hash);
+    assertEquals(hash, m_hash.getHash(CLUSTERNAME, HOSTNAME));
+  }
+
+  /**
+   * Test method for {@link org.apache.ambari.server.state.alert.AlertDefinitionHash#getAlertDefinitions(java.lang.String, java.lang.String)}.
+   */
+  @Test
+  public void testGetAlertDefinitions() {
+    Set<AlertDefinitionEntity> definitions = m_hash.getAlertDefinitions(
+        CLUSTERNAME, HOSTNAME);
+
+    assertEquals(3, definitions.size());
+  }
+
+  /**
+   *
+   */
+  private class MockModule implements Module {
+    /**
+     *
+     */
+    @Override
+    public void configure(Binder binder) {
+      binder.bind(Clusters.class).toInstance(
+          EasyMock.createNiceMock(Clusters.class));
+      binder.bind(Cluster.class).toInstance(
+          EasyMock.createNiceMock(Cluster.class));
+      binder.bind(AlertDefinitionDAO.class).toInstance(
+          EasyMock.createNiceMock(AlertDefinitionDAO.class));
+    }
+  }
+}