You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by ao...@apache.org on 2016/03/10 12:05:31 UTC

[4/4] ambari git commit: AMBARI-15228. Ambari overwrites permissions on HDFS directories (aonishuk)

AMBARI-15228. Ambari overwrites permissions on HDFS directories (aonishuk)


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

Branch: refs/heads/branch-2.2
Commit: 23386d0d5207a7176c2b940c67b10e9b081b4937
Parents: 3efb9f1
Author: Andrew Onishuk <ao...@hortonworks.com>
Authored: Thu Mar 10 13:04:36 2016 +0200
Committer: Andrew Onishuk <ao...@hortonworks.com>
Committed: Thu Mar 10 13:04:36 2016 +0200

----------------------------------------------------------------------
 .../functions/get_not_managed_resources.py      |  50 +++++++++++++++++++
 .../libraries/providers/hdfs_resource.py        |  20 ++++++--
 .../libraries/resources/hdfs_resource.py        |   9 +++-
 .../ambari/server/agent/ExecutionCommand.java   |   1 +
 .../AmbariCustomCommandExecutionHelper.java     |  14 +++++-
 .../AmbariManagementControllerImpl.java         |   5 ++
 .../internal/ClientConfigResourceProvider.java  |   5 ++
 .../ambari/server/state/PropertyInfo.java       |   3 +-
 .../1.6.1.2.2.0/package/scripts/params.py       |   4 +-
 .../0.1.0/package/scripts/params.py             |   4 +-
 .../0.5.0.2.1/configuration/falcon-env.xml      |  14 +++++-
 .../FALCON/0.5.0.2.1/package/scripts/falcon.py  |   2 +-
 .../0.5.0.2.1/package/scripts/params_linux.py   |   6 ++-
 .../HAWQ/2.0.0/package/scripts/params.py        |   4 +-
 .../0.96.0.2.0/package/scripts/params_linux.py  |   4 +-
 .../HDFS/2.1.0.2.0/configuration/hadoop-env.xml |  12 +++++
 .../2.1.0.2.0/package/scripts/hdfs_namenode.py  |   2 +-
 .../2.1.0.2.0/package/scripts/params_linux.py   |   6 ++-
 .../2.1.0.2.0/package/scripts/service_check.py  |   4 +-
 .../HIVE/0.12.0.2.0/configuration/hive-site.xml |   1 +
 .../0.12.0.2.0/package/scripts/params_linux.py  |   4 +-
 .../KAFKA/0.8.1.2.2/package/scripts/params.py   |   5 +-
 .../0.5.0.2.2/package/scripts/params_linux.py   |   4 +-
 .../MAHOUT/1.0.0.2.3/package/scripts/params.py  |   4 +-
 .../4.0.0.2.0/package/scripts/params_linux.py   |   4 +-
 .../0.12.0.2.0/package/scripts/params_linux.py  |   4 +-
 .../PXF/3.0.0/package/scripts/params.py         |   4 +-
 .../0.60.0.2.2/package/scripts/params_linux.py  |   4 +-
 .../SPARK/1.2.0.2.2/package/scripts/params.py   |   4 +-
 .../0.9.1.2.1/package/scripts/params_linux.py   |   4 +-
 .../0.4.0.2.1/package/scripts/params_linux.py   |   4 +-
 .../configuration-mapred/mapred-site.xml        |   1 +
 .../YARN/2.1.0.2.0/configuration/yarn-site.xml  |   1 +
 .../2.1.0.2.0/package/scripts/params_linux.py   |   4 +-
 .../HDP/2.0.6/configuration/cluster-env.xml     |   9 ++++
 .../before-START/files/fast-hdfs-resource.jar   | Bin 19285461 -> 19285554 bytes
 .../services/HIVE/configuration/hive-site.xml   |   1 +
 .../services/HIVE/configuration/hive-site.xml   |   1 +
 .../services/HIVE/configuration/hive-site.xml   |   1 +
 .../AmbariManagementControllerImplTest.java     |  23 ++++++++-
 .../AMBARI_METRICS/test_metrics_collector.py    |   3 ++
 .../stacks/2.0.6/HBASE/test_hbase_master.py     |   9 ++++
 .../python/stacks/2.0.6/HDFS/test_namenode.py   |  27 ++++++++++
 .../stacks/2.0.6/HDFS/test_service_check.py     |   4 ++
 .../stacks/2.0.6/HIVE/test_hive_server.py       |  14 ++++++
 .../2.0.6/HIVE/test_hive_service_check.py       |   6 +++
 .../stacks/2.0.6/OOZIE/test_oozie_server.py     |  16 ++++++
 .../stacks/2.0.6/OOZIE/test_service_check.py    |   5 ++
 .../stacks/2.0.6/PIG/test_pig_service_check.py  |   6 +++
 .../stacks/2.0.6/YARN/test_historyserver.py     |  16 ++++++
 .../2.0.6/YARN/test_mapreduce2_service_check.py |   6 +++
 .../stacks/2.0.6/configs/altfs_plus_hdfs.json   |   3 ++
 .../stacks/2.0.6/configs/client-upgrade.json    |   2 +
 .../2.0.6/configs/default.hbasedecom.json       |   2 +
 .../python/stacks/2.0.6/configs/default.json    |   4 ++
 .../2.0.6/configs/default.non_gmetad_host.json  |   4 +-
 .../2.0.6/configs/default_ams_embedded.json     |   2 +
 .../stacks/2.0.6/configs/default_client.json    |   2 +
 .../2.0.6/configs/default_hive_nn_ha.json       |   2 +
 .../2.0.6/configs/default_hive_nn_ha_2.json     |   2 +
 .../2.0.6/configs/default_hive_non_hdfs.json    |   2 +
 .../2.0.6/configs/default_no_install.json       |   2 +
 .../2.0.6/configs/default_oozie_mysql.json      |   2 +
 .../default_update_exclude_file_only.json       |   2 +
 .../python/stacks/2.0.6/configs/flume_22.json   |   4 +-
 .../python/stacks/2.0.6/configs/flume_only.json |   3 +-
 .../stacks/2.0.6/configs/flume_target.json      |   3 +-
 .../2.0.6/configs/ha_bootstrap_active_node.json |   3 ++
 .../configs/ha_bootstrap_standby_node.json      |   3 ++
 ...ha_bootstrap_standby_node_initial_start.json |   3 ++
 .../python/stacks/2.0.6/configs/ha_default.json |   3 ++
 .../python/stacks/2.0.6/configs/ha_secured.json |   3 ++
 .../python/stacks/2.0.6/configs/hbase-2.2.json  |   2 +
 .../stacks/2.0.6/configs/hbase-check-2.2.json   |   2 +
 .../stacks/2.0.6/configs/hbase-preupgrade.json  |   2 +
 .../2.0.6/configs/hbase-rs-2.2-phoenix.json     |   2 +
 .../stacks/2.0.6/configs/hbase-rs-2.2.json      |   2 +
 .../stacks/2.0.6/configs/hbase_no_phx.json      |   2 +
 .../stacks/2.0.6/configs/hbase_with_phx.json    |   2 +
 .../test/python/stacks/2.0.6/configs/nn_eu.json |   5 +-
 .../stacks/2.0.6/configs/nn_eu_standby.json     |   5 +-
 .../python/stacks/2.0.6/configs/nn_ru_lzo.json  |   3 ++
 .../2.0.6/configs/oozie_existing_sqla.json      |   2 +
 .../2.0.6/configs/ranger-namenode-start.json    |   2 +
 .../2.0.6/configs/rebalancehdfs_default.json    |   2 +
 .../2.0.6/configs/rebalancehdfs_secured.json    |   2 +
 .../python/stacks/2.0.6/configs/secured.json    |   3 ++
 .../stacks/2.0.6/configs/secured_client.json    |   2 +
 .../2.0.6/configs/secured_no_jce_name.json      |   3 +-
 .../2.0.6/configs/zk-service_check_2.2.json     |   4 +-
 .../stacks/2.1/FALCON/test_falcon_server.py     |   6 +++
 .../python/stacks/2.1/TEZ/test_service_check.py |   8 +++
 .../stacks/2.1/YARN/test_apptimelineserver.py   |   1 +
 .../stacks/2.1/configs/client-upgrade.json      |   2 +
 .../stacks/2.1/configs/default-storm-start.json |   2 +
 .../test/python/stacks/2.1/configs/default.json |   3 ++
 .../2.1/configs/hive-metastore-upgrade.json     |   4 +-
 .../stacks/2.1/configs/secured-storm-start.json |   6 ++-
 .../test/python/stacks/2.1/configs/secured.json |   2 +
 .../stacks/2.2/PIG/test_pig_service_check.py    |   6 +++
 .../stacks/2.2/SPARK/test_job_history_server.py |   6 +++
 .../test/python/stacks/2.2/configs/default.json |   2 +
 .../2.2/configs/default_custom_path_config.json |   2 +
 .../stacks/2.2/configs/falcon-upgrade.json      |   5 +-
 .../python/stacks/2.2/configs/hive-upgrade.json |   2 +
 .../journalnode-upgrade-hdfs-secure.json        |   2 +
 .../stacks/2.2/configs/journalnode-upgrade.json |   2 +
 .../python/stacks/2.2/configs/knox_upgrade.json |   6 ++-
 .../stacks/2.2/configs/oozie-downgrade.json     |   2 +
 .../stacks/2.2/configs/oozie-upgrade.json       |   2 +
 .../2.2/configs/pig-service-check-secure.json   |   4 +-
 .../2.2/configs/ranger-admin-default.json       |   2 +
 .../2.2/configs/ranger-admin-secured.json       |   2 +
 .../2.2/configs/ranger-admin-upgrade.json       |   4 +-
 .../2.2/configs/ranger-usersync-upgrade.json    |   4 +-
 .../test/python/stacks/2.2/configs/secured.json |   2 +
 .../2.2/configs/spark-job-history-server.json   |   4 +-
 .../2.3/MAHOUT/test_mahout_service_check.py     |   4 ++
 .../2.3/SPARK/test_spark_thrift_server.py       |   2 +
 .../test/python/stacks/2.3/YARN/test_ats_1_5.py |   5 ++
 .../test/python/stacks/2.3/configs/ats_1_5.json |   4 +-
 .../stacks/2.3/configs/default.hbasedecom.json  |   2 +
 .../test/python/stacks/2.3/configs/default.json |   2 +
 .../stacks/2.3/configs/hbase_default.json       |   2 +
 .../python/stacks/2.3/configs/hbase_secure.json |   2 +
 .../python/stacks/2.3/configs/pxf_default.json  |   2 +
 .../stacks/2.3/configs/spark_default.json       |   2 +
 .../stacks/2.3/configs/storm_default.json       |   4 +-
 .../2.3/configs/storm_default_secure.json       |   4 +-
 .../ambari/fast_hdfs_resource/Resource.java     |  15 ++++--
 .../ambari/fast_hdfs_resource/Runner.java       |   6 +++
 131 files changed, 561 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-common/src/main/python/resource_management/libraries/functions/get_not_managed_resources.py
