瀏覽代碼

AMBARI-7336 - Alerts: Create Default AlertGroup When Service Is Installed (jonathanhurley)

Jonathan Hurley 10 年之前
父節點
當前提交
b7ae2d74ad

+ 2 - 0
ambari-server/src/main/java/org/apache/ambari/server/controller/ControllerModule.java

@@ -52,6 +52,7 @@ import org.apache.ambari.server.controller.internal.MemberResourceProvider;
 import org.apache.ambari.server.controller.internal.ServiceResourceProvider;
 import org.apache.ambari.server.controller.spi.ResourceProvider;
 import org.apache.ambari.server.events.listeners.AlertReceivedListener;
+import org.apache.ambari.server.events.listeners.AlertServiceStateListener;
 import org.apache.ambari.server.events.listeners.AlertStateChangedListener;
 import org.apache.ambari.server.orm.DBAccessor;
 import org.apache.ambari.server.orm.DBAccessorImpl;
@@ -313,5 +314,6 @@ public class ControllerModule extends AbstractModule {
     // them as eager singletons to have them register with the eventbus
     bind(AlertReceivedListener.class).asEagerSingleton();
     bind(AlertStateChangedListener.class).asEagerSingleton();
+    bind(AlertServiceStateListener.class).asEagerSingleton();
   }
 }

+ 1 - 1
ambari-server/src/main/java/org/apache/ambari/server/events/AlertEvent.java

