You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by nc...@apache.org on 2014/07/23 22:47:29 UTC

git commit: AMBARI-6569. Alerts: Add remaining CRUD operations for AlertDefinitions (ncole)

Repository: ambari
Updated Branches:
  refs/heads/trunk 98f531d39 -> 49d7a3da8


AMBARI-6569. Alerts: Add remaining CRUD operations for AlertDefinitions (ncole)


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

Branch: refs/heads/trunk
Commit: 49d7a3da8db3aae5c4f0f94def37bb5a3f40989a
Parents: 98f531d
Author: Nate Cole <nc...@hortonworks.com>
Authored: Tue Jul 22 21:39:12 2014 -0400
Committer: Nate Cole <nc...@hortonworks.com>
Committed: Wed Jul 23 16:22:21 2014 -0400

----------------------------------------------------------------------
 .../api/services/AlertDefinitionService.java    |  34 ++-
 .../server/api/services/AmbariMetaInfo.java     |  50 ++++-
 .../AlertDefinitionResourceProvider.java        | 197 +++++++++++++++--
 .../server/state/alert/AlertDefinition.java     |  41 ++--
 .../ambari/server/state/alert/MetricAlert.java  |  57 -----
 .../ambari/server/state/alert/MetricSource.java |  57 +++++
 .../ambari/server/state/alert/Source.java       |  34 +++
 .../src/main/resources/properties.json          |   4 +-
 .../stacks/HDP/2.0.6/services/HDFS/alerts.json  |  19 +-
 .../AlertDefinitionResourceProviderTest.java    | 220 ++++++++++++++++---
 .../stacks/HDP/2.0.5/services/HDFS/alerts.json  |  19 +-
 11 files changed, 583 insertions(+), 149 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/49d7a3da/ambari-server/src/main/java/org/apache/ambari/server/api/services/AlertDefinitionService.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/api/services/AlertDefinitionService.java b/ambari-server/src/main/java/org/apache/ambari/server/api/services/AlertDefinitionService.java
index ee3d29a..506f911 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/api/services/AlertDefinitionService.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/api/services/AlertDefinitionService.java
@@ -20,8 +20,10 @@ package org.apache.ambari.server.api.services;
 import java.util.HashMap;
 import java.util.Map;
 
+import javax.ws.rs.DELETE;
 import javax.ws.rs.GET;
 import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
@@ -38,10 +40,10 @@ import org.apache.ambari.server.controller.spi.Resource;
  */
 public class AlertDefinitionService extends BaseService {
 
-  private String m_clusterName = null;
+  private String clusterName = null;
   
   AlertDefinitionService(String clusterName) {
-    m_clusterName = clusterName;
+    this.clusterName = clusterName;
   }
   
   @GET
@@ -50,7 +52,7 @@ public class AlertDefinitionService extends BaseService {
       @Context HttpHeaders headers,
       @Context UriInfo ui) {
     return handleRequest(headers, body, ui, Request.Type.GET,
-      createResourceInstance(m_clusterName, null));
+      createResourceInstance(clusterName, null));
   }
   
   @POST
@@ -59,9 +61,31 @@ public class AlertDefinitionService extends BaseService {
       @Context HttpHeaders headers,
       @Context UriInfo ui) {
     return handleRequest(headers, body, ui, Request.Type.POST,
-      createResourceInstance(m_clusterName, null));
+      createResourceInstance(clusterName, null));
   }
   
+  @PUT
+  @Path("{alertDefinitionId}")
+  @Produces("text/plain")
+  public Response updateDefinition(String body,
+      @Context HttpHeaders headers,
+      @Context UriInfo ui,
+      @PathParam("alertDefinitionId") Long id) {
+    return handleRequest(headers, body, ui, Request.Type.PUT,
+      createResourceInstance(clusterName, id));
+  }
+  
+  @DELETE
+  @Path("{alertDefinitionId}")
+  @Produces("text/plain")
+  public Response deleteDefinition(String body,
+      @Context HttpHeaders headers,
+      @Context UriInfo ui,
+      @PathParam("alertDefinitionId") Long id) {
+    return handleRequest(headers, body, ui, Request.Type.DELETE,
+      createResourceInstance(clusterName, id));
+  }
+
   
   @GET
   @Path("{alertDefinitionId}")