----------------------------------------------------------------------
diff --git a/ambari-common/src/main/python/resource_management/libraries/functions/get_not_managed_resources.py b/ambari-common/src/main/python/resource_management/libraries/functions/get_not_managed_resources.py
new file mode 100644
index 0000000..5f8bc67
--- /dev/null
+++ b/ambari-common/src/main/python/resource_management/libraries/functions/get_not_managed_resources.py
@@ -0,0 +1,50 @@
+#!/usr/bin/env python
+"""
+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.
+
+Ambari Agent
+
+"""
+__all__ = ["get_not_managed_resources"]
+
+import json
+from resource_management.libraries.script import Script
+from resource_management.core.logger import Logger
+from resource_management.libraries.functions.default import default
+
+def get_not_managed_resources():
+  """
+  Returns a list of not managed hdfs paths.
+  The result contains all paths from hostLevelParams/not_managed_hdfs_path_list
+  except config values from cluster-env/managed_hdfs_resource_property_names
+  """
+  config = Script.get_config()
+  not_managed_hdfs_path_list = json.loads(config['hostLevelParams']['not_managed_hdfs_path_list'])[:]
+  managed_hdfs_resource_property_names = config['configurations']['cluster-env']['managed_hdfs_resource_property_names']
+  managed_hdfs_resource_property_list = filter(None, [property.strip() for property in managed_hdfs_resource_property_names.split(',')])
+
+  for property_name in managed_hdfs_resource_property_list:
+    property_value = default('/configurations/' + property_name, None)
+
+    if property_value == None:
+      Logger.warning(("Property {0} from cluster-env/managed_hdfs_resource_property_names not found in configurations. "
+                     "Management of this DFS resource will not be forced.").format(property_name))
+    else:
+      while property_value in not_managed_hdfs_path_list:
+        not_managed_hdfs_path_list.remove(property_value)
+
+  return not_managed_hdfs_path_list
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-common/src/main/python/resource_management/libraries/providers/hdfs_resource.py
----------------------------------------------------------------------
diff --git a/ambari-common/src/main/python/resource_management/libraries/providers/hdfs_resource.py b/ambari-common/src/main/python/resource_management/libraries/providers/hdfs_resource.py
index ed9a642..d200956 100644
--- a/ambari-common/src/main/python/resource_management/libraries/providers/hdfs_resource.py
+++ b/ambari-common/src/main/python/resource_management/libraries/providers/hdfs_resource.py
@@ -51,7 +51,9 @@ RESOURCE_TO_JSON_FIELDS = {
   'mode': 'mode',
   'recursive_chown': 'recursiveChown',
   'recursive_chmod': 'recursiveChmod',
-  'change_permissions_for_parents': 'changePermissionforParents'
+  'change_permissions_for_parents': 'changePermissionforParents',
+  'manage_if_exists': 'manageIfExists'
+
 }
 
 class HdfsResourceJar:
@@ -76,6 +78,8 @@ class HdfsResourceJar:
         resource[json_field_name] = action_name
       elif field_name == 'mode' and main_resource.resource.mode:
         resource[json_field_name] = oct(main_resource.resource.mode)[1:]
+      elif field_name == 'manage_if_exists':
+        resource[json_field_name] = main_resource.manage_if_exists
       elif getattr(main_resource.resource, field_name):
         resource[json_field_name] = getattr(main_resource.resource, field_name)
 
@@ -244,7 +248,12 @@ class HdfsResourceWebHDFS:
     self.mode_set = False
     self.main_resource = main_resource
     self._assert_valid()
-
+    
+    if self.main_resource.manage_if_exists == False and self.target_status:
+      Logger.info("Skipping the operation for not managed DFS directory " + str(self.main_resource.resource.target) +
+                  " since immutable_paths contains it.")
+      return        
+    
     if action_name == "create":
       self._create_resource()
       self._set_mode(self.target_status)
@@ -437,7 +446,12 @@ class HdfsResourceProvider(Provider):
   def action_delayed(self, action_name):
     self.assert_parameter_is_set('type')
 
