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/12/16 02:22:45 UTC

ambari git commit: AMBARI-8723 - Alerts: Thresholds For Percent Alerts Do Not Transition To Warning (jonathanhurley)

Repository: ambari
Updated Branches:
  refs/heads/trunk 5b0ccc012 -> c78d584e3


AMBARI-8723 - Alerts: Thresholds For Percent Alerts Do Not Transition To Warning (jonathanhurley)


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

Branch: refs/heads/trunk
Commit: c78d584e3979bb253cbd681fa80be9e70fe16787
Parents: 5b0ccc0
Author: Jonathan Hurley <jh...@hortonworks.com>
Authored: Mon Dec 15 16:41:40 2014 -0500
Committer: Jonathan Hurley <jh...@hortonworks.com>
Committed: Mon Dec 15 20:13:10 2014 -0500

----------------------------------------------------------------------
 .../python/ambari_agent/alerts/metric_alert.py  | 18 ++---
 .../events/AlertDefinitionChangedEvent.java     | 55 +++++++++++++++
 .../events/AlertDefinitionDisabledEvent.java    |  2 +-
 .../ambari/server/events/AmbariEvent.java       |  5 ++
 .../alerts/AlertAggregateListener.java          | 37 +++++-----
 .../alerts/AlertLifecycleListener.java          | 25 ++++++-
 .../server/orm/dao/AlertDefinitionDAO.java      | 25 +++++--
 .../apache/ambari/server/orm/dao/AlertsDAO.java |  3 +-
 .../ambari/server/state/alert/Source.java       | 10 ++-
 .../services/AlertNoticeDispatchService.java    |  4 +-
 .../ambari/server/orm/dao/AlertsDAOTest.java    |  2 +-
 .../state/alerts/AlertEventPublisherTest.java   | 71 ++++++++++++++++++++
 12 files changed, 221 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/c78d584e/ambari-agent/src/main/python/ambari_agent/alerts/metric_alert.py
----------------------------------------------------------------------
diff --git a/ambari-agent/src/main/python/ambari_agent/alerts/metric_alert.py b/ambari-agent/src/main/python/ambari_agent/alerts/metric_alert.py
index 6a9fb95..ebd205d 100644
--- a/ambari-agent/src/main/python/ambari_agent/alerts/metric_alert.py
+++ b/ambari-agent/src/main/python/ambari_agent/alerts/metric_alert.py
@@ -90,17 +90,17 @@ class MetricAlert(BaseAlert):
     crit_value = self.__find_threshold('critical')
     
     # critical values are higher
-    crit_direction_up = crit_value >= warn_value
+    critical_direction_up = crit_value >= warn_value
     
-    if crit_direction_up: 
-      # critcal values are higher
-      if value > crit_value:
+    if critical_direction_up:
+      # critical values are higher
+      if value >= crit_value:
         return self.RESULT_CRITICAL
-      elif value > warn_value:
+      elif value >= warn_value:
         return self.RESULT_WARNING
       else:
         if ok_value is not None:
-          if value > ok_value and value <= warn_value:
+          if value >= ok_value and value < warn_value:
             return self.RESULT_OK
           else:
             return self.RESULT_UNKNOWN
@@ -108,13 +108,13 @@ class MetricAlert(BaseAlert):
           return self.RESULT_OK
     else:
       # critical values are lower
-      if value < crit_value:
+      if value <= crit_value:
         return self.RESULT_CRITICAL
-      elif value < warn_value:
+      elif value <= warn_value:
         return self.RESULT_WARNING
       else:
         if ok_value is not None:
-          if value < ok_value and value >= warn_value:
+          if value <= ok_value and value > warn_value:
             return self.RESULT_OK
           else:
             return self.RESULT_UNKNOWN