@@ -71,7 +95,7 @@ public class AlertDefinitionService extends BaseService {
       @Context UriInfo ui,
       @PathParam("alertDefinitionId") Long id) {
     return handleRequest(headers, body, ui, Request.Type.GET,
-      createResourceInstance(m_clusterName, id));
+      createResourceInstance(clusterName, id));
   }
   
   

http://git-wip-us.apache.org/repos/asf/ambari/blob/49d7a3da/ambari-server/src/main/java/org/apache/ambari/server/api/services/AmbariMetaInfo.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/api/services/AmbariMetaInfo.java b/ambari-server/src/main/java/org/apache/ambari/server/api/services/AmbariMetaInfo.java
index c8ccf9a..3937633 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/api/services/AmbariMetaInfo.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/api/services/AmbariMetaInfo.java
@@ -60,6 +60,9 @@ import org.apache.ambari.server.state.Stack;
 import org.apache.ambari.server.state.StackId;
 import org.apache.ambari.server.state.StackInfo;
 import org.apache.ambari.server.state.alert.AlertDefinition;
+import org.apache.ambari.server.state.alert.MetricSource;
+import org.apache.ambari.server.state.alert.Source;
+import org.apache.ambari.server.state.alert.SourceType;
 import org.apache.ambari.server.state.stack.LatestRepoCallable;
 import org.apache.ambari.server.state.stack.MetricDefinition;
 import org.apache.ambari.server.state.stack.RepositoryXml;
@@ -69,6 +72,12 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonDeserializationContext;
+import com.google.gson.JsonDeserializer;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParseException;
 import com.google.gson.reflect.TypeToken;
 import com.google.inject.Inject;
 import com.google.inject.Injector;
@@ -1053,6 +1062,13 @@ public class AmbariMetaInfo {
     return requiredProperties;
   }
   