@@ -20,7 +20,7 @@ package org.apache.ambari.server.events;
 import org.apache.ambari.server.state.Alert;
 
 /**
- * The {@link AlertEvent} class the base for all events related to alerts.
+ * The {@link AlertEvent} class is the base for all events related to alerts.
  */
 public abstract class AlertEvent {
 

+ 75 - 0
ambari-server/src/main/java/org/apache/ambari/server/events/AmbariEvent.java

@@ -0,0 +1,75 @@
+/**
+ * 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;
+
+
+/**
+ * The {@link AmbariEvent} class is the base for all events in Ambari.
+ */
+public abstract class AmbariEvent {
+
+  /**
+   * The {@link AmbariEventType} defines the type of Ambari event.
+   */
+  public enum AmbariEventType {
+    /**
+     * A service was successfully installed.
+     */
+    SERVICE_INSTALL_SUCCESS;
+  }
+
+  /**
+   * The concrete event's type.
+   */
+  protected final AmbariEventType m_eventType;
+
+  /**
+   * The cluster ID.
+   */
+  protected final long m_clusterId;
+
+  /**
+   * Constructor.
+   *
+   * @param eventType
+   *          the type of event (not {@code null}).
+   * @param clusterId
+   */
+  public AmbariEvent(AmbariEventType eventType, long clusterId) {
+    m_eventType = eventType;
+    m_clusterId = clusterId;
+  }
+
+  /**
+   * Gets the cluster ID that the event belongs to.
+   *
+   * @return the ID of the cluster.
+   */
+  public long getClusterId() {
+    return m_clusterId;
+  }
+
+  /**
+   * Gets the type of {@link AmbariEvent}.
+   *
+   * @return the event type (never {@code null}).
+   */
+  public AmbariEventType getType() {
+    return m_eventType;
+  }
+}

+ 75 - 0
ambari-server/src/main/java/org/apache/ambari/server/events/ServiceEvent.java

@@ -0,0 +1,75 @@
+/**
+ * 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;
+
+
+/**
+ * The {@link ServiceEvent} class is the base for all service events in Ambari.
+ */
+public abstract class ServiceEvent extends AmbariEvent {
+
+  /**
+   * The name of the service.
+   */
+  protected final String m_serviceName;
+
+  /**
+   * The name of the services' stack.
+   */
+  protected final String m_stackName;
+
+  /**
+   * The version of the services' stack.
+   */
+  protected final String m_stackVersion;
+
+  /**
+   * Constructor.
+   *
+   * @param eventType
+   * @param clusterId
+   */
+  public ServiceEvent(AmbariEventType eventType, long clusterId,
+      String stackName, String stackVersion, String serviceName) {
+    super(eventType, clusterId);
+    m_stackName = stackName;
+    m_stackVersion = stackVersion;
+    m_serviceName = serviceName;
+  }
+
+  /**
+   * @return the serviceName (never {@code null}).
+   */
+  public String getServiceName() {
+    return m_serviceName;
+  }
+
+  /**
+   * @return the stackName (never {@code null}).
+   */
+  public String getStackName() {
+    return m_stackName;
+  }
+
+  /**
+   * @return the stackVersion (never {@code null}).
+   */
+  public String getStackVersion() {
+    return m_stackVersion;
+  }
+}

+ 52 - 0
ambari-server/src/main/java/org/apache/ambari/server/events/ServiceInstalledEvent.java

@@ -0,0 +1,52 @@
+/**
+ * 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;
+
+/**
+ * The {@link ServiceInstalledEvent} class is fired when a service is
+ * successfully installed.
+ */
+public class ServiceInstalledEvent extends ServiceEvent {
+  /**
+   * Constructor.
+   *
+   * @param clusterId
+   * @param stackName
+   * @param stackVersion
+   * @param serviceName
+   */
+  public ServiceInstalledEvent(long clusterId, String stackName,
+      String stackVersion, String serviceName) {
+    super(AmbariEventType.SERVICE_INSTALL_SUCCESS, clusterId, stackName,
+        stackVersion, serviceName);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public String toString() {
+    StringBuilder buffer = new StringBuilder("ServiceInstalledEvent{ ");
+    buffer.append("cluserId=").append(m_clusterId);
+    buffer.append(", stackName=").append(m_stackName);
+    buffer.append(", stackVersion=").append(m_stackVersion);
+    buffer.append(", serviceName=").append(m_serviceName);
+    buffer.append("}");
+    return buffer.toString();
+  }
+}

+ 3 - 1
ambari-server/src/main/java/org/apache/ambari/server/events/listeners/AlertReceivedListener.java

@@ -38,7 +38,7 @@ import com.google.inject.Singleton;
 
 /**
  * The {@link AlertReceivedListener} class handles {@link AlertReceivedEvent}
- * and updates the appropirate DAOs. It may also fire new
+ * and updates the appropriate DAOs. It may also fire new
  * {@link AlertStateChangeEvent} when an {@link AlertState} change is detected.
  */
 @Singleton
@@ -80,6 +80,8 @@ public class AlertReceivedListener {
   @Subscribe
   @AllowConcurrentEvents
   public void onAlertEvent(AlertReceivedEvent event) {
+    LOG.debug(event);
+
     long clusterId = event.getClusterId();
     Alert alert = event.getAlert();
 

+ 147 - 0
ambari-server/src/main/java/org/apache/ambari/server/events/listeners/AlertServiceStateListener.java

@@ -0,0 +1,147 @@
+/**
+ * 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.listeners;
+
+import java.text.MessageFormat;
+import java.util.Set;
+
+import org.apache.ambari.server.AmbariException;
+import org.apache.ambari.server.api.services.AmbariMetaInfo;
+import org.apache.ambari.server.controller.AmbariServer;
+import org.apache.ambari.server.controller.ControllerModule;
+import org.apache.ambari.server.events.ServiceInstalledEvent;
+import org.apache.ambari.server.events.publishers.AmbariEventPublisher;
+import org.apache.ambari.server.orm.GuiceJpaInitializer;
+import org.apache.ambari.server.orm.dao.AlertDefinitionDAO;
+import org.apache.ambari.server.orm.dao.AlertDispatchDAO;
+import org.apache.ambari.server.orm.entities.AlertDefinitionEntity;
+import org.apache.ambari.server.orm.entities.AlertGroupEntity;
+import org.apache.ambari.server.state.alert.AlertDefinition;
+import org.apache.ambari.server.state.alert.AlertDefinitionFactory;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import com.google.common.eventbus.AllowConcurrentEvents;
+import com.google.common.eventbus.Subscribe;
+import com.google.inject.Inject;
+import com.google.inject.Provider;
+import com.google.inject.Singleton;
+
+/**
+ * The {@link AlertServiceStateListener} class handles
+ * {@link ServiceInstalledEvent} and ensures that {@link AlertDefinitionEntity}
+ * and {@link AlertGroupEntity} instances are correctly populated.
+ */
+@Singleton
+public class AlertServiceStateListener {
+  /**
+   * Logger.
+   */
+  private static Log LOG = LogFactory.getLog(AlertServiceStateListener.class);
+
+  /**
+   * Services metainfo; injected lazily as a {@link Provider} since JPA is not
+   * fully initialized when this singleton is eagerly instantiated. See
+   * {@link AmbariServer#main(String[])} and the ordering of
+   * {@link ControllerModule} and {@link GuiceJpaInitializer}.
+   */
+  @Inject
+  private Provider<AmbariMetaInfo> m_metaInfoProvider;
+
+  /**
+   * Used when a service is installed to read alert definitions from the stack
+   * and coerce them into {@link AlertDefinitionEntity}.
+   */
+  @Inject
+  private AlertDefinitionFactory m_alertDefinitionFactory;
+
+  /**
+   * Used when a service is installed to insert a default
+   * {@link AlertGroupEntity} into the database.
+   */
+  @Inject
+  private AlertDispatchDAO m_alertDispatchDao;
+
+  /**
+   * Used when a service is installed to insert {@link AlertDefinitionEntity}
+   * into the database.
+   */
+  @Inject
+  private AlertDefinitionDAO m_definitionDao;
+
+  /**
+   * Constructor.
+   *
+   * @param publisher
+   */
+  @Inject
+  public AlertServiceStateListener(AmbariEventPublisher publisher) {
+    publisher.register(this);
+  }
+
+
+  /**
+   * Handles service installed events by populating the database with all known
+   * alert definitions for the newly installed service and creates the service's
+   * default alert group.
+   *
+   * @param event
+   *          the published event being handled (not {@code null}).
+   */
+  @Subscribe
+  @AllowConcurrentEvents
+  public void onAmbariEvent(ServiceInstalledEvent event) {
+    LOG.debug(event);
+
+    long clusterId = event.getClusterId();
+    String stackName = event.getStackName();
+    String stackVersion = event.getStackVersion();
+    String serviceName = event.getServiceName();
+
+    // populate alert definitions for the new service from the database, but
+    // don't worry about sending down commands to the agents; the host
+    // components are not yet bound to the hosts so we'd have no way of knowing
+    // which hosts are invalidated; do that in another impl
+    try {
+      Set<AlertDefinition> alertDefinitions = m_metaInfoProvider.get().getAlertDefinitions(
+          stackName, stackVersion, serviceName);
+
+      for (AlertDefinition definition : alertDefinitions) {
+        AlertDefinitionEntity entity = m_alertDefinitionFactory.coerce(
+            clusterId,
+            definition);
+
+        m_definitionDao.create(entity);
+      }
+    } catch (AmbariException ae) {
+      String message = MessageFormat.format(
+          "Unable to populate alert definitions from the database during installation of {0}",
+          serviceName);
+      LOG.error(message, ae);
+    }
+
+    // create the default alert group for the new service
+    AlertGroupEntity serviceAlertGroup = new AlertGroupEntity();
+    serviceAlertGroup.setClusterId(clusterId);
+    serviceAlertGroup.setDefault(true);
+    serviceAlertGroup.setGroupName(serviceName);
+    serviceAlertGroup.setServiceName(serviceName);
+
+    m_alertDispatchDao.create(serviceAlertGroup);
+  }
+}

+ 2 - 0
ambari-server/src/main/java/org/apache/ambari/server/events/listeners/AlertStateChangedListener.java

@@ -72,6 +72,8 @@ public class AlertStateChangedListener {
   @Subscribe
   @AllowConcurrentEvents
   public void onAlertEvent(AlertStateChangeEvent event) {
+    LOG.debug(event);
+
     AlertHistoryEntity history = event.getNewHistoricalEntry();
     AlertDefinitionEntity definition = history.getAlertDefinition();
 

+ 4 - 4
ambari-server/src/main/java/org/apache/ambari/server/events/publishers/AlertEventPublisher.java

@@ -42,13 +42,13 @@ public final class AlertEventPublisher {
   /**
    * A multi-threaded event bus that can handle dispatching {@link AlertEvent}s.
    */
-  private final EventBus s_eventBus;
+  private final EventBus m_eventBus;
 
   /**
    * Constructor.
    */
   public AlertEventPublisher() {
-    s_eventBus = new AsyncEventBus(Executors.newFixedThreadPool(2,
+    m_eventBus = new AsyncEventBus(Executors.newFixedThreadPool(2,
         new AlertEventBusThreadFactory()));
   }
 
@@ -59,7 +59,7 @@ public final class AlertEventPublisher {
    * @param event
    */
   public void publish(AlertEvent event) {
-    s_eventBus.post(event);
+    m_eventBus.post(event);
   }
 
   /**
@@ -70,7 +70,7 @@ public final class AlertEventPublisher {
    *          the listener to receive events.
    */
   public void register(Object object) {
-    s_eventBus.register(object);
+    m_eventBus.register(object);
   }
 
   /**

+ 66 - 0
ambari-server/src/main/java/org/apache/ambari/server/events/publishers/AmbariEventPublisher.java

@@ -0,0 +1,66 @@
+/**
+ * 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.publishers;
+
+import org.apache.ambari.server.events.AmbariEvent;
+
+import com.google.common.eventbus.EventBus;
+import com.google.common.eventbus.Subscribe;
+import com.google.inject.Singleton;
+
+/**
+ * The {@link AmbariEventPublisher} is used to publish instances of
+ * {@link AmbariEvent} to any {@link Subscribe} interested. It uses a
+ * single-threaded, serial {@link EventBus}.
+ */
+@Singleton
+public final class AmbariEventPublisher {
+
+  /**
+   * A single threaded event bus for processing Ambari events in serial.
+   */
+  private final EventBus m_eventBus;
+
+  /**
+   * Constructor.
+   */
+  public AmbariEventPublisher() {
+    m_eventBus = new EventBus("ambari-event-bus");
+  }
+
+  /**
+   * Publishes the specified event to all registered listeners that
+   * {@link Subscribe} to any of the {@link AmbariEvent} instances.
+   *
+   * @param event
+   */
+  public void publish(AmbariEvent event) {
+    m_eventBus.post(event);
+  }
+
+  /**
+   * Register a listener to receive events. The listener should use the
+   * {@link Subscribe} annotation.
+   *
+   * @param object
+   *          the listener to receive events.
+   */
+  public void register(Object object) {
+    m_eventBus.register(object);
+  }
+}

+ 13 - 36
ambari-server/src/main/java/org/apache/ambari/server/state/ServiceImpl.java

@@ -20,7 +20,6 @@ package org.apache.ambari.server.state;
 
 import java.util.HashMap;
 import java.util.Map;
-import java.util.Set;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
@@ -28,18 +27,16 @@ import org.apache.ambari.server.AmbariException;
 import org.apache.ambari.server.ServiceComponentNotFoundException;
 import org.apache.ambari.server.api.services.AmbariMetaInfo;
 import org.apache.ambari.server.controller.ServiceResponse;
-import org.apache.ambari.server.orm.dao.AlertDefinitionDAO;
+import org.apache.ambari.server.events.ServiceInstalledEvent;
+import org.apache.ambari.server.events.publishers.AmbariEventPublisher;
 import org.apache.ambari.server.orm.dao.ClusterDAO;
 import org.apache.ambari.server.orm.dao.ClusterServiceDAO;
 import org.apache.ambari.server.orm.dao.ServiceDesiredStateDAO;
-import org.apache.ambari.server.orm.entities.AlertDefinitionEntity;
 import org.apache.ambari.server.orm.entities.ClusterEntity;
 import org.apache.ambari.server.orm.entities.ClusterServiceEntity;
 import org.apache.ambari.server.orm.entities.ClusterServiceEntityPK;
 import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntity;
 import org.apache.ambari.server.orm.entities.ServiceDesiredStateEntity;
-import org.apache.ambari.server.state.alert.AlertDefinition;
-import org.apache.ambari.server.state.alert.AlertDefinitionFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -80,18 +77,10 @@ public class ServiceImpl implements Service {
   private AmbariMetaInfo ambariMetaInfo;
 
   /**
-   * Used when a service is installed to insert {@link AlertDefinitionEntity}
-   * into the database.
+   * Used to publish events relating to service CRUD operations.
    */
   @Inject
-  private AlertDefinitionDAO alertDefinitionDAO;
-
-  /**
-   * Used when a service is installed to read alert definitions from the stack
-   * and coerce them into {@link AlertDefinitionEntity}.
-   */
-  @Inject
-  private AlertDefinitionFactory alertDefinitionFactory;
+  private AmbariEventPublisher eventPublisher;
 
   private void init() {
     // TODO load from DB during restart?
@@ -476,6 +465,15 @@ public class ServiceImpl implements Service {
           refresh();
           cluster.refresh();
           persisted = true;
+
+          // publish the service installed event
+          StackId stackId = cluster.getDesiredStackVersion();
+
+          ServiceInstalledEvent event = new ServiceInstalledEvent(
+              getClusterId(), stackId.getStackName(),
+              stackId.getStackVersion(), getName());
+
+          eventPublisher.publish(event);
         } else {
           saveIfPersisted();
         }
@@ -491,7 +489,6 @@ public class ServiceImpl implements Service {
   @Transactional
   protected void persistEntities() {
     long clusterId = cluster.getClusterId();
-    StackId stackId = cluster.getDesiredStackVersion();
 
     ClusterEntity clusterEntity = clusterDAO.findById(clusterId);
     serviceEntity.setClusterEntity(clusterEntity);
@@ -501,26 +498,6 @@ public class ServiceImpl implements Service {
     clusterDAO.merge(clusterEntity);
     clusterServiceDAO.merge(serviceEntity);
     serviceDesiredStateDAO.merge(serviceDesiredStateEntity);
-
-    // populate alert definitions for the new service from the database, but
-    // don't worry about sending down commands to the agents; the host
-    // components are not yet bound to the hosts so we'd have no way of knowing
-    // which hosts are invalidated; do that in another impl
-    try{
-      Set<AlertDefinition> alertDefinitions = ambariMetaInfo.getAlertDefinitions(
-          stackId.getStackName(), stackId.getStackVersion(), getName());
-
-      for (AlertDefinition definition : alertDefinitions) {
-        AlertDefinitionEntity entity = alertDefinitionFactory.coerce(clusterId,
-            definition);
-
-        alertDefinitionDAO.create(entity);
-      }
-    } catch( AmbariException ae ){
-      LOG.error(
-          "Unable to populate alert definitions from the database during installation of {}",
-          getName(), ae);
-    }
   }
 
   @Transactional

+ 132 - 0
ambari-server/src/test/java/org/apache/ambari/server/state/alerts/AlertEventPublisherTest.java

@@ -0,0 +1,132 @@
+/**
+ * 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 junit.framework.Assert;
+
+import org.apache.ambari.server.api.services.AmbariMetaInfo;
+import org.apache.ambari.server.events.AmbariEvent;
+import org.apache.ambari.server.events.listeners.AlertServiceStateListener;
+import org.apache.ambari.server.orm.GuiceJpaInitializer;
+import org.apache.ambari.server.orm.InMemoryDefaultTestModule;
+import org.apache.ambari.server.orm.dao.AlertDefinitionDAO;
+import org.apache.ambari.server.orm.dao.AlertDispatchDAO;
+import org.apache.ambari.server.orm.entities.AlertDefinitionEntity;
+import org.apache.ambari.server.orm.entities.AlertGroupEntity;
+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.ServiceFactory;
+import org.apache.ambari.server.state.StackId;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+import com.google.inject.persist.PersistService;
+
+/**
+ * Tests that {@link AmbariEvent} instances are fired correctly and that alert
+ * data is bootstrapped into the database.
+ */
+public class AlertEventPublisherTest {
+
+  private AlertDispatchDAO dispatchDao;
+  private AlertDefinitionDAO definitionDao;
+  private Clusters clusters;
+  private Cluster cluster;
+  private String clusterName;
+  private Injector injector;
+  private ServiceFactory serviceFactory;
+  private AmbariMetaInfo metaInfo;
+
+  /**
+   *
+   */
+  @Before
+  public void setup() throws Exception {
+    injector = Guice.createInjector(new InMemoryDefaultTestModule());
+    injector.getInstance(GuiceJpaInitializer.class);
+
+    // force singleton init via Guice so the listener registers with the bus
+    injector.getInstance(AlertServiceStateListener.class);
+
+    dispatchDao = injector.getInstance(AlertDispatchDAO.class);
+    definitionDao = injector.getInstance(AlertDefinitionDAO.class);
+    clusters = injector.getInstance(Clusters.class);
+    serviceFactory = injector.getInstance(ServiceFactory.class);
+
+    metaInfo = injector.getInstance(AmbariMetaInfo.class);
+    metaInfo.init();
+
+    clusterName = "foo";
+    clusters.addCluster(clusterName);
+    cluster = clusters.getCluster(clusterName);
+    cluster.setDesiredStackVersion(new StackId("HDP", "2.0.6"));
+    Assert.assertNotNull(cluster);
+  }
+
+  /**
+   * @throws Exception
+   */
+  @After
+  public void teardown() throws Exception {
+    injector.getInstance(PersistService.class).stop();
+    injector = null;
+  }
+
+  /**
+   * Tests that a default {@link AlertGroupEntity} is created when a service is
+   * installed.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testDefaultAlertGroupCreation() throws Exception {
+    Assert.assertEquals(0, dispatchDao.findAllGroups().size());
+    installHdfsService();
+    Assert.assertEquals(1, dispatchDao.findAllGroups().size());
+  }
+
+  /**
+   * Tests that all {@link AlertDefinitionEntity} instances are created for the
+   * installed service.
+   * 
+   * @throws Exception
+   */
+  @Test
+  public void testAlertDefinitionInsertion() throws Exception {
+    Assert.assertEquals(0, definitionDao.findAll().size());
+    installHdfsService();
+    Assert.assertEquals(4, definitionDao.findAll().size());
+  }
+
+  /**
+   * Calls {@link Service#persist()} to mock a service install.
+   */
+  private void installHdfsService() throws Exception {
+    String serviceName = "HDFS";
+    Service service = serviceFactory.createNew(cluster, serviceName);
+    cluster.addService(service);
+    service.persist();
+    service = cluster.getService(serviceName);
+
+    Assert.assertNotNull(service);
+  }
+}