-    if HdfsResourceProvider.parse_path(self.resource.target) in self.ignored_resources_list:
+    parsed_path = HdfsResourceProvider.parse_path(self.resource.target)
+
+    parsed_not_managed_paths = [HdfsResourceProvider.parse_path(path) for path in self.resource.immutable_paths]
+    self.manage_if_exists = not parsed_path in parsed_not_managed_paths
+
+    if parsed_path in self.ignored_resources_list:
       Logger.info("Skipping '{0}' because it is in ignore file {1}.".format(self.resource, self.resource.hdfs_resource_ignore_file))
       return
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-common/src/main/python/resource_management/libraries/resources/hdfs_resource.py
----------------------------------------------------------------------
diff --git a/ambari-common/src/main/python/resource_management/libraries/resources/hdfs_resource.py b/ambari-common/src/main/python/resource_management/libraries/resources/hdfs_resource.py
index 06d0f2b..03221ac 100644
--- a/ambari-common/src/main/python/resource_management/libraries/resources/hdfs_resource.py
+++ b/ambari-common/src/main/python/resource_management/libraries/resources/hdfs_resource.py
@@ -87,7 +87,14 @@ class HdfsResource(Resource):
   /var/lib/ambari-agent/data/.hdfs_resource_ignore
   """
   hdfs_resource_ignore_file = ResourceArgument()
-  
+
+  """
+  If the name of the HdfsResource is in immutable_paths
+  and it is already created, any actions on it will be skipped
+  (like changing permissions/recursive permissions, copying from source, deleting etc.)
+  """
+  immutable_paths = ResourceArgument(default=[])
+
   # WebHDFS needs these
   hdfs_site = ResourceArgument()
   default_fs = ResourceArgument()

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/java/org/apache/ambari/server/agent/ExecutionCommand.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/agent/ExecutionCommand.java b/ambari-server/src/main/java/org/apache/ambari/server/agent/ExecutionCommand.java
index 2ae4a80..4ffc663 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/agent/ExecutionCommand.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/agent/ExecutionCommand.java
@@ -343,6 +343,7 @@ public class ExecutionCommand extends AgentCommand {
     String REFRESH_ADITIONAL_COMPONENT_TAGS = "forceRefreshConfigTags";
     String USER_LIST = "user_list";
     String GROUP_LIST = "group_list";
+    String NOT_MANAGED_HDFS_PATH_LIST = "not_managed_hdfs_path_list";
     String VERSION = "version";
     String REFRESH_TOPOLOGY = "refresh_topology";
     String HOST_SYS_PREPPED = "host_sys_prepped";

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelper.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelper.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelper.java
index d85f17e..5688df2 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelper.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelper.java
@@ -35,6 +35,7 @@ import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.JCE_NAME;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.JDK_LOCATION;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.JDK_NAME;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.MYSQL_JDBC_URL;
+import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.NOT_MANAGED_HDFS_PATH_LIST;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.ORACLE_JDBC_URL;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.REPO_INFO;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.SCRIPT;
@@ -373,6 +374,10 @@ public class AmbariCustomCommandExecutionHelper {
       String groupList = gson.toJson(groupSet);
       hostLevelParams.put(GROUP_LIST, groupList);
 
+      Set<String> notManagedHdfsPathSet = configHelper.getPropertyValuesWithPropertyType(stackId, PropertyType.NOT_MANAGED_HDFS_PATH, cluster);
+      String notManagedHdfsPathList = gson.toJson(notManagedHdfsPathSet);
+      hostLevelParams.put(NOT_MANAGED_HDFS_PATH_LIST, notManagedHdfsPathList);
+
       execCmd.setHostLevelParams(hostLevelParams);
 
       Map<String, String> commandParams = new TreeMap<String, String>();
@@ -1103,12 +1108,12 @@ public class AmbariCustomCommandExecutionHelper {
         hostParamsStageJson);
   }
 
-  Map<String, String> createDefaultHostParams(Cluster cluster) {
+  Map<String, String> createDefaultHostParams(Cluster cluster) throws AmbariException {
     StackId stackId = cluster.getDesiredStackVersion();
     return createDefaultHostParams(cluster, stackId);
   }
 
-  Map<String, String> createDefaultHostParams(Cluster cluster, StackId stackId) {
+  Map<String, String> createDefaultHostParams(Cluster cluster, StackId stackId) throws AmbariException{
     TreeMap<String, String> hostLevelParams = new TreeMap<String, String>();
     hostLevelParams.put(JDK_LOCATION, managementController.getJdkResourceUrl());
     hostLevelParams.put(JAVA_HOME, managementController.getJavaHome());
@@ -1125,6 +1130,11 @@ public class AmbariCustomCommandExecutionHelper {
     hostLevelParams.put(HOST_SYS_PREPPED, configs.areHostsSysPrepped());
     hostLevelParams.put(AGENT_STACK_RETRY_ON_UNAVAILABILITY, configs.isAgentStackRetryOnInstallEnabled());
     hostLevelParams.put(AGENT_STACK_RETRY_COUNT, configs.getAgentStackRetryOnInstallCount());
+
+    Set<String> notManagedHdfsPathSet = configHelper.getPropertyValuesWithPropertyType(stackId, PropertyType.NOT_MANAGED_HDFS_PATH, cluster);
+    String notManagedHdfsPathList = gson.toJson(notManagedHdfsPathSet);
+    hostLevelParams.put(NOT_MANAGED_HDFS_PATH_LIST, notManagedHdfsPathList);
+
     ClusterVersionEntity clusterVersionEntity = clusterVersionDAO.findByClusterAndStateCurrent(cluster.getClusterName());
     if (clusterVersionEntity == null) {
       List<ClusterVersionEntity> clusterVersionEntityList = clusterVersionDAO

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
index 157b961..0461a0d 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
@@ -170,6 +170,7 @@ import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.COMMAND_R
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.COMMAND_TIMEOUT;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.DB_DRIVER_FILENAME;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.GROUP_LIST;
+import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.NOT_MANAGED_HDFS_PATH_LIST;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.HOOKS_FOLDER;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.MAX_DURATION_OF_RETRIES;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.PACKAGE_LIST;
@@ -2053,6 +2054,10 @@ public class AmbariManagementControllerImpl implements AmbariManagementControlle
     String groupList = gson.toJson(groupSet);
     hostParams.put(GROUP_LIST, groupList);
 
+    Set<String> notManagedHdfsPathSet = configHelper.getPropertyValuesWithPropertyType(stackId, PropertyType.NOT_MANAGED_HDFS_PATH, cluster);
+    String notManagedHdfsPathList = gson.toJson(notManagedHdfsPathSet);
+    hostParams.put(NOT_MANAGED_HDFS_PATH_LIST, notManagedHdfsPathList);
+
     DatabaseType databaseType = configs.getDatabaseType();
     if (databaseType == DatabaseType.ORACLE) {
       hostParams.put(DB_DRIVER_FILENAME, configs.getOjdbcJarName());

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/ClientConfigResourceProvider.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/ClientConfigResourceProvider.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/ClientConfigResourceProvider.java
index 43aba04..2d26e4b 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/ClientConfigResourceProvider.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/ClientConfigResourceProvider.java
@@ -83,6 +83,7 @@ import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.JCE_NAME;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.JDK_LOCATION;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.JDK_NAME;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.MYSQL_JDBC_URL;
+import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.NOT_MANAGED_HDFS_PATH_LIST;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.ORACLE_JDBC_URL;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.PACKAGE_LIST;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.SERVICE_REPO_INFO;
@@ -339,6 +340,10 @@ public class ClientConfigResourceProvider extends AbstractControllerResourceProv
       String groupList = gson.toJson(groupSet);
       hostLevelParams.put(GROUP_LIST, groupList);
 
+      Set<String> notManagedHdfsPathSet = configHelper.getPropertyValuesWithPropertyType(stackId, PropertyType.NOT_MANAGED_HDFS_PATH, cluster);
+      String notManagedHdfsPathList = gson.toJson(notManagedHdfsPathSet);
+      hostLevelParams.put(NOT_MANAGED_HDFS_PATH_LIST, notManagedHdfsPathList);
+
       String jsonConfigurations = null;
       Map<String, Object> commandParams = new HashMap<String, Object>();
       List<Map<String, String>> xmlConfigs = new LinkedList<Map<String, String>>();

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/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 bc90a8c..deb0077 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
@@ -234,6 +234,7 @@ public class PropertyInfo {
     GROUP,
     TEXT,
     ADDITIONAL_USER_PROPERTY,
-    DONT_ADD_ON_UPGRADE
+    DONT_ADD_ON_UPGRADE,
+    NOT_MANAGED_HDFS_PATH
   }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/ACCUMULO/1.6.1.2.2.0/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/ACCUMULO/1.6.1.2.2.0/package/scripts/params.py b/ambari-server/src/main/resources/common-services/ACCUMULO/1.6.1.2.2.0/package/scripts/params.py
index 09af8a9..3b8505b 100644
--- a/ambari-server/src/main/resources/common-services/ACCUMULO/1.6.1.2.2.0/package/scripts/params.py
+++ b/ambari-server/src/main/resources/common-services/ACCUMULO/1.6.1.2.2.0/package/scripts/params.py
@@ -21,6 +21,7 @@ from resource_management.libraries.functions import conf_select
 from resource_management.libraries.functions import hdp_select
 from resource_management.libraries.resources.hdfs_resource import HdfsResource
 from resource_management.libraries.functions import format
+from resource_management.libraries.functions.get_not_managed_resources import get_not_managed_resources
 from resource_management.libraries.functions.version import format_hdp_stack_version
 from resource_management.libraries.functions.default import default
 from resource_management.libraries.functions.get_bare_principal import get_bare_principal
@@ -191,5 +192,6 @@ HdfsResource = functools.partial(
   hadoop_conf_dir = hadoop_conf_dir,
   principal_name = hdfs_principal_name,
   hdfs_site = hdfs_site,
-  default_fs = default_fs
+  default_fs = default_fs,
+  immutable_paths = get_not_managed_resources()
 )

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/AMBARI_METRICS/0.1.0/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/AMBARI_METRICS/0.1.0/package/scripts/params.py b/ambari-server/src/main/resources/common-services/AMBARI_METRICS/0.1.0/package/scripts/params.py
index acc0fcb..12c0f25 100644
--- a/ambari-server/src/main/resources/common-services/AMBARI_METRICS/0.1.0/package/scripts/params.py
+++ b/ambari-server/src/main/resources/common-services/AMBARI_METRICS/0.1.0/package/scripts/params.py
@@ -23,6 +23,7 @@ from functions import check_append_heap_property
 from functions import trim_heap_property
 from resource_management.core.logger import Logger
 from resource_management import *
+from resource_management.libraries.functions.get_not_managed_resources import get_not_managed_resources
 import status_params
 from ambari_commons import OSCheck
 import ConfigParser
@@ -318,7 +319,8 @@ HdfsResource = functools.partial(
   hadoop_conf_dir = hadoop_conf_dir,
   principal_name = hdfs_principal_name,
   hdfs_site = hdfs_site,
-  default_fs = default_fs
+  default_fs = default_fs,
+  immutable_paths = get_not_managed_resources()
  )
 
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/configuration/falcon-env.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/configuration/falcon-env.xml b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/configuration/falcon-env.xml
index 5f942f6..88f5bf6 100644
--- a/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/configuration/falcon-env.xml
+++ b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/configuration/falcon-env.xml
@@ -98,7 +98,19 @@
       <overridable>false</overridable>
     </value-attributes>
   </property>
-  
+  <property>
+    <name>falcon_apps_hdfs_dir</name>
+    <value>/apps/falcon</value>
+    <description>Falcon Apps HDFS Dir</description>
+    <display-name>Falcon Apps HDFS Dir</display-name>
+    <property-type>NOT_MANAGED_HDFS_PATH</property-type>
+    <value-attributes>
+      <read-only>true</read-only>
+      <overridable>false</overridable>
+      <visible>false</visible>
+    </value-attributes>
+  </property>
+
   <!-- falcon-env.sh -->
   <property>
     <name>content</name>

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/falcon.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/falcon.py b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/falcon.py
index 6c8d5ff..c63ef98 100644
--- a/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/falcon.py
+++ b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/falcon.py
@@ -120,7 +120,7 @@ def falcon(type, action = None, upgrade_type=None):
           recursive = True)
 
       # TODO change to proper mode
-      params.HdfsResource(params.flacon_apps_dir,
+      params.HdfsResource(params.falcon_apps_dir,
         type = "directory",
         action = "create_on_execute",
         owner = params.falcon_user,

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/params_linux.py b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/params_linux.py
index 7564905..d442eed 100644
--- a/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/params_linux.py
+++ b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/params_linux.py
@@ -23,6 +23,7 @@ from resource_management.libraries.functions import hdp_select
 from resource_management.libraries.functions import format
 from resource_management.libraries.functions.version import format_hdp_stack_version
 from resource_management.libraries.functions.default import default
+from resource_management.libraries.functions.get_not_managed_resources import get_not_managed_resources
 from resource_management.libraries.functions import get_kinit_path
 from resource_management.libraries.script.script import Script
 
@@ -92,7 +93,7 @@ falcon_startup_properties = config['configurations']['falcon-startup.properties'
 smokeuser_keytab = config['configurations']['cluster-env']['smokeuser_keytab']
 falcon_env_sh_template = config['configurations']['falcon-env']['content']
 
-flacon_apps_dir = '/apps/falcon'
+falcon_apps_dir = config['configurations']['falcon-env']['falcon_apps_hdfs_dir']
 #for create_hdfs_directory
 security_enabled = config['configurations']['cluster-env']['security_enabled']
 hostname = config["hostname"]
@@ -125,6 +126,7 @@ HdfsResource = functools.partial(
   hadoop_conf_dir = hadoop_conf_dir,
   principal_name = hdfs_principal_name,
   hdfs_site = hdfs_site,
-  default_fs = default_fs
+  default_fs = default_fs,
+  immutable_paths = get_not_managed_resources()
  )
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/HAWQ/2.0.0/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HAWQ/2.0.0/package/scripts/params.py b/ambari-server/src/main/resources/common-services/HAWQ/2.0.0/package/scripts/params.py
index 50a8fda..1bdc5aa 100644
--- a/ambari-server/src/main/resources/common-services/HAWQ/2.0.0/package/scripts/params.py
+++ b/ambari-server/src/main/resources/common-services/HAWQ/2.0.0/package/scripts/params.py
@@ -24,6 +24,7 @@ from resource_management.libraries.functions.default import default
 from resource_management.libraries.resources.hdfs_resource import HdfsResource
 from resource_management.libraries.resources.xml_config import XmlConfig
 from resource_management.libraries.functions import get_kinit_path
+from resource_management.libraries.functions.get_not_managed_resources import get_not_managed_resources
 
 config = Script.get_config()
 config_attrs = config['configuration_attributes']
@@ -74,7 +75,8 @@ HdfsResource = functools.partial(HdfsResource,
                                  kinit_path_local=kinit_path_local,
                                  principal_name=hdfs_principal_name,
                                  hdfs_site=hdfs_site,
-                                 default_fs=default_fs)
+                                 default_fs=default_fs,
+                                 immutable_paths = get_not_managed_resources())
 
 # File partial function
 File = functools.partial(File,

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/params_linux.py b/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/params_linux.py
index 7e7a070..b517eeb 100644
--- a/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/params_linux.py
+++ b/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/params_linux.py
@@ -35,6 +35,7 @@ from resource_management.libraries.functions.default import default
 from resource_management.libraries.functions import get_kinit_path
 from resource_management.libraries.functions import is_empty
 from resource_management.libraries.functions import get_unique_id_and_date
+from resource_management.libraries.functions.get_not_managed_resources import get_not_managed_resources
 from resource_management.libraries.script.script import Script
 
 
@@ -240,7 +241,8 @@ HdfsResource = functools.partial(
   hadoop_conf_dir = hadoop_conf_dir,
   principal_name = hdfs_principal_name,
   hdfs_site = hdfs_site,
-  default_fs = default_fs
+  default_fs = default_fs,
+  immutable_paths = get_not_managed_resources()
 )
 
 # ranger host

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/configuration/hadoop-env.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/configuration/hadoop-env.xml b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/configuration/hadoop-env.xml
index 34bb032..cade180 100644
--- a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/configuration/hadoop-env.xml
+++ b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/configuration/hadoop-env.xml
@@ -179,6 +179,18 @@
     <description>User to run HDFS as</description>
   </property>
   <property>
+    <name>hdfs_tmp_dir</name>
+    <value>/tmp</value>
+    <description>HDFS tmp Dir</description>
+    <display-name>HDFS tmp Dir</display-name>
+    <property-type>NOT_MANAGED_HDFS_PATH</property-type>
+    <value-attributes>
+      <read-only>true</read-only>
+      <overridable>false</overridable>
+      <visible>false</visible>
+    </value-attributes>
+  </property>
+  <property>
     <name>hdfs_user_nofile_limit</name>
     <value>128000</value>
     <description>Max open files limit setting for HDFS user.</description>

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/hdfs_namenode.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/hdfs_namenode.py b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/hdfs_namenode.py
index d61dc2e..bac5c89 100644
--- a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/hdfs_namenode.py
+++ b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/hdfs_namenode.py
@@ -259,7 +259,7 @@ def create_name_dirs(directories):
 def create_hdfs_directories(check):
   import params
 
-  params.HdfsResource("/tmp",
+  params.HdfsResource(params.hdfs_tmp_dir,
                        type="directory",
                        action="create_on_execute",
                        owner=params.hdfs_user,

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params_linux.py b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params_linux.py
index 8079631..60dfc6e 100644
--- a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params_linux.py
+++ b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params_linux.py
@@ -33,6 +33,7 @@ from resource_management.libraries.functions.version import format_hdp_stack_ver
 from resource_management.libraries.functions.default import default
 from resource_management.libraries.functions import get_klist_path
 from resource_management.libraries.functions import get_kinit_path
+from resource_management.libraries.functions.get_not_managed_resources import get_not_managed_resources
 from resource_management.libraries.script.script import Script
 from resource_management.libraries.resources.hdfs_resource import HdfsResource
 
@@ -75,6 +76,8 @@ dfs_http_policy = default('/configurations/hdfs-site/dfs.http.policy', None)
 dfs_dn_ipc_address = config['configurations']['hdfs-site']['dfs.datanode.ipc.address']
 secure_dn_ports_are_in_use = False
 
+hdfs_tmp_dir = config['configurations']['hadoop-env']['hdfs_tmp_dir']
+
 # hadoop default parameters
 mapreduce_libs_path = "/usr/lib/hadoop-mapreduce/*"
 hadoop_libexec_dir = hdp_select.get_hadoop_dir("libexec")
@@ -333,7 +336,8 @@ HdfsResource = functools.partial(
   hadoop_conf_dir = hadoop_conf_dir,
   principal_name = hdfs_principal_name,
   hdfs_site = hdfs_site,
-  default_fs = default_fs
+  default_fs = default_fs,
+  immutable_paths = get_not_managed_resources()
 )
 
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/service_check.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/service_check.py b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/service_check.py
index 6ec3996..30f9daa 100644
--- a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/service_check.py
+++ b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/service_check.py
@@ -34,7 +34,7 @@ class HdfsServiceCheckDefault(HdfsServiceCheck):
 
     env.set_params(params)
     unique = functions.get_unique_id_and_date()
-    dir = '/tmp'
+    dir = params.hdfs_tmp_dir
     tmp_file = format("{dir}/{unique}")
 
     safemode_command = format("dfsadmin -fs {namenode_address} -safemode get | grep OFF")
@@ -119,7 +119,7 @@ class HdfsServiceCheckWindows(HdfsServiceCheck):
     unique = functions.get_unique_id_and_date()
 
     #Hadoop uses POSIX-style paths, separator is always /
-    dir = '/tmp'
+    dir = params.hdfs_tmp_dir
     tmp_file = dir + '/' + unique
 
     #commands for execution

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/configuration/hive-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/configuration/hive-site.xml b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/configuration/hive-site.xml
index 078900e..190a3c3 100644
--- a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/configuration/hive-site.xml
+++ b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/configuration/hive-site.xml
@@ -90,6 +90,7 @@ limitations under the License.
     <name>hive.metastore.warehouse.dir</name>
     <value>/apps/hive/warehouse</value>
     <description>location of default database for the warehouse</description>
+    <property-type>NOT_MANAGED_HDFS_PATH</property-type>
   </property>
 
   <property>

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/params_linux.py b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/params_linux.py
index b619b78..25b79ab 100644
--- a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/params_linux.py
+++ b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/params_linux.py
@@ -35,6 +35,7 @@ from resource_management.libraries.functions.is_empty import is_empty
 from resource_management.libraries.functions.version import format_hdp_stack_version
 from resource_management.libraries.functions.copy_tarball import STACK_VERSION_PATTERN
 from resource_management.libraries.functions import get_kinit_path
+from resource_management.libraries.functions.get_not_managed_resources import get_not_managed_resources
 from resource_management.libraries.script.script import Script
 from resource_management.libraries.functions.get_port_from_url import get_port_from_url
 from resource_management.libraries import functions
@@ -469,7 +470,8 @@ HdfsResource = functools.partial(
   hadoop_conf_dir = hadoop_conf_dir,
   principal_name = hdfs_principal_name,
   hdfs_site = hdfs_site,
-  default_fs = default_fs
+  default_fs = default_fs,
+  immutable_paths = get_not_managed_resources()
  )
 
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/KAFKA/0.8.1.2.2/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/KAFKA/0.8.1.2.2/package/scripts/params.py b/ambari-server/src/main/resources/common-services/KAFKA/0.8.1.2.2/package/scripts/params.py
index dc46749..f75d578 100644
--- a/ambari-server/src/main/resources/common-services/KAFKA/0.8.1.2.2/package/scripts/params.py
+++ b/ambari-server/src/main/resources/common-services/KAFKA/0.8.1.2.2/package/scripts/params.py
@@ -30,7 +30,7 @@ from resource_management.libraries.resources.hdfs_resource import HdfsResource
 from resource_management.libraries.functions import hdp_select
 from resource_management.libraries.functions import conf_select
 from resource_management.libraries.functions import get_kinit_path
-
+from resource_management.libraries.functions.get_not_managed_resources import get_not_managed_resources
 
 # server configurations
 config = Script.get_config()
@@ -282,5 +282,6 @@ HdfsResource = functools.partial(
   hadoop_conf_dir = hadoop_conf_dir,
   principal_name = hdfs_principal_name,
   hdfs_site = hdfs_site,
-  default_fs = default_fs
+  default_fs = default_fs,
+  immutable_paths = get_not_managed_resources()
 )
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/params_linux.py b/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/params_linux.py
index 9d4847f..3818e81 100644
--- a/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/params_linux.py
+++ b/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/params_linux.py
@@ -32,6 +32,7 @@ from status_params import *
 from resource_management.libraries.resources.hdfs_resource import HdfsResource
 from resource_management.libraries.functions import hdp_select
 from resource_management.libraries.functions import conf_select
+from resource_management.libraries.functions.get_not_managed_resources import get_not_managed_resources
 
 # server configurations
 config = Script.get_config()
@@ -331,5 +332,6 @@ HdfsResource = functools.partial(
   hadoop_conf_dir = hadoop_conf_dir,
   principal_name = hdfs_principal_name,
   hdfs_site = hdfs_site,
-  default_fs = default_fs
+  default_fs = default_fs,
+  immutable_paths = get_not_managed_resources()
 )
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/MAHOUT/1.0.0.2.3/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/MAHOUT/1.0.0.2.3/package/scripts/params.py b/ambari-server/src/main/resources/common-services/MAHOUT/1.0.0.2.3/package/scripts/params.py
index b261b23..555570e 100644
--- a/ambari-server/src/main/resources/common-services/MAHOUT/1.0.0.2.3/package/scripts/params.py
+++ b/ambari-server/src/main/resources/common-services/MAHOUT/1.0.0.2.3/package/scripts/params.py
@@ -25,6 +25,7 @@ from resource_management.libraries.functions import format
 from resource_management.libraries.functions.version import format_hdp_stack_version
 from resource_management.libraries.functions.default import default
 from resource_management.libraries.functions import get_kinit_path
+from resource_management.libraries.functions.get_not_managed_resources import get_not_managed_resources
 from resource_management.libraries.script.script import Script
 
 # server configurations
@@ -88,5 +89,6 @@ HdfsResource = functools.partial(
   hadoop_conf_dir = hadoop_conf_dir,
   principal_name = hdfs_principal_name,
   hdfs_site = hdfs_site,
-  default_fs = default_fs
+  default_fs = default_fs,
+  immutable_paths = get_not_managed_resources()
 )

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/OOZIE/4.0.0.2.0/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/OOZIE/4.0.0.2.0/package/scripts/params_linux.py b/ambari-server/src/main/resources/common-services/OOZIE/4.0.0.2.0/package/scripts/params_linux.py
index 0073865..269b602 100644
--- a/ambari-server/src/main/resources/common-services/OOZIE/4.0.0.2.0/package/scripts/params_linux.py
+++ b/ambari-server/src/main/resources/common-services/OOZIE/4.0.0.2.0/package/scripts/params_linux.py
@@ -27,6 +27,7 @@ from resource_management.libraries.functions.version import format_hdp_stack_ver
 from resource_management.libraries.functions.default import default
 from resource_management.libraries.functions import get_kinit_path
 from resource_management.libraries.functions import get_port_from_url
+from resource_management.libraries.functions.get_not_managed_resources import get_not_managed_resources
 from resource_management.libraries.script.script import Script
 
 from resource_management.libraries.functions.get_lzo_packages import get_lzo_packages
@@ -273,7 +274,8 @@ HdfsResource = functools.partial(
   hadoop_conf_dir = hadoop_conf_dir,
   principal_name = hdfs_principal_name,
   hdfs_site = hdfs_site,
-  default_fs = default_fs
+  default_fs = default_fs,
+  immutable_paths = get_not_managed_resources()
 )
 
 is_webhdfs_enabled = config['configurations']['hdfs-site']['dfs.webhdfs.enabled']

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/params_linux.py b/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/params_linux.py
index 17998c6..aae6a3b 100644
--- a/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/params_linux.py
+++ b/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/params_linux.py
@@ -26,6 +26,7 @@ from resource_management.libraries.functions import hdp_select
 from resource_management.libraries.functions.version import format_hdp_stack_version
 from resource_management.libraries.functions.default import default
 from resource_management.libraries.functions import get_kinit_path
+from resource_management.libraries.functions.get_not_managed_resources import get_not_managed_resources
 
 # server configurations
 config = Script.get_config()
@@ -89,6 +90,7 @@ HdfsResource = functools.partial(
   hadoop_conf_dir = hadoop_conf_dir,
   principal_name = hdfs_principal_name,
   hdfs_site = hdfs_site,
-  default_fs = default_fs
+  default_fs = default_fs,
+  immutable_paths = get_not_managed_resources()
  )
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/PXF/3.0.0/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/PXF/3.0.0/package/scripts/params.py b/ambari-server/src/main/resources/common-services/PXF/3.0.0/package/scripts/params.py
index 1dbed45..0bae970 100644
--- a/ambari-server/src/main/resources/common-services/PXF/3.0.0/package/scripts/params.py
+++ b/ambari-server/src/main/resources/common-services/PXF/3.0.0/package/scripts/params.py
@@ -23,6 +23,7 @@ from resource_management.libraries.functions.default import default
 from resource_management.libraries.functions import get_kinit_path
 from resource_management.libraries.resources.hdfs_resource import HdfsResource
 from resource_management.libraries.functions.namenode_ha_utils import get_active_namenode
+from resource_management.libraries.functions.get_not_managed_resources import get_not_managed_resources
 
 config = Script.get_config()
 
@@ -83,5 +84,6 @@ HdfsResource = functools.partial(HdfsResource,
     kinit_path_local=kinit_path_local,
     principal_name=hdfs_principal_name,
     hdfs_site=hdfs_site,
-    default_fs=default_fs)
+    default_fs=default_fs,
+    immutable_paths = get_not_managed_resources())
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/SLIDER/0.60.0.2.2/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/SLIDER/0.60.0.2.2/package/scripts/params_linux.py b/ambari-server/src/main/resources/common-services/SLIDER/0.60.0.2.2/package/scripts/params_linux.py
index 132ff77..52ee1fc 100644
--- a/ambari-server/src/main/resources/common-services/SLIDER/0.60.0.2.2/package/scripts/params_linux.py
+++ b/ambari-server/src/main/resources/common-services/SLIDER/0.60.0.2.2/package/scripts/params_linux.py
@@ -23,6 +23,7 @@ from resource_management.libraries.script.script import Script
 from resource_management.libraries.functions.format import format
 from resource_management.libraries.functions.default import default
 from resource_management.libraries.functions import get_kinit_path
+from resource_management.libraries.functions.get_not_managed_resources import get_not_managed_resources
 
 # server configurations
 config = Script.get_config()
@@ -72,5 +73,6 @@ HdfsResource = functools.partial(
   hadoop_conf_dir = hadoop_conf_dir,
   principal_name = hdfs_principal_name,
   hdfs_site = hdfs_site,
-  default_fs = default_fs
+  default_fs = default_fs,
+  immutable_paths = get_not_managed_resources()
 )

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/SPARK/1.2.0.2.2/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/SPARK/1.2.0.2.2/package/scripts/params.py b/ambari-server/src/main/resources/common-services/SPARK/1.2.0.2.2/package/scripts/params.py
index 52c78c5..19666d8 100644
--- a/ambari-server/src/main/resources/common-services/SPARK/1.2.0.2.2/package/scripts/params.py
+++ b/ambari-server/src/main/resources/common-services/SPARK/1.2.0.2.2/package/scripts/params.py
@@ -31,6 +31,7 @@ from resource_management.libraries.functions.get_hdp_version import get_hdp_vers
 from resource_management.libraries.functions.version import format_hdp_stack_version
 from resource_management.libraries.functions.default import default
 from resource_management.libraries.functions import get_kinit_path
+from resource_management.libraries.functions.get_not_managed_resources import get_not_managed_resources
 
 from resource_management.libraries.script.script import Script
 
@@ -189,5 +190,6 @@ HdfsResource = functools.partial(
   hadoop_conf_dir = hadoop_conf_dir,
   principal_name = hdfs_principal_name,
   hdfs_site = hdfs_site,
-  default_fs = default_fs
+  default_fs = default_fs,
+  immutable_paths = get_not_managed_resources()
  )

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/params_linux.py b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/params_linux.py
index 94988ce..a022282 100644
--- a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/params_linux.py
+++ b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/params_linux.py
@@ -33,6 +33,7 @@ from resource_management.libraries.resources.hdfs_resource import HdfsResource
 from resource_management.libraries.functions import hdp_select
 from resource_management.libraries.functions import conf_select
 from resource_management.libraries.functions import get_kinit_path
+from resource_management.libraries.functions.get_not_managed_resources import get_not_managed_resources
 
 # server configurations
 config = Script.get_config()
@@ -303,5 +304,6 @@ HdfsResource = functools.partial(
   hadoop_conf_dir = hadoop_conf_dir,
   principal_name = hdfs_principal_name,
   hdfs_site = hdfs_site,
-  default_fs = default_fs
+  default_fs = default_fs,
+  immutable_paths = get_not_managed_resources()
 )
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/params_linux.py b/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/params_linux.py
index f0ba617..399b870 100644
--- a/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/params_linux.py
+++ b/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/params_linux.py
@@ -25,6 +25,7 @@ from resource_management.libraries.functions import hdp_select
 from resource_management.libraries.functions.version import format_hdp_stack_version
 from resource_management.libraries.functions.default import default
 from resource_management.libraries.functions import get_kinit_path
+from resource_management.libraries.functions.get_not_managed_resources import get_not_managed_resources
 from resource_management.libraries.script.script import Script
 
 # server configurations
@@ -93,7 +94,8 @@ HdfsResource = functools.partial(
   hadoop_conf_dir = hadoop_conf_dir,
   principal_name = hdfs_principal_name,
   hdfs_site = hdfs_site,
-  default_fs = default_fs
+  default_fs = default_fs,
+  immutable_paths = get_not_managed_resources()
 )
 
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/configuration-mapred/mapred-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/configuration-mapred/mapred-site.xml b/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/configuration-mapred/mapred-site.xml
index f30b807..3e78c37 100644
--- a/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/configuration-mapred/mapred-site.xml
+++ b/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/configuration-mapred/mapred-site.xml
@@ -252,6 +252,7 @@
     <description>
       Directory where history files are managed by the MR JobHistory Server.
     </description>
+    <property-type>NOT_MANAGED_HDFS_PATH</property-type>
   </property>
 
   <property>       

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/configuration/yarn-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/configuration/yarn-site.xml b/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/configuration/yarn-site.xml
index 59d4964..8b00139 100644
--- a/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/configuration/yarn-site.xml
+++ b/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/configuration/yarn-site.xml
@@ -292,6 +292,7 @@
     <name>yarn.nodemanager.remote-app-log-dir</name>
     <value>/app-logs</value>
     <description>Location to aggregate logs to. </description>
+    <property-type>NOT_MANAGED_HDFS_PATH</property-type>
   </property>
 
   <property>

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/scripts/params_linux.py b/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/scripts/params_linux.py
index 8d7427d..c3a499e 100644
--- a/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/scripts/params_linux.py
+++ b/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/scripts/params_linux.py
@@ -26,6 +26,7 @@ from resource_management.libraries.functions import conf_select
 from resource_management.libraries.functions import hdp_select
 from resource_management.libraries.functions import format
 from resource_management.libraries.functions import get_kinit_path
+from resource_management.libraries.functions.get_not_managed_resources import get_not_managed_resources
 from resource_management.libraries.functions.version import format_hdp_stack_version
 from resource_management.libraries.functions.default import default
 from resource_management.libraries import functions
@@ -278,7 +279,8 @@ HdfsResource = functools.partial(
   hadoop_conf_dir = hadoop_conf_dir,
   principal_name = hdfs_principal_name,
   hdfs_site = hdfs_site,
-  default_fs = default_fs
+  default_fs = default_fs,
+  immutable_paths = get_not_managed_resources()
  )
 update_exclude_file_only = default("/commandParams/update_exclude_file_only",False)
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/stacks/HDP/2.0.6/configuration/cluster-env.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/configuration/cluster-env.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.6/configuration/cluster-env.xml
index a939302..7a4aa35 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/configuration/cluster-env.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/configuration/cluster-env.xml
@@ -103,4 +103,13 @@ gpgcheck=0</value>
         </value-attributes>
     </property>
 
+    <property>
+        <name>managed_hdfs_resource_property_names</name>
+        <value></value>
+        <description>Comma separated list of property names with HDFS resource paths.
+        Resource from this list will be managed even if it is marked as not managed in the stack</description>
+        <value-attributes>
+            <overridable>false</overridable>
+        </value-attributes>
+    </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/stacks/HDP/2.0.6/hooks/before-START/files/fast-hdfs-resource.jar
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/hooks/before-START/files/fast-hdfs-resource.jar b/ambari-server/src/main/resources/stacks/HDP/2.0.6/hooks/before-START/files/fast-hdfs-resource.jar
index 4544f6b..60de52b 100644
Binary files a/ambari-server/src/main/resources/stacks/HDP/2.0.6/hooks/before-START/files/fast-hdfs-resource.jar and b/ambari-server/src/main/resources/stacks/HDP/2.0.6/hooks/before-START/files/fast-hdfs-resource.jar differ

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/stacks/HDP/2.1/services/HIVE/configuration/hive-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.1/services/HIVE/configuration/hive-site.xml b/ambari-server/src/main/resources/stacks/HDP/2.1/services/HIVE/configuration/hive-site.xml
index b444c11..efaefc9 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.1/services/HIVE/configuration/hive-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.1/services/HIVE/configuration/hive-site.xml
@@ -88,6 +88,7 @@ limitations under the License.
     <name>hive.metastore.warehouse.dir</name>
     <value>/apps/hive/warehouse</value>
     <description>location of default database for the warehouse</description>
+    <property-type>NOT_MANAGED_HDFS_PATH</property-type>
   </property>
 
   <property>

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/stacks/HDP/2.2/services/HIVE/configuration/hive-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.2/services/HIVE/configuration/hive-site.xml b/ambari-server/src/main/resources/stacks/HDP/2.2/services/HIVE/configuration/hive-site.xml
index 64094c0..fbc575a 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.2/services/HIVE/configuration/hive-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.2/services/HIVE/configuration/hive-site.xml
@@ -342,6 +342,7 @@ limitations under the License.
     <name>hive.metastore.warehouse.dir</name>
     <value>/apps/hive/warehouse</value>
     <description>location of default database for the warehouse</description>
+    <property-type>NOT_MANAGED_HDFS_PATH</property-type>
   </property>
 
   <property>

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/main/resources/stacks/HDPWIN/2.2/services/HIVE/configuration/hive-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.2/services/HIVE/configuration/hive-site.xml b/ambari-server/src/main/resources/stacks/HDPWIN/2.2/services/HIVE/configuration/hive-site.xml
index 3059db6..0818ed5 100644
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.2/services/HIVE/configuration/hive-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDPWIN/2.2/services/HIVE/configuration/hive-site.xml
@@ -310,6 +310,7 @@ limitations under the License.
     <name>hive.metastore.warehouse.dir</name>
     <value>/apps/hive/warehouse</value>
     <description>location of default database for the warehouse</description>
+    <property-type>NOT_MANAGED_HDFS_PATH</property-type>
   </property>
 
   <property>

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariManagementControllerImplTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariManagementControllerImplTest.java b/ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariManagementControllerImplTest.java
index 9c2a8bf..bf39c3a 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariManagementControllerImplTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/controller/AmbariManagementControllerImplTest.java
@@ -53,6 +53,7 @@ import org.apache.ambari.server.security.ldap.LdapBatchDto;
 import org.apache.ambari.server.state.Cluster;
 import org.apache.ambari.server.state.Clusters;
 import org.apache.ambari.server.state.ComponentInfo;
+import org.apache.ambari.server.state.ConfigHelper;
 import org.apache.ambari.server.state.ConfigImpl;
 import org.apache.ambari.server.state.Host;
 import org.apache.ambari.server.state.MaintenanceState;
@@ -75,6 +76,7 @@ import javax.persistence.RollbackException;
 import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -87,6 +89,7 @@ import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.HOST_SYS_
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.JAVA_VERSION;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.STACK_NAME;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.STACK_VERSION;
+import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.NOT_MANAGED_HDFS_PATH_LIST;
 import static org.easymock.EasyMock.anyBoolean;
 import static org.easymock.EasyMock.anyObject;
 import static org.easymock.EasyMock.capture;
@@ -1958,6 +1961,8 @@ public class AmbariManagementControllerImplTest {
     String JCE_NAME = "jceName";
     String OJDBC_JAR_NAME = "OjdbcJarName";
     String SERVER_DB_NAME = "ServerDBName";
+    Set<String> notManagedHdfsPathSet = new HashSet<>(Arrays.asList("/tmp", "/apps/falcon"));
+    Gson gson = new Gson();
 
     ActionManager manager = createNiceMock(ActionManager.class);
     StackId stackId = createNiceMock(StackId.class);
@@ -1967,6 +1972,7 @@ public class AmbariManagementControllerImplTest {
     ClusterVersionDAO clusterVersionDAO = createNiceMock(ClusterVersionDAO.class);
     ClusterVersionEntity clusterVersionEntity = createNiceMock(ClusterVersionEntity.class);
     RepositoryVersionEntity repositoryVersionEntity = createNiceMock(RepositoryVersionEntity.class);
+    ConfigHelper configHelper = createNiceMock(ConfigHelper.class);
 
     expect(cluster.getClusterName()).andReturn(clusterName);
     expect(cluster.getDesiredStackVersion()).andReturn(stackId);
@@ -1983,8 +1989,11 @@ public class AmbariManagementControllerImplTest {
     expect(clusterVersionDAO.findByClusterAndStateCurrent(clusterName)).andReturn(clusterVersionEntity).anyTimes();
     expect(clusterVersionEntity.getRepositoryVersion()).andReturn(repositoryVersionEntity).anyTimes();
     expect(repositoryVersionEntity.getVersion()).andReturn("1234").anyTimes();
+    expect(configHelper.getPropertyValuesWithPropertyType(stackId, PropertyInfo.PropertyType.NOT_MANAGED_HDFS_PATH,
+        cluster)).andReturn(notManagedHdfsPathSet);
 
-    replay(manager, clusters, cluster, injector, stackId, configuration, clusterVersionDAO, clusterVersionEntity, repositoryVersionEntity);
+    replay(manager, clusters, cluster, injector, stackId, configuration, clusterVersionDAO, clusterVersionEntity,
+        repositoryVersionEntity, configHelper);
 
     AmbariManagementControllerImpl ambariManagementControllerImpl =
             createMockBuilder(AmbariManagementControllerImpl.class)
@@ -2015,14 +2024,24 @@ public class AmbariManagementControllerImplTest {
     f.setAccessible(true);
     f.set(helper, clusterVersionDAO);
 
+    f = helperClass.getDeclaredField("configHelper");
+    f.setAccessible(true);
+    f.set(helper, configHelper);
+
+    f = helperClass.getDeclaredField("gson");
+    f.setAccessible(true);
+    f.set(helper, gson);
+
     Map<String, String> defaultHostParams = helper.createDefaultHostParams(cluster);
 
-    assertEquals(defaultHostParams.size(), 15);
+    assertEquals(defaultHostParams.size(), 16);
     assertEquals(defaultHostParams.get(DB_DRIVER_FILENAME), MYSQL_JAR);
     assertEquals(defaultHostParams.get(STACK_NAME), SOME_STACK_NAME);
     assertEquals(defaultHostParams.get(STACK_VERSION), SOME_STACK_VERSION);
     assertEquals("true", defaultHostParams.get(HOST_SYS_PREPPED));
     assertEquals("8", defaultHostParams.get(JAVA_VERSION));
+    assertNotNull(defaultHostParams.get(NOT_MANAGED_HDFS_PATH_LIST));
+    assertTrue(defaultHostParams.get(NOT_MANAGED_HDFS_PATH_LIST).contains("/tmp"));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/test/python/stacks/2.0.6/AMBARI_METRICS/test_metrics_collector.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/python/stacks/2.0.6/AMBARI_METRICS/test_metrics_collector.py b/ambari-server/src/test/python/stacks/2.0.6/AMBARI_METRICS/test_metrics_collector.py
index 9670b1c..e461421 100644
--- a/ambari-server/src/test/python/stacks/2.0.6/AMBARI_METRICS/test_metrics_collector.py
+++ b/ambari-server/src/test/python/stacks/2.0.6/AMBARI_METRICS/test_metrics_collector.py
@@ -320,6 +320,7 @@ class TestMetricsCollector(RMFTestCase):
     if name == 'master':
       if distributed:
         self.assertResourceCalled('HdfsResource', 'hdfs://localhost:8020/apps/hbase/data',
+                                  immutable_paths = ['/apps/hive/warehouse', '/apps/falcon', '/mr-history/done', '/app-logs', '/tmp'],
                                   security_enabled = False,
                                   hadoop_bin_dir = '/usr/bin',
                                   keytab = UnknownConfigurationMock(),
@@ -335,6 +336,7 @@ class TestMetricsCollector(RMFTestCase):
                                   default_fs='hdfs://c6401.ambari.apache.org:8020',
                                   )
         self.assertResourceCalled('HdfsResource', '/amshbase/staging',
+                                  immutable_paths = ['/apps/hive/warehouse', '/apps/falcon', '/mr-history/done', '/app-logs', '/tmp'],
                                   security_enabled = False,
                                   hadoop_bin_dir = '/usr/bin',
                                   keytab = UnknownConfigurationMock(),
@@ -350,6 +352,7 @@ class TestMetricsCollector(RMFTestCase):
                                   default_fs='hdfs://c6401.ambari.apache.org:8020',
                                   )
         self.assertResourceCalled('HdfsResource', None,
+                                  immutable_paths = ['/apps/hive/warehouse', '/apps/falcon', '/mr-history/done', '/app-logs', '/tmp'],
                                   security_enabled = False,
                                   hadoop_bin_dir = '/usr/bin',
                                   keytab = UnknownConfigurationMock(),

http://git-wip-us.apache.org/repos/asf/ambari/blob/23386d0d/ambari-server/src/test/python/stacks/2.0.6/HBASE/test_hbase_master.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/python/stacks/2.0.6/HBASE/test_hbase_master.py b/ambari-server/src/test/python/stacks/2.0.6/HBASE/test_hbase_master.py
index 6a4a2ef..1e9255c 100644
--- a/ambari-server/src/test/python/stacks/2.0.6/HBASE/test_hbase_master.py
+++ b/ambari-server/src/test/python/stacks/2.0.6/HBASE/test_hbase_master.py
@@ -329,6 +329,7 @@ class TestHBaseMaster(RMFTestCase):
     )
 
     self.assertResourceCalled('HdfsResource', 'hdfs://c6401.ambari.apache.org:8020/apps/hbase/data',
+        immutable_paths = ['/apps/hive/warehouse', '/apps/falcon', '/mr-history/done', '/app-logs', '/tmp'],
         security_enabled = False,
         hadoop_bin_dir = '/usr/bin',
         keytab = UnknownConfigurationMock(),
@@ -341,6 +342,7 @@ class TestHBaseMaster(RMFTestCase):
         action = ['create_on_execute'], hdfs_resource_ignore_file='/var/lib/ambari-agent/data/.hdfs_resource_ignore', hdfs_site=self.getConfig()['configurations']['hdfs-site'], principal_name=UnknownConfigurationMock(), default_fs='hdfs://c6401.ambari.apache.org:8020',
     )
     self.assertResourceCalled('HdfsResource', '/apps/hbase/staging',
+        immutable_paths = ['/apps/hive/warehouse', '/apps/falcon', '/mr-history/done', '/app-logs', '/tmp'],
         security_enabled = False,
         hadoop_conf_dir = '/etc/hadoop/conf',
         keytab = UnknownConfigurationMock(),
@@ -354,6 +356,7 @@ class TestHBaseMaster(RMFTestCase):
         mode = 0711,
     )
     self.assertResourceCalled('HdfsResource', None,
+        immutable_paths = ['/apps/hive/warehouse', '/apps/falcon', '/mr-history/done', '/app-logs', '/tmp'],
         security_enabled = False,
         hadoop_bin_dir = '/usr/bin',
         keytab = UnknownConfigurationMock(),
@@ -464,6 +467,7 @@ class TestHBaseMaster(RMFTestCase):
                               content='log4jproperties\nline2'
     )
     self.assertResourceCalled('HdfsResource', 'hdfs://c6401.ambari.apache.org:8020/apps/hbase/data',
+        immutable_paths = ['/apps/hive/warehouse', '/apps/falcon', '/mr-history/done', '/app-logs', '/tmp'],
         security_enabled = True,
         hadoop_bin_dir = '/usr/bin',
         keytab = '/etc/security/keytabs/hdfs.headless.keytab',
@@ -476,6 +480,7 @@ class TestHBaseMaster(RMFTestCase):
         action = ['create_on_execute'], hdfs_resource_ignore_file='/var/lib/ambari-agent/data/.hdfs_resource_ignore', hdfs_site=self.getConfig()['configurations']['hdfs-site'], principal_name='hdfs', default_fs='hdfs://c6401.ambari.apache.org:8020',
     )
     self.assertResourceCalled('HdfsResource', '/apps/hbase/staging',
+        immutable_paths = ['/apps/hive/warehouse', '/apps/falcon', '/mr-history/done', '/app-logs', '/tmp'],
         security_enabled = True,
         hadoop_conf_dir = '/etc/hadoop/conf',
         keytab = '/etc/security/keytabs/hdfs.headless.keytab',
@@ -489,6 +494,7 @@ class TestHBaseMaster(RMFTestCase):
         mode = 0711,
     )
     self.assertResourceCalled('HdfsResource', None,
+        immutable_paths = ['/apps/hive/warehouse', '/apps/falcon', '/mr-history/done', '/app-logs', '/tmp'],
         security_enabled = True,
         hadoop_bin_dir = '/usr/bin',
         keytab = '/etc/security/keytabs/hdfs.headless.keytab',
@@ -608,6 +614,7 @@ class TestHBaseMaster(RMFTestCase):
                               content='log4jproperties\nline2')
 
     self.assertResourceCalled('HdfsResource', 'hdfs://nn1/apps/hbase/data',
+        immutable_paths = ['/apps/hive/warehouse', '/apps/falcon', '/mr-history/done', '/app-logs', '/tmp'],
         security_enabled = False,
         hadoop_bin_dir = '/usr/hdp/current/hadoop-client/bin',
         keytab = UnknownConfigurationMock(),
@@ -622,6 +629,7 @@ class TestHBaseMaster(RMFTestCase):
         action = ['create_on_execute'], hdfs_resource_ignore_file='/var/lib/ambari-agent/data/.hdfs_resource_ignore',
     )
     self.assertResourceCalled('HdfsResource', '/apps/hbase/staging',
+        immutable_paths = ['/apps/hive/warehouse', '/apps/falcon', '/mr-history/done', '/app-logs', '/tmp'],
         security_enabled = False,
         hadoop_bin_dir = '/usr/hdp/current/hadoop-client/bin',
         keytab = UnknownConfigurationMock(),
@@ -637,6 +645,7 @@ class TestHBaseMaster(RMFTestCase):
         mode = 0711,
     )
     self.assertResourceCalled('HdfsResource', None,
+        immutable_paths = ['/apps/hive/warehouse', '/apps/falcon', '/mr-history/done', '/app-logs', '/tmp'],
         security_enabled = False,
         hadoop_bin_dir = '/usr/hdp/current/hadoop-client/bin',
         keytab = UnknownConfigurationMock(),