+  /**
+   * @param stackName the stack name
+   * @param stackVersion the stack version
+   * @param serviceName the service name
+   * @return the alert definitions for a stack
+   * @throws AmbariException
+   */
   public Set<AlertDefinition> getAlertDefinitions(String stackName, String stackVersion,
       String serviceName) throws AmbariException {
     
@@ -1063,16 +1079,38 @@ public class AmbariMetaInfo {
       return null;
     }
     
-    
     Map<String, List<AlertDefinition>> map = null;
+
+    GsonBuilder builder = new GsonBuilder().registerTypeAdapter(Source.class,
+        new JsonDeserializer<Source>() {
+          @Override
+          public Source deserialize(JsonElement json, Type typeOfT,
+              JsonDeserializationContext context) throws JsonParseException {
+            JsonObject jsonObj = (JsonObject) json;
+
+            SourceType type = SourceType.valueOf(jsonObj.get("type").getAsString());
+            Class<? extends Source> cls = null;
+            
+            switch (type) {
+              case METRIC:
+                cls = MetricSource.class;
+                break;
+              default:
+                break;
+            }
+
+            if (null != cls)
+              return context.deserialize(json, cls);
+            else
+              return null;
+          }
+        });
     
-    Type type = new TypeToken<Map<String, List<AlertDefinition>>>(){}.getType();
-    
-    Gson gson = new Gson();
+    Gson gson = builder.create();
 
     try {
+      Type type = new TypeToken<Map<String, List<AlertDefinition>>>(){}.getType();
       map = gson.fromJson(new FileReader(svc.getAlertsFile()), type);
-
     } catch (Exception e) {
       LOG.error ("Could not read the alert definition file", e);
       throw new AmbariException("Could not read alert definition file", e);
@@ -1092,5 +1130,5 @@ public class AmbariMetaInfo {
     
     return defs;
   }
-
+  
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/49d7a3da/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 c83615a..966d8b5 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
@@ -17,11 +17,15 @@
  */
 package org.apache.ambari.server.controller.internal;
 
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
+import java.util.UUID;
 
 import org.apache.ambari.server.AmbariException;
 import org.apache.ambari.server.controller.AmbariManagementController;
@@ -34,12 +38,14 @@ import org.apache.ambari.server.controller.spi.Resource;
 import org.apache.ambari.server.controller.spi.ResourceAlreadyExistsException;
 import org.apache.ambari.server.controller.spi.SystemException;
 import org.apache.ambari.server.controller.spi.UnsupportedPropertyException;
+import org.apache.ambari.server.controller.utilities.PropertyHelper;
 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.alert.MetricAlert;
+import org.apache.ambari.server.state.alert.SourceType;
 
 import com.google.gson.Gson;
+import com.google.gson.JsonObject;
 import com.google.inject.Inject;
 
 /**
@@ -51,7 +57,8 @@ public class AlertDefinitionResourceProvider extends AbstractControllerResourceP
   protected static final String ALERT_DEF_ID = "AlertDefinition/id";
   protected static final String ALERT_DEF_NAME = "AlertDefinition/name";
   protected static final String ALERT_DEF_INTERVAL = "AlertDefinition/interval";
-  protected static final String ALERT_DEF_SOURCE_TYPE = "AlertDefinition/source";
+  protected static final String ALERT_DEF_SOURCE_TYPE = "AlertDefinition/source/type";
+  protected static final String ALERT_DEF_SOURCE = "AlertDefinition/source";
   protected static final String ALERT_DEF_SERVICE_NAME = "AlertDefinition/service_name";
   protected static final String ALERT_DEF_COMPONENT_NAME = "AlertDefinition/component_name";
   protected static final String ALERT_DEF_ENABLED = "AlertDefinition/enabled";
@@ -61,6 +68,8 @@ public class AlertDefinitionResourceProvider extends AbstractControllerResourceP
       Arrays.asList(ALERT_DEF_ID, ALERT_DEF_NAME));
   private static AlertDefinitionDAO alertDefinitionDAO = null;
   
+  private static Gson gson = new Gson();
+  
   /**
    * @param instance
    */
@@ -81,11 +90,92 @@ public class AlertDefinitionResourceProvider extends AbstractControllerResourceP
   }
 
   @Override
-  public RequestStatus createResources(Request request) throws SystemException,
+  public RequestStatus createResources(final Request request) throws SystemException,
       UnsupportedPropertyException, ResourceAlreadyExistsException,
       NoSuchParentResourceException {
-    // TODO Auto-generated method stub
-    return null;
+
+    createResources(new Command<Void>() {
+      @Override
+      public Void invoke() throws AmbariException {
+        createAlertDefinitions(request.getProperties());
+        return null;
+      }
+    });
+    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)
+      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())
+      throw new IllegalArgumentException("Invalid argument, cluster name is required");
+    
+    if (!requestMap.containsKey(ALERT_DEF_INTERVAL))
+      throw new IllegalArgumentException("Check interval must be specified");
+    Long interval = Long.valueOf((String) requestMap.get(ALERT_DEF_INTERVAL));
+    
+    if (!requestMap.containsKey(ALERT_DEF_NAME))
+      throw new IllegalArgumentException("Definition name must be specified");
+    
+    if (!requestMap.containsKey(ALERT_DEF_SERVICE_NAME))
+      throw new IllegalArgumentException("Service name must be specified");
+    
+    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());
+
+      if (propCat.equals(ALERT_DEF_SOURCE)) {
+        jsonObj.addProperty(propName, entry.getValue().toString());
+      }
+    }
+
+    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));
+    entity.setDefinitionName((String) requestMap.get(ALERT_DEF_NAME));
+
+    boolean b = requestMap.containsKey(ALERT_DEF_ENABLED) ?
+        Boolean.parseBoolean((String)requestMap.get(ALERT_DEF_ENABLED)) : true;
+    entity.setEnabled(b);
+    
+    entity.setHash(UUID.randomUUID().toString());
+    entity.setScheduleInterval(interval);
+    entity.setScope((String) requestMap.get(ALERT_DEF_SCOPE));
+    entity.setServiceName((String) requestMap.get(ALERT_DEF_SERVICE_NAME));
+    entity.setSourceType((String) requestMap.get(ALERT_DEF_SOURCE_TYPE));
+    entity.setSource(jsonObj.toString());
+    
+    return entity;
   }
 
   @Override