http://git-wip-us.apache.org/repos/asf/ambari/blob/c78d584e/ambari-server/src/main/java/org/apache/ambari/server/events/AlertDefinitionChangedEvent.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/events/AlertDefinitionChangedEvent.java b/ambari-server/src/main/java/org/apache/ambari/server/events/AlertDefinitionChangedEvent.java
new file mode 100644
index 0000000..ad063a3
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/events/AlertDefinitionChangedEvent.java
@@ -0,0 +1,55 @@
+/**
+ * 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.events;
+
+import org.apache.ambari.server.state.alert.AlertDefinition;
+
+/**
+ * The {@link AlertDefinitionChangedEvent} is used to represent that an
+ * {@link AlertDefinition} has been changed.
+ */
+public class AlertDefinitionChangedEvent extends ClusterEvent {
+
+  /**
+   * The changed alert defintiion
+   */
+  private final AlertDefinition m_definition;
+
+  /**
+   * Constructor.
+   *
+   * @param clusterId
+   *          the ID of the cluster that the definition is in.
+   * @param definition
+   *          the alert definition that was changed.
+   */
+  public AlertDefinitionChangedEvent(long clusterId,
+ AlertDefinition definition) {
+    super(AmbariEventType.ALERT_DEFINITION_CHANGED, clusterId);
+    m_definition = definition;
+  }
+
+  /**
+   * Get the registered alert definition.
+   *
+   * @return the alert definition (not {@code null}).
+   */
+  public AlertDefinition getDefinition() {
+    return m_definition;
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/c78d584e/ambari-server/src/main/java/org/apache/ambari/server/events/AlertDefinitionDisabledEvent.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/events/AlertDefinitionDisabledEvent.java b/ambari-server/src/main/java/org/apache/ambari/server/events/AlertDefinitionDisabledEvent.java
index 2fe3ba9..a3c6e68 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/events/AlertDefinitionDisabledEvent.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/events/AlertDefinitionDisabledEvent.java
@@ -39,7 +39,7 @@ public class AlertDefinitionDisabledEvent extends ClusterEvent {
    *          the alert definition being registered.
    */
   public AlertDefinitionDisabledEvent(long clusterId, long definitionId) {
-    super(AmbariEventType.ALERT_DEFINITION_REMOVAL, clusterId);
+    super(AmbariEventType.ALERT_DEFINITION_DISABLED, clusterId);
     m_definitionId = definitionId;
   }
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/c78d584e/ambari-server/src/main/java/org/apache/ambari/server/events/AmbariEvent.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/events/AmbariEvent.java b/ambari-server/src/main/java/org/apache/ambari/server/events/AmbariEvent.java
index 04217f1..7944908 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/events/AmbariEvent.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/events/AmbariEvent.java
@@ -53,6 +53,11 @@ public abstract class AmbariEvent {
     ALERT_DEFINITION_REGISTRATION,
 
     /**
+     * An alert definition was updated.
+     */
+    ALERT_DEFINITION_CHANGED,
+
+    /**
      * An alert definition is removed from the system.
      */
     ALERT_DEFINITION_REMOVAL,

http://git-wip-us.apache.org/repos/asf/ambari/blob/c78d584e/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/alerts/AlertAggregateListener.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/alerts/AlertAggregateListener.java b/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/alerts/AlertAggregateListener.java
index 6d8f34a..47fc4e3 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/alerts/AlertAggregateListener.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/alerts/AlertAggregateListener.java
@@ -71,41 +71,48 @@ public class AlertAggregateListener {
    */
   @Subscribe
   public void onAlertEvent(AlertReceivedEvent event) {
-    AlertDefinition def = m_aggregateMapping.getAggregateDefinition(
+    AlertDefinition aggregateDefinition = m_aggregateMapping.getAggregateDefinition(
         event.getClusterId(), event.getAlert().getName());
 
-    if (null == def || null == m_alertsDao) {
+    if (null == aggregateDefinition || null == m_alertsDao) {
       return;
     }
 
-    AggregateSource as = (AggregateSource) def.getSource();
+    AggregateSource aggregateSource = (AggregateSource) aggregateDefinition.getSource();
 
     AlertSummaryDTO summary = m_alertsDao.findAggregateCounts(
-        event.getClusterId(), as.getAlertName());
+        event.getClusterId(), aggregateSource.getAlertName());
 
-    Alert alert = new Alert(def.getName(), null, def.getServiceName(),
-        null, null, AlertState.UNKNOWN);
+    int okCount = summary.getOkCount();
+    int warningCount = summary.getWarningCount();
+    int criticalCount = summary.getCriticalCount();
+    int unknownCount = summary.getUnknownCount();
+    int totalCount = okCount + warningCount + criticalCount + unknownCount;
 
-    alert.setLabel(def.getLabel());
+    Alert alert = new Alert(aggregateDefinition.getName(), null,
+        aggregateDefinition.getServiceName(), null, null, AlertState.UNKNOWN);
+
+    alert.setLabel(aggregateDefinition.getLabel());
     alert.setTimestamp(System.currentTimeMillis());
 
-    if (0 == summary.getOkCount()) {
-      alert.setText("Cannot determine, there are no records");
+    if (0 == totalCount) {
+      alert.setText("There are no instances of the aggregated alert.");
     } else if (summary.getUnknownCount() > 0) {
-      alert.setText("There are alerts with status UNKNOWN.");
+      alert.setText("There are alerts with a state of UNKNOWN.");
     } else {
-      Reporting reporting = as.getReporting();
+      Reporting reporting = aggregateSource.getReporting();
 
       int numerator = summary.getCriticalCount() + summary.getWarningCount();
-      int denominator = summary.getOkCount();
-      double value = (double)(numerator) / denominator;
+      int denominator = totalCount;
+
+      double value = (double) (numerator) / denominator;
 
-      if (value > reporting.getCritical().getValue()) {
+      if (value >= reporting.getCritical().getValue()) {
         alert.setState(AlertState.CRITICAL);
         alert.setText(MessageFormat.format(reporting.getCritical().getText(),
             denominator, numerator));
 
-      } else if (value > reporting.getWarning().getValue()) {
+      } else if (value >= reporting.getWarning().getValue()) {
         alert.setState(AlertState.WARNING);
         alert.setText(MessageFormat.format(reporting.getWarning().getText(),
             denominator, numerator));

http://git-wip-us.apache.org/repos/asf/ambari/blob/c78d584e/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/alerts/AlertLifecycleListener.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/alerts/AlertLifecycleListener.java b/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/alerts/AlertLifecycleListener.java
index df95d3d..f21d800 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/alerts/AlertLifecycleListener.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/events/listeners/alerts/AlertLifecycleListener.java
@@ -20,9 +20,10 @@ package org.apache.ambari.server.events.listeners.alerts;
 import java.util.Set;
 
 import org.apache.ambari.server.EagerSingleton;
+import org.apache.ambari.server.events.AlertDefinitionChangedEvent;
 import org.apache.ambari.server.events.AlertDefinitionDeleteEvent;
-import org.apache.ambari.server.events.AlertHashInvalidationEvent;
 import org.apache.ambari.server.events.AlertDefinitionRegistrationEvent;
+import org.apache.ambari.server.events.AlertHashInvalidationEvent;
 import org.apache.ambari.server.events.publishers.AmbariEventPublisher;
 import org.apache.ambari.server.state.alert.AggregateDefinitionMapping;
 import org.apache.ambari.server.state.alert.AlertDefinition;
@@ -101,6 +102,28 @@ public class AlertLifecycleListener {
   }
 
   /**
+   * Handles {@link AlertDefinitionChangedEvent} by performing the following
+   * tasks:
+   * <ul>
+   * <li>Updating definition with {@link AggregateDefinitionMapping}</li>
+   * </ul>
+   *
+   * @param event
+   *          the event being handled.
+   */
+  @Subscribe
+  @AllowConcurrentEvents
+  public void onAmbariEvent(AlertDefinitionChangedEvent event) {
+    AlertDefinition definition = event.getDefinition();
+
+    LOG.debug("Updating alert definition {}", definition);
+
+    if (definition.getSource().getType() == SourceType.AGGREGATE) {
+      m_aggregateMapping.registerAggregate(event.getClusterId(), definition);
+    }
+  }
+
+  /**
    * Handles {@link AlertDefinitionDeleteEvent} by performing the following
    * tasks:
    * <ul>

http://git-wip-us.apache.org/repos/asf/ambari/blob/c78d584e/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 23de17e..2daf804 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
@@ -26,6 +26,8 @@ import javax.persistence.TypedQuery;
 
 import org.apache.ambari.server.AmbariException;
 import org.apache.ambari.server.controller.RootServiceResponseFactory;
+import org.apache.ambari.server.controller.internal.AlertDefinitionResourceProvider;
+import org.apache.ambari.server.events.AlertDefinitionChangedEvent;
 import org.apache.ambari.server.events.AlertDefinitionDeleteEvent;
 import org.apache.ambari.server.events.AlertDefinitionRegistrationEvent;
 import org.apache.ambari.server.events.publishers.AmbariEventPublisher;
@@ -81,7 +83,9 @@ public class AlertDefinitionDAO {
    * Publishes the following events:
    * <ul>
    * <li>{@link AlertDefinitionRegistrationEvent} when new alerts are merged
-   * from the stack</li>
+   * from the stack or created from the {@link AlertDefinitionResourceProvider}</li>
+   * <li>{@link AlertDefinitionChangedEvent} when alerts are updated.</li>
+   * <li>{@link AlertDefinitionDeleteEvent} when alerts are removed</li>
    * </ul>
    */
   @Inject
@@ -279,7 +283,7 @@ public class AlertDefinitionDAO {
   /**
    * Persists a new alert definition, also creating the associated
    * {@link AlertGroupEntity} relationship for the definition's service default
-   * group.
+   * group. Fires an {@link AlertDefinitionRegistrationEvent}.
    *
    * @param alertDefinition
    *          the definition to persist (not {@code null}).
@@ -330,7 +334,7 @@ public class AlertDefinitionDAO {
 
   /**
    * Merge the speicified alert definition with the existing definition in the
-   * database.
+   * database. Fires an {@link AlertDefinitionChangedEvent}.
    *
    * @param alertDefinition
    *          the definition to merge (not {@code null}).
@@ -338,7 +342,17 @@ public class AlertDefinitionDAO {
    */
   @Transactional
   public AlertDefinitionEntity merge(AlertDefinitionEntity alertDefinition) {
-    return entityManagerProvider.get().merge(alertDefinition);
+    AlertDefinitionEntity entity = entityManagerProvider.get().merge(
+        alertDefinition);
+
+    AlertDefinition definition = alertDefinitionFactory.coerce(entity);
+
+    AlertDefinitionChangedEvent event = new AlertDefinitionChangedEvent(
+        alertDefinition.getClusterId(), definition);
+
+    eventPublisher.publish(event);
+
+    return entity;
   }
 
   /**
@@ -360,7 +374,8 @@ public class AlertDefinitionDAO {
 
   /**
    * Removes the specified alert definition and all related history and
-   * associations from the database.
+   * associations from the database. Fires an {@link AlertDefinitionDeleteEvent}
+   * .
    *
    * @param alertDefinition
    *          the definition to remove.

http://git-wip-us.apache.org/repos/asf/ambari/blob/c78d584e/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/AlertsDAO.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/AlertsDAO.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/AlertsDAO.java
index 797d759..6fac728 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/AlertsDAO.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/AlertsDAO.java
@@ -676,7 +676,7 @@ public class AlertsDAO {
   public AlertSummaryDTO findAggregateCounts(long clusterId, String alertName) {
     StringBuilder sb = new StringBuilder();
     sb.append("SELECT NEW %s (");
-    sb.append("COUNT(history), ");
+    sb.append("SUM(CASE WHEN history.alertState = %s.%s THEN 1 ELSE 0 END), ");
     sb.append("SUM(CASE WHEN history.alertState = %s.%s THEN 1 ELSE 0 END), ");
     sb.append("SUM(CASE WHEN history.alertState = %s.%s THEN 1 ELSE 0 END), ");
     sb.append("SUM(CASE WHEN history.alertState = %s.%s THEN 1 ELSE 0 END)) ");
@@ -685,6 +685,7 @@ public class AlertsDAO {
 
     String str = String.format(sb.toString(),
         AlertSummaryDTO.class.getName(),
+        AlertState.class.getName(), AlertState.OK.name(),
         AlertState.class.getName(), AlertState.WARNING.name(),
         AlertState.class.getName(), AlertState.CRITICAL.name(),
         AlertState.class.getName(), AlertState.UNKNOWN.name());

http://git-wip-us.apache.org/repos/asf/ambari/blob/c78d584e/ambari-server/src/main/java/org/apache/ambari/server/state/alert/Source.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/alert/Source.java b/ambari-server/src/main/java/org/apache/ambari/server/state/alert/Source.java
index 09ddeb3..25a4a0a 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/alert/Source.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/alert/Source.java
@@ -40,6 +40,14 @@ public abstract class Source {
   }
 
   /**
+   * @param type
+   *          the type to set.
+   */
+  public void setType(SourceType type) {
+    this.type = type;
+  }
+
+  /**
    * @return
    */
   public Reporting getReporting() {
@@ -48,7 +56,7 @@ public abstract class Source {
 
   /**
    * Sets the OK/WARNING/CRTICAL structures.
-   * 
+   *
    * @param reporting
    *          the reporting structure or {@code null} for none.
    */

http://git-wip-us.apache.org/repos/asf/ambari/blob/c78d584e/ambari-server/src/main/java/org/apache/ambari/server/state/services/AlertNoticeDispatchService.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/services/AlertNoticeDispatchService.java b/ambari-server/src/main/java/org/apache/ambari/server/state/services/AlertNoticeDispatchService.java
index 69f3393..b085112 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/services/AlertNoticeDispatchService.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/services/AlertNoticeDispatchService.java
@@ -82,8 +82,8 @@ import com.google.inject.Provider;
 
 /**
  * The {@link AlertNoticeDispatchService} is used to scan the database for
- * {@link AlertNoticeEntity} that are in the {@link NotificationState#PENDING}.
- * It will then process them through the dispatch system.
+ * {@link AlertNoticeEntity} that are in the {@link NotificationState#PENDING}
+ * state. It will then process them through the dispatch system.
  * <p/>
  * The dispatch system will then make a callback to
  * {@link AlertNoticeDispatchCallback} so that the {@link NotificationState} can

http://git-wip-us.apache.org/repos/asf/ambari/blob/c78d584e/ambari-server/src/test/java/org/apache/ambari/server/orm/dao/AlertsDAOTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/orm/dao/AlertsDAOTest.java b/ambari-server/src/test/java/org/apache/ambari/server/orm/dao/AlertsDAOTest.java
index 31e93e4..f831d14 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/orm/dao/AlertsDAOTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/orm/dao/AlertsDAOTest.java
@@ -616,7 +616,7 @@ public class AlertsDAOTest {
 
     summary = m_dao.findAggregateCounts(m_cluster.getClusterId(),
         "many_per_cluster");
-    assertEquals(2, summary.getOkCount());
+    assertEquals(1, summary.getOkCount());
     assertEquals(0, summary.getWarningCount());
     assertEquals(1, summary.getCriticalCount());
     assertEquals(0, summary.getUnknownCount());

http://git-wip-us.apache.org/repos/asf/ambari/blob/c78d584e/ambari-server/src/test/java/org/apache/ambari/server/state/alerts/AlertEventPublisherTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/state/alerts/AlertEventPublisherTest.java b/ambari-server/src/test/java/org/apache/ambari/server/state/alerts/AlertEventPublisherTest.java
index 271c078..6331516 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/state/alerts/AlertEventPublisherTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/state/alerts/AlertEventPublisherTest.java
@@ -18,10 +18,12 @@
 package org.apache.ambari.server.state.alerts;
 
 import java.lang.reflect.Field;
+import java.util.UUID;
 
 import junit.framework.Assert;
 
 import org.apache.ambari.server.api.services.AmbariMetaInfo;
+import org.apache.ambari.server.events.AlertDefinitionChangedEvent;
 import org.apache.ambari.server.events.AlertDefinitionDeleteEvent;
 import org.apache.ambari.server.events.AmbariEvent;
 import org.apache.ambari.server.events.listeners.alerts.AlertLifecycleListener;
@@ -43,12 +45,16 @@ import org.apache.ambari.server.state.StackId;
 import org.apache.ambari.server.state.alert.AggregateDefinitionMapping;
 import org.apache.ambari.server.state.alert.AggregateSource;
 import org.apache.ambari.server.state.alert.AlertDefinition;
+import org.apache.ambari.server.state.alert.Reporting;
+import org.apache.ambari.server.state.alert.Reporting.ReportTemplate;
 import org.apache.ambari.server.state.alert.Scope;
+import org.apache.ambari.server.state.alert.SourceType;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
 import com.google.common.eventbus.EventBus;
+import com.google.gson.Gson;
 import com.google.inject.Guice;
 import com.google.inject.Injector;
 import com.google.inject.persist.PersistService;
@@ -160,6 +166,71 @@ public class AlertEventPublisherTest {
   }
 
   /**
+   * Tests that {@link AlertDefinitionChangedEvent} instances are fired when a
+   * definition is updated.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testAlertDefinitionChanged() throws Exception {
+    installHdfsService();
+
+    int definitionCount = definitionDao.findAll().size();
+    AlertDefinitionEntity definition = ormHelper.createAlertDefinition(1L);
+    Assert.assertEquals(definitionCount + 1, definitionDao.findAll().size());
+
+    AggregateSource source = new AggregateSource();
+    Reporting reporting = new Reporting();
+    ReportTemplate okTemplate = new ReportTemplate();
+    okTemplate.setValue(50.0d);
+    okTemplate.setText("foo");
+    reporting.setOk(okTemplate);
+    source.setReporting(reporting);
+    source.setAlertName(definition.getDefinitionName());
+    source.setType(SourceType.AGGREGATE);
+
+    AlertDefinitionEntity aggregateEntity = new AlertDefinitionEntity();
+    aggregateEntity.setClusterId(1L);
+    aggregateEntity.setComponentName("DATANODE");
+    aggregateEntity.setEnabled(true);
+    aggregateEntity.setDefinitionName("datanode_aggregate");
+    aggregateEntity.setScope(Scope.ANY);
+    aggregateEntity.setServiceName("HDFS");
+    aggregateEntity.setSource(new Gson().toJson(source));
+    aggregateEntity.setHash(UUID.randomUUID().toString());
+    aggregateEntity.setScheduleInterval(1);
+    aggregateEntity.setSourceType(SourceType.AGGREGATE);
+
+    // creating the aggregate alert will register it with the mapping
+    definitionDao.create(aggregateEntity);
+
+    // pull it out of the mapping and compare fields
+    AlertDefinition aggregate = aggregateMapping.getAggregateDefinition(1L,
+        source.getAlertName());
+
+    Assert.assertNotNull(aggregate);
+    Assert.assertEquals("foo",
+        aggregate.getSource().getReporting().getOk().getText());
+
+    // change something about the aggregate's reporting
+    String sourceText = aggregateEntity.getSource();
+    sourceText = sourceText.replace("foo", "bar");
+    aggregateEntity.setSource(sourceText);
+
+    // save the aggregate; this should trigger the event,
+    // causing the updated aggregate definition to be mapped
+    definitionDao.merge(aggregateEntity);
+
+    // check the aggregate mapping for the new value
+    aggregate = aggregateMapping.getAggregateDefinition(1L,
+        source.getAlertName());
+
+    Assert.assertNotNull(aggregate);
+    Assert.assertEquals("bar",
+        aggregate.getSource().getReporting().getOk().getText());
+  }
+
+  /**
    * Tests that {@link AlertDefinitionDeleteEvent} instances are fired when a
    * definition is removed.
    *