You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by sr...@apache.org on 2014/07/10 02:12:45 UTC

[2/2] git commit: AMBARI-6329. Load 'final' attribute of service config-type files into StackServiceResponse

AMBARI-6329. Load 'final' attribute of service config-type files into StackServiceResponse


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

Branch: refs/heads/trunk
Commit: b08909a1f385b66d45cbf90785a41a5e8e5a3687
Parents: 0bc1444
Author: Srimanth Gunturi <sg...@hortonworks.com>
Authored: Tue Jul 8 17:30:54 2014 -0700
Committer: Srimanth Gunturi <sg...@hortonworks.com>
Committed: Wed Jul 9 17:10:53 2014 -0700

----------------------------------------------------------------------
 .../server/api/util/StackExtensionHelper.java   | 134 ++++++--
 .../controller/StackConfigurationResponse.java  |  17 +-
 .../server/controller/StackServiceResponse.java |   8 +-
 .../StackConfigurationResourceProvider.java     |   6 +
 .../ambari/server/state/PropertyInfo.java       |  13 +-
 .../apache/ambari/server/state/ServiceInfo.java |  14 +-
 .../server/state/stack/ConfigurationXml.java    |  12 +
 .../src/main/resources/properties.json          |   1 +
 .../GLUSTERFS/configuration/core-site.xml       |   2 +-
 .../GLUSTERFS/configuration/hadoop-policy.xml   |   2 +-
 .../GLUSTERFS/configuration/hdfs-site.xml       |   2 +-
 .../services/HDFS/configuration/core-site.xml   |   2 +-
 .../services/HDFS/configuration/global.xml      |   2 +-
 .../HDFS/configuration/hadoop-policy.xml        |   2 +-
 .../services/HDFS/configuration/hdfs-site.xml   |   2 +-
 .../services/GANGLIA/configuration/global.xml   |   2 +-
 .../services/HBASE/configuration/global.xml     |   2 +-
 .../HBASE/configuration/hbase-log4j.xml         |   2 +-
 .../HBASE/configuration/hbase-policy.xml        |   2 +-
 .../services/HBASE/configuration/hbase-site.xml |   2 +-
 .../services/HDFS/configuration/core-site.xml   |   2 +-
 .../services/HDFS/configuration/global.xml      |   2 +-
 .../HDFS/configuration/hadoop-policy.xml        |   2 +-
 .../services/HDFS/configuration/hdfs-log4j.xml  |   2 +-
 .../services/HDFS/configuration/hdfs-site.xml   |   2 +-
 .../services/HIVE/configuration/global.xml      |   2 +-
 .../HIVE/configuration/hive-exec-log4j.xml      |   2 +-
 .../services/HIVE/configuration/hive-log4j.xml  |   2 +-
 .../services/HIVE/configuration/hive-site.xml   |   2 +-
 .../1.3.2/services/HUE/configuration/global.xml |   2 +-
 .../services/HUE/configuration/hue-site.xml     |   2 +-
 .../configuration/capacity-scheduler.xml        |   2 +-
 .../MAPREDUCE/configuration/core-site.xml       |   2 +-
 .../services/MAPREDUCE/configuration/global.xml |   2 +-
 .../configuration/mapred-queue-acls.xml         |   2 +-
 .../MAPREDUCE/configuration/mapred-site.xml     |   2 +-
 .../MAPREDUCE/configuration/mapreduce-log4j.xml |   2 +-
 .../services/NAGIOS/configuration/global.xml    |   2 +-
 .../services/OOZIE/configuration/global.xml     |   2 +-
 .../OOZIE/configuration/oozie-log4j.xml         |   2 +-
 .../services/OOZIE/configuration/oozie-site.xml |   2 +-
 .../services/PIG/configuration/pig-log4j.xml    |   2 +-
 .../WEBHCAT/configuration/webhcat-site.xml      |   2 +-
 .../services/ZOOKEEPER/configuration/global.xml |   2 +-
 .../ZOOKEEPER/configuration/zookeeper-log4j.xml |   2 +-
 .../GLUSTERFS/configuration/core-site.xml       |   2 +-
 .../services/GLUSTERFS/configuration/global.xml |   2 +-
 .../services/HDFS/configuration/core-site.xml   |   2 +-
 .../services/HDFS/configuration/global.xml      |   2 +-
 .../HDFS/configuration/hadoop-policy.xml        |   2 +-
 .../services/HDFS/configuration/hdfs-site.xml   |   2 +-
 .../services/FLUME/configuration/flume-conf.xml |   2 +-
 .../FLUME/configuration/flume-log4j.xml         |   2 +-
 .../services/FLUME/configuration/global.xml     |   2 +-
 .../services/GANGLIA/configuration/global.xml   |   2 +-
 .../services/HBASE/configuration/global.xml     |   2 +-
 .../HBASE/configuration/hbase-log4j.xml         |   2 +-
 .../HBASE/configuration/hbase-policy.xml        |   2 +-
 .../services/HBASE/configuration/hbase-site.xml |   2 +-
 .../services/HDFS/configuration/core-site.xml   |   2 +-
 .../services/HDFS/configuration/global.xml      |   2 +-
 .../HDFS/configuration/hadoop-policy.xml        |   2 +-
 .../services/HDFS/configuration/hdfs-log4j.xml  |   2 +-
 .../services/HDFS/configuration/hdfs-site.xml   |   2 +-
 .../services/HIVE/configuration/global.xml      |   2 +-
 .../HIVE/configuration/hive-exec-log4j.xml      |   2 +-
 .../services/HIVE/configuration/hive-log4j.xml  |   2 +-
 .../services/HIVE/configuration/hive-site.xml   |   2 +-
 .../services/NAGIOS/configuration/global.xml    |   2 +-
 .../package/files/hdp_mon_nagios_addons.conf    |  17 +
 .../services/OOZIE/configuration/global.xml     |   2 +-
 .../OOZIE/configuration/oozie-log4j.xml         |   2 +-
 .../services/OOZIE/configuration/oozie-site.xml |   2 +-
 .../services/PIG/configuration/pig-log4j.xml    |   2 +-
 .../PIG/configuration/pig-properties.xml        |   2 +-
 .../WEBHCAT/configuration/webhcat-site.xml      |   2 +-
 .../YARN/configuration-mapred/core-site.xml     |   2 +-
 .../YARN/configuration-mapred/global.xml        |   2 +-
 .../configuration-mapred/mapred-queue-acls.xml  |   2 +-
 .../YARN/configuration-mapred/mapred-site.xml   |   2 +-
 .../YARN/configuration/capacity-scheduler.xml   |   2 +-
 .../services/YARN/configuration/core-site.xml   |   2 +-
 .../services/YARN/configuration/global.xml      |   2 +-
 .../services/YARN/configuration/yarn-log4j.xml  |   2 +-
 .../services/YARN/configuration/yarn-site.xml   |   2 +-
 .../services/ZOOKEEPER/configuration/global.xml |   2 +-
 .../ZOOKEEPER/configuration/zookeeper-log4j.xml |   2 +-
 .../configuration/falcon-runtime.properties.xml |   2 +-
 .../configuration/falcon-startup.properties.xml |   2 +-
 .../services/FALCON/configuration/global.xml    |   2 +-
 .../FALCON/configuration/oozie-site.xml         |   2 +-
 .../GLUSTERFS/configuration/core-site.xml       |   2 +-
 .../services/GLUSTERFS/configuration/global.xml |   2 +-
 .../services/HIVE/configuration/hive-site.xml   |   2 +-
 .../services/OOZIE/configuration/oozie-site.xml |   2 +-
 .../PIG/configuration/pig-properties.xml        |   2 +-
 .../services/STORM/configuration/global.xml     |   2 +-
 .../services/STORM/configuration/storm-site.xml |   2 +-
 .../services/TEZ/configuration/global.xml       |   2 +-
 .../services/TEZ/configuration/tez-site.xml     |   2 +-
 .../WEBHCAT/configuration/webhcat-site.xml      |   2 +-
 .../YARN/configuration-mapred/mapred-site.xml   |   2 +-
 .../YARN/configuration/capacity-scheduler.xml   |   2 +-
 .../services/YARN/configuration/core-site.xml   |   2 +-
 .../services/YARN/configuration/global.xml      |   2 +-
 .../services/YARN/configuration/yarn-site.xml   |   2 +-
 .../configuration/falcon-runtime.properties.xml |   2 +-
 .../configuration/falcon-startup.properties.xml |   2 +-
 .../services/FALCON/configuration/global.xml    |   2 +-
 .../FALCON/configuration/oozie-site.xml         |   2 +-
 .../services/HIVE/configuration/hive-site.xml   |   2 +-
 .../services/OOZIE/configuration/oozie-site.xml |   2 +-
 .../PIG/configuration/pig-properties.xml        |   2 +-
 .../2.1/services/STORM/configuration/global.xml |   2 +-
 .../services/STORM/configuration/storm-site.xml |   2 +-
 .../2.1/services/TEZ/configuration/global.xml   |   2 +-
 .../2.1/services/TEZ/configuration/tez-site.xml |   2 +-
 .../WEBHCAT/configuration/webhcat-site.xml      |   2 +-
 .../YARN/configuration/capacity-scheduler.xml   |   2 +-
 .../services/YARN/configuration/core-site.xml   |   2 +-
 .../2.1/services/YARN/configuration/global.xml  |   2 +-
 .../services/YARN/configuration/yarn-site.xml   |   2 +-
 .../api/util/StackExtensionHelperTest.java      | 319 ++++++++++++++++++-
 .../StackConfigurationResourceProviderTest.java |   7 +-
 .../YARN/configuration/capacity-scheduler.xml   |  42 +++
 .../services/YARN/configuration/yarn-site.xml   |  60 ++++
 .../services/HDFS/configuration/core-site.xml   |   2 +-
 .../WEBHCAT/configuration/webhcat-site.xml      |   2 +-
 .../HDP/2.0.6/services/WEBHCAT/metainfo.xml     |  27 ++
 .../services/HDFS/configuration/core-site.xml   |   2 +-
 .../services/HDFS/configuration/global.xml      |   2 +-
 131 files changed, 754 insertions(+), 157 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/java/org/apache/ambari/server/api/util/StackExtensionHelper.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/api/util/StackExtensionHelper.java b/ambari-server/src/main/java/org/apache/ambari/server/api/util/StackExtensionHelper.java