@@ -134,16 +224,90 @@ public class AlertDefinitionResourceProvider extends AbstractControllerResourceP
   public RequestStatus updateResources(Request request, Predicate predicate)
       throws SystemException, UnsupportedPropertyException,
       NoSuchResourceException, NoSuchParentResourceException {
-    throw new UnsupportedOperationException("Not currently supported.");
+
+    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)
+          continue;
+
+        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(Long.valueOf(
+              (String) propertyMap.get(ALERT_DEF_INTERVAL)));
+        }
+        
+        if (propertyMap.containsKey(ALERT_DEF_SCOPE))
+          entity.setScope((String) propertyMap.get(ALERT_DEF_SCOPE));
+        
+
+        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());
+
+          if (propCat.equals(ALERT_DEF_SOURCE)) {
+            jsonObj.addProperty(propName, entry.getValue().toString());
+          }
+        }
+        
+        entity.setHash(UUID.randomUUID().toString());
+        
+        alertDefinitionDAO.merge(entity);
+      }
+    }
+    
+    notifyUpdate(Resource.Type.AlertDefinition, request, predicate);
+
+    return getRequestStatus(null);    
   }
 
   @Override
   public RequestStatus deleteResources(Predicate predicate)
       throws SystemException, UnsupportedPropertyException,
       NoSuchResourceException, NoSuchParentResourceException {
-    throw new UnsupportedOperationException("Not currently supported.");
-  }
 
+    Set<Resource> resources = getResources(
+        new RequestImpl(null, null, null, null), predicate);
+    
+    Set<Long> definitionIds = new HashSet<Long>();
+
+    for (final Resource resource : resources) {
+      definitionIds.add((Long) resource.getPropertyValue(ALERT_DEF_ID));
+    }
+
+    for (Long definitionId : definitionIds) {
+
+      LOG.info("Deleting alert definition {}", definitionId);
+      
+      final AlertDefinitionEntity ad = alertDefinitionDAO.findById(definitionId.longValue());
+
+      modifyResources(new Command<Void>() {
+        @Override
+        public Void invoke() throws AmbariException {
+          alertDefinitionDAO.remove(ad);
+          return null;
+        }
+      });
+    }
+
+    notifyDelete(Resource.Type.AlertDefinition, predicate);
+    return getRequestStatus(null);
+
+  }
   
   private Resource toResource(boolean isCollection, String clusterName,
       AlertDefinitionEntity entity, Set<String> requestedIds) {
@@ -153,22 +317,23 @@ public class AlertDefinitionResourceProvider extends AbstractControllerResourceP
     setResourceProperty(resource, ALERT_DEF_ID, entity.getDefinitionId(), requestedIds);
     setResourceProperty(resource, ALERT_DEF_NAME, entity.getDefinitionName(), requestedIds);
     setResourceProperty(resource, ALERT_DEF_INTERVAL, entity.getScheduleInterval(), requestedIds);
-    setResourceProperty(resource, ALERT_DEF_SOURCE_TYPE, entity.getSourceType(), requestedIds);
     setResourceProperty(resource, ALERT_DEF_SERVICE_NAME, entity.getServiceName(), requestedIds);
     setResourceProperty(resource, ALERT_DEF_COMPONENT_NAME, entity.getComponentName(), requestedIds);
     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);
     
     if (!isCollection && null != resource.getPropertyValue(ALERT_DEF_SOURCE_TYPE)) {
-      Gson gson = new Gson();
       
-      if (entity.getSourceType().equals("metric")) {
-        try {
-          MetricAlert ma = gson.fromJson(entity.getSource(), MetricAlert.class);
-          resource.setProperty("AlertDefinition/metric", ma);
-        } catch (Exception e) {
-          LOG.error("Could not coerce alert source into a 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());  
         }
+      } catch (Exception e) {
+        LOG.error("Could not coerce alert JSON into a type");
       }
     }
     

http://git-wip-us.apache.org/repos/asf/ambari/blob/49d7a3da/ambari-server/src/main/java/org/apache/ambari/server/state/alert/AlertDefinition.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/alert/AlertDefinition.java b/ambari-server/src/main/java/org/apache/ambari/server/state/alert/AlertDefinition.java
index f8df108..e6f9b39 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/alert/AlertDefinition.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/alert/AlertDefinition.java
@@ -17,11 +17,9 @@
  */
 package org.apache.ambari.server.state.alert;
 
-import com.google.gson.annotations.SerializedName;
 
 /**
- * @author ncole
- *
+ * Represents information required to run and collection of alerts.
  */
 public class AlertDefinition {
 
@@ -31,25 +29,35 @@ public class AlertDefinition {
   private String name = null;
   private String label = null;
   private String scope = null;
-  private String source = null;
   private int interval = 1;
   private boolean enabled = true;
-  
-  @SerializedName("metric")
-  private MetricAlert metricAlert = null;
-  
+  private Source source = null;
+
+  /**
+   * @return the service name
+   */
   public String getServiceName() {
     return serviceName;
   }
   
+  /**
+   * @param name the service name
+   */
   public void setServiceName(String name) {
     serviceName = name;
   }
-  
+
+  /**
+   * @return the component name
+   */
   public String getComponentName() {
     return componentName;
   }
   
+  /**
+   * 
+   * @param name the component name
+   */
   public void setComponentName(String name) {
     componentName = name;
   }
@@ -76,26 +84,23 @@ public class AlertDefinition {
   }
   
   /**
-   * @return the source
-   */
-  public String getSource() {
-    return source;
-  }
-  
-  /**
-  * @return the interval
+   * @return the interval
    */
   public int getInterval() {
     return interval;
   }
   
   /**
-   * @return <code>true</code> if enabled
+   * @return {@code true} if enabled
    */
   public boolean isEnabled() {
     return enabled;
   }
   
+  public Source getSource() {
+    return source;
+  }
+  
   @Override
   public boolean equals(Object obj) {
     if (null == obj || !obj.getClass().equals(AlertDefinition.class))

http://git-wip-us.apache.org/repos/asf/ambari/blob/49d7a3da/ambari-server/src/main/java/org/apache/ambari/server/state/alert/MetricAlert.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/alert/MetricAlert.java b/ambari-server/src/main/java/org/apache/ambari/server/state/alert/MetricAlert.java
deleted file mode 100644
index b565a3f..0000000
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/alert/MetricAlert.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/**
- * 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 com.google.gson.annotations.SerializedName;
-
-/**
- * Alert when the source type is defined as {@link SourceType#METRIC}
- */
-public class MetricAlert {
-  
-  private String host = null;
-  
-  @SerializedName("jmx")
-  private String jmxInfo = null;
-  
-  @SerializedName("ganglia")
-  private String gangliaInfo = null;
-  
-  /**
-   * @return the jmx info, if this metric is jmx-based
-   */
-  public String getJmxInfo() {
-    return jmxInfo;
-  }
-  
-  /**
-   * @return the ganglia info, if this metric is ganglia-based
-   */
-  public String getGangliaInfo() {
-    return gangliaInfo;
-  }
-  
-  /**
-   * @return the host info, which may include port information
-   */
-  public String getHost() {
-    return host;
-  }
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/49d7a3da/ambari-server/src/main/java/org/apache/ambari/server/state/alert/MetricSource.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/alert/MetricSource.java b/ambari-server/src/main/java/org/apache/ambari/server/state/alert/MetricSource.java
new file mode 100644
index 0000000..9ad4a13
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/alert/MetricSource.java
@@ -0,0 +1,57 @@
+/**
+ * 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 com.google.gson.annotations.SerializedName;
+
+/**
+ * Alert when the source type is defined as {@link SourceType#METRIC}
+ */
+public class MetricSource extends Source {
+  
+  private String host = null;
+  
+  @SerializedName("jmx")
+  private String jmxInfo = null;
+  
+  @SerializedName("ganglia")
+  private String gangliaInfo = null;
+  
+  /**
+   * @return the jmx info, if this metric is jmx-based
+   */
+  public String getJmxInfo() {
+    return jmxInfo;
+  }
+  
+  /**
+   * @return the ganglia info, if this metric is ganglia-based
+   */
+  public String getGangliaInfo() {
+    return gangliaInfo;
+  }
+  
+  /**
+   * @return the host info, which may include port information
+   */
+  public String getHost() {
+    return host;
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/49d7a3da/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
new file mode 100644
index 0000000..f64b7d2
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/alert/Source.java
@@ -0,0 +1,34 @@
+/**
+ * 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;
+
+/**
+ * Abstract class that all known alert sources should extend.
+ */
+public abstract class Source {
+
+  private SourceType type;
+
+  /**
+   * @return the type
+   */
+  public SourceType getType() {
+    return type;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/49d7a3da/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 4d5407c..825be7d 100644
--- a/ambari-server/src/main/resources/properties.json
+++ b/ambari-server/src/main/resources/properties.json
@@ -363,8 +363,8 @@
       "AlertDefinition/id",
       "AlertDefinition/name",
       "AlertDefinition/interval",
-      "AlertDefinition/source",
       "AlertDefinition/enabled",
-      "AlertDefinition/scope"      
+      "AlertDefinition/scope",
+      "AlertDefinition/source"
     ]
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/49d7a3da/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/alerts.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/alerts.json b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/alerts.json
index c1918e5..6a06d4d 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/alerts.json
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/alerts.json
@@ -9,8 +9,8 @@
       "label": "Secondary NameNode process",
       "interval": 1,
       "scope": "service",
-      "source": "port",
-      "port": {
+      "source": {
+        "type": "PORT",
         "config": "{{hdfs-site/dfs.namenode.secondary.http-address}}:50071",
         "default": 50071
       }
@@ -22,10 +22,9 @@
       "name": "namenode_cpu",
       "label": "NameNode host CPU Utilization",
       "scope": "host",
-      "source": "metric",
-      "metric": {
-        "jmx_object": "java.lang:type=OperatingSystem",
-        "jmx_attribute": "SystemCpuLoad",
+      "source": {
+        "type": "METRIC",
+        "jmx": "java.lang:type=OperatingSystem/SystemCpuLoad",
         "host": "{{hdfs-site/dfs.namenode.secondary.http-address}}"
       }
     },
@@ -35,8 +34,8 @@
       "label": "NameNode process",
       "interval": 1,
       "scope": "host",
-      "source": "port",
-      "port": {
+      "source": {
+        "type": "PORT",
         "uri": "{{hdfs-site/dfs.namenode.http-address}}:50070"
        }
     },
@@ -45,9 +44,9 @@
       "label": "Last Checkpoint Time",
       "interval": 1,
       "scope": "service",
-      "source": "script",
       "enabled": false
-      "script": {
+      "source": {
+        "type": "SCRIPT",
         "path": "scripts/alerts/last_checkpoint.py"
       }
     }

http://git-wip-us.apache.org/repos/asf/ambari/blob/49d7a3da/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 de48756..b7bcab6 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
@@ -17,10 +17,22 @@
  */
 package org.apache.ambari.server.controller.internal;
 
+import static org.easymock.EasyMock.anyObject;
+import static org.easymock.EasyMock.capture;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.createStrictMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.expectLastCall;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.resetToStrict;
+import static org.easymock.EasyMock.verify;
 import static org.junit.Assert.assertEquals;
 
 import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.ambari.server.controller.AmbariManagementController;
@@ -33,8 +45,7 @@ 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.alert.MetricAlert;
-import org.easymock.EasyMock;
+import org.easymock.Capture;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -48,7 +59,7 @@ public class AlertDefinitionResourceProviderTest {
   
   @Before
   public void before() {
-    dao = EasyMock.createStrictMock(AlertDefinitionDAO.class);
+    dao = createStrictMock(AlertDefinitionDAO.class);
     
     AlertDefinitionResourceProvider.init(dao);
   }
@@ -60,10 +71,6 @@ public class AlertDefinitionResourceProviderTest {
     Request request = PropertyHelper.getReadRequest("AlertDefinition/cluster_name",
         "AlertDefinition/id");
     
-    EasyMock.expect(dao.findAll()).andReturn(getMockEntities());
-
-    EasyMock.replay(dao);
-    
     Set<Resource> results = provider.getResources(request, null);
     
     assertEquals(0, results.size());
@@ -76,19 +83,19 @@ public class AlertDefinitionResourceProviderTest {
         AlertDefinitionResourceProvider.ALERT_DEF_ID,
         AlertDefinitionResourceProvider.ALERT_DEF_NAME);
     
-    AmbariManagementController amc = EasyMock.createMock(AmbariManagementController.class);
-    Clusters clusters = EasyMock.createMock(Clusters.class);
-    Cluster cluster = EasyMock.createMock(Cluster.class);
-    EasyMock.expect(amc.getClusters()).andReturn(clusters).atLeastOnce();
-    EasyMock.expect(clusters.getCluster(EasyMock.<String>anyObject())).andReturn(cluster).atLeastOnce();
-    EasyMock.expect(cluster.getClusterId()).andReturn(Long.valueOf(1)).anyTimes();
+    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();    
     
-    EasyMock.expect(dao.findAll(1L)).andReturn(getMockEntities());
+    expect(dao.findAll(1L)).andReturn(getMockEntities());
 
-    EasyMock.replay(amc, clusters, cluster, dao);
+    replay(amc, clusters, cluster, dao);
     
     AlertDefinitionResourceProvider provider = createProvider(amc);    
     Set<Resource> results = provider.getResources(request, predicate);
@@ -98,6 +105,8 @@ public class AlertDefinitionResourceProviderTest {
     Resource r = results.iterator().next();
     
     Assert.assertEquals("my_def", r.getPropertyValue(AlertDefinitionResourceProvider.ALERT_DEF_NAME));
+    
+    verify(amc, clusters, cluster, dao);
   }
   
   @Test
@@ -108,20 +117,20 @@ public class AlertDefinitionResourceProviderTest {
         AlertDefinitionResourceProvider.ALERT_DEF_NAME,
         AlertDefinitionResourceProvider.ALERT_DEF_SOURCE_TYPE);
     
-    AmbariManagementController amc = EasyMock.createMock(AmbariManagementController.class);
-    Clusters clusters = EasyMock.createMock(Clusters.class);
-    Cluster cluster = EasyMock.createMock(Cluster.class);
-    EasyMock.expect(amc.getClusters()).andReturn(clusters).atLeastOnce();
-    EasyMock.expect(clusters.getCluster(EasyMock.<String>anyObject())).andReturn(cluster).atLeastOnce();
-    EasyMock.expect(cluster.getClusterId()).andReturn(Long.valueOf(1)).anyTimes();
+    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();    
     
-    EasyMock.expect(dao.findById(1L)).andReturn(getMockEntities().get(0));
+    expect(dao.findById(1L)).andReturn(getMockEntities().get(0));
 
-    EasyMock.replay(amc, clusters, cluster, dao);
+    replay(amc, clusters, cluster, dao);
     
     AlertDefinitionResourceProvider provider = createProvider(amc);    
     Set<Resource> results = provider.getResources(request, predicate);
@@ -132,8 +141,169 @@ public class AlertDefinitionResourceProviderTest {
     
     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/metric"));
-    Assert.assertEquals(MetricAlert.class, r.getPropertyValue("AlertDefinition/metric").getClass());
+    Assert.assertNotNull(r.getPropertyValue("AlertDefinition/source/type"));
+  }
+
+  @Test
+  public void testCreateResources() throws Exception {
+    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();
+
+    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(); 
+    Assert.assertNotNull(entity);
+
+    Assert.assertEquals(Long.valueOf(1), entity.getClusterId());
+    Assert.assertNull(entity.getComponentName());
+    Assert.assertEquals("my_def", entity.getDefinitionName());
+    Assert.assertTrue(entity.getEnabled());
+    Assert.assertNotNull(entity.getHash());
+    Assert.assertEquals(Long.valueOf(1), entity.getScheduleInterval());
+    Assert.assertNull(entity.getScope());
+    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);
+    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();
+
+    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(); 
+    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");
+    requestProps.put(AlertDefinitionResourceProvider.ALERT_DEF_NAME, "my_def1");
+    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);
+    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();
+
+    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(); 
+    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) {

http://git-wip-us.apache.org/repos/asf/ambari/blob/49d7a3da/ambari-server/src/test/resources/stacks/HDP/2.0.5/services/HDFS/alerts.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/resources/stacks/HDP/2.0.5/services/HDFS/alerts.json b/ambari-server/src/test/resources/stacks/HDP/2.0.5/services/HDFS/alerts.json
index 7988d8b..85aa3ab 100644
--- a/ambari-server/src/test/resources/stacks/HDP/2.0.5/services/HDFS/alerts.json
+++ b/ambari-server/src/test/resources/stacks/HDP/2.0.5/services/HDFS/alerts.json
@@ -7,10 +7,9 @@
       "label": "Secondary NameNode process",
       "interval": 1,
       "scope": "service",
-      "source": "port",
-      "port": {
-        "config": "{{hdfs-site/dfs.namenode.secondary.http-address}}:50071",
-        "default": 50071
+      "source": {
+        "type": "PORT",
+        "config": "{{hdfs-site/dfs.namenode.secondary.http-address}}:50071"
       }
     }
   ],
@@ -19,8 +18,8 @@
       "name": "namenode_cpu",
       "label": "NameNode host CPU Utilization",
       "scope": "host",
-      "source": "metric",
-      "metric": {
+      "source": {
+        "type": "METRIC",
         "jmx": "java.lang:type=OperatingSystem/SystemCpuLoad",
         "host": "{{hdfs-site/dfs.namenode.secondary.http-address}}"
       }
@@ -30,8 +29,8 @@
       "label": "NameNode process",
       "interval": 1,
       "scope": "host",
-      "source": "port",
-      "port": {
+      "source": {
+        "type": "PORT",
         "uri": "{{hdfs-site/dfs.namenode.http-address}}:50070"
        }
     },
@@ -40,9 +39,9 @@
       "label": "Last Checkpoint Time",
       "interval": 1,
       "scope": "service",
-      "source": "script",
       "enabled": false,
-      "script": {
+      "source": {
+        "type": "SCRIPT",
         "path": "scripts/alerts/last_checkpoint.py"
       }
     }