index da1a224..6929ea9 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/api/util/StackExtensionHelper.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/api/util/StackExtensionHelper.java
@@ -30,6 +30,7 @@ import java.util.Map;
 import javax.xml.bind.JAXBContext;
 import javax.xml.bind.JAXBException;
 import javax.xml.bind.Unmarshaller;
+import javax.xml.namespace.QName;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.ParserConfigurationException;
@@ -131,8 +132,11 @@ public class StackExtensionHelper {
     mergedServiceInfo.setComment(childService.getComment());
     mergedServiceInfo.setVersion(childService.getVersion());
     mergedServiceInfo.setConfigDependencies(
-      childService.getConfigDependencies() != null ?
-        childService.getConfigDependencies() : new ArrayList<String>());
+        childService.getConfigDependencies() != null ?
+            childService.getConfigDependencies() : parentService.getConfigDependencies());
+    mergedServiceInfo.setConfigTypes(
+        childService.getConfigTypes() != null ?
+            childService.getConfigTypes() : parentService.getConfigTypes());
     
     mergedServiceInfo.setRestartRequiredAfterChange(
             (childService.isRestartRequiredAfterChange() != null) 
@@ -396,7 +400,6 @@ public class StackExtensionHelper {
           // get metrics file, if it exists
           File metricsJson = new File(serviceFolder.getAbsolutePath()
             + File.separator + AmbariMetaInfo.SERVICE_METRIC_FILE_NAME);
-          String version = getSchemaVersion(metainfoFile);
 
           //Reading v2 service metainfo (may contain multiple services)
           // Get services from metadata
@@ -405,6 +408,7 @@ public class StackExtensionHelper {
           List<ServiceInfo> serviceInfos = smiv2x.getServices();
           for (ServiceInfo serviceInfo : serviceInfos) {
             serviceInfo.setSchemaVersion(AmbariMetaInfo.SCHEMA_VERSION_2);
+            populateConfigTypesFromDependencies(serviceInfo);
 
             // Find service package folder
             String servicePackageDir = resolveServicePackageFolder(
@@ -421,7 +425,7 @@ public class StackExtensionHelper {
 
             // Add now to be removed while iterating extension graph
             services.add(serviceInfo);
-            }
+          }
         }
       } catch (Exception e) {
         LOG.error("Error while parsing metainfo.xml for a service", e);
@@ -587,26 +591,73 @@ public class StackExtensionHelper {
     return stackInfo;
   }
 
+  private List<PropertyInfo> getProperties(ConfigurationXml configuration, String fileName) {
+    List<PropertyInfo> list = new ArrayList<PropertyInfo>();
+    for (PropertyInfo pi : configuration.getProperties()) {
+      pi.setFilename(fileName);
+      list.add(pi);
+    }
+    return list;
+  }
 
-  private List<PropertyInfo> getProperties(File propertyFile) {
-    
-    try {
-      ConfigurationXml cx = unmarshal(ConfigurationXml.class, propertyFile);
-
-      List<PropertyInfo> list = new ArrayList<PropertyInfo>();
-      
-      for (PropertyInfo pi : cx.getProperties()) {
-        pi.setFilename(propertyFile.getName());
-        list.add(pi);
+  /**
+   * Add properties and config type's properties from configuration file
+   */
+  void populateServiceProperties(File configFile, ServiceInfo serviceInfo) throws JAXBException {
+    ConfigurationXml configuration = unmarshal(ConfigurationXml.class, configFile);
+    String fileName = configFile.getName();
+    serviceInfo.getProperties().addAll(getProperties(configuration, fileName));
+    int extIndex = fileName.indexOf(AmbariMetaInfo.SERVICE_CONFIG_FILE_NAME_POSTFIX);
+    String configType = fileName.substring(0, extIndex);
+    for (Map.Entry<QName, String> attribute : configuration.getAttributes().entrySet()) {
+      for (Supports supportsProperty : Supports.values()) {
+        String attributeName = attribute.getKey().getLocalPart();
+        String attributeValue = attribute.getValue();
+        if (attributeName.equals(supportsProperty.getXmlAttributeName())) {
+          addConfigTypeProperty(serviceInfo, configType, Supports.KEYWORD,
+              supportsProperty.getPropertyName(), Boolean.valueOf(attributeValue).toString());
+        }
+      }
+    }
+  }
 
+  /**
+   * Populate ServiceInfo#configTypes with default entries based on ServiceInfo#configDependencies property
+   */
+  void populateConfigTypesFromDependencies(ServiceInfo serviceInfo) {
+    List<String> configDependencies = serviceInfo.getConfigDependencies();
+    if (configDependencies != null) {
+      Map<String, Map<String, Map<String, String>>> configTypes = new HashMap<String, Map<String, Map<String, String>>>();
+      for (String configDependency : configDependencies) {
+        if (!configTypes.containsKey(configDependency)) {
+          Map<String, Map<String, String>> properties = new HashMap<String, Map<String, String>>();
+          Map<String, String> supportsProperties = new HashMap<String, String>();
+          for (Supports supportsProperty : Supports.values()) {
+            supportsProperties.put(supportsProperty.getPropertyName(), supportsProperty.getDefaultValue());
+          }
+          properties.put(Supports.KEYWORD, supportsProperties);
+          configTypes.put(configDependency, properties);
+        }
       }
-      return list;
-    } catch (Exception e) {
-      LOG.error("Could not load configuration for " + propertyFile, e);
-      return null;
+      serviceInfo.setConfigTypes(configTypes);
     }
   }
 
+  /**
+   * Put new property entry to ServiceInfo#configTypes collection for specified configType
+   */
+  void addConfigTypeProperty(ServiceInfo serviceInfo, String configType,
+                             String propertiesGroupName, String key, String value) {
+    Map<String, Map<String, Map<String, String>>> configTypes = serviceInfo.getConfigTypes();
+    if (configTypes.containsKey(configType)) {
+      Map<String, Map<String, String>> configDependencyProperties = configTypes.get(configType);
+      if (!configDependencyProperties.containsKey(propertiesGroupName)) {
+        configDependencyProperties.put(propertiesGroupName, new HashMap<String, String>());
+      }
+      Map<String, String> propertiesGroup = configDependencyProperties.get(propertiesGroupName);
+      propertiesGroup.put(key, value);
+    }
+  }
 
   /**
    * Get all properties from all "configs/*-site.xml" files
@@ -621,9 +672,13 @@ public class StackExtensionHelper {
     
     File[] configFiles = serviceConfigFolder.listFiles(AmbariMetaInfo.FILENAME_FILTER);
     if (configFiles != null) {
-      for (File config : configFiles) {
-        if (config.getName().endsWith(AmbariMetaInfo.SERVICE_CONFIG_FILE_NAME_POSTFIX)) {
-          serviceInfo.getProperties().addAll(getProperties(config));
+      for (File configFile : configFiles) {
+        if (configFile.getName().endsWith(AmbariMetaInfo.SERVICE_CONFIG_FILE_NAME_POSTFIX)) {
+          try {
+            populateServiceProperties(configFile, serviceInfo);
+          } catch (Exception e) {
+            LOG.error("Could not load configuration for " + configFile, e);
+          }
         }
       }
     }
@@ -635,4 +690,41 @@ public class StackExtensionHelper {
     return clz.cast(u.unmarshal(file));
   }  
   
+  /**
+   * Service configuration-types can support different abilities. This
+   * enumerates the various abilities that configuration-types can support.
+   * 
+   * For example, Hadoop configuration types like 'core-site' and 'hdfs-site'
+   * can support the ability to define certain configs as 'final'.
+   */
+  protected enum Supports {
+
+    FINAL("supports_final");
+
+    public static final String KEYWORD = "supports";
+
+    private String defaultValue;
+    private String xmlAttributeName;
+
+    private Supports(String xmlAttributeName) {
+      this(xmlAttributeName, Boolean.FALSE.toString());
+    }
+
+    private Supports(String xmlAttributeName, String defaultValue) {
+      this.defaultValue = defaultValue;
+      this.xmlAttributeName = xmlAttributeName;
+    }
+
+    public String getDefaultValue() {
+      return defaultValue;
+    }
+
+    public String getXmlAttributeName() {
+      return xmlAttributeName;
+    }
+
+    public String getPropertyName() {
+      return name().toLowerCase();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/java/org/apache/ambari/server/controller/StackConfigurationResponse.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/StackConfigurationResponse.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/StackConfigurationResponse.java
index c4baefb..44454ec 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/StackConfigurationResponse.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/StackConfigurationResponse.java
@@ -27,12 +27,14 @@ public class StackConfigurationResponse {
    * @param propertyValue Property Value
    * @param propertyDescription Property Description
    * @param type Configuration type
+   * @param isFinal Is property final
    */
-  public StackConfigurationResponse(String propertyName, String propertyValue, String propertyDescription, String type) {
+  public StackConfigurationResponse(String propertyName, String propertyValue, String propertyDescription, String type, Boolean isFinal) {
     setPropertyName(propertyName);
     setPropertyValue(propertyValue);
     setPropertyDescription(propertyDescription);
     setType(type);
+    setFinal(isFinal);
   }
 
   /**
@@ -43,16 +45,18 @@ public class StackConfigurationResponse {
    * @param type Configuration type
    * @param isRequired Is required to be set
    * @param propertyType Property Type
+   * @param isFinal Is property final
    */
   public StackConfigurationResponse(String propertyName, String propertyValue,
                                     String propertyDescription, String type,
-                                    Boolean isRequired, String propertyType) {
+                                    Boolean isRequired, String propertyType, Boolean isFinal) {
     setPropertyName(propertyName);
     setPropertyValue(propertyValue);
     setPropertyDescription(propertyDescription);
     setType(type);
     setRequired(isRequired);
     setPropertyType(propertyType);
+    setFinal(isFinal);
   }
 
   private String stackName;
@@ -62,6 +66,7 @@ public class StackConfigurationResponse {
   private String propertyValue;
   private String propertyDescription;
   private String type;
+  private Boolean isFinal;
   private Boolean isRequired;
   private String propertyType;
 
@@ -125,6 +130,14 @@ public class StackConfigurationResponse {
     this.type = type;
   }
 
+  public Boolean isFinal() {
+    return isFinal;
+  }
+
+  public void setFinal(Boolean isFinal) {
+    this.isFinal = isFinal;
+  }
+
   /**
    * Is property a isRequired property
    * @return True/False

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/java/org/apache/ambari/server/controller/StackServiceResponse.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/StackServiceResponse.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/StackServiceResponse.java
index 3c90c4b..df91239 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/StackServiceResponse.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/StackServiceResponse.java
@@ -18,7 +18,7 @@
 
 package org.apache.ambari.server.controller;
 
-import java.util.List;
+import java.util.Map;
 
 public class StackServiceResponse {
 
@@ -32,10 +32,10 @@ public class StackServiceResponse {
   
   private String serviceVersion;
   
-  private List<String> configTypes;
+  private Map<String, Map<String, Map<String, String>>> configTypes;
 
   public StackServiceResponse(String serviceName, String userName, String comments, String serviceVersion,
-      List<String> types) {
+      Map<String, Map<String, Map<String, String>>> types) {
     setServiceName(serviceName);
     setUserName(userName);
     setComments(comments);
@@ -91,7 +91,7 @@ public class StackServiceResponse {
     this.serviceVersion = serviceVersion;
   }
   
-  public List<String> getConfigTypes() {
+  public Map<String, Map<String, Map<String, String>>> getConfigTypes() {
     return configTypes;
   }
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StackConfigurationResourceProvider.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StackConfigurationResourceProvider.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StackConfigurationResourceProvider.java
index 7ce98af..4aa56f0 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StackConfigurationResourceProvider.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StackConfigurationResourceProvider.java
@@ -63,6 +63,9 @@ public class StackConfigurationResourceProvider extends
   public static final String PROPERTY_TYPE_PROPERTY_ID = PropertyHelper
       .getPropertyId("StackConfigurations", "type");
 
+  public static final String PROPERTY_FINAL_PROPERTY_ID = PropertyHelper
+      .getPropertyId("StackConfigurations", "final");
+
 
   private static Set<String> pkPropertyIds = new HashSet<String>(
       Arrays.asList(new String[] { STACK_NAME_PROPERTY_ID,
@@ -125,6 +128,9 @@ public class StackConfigurationResourceProvider extends
       setResourceProperty(resource, PROPERTY_TYPE_PROPERTY_ID,
           response.getType(), requestedIds);
 
+      setResourceProperty(resource, PROPERTY_FINAL_PROPERTY_ID,
+          response.isFinal(), requestedIds);
+
       resources.add(resource);
     }
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/java/org/apache/ambari/server/state/PropertyInfo.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/PropertyInfo.java b/ambari-server/src/main/java/org/apache/ambari/server/state/PropertyInfo.java
index 6257157..8789383 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/PropertyInfo.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/PropertyInfo.java
@@ -22,6 +22,7 @@ package org.apache.ambari.server.state;
 import org.apache.ambari.server.controller.StackConfigurationResponse;
 
 import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlElement;
 
 public class PropertyInfo {
   private String name;
@@ -31,6 +32,7 @@ public class PropertyInfo {
   private boolean deleted;
   private boolean requireInput;
   private PropertyType type = PropertyType.DEFAULT;
+  private boolean isFinal;
 
   public String getName() {
     return name;
@@ -66,7 +68,7 @@ public class PropertyInfo {
   
   public StackConfigurationResponse convertToResponse() {
     return new StackConfigurationResponse(getName(), getValue(),
-      getDescription() , getFilename(), isRequireInput(), getType().name());
+      getDescription() , getFilename(), isRequireInput(), getType().name(), isFinal());
   }
 
   public boolean isDeleted() {
@@ -77,6 +79,15 @@ public class PropertyInfo {
     this.deleted = deleted;
   }
 
+  @XmlElement(name="final")
+  public boolean isFinal() {
+    return isFinal;
+  }
+
+  public void setFinal(boolean isFinal) {
+    this.isFinal = isFinal;
+  }
+
   @XmlAttribute(name = "require-input")
   public boolean isRequireInput() {
     return requireInput;

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/java/org/apache/ambari/server/state/ServiceInfo.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/ServiceInfo.java b/ambari-server/src/main/java/org/apache/ambari/server/state/ServiceInfo.java
index fab4867..44bc369 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/ServiceInfo.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/ServiceInfo.java
@@ -32,6 +32,7 @@ import javax.xml.bind.annotation.*;
 @XmlAccessorType(XmlAccessType.FIELD)
 @JsonFilter("propertiesfilter")
 public class ServiceInfo {
+
   /**
    * Format version. Added at schema ver 2
    */
@@ -58,6 +59,9 @@ public class ServiceInfo {
   @XmlElement(name="config-type")
   private List<String> configDependencies;
 
+  @XmlTransient
+  private Map<String, Map<String, Map<String, String>>> configTypes;
+
   @JsonIgnore
   private Boolean monitoringService;
   
@@ -212,12 +216,14 @@ public class ServiceInfo {
         getConfigTypes());
   }
   
-  public List<String> getConfigTypes() {
-    return configDependencies != null ? configDependencies :
-      Collections.unmodifiableList(new ArrayList<String>());
+  public Map<String, Map<String, Map<String, String>>> getConfigTypes() {
+    return configTypes;
+  }
+
+  public void setConfigTypes(Map<String, Map<String, Map<String, String>>> configTypes) {
+    this.configTypes = configTypes;
   }
 
-  
   /**
    * @param type the config type
    * @return <code>true</code> if the service defines the supplied type

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/java/org/apache/ambari/server/state/stack/ConfigurationXml.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/stack/ConfigurationXml.java b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/ConfigurationXml.java
index 8123716..50ab901 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/stack/ConfigurationXml.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/stack/ConfigurationXml.java
@@ -18,10 +18,15 @@
 package org.apache.ambari.server.state.stack;
 
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
+import javax.xml.bind.annotation.XmlAnyAttribute;
+import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.namespace.QName;
 
 import org.apache.ambari.server.state.PropertyInfo;
 
@@ -31,6 +36,9 @@ import org.apache.ambari.server.state.PropertyInfo;
 @XmlRootElement(name="configuration")
 public class ConfigurationXml {
   
+  @XmlAnyAttribute
+  private Map<QName, String> attributes = new HashMap<QName, String>();
+
   @XmlElement(name="property")
   private List<PropertyInfo> properties = new ArrayList<PropertyInfo>();
   
@@ -41,4 +49,8 @@ public class ConfigurationXml {
     return properties;
   }
 
+  public Map<QName, String> getAttributes() {
+    return attributes;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/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 eb364fb..a6cdac4 100644
--- a/ambari-server/src/main/resources/properties.json
+++ b/ambari-server/src/main/resources/properties.json
@@ -222,6 +222,7 @@
         "StackConfigurations/property_value",
         "StackConfigurations/property_description",
         "StackConfigurations/type",
+        "StackConfigurations/final",
         "_"
     ],
     "StackServiceComponent":[

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/GLUSTERFS/configuration/core-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/GLUSTERFS/configuration/core-site.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/GLUSTERFS/configuration/core-site.xml
index 9644ec8..b59a345 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/GLUSTERFS/configuration/core-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/GLUSTERFS/configuration/core-site.xml
@@ -20,7 +20,7 @@
  
 <!-- Put site-specific property overrides in this file. -->
 
-<configuration xmlns:xi="http://www.w3.org/2001/XInclude">
+<configuration supports_final="true" xmlns:xi="http://www.w3.org/2001/XInclude">
 
 <!-- i/o properties -->
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/GLUSTERFS/configuration/hadoop-policy.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/GLUSTERFS/configuration/hadoop-policy.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/GLUSTERFS/configuration/hadoop-policy.xml
index 900da99..14dbc8b 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/GLUSTERFS/configuration/hadoop-policy.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/GLUSTERFS/configuration/hadoop-policy.xml
@@ -20,7 +20,7 @@
 
 <!-- Put site-specific property overrides in this file. -->
 
-<configuration>
+<configuration supports_final="true">
   <property>
     <name>security.client.protocol.acl</name>
     <value>*</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/GLUSTERFS/configuration/hdfs-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/GLUSTERFS/configuration/hdfs-site.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/GLUSTERFS/configuration/hdfs-site.xml
index 3f9dae4..fc19139 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/GLUSTERFS/configuration/hdfs-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/GLUSTERFS/configuration/hdfs-site.xml
@@ -20,7 +20,7 @@
 
 <!-- Put site-specific property overrides in this file. -->
 
-<configuration>
+<configuration supports_final="true">
 
 <!-- file system properties -->
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/HDFS/configuration/core-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/HDFS/configuration/core-site.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/HDFS/configuration/core-site.xml
index fe0bcba..ae298d4 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/HDFS/configuration/core-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/HDFS/configuration/core-site.xml
@@ -20,7 +20,7 @@
  
 <!-- Put site-specific property overrides in this file. -->
 
-<configuration xmlns:xi="http://www.w3.org/2001/XInclude">
+<configuration supports_final="true" xmlns:xi="http://www.w3.org/2001/XInclude">
 
 <!-- i/o properties -->
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/HDFS/configuration/global.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/HDFS/configuration/global.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/HDFS/configuration/global.xml
index f10b9f9..d09b6d2 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/HDFS/configuration/global.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/HDFS/configuration/global.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
   <property>
     <name>namenode_host</name>
     <value></value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/HDFS/configuration/hadoop-policy.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/HDFS/configuration/hadoop-policy.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/HDFS/configuration/hadoop-policy.xml
index 900da99..14dbc8b 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/HDFS/configuration/hadoop-policy.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/HDFS/configuration/hadoop-policy.xml
@@ -20,7 +20,7 @@
 
 <!-- Put site-specific property overrides in this file. -->
 
-<configuration>
+<configuration supports_final="true">
   <property>
     <name>security.client.protocol.acl</name>
     <value>*</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/HDFS/configuration/hdfs-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/HDFS/configuration/hdfs-site.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/HDFS/configuration/hdfs-site.xml
index 640b748..c3b51f2 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/HDFS/configuration/hdfs-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2.GlusterFS/services/HDFS/configuration/hdfs-site.xml
@@ -20,7 +20,7 @@
 
 <!-- Put site-specific property overrides in this file. -->
 
-<configuration>
+<configuration supports_final="true">
 
 <!-- file system properties -->
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/GANGLIA/configuration/global.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/GANGLIA/configuration/global.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/GANGLIA/configuration/global.xml
index 68d94f2..d1f369a 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/GANGLIA/configuration/global.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/GANGLIA/configuration/global.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
   <property>
     <name>ganglia_conf_dir</name>
     <value>/etc/ganglia/hdp</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/configuration/global.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/configuration/global.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/configuration/global.xml
index 9d89400..2a00566 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/configuration/global.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/configuration/global.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
   <property>
     <name>hbase_log_dir</name>
     <value>/var/log/hbase</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/configuration/hbase-log4j.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/configuration/hbase-log4j.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/configuration/hbase-log4j.xml
index aa758be..5456029 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/configuration/hbase-log4j.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/configuration/hbase-log4j.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
 
   <property>
     <name>content</name>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/configuration/hbase-policy.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/configuration/hbase-policy.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/configuration/hbase-policy.xml
index e45f23c..2f12801 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/configuration/hbase-policy.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/configuration/hbase-policy.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="true">
   <property>
     <name>security.client.protocol.acl</name>
     <value>*</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/configuration/hbase-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/configuration/hbase-site.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/configuration/hbase-site.xml
index 11c5b95..a4ec9ba 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/configuration/hbase-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HBASE/configuration/hbase-site.xml
@@ -19,7 +19,7 @@
  * limitations under the License.
  */
 -->
-<configuration>
+<configuration supports_final="true">
   <property>
     <name>hbase.rootdir</name>
     <value>hdfs://localhost:8020/apps/hbase/data</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/core-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/core-site.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/core-site.xml
index 354d194..f9d6ebe 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/core-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/core-site.xml
@@ -20,7 +20,7 @@
  
 <!-- Put site-specific property overrides in this file. -->
 
-<configuration xmlns:xi="http://www.w3.org/2001/XInclude">
+<configuration supports_final="true" xmlns:xi="http://www.w3.org/2001/XInclude">
 
 <!-- i/o properties -->
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/global.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/global.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/global.xml
index 3cc6460..3eb0a22 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/global.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/global.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
   <property>
     <name>hdfs_log_dir_prefix</name>
     <value>/var/log/hadoop</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/hadoop-policy.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/hadoop-policy.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/hadoop-policy.xml
index f293f43..def654b 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/hadoop-policy.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/hadoop-policy.xml
@@ -20,7 +20,7 @@
 
 <!-- Put site-specific property overrides in this file. -->
 
-<configuration>
+<configuration supports_final="true">
   <property>
     <name>security.client.protocol.acl</name>
     <value>*</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/hdfs-log4j.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/hdfs-log4j.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/hdfs-log4j.xml
index 9e9f884..360b18b 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/hdfs-log4j.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/hdfs-log4j.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
 
   <property>
     <name>content</name>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/hdfs-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/hdfs-site.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/hdfs-site.xml
index 5943358..5082aa5 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/hdfs-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HDFS/configuration/hdfs-site.xml
@@ -20,7 +20,7 @@
 
 <!-- Put site-specific property overrides in this file. -->
 
-<configuration>
+<configuration supports_final="true">
 
   <!-- file system properties -->
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HIVE/configuration/global.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HIVE/configuration/global.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HIVE/configuration/global.xml
index 601edeb..90e7627 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HIVE/configuration/global.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HIVE/configuration/global.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
   <property>
     <name>hive_database_type</name>
     <value>mysql</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HIVE/configuration/hive-exec-log4j.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HIVE/configuration/hive-exec-log4j.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HIVE/configuration/hive-exec-log4j.xml
index 9a6d574..f32de10 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HIVE/configuration/hive-exec-log4j.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HIVE/configuration/hive-exec-log4j.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
 
   <property>
     <name>content</name>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HIVE/configuration/hive-log4j.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HIVE/configuration/hive-log4j.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HIVE/configuration/hive-log4j.xml
index 43c6e76..c9552b2 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HIVE/configuration/hive-log4j.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HIVE/configuration/hive-log4j.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
 
   <property>
     <name>content</name>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HIVE/configuration/hive-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HIVE/configuration/hive-site.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HIVE/configuration/hive-site.xml
index 6ed4c2a..fabcdf3 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HIVE/configuration/hive-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HIVE/configuration/hive-site.xml
@@ -17,7 +17,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-<configuration>
+<configuration supports_final="true">
 
   <property>
     <name>hive.heapsize</name>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HUE/configuration/global.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HUE/configuration/global.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HUE/configuration/global.xml
index c49480f..2028064 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HUE/configuration/global.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HUE/configuration/global.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
   <property>
     <name>hue_pid_dir</name>
     <value>/var/run/hue</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HUE/configuration/hue-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HUE/configuration/hue-site.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HUE/configuration/hue-site.xml
index 6eb52a2..3fb44d0 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HUE/configuration/hue-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/HUE/configuration/hue-site.xml
@@ -17,7 +17,7 @@
    limitations under the License.# limitations under the License.
 -->
 
-<configuration>
+<configuration supports_final="true">
   <!-- General Hue server configuration properties -->
   <property>
       <name>send_debug_messages</name>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/capacity-scheduler.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/capacity-scheduler.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/capacity-scheduler.xml
index 8034d19..010a313 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/capacity-scheduler.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/capacity-scheduler.xml
@@ -22,7 +22,7 @@
 <!-- The properties for a queue follow a naming convention,such as, -->
 <!-- mapred.capacity-scheduler.queue.<queue-name>.property-name. -->
 
-<configuration>
+<configuration supports_final="true">
 
   <property>
     <name>mapred.capacity-scheduler.maximum-system-jobs</name>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/core-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/core-site.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/core-site.xml
index 3a2af49..60f01ad 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/core-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/core-site.xml
@@ -16,5 +16,5 @@
    See the License for the specific language governing permissions and
    limitations under the License.
 -->
-<configuration>
+<configuration supports_final="true">
 </configuration>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/global.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/global.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/global.xml
index 00f3825..41bb735 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/global.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/global.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
   <property>
     <name>mapred_local_dir</name>
     <value>/hadoop/mapred</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/mapred-queue-acls.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/mapred-queue-acls.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/mapred-queue-acls.xml
index 9389ed0..f1f0ec5 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/mapred-queue-acls.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/mapred-queue-acls.xml
@@ -19,7 +19,7 @@
 -->
 
 <!-- mapred-queue-acls.xml -->
-<configuration>
+<configuration supports_final="true">
 
 
 <!-- queue default -->

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/mapred-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/mapred-site.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/mapred-site.xml
index 487cbd7..7ca3d80 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/mapred-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/mapred-site.xml
@@ -20,7 +20,7 @@
 
 <!-- Put site-specific property overrides in this file. -->
 
-<configuration xmlns:xi="http://www.w3.org/2001/XInclude">
+<configuration supports_final="true" xmlns:xi="http://www.w3.org/2001/XInclude">
 
   <!-- i/o properties -->
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/mapreduce-log4j.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/mapreduce-log4j.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/mapreduce-log4j.xml
index c76c088..54d6331 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/mapreduce-log4j.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/MAPREDUCE/configuration/mapreduce-log4j.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
 
   <property>
     <name>content</name>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/NAGIOS/configuration/global.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/NAGIOS/configuration/global.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/NAGIOS/configuration/global.xml
index 54c742c..f36020d 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/NAGIOS/configuration/global.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/NAGIOS/configuration/global.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
   <property>
     <name>nagios_user</name>
     <value>nagios</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/OOZIE/configuration/global.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/OOZIE/configuration/global.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/OOZIE/configuration/global.xml
index 6799fc5..1410bac 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/OOZIE/configuration/global.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/OOZIE/configuration/global.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
   <property>
     <name>oozie_user</name>
     <value>oozie</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/OOZIE/configuration/oozie-log4j.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/OOZIE/configuration/oozie-log4j.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/OOZIE/configuration/oozie-log4j.xml
index 147361a..cb77566 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/OOZIE/configuration/oozie-log4j.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/OOZIE/configuration/oozie-log4j.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
 
   <property>
     <name>content</name>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/OOZIE/configuration/oozie-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/OOZIE/configuration/oozie-site.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/OOZIE/configuration/oozie-site.xml
index 02e7a76..ae22d5d 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/OOZIE/configuration/oozie-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/OOZIE/configuration/oozie-site.xml
@@ -17,7 +17,7 @@
   limitations under the License.
 -->
 
-<configuration>
+<configuration supports_final="true">
 
   <!--
       Refer to the oozie-default.xml file for the complete list of

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/PIG/configuration/pig-log4j.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/PIG/configuration/pig-log4j.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/PIG/configuration/pig-log4j.xml
index f34724e..cbdd452 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/PIG/configuration/pig-log4j.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/PIG/configuration/pig-log4j.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
 
   <property>
     <name>content</name>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/WEBHCAT/configuration/webhcat-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/WEBHCAT/configuration/webhcat-site.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/WEBHCAT/configuration/webhcat-site.xml
index 653df05..e6b9f30 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/WEBHCAT/configuration/webhcat-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/WEBHCAT/configuration/webhcat-site.xml
@@ -21,7 +21,7 @@ limitations under the License.
 <!-- Edit templeton-site.xml to change settings for your local -->
 <!-- install. -->
 
-<configuration>
+<configuration supports_final="true">
 
   <property>
     <name>templeton.port</name>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/ZOOKEEPER/configuration/global.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/ZOOKEEPER/configuration/global.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/ZOOKEEPER/configuration/global.xml
index 840e991..d05c689 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/ZOOKEEPER/configuration/global.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/ZOOKEEPER/configuration/global.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
   <property>
     <name>zk_user</name>
     <value>zookeeper</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/ZOOKEEPER/configuration/zookeeper-log4j.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/ZOOKEEPER/configuration/zookeeper-log4j.xml b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/ZOOKEEPER/configuration/zookeeper-log4j.xml
index 687f8f9..4240099 100644
--- a/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/ZOOKEEPER/configuration/zookeeper-log4j.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/1.3.2/services/ZOOKEEPER/configuration/zookeeper-log4j.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
 
   <property>
     <name>content</name>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/GLUSTERFS/configuration/core-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/GLUSTERFS/configuration/core-site.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/GLUSTERFS/configuration/core-site.xml
index 4fd938c..afeada6 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/GLUSTERFS/configuration/core-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/GLUSTERFS/configuration/core-site.xml
@@ -20,7 +20,7 @@
  
 <!-- Put site-specific property overrides in this file. -->
 
-<configuration xmlns:xi="http://www.w3.org/2001/XInclude">
+<configuration supports_final="true" xmlns:xi="http://www.w3.org/2001/XInclude">
 
 <!-- i/o properties -->
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/GLUSTERFS/configuration/global.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/GLUSTERFS/configuration/global.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/GLUSTERFS/configuration/global.xml
index 99d493c..bb1f845 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/GLUSTERFS/configuration/global.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/GLUSTERFS/configuration/global.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
   <property>
     <name>hadoop_pid_dir_prefix</name>
     <value>/var/run/hadoop</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/HDFS/configuration/core-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/HDFS/configuration/core-site.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/HDFS/configuration/core-site.xml
index e244fc7..e335fe2 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/HDFS/configuration/core-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/HDFS/configuration/core-site.xml
@@ -20,7 +20,7 @@
  
 <!-- Put site-specific property overrides in this file. -->
 
-<configuration xmlns:xi="http://www.w3.org/2001/XInclude">
+<configuration supports_final="true" xmlns:xi="http://www.w3.org/2001/XInclude">
 
 <!-- i/o properties -->
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/HDFS/configuration/global.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/HDFS/configuration/global.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/HDFS/configuration/global.xml
index c17de27..dbfaddc 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/HDFS/configuration/global.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/HDFS/configuration/global.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
   <property>
     <name>namenode_host</name>
     <value></value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/HDFS/configuration/hadoop-policy.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/HDFS/configuration/hadoop-policy.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/HDFS/configuration/hadoop-policy.xml
index 51b01bb..41bde16 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/HDFS/configuration/hadoop-policy.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/HDFS/configuration/hadoop-policy.xml
@@ -20,7 +20,7 @@
 
 <!-- Put site-specific property overrides in this file. -->
 
-<configuration>
+<configuration supports_final="true">
   <property>
     <name>security.client.protocol.acl</name>
     <value>*</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/HDFS/configuration/hdfs-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/HDFS/configuration/hdfs-site.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/HDFS/configuration/hdfs-site.xml
index 64c7acd..f72feee 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/HDFS/configuration/hdfs-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6.GlusterFS/services/HDFS/configuration/hdfs-site.xml
@@ -20,7 +20,7 @@
 
 <!-- Put site-specific property overrides in this file. -->
 
-<configuration>
+<configuration supports_final="true">
 
 <!-- file system properties -->
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/FLUME/configuration/flume-conf.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/FLUME/configuration/flume-conf.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/FLUME/configuration/flume-conf.xml
index e8ff5fd..74a4c15 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/FLUME/configuration/flume-conf.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/FLUME/configuration/flume-conf.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
   <property>
     <name>content</name>
     <description>Describe all the Flume agent configurations</description>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/FLUME/configuration/flume-log4j.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/FLUME/configuration/flume-log4j.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/FLUME/configuration/flume-log4j.xml
index e5bd8c6..66c2a14 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/FLUME/configuration/flume-log4j.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/FLUME/configuration/flume-log4j.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
   <property>
     <name>content</name>
     <value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/FLUME/configuration/global.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/FLUME/configuration/global.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/FLUME/configuration/global.xml
index 7107248..8c06c95 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/FLUME/configuration/global.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/FLUME/configuration/global.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
   <property>
     <name>flume_conf_dir</name>
     <value>/etc/flume/conf</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/GANGLIA/configuration/global.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/GANGLIA/configuration/global.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/GANGLIA/configuration/global.xml
index d340a94..901f855 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/GANGLIA/configuration/global.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/GANGLIA/configuration/global.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
   <property>
     <name>ganglia_conf_dir</name>
     <value>/etc/ganglia/hdp</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/configuration/global.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/configuration/global.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/configuration/global.xml
index 2f64ce0..47dfa1c 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/configuration/global.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/configuration/global.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
   <property>
     <name>hbase_log_dir</name>
     <value>/var/log/hbase</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/configuration/hbase-log4j.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/configuration/hbase-log4j.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/configuration/hbase-log4j.xml
index 3bbc549..1895df8 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/configuration/hbase-log4j.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/configuration/hbase-log4j.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
 
   <property>
     <name>content</name>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/configuration/hbase-policy.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/configuration/hbase-policy.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/configuration/hbase-policy.xml
index e45f23c..2f12801 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/configuration/hbase-policy.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/configuration/hbase-policy.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="true">
   <property>
     <name>security.client.protocol.acl</name>
     <value>*</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/configuration/hbase-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/configuration/hbase-site.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/configuration/hbase-site.xml
index a9711d3..a0ca6d7 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/configuration/hbase-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HBASE/configuration/hbase-site.xml
@@ -19,7 +19,7 @@
  * limitations under the License.
  */
 -->
-<configuration>
+<configuration supports_final="true">
   <property>
     <name>hbase.rootdir</name>
     <value>hdfs://localhost:8020/apps/hbase/data</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/core-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/core-site.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/core-site.xml
index 356cba4..569b1fe 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/core-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/core-site.xml
@@ -20,7 +20,7 @@
  
 <!-- Put site-specific property overrides in this file. -->
 
-<configuration xmlns:xi="http://www.w3.org/2001/XInclude">
+<configuration supports_final="true" xmlns:xi="http://www.w3.org/2001/XInclude">
 
 <!-- i/o properties -->
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/global.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/global.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/global.xml
index 17f1a0b..e5163f8 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/global.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/global.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
   <property>
     <name>hdfs_log_dir_prefix</name>
     <value>/var/log/hadoop</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/hadoop-policy.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/hadoop-policy.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/hadoop-policy.xml
index 51b01bb..41bde16 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/hadoop-policy.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/hadoop-policy.xml
@@ -20,7 +20,7 @@
 
 <!-- Put site-specific property overrides in this file. -->
 
-<configuration>
+<configuration supports_final="true">
   <property>
     <name>security.client.protocol.acl</name>
     <value>*</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/hdfs-log4j.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/hdfs-log4j.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/hdfs-log4j.xml
index c0299f4..b592be5 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/hdfs-log4j.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/hdfs-log4j.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
 
   <property>
     <name>content</name>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/hdfs-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/hdfs-site.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/hdfs-site.xml
index 041ffee..8fe9603 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/hdfs-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HDFS/configuration/hdfs-site.xml
@@ -20,7 +20,7 @@
 
 <!-- Put site-specific property overrides in this file. -->
 
-<configuration>
+<configuration supports_final="true">
 
   <!-- file system properties -->
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HIVE/configuration/global.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HIVE/configuration/global.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HIVE/configuration/global.xml
index b5877cf..0953ce7 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HIVE/configuration/global.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HIVE/configuration/global.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
   <property>
     <name>hive_database_type</name>
     <value>mysql</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HIVE/configuration/hive-exec-log4j.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HIVE/configuration/hive-exec-log4j.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HIVE/configuration/hive-exec-log4j.xml
index f0cc298..0983152 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HIVE/configuration/hive-exec-log4j.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HIVE/configuration/hive-exec-log4j.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
 
   <property>
     <name>content</name>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HIVE/configuration/hive-log4j.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HIVE/configuration/hive-log4j.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HIVE/configuration/hive-log4j.xml
index 8316698..d8749da 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HIVE/configuration/hive-log4j.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HIVE/configuration/hive-log4j.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
 
   <property>
     <name>content</name>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HIVE/configuration/hive-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HIVE/configuration/hive-site.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HIVE/configuration/hive-site.xml
index 3961cb0..4bc2147 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HIVE/configuration/hive-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/HIVE/configuration/hive-site.xml
@@ -17,7 +17,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-<configuration>
+<configuration supports_final="true">
 
   <property>
     <name>hive.heapsize</name>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/NAGIOS/configuration/global.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/NAGIOS/configuration/global.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/NAGIOS/configuration/global.xml
index 54c742c..f36020d 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/NAGIOS/configuration/global.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/NAGIOS/configuration/global.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
   <property>
     <name>nagios_user</name>
     <value>nagios</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/NAGIOS/package/files/hdp_mon_nagios_addons.conf
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/NAGIOS/package/files/hdp_mon_nagios_addons.conf b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/NAGIOS/package/files/hdp_mon_nagios_addons.conf
index fbaeb2a..87717d2 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/NAGIOS/package/files/hdp_mon_nagios_addons.conf
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/NAGIOS/package/files/hdp_mon_nagios_addons.conf
@@ -1,3 +1,20 @@
+# 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.
+#
 Alias /ambarinagios /usr/share/hdp
 <Directory /usr/share/hdp>
   Options None

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/OOZIE/configuration/global.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/OOZIE/configuration/global.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/OOZIE/configuration/global.xml
index 03cff07..24422b3 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/OOZIE/configuration/global.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/OOZIE/configuration/global.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
   <property>
     <name>oozie_user</name>
     <value>oozie</value>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/OOZIE/configuration/oozie-log4j.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/OOZIE/configuration/oozie-log4j.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/OOZIE/configuration/oozie-log4j.xml
index 147361a..cb77566 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/OOZIE/configuration/oozie-log4j.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/OOZIE/configuration/oozie-log4j.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
 
   <property>
     <name>content</name>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/OOZIE/configuration/oozie-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/OOZIE/configuration/oozie-site.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/OOZIE/configuration/oozie-site.xml
index 158520e..632d606 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/OOZIE/configuration/oozie-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/OOZIE/configuration/oozie-site.xml
@@ -17,7 +17,7 @@
   limitations under the License.
 -->
 
-<configuration>
+<configuration supports_final="true">
 
   <!--
       Refer to the oozie-default.xml file for the complete list of

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/PIG/configuration/pig-log4j.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/PIG/configuration/pig-log4j.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/PIG/configuration/pig-log4j.xml
index f34724e..cbdd452 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/PIG/configuration/pig-log4j.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/PIG/configuration/pig-log4j.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
 
   <property>
     <name>content</name>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/PIG/configuration/pig-properties.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/PIG/configuration/pig-properties.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/PIG/configuration/pig-properties.xml
index 185d3b0..d9b582e 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/PIG/configuration/pig-properties.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/PIG/configuration/pig-properties.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
 
   <property>
     <name>pig-content</name>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/WEBHCAT/configuration/webhcat-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/WEBHCAT/configuration/webhcat-site.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/WEBHCAT/configuration/webhcat-site.xml
index 0ff2cde..0523dab 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/WEBHCAT/configuration/webhcat-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/WEBHCAT/configuration/webhcat-site.xml
@@ -21,7 +21,7 @@ limitations under the License.
 <!-- Edit templeton-site.xml to change settings for your local -->
 <!-- install. -->
 
-<configuration>
+<configuration supports_final="true">
 
   <property>
     <name>templeton.port</name>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/YARN/configuration-mapred/core-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/YARN/configuration-mapred/core-site.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/YARN/configuration-mapred/core-site.xml
index 3a2af49..60f01ad 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/YARN/configuration-mapred/core-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/YARN/configuration-mapred/core-site.xml
@@ -16,5 +16,5 @@
    See the License for the specific language governing permissions and
    limitations under the License.
 -->
-<configuration>
+<configuration supports_final="true">
 </configuration>

http://git-wip-us.apache.org/repos/asf/ambari/blob/b08909a1/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/YARN/configuration-mapred/global.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/YARN/configuration-mapred/global.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/YARN/configuration-mapred/global.xml
index e21a11e..27cffdf 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/YARN/configuration-mapred/global.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/YARN/configuration-mapred/global.xml
@@ -20,7 +20,7 @@
  */
 -->
 
-<configuration>
+<configuration supports_final="false">
   <property>
     <name>mapred_log_dir_prefix</name>
     <value>/var/log/hadoop-mapreduce</value>