You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ji...@apache.org on 2016/12/07 21:10:01 UTC

[01/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe) [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/yarn-native-services 69283ee84 -> fc259d505 (forced update)


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/proto/SliderClusterMessages.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/proto/SliderClusterMessages.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/proto/SliderClusterMessages.proto
new file mode 100644
index 0000000..b8bdc59
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/proto/SliderClusterMessages.proto
@@ -0,0 +1,396 @@
+/**
+ * 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.
+ */
+
+option java_package = "org.apache.slider.api.proto";
+option java_outer_classname = "Messages";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+package org.apache.slider.api;
+
+//import "Security.proto";
+
+/*
+  Look at SliderClusterProtocol.proto to see how to build this
+*/
+
+message RoleInstanceState {
+  required string name =        1;
+  optional string role =        2;
+  required uint32 state =       4;
+  required uint32 exitCode =    5;
+  optional string command =     6;
+  optional string diagnostics = 7;
+  repeated string output =      8;
+  repeated string environment = 9;
+  required uint32 roleId =     10;
+  required bool released =     11;
+  required int64 createTime =  12;
+  required int64 startTime =   13;
+  required string host =       14;
+  required string hostURL =    15;
+  optional string appVersion = 16;
+}
+
+/**
+ * stop the cluster
+ */
+message StopClusterRequestProto {
+  /**
+  message to include
+  */
+  required string message = 1;
+}
+
+/**
+ * stop the cluster
+ */
+message StopClusterResponseProto {
+}
+
+/**
+ * upgrade the containers
+ */
+message UpgradeContainersRequestProto {
+  /**
+  message to include
+  */
+  required string message =     1;
+  repeated string container =   2;
+  repeated string component =   3;
+}
+
+/**
+ * upgrade the containers
+ */
+message UpgradeContainersResponseProto {
+}
+
+/**
+ * flex the cluster
+ */
+message FlexClusterRequestProto {
+  required string clusterSpec = 1;
+}
+
+
+/**
+ * flex the cluster
+ */
+message FlexClusterResponseProto {
+  required bool response = 1;
+}
+
+
+/**
+ * void request
+ */
+message GetJSONClusterStatusRequestProto {
+}
+
+/**
+ * response
+ */
+message GetJSONClusterStatusResponseProto {
+  required string clusterSpec = 1;
+}
+
+/**
+ * list the nodes in a role
+ */
+message ListNodeUUIDsByRoleRequestProto {
+  required string role = 1;
+}
+
+/**
+ * list the nodes in a role
+ */
+message ListNodeUUIDsByRoleResponseProto {
+  repeated string uuid = 1 ;
+}
+
+/**
+ * get a node
+ */
+message GetNodeRequestProto {
+  required string uuid = 1;
+}
+
+
+/**
+ * response on a node
+ */
+message GetNodeResponseProto {
+   required RoleInstanceState clusterNode = 1 ;
+}
+
+/**
+ * list the nodes for the UUDs
+ */
+message GetClusterNodesRequestProto {
+  repeated string uuid = 1 ;
+}
+
+/**
+ * list the nodes in a role
+ */
+message GetClusterNodesResponseProto {
+  repeated RoleInstanceState clusterNode = 1 ;
+}
+
+/**
+ * Echo
+ */
+message EchoRequestProto {
+  required string text = 1;
+}
+
+/**
+ * Echo reply
+ */
+message EchoResponseProto {
+  required string text = 1;
+}
+
+
+/**
+ * Kill a container
+ */
+message KillContainerRequestProto {
+  required string id = 1;
+}
+
+/**
+ * Kill reply
+ */
+message KillContainerResponseProto {
+  required bool success = 1;
+}
+
+/**
+ * AM suicide
+ */
+message AMSuicideRequestProto {
+  required string text =      1;
+  required int32 signal =     2;
+  required int32 delay =      3;
+}
+
+/**
+ * AM suicide reply. For this to be returned implies
+ * a failure of the AM to kill itself
+ */
+message AMSuicideResponseProto {
+
+}
+
+
+/**
+ * Ask for the instance definition details
+ */
+message GetInstanceDefinitionRequestProto {
+
+}
+
+/**
+ * Get the definition back as three separate JSON strings
+ */
+message GetInstanceDefinitionResponseProto {
+  required string internal =        1;
+  required string resources =       2;
+  required string application =     3;
+}
+
+
+  /* ************************************************************************
+  
+  REST model and operations.
+  Below here the operations and payloads designed to mimic
+  the REST API. That API is now the source of those
+  specificatations; this is simply a derivative.
+  
+  **************************************************************************/
+
+/**
+ * See org.apache.slider.api.types.ApplicationLivenessInformation
+ */
+message ApplicationLivenessInformationProto {
+  optional bool allRequestsSatisfied = 1;
+  optional int32 requestsOutstanding = 2;
+}
+
+/*
+ * see org.apache.slider.api.types.ComponentInformation
+ */
+message ComponentInformationProto {
+  optional string name =           1;
+  optional int32 priority =        2;
+  optional int32 desired =         3;
+  optional int32 actual =          4;
+  optional int32 releasing =       5;
+  optional int32 requested =       6;
+  optional int32 failed =          7;
+  optional int32 started =         8;
+  optional int32 startFailed =     9;
+  optional int32 completed =      10;
+  optional int32 totalRequested = 11;
+  optional string failureMessage =12;
+  optional int32 placementPolicy =13;
+  repeated string containers =    14;
+  optional int32 failedRecently = 15;
+  optional int32 nodeFailed =     16;
+  optional int32 preempted =      17;
+  optional int32 pendingAntiAffineRequestCount = 18;
+  optional bool isAARequestOutstanding = 19;
+}
+
+/*
+ * see org.apache.slider.api.types.ContainerInformation
+ */
+message ContainerInformationProto {
+  optional string containerId =   1;
+  optional string component =     2;
+  optional bool released =        3;
+  optional int32 state =          4;
+  optional int32 exitCode =       5;
+  optional string diagnostics =   6;
+  optional int64 createTime =     7;
+  optional int64 startTime =      8;
+  repeated string output =        9;
+  optional string host =         10;
+  optional string hostURL =      11;
+  optional string placement =    12;
+  optional string appVersion =   13;
+}
+
+
+/*
+ * see org.apache.slider.api.types.PingInformation
+ */
+message PingInformationProto {
+  optional string text = 1;
+  optional string verb = 2;
+  optional string body = 3;
+  optional int64 time =  4;
+}
+
+message NodeEntryInformationProto {
+  required int32 priority =      1;
+  required int32 requested =     2;
+  required int32 starting =      3;
+  required int32 startFailed =   4;
+  required int32 failed =        5;
+  required int32 failedRecently= 6;
+  required int32 preempted =     7;
+  required int32 live =          8;
+  required int32 releasing =     9;
+  required int64 lastUsed =     10;
+  required string name =        11;
+}
+
+message NodeInformationProto {
+  required string hostname =    1;
+  required string state =       2;
+  required string httpAddress = 3;
+  required string rackName =    4;
+  required string labels =      5;
+  required string healthReport= 6;
+  required int64 lastUpdated =  7;
+  repeated NodeEntryInformationProto entries = 8;
+}
+
+message GetModelRequestProto {
+}
+
+message GetModelDesiredRequestProto {
+}
+
+message GetModelDesiredAppconfRequestProto {
+}
+
+message GetModelDesiredResourcesRequestProto {
+}
+
+message GetModelResolvedAppconfRequestProto {
+}
+
+message GetModelResolvedResourcesRequestProto {
+}
+
+message GetModelLiveResourcesRequestProto {
+}
+
+message GetLiveContainersRequestProto {
+}
+
+message GetLiveContainersResponseProto {
+  repeated string names = 1;
+  repeated ContainerInformationProto containers = 2;
+}
+
+message GetLiveContainerRequestProto {
+  required string containerId = 1;
+}
+
+
+message GetLiveComponentsRequestProto {
+}
+
+message GetLiveComponentsResponseProto {
+
+  repeated string names = 1;
+  repeated ComponentInformationProto components = 2;
+}
+
+message GetLiveComponentRequestProto {
+  required string name = 1;
+}
+
+message GetApplicationLivenessRequestProto {
+}
+
+message EmptyPayloadProto {
+}
+
+/**
+  Generic JSON, often containing data structures serialized as a string
+*/
+message WrappedJsonProto {
+  required string json = 1;
+}
+
+message GetCertificateStoreRequestProto {
+  optional string hostname =    1;
+  required string requesterId = 2;
+  required string password =    3;
+  required string type =        4;
+}
+
+message GetCertificateStoreResponseProto {
+  required bytes store = 1;
+}
+
+message GetLiveNodesRequestProto {
+}
+
+message GetLiveNodesResponseProto {
+  repeated NodeInformationProto nodes = 1;
+}
+
+message GetLiveNodeRequestProto {
+  required string name = 1;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/proto/SliderClusterProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/proto/SliderClusterProtocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/proto/SliderClusterProtocol.proto
new file mode 100644
index 0000000..4221b1d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/proto/SliderClusterProtocol.proto
@@ -0,0 +1,184 @@
+/*
+ * 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.
+ */
+
+option java_package = "org.apache.slider.api.proto";
+option java_outer_classname = "SliderClusterAPI";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+package org.apache.slider.api;
+
+/*
+
+Compiling
+
+Maven: How to do it as part of the build
+ mvn install -DskipTests -Pcompile-protobuf
+
+How to do it so as to get error messages
+
+protoc --java_out=src/main/java \
+ -Isrc/main/proto src/main/proto/SliderClusterMessages.proto \
+  src/main/proto/SliderClusterProtocol.proto
+
+Once happy: commit the changes
+
+*/
+
+//import "Security.proto";
+import "SliderClusterMessages.proto";
+
+
+/**
+ * Protocol used from between Slider Client and AM
+ */
+service SliderClusterProtocolPB {
+
+  /**
+   * Stop the cluster
+   */
+
+  rpc stopCluster(StopClusterRequestProto) 
+    returns(StopClusterResponseProto);
+    
+  /**
+   * Upgrade containers 
+   */
+  rpc upgradeContainers(UpgradeContainersRequestProto) 
+    returns(UpgradeContainersResponseProto);
+
+  /**
+   * Flex the cluster. 
+   */
+  rpc flexCluster(FlexClusterRequestProto) 
+    returns(FlexClusterResponseProto);
+
+  /**
+   * Get the current cluster status
+   */
+  rpc getJSONClusterStatus(GetJSONClusterStatusRequestProto)
+    returns(GetJSONClusterStatusResponseProto);
+      
+  /**
+   * Get the instance definition
+   */
+  rpc getInstanceDefinition(GetInstanceDefinitionRequestProto)
+   returns(GetInstanceDefinitionResponseProto);
+      
+  /**
+   * List all running nodes in a role
+   */
+  rpc listNodeUUIDsByRole(ListNodeUUIDsByRoleRequestProto)
+    returns(ListNodeUUIDsByRoleResponseProto);
+
+  /**
+   * Get the details on a node
+   */
+  rpc getNode(GetNodeRequestProto)
+    returns(GetNodeResponseProto);
+
+  /**
+   * Get the 
+   * details on a list of nodes.
+   * Unknown nodes are not returned
+   * <i>Important: the order of the results are undefined</i>
+   */
+  rpc getClusterNodes(GetClusterNodesRequestProto)
+    returns(GetClusterNodesResponseProto);
+    
+   /**
+    * echo some text
+    */
+   rpc echo(EchoRequestProto)
+     returns(EchoResponseProto); 
+
+   /**
+    * kill a container
+    */
+   rpc killContainer(KillContainerRequestProto)
+     returns(KillContainerResponseProto);
+      
+   /**
+    * kill the AM
+    */
+   rpc amSuicide(AMSuicideRequestProto)
+     returns(AMSuicideResponseProto);
+
+  /* ************************************************************************
+  
+  REST model and operations.
+  Below here the operations and payloads designed to mimic
+  the REST API. That API is now the source of those
+  specificatations; this is simply a derivative.
+  
+  **************************************************************************/
+
+  rpc getLivenessInformation(GetApplicationLivenessRequestProto) 
+    returns(ApplicationLivenessInformationProto);
+
+  rpc getLiveContainers(GetLiveContainersRequestProto) 
+    returns(GetLiveContainersResponseProto);
+
+  rpc getLiveContainer(GetLiveContainerRequestProto) 
+    returns(ContainerInformationProto);
+
+  rpc getLiveComponents(GetLiveComponentsRequestProto) 
+    returns(GetLiveComponentsResponseProto);
+
+  rpc getLiveComponent(GetLiveComponentRequestProto) 
+    returns(ComponentInformationProto);
+
+  rpc getLiveNodes(GetLiveNodesRequestProto)
+    returns(GetLiveNodesResponseProto);
+
+  rpc getLiveNode(GetLiveNodeRequestProto)
+    returns(NodeInformationProto);
+  
+// AggregateConf getModelDesired()
+  rpc getModelDesired(EmptyPayloadProto) 
+    returns(WrappedJsonProto);
+
+  // ConfTree getModelDesiredAppconf
+  rpc getModelDesiredAppconf(EmptyPayloadProto) 
+    returns(WrappedJsonProto);
+
+    // ConfTree getModelDesiredResources
+  rpc getModelDesiredResources(EmptyPayloadProto) 
+    returns(WrappedJsonProto);
+  
+// AggregateConf getModelResolved()
+  rpc getModelResolved(EmptyPayloadProto) 
+    returns(WrappedJsonProto);
+
+  // ConfTree getModelResolvedAppconf
+  rpc getModelResolvedAppconf(EmptyPayloadProto) 
+    returns(WrappedJsonProto);
+
+    // ConfTree getModelResolvedResources
+  rpc getModelResolvedResources(EmptyPayloadProto) 
+    returns(WrappedJsonProto);
+
+    // ConfTree getLiveResources
+  rpc getLiveResources(EmptyPayloadProto) 
+    returns(WrappedJsonProto);
+
+  rpc getClientCertificateStore(GetCertificateStoreRequestProto)
+    returns(GetCertificateStoreResponseProto);
+
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo
new file mode 100644
index 0000000..9e67c15
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo
@@ -0,0 +1,15 @@
+# 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.
+org.apache.slider.server.appmaster.rpc.SliderRPCSecurityInfo

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/log4j.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/log4j.properties
new file mode 100644
index 0000000..65a7ad0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/log4j.properties
@@ -0,0 +1,52 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#  
+#       http://www.apache.org/licenses/LICENSE-2.0
+#  
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+#
+
+# This is a log4j config for slider
+
+log4j.rootLogger=INFO,stdout
+log4j.threshhold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{3} (%F:%M(%L)) - %m%n
+
+log4j.appender.subprocess=org.apache.log4j.ConsoleAppender
+log4j.appender.subprocess.layout=org.apache.log4j.PatternLayout
+log4j.appender.subprocess.layout.ConversionPattern=%c{1}: %m%n
+
+
+#at debug this provides details on what is going on
+log4j.logger.org.apache.slider=DEBUG
+#log4j.logger.org.apache.slider.exec.RunLongLivedApp=ERROR
+
+log4j.logger.org.apache.hadoop.security=DEBUG
+log4j.logger.org.apache.hadoop.yarn.service.launcher=DEBUG
+log4j.logger.org.apache.hadoop.yarn.service=DEBUG
+log4j.logger.org.apache.hadoop.yarn.client=DEBUG
+#crank back on some noise
+log4j.logger.org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor=WARN
+log4j.logger.org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdaterImpl=WARN
+log4j.logger.org.apache.hadoop.yarn.client.RMProxy=WARN
+
+# for test runs we don't care about native code
+log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR
+# HDFS is noise on tets
+log4j.logger.org.apache.hadoop.hdfs.server.datanode=WARN
+log4j.logger.org.apache.hadoop.hdfs.server.namenode=WARN
+log4j.logger.org.apache.hadoop.hdfs.server.blockmanagement=WARN
+log4j.logger.org.apache.hadoop.hdfs=WARN
+
+log4j.logger.org.apache.zookeeper=WARN

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/conf/agent.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/conf/agent.txt b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/conf/agent.txt
new file mode 100644
index 0000000..79c1972
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/conf/agent.txt
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+ 
+This is the conf directory for the python agent
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/conf/command.json
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/conf/command.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/conf/command.json
new file mode 100644
index 0000000..197a046
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/conf/command.json
@@ -0,0 +1,168 @@
+{
+  "roleCommand": "START",
+  "clusterName": "c1",
+  "hostname": "c6402.ambari.apache.org",
+  "hostLevelParams": {
+    "java_home": "/usr/jdk64/jdk1.7.0_45"
+  },
+  "commandType": "EXECUTION_COMMAND",
+  "roleParams": {},
+  "serviceName": "HBASE",
+  "role": "HBASE_MASTER",
+  "commandParams": {},
+  "taskId": 24,
+  "public_hostname": "c6402.ambari.apache.org",
+  "configurations": {
+    "hbase-log4j": {
+      "log4j.threshold": "ALL",
+      "log4j.rootLogger": "${hbase.root.logger}",
+      "log4j.logger.org.apache.zookeeper": "INFO",
+      "log4j.logger.org.apache.hadoop.hbase": "DEBUG",
+      "log4j.logger.org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher": "INFO",
+      "log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKUtil": "INFO",
+      "log4j.category.SecurityLogger": "${hbase.security.logger}",
+      "log4j.appender.console": "org.apache.log4j.ConsoleAppender",
+      "log4j.appender.console.target": "System.err",
+      "log4j.appender.console.layout": "org.apache.log4j.PatternLayout",
+      "log4j.appender.console.layout.ConversionPattern": "%d{ISO8601} %-5p [%t] %c{2}: %m%n",
+      "log4j.appender.RFAS": "org.apache.log4j.RollingFileAppender",
+      "log4j.appender.RFAS.layout": "org.apache.log4j.PatternLayout",
+      "log4j.appender.RFAS.layout.ConversionPattern": "%d{ISO8601} %p %c: %m%n",
+      "log4j.appender.RFAS.MaxFileSize": "${hbase.security.log.maxfilesize}",
+      "log4j.appender.RFAS.MaxBackupIndex": "${hbase.security.log.maxbackupindex}",
+      "log4j.appender.RFAS.File": "${hbase.log.dir}/${hbase.security.log.file}",
+      "log4j.appender.RFA": "org.apache.log4j.RollingFileAppender",
+      "log4j.appender.RFA.layout": "org.apache.log4j.PatternLayout",
+      "log4j.appender.RFA.layout.ConversionPattern": "%d{ISO8601} %-5p [%t] %c{2}: %m%n",
+      "log4j.appender.RFA.MaxFileSize": "${hbase.log.maxfilesize}",
+      "log4j.appender.RFA.MaxBackupIndex": "${hbase.log.maxbackupindex}",
+      "log4j.appender.RFA.File": "${hbase.log.dir}/${hbase.log.file}",
+      "log4j.appender.NullAppender": "org.apache.log4j.varia.NullAppender",
+      "log4j.appender.DRFA": "org.apache.log4j.DailyRollingFileAppender",
+      "log4j.appender.DRFA.layout": "org.apache.log4j.PatternLayout",
+      "log4j.appender.DRFA.layout.ConversionPattern": "%d{ISO8601} %-5p [%t] %c{2}: %m%n",
+      "log4j.appender.DRFA.File": "${hbase.log.dir}/${hbase.log.file}",
+      "log4j.appender.DRFA.DatePattern": ".yyyy-MM-dd",
+      "log4j.additivity.SecurityLogger": "false",
+      "hbase.security.logger": "INFO,console",
+      "hbase.security.log.maxfilesize": "256MB",
+      "hbase.security.log.maxbackupindex": "20",
+      "hbase.security.log.file": "SecurityAuth.audit",
+      "hbase.root.logger": "INFO,console",
+      "hbase.log.maxfilesize": "256MB",
+      "hbase.log.maxbackupindex": "20",
+      "hbase.log.file": "hbase.log",
+      "hbase.log.dir": "."
+    },
+    "global": {
+      "hbase_root": "/share/hbase/hbase-0.96.1-hadoop2",
+      "hbase_pid_dir": "/var/run/hbase",
+      "proxyuser_group": "users",
+      "syncLimit": "5",
+      "hbase_regionserver_heapsize": "1024m",
+      "rca_enabled": "false",
+      "tickTime": "2000",
+      "hbase_master_heapsize": "1024m",
+      "initLimit": "10",
+      "user_group": "hadoop",
+      "hbase_user": "hbase",
+      "hbase_log_dir": "/var/log/hbase"
+    },
+    "hdfs-site": {
+      "dfs.namenode.checkpoint.period": "21600",
+      "dfs.namenode.avoid.write.stale.datanode": "true",
+      "dfs.namenode.checkpoint.txns": "1000000",
+      "dfs.block.access.token.enable": "true",
+      "dfs.support.append": "true",
+      "dfs.datanode.address": "0.0.0.0:${ambari.dfs.datanode.port}",
+      "dfs.cluster.administrators": " hdfs",
+      "dfs.replication": "3",
+      "ambari.dfs.datanode.http.port": "50075",
+      "dfs.datanode.balance.bandwidthPerSec": "6250000",
+      "dfs.namenode.safemode.threshold-pct": "1.0f",
+      "dfs.namenode.checkpoint.edits.dir": "${dfs.namenode.checkpoint.dir}",
+      "dfs.permissions.enabled": "true",
+      "dfs.client.read.shortcircuit": "true",
+      "dfs.namenode.https-address": "c6402.ambari.apache.org:50470",
+      "dfs.journalnode.edits.dir": "/grid/0/hdfs/journal",
+      "dfs.blocksize": "134217728",
+      "dfs.datanode.max.transfer.threads": "1024",
+      "dfs.datanode.du.reserved": "1073741824",
+      "dfs.webhdfs.enabled": "true",
+      "dfs.namenode.handler.count": "100",
+      "dfs.namenode.checkpoint.dir": "/hadoop/hdfs/namesecondary",
+      "fs.permissions.umask-mode": "022",
+      "dfs.datanode.http.address": "0.0.0.0:${ambari.dfs.datanode.http.port}",
+      "dfs.datanode.ipc.address": "0.0.0.0:8010",
+      "dfs.datanode.data.dir": "/hadoop/hdfs/data",
+      "dfs.namenode.http-address": "c6402.ambari.apache.org:50070",
+      "dfs.blockreport.initialDelay": "120",
+      "dfs.datanode.failed.volumes.tolerated": "0",
+      "dfs.namenode.accesstime.precision": "0",
+      "ambari.dfs.datanode.port": "50010",
+      "dfs.namenode.avoid.read.stale.datanode": "true",
+      "dfs.namenode.secondary.http-address": "c6402.ambari.apache.org:50090",
+      "dfs.namenode.stale.datanode.interval": "30000",
+      "dfs.heartbeat.interval": "3",
+      "dfs.client.read.shortcircuit.streams.cache.size": "4096",
+      "dfs.permissions.superusergroup": "hdfs",
+      "dfs.https.port": "50470",
+      "dfs.journalnode.http-address": "0.0.0.0:8480",
+      "dfs.domain.socket.path": "/var/lib/hadoop-hdfs/dn_socket",
+      "dfs.namenode.write.stale.datanode.ratio": "1.0f",
+      "dfs.hosts.exclude": "/etc/hadoop/conf/dfs.exclude",
+      "dfs.datanode.data.dir.perm": "750",
+      "dfs.namenode.name.dir.restore": "true",
+      "dfs.replication.max": "50",
+      "dfs.namenode.name.dir": "/hadoop/hdfs/namenode"
+    },
+    "hbase-site": {
+      "hbase.hstore.flush.retries.number": "120",
+      "hbase.client.keyvalue.maxsize": "10485760",
+      "hbase.hstore.compactionThreshold": "3",
+      "hbase.rootdir": "hdfs://c6402.ambari.apache.org:8020/apps/hbase/data",
+      "hbase.stagingdir": "hdfs://c6402.ambari.apache.org:8020/apps/hbase/staging",
+      "hbase.regionserver.handler.count": "60",
+      "hbase.regionserver.global.memstore.lowerLimit": "0.38",
+      "hbase.hregion.memstore.block.multiplier": "2",
+      "hbase.hregion.memstore.flush.size": "134217728",
+      "hbase.superuser": "hbase",
+      "hbase.zookeeper.property.clientPort": "2181",
+      "hbase.regionserver.global.memstore.upperLimit": "0.4",
+      "zookeeper.session.timeout": "30000",
+      "hbase.tmp.dir": "/hadoop/hbase",
+      "hbase.hregion.max.filesize": "10737418240",
+      "hfile.block.cache.size": "0.40",
+      "hbase.security.authentication": "simple",
+      "hbase.defaults.for.version.skip": "true",
+      "hbase.zookeeper.quorum": "c6402.ambari.apache.org",
+      "zookeeper.znode.parent": "/hbase-unsecure",
+      "hbase.hstore.blockingStoreFiles": "10",
+      "hbase.hregion.majorcompaction": "86400000",
+      "hbase.security.authorization": "false",
+      "hbase.cluster.distributed": "true",
+      "hbase.hregion.memstore.mslab.enabled": "true",
+      "hbase.client.scanner.caching": "100",
+      "hbase.zookeeper.useMulti": "true",
+      "hbase.regionserver.info.port": "0",
+      "hbase.master.info.port": "60010"
+    },
+    "core-site": {
+      "io.serializations": "org.apache.hadoop.io.serializer.WritableSerialization",
+      "gluster.daemon.user": "null",
+      "fs.trash.interval": "360",
+      "hadoop.security.authentication": "simple",
+      "io.compression.codecs": "org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.DefaultCodec",
+      "mapreduce.jobtracker.webinterface.trusted": "false",
+      "fs.AbstractFileSystem.glusterfs.impl": "null",
+      "fs.defaultFS": "hdfs://c6402.ambari.apache.org:8020",
+      "ipc.client.connect.max.retries": "50",
+      "ipc.client.idlethreshold": "8000",
+      "io.file.buffer.size": "131072",
+      "hadoop.security.authorization": "false",
+      "hadoop.security.auth_to_local": "\n        RULE:[2:$1@$0]([rn]m@.*)s/.*/yarn/\n        RULE:[2:$1@$0](jhs@.*)s/.*/mapred/\n        RULE:[2:$1@$0]([nd]n@.*)s/.*/hdfs/\n        RULE:[2:$1@$0](hm@.*)s/.*/hbase/\n        RULE:[2:$1@$0](rs@.*)s/.*/hbase/\n        DEFAULT",
+      "ipc.client.connection.maxidletime": "30000"
+    }
+  },
+  "commandId": "2-2"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/conf/command_template.json
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/conf/command_template.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/conf/command_template.json
new file mode 100644
index 0000000..da06c13
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/conf/command_template.json
@@ -0,0 +1,168 @@
+{
+  "roleCommand": "{{COMMAND}}",
+  "clusterName": "{{CLUSTER_NAME}}",
+  "hostname": "{{HOST_NAME}}",
+  "hostLevelParams": {
+    "java_home": "/usr/jdk64/jdk1.7.0_45"
+  },
+  "commandType": "EXECUTION_COMMAND",
+  "roleParams": {},
+  "serviceName": "{{SERVICE_NAME}}",
+  "role": "{{ROLE_NAME}}",
+  "commandParams": {},
+  "taskId": "{{TASK_ID}}",
+  "public_hostname": "{{HOST_NAME}}",
+  "configurations": {
+    "hbase-log4j": {
+      "log4j.threshold": "ALL",
+      "log4j.rootLogger": "${hbase.root.logger}",
+      "log4j.logger.org.apache.zookeeper": "INFO",
+      "log4j.logger.org.apache.hadoop.hbase": "DEBUG",
+      "log4j.logger.org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher": "INFO",
+      "log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKUtil": "INFO",
+      "log4j.category.SecurityLogger": "${hbase.security.logger}",
+      "log4j.appender.console": "org.apache.log4j.ConsoleAppender",
+      "log4j.appender.console.target": "System.err",
+      "log4j.appender.console.layout": "org.apache.log4j.PatternLayout",
+      "log4j.appender.console.layout.ConversionPattern": "%d{ISO8601} %-5p [%t] %c{2}: %m%n",
+      "log4j.appender.RFAS": "org.apache.log4j.RollingFileAppender",
+      "log4j.appender.RFAS.layout": "org.apache.log4j.PatternLayout",
+      "log4j.appender.RFAS.layout.ConversionPattern": "%d{ISO8601} %p %c: %m%n",
+      "log4j.appender.RFAS.MaxFileSize": "${hbase.security.log.maxfilesize}",
+      "log4j.appender.RFAS.MaxBackupIndex": "${hbase.security.log.maxbackupindex}",
+      "log4j.appender.RFAS.File": "${hbase.log.dir}/${hbase.security.log.file}",
+      "log4j.appender.RFA": "org.apache.log4j.RollingFileAppender",
+      "log4j.appender.RFA.layout": "org.apache.log4j.PatternLayout",
+      "log4j.appender.RFA.layout.ConversionPattern": "%d{ISO8601} %-5p [%t] %c{2}: %m%n",
+      "log4j.appender.RFA.MaxFileSize": "${hbase.log.maxfilesize}",
+      "log4j.appender.RFA.MaxBackupIndex": "${hbase.log.maxbackupindex}",
+      "log4j.appender.RFA.File": "${hbase.log.dir}/${hbase.log.file}",
+      "log4j.appender.NullAppender": "org.apache.log4j.varia.NullAppender",
+      "log4j.appender.DRFA": "org.apache.log4j.DailyRollingFileAppender",
+      "log4j.appender.DRFA.layout": "org.apache.log4j.PatternLayout",
+      "log4j.appender.DRFA.layout.ConversionPattern": "%d{ISO8601} %-5p [%t] %c{2}: %m%n",
+      "log4j.appender.DRFA.File": "${hbase.log.dir}/${hbase.log.file}",
+      "log4j.appender.DRFA.DatePattern": ".yyyy-MM-dd",
+      "log4j.additivity.SecurityLogger": "false",
+      "hbase.security.logger": "INFO,console",
+      "hbase.security.log.maxfilesize": "256MB",
+      "hbase.security.log.maxbackupindex": "20",
+      "hbase.security.log.file": "SecurityAuth.audit",
+      "hbase.root.logger": "INFO,console",
+      "hbase.log.maxfilesize": "256MB",
+      "hbase.log.maxbackupindex": "20",
+      "hbase.log.file": "hbase.log",
+      "hbase.log.dir": "."
+    },
+    "global": {
+      "hbase_root": "{{HBASE_HOME}}",
+      "hbase_pid_dir": "{{PID_DIR}}",
+      "proxyuser_group": "users",
+      "syncLimit": "5",
+      "hbase_regionserver_heapsize": "{{REGION_SERVER_HEAP_SIZE}}",
+      "rca_enabled": "false",
+      "tickTime": "2000",
+      "hbase_master_heapsize": "{{MASTER_HEAP_SIZE}}",
+      "initLimit": "10",
+      "user_group": "{{GROUP_NAME}}",
+      "hbase_user": "{{USER_NAME}}",
+      "hbase_log_dir": "{{LOG_DIR}}"
+    },
+    "hdfs-site": {
+      "dfs.namenode.checkpoint.period": "21600",
+      "dfs.namenode.avoid.write.stale.datanode": "true",
+      "dfs.namenode.checkpoint.txns": "1000000",
+      "dfs.block.access.token.enable": "true",
+      "dfs.support.append": "true",
+      "dfs.datanode.address": "0.0.0.0:${ambari.dfs.datanode.port}",
+      "dfs.cluster.administrators": " hdfs",
+      "dfs.replication": "3",
+      "ambari.dfs.datanode.http.port": "50075",
+      "dfs.datanode.balance.bandwidthPerSec": "6250000",
+      "dfs.namenode.safemode.threshold-pct": "1.0f",
+      "dfs.namenode.checkpoint.edits.dir": "${dfs.namenode.checkpoint.dir}",
+      "dfs.permissions.enabled": "true",
+      "dfs.client.read.shortcircuit": "true",
+      "dfs.namenode.https-address": "{{NAMENODE_HTTPS_ADDRESS}}",
+      "dfs.journalnode.edits.dir": "/grid/0/hdfs/journal",
+      "dfs.blocksize": "134217728",
+      "dfs.datanode.max.transfer.threads": "1024",
+      "dfs.datanode.du.reserved": "1073741824",
+      "dfs.webhdfs.enabled": "true",
+      "dfs.namenode.handler.count": "100",
+      "dfs.namenode.checkpoint.dir": "/hadoop/hdfs/namesecondary",
+      "fs.permissions.umask-mode": "022",
+      "dfs.datanode.http.address": "0.0.0.0:${ambari.dfs.datanode.http.port}",
+      "dfs.datanode.ipc.address": "0.0.0.0:8010",
+      "dfs.datanode.data.dir": "/hadoop/hdfs/data",
+      "dfs.namenode.http-address": "{{NAMENODE_HTTP_ADDRESS}}",
+      "dfs.blockreport.initialDelay": "120",
+      "dfs.datanode.failed.volumes.tolerated": "0",
+      "dfs.namenode.accesstime.precision": "0",
+      "ambari.dfs.datanode.port": "50010",
+      "dfs.namenode.avoid.read.stale.datanode": "true",
+      "dfs.namenode.secondary.http-address": "c6402.ambari.apache.org:50090",
+      "dfs.namenode.stale.datanode.interval": "30000",
+      "dfs.heartbeat.interval": "3",
+      "dfs.client.read.shortcircuit.streams.cache.size": "4096",
+      "dfs.permissions.superusergroup": "hdfs",
+      "dfs.https.port": "50470",
+      "dfs.journalnode.http-address": "0.0.0.0:8480",
+      "dfs.domain.socket.path": "/var/lib/hadoop-hdfs/dn_socket",
+      "dfs.namenode.write.stale.datanode.ratio": "1.0f",
+      "dfs.hosts.exclude": "/etc/hadoop/conf/dfs.exclude",
+      "dfs.datanode.data.dir.perm": "750",
+      "dfs.namenode.name.dir.restore": "true",
+      "dfs.replication.max": "50",
+      "dfs.namenode.name.dir": "/hadoop/hdfs/namenode"
+    },
+    "hbase-site": {
+      "hbase.hstore.flush.retries.number": "120",
+      "hbase.client.keyvalue.maxsize": "10485760",
+      "hbase.hstore.compactionThreshold": "3",
+      "hbase.rootdir": "{{HBASE_ROOT_DIR}}",
+      "hbase.stagingdir": "{{HBASE_STAGING_DIR}}",
+      "hbase.regionserver.handler.count": "60",
+      "hbase.regionserver.global.memstore.lowerLimit": "0.38",
+      "hbase.hregion.memstore.block.multiplier": "2",
+      "hbase.hregion.memstore.flush.size": "134217728",
+      "hbase.superuser": "{{HBASE_SUPERUSER}}",
+      "hbase.zookeeper.property.clientPort": "{{ZK_CLIENT_PORT}}",
+      "hbase.regionserver.global.memstore.upperLimit": "0.4",
+      "zookeeper.session.timeout": "30000",
+      "hbase.tmp.dir": "/hadoop/hbase",
+      "hbase.hregion.max.filesize": "10737418240",
+      "hfile.block.cache.size": "0.40",
+      "hbase.security.authentication": "simple",
+      "hbase.defaults.for.version.skip": "true",
+      "hbase.zookeeper.quorum": "{{ZK_HOSTS}}",
+      "zookeeper.znode.parent": "{{ZK_NODE_PARENT}}",
+      "hbase.hstore.blockingStoreFiles": "10",
+      "hbase.hregion.majorcompaction": "86400000",
+      "hbase.security.authorization": "false",
+      "hbase.cluster.distributed": "true",
+      "hbase.hregion.memstore.mslab.enabled": "true",
+      "hbase.client.scanner.caching": "100",
+      "hbase.zookeeper.useMulti": "true",
+      "hbase.regionserver.info.port": "{{REGION_SERVER_INFO_PORT}}",
+      "hbase.master.info.port": "{{MASTER_INFO_PORT}}"
+    },
+    "core-site": {
+      "io.serializations": "org.apache.hadoop.io.serializer.WritableSerialization",
+      "gluster.daemon.user": "null",
+      "fs.trash.interval": "360",
+      "hadoop.security.authentication": "simple",
+      "io.compression.codecs": "org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.DefaultCodec",
+      "mapreduce.jobtracker.webinterface.trusted": "false",
+      "fs.AbstractFileSystem.glusterfs.impl": "null",
+      "fs.defaultFS": "{{DEFAULT_FS}}",
+      "ipc.client.connect.max.retries": "50",
+      "ipc.client.idlethreshold": "8000",
+      "io.file.buffer.size": "131072",
+      "hadoop.security.authorization": "false",
+      "hadoop.security.auth_to_local": "\n        RULE:[2:$1@$0]([rn]m@.*)s/.*/yarn/\n        RULE:[2:$1@$0](jhs@.*)s/.*/mapred/\n        RULE:[2:$1@$0]([nd]n@.*)s/.*/hdfs/\n        RULE:[2:$1@$0](hm@.*)s/.*/hbase/\n        RULE:[2:$1@$0](rs@.*)s/.*/hbase/\n        DEFAULT",
+      "ipc.client.connection.maxidletime": "30000"
+    }
+  },
+  "commandId": "{{COMMAND_ID}}"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/role-node.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/role-node.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/role-node.xml
new file mode 100644
index 0000000..aff1e05
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/agent/role-node.xml
@@ -0,0 +1,65 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  ~ 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.
+  -->
+
+  <!--
+  Role options for an agent-managed node
+  -->
+<configuration>
+  <property>
+    <name>role.name</name>
+    <value>node</value>
+  </property>
+  
+  <property>
+    <name>role.instances</name>
+    <value>1</value>
+  </property>
+    
+  <property>
+    <name>role.priority</name>
+    <value>1</value>
+  </property>
+      
+  <property>
+    <name>role.placement.policy</name>
+    <value>2</value>
+  </property>
+  
+  <property>
+    <name>yarn.memory</name>
+    <value>256</value>
+  </property>
+  
+  <property>
+    <name>yarn.vcores</name>
+    <value>1</value>
+  </property>
+  
+  <property>
+    <name>jvm.heapsize</name>
+    <value>256M</value>
+  </property>
+  
+  <property>
+    <name>env.MALLOC_ARENA_MAX</name>
+    <value>4</value>
+  </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/dynamic/application.properties
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/dynamic/application.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/dynamic/application.properties
new file mode 100644
index 0000000..d9b42de
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/dynamic/application.properties
@@ -0,0 +1,25 @@
+# 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.
+#
+
+# gets updated at build time
+application.name=${pom.name}
+application.version=${pom.version}
+application.build=${buildNumber}
+application.build.java.version=${java.version}
+application.build.user=${user.name}
+application.build.info=${pom.name}-${pom.version} Built against commit# ${buildNumber} on Java ${java.version} by ${user.name}
+hadoop.build.info=${hadoop.version}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/appconf.json
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/appconf.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/appconf.json
new file mode 100644
index 0000000..81239a2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/appconf.json
@@ -0,0 +1,19 @@
+{
+  "schema": "http://example.org/specification/v2.0.0",
+
+  "metadata": {
+
+
+  },
+
+  "global": {
+    "env.MALLOC_ARENA_MAX": "4"
+  },
+
+  "components": {
+    "slider-appmaster" : {
+      "jvm.heapsize": "256M"
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/internal.json
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/internal.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/internal.json
new file mode 100644
index 0000000..2367d8f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/internal.json
@@ -0,0 +1,17 @@
+{
+  "schema": "http://example.org/specification/v2.0.0",
+
+  "metadata": {
+  },
+
+  "global": {
+    "internal.container.failure.shortlife": "60000",
+    "internal.container.failure.threshold": "5",
+    "slider.cluster.directory.permissions": "0770",
+    "slider.data.directory.permissions": "0770"
+  },
+
+  "components": {
+
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/resources.json
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/resources.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/resources.json
new file mode 100644
index 0000000..478ab7e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/resources.json
@@ -0,0 +1,18 @@
+{
+  "schema": "http://example.org/specification/v2.0.0",
+
+  "metadata": {
+ 
+  },
+
+  "global": {
+  },
+
+  "components": {
+    "slider-appmaster": {
+      "yarn.component.instances": "1",
+      "yarn.vcores": "1",
+      "yarn.memory": "1024"
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/slider.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/slider.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/slider.xml
new file mode 100644
index 0000000..37ac65c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/slider.xml
@@ -0,0 +1,30 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  ~ 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.
+  -->
+
+<configuration>
+  <property>
+    <name>slider.config.loaded</name>
+    <value>true</value>
+  </property>
+  <property>
+    <name>slider.provider.agent</name>
+    <value>org.apache.slider.providers.agent.AgentProviderFactory</value>
+  </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/scripts/slider_keytabs.sh
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/scripts/slider_keytabs.sh b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/scripts/slider_keytabs.sh
new file mode 100644
index 0000000..f0a8fc2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/scripts/slider_keytabs.sh
@@ -0,0 +1,67 @@
+#!/usr/bin/env bash
+
+# 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.
+
+# This script exists to create the keytab set for a node on the cluster
+# including hbase and ZK alongside then YARN cores.
+
+# usage
+# keytabs <realm> <hostname>
+# validate the args
+
+num_vars=$#
+if [[ $num_vars < 2 ]]
+then
+  echo "Usage: $0 <realm> <hostname>"
+  exit -2
+fi
+
+realm="$1"
+hostname="$2"
+dest="."
+
+kadmin=kadmin.local
+
+${kadmin} <<EOF
+addprinc -randkey hdfs/${hostname}@${realm}
+addprinc -randkey yarn/${hostname}@${realm}
+addprinc -randkey HTTP/${hostname}@${realm}
+addprinc -randkey hbase/${hostname}@${realm}
+addprinc -randkey zookeeper/${hostname}@${realm}
+
+ktadd -norandkey -k ${dest}/hdfs.keytab  \
+  hdfs/${hostname}@${realm} \
+  HTTP/${hostname}@${realm}
+
+ktadd -norandkey -k ${dest}/yarn.keytab  \
+  yarn/${hostname}@${realm} \
+  HTTP/${hostname}@${realm}
+
+ktadd -norandkey -k ${dest}/hbase.keytab  \
+  hbase/${hostname}@${realm} 
+
+ktadd -norandkey -k ${dest}/zookeeper.keytab  \
+  zookeeper/${hostname}@${realm} 
+EOF
+
+exitcode=$?
+if  [[ $exitcode != 0 ]]
+then
+  echo "keytab generation from ${kadmin} failed with exit code $exitcode"
+  exit $exitcode
+else
+  echo "keytab files for ${hostname}@${realm} created"
+fi

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/scripts/yarnservice.py
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/scripts/yarnservice.py b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/scripts/yarnservice.py
new file mode 100644
index 0000000..1208c28
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/scripts/yarnservice.py
@@ -0,0 +1,383 @@
+#!/usr/bin/env python
+# -*- coding: utf-8 -*-
+#
+# 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.
+
+
+"""Launches a yarn service
+
+WORK IN PROGRESS, IGNORE
+
+This is as work in progress project to build as new launcher script for
+any Hadoop service
+A key feature here is that the configs are defined in JSON files -
+files that are read in the order passed down, and merged into each other.
+
+The final merged file is used to define the java command to execute
+-and hadoop XML files.
+
+
+It uses a JSON config file 
+  --jfile configuration file (JSON format)
+  -class classname
+  -Dname=value -arbitrary value to pass down to the JVM
+  --java: any JVM arg
+  -javaX: javaX value
+
+
+ after an -- , all following commands are passed straight down to the invoked process.
+  # -xJ name=value JVM options. No: this is just another param
+  -xF file  file to load next. Files are loaded in order. 
+  -xD name=value again, values are loaded in order
+  -xU undefine
+  -xX main class, 'eXecute'
+
+  --  end of arguments
+  
+
+"""
+
+import sys
+# see : http://simplejson.readthedocs.org/en/latest/
+# and install w/ easy_install simplejson
+import simplejson
+
+KEY_JFILE = "-xF"
+KEY_DEF = "-xD"
+KEY_UNDEF = "-xU"
+KEY_EXEC = "-xX"
+KEY_ARGS = "--"
+
+COMMANDS = [KEY_JFILE, KEY_DEF, KEY_EXEC]
+
+#
+
+def debug(string) :
+  print string
+
+
+def pop_required_arg(arglist, previousArg) :
+  """
+  Pop the first element off the list and return it.
+  If the list is empty, raise an exception about a missing argument after the $previousArgument
+  """
+  if not len(arglist) :
+    raise Exception, "Missing required parameter after %s" % previousArg
+  head = arglist[0]
+  del arglist[0]
+  return head
+
+
+def parse_one_jfile(filename) :
+  """
+  read in the given config file
+  """
+  parsed = simplejson.load(open(filename, "r"))
+  return parsed
+
+# hand down sys.argv:
+def extract_jfiles(args) :
+  """ takes a list of arg strings and separates them into jfile references
+  and other arguments.
+  """
+  l = len(args)
+  stripped = []
+  jfiles = []
+  index = 0
+  while index < l :
+    elt = args[index]
+    index += 1
+    if KEY_JFILE == elt :
+      # a match
+      if index == l :
+        #overshoot
+        raise Exception("Missing filename after " + KEY_JFILE)
+      filename = args[index]
+      debug("jfile " + filename)
+      jfiles.append(filename)
+      index += 1
+    else :
+      stripped.append(elt)
+  return jfiles, stripped
+
+
+def extract_args(args) :
+  """
+  Take a list of args, parse them or fail, generating a dictionary of actions
+  Return: dictionary and all leftover arguments
+  """
+  jfiles = []
+  execs = []
+  defs = []
+  remainder = []
+  while len(args) :
+    # the next call cannot fail, because of the len(args)
+    arg = pop_required_arg(args, "")
+    if KEY_JFILE == arg :
+      jfiles.append(pop_required_arg(args, KEY_JFILE))
+    elif KEY_DEF == arg :
+      defs.append((KEY_DEF, pop_required_arg(args, KEY_DEF)))
+    elif KEY_UNDEF == arg :
+      defs.append((KEY_UNDEF, pop_required_arg(args, KEY_UNDEF)))
+    elif KEY_EXEC == arg :
+      execs.append(pop_required_arg(args, KEY_EXEC))
+    elif KEY_ARGS == arg :
+      remainder += args
+      args = []
+    else :
+      remainder.append(arg)
+      #build the action list
+  actions = {
+    KEY_JFILE : jfiles,
+    KEY_EXEC : execs,
+    KEY_DEF : defs,
+    KEY_ARGS : remainder
+  }
+  #end of the run, there's a dictionary and a list of unparsed values
+  return actions
+
+
+def get(conf, key, defVal) :
+  if conf.has_key(key) :
+    return conf[key]
+  else :
+    return defVal
+
+
+def merge_json(conf, json) :
+  """ merge in a json dict with the existing one
+  in: configuration dict, json dict
+  out: configuration'
+  """
+  for (key, val) in json.items() :
+    if key in conf :
+      #there's a match, do a more detailed merge
+      oldval = conf[key]
+      if type(oldval) == dict and type(val) == dict :
+      # two dictionary instances -merge
+        merge_json(oldval, val)
+      else :
+        conf[key] = val
+    else :
+      conf[key] = val
+  return conf
+
+
+def merge_jfile(conf, filename) :
+  json = parse_one_jfile(filename)
+  return merge_json(conf, json)
+
+
+def merge_jfile_list(conf, jfiles) :
+  """ merge a list of jfiles on top of a conf dict
+  """
+  for jfile in jfiles :
+    conf = merge_jfile(conf, jfile)
+  return conf
+
+
+def split_to_keyval_tuple(param) :
+  """
+  Split a key=value string into the (key,value) tuple
+  * an exception is raised on any string "=value"
+  * if there is no string: exception.
+  * a key only definition maps to (key, None)
+  * a "key=" definition maps to (key, "")
+  """
+  if not len(param) :
+    raise Exception, "Empty string cannot be a key=value definition"
+  equalsPos = param.find("=")
+  if equalsPos < 0 :
+    return param, None
+  elif not equalsPos :
+    raise Exception, "no key in argument %s" % param
+  else :
+    key = param[:(equalsPos - 1)]
+    value = param[(equalsPos + 1) :]
+    return key, value
+
+
+def recursive_define(conf, path, value) :
+  if not len(path) :
+    #fallen off the end of the world
+    return
+  entry = path[0]
+  if len(path) == 1 :
+    #end of list, apply it.
+    conf[entry] = value
+  else :
+    #there's 1+ elements below, yet there's a subdir here.
+    if conf.has_key(entry) and type(conf[entry]) == dict :
+      #it's a subdir, simple: recurse.
+      recursive_define(conf[entry], path[1 :], value)
+    else :
+      #either there is an entry that isn't a conf, or its not there. Same outcome.
+      subconf = {}
+      conf[entry] = subconf
+      recursive_define(subconf, path[1 :], value)
+
+def recursive_undef(conf, path) :
+  if not len(path) :
+    #fallen off the end of the world
+    return
+  entry = path[0]
+  if len(path) == 1 :
+    #end of list, apply it.
+    del conf[entry]
+  else :
+    #there's 1+ elements below, yet there's a subdir here.
+    if conf.has_key(entry) and type(conf[entry]) == dict :
+      #it's a subdir, simple: recurse.
+      recursive_undef(conf[entry], path[1 :])
+    else :
+      #either there is an entry that isn't a conf, or its not there. Same outcome.
+      pass
+
+def apply_action(conf, action, key, value) :
+  """
+  Apply either a def or undef action; splitting the key into a path and running through it.
+  """
+  keypath = key.split("/")
+  #now have a split key,
+  if KEY_DEF == action :
+    recursive_define(conf, keypath, value)
+  elif KEY_UNDEF == action :
+    recursive_undef(conf, keypath)
+
+
+def apply_local_definitions(conf, definitions) :
+  """
+  Run through the definition actions and apply them one by one
+  """
+  for defn in definitions :
+    # split into key=value; no value -> empty string
+    (action, param) = defn
+    if KEY_DEF == action :
+      (key, val) = split_to_keyval_tuple(param)
+      apply_action(conf, KEY_DEF, key, val)
+
+  return conf
+
+
+#def parse_args(conf, args) :
+#  """
+#   split an arg string, parse the jfiles & merge over the conf
+#  (configuration, args[]) -> (conf', stripped, jfiles[])
+#  """
+#  (jfiles, stripped) = extract_jfiles(args)
+#
+#  actions = extract_args(args)
+#  jfiles = actions[KEY_JFILE]
+#  conf = merge_jfile_list(conf, jfiles)
+#  return conf, actions
+
+
+def print_conf(conf) :
+  """ dump the configuration to the console
+  """
+  print "{"
+  for (key, val) in conf.items() :
+    if type(val) == dict :
+      print key
+      print_conf(val)
+    else :
+      print "" + key + " => " + str(val)
+  print "}"
+
+
+def list_to_str(l, spacer) :
+  result = ""
+  for elt in l :
+    if len(result) > 0 :
+      result += spacer
+    result += elt
+  return result
+
+
+def list_to_hxml_str(l) :
+  return list_to_str(l, ",")
+
+
+def export_kv_xml(output, key, value) :
+  line = "<property><name>" + key + "</name><value>" + str(value) + "</value>\n"
+  print line
+  output.write(line)
+
+
+def export_to_hadoop_xml(output, conf) :
+  """ export the conf to hadoop XML
+  dictionaries are skipped.
+  """
+  output.write("<configuration>\n")
+  for (key, value) in conf.items() :
+    if type(value) is list :
+      # list print
+      export_kv_xml(output, key, list_to_hxml_str(value))
+    else :
+      if type(value) is dict :
+        print "skipping dict " + key
+      else :
+        export_kv_xml(output, key, value)
+  output.write("</configuration>\n")
+
+
+def start(conf, stripped_args) :
+  """
+  start the process by grabbing exec/args for the arguments
+  """
+  ex = conf["exec"]
+  args = []
+  jsonargs = get(ex, "args", [])
+  args.extend(jsonargs)
+  args.extend(stripped_args)
+  classname = get(ex, "classname", "")
+  if not len(classname) :
+    raise Exception, "No classname supplied"
+  classname = get(ex, "classname", "")
+  commandline = ["java"]
+  classpath = []
+  jvmargs = []
+  commandline.extend(jvmargs)
+  commandline.append("-classpath")
+  commandline.append(list_to_str(classpath, ":"))
+  commandline.append("org.apache.hadoop.yarn.service.launcher.ServiceLauncher")
+  commandline.append(classname)
+  commandline.extend(args)
+  print "ready to exec : %s" % commandline
+
+
+def main() :
+#  (conf, stripped, jfiles) = parse_args({}, sys.argv[1 :])
+  actions = extract_args(sys.argv[1 :])
+  jfiles = actions[KEY_JFILE]
+  conf = merge_jfile_list({}, jfiles)
+  apply_local_definitions(conf, actions[KEY_DEF])
+  exec_args = actions[KEY_ARGS]
+
+  print_conf(conf)
+  #  if len(stripped) > 0 :
+  #got an output file
+  #    filename = stripped[0]
+  #    print "Writing XML configuration to " + filename
+  #    output = open(filename, "w")
+  #    export_to_hadoop_xml(output, conf["site"])
+  start(conf, exec_args)
+
+
+if __name__ == "__main__" :
+  main()
+
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/site/site.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/site/site.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/site/site.xml
new file mode 100644
index 0000000..3b5df7a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/site/site.xml
@@ -0,0 +1,26 @@
+<?xml version="1.0"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~  or more contributor license agreements.  See the NOTICE file
+  ~  distributed with this work for additional information
+  ~  regarding copyright ownership.  The ASF licenses this file
+  ~  to you under the Apache License, Version 2.0 (the
+  ~  "License"); you may not use this file except in compliance
+  ~  with the License.  You may obtain a copy of the License at
+  ~
+  ~       http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~  Unless required by applicable law or agreed to in writing, software
+  ~  distributed under the License is distributed on an "AS IS" BASIS,
+  ~  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~  See the License for the specific language governing permissions and
+  ~  limitations under the License.
+  -->
+
+<project name="Slider">
+
+  <version position="right"/>
+  <body>
+    <menu ref="reports"/>
+  </body>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/pom.xml
new file mode 100644
index 0000000..ad374f8d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/pom.xml
@@ -0,0 +1,39 @@
+<?xml version="1.0"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
+                      http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>hadoop-yarn</artifactId>
+        <groupId>org.apache.hadoop</groupId>
+        <version>3.0.0-alpha2-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-yarn-slider</artifactId>
+    <name>Apache Hadoop YARN Slider</name>
+    <packaging>pom</packaging>
+
+    <properties>
+        <hadoop.common.build.dir>${basedir}/../../../../hadoop-common-project/hadoop-common/target</hadoop.common.build.dir>
+    </properties>
+
+    <!-- Do not add dependencies here, add them to the POM of the leaf module -->
+
+    <modules>
+        <module>hadoop-yarn-slider-core</module>
+    </modules>
+</project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml
index 233a353..493e03a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml
@@ -36,6 +36,8 @@
   <modules>
     <module>hadoop-yarn-applications-distributedshell</module>
     <module>hadoop-yarn-applications-unmanaged-am-launcher</module>
+    <module>hadoop-yarn-slider</module>
+
   </modules>
 
  <profiles>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[42/76] [abbrv] hadoop git commit: YARN-5538. Apply SLIDER-875 to yarn-native-services. Contributed by Billie Rinaldi

Posted by ji...@apache.org.
YARN-5538. Apply SLIDER-875 to yarn-native-services. Contributed by Billie Rinaldi


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

Branch: refs/heads/yarn-native-services
Commit: 86a29d4f3dd31796848335036c797e98f565ca38
Parents: a4beaeb
Author: Jian He <ji...@apache.org>
Authored: Mon Aug 22 13:58:50 2016 +0800
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/slider/api/RoleKeys.java    |   5 +
 .../org/apache/slider/client/SliderClient.java  |  14 +-
 .../org/apache/slider/common/SliderKeys.java    |  13 ++
 .../apache/slider/common/tools/SliderUtils.java |  88 +++++++
 .../slider/core/buildutils/InstanceBuilder.java | 205 +++++++++++++++++
 .../slider/core/conf/ConfTreeOperations.java    |  50 ++++
 .../providers/AbstractProviderService.java      |   3 +-
 .../slider/providers/ProviderService.java       |   4 +-
 .../providers/agent/AgentClientProvider.java    |  29 ++-
 .../slider/providers/agent/AgentKeys.java       |   3 +
 .../providers/agent/AgentProviderService.java   | 227 ++++++++++++++-----
 .../slider/providers/agent/AgentUtils.java      |  16 ++
 .../providers/agent/ComponentCommandOrder.java  | 112 ++++++---
 .../server/appmaster/SliderAppMaster.java       |   6 +-
 .../slider/server/appmaster/state/AppState.java |   5 +
 .../appmaster/web/rest/agent/AgentResource.java |   4 +-
 16 files changed, 669 insertions(+), 115 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/86a29d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/RoleKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/RoleKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/RoleKeys.java
index 812a6b3..ce413ff 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/RoleKeys.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/RoleKeys.java
@@ -35,6 +35,11 @@ public interface RoleKeys {
   String ROLE_GROUP = "role.group";
 
   /**
+   * The prefix of a role: {@value}
+   */
+  String ROLE_PREFIX = "role.prefix";
+
+  /**
    * Status report: number actually granted : {@value} 
    */
   String ROLE_ACTUAL_INSTANCES = "role.actual.instances";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86a29d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
index d464ce0..3129f6f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
@@ -151,6 +151,7 @@ import org.apache.slider.core.registry.YarnAppListClient;
 import org.apache.slider.core.registry.docstore.ConfigFormat;
 import org.apache.slider.core.registry.docstore.PublishedConfigSet;
 import org.apache.slider.core.registry.docstore.PublishedConfiguration;
+import org.apache.slider.core.registry.docstore.PublishedConfigurationOutputter;
 import org.apache.slider.core.registry.docstore.PublishedExports;
 import org.apache.slider.core.registry.docstore.PublishedExportsOutputter;
 import org.apache.slider.core.registry.docstore.PublishedExportsSet;
@@ -724,7 +725,8 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     AggregateConf instanceDefinition = loadInstanceDefinitionUnresolved(
         clustername, clusterDirectory);
     try {
-      checkForCredentials(getConfig(), instanceDefinition.getAppConf());
+      checkForCredentials(getConfig(), instanceDefinition.getAppConf(),
+          clustername);
     } catch (IOException e) {
       sliderFileSystem.getFileSystem().delete(clusterDirectory, true);
       throw e;
@@ -906,7 +908,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   }
 
   protected static void checkForCredentials(Configuration conf,
-      ConfTree tree) throws IOException {
+      ConfTree tree, String clusterName) throws IOException {
     if (tree.credentials == null || tree.credentials.isEmpty()) {
       log.info("No credentials requested");
       return;
@@ -915,7 +917,9 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     BufferedReader br = null;
     try {
       for (Entry<String, List<String>> cred : tree.credentials.entrySet()) {
-        String provider = cred.getKey();
+        String provider = cred.getKey()
+            .replaceAll(Pattern.quote("${CLUSTER_NAME}"), clusterName)
+            .replaceAll(Pattern.quote("${CLUSTER}"), clusterName);
         List<String> aliases = cred.getValue();
         if (aliases == null || aliases.isEmpty()) {
           continue;
@@ -1727,6 +1731,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     resources.mergeComponents(buildInfo.getResourceCompOptionMap());
 
     builder.init(providerName, instanceDefinition);
+    builder.resolve();
     builder.propagateFilename();
     builder.propagatePrincipals();
     builder.setImageDetailsIfAvailable(buildInfo.getImage(),
@@ -1917,8 +1922,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   private static String replaceTokens(String s, String userName,
       String clusterName) throws IOException {
     return s.replaceAll(Pattern.quote("${USER}"), userName)
-        .replaceAll(Pattern.quote("${USER_NAME}"), userName)
-        .replaceAll(Pattern.quote("${CLUSTER_NAME}"), clusterName);
+        .replaceAll(Pattern.quote("${USER_NAME}"), userName);
   }
 
   public FsPermission getClusterDirectoryPermissions(Configuration conf) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86a29d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
index ba3effc..120b1fc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
@@ -70,6 +70,19 @@ public interface SliderKeys extends SliderXmlConfKeys {
   String APP_TYPE = "org-apache-slider";
 
   /**
+   * Key for component type. This MUST NOT be set in app_config/global {@value}
+   */
+  String COMPONENT_TYPE_KEY = "site.global.component_type";
+  /**
+   * A component type for an external app that has been predefined using the
+   * slider build command
+   */
+  String COMPONENT_TYPE_EXTERNAL_APP = "external_app";
+  String COMPONENT_SEPARATOR = "-";
+  String[] COMPONENT_KEYS_TO_SKIP = {"zookeeper.", "env.MALLOC_ARENA_MAX",
+      "site.fs.", "site.dfs."};
+
+  /**
    * Key for application version. This must be set in app_config/global {@value}
    */
   String APP_VERSION = "site.global.app_version";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86a29d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
index 73e0879..e9f65ba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
@@ -69,6 +69,7 @@ import org.apache.slider.core.exceptions.SliderException;
 import org.apache.slider.core.launch.ClasspathConstructor;
 import org.apache.slider.core.main.LauncherExitCodes;
 import org.apache.slider.providers.agent.AgentKeys;
+import org.apache.slider.providers.agent.application.metadata.Component;
 import org.apache.slider.server.services.utility.PatternValidator;
 import org.apache.slider.server.services.workflow.ForkedProcessService;
 import org.apache.zookeeper.server.util.KerberosUtil;
@@ -122,6 +123,8 @@ import java.util.zip.GZIPOutputStream;
 import java.util.zip.ZipEntry;
 import java.util.zip.ZipOutputStream;
 
+import static org.apache.slider.common.SliderKeys.COMPONENT_SEPARATOR;
+
 /**
  * These are slider-specific Util methods
  */
@@ -475,6 +478,32 @@ public final class SliderUtils {
     return srcFileCount;
   }
 
+  /**
+   * Copy a file to a new FS -both paths must be qualified.
+   * @param conf conf file
+   * @param srcFile src file
+   * @param destFile dest file
+   */
+  public static void copy(Configuration conf,
+      Path srcFile,
+      Path destFile) throws
+      IOException,
+      BadClusterStateException {
+    FileSystem srcFS = FileSystem.get(srcFile.toUri(), conf);
+    //list all paths in the src.
+    if (!srcFS.exists(srcFile)) {
+      throw new FileNotFoundException("Source file not found " + srcFile);
+    }
+    if (!srcFS.isFile(srcFile)) {
+      throw new FileNotFoundException(
+          "Source file not a file " + srcFile);
+    }
+    FileSystem destFS = FileSystem.get(destFile.toUri(), conf);
+    if (destFS.exists(destFile)) {
+      throw new IOException("Dest file already exists " + destFile);
+    }
+    FileUtil.copy(srcFS, srcFile, destFS, destFile, false, true, conf);
+  }
 
   public static String stringify(Throwable t) {
     StringWriter sw = new StringWriter();
@@ -927,6 +956,38 @@ public final class SliderUtils {
   }
 
   /**
+   * Merge string maps excluding prefixes
+   * @param first first map
+   * @param second second map
+   * @param  prefixes prefixes to ignore
+   * @return 'first' merged with the second
+   */
+  public static Map<String, String> mergeMapsIgnoreDuplicateKeysAndPrefixes(
+      Map<String, String> first, Map<String, String> second,
+      String... prefixes) {
+    Preconditions.checkArgument(first != null, "Null 'first' value");
+    Preconditions.checkArgument(second != null, "Null 'second' value");
+    Preconditions.checkArgument(prefixes != null, "Null 'prefixes' value");
+    for (Map.Entry<String, String> entry : second.entrySet()) {
+      String key = entry.getKey();
+      boolean hasPrefix = false;
+      for (String prefix : prefixes) {
+        if (key.startsWith(prefix)) {
+          hasPrefix = true;
+          break;
+        }
+      }
+      if (hasPrefix) {
+        continue;
+      }
+      if (!first.containsKey(key)) {
+        first.put(key, entry.getValue());
+      }
+    }
+    return first;
+  }
+
+  /**
    * Convert a map to a multi-line string for printing
    * @param map map to stringify
    * @return a string representation of the map
@@ -2352,8 +2413,28 @@ public final class SliderUtils {
    */
   public static String getApplicationDefinitionPath(ConfTreeOperations conf)
       throws BadConfigException {
+    return getApplicationDefinitionPath(conf, null);
+  }
+
+  /**
+   * return the HDFS path where the application package has been uploaded
+   * manually or by using slider client (install package command)
+   *
+   * @param conf configuration
+   * @param roleGroup name of component
+   * @return
+   */
+  public static String getApplicationDefinitionPath(ConfTreeOperations conf,
+      String roleGroup)
+      throws BadConfigException {
     String appDefPath = conf.getGlobalOptions().getMandatoryOption(
         AgentKeys.APP_DEF);
+    if (roleGroup != null) {
+      MapOperations component = conf.getComponent(roleGroup);
+      if (component != null) {
+        appDefPath = component.getOption(AgentKeys.APP_DEF, appDefPath);
+      }
+    }
     return appDefPath;
   }
 
@@ -2545,4 +2626,11 @@ public final class SliderUtils {
     }
     return buffer.toString();
   }
+
+  public static String trimPrefix(String prefix) {
+    if (prefix != null && prefix.endsWith(COMPONENT_SEPARATOR)) {
+      return prefix.substring(0, prefix.length()-1);
+    }
+    return prefix;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86a29d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/buildutils/InstanceBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/buildutils/InstanceBuilder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/buildutils/InstanceBuilder.java
index 4250d79..25c65fc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/buildutils/InstanceBuilder.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/buildutils/InstanceBuilder.java
@@ -25,7 +25,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.slider.api.InternalKeys;
 import org.apache.slider.api.OptionKeys;
+import org.apache.slider.api.ResourceKeys;
 import org.apache.slider.api.StatusKeys;
+import org.apache.slider.common.SliderKeys;
 import org.apache.slider.common.SliderXmlConfKeys;
 import org.apache.slider.common.tools.CoreFileSystem;
 import org.apache.slider.common.tools.SliderUtils;
@@ -42,11 +44,17 @@ import org.apache.slider.core.persist.LockAcquireFailedException;
 import org.apache.slider.core.persist.LockHeldAction;
 import org.apache.slider.core.zk.ZKPathBuilder;
 import org.apache.slider.core.zk.ZookeeperUtils;
+import org.apache.slider.providers.agent.AgentKeys;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeSet;
 
 import static org.apache.slider.api.InternalKeys.INTERNAL_ADDONS_DIR_PATH;
 import static org.apache.slider.api.InternalKeys.INTERNAL_APPDEF_DIR_PATH;
@@ -61,6 +69,12 @@ import static org.apache.slider.api.OptionKeys.INTERNAL_SNAPSHOT_CONF_PATH;
 import static org.apache.slider.api.OptionKeys.ZOOKEEPER_HOSTS;
 import static org.apache.slider.api.OptionKeys.ZOOKEEPER_PATH;
 import static org.apache.slider.api.OptionKeys.ZOOKEEPER_QUORUM;
+import static org.apache.slider.api.RoleKeys.ROLE_PREFIX;
+import static org.apache.slider.common.SliderKeys.COMPONENT_AM;
+import static org.apache.slider.common.SliderKeys.COMPONENT_SEPARATOR;
+import static org.apache.slider.common.SliderKeys.COMPONENT_TYPE_EXTERNAL_APP;
+import static org.apache.slider.common.SliderKeys.COMPONENT_TYPE_KEY;
+import static org.apache.slider.common.tools.SliderUtils.isClusternameValid;
 
 /**
  * Build up the instance of a cluster.
@@ -72,6 +86,8 @@ public class InstanceBuilder {
   private final CoreFileSystem coreFS;
   private final InstancePaths instancePaths;
   private AggregateConf instanceDescription;
+  private Map<String, Path> externalAppDefs = new HashMap<>();
+  private TreeSet<Integer> priorities = new TreeSet<>();
 
   private static final Logger log =
     LoggerFactory.getLogger(InstanceBuilder.class);
@@ -244,6 +260,192 @@ public class InstanceBuilder {
   }
 
 
+  private Set<String> getExternalComponents(ConfTreeOperations ops)
+      throws BadConfigException {
+    Set<String> externalComponents = new HashSet<>();
+    if (ops.getGlobalOptions().containsKey(COMPONENT_TYPE_KEY)) {
+      throw new BadConfigException(COMPONENT_TYPE_KEY + " must be " +
+          "specified per-component, not in global");
+    }
+
+    for (Entry<String, Map<String, String>> entry : ops.getComponents()
+        .entrySet()) {
+      if (COMPONENT_AM.equals(entry.getKey())) {
+        continue;
+      }
+      Map<String, String> options = entry.getValue();
+      if (COMPONENT_TYPE_EXTERNAL_APP.equals(options.get(COMPONENT_TYPE_KEY))) {
+        externalComponents.add(entry.getKey());
+      }
+    }
+    return externalComponents;
+  }
+
+  private void mergeExternalComponent(ConfTreeOperations ops,
+      ConfTreeOperations externalOps, String externalComponent,
+      Integer priority) throws BadConfigException {
+    for (String subComponent : externalOps.getComponentNames()) {
+      if (COMPONENT_AM.equals(subComponent)) {
+        continue;
+      }
+      String prefix = externalComponent + COMPONENT_SEPARATOR;
+      log.debug("Merging options for {} into {}", subComponent,
+          prefix + subComponent);
+      MapOperations subComponentOps = ops.getOrAddComponent(
+          prefix + subComponent);
+      if (priority == null) {
+        SliderUtils.mergeMaps(subComponentOps,
+            ops.getComponent(externalComponent).options);
+        subComponentOps.remove(COMPONENT_TYPE_KEY);
+      }
+
+      SliderUtils.mergeMapsIgnoreDuplicateKeysAndPrefixes(subComponentOps,
+          externalOps.getComponent(subComponent),
+          SliderKeys.COMPONENT_KEYS_TO_SKIP);
+
+      // add prefix to existing prefix
+      String existingPrefix = subComponentOps.get(ROLE_PREFIX);
+      if (existingPrefix != null) {
+        if (!subComponent.startsWith(existingPrefix)) {
+          throw new BadConfigException("Bad prefix " + existingPrefix +
+              " for subcomponent " + subComponent + " of " + externalComponent);
+        }
+        prefix = prefix + existingPrefix;
+      }
+      subComponentOps.set(ROLE_PREFIX, prefix);
+
+      // adjust priority
+      if (priority != null) {
+        subComponentOps.put(ResourceKeys.COMPONENT_PRIORITY,
+            Integer.toString(priority));
+        priorities.add(priority);
+        priority++;
+      }
+    }
+  }
+
+  private int getNextPriority() {
+    if (priorities.isEmpty()) {
+      return 1;
+    } else {
+      return priorities.last() + 1;
+    }
+  }
+
+  public void resolve()
+      throws BadConfigException, IOException, BadClusterStateException {
+    ConfTreeOperations appConf = instanceDescription.getAppConfOperations();
+    ConfTreeOperations resources = instanceDescription.getResourceOperations();
+
+    for (Entry<String, Map<String, String>> entry : resources.getComponents()
+        .entrySet()) {
+      if (COMPONENT_AM.equals(entry.getKey())) {
+        continue;
+      }
+      if (entry.getValue().containsKey(ResourceKeys.COMPONENT_PRIORITY)) {
+        priorities.add(Integer.parseInt(entry.getValue().get(
+            ResourceKeys.COMPONENT_PRIORITY)));
+      }
+    }
+
+    Set<String> externalComponents = getExternalComponents(appConf);
+    if (!externalComponents.isEmpty()) {
+      log.info("Found external components {}", externalComponents);
+    }
+
+    for (String component : externalComponents) {
+      if (!isClusternameValid(component)) {
+        throw new BadConfigException(component + " is not a valid external " +
+            "component");
+      }
+      Path componentClusterDir = coreFS.buildClusterDirPath(component);
+      try {
+        coreFS.verifyPathExists(componentClusterDir);
+      } catch (IOException e) {
+        throw new BadConfigException("external component " + component +
+            " doesn't exist");
+      }
+      AggregateConf componentConf = new AggregateConf();
+      ConfPersister persister = new ConfPersister(coreFS, componentClusterDir);
+      try {
+        persister.load(componentConf);
+      } catch (Exception e) {
+        throw new BadConfigException("Couldn't read configuration for " +
+            "external component " + component);
+      }
+
+      ConfTreeOperations componentAppConf = componentConf.getAppConfOperations();
+      String externalAppDef = componentAppConf.get(AgentKeys.APP_DEF);
+      if (SliderUtils.isSet(externalAppDef)) {
+        Path newAppDef = new Path(coreFS.buildAppDefDirPath(clustername),
+            component + "_" + SliderKeys.DEFAULT_APP_PKG);
+        componentAppConf.set(AgentKeys.APP_DEF, newAppDef);
+        componentAppConf.append(AgentKeys.APP_DEF_ORIGINAL, externalAppDef);
+        log.info("Copying external appdef {} to {} for {}", externalAppDef,
+            newAppDef, component);
+        externalAppDefs.put(externalAppDef, newAppDef);
+        externalAppDef = newAppDef.toString();
+      }
+
+      for (String rcomp : componentConf.getResourceOperations()
+          .getComponentNames()) {
+        if (COMPONENT_AM.equals(rcomp)) {
+          continue;
+        }
+        log.debug("Adding component {} to appConf for {}", rcomp, component);
+        componentAppConf.getOrAddComponent(rcomp);
+      }
+      componentConf.resolve();
+
+      for (String rcomp : componentConf.getResourceOperations()
+          .getComponentNames()) {
+        if (COMPONENT_AM.equals(rcomp)) {
+          continue;
+        }
+        String componentAppDef = componentAppConf.getComponentOpt(
+            rcomp, AgentKeys.APP_DEF, null);
+        if (SliderUtils.isUnset(componentAppDef) ||
+            componentAppDef.equals(externalAppDef)) {
+          continue;
+        }
+        if (externalAppDefs.containsKey(componentAppDef)) {
+          log.info("Using external appdef {} for {}",
+              externalAppDefs.get(componentAppDef), rcomp);
+        } else {
+          String existingPrefix = componentAppConf.getComponentOpt(rcomp,
+              ROLE_PREFIX, null);
+          if (SliderUtils.isUnset(existingPrefix)) {
+            existingPrefix = "";
+          } else {
+            existingPrefix = COMPONENT_SEPARATOR + SliderUtils.trimPrefix(
+                existingPrefix);
+          }
+          Path newAppDef = new Path(coreFS.buildAppDefDirPath(clustername),
+              component + existingPrefix + "_" + SliderKeys.DEFAULT_APP_PKG);
+          externalAppDefs.put(componentAppDef, newAppDef);
+          log.info("Copying external appdef {} to {} for {}", componentAppDef,
+              newAppDef, component + COMPONENT_SEPARATOR + rcomp);
+        }
+        componentAppConf.setComponentOpt(rcomp, AgentKeys.APP_DEF,
+            externalAppDefs.get(componentAppDef).toString());
+        componentAppConf.appendComponentOpt(rcomp,
+            AgentKeys.APP_DEF_ORIGINAL, componentAppDef);
+      }
+      Set<Path> newAppDefs = new HashSet<>();
+      newAppDefs.addAll(externalAppDefs.values());
+      if (newAppDefs.size() != externalAppDefs.size()) {
+        throw new IllegalStateException("Values repeat in external appdefs "
+            + externalAppDefs);
+      }
+      log.info("External appdefs after {}: {}", component, externalAppDefs);
+
+      mergeExternalComponent(appConf, componentAppConf, component, null);
+      mergeExternalComponent(resources, componentConf.getResourceOperations(),
+          component, getNextPriority());
+    }
+  }
+
+
   /**
    * Persist this
    * @param appconfdir conf dir
@@ -266,6 +468,9 @@ public class InstanceBuilder {
       action = new ConfDirSnapshotAction(appconfdir);
     }
     persister.save(instanceDescription, action);
+    for (Entry<String, Path> appDef : externalAppDefs.entrySet()) {
+      SliderUtils.copy(conf, new Path(appDef.getKey()), appDef.getValue());
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86a29d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java
index d24a158..526e17d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java
@@ -214,6 +214,23 @@ public class ConfTreeOperations {
   public String get(String key) {
     return globalOptions.get(key);
   }
+  /**
+   * append to a global option
+   * @param key key
+   * @return value
+   *
+   */
+  public String append(String key, String value) {
+    if (SliderUtils.isUnset(value)) {
+      return null;
+    }
+    if (globalOptions.containsKey(key)) {
+      globalOptions.put(key, globalOptions.get(key) + "," + value);
+    } else {
+      globalOptions.put(key, value);
+    }
+    return globalOptions.get(key);
+  }
 
   /**
    * Propagate all global keys matching a prefix
@@ -257,6 +274,17 @@ public class ConfTreeOperations {
                                             Map<String, String> map,
                                             String prefix,
                                             boolean overwrite) {
+    boolean needsMerge = false;
+    for (Map.Entry<String, String> entry : map.entrySet()) {
+      String key = entry.getKey();
+      if (key.startsWith(prefix)) {
+        needsMerge = true;
+        break;
+      }
+    }
+    if (!needsMerge) {
+      return;
+    }
     MapOperations comp = getOrAddComponent(component);
     comp.mergeMapPrefixedKeys(map,prefix, overwrite);
   }
@@ -474,4 +502,26 @@ public class ConfTreeOperations {
     setComponentOpt(role, option, Long.toString(val));
   }
 
+  /**
+   * append to a component option
+   * @param key key
+   * @return value
+   *
+   */
+  public String appendComponentOpt(String role, String key, String value) {
+    if (SliderUtils.isUnset(value)) {
+      return null;
+    }
+    MapOperations roleopts = getComponent(role);
+    if (roleopts == null) {
+      return null;
+    }
+
+    if (roleopts.containsKey(key)) {
+      roleopts.put(key, roleopts.get(key) + "," + value);
+    } else {
+      roleopts.put(key, value);
+    }
+    return roleopts.get(key);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86a29d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
index 61b2655..92766f5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
@@ -169,7 +169,8 @@ public abstract class AbstractProviderService
    */
   @Override
   public void initializeApplicationConfiguration(
-      AggregateConf instanceDefinition, SliderFileSystem fileSystem)
+      AggregateConf instanceDefinition, SliderFileSystem fileSystem,
+      String roleGroup)
       throws IOException, SliderException {
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86a29d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
index f754eee..3f24665 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
@@ -118,11 +118,13 @@ public interface ProviderService extends ProviderCore,
    * 
    * @param instanceDefinition
    * @param fileSystem
+   * @param roleGroup
    * @throws IOException
    * @throws SliderException
    */
   void initializeApplicationConfiguration(AggregateConf instanceDefinition,
-      SliderFileSystem fileSystem) throws IOException, SliderException;
+      SliderFileSystem fileSystem, String roleGroup) throws IOException,
+      SliderException;
 
   /**
    * This is a validation of the application configuration on the AM.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86a29d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
index 4c6a50b..8203cf0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
@@ -176,13 +176,22 @@ public class AgentClientProvider extends AbstractClientProvider
     names.remove(SliderKeys.COMPONENT_AM);
     Map<Integer, String> priorityMap = new HashMap<Integer, String>();
 
-    Metainfo metaInfo = getMetainfo(fs, appDef);
-
     for (String name : names) {
+      try {
+        // Validate the app definition
+        appDef = SliderUtils.getApplicationDefinitionPath(instanceDefinition
+            .getAppConfOperations(), name);
+      } catch (BadConfigException bce) {
+        throw new BadConfigException("Application definition must be provided. " + bce.getMessage());
+      }
+      Metainfo metaInfo = getMetainfo(fs, appDef);
+
       MapOperations component = resources.getMandatoryComponent(name);
 
       if (metaInfo != null) {
-        Component componentDef = metaInfo.getApplicationComponent(name);
+        Component componentDef = metaInfo.getApplicationComponent(
+            AgentUtils.getMetainfoComponentName(name,
+                instanceDefinition.getAppConfOperations()));
         if (componentDef == null) {
           throw new BadConfigException(
               "Component %s is not a member of application.", name);
@@ -208,16 +217,12 @@ public class AgentClientProvider extends AbstractClientProvider
             existing);
       }
       priorityMap.put(priority, name);
-    }
 
-    // fileSystem may be null for tests
-    if (metaInfo != null) {
-      for (String name : names) {
-        Component componentDef = metaInfo.getApplicationComponent(name);
-        if (componentDef == null) {
-          throw new BadConfigException(
-              "Component %s is not a member of application.", name);
-        }
+      // fileSystem may be null for tests
+      if (metaInfo != null) {
+        Component componentDef = metaInfo.getApplicationComponent(
+            AgentUtils.getMetainfoComponentName(name,
+                instanceDefinition.getAppConfOperations()));
 
         // ensure that intance count is 0 for client components
         if ("CLIENT".equals(componentDef.getCategory())) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86a29d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java
index 01a3f1a..9ea984c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java
@@ -69,6 +69,7 @@ public interface AgentKeys {
   String AGENT_MAIN_SCRIPT = "agent/main.py";
 
   String APP_DEF = "application.def";
+  String APP_DEF_ORIGINAL = "application.def.original";
   String ADDON_PREFIX = "application.addon.";
   String ADDONS = "application.addons";
   String AGENT_VERSION = "agent.version";
@@ -104,6 +105,8 @@ public interface AgentKeys {
   String KEY_CONTAINER_LAUNCH_DELAY = "container.launch.delay.sec";
   String TEST_RELAX_VERIFICATION = "test.relax.validation";
   String AM_CONFIG_GENERATION = "am.config.generation";
+
+  String DEFAULT_METAINFO_MAP_KEY = "DEFAULT_KEY";
 }
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86a29d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
index 4ffae7c..2ab5c6f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
@@ -75,6 +75,7 @@ import org.apache.slider.providers.ProviderRole;
 import org.apache.slider.providers.ProviderUtils;
 import org.apache.slider.providers.agent.application.metadata.AbstractComponent;
 import org.apache.slider.providers.agent.application.metadata.Application;
+import org.apache.slider.providers.agent.application.metadata.CommandOrder;
 import org.apache.slider.providers.agent.application.metadata.CommandScript;
 import org.apache.slider.providers.agent.application.metadata.Component;
 import org.apache.slider.providers.agent.application.metadata.ComponentCommand;
@@ -128,6 +129,7 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Scanner;
 import java.util.Set;
 import java.util.TreeMap;
@@ -135,7 +137,9 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.regex.Pattern;
 
+import static org.apache.slider.api.RoleKeys.ROLE_PREFIX;
 import static org.apache.slider.server.appmaster.web.rest.RestPaths.SLIDER_PATH_AGENTS;
 
 /**
@@ -170,10 +174,10 @@ public class AgentProviderService extends AbstractProviderService implements
   private int heartbeatMonitorInterval = 0;
   private AgentClientProvider clientProvider;
   private AtomicInteger taskId = new AtomicInteger(0);
-  private volatile Metainfo metaInfo = null;
+  private volatile Map<String, MetainfoHolder> metaInfoMap = new HashMap<>();
   private SliderFileSystem fileSystem = null;
   private Map<String, DefaultConfig> defaultConfigs = null;
-  private ComponentCommandOrder commandOrder = null;
+  private ComponentCommandOrder commandOrder = new ComponentCommandOrder();
   private HeartbeatMonitor monitor;
   private Boolean canAnyMasterPublish = null;
   private AgentLaunchParameter agentLaunchParameter = null;
@@ -208,6 +212,17 @@ public class AgentProviderService extends AbstractProviderService implements
   private final Map<String, Set<String>> containerExportsMap =
       new HashMap<String, Set<String>>();
 
+  private static class MetainfoHolder {
+    Metainfo metaInfo;
+    private Map<String, DefaultConfig> defaultConfigs = null;
+
+    public MetainfoHolder(Metainfo metaInfo,
+        Map<String, DefaultConfig> defaultConfigs) {
+      this.metaInfo = metaInfo;
+      this.defaultConfigs = defaultConfigs;
+    }
+  }
+
   /**
    * Create an instance of AgentProviderService
    */
@@ -252,10 +267,11 @@ public class AgentProviderService extends AbstractProviderService implements
     Set<String> names = resources.getComponentNames();
     names.remove(SliderKeys.COMPONENT_AM);
     for (String name : names) {
-      Component componentDef = getMetaInfo().getApplicationComponent(name);
+      Component componentDef = getApplicationComponent(name);
       if (componentDef == null) {
-        throw new BadConfigException(
-            "Component %s is not a member of application.", name);
+        // component member is validated elsewhere, so we don't need to throw
+        // an exception here
+        continue;
       }
 
       MapOperations componentConfig = resources.getMandatoryComponent(name);
@@ -277,32 +293,67 @@ public class AgentProviderService extends AbstractProviderService implements
 
   // Reads the metainfo.xml in the application package and loads it
   private void buildMetainfo(AggregateConf instanceDefinition,
-                             SliderFileSystem fileSystem) throws IOException, SliderException {
-    String appDef = SliderUtils.getApplicationDefinitionPath(instanceDefinition
-        .getAppConfOperations());
+                             SliderFileSystem fileSystem,
+                             String roleGroup)
+      throws IOException, SliderException {
+    String mapKey = instanceDefinition.getAppConfOperations()
+        .getComponentOpt(roleGroup, ROLE_PREFIX, DEFAULT_METAINFO_MAP_KEY);
+    String appDef = SliderUtils.getApplicationDefinitionPath(
+        instanceDefinition.getAppConfOperations(), roleGroup);
+    MapOperations component = null;
+    if (roleGroup != null) {
+      component = instanceDefinition.getAppConfOperations().getComponent(roleGroup);
+    }
 
-    if (metaInfo == null) {
+    MetainfoHolder metaInfoHolder = metaInfoMap.get(mapKey);
+    if (metaInfoHolder == null) {
       synchronized (syncLock) {
-        if (metaInfo == null) {
+        if (this.fileSystem == null) {
           this.fileSystem = fileSystem;
+        }
+        metaInfoHolder = metaInfoMap.get(mapKey);
+        if (metaInfoHolder == null) {
           readAndSetHeartbeatMonitoringInterval(instanceDefinition);
           initializeAgentDebugCommands(instanceDefinition);
 
-          metaInfo = getApplicationMetainfo(fileSystem, appDef, false);
+          Metainfo metaInfo = getApplicationMetainfo(fileSystem, appDef, false);
           log.info("Master package metainfo: {}", metaInfo.toString());
           if (metaInfo == null || metaInfo.getApplication() == null) {
             log.error("metainfo.xml is unavailable or malformed at {}.", appDef);
             throw new SliderException(
                 "metainfo.xml is required in app package.");
           }
-          commandOrder = new ComponentCommandOrder(metaInfo.getApplication().getCommandOrders());
-          defaultConfigs = initializeDefaultConfigs(fileSystem, appDef, metaInfo);
+          List<CommandOrder> commandOrders = metaInfo.getApplication()
+              .getCommandOrders();
+          if (!DEFAULT_METAINFO_MAP_KEY.equals(mapKey)) {
+            for (Component comp : metaInfo.getApplication().getComponents()) {
+              comp.setName(mapKey + comp.getName());
+              log.info("Modifying external metainfo component name to {}",
+                  comp.getName());
+            }
+            for (CommandOrder co : commandOrders) {
+              log.info("Adding prefix {} to command order {}",
+                  mapKey, co);
+              co.setCommand(mapKey + co.getCommand());
+              co.setRequires(mapKey + co.getRequires());
+            }
+          }
+          log.debug("Merging command orders {} for {}", commandOrders,
+              roleGroup);
+          commandOrder.mergeCommandOrders(commandOrders,
+              instanceDefinition.getResourceOperations());
+          Map<String, DefaultConfig> defaultConfigs =
+              initializeDefaultConfigs(fileSystem, appDef, metaInfo);
+          metaInfoMap.put(mapKey, new MetainfoHolder(metaInfo, defaultConfigs));
           monitor = new HeartbeatMonitor(this, getHeartbeatMonitorInterval());
           monitor.start();
 
           // build a map from component to metainfo
           String addonAppDefString = instanceDefinition.getAppConfOperations()
               .getGlobalOptions().getOption(AgentKeys.ADDONS, null);
+          if (component != null) {
+            addonAppDefString = component.getOption(AgentKeys.ADDONS, addonAppDefString);
+          }
           log.debug("All addon appdefs: {}", addonAppDefString);
           if (addonAppDefString != null) {
             Scanner scanner = new Scanner(addonAppDefString).useDelimiter(",");
@@ -310,6 +361,9 @@ public class AgentProviderService extends AbstractProviderService implements
               String addonAppDef = scanner.next();
               String addonAppDefPath = instanceDefinition
                   .getAppConfOperations().getGlobalOptions().get(addonAppDef);
+              if (component != null) {
+                addonAppDefPath = component.getOption(addonAppDef, addonAppDefPath);
+              }
               log.debug("Addon package {} is stored at: {}", addonAppDef
                   + addonAppDefPath);
               Metainfo addonMetaInfo = getApplicationMetainfo(fileSystem,
@@ -328,9 +382,10 @@ public class AgentProviderService extends AbstractProviderService implements
 
   @Override
   public void initializeApplicationConfiguration(
-      AggregateConf instanceDefinition, SliderFileSystem fileSystem)
+      AggregateConf instanceDefinition, SliderFileSystem fileSystem,
+      String roleGroup)
       throws IOException, SliderException {
-    buildMetainfo(instanceDefinition, fileSystem);
+    buildMetainfo(instanceDefinition, fileSystem, roleGroup);
   }
 
   @Override
@@ -349,9 +404,9 @@ public class AgentProviderService extends AbstractProviderService implements
     String roleName = providerRole.name;
     String roleGroup = providerRole.group;
     String appDef = SliderUtils.getApplicationDefinitionPath(instanceDefinition
-        .getAppConfOperations());
+        .getAppConfOperations(), roleGroup);
 
-    initializeApplicationConfiguration(instanceDefinition, fileSystem);
+    initializeApplicationConfiguration(instanceDefinition, fileSystem, roleGroup);
 
     log.info("Build launch context for Agent");
     log.debug(instanceDefinition.toString());
@@ -439,7 +494,7 @@ public class AgentProviderService extends AbstractProviderService implements
         LocalResourceType.ARCHIVE);
     launcher.addLocalResource(AgentKeys.APP_DEFINITION_DIR, appDefRes);
 
-    for (Package pkg : getMetaInfo().getApplication().getPackages()) {
+    for (Package pkg : getMetaInfo(roleGroup).getApplication().getPackages()) {
       Path pkgPath = fileSystem.buildResourcePath(pkg.getName());
       if (!fileSystem.isFile(pkgPath)) {
         pkgPath = fileSystem.buildResourcePath(getClusterName(),
@@ -505,7 +560,7 @@ public class AgentProviderService extends AbstractProviderService implements
       Map<String, Map<String, String>> configurations =
           buildCommandConfigurations(instanceDefinition.getAppConfOperations(),
               container.getId().toString(), roleName, roleGroup);
-      localizeConfigFiles(launcher, roleName, roleGroup, getMetaInfo(),
+      localizeConfigFiles(launcher, roleName, roleGroup, getMetaInfo(roleGroup),
           configurations, launcher.getEnv(), fileSystem);
     }
 
@@ -597,7 +652,7 @@ public class AgentProviderService extends AbstractProviderService implements
     // initialize the component instance state
     getComponentStatuses().put(label,
                                new ComponentInstanceState(
-                                   roleName,
+                                   roleGroup,
                                    container.getId(),
                                    getClusterInfoPropertyValue(OptionKeys.APPLICATION_NAME),
                                    pkgStatuses));
@@ -610,6 +665,22 @@ public class AgentProviderService extends AbstractProviderService implements
                                                AggregateConf instanceDefinition,
                                                MapOperations compOps)
       throws SliderException, IOException {
+    // substitute CLUSTER_NAME into credentials
+    Map<String,List<String>> newcred = new HashMap<>();
+    for (Entry<String,List<String>> entry : instanceDefinition.getAppConf().credentials.entrySet()) {
+      List<String> resultList = new ArrayList<>();
+      for (String v : entry.getValue()) {
+        resultList.add(v.replaceAll(Pattern.quote("${CLUSTER_NAME}"),
+            clusterName).replaceAll(Pattern.quote("${CLUSTER}"),
+            clusterName));
+      }
+      newcred.put(entry.getKey().replaceAll(Pattern.quote("${CLUSTER_NAME}"),
+          clusterName).replaceAll(Pattern.quote("${CLUSTER}"),
+          clusterName),
+          resultList);
+    }
+    instanceDefinition.getAppConf().credentials = newcred;
+
     // generate and localize security stores
     SecurityStore[] stores = generateSecurityStores(container, role,
                                                     instanceDefinition, compOps);
@@ -858,11 +929,12 @@ public class AgentProviderService extends AbstractProviderService implements
                                                   .extractRole(container));
       if (role != null) {
         String roleName = role.name;
-        String label = getContainerLabel(container, roleName, role.group);
+        String roleGroup = role.group;
+        String label = getContainerLabel(container, roleName, roleGroup);
         log.info("Rebuilding in-memory: container {} in role {} in cluster {}",
                  container.getId(), roleName, applicationId);
         getComponentStatuses().put(label,
-            new ComponentInstanceState(roleName, container.getId(),
+            new ComponentInstanceState(roleGroup, container.getId(),
                                        applicationId));
       } else {
         log.warn("Role not found for container {} in cluster {}",
@@ -983,7 +1055,7 @@ public class AgentProviderService extends AbstractProviderService implements
 
     StateAccessForProviders accessor = getAmState();
     CommandScript cmdScript = getScriptPathForMasterPackage(roleGroup);
-    List<ComponentCommand> commands = getMetaInfo().getApplicationComponent(roleGroup).getCommands();
+    List<ComponentCommand> commands = getApplicationComponent(roleGroup).getCommands();
 
     if (!isDockerContainer(roleGroup) && !isYarnDockerContainer(roleGroup)
         && (cmdScript == null || cmdScript.getScript() == null)
@@ -1261,7 +1333,7 @@ public class AgentProviderService extends AbstractProviderService implements
   }
 
   private boolean isDockerContainer(String roleGroup) {
-    String type = getMetaInfo().getApplicationComponent(roleGroup).getType();
+    String type = getApplicationComponent(roleGroup).getType();
     if (SliderUtils.isSet(type)) {
       return type.toLowerCase().equals(SliderUtils.DOCKER) || type.toLowerCase().equals(SliderUtils.DOCKER_YARN);
     }
@@ -1269,7 +1341,7 @@ public class AgentProviderService extends AbstractProviderService implements
   }
 
   private boolean isYarnDockerContainer(String roleGroup) {
-    String type = getMetaInfo().getApplicationComponent(roleGroup).getType();
+    String type = getApplicationComponent(roleGroup).getType();
     if (SliderUtils.isSet(type)) {
       return type.toLowerCase().equals(SliderUtils.DOCKER_YARN);
     }
@@ -1393,23 +1465,21 @@ public class AgentProviderService extends AbstractProviderService implements
       throw new IOException(e);
     }
 
-    for (ConfigFile configFile : getMetaInfo()
-        .getComponentConfigFiles(client.getName())) {
+    for (ConfigFile configFile : getMetaInfo().getComponentConfigFiles(client.getName())) {
       addNamedConfiguration(configFile.getDictionaryName(),
           appConf.getGlobalOptions().options, configurations, tokens, null,
-          client.getName());
+          client.getName(), client.getName());
       if (appConf.getComponent(client.getName()) != null) {
         addNamedConfiguration(configFile.getDictionaryName(),
             appConf.getComponent(client.getName()).options, configurations,
-            tokens, null, client.getName());
+            tokens, null, client.getName(), client.getName());
       }
     }
 
     //do a final replacement of re-used configs
     dereferenceAllConfigs(configurations);
 
-    for (ConfigFile configFile : getMetaInfo()
-        .getComponentConfigFiles(client.getName())) {
+    for (ConfigFile configFile : getMetaInfo().getComponentConfigFiles(client.getName())) {
       ConfigFormat configFormat = ConfigFormat.resolve(configFile.getType());
 
       Map<String, String> config = configurations.get(configFile.getDictionaryName());
@@ -1525,9 +1595,23 @@ public class AgentProviderService extends AbstractProviderService implements
     return workFolderExports;
   }
 
-  @VisibleForTesting
   protected Metainfo getMetaInfo() {
-    return this.metaInfo;
+    return getMetaInfo(null);
+  }
+
+  @VisibleForTesting
+  protected Metainfo getMetaInfo(String roleGroup) {
+    String mapKey = DEFAULT_METAINFO_MAP_KEY;
+    if (roleGroup != null) {
+      ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
+      mapKey = appConf.getComponentOpt(roleGroup, ROLE_PREFIX,
+          DEFAULT_METAINFO_MAP_KEY);
+    }
+    MetainfoHolder mh = this.metaInfoMap.get(mapKey);
+    if (mh == null) {
+      return null;
+    }
+    return mh.metaInfo;
   }
 
   @VisibleForTesting
@@ -1597,8 +1681,11 @@ public class AgentProviderService extends AbstractProviderService implements
     return defaultConfigMap;
   }
 
-  protected Map<String, DefaultConfig> getDefaultConfigs() {
-    return defaultConfigs;
+  protected Map<String, DefaultConfig> getDefaultConfigs(String roleGroup) {
+    ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
+    String mapKey = appConf.getComponentOpt(roleGroup, ROLE_PREFIX,
+        DEFAULT_METAINFO_MAP_KEY);
+    return metaInfoMap.get(mapKey).defaultConfigs;
   }
 
   private int getHeartbeatMonitorInterval() {
@@ -1764,9 +1851,9 @@ public class AgentProviderService extends AbstractProviderService implements
         log.info("Status report: {}", status.toString());
 
         if (status.getConfigs() != null) {
-          Application application = getMetaInfo().getApplication();
+          Application application = getMetaInfo(componentGroup).getApplication();
 
-          if ((!canAnyMasterPublishConfig() || canPublishConfig(componentGroup)) &&
+          if ((!canAnyMasterPublishConfig(componentGroup) || canPublishConfig(componentGroup)) &&
               !getAmState().getAppConfSnapshot().getComponentOptBool(
                   componentGroup, AgentKeys.AM_CONFIG_GENERATION, false)) {
             // If no Master can explicitly publish then publish if its a master
@@ -1914,7 +2001,7 @@ public class AgentProviderService extends AbstractProviderService implements
     String hostNamePattern = "${THIS_HOST}";
     Map<String, String> toPublish = new HashMap<String, String>();
 
-    Application application = getMetaInfo().getApplication();
+    Application application = getMetaInfo(componentGroup).getApplication();
     for (Component component : application.getComponents()) {
       if (component.getName().equals(componentGroup)) {
         if (component.getComponentExports().size() > 0) {
@@ -1965,8 +2052,8 @@ public class AgentProviderService extends AbstractProviderService implements
     String portVarFormat = "${site.%s}";
     String hostNamePattern = "${" + compGroup + "_HOST}";
 
-    List<ExportGroup> appExportGroups = getMetaInfo().getApplication().getExportGroups();
-    Component component = getMetaInfo().getApplicationComponent(compGroup);
+    List<ExportGroup> appExportGroups = getMetaInfo(compGroup).getApplication().getExportGroups();
+    Component component = getApplicationComponent(compGroup);
     if (component != null && SliderUtils.isSet(component.getCompExports())
         && SliderUtils.isNotEmpty(appExportGroups)) {
 
@@ -2068,7 +2155,11 @@ public class AgentProviderService extends AbstractProviderService implements
    * @return the component entry or null for no match
    */
   protected Component getApplicationComponent(String roleGroup) {
-    return getMetaInfo().getApplicationComponent(roleGroup);
+    Metainfo metainfo = getMetaInfo(roleGroup);
+    if (metainfo == null) {
+      return null;
+    }
+    return metainfo.getApplicationComponent(roleGroup);
   }
 
   /**
@@ -2137,9 +2228,9 @@ public class AgentProviderService extends AbstractProviderService implements
    *
    * @return true if the condition holds
    */
-  protected boolean canAnyMasterPublishConfig() {
+  protected boolean canAnyMasterPublishConfig(String roleGroup) {
     if (canAnyMasterPublish == null) {
-      Application application = getMetaInfo().getApplication();
+      Application application = getMetaInfo(roleGroup).getApplication();
       if (application == null) {
         log.error("Malformed app definition: Expect application as root element in the metainfo.xml");
       } else {
@@ -2214,7 +2305,7 @@ public class AgentProviderService extends AbstractProviderService implements
     cmd.setPkg(pkg);
     Map<String, String> hostLevelParams = new TreeMap<String, String>();
     hostLevelParams.put(JAVA_HOME, appConf.getGlobalOptions().getOption(JAVA_HOME, getJDKDir()));
-    hostLevelParams.put(PACKAGE_LIST, getPackageList());
+    hostLevelParams.put(PACKAGE_LIST, getPackageList(roleGroup));
     hostLevelParams.put(CONTAINER_ID, containerId);
     cmd.setHostLevelParams(hostLevelParams);
 
@@ -2263,7 +2354,7 @@ public class AgentProviderService extends AbstractProviderService implements
     cmd.setComponentName(roleName);
     cmd.setRole(roleName);
     Map<String, String> hostLevelParams = new TreeMap<String, String>();
-    hostLevelParams.put(PACKAGE_LIST, getPackageList());
+    hostLevelParams.put(PACKAGE_LIST, getPackageList(roleGroup));
     hostLevelParams.put(CONTAINER_ID, containerId);
     cmd.setHostLevelParams(hostLevelParams);
 
@@ -2283,7 +2374,7 @@ public class AgentProviderService extends AbstractProviderService implements
     configurations.get("global").put("exec_cmd", effectiveCommand.getExec());
 
     cmd.setHostname(getClusterInfoPropertyValue(StatusKeys.INFO_AM_HOSTNAME));
-    cmd.addContainerDetails(roleGroup, getMetaInfo());
+    cmd.addContainerDetails(roleGroup, getMetaInfo(roleGroup));
 
     Map<String, String> dockerConfig = new HashMap<String, String>();
     if(isYarnDockerContainer(roleGroup)){
@@ -2366,8 +2457,8 @@ public class AgentProviderService extends AbstractProviderService implements
     }
   }
 
-  private String getPackageList() {
-    return getPackageListFromApplication(getMetaInfo().getApplication());
+  private String getPackageList(String roleGroup) {
+    return getPackageListFromApplication(getMetaInfo(roleGroup).getApplication());
   }
 
   private void prepareExecutionCommand(ExecutionCommand cmd) {
@@ -2532,7 +2623,7 @@ public class AgentProviderService extends AbstractProviderService implements
   private String getConfigFromMetaInfoWithAppConfigOverriding(String roleGroup,
       String configName){
     ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
-    String containerName = getMetaInfo().getApplicationComponent(roleGroup)
+    String containerName = getApplicationComponent(roleGroup)
         .getDockerContainers().get(0).getName();
     String composedConfigName = null;
     String appConfigValue = null;
@@ -2673,7 +2764,7 @@ public class AgentProviderService extends AbstractProviderService implements
     
     cmd.setConfigurations(configurations);
    // configurations.get("global").put("exec_cmd", startCommand.getExec());
-    cmd.addContainerDetails(roleGroup, getMetaInfo());
+    cmd.addContainerDetails(roleGroup, getMetaInfo(roleGroup));
 
     log.info("Docker- command: {}", cmd.toString());
 
@@ -2683,7 +2774,7 @@ public class AgentProviderService extends AbstractProviderService implements
   private String getConfigFromMetaInfo(String roleGroup, String configName) {
     String result = null;
 
-    List<DockerContainer> containers = getMetaInfo().getApplicationComponent(
+    List<DockerContainer> containers = getApplicationComponent(
         roleGroup).getDockerContainers();// to support multi container per
                                              // component later
     log.debug("Docker- containers metainfo: {}", containers.toString());
@@ -2985,10 +3076,11 @@ public class AgentProviderService extends AbstractProviderService implements
 
     for (String configType : configs) {
       addNamedConfiguration(configType, appConf.getGlobalOptions().options,
-                            configurations, tokens, containerId, roleName);
+                            configurations, tokens, containerId, roleName,
+                            roleGroup);
       if (appConf.getComponent(roleGroup) != null) {
         addNamedConfiguration(configType, appConf.getComponent(roleGroup).options,
-            configurations, tokens, containerId, roleName);
+            configurations, tokens, containerId, roleName, roleGroup);
       }
     }
 
@@ -3058,15 +3150,32 @@ public class AgentProviderService extends AbstractProviderService implements
     tokens.put("${NN_HOST}", URI.create(nnuri).getHost());
     tokens.put("${ZK_HOST}", appConf.get(OptionKeys.ZOOKEEPER_HOSTS));
     tokens.put("${DEFAULT_ZK_PATH}", appConf.get(OptionKeys.ZOOKEEPER_PATH));
+    String prefix = appConf.getComponentOpt(componentGroup, ROLE_PREFIX,
+        null);
+    String dataDirSuffix = "";
+    if (prefix == null) {
+      prefix = "";
+    } else {
+      dataDirSuffix = "_" + SliderUtils.trimPrefix(prefix);
+    }
     tokens.put("${DEFAULT_DATA_DIR}", getAmState()
         .getInternalsSnapshot()
         .getGlobalOptions()
-        .getMandatoryOption(InternalKeys.INTERNAL_DATA_DIR_PATH));
+        .getMandatoryOption(InternalKeys.INTERNAL_DATA_DIR_PATH) + dataDirSuffix);
     tokens.put("${JAVA_HOME}", appConf.get(AgentKeys.JAVA_HOME));
     tokens.put("${COMPONENT_NAME}", componentName);
+    tokens.put("${COMPONENT_NAME.lc}", componentName.toLowerCase());
+    tokens.put("${COMPONENT_PREFIX}", prefix);
+    tokens.put("${COMPONENT_PREFIX.lc}", prefix.toLowerCase());
     if (!componentName.equals(componentGroup) && componentName.startsWith(componentGroup)) {
       tokens.put("${COMPONENT_ID}", componentName.substring(componentGroup.length()));
     }
+    tokens.put("${CLUSTER_NAME}", getClusterName());
+    tokens.put("${CLUSTER_NAME.lc}", getClusterName().toLowerCase());
+    tokens.put("${APP_NAME}", getClusterName());
+    tokens.put("${APP_NAME.lc}", getClusterName().toLowerCase());
+    tokens.put("${APP_COMPONENT_NAME}", componentName);
+    tokens.put("${APP_COMPONENT_NAME.lc}", componentName.toLowerCase());
     return tokens;
   }
 
@@ -3091,12 +3200,12 @@ public class AgentProviderService extends AbstractProviderService implements
     List<String> configList = new ArrayList<String>();
     configList.add(GLOBAL_CONFIG_TAG);
 
-    List<ConfigFile> configFiles = getMetaInfo().getApplication().getConfigFiles();
+    List<ConfigFile> configFiles = getMetaInfo(roleGroup).getApplication().getConfigFiles();
     for (ConfigFile configFile : configFiles) {
       log.info("Expecting config type {}.", configFile.getDictionaryName());
       configList.add(configFile.getDictionaryName());
     }
-    for (Component component : getMetaInfo().getApplication().getComponents()) {
+    for (Component component : getMetaInfo(roleGroup).getApplication().getComponents()) {
       if (!component.getName().equals(roleGroup)) {
         continue;
       }
@@ -3121,7 +3230,7 @@ public class AgentProviderService extends AbstractProviderService implements
   private void addNamedConfiguration(String configName, Map<String, String> sourceConfig,
                                      Map<String, Map<String, String>> configurations,
                                      Map<String, String> tokens, String containerId,
-                                     String roleName) {
+                                     String roleName, String roleGroup) {
     Map<String, String> config = new HashMap<String, String>();
     if (configName.equals(GLOBAL_CONFIG_TAG)) {
       addDefaultGlobalConfig(config, containerId, roleName);
@@ -3150,9 +3259,9 @@ public class AgentProviderService extends AbstractProviderService implements
     }
 
     //apply defaults only if the key is not present and value is not empty
-    if (getDefaultConfigs().containsKey(configName)) {
+    if (getDefaultConfigs(roleGroup).containsKey(configName)) {
       log.info("Adding default configs for type {}.", configName);
-      for (PropertyInfo defaultConfigProp : getDefaultConfigs().get(configName).getPropertyInfos()) {
+      for (PropertyInfo defaultConfigProp : getDefaultConfigs(roleGroup).get(configName).getPropertyInfos()) {
         if (!config.containsKey(defaultConfigProp.getName())) {
           if (!defaultConfigProp.getName().isEmpty() &&
               defaultConfigProp.getValue() != null &&

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86a29d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentUtils.java
index cfcfc5d..23e05a3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentUtils.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.slider.common.tools.SliderFileSystem;
 import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.conf.ConfTreeOperations;
 import org.apache.slider.core.exceptions.BadConfigException;
 import org.apache.slider.providers.agent.application.metadata.AbstractMetainfoParser;
 import org.apache.slider.providers.agent.application.metadata.AddonPackageMetainfoParser;
@@ -35,6 +36,8 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 
+import static org.apache.slider.api.RoleKeys.ROLE_PREFIX;
+
 /**
  *
  */
@@ -131,4 +134,17 @@ public class AgentUtils {
 
     return new DefaultConfigParser().parse(configStream);
   }
+
+  static String getMetainfoComponentName(String roleGroup,
+      ConfTreeOperations appConf) throws BadConfigException {
+    String prefix = appConf.getComponentOpt(roleGroup, ROLE_PREFIX, null);
+    if (prefix == null) {
+      return roleGroup;
+    }
+    if (!roleGroup.startsWith(prefix)) {
+      throw new BadConfigException("Component " + roleGroup + " doesn't start" +
+          " with prefix " + prefix);
+    }
+    return roleGroup.substring(prefix.length());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86a29d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentCommandOrder.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentCommandOrder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentCommandOrder.java
index 91f1259..4abac7a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentCommandOrder.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentCommandOrder.java
@@ -18,6 +18,8 @@
 
 package org.apache.slider.providers.agent;
 
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.conf.ConfTreeOperations;
 import org.apache.slider.providers.agent.application.metadata.CommandOrder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -25,9 +27,12 @@ import org.slf4j.LoggerFactory;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.slider.api.RoleKeys.ROLE_PREFIX;
+
 /**
  * Stores the command dependency order for all components in a service. <commandOrder>
  * <command>SUPERVISOR-START</command> <requires>NIMBUS-STARTED</requires> </commandOrder> Means, SUPERVISOR START
@@ -39,13 +44,36 @@ public class ComponentCommandOrder {
   private static char SPLIT_CHAR = '-';
   Map<Command, Map<String, List<ComponentState>>> dependencies =
       new HashMap<Command, Map<String, List<ComponentState>>>();
+  Map<String, Collection<String>> prefixRoleMap = new HashMap<>();
+  Map<String, String> rolePrefixMap = new HashMap<>();
+
+  public ComponentCommandOrder() {}
 
-  public ComponentCommandOrder(List<CommandOrder> commandOrders) {
+  public ComponentCommandOrder(List<CommandOrder> commandOrders,
+      ConfTreeOperations resources) {
+    mergeCommandOrders(commandOrders, resources);
+  }
+
+  void mergeCommandOrders(List<CommandOrder> commandOrders,
+      ConfTreeOperations resources) {
+    for (String component : resources.getComponentNames()) {
+      String prefix = SliderUtils.trimPrefix(
+          resources.getComponentOpt(component, ROLE_PREFIX, null));
+      if (prefix != null) {
+        rolePrefixMap.put(component, prefix);
+        if (!prefixRoleMap.containsKey(prefix)) {
+          prefixRoleMap.put(prefix, new HashSet<String>());
+        }
+        prefixRoleMap.get(prefix).add(component);
+      }
+    }
     if (commandOrders != null && commandOrders.size() > 0) {
       for (CommandOrder commandOrder : commandOrders) {
-        ComponentCommand componentCmd = getComponentCommand(commandOrder.getCommand());
+        ComponentCommand componentCmd = getComponentCommand(
+            commandOrder.getCommand(), resources);
         String requires = commandOrder.getRequires();
-        List<ComponentState> requiredStates = parseRequiredStates(requires);
+        List<ComponentState> requiredStates = parseRequiredStates(requires,
+            resources);
         if (requiredStates.size() > 0) {
           Map<String, List<ComponentState>> compDep = dependencies.get(componentCmd.command);
           if (compDep == null) {
@@ -65,7 +93,8 @@ public class ComponentCommandOrder {
     }
   }
 
-  private List<ComponentState> parseRequiredStates(String requires) {
+  private List<ComponentState> parseRequiredStates(String requires,
+      ConfTreeOperations resources) {
     if (requires == null || requires.length() < 2) {
       throw new IllegalArgumentException("Input cannot be null and must contain component and state.");
     }
@@ -73,13 +102,14 @@ public class ComponentCommandOrder {
     String[] componentStates = requires.split(",");
     List<ComponentState> retList = new ArrayList<ComponentState>();
     for (String componentStateStr : componentStates) {
-      retList.add(getComponentState(componentStateStr));
+      retList.add(getComponentState(componentStateStr, resources));
     }
 
     return retList;
   }
 
-  private ComponentCommand getComponentCommand(String compCmdStr) {
+  private ComponentCommand getComponentCommand(String compCmdStr,
+      ConfTreeOperations resources) {
     if (compCmdStr == null || compCmdStr.trim().length() < 2) {
       throw new IllegalArgumentException("Input cannot be null and must contain component and command.");
     }
@@ -92,6 +122,11 @@ public class ComponentCommandOrder {
     String compStr = compCmdStr.substring(0, splitIndex);
     String cmdStr = compCmdStr.substring(splitIndex + 1);
 
+    if (resources.getComponent(compStr) == null && !prefixRoleMap.containsKey(compStr)) {
+      throw new IllegalArgumentException("Component " + compStr + " specified" +
+          " in command order does not exist");
+    }
+
     Command cmd = Command.valueOf(cmdStr);
 
     if (cmd != Command.START) {
@@ -100,7 +135,8 @@ public class ComponentCommandOrder {
     return new ComponentCommand(compStr, cmd);
   }
 
-  private ComponentState getComponentState(String compStStr) {
+  private ComponentState getComponentState(String compStStr,
+      ConfTreeOperations resources) {
     if (compStStr == null || compStStr.trim().length() < 2) {
       throw new IllegalArgumentException("Input cannot be null.");
     }
@@ -113,6 +149,11 @@ public class ComponentCommandOrder {
     String compStr = compStStr.substring(0, splitIndex);
     String stateStr = compStStr.substring(splitIndex + 1);
 
+    if (resources.getComponent(compStr) == null && !prefixRoleMap.containsKey(compStr)) {
+      throw new IllegalArgumentException("Component " + compStr + " specified" +
+          " in command order does not exist");
+    }
+
     State state = State.valueOf(stateStr);
     if (state != State.STARTED && state != State.INSTALLED) {
       throw new IllegalArgumentException("Dependency order can only be specified against STARTED/INSTALLED.");
@@ -123,40 +164,43 @@ public class ComponentCommandOrder {
   // dependency is still on component level, but not package level
   // so use component name to check dependency, not component-package
   public boolean canExecute(String component, Command command, Collection<ComponentInstanceState> currentStates) {
-    boolean canExecute = true;
-    if (dependencies.containsKey(command) && dependencies.get(command).containsKey(component)) {
-      List<ComponentState> required = dependencies.get(command).get(component);
-      for (ComponentState stateToMatch : required) {
-        for (ComponentInstanceState currState : currentStates) {
-          log.debug("Checking schedule {} {} against dependency {} is {}",
-                    component, command, currState.getComponentName(), currState.getState());
-          if (currState.getComponentName().equals(stateToMatch.componentName)) {
-            if (currState.getState() != stateToMatch.state) {
-              if (stateToMatch.state == State.STARTED) {
+    if (!dependencies.containsKey(command)) {
+      return true;
+    }
+    List<ComponentState> required = new ArrayList<>();
+    if (dependencies.get(command).containsKey(component)) {
+      required.addAll(dependencies.get(command).get(component));
+    }
+    String prefix = rolePrefixMap.get(component);
+    if (prefix != null && dependencies.get(command).containsKey(prefix)) {
+      required.addAll(dependencies.get(command).get(prefix));
+    }
+
+    for (ComponentState stateToMatch : required) {
+      for (ComponentInstanceState currState : currentStates) {
+        log.debug("Checking schedule {} {} against dependency {} is {}",
+            component, command, currState.getComponentName(), currState.getState());
+        if (currState.getComponentName().equals(stateToMatch.componentName) ||
+            (prefixRoleMap.containsKey(stateToMatch.componentName) &&
+                prefixRoleMap.get(stateToMatch.componentName).contains(currState.getComponentName()))) {
+          if (currState.getState() != stateToMatch.state) {
+            if (stateToMatch.state == State.STARTED) {
+              log.info("Cannot schedule {} {} as dependency {} is {}",
+                  component, command, currState.getComponentName(), currState.getState());
+              return false;
+            } else {
+              //state is INSTALLED
+              if (currState.getState() != State.STARTING && currState.getState() != State.STARTED) {
                 log.info("Cannot schedule {} {} as dependency {} is {}",
-                         component, command, currState.getComponentName(), currState.getState());
-                canExecute = false;
-              } else {
-                //state is INSTALLED
-                if (currState.getState() != State.STARTING && currState.getState() != State.STARTED) {
-                  log.info("Cannot schedule {} {} as dependency {} is {}",
-                           component, command, currState.getComponentName(), currState.getState());
-                  canExecute = false;
-                }
+                    component, command, currState.getComponentName(), currState.getState());
+                return false;
               }
             }
           }
-          if (!canExecute) {
-            break;
-          }
-        }
-        if (!canExecute) {
-          break;
         }
       }
     }
-
-    return canExecute;
+    return true;
   }
 
   static class ComponentState {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86a29d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
index addb3f7..983b5ba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
@@ -20,6 +20,9 @@ package org.apache.slider.server.appmaster;
 
 import com.codahale.metrics.MetricRegistry;
 import com.codahale.metrics.health.HealthCheckRegistry;
+import com.codahale.metrics.jvm.GarbageCollectorMetricSet;
+import com.codahale.metrics.jvm.MemoryUsageGaugeSet;
+import com.codahale.metrics.jvm.ThreadStatesGaugeSet;
 import com.google.common.base.Preconditions;
 import com.google.protobuf.BlockingService;
 
@@ -879,7 +882,8 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
       Configuration providerConf =
         providerService.loadProviderConfigurationInformation(confDir);
 
-      providerService.initializeApplicationConfiguration(instanceDefinition, fs);
+      providerService.initializeApplicationConfiguration(instanceDefinition,
+          fs, null);
 
       providerService.validateApplicationConfiguration(instanceDefinition,
           confDir,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86a29d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
index 16c2435..49e7b78 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
@@ -1818,6 +1818,11 @@ public class AppState {
         SliderUtils.mergeMapsIgnoreDuplicateKeys(cd.getRole(rolename),
             groupOptions.options);
       }
+      String prefix = instanceDefinition.getAppConfOperations()
+          .getComponentOpt(role.getGroup(), ROLE_PREFIX, null);
+      if (SliderUtils.isSet(prefix)) {
+        cd.setRoleOpt(rolename, ROLE_PREFIX, SliderUtils.trimPrefix(prefix));
+      }
       List<String> instances = instanceMap.get(rolename);
       int nodeCount = instances != null ? instances.size(): 0;
       cd.setRoleOpt(rolename, COMPONENT_INSTANCES,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/86a29d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentResource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentResource.java
index f1e105a..20ef068 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentResource.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentResource.java
@@ -73,7 +73,7 @@ public class AgentResource extends AbstractSliderResource {
   }
 
   @POST
-  @Path("/{agent_name: [a-zA-Z][a-zA-Z_0-9]*}/register")
+  @Path("/{agent_name: [a-zA-Z][a-zA-Z0-9_-]*}/register")
   @Consumes({MediaType.APPLICATION_JSON})
   @Produces({MediaType.APPLICATION_JSON})
   public RegistrationResponse register(Register registration,
@@ -87,7 +87,7 @@ public class AgentResource extends AbstractSliderResource {
   }
 
   @POST
-  @Path("/{agent_name: [a-zA-Z][a-zA-Z_0-9]*}/heartbeat")
+  @Path("/{agent_name: [a-zA-Z][a-zA-Z0-9_-]*}/heartbeat")
   @Consumes(MediaType.APPLICATION_JSON)
   @Produces({MediaType.APPLICATION_JSON})
   public HeartBeatResponse heartbeat(HeartBeat message,


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[33/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/ApplicationLivenessInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/ApplicationLivenessInformation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/ApplicationLivenessInformation.java
new file mode 100644
index 0000000..9879d05
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/ApplicationLivenessInformation.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.api.types;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+/**
+ * Serialized information about liveness
+ * <p>
+ *   If true liveness probes are implemented, this
+ *   datatype can be extended to publish their details.
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class ApplicationLivenessInformation {
+  /** flag set if the cluster is at size */
+  public boolean allRequestsSatisfied;
+
+  /** number of outstanding requests: those needed to satisfy */
+  public int requestsOutstanding;
+
+  /** number of requests submitted to YARN */
+  public int activeRequests;
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder("ApplicationLivenessInformation{");
+    sb.append("allRequestsSatisfied=").append(allRequestsSatisfied);
+    sb.append(", requestsOutstanding=").append(requestsOutstanding);
+    sb.append('}');
+    return sb.toString();
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/ComponentInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/ComponentInformation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/ComponentInformation.java
new file mode 100644
index 0000000..c46a59f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/ComponentInformation.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.api.types;
+
+import org.apache.slider.api.StatusKeys;
+import org.apache.slider.server.appmaster.state.RoleStatus;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Serializable version of component data.
+ * <p>
+ * This is sent in REST calls as a JSON object \u2014but is also marshalled into
+ * a protobuf structure. Look at {@link org.apache.slider.api.proto.RestTypeMarshalling}
+ * for the specifics there.
+ * <p>
+ * This means that if any fields are added here. they must be added to
+ * <code>src/main/proto/SliderClusterMessages.proto</code> and
+ * the protobuf structures rebuilt via a {@code mvn generate-sources -Pcompile-protobuf}
+ *
+ * See also {@link RoleStatus#serialize()}
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+
+public class ComponentInformation {
+
+  public String name;
+  public int priority;
+  public int desired, actual, releasing;
+  public int placementPolicy;
+  public int requested;
+  public int failed, started, startFailed, completed, totalRequested;
+  public int nodeFailed, failedRecently, preempted;
+  public int pendingAntiAffineRequestCount;
+  public boolean isAARequestOutstanding;
+
+  public String failureMessage;
+  public List<String> containers;
+
+  /**
+   * Build the statistics map from the current data
+   * @return a map for use in statistics reports
+   */
+  public Map<String, Integer> buildStatistics() {
+    Map<String, Integer> stats = new HashMap<>();
+    stats.put(StatusKeys.STATISTICS_CONTAINERS_ACTIVE_REQUESTS, requested);
+    stats.put(StatusKeys.STATISTICS_CONTAINERS_ANTI_AFFINE_PENDING, pendingAntiAffineRequestCount);
+    stats.put(StatusKeys.STATISTICS_CONTAINERS_COMPLETED, completed);
+    stats.put(StatusKeys.STATISTICS_CONTAINERS_DESIRED, desired);
+    stats.put(StatusKeys.STATISTICS_CONTAINERS_FAILED, failed);
+    stats.put(StatusKeys.STATISTICS_CONTAINERS_FAILED_NODE, nodeFailed);
+    stats.put(StatusKeys.STATISTICS_CONTAINERS_FAILED_RECENTLY, failedRecently);
+    stats.put(StatusKeys.STATISTICS_CONTAINERS_LIVE, actual);
+    stats.put(StatusKeys.STATISTICS_CONTAINERS_PREEMPTED, preempted);
+    stats.put(StatusKeys.STATISTICS_CONTAINERS_REQUESTED, totalRequested);
+    stats.put(StatusKeys.STATISTICS_CONTAINERS_STARTED, started);
+    stats.put(StatusKeys.STATISTICS_CONTAINERS_START_FAILED, startFailed);
+    return stats;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder("ComponentInformation{");
+    sb.append(", name='").append(name).append('\'');
+    sb.append(", actual=").append(actual);
+    sb.append(", completed=").append(completed);
+    sb.append(", desired=").append(desired);
+    sb.append(", failed=").append(failed);
+    sb.append(", failureMessage='").append(failureMessage).append('\'');
+    sb.append(", placementPolicy=").append(placementPolicy);
+    sb.append(", isAARequestOutstanding=").append(isAARequestOutstanding);
+    sb.append(", pendingAntiAffineRequestCount=").append(pendingAntiAffineRequestCount);
+    sb.append(", priority=").append(priority);
+    sb.append(", releasing=").append(releasing);
+    sb.append(", requested=").append(requested);
+    sb.append(", started=").append(started);
+    sb.append(", startFailed=").append(startFailed);
+    sb.append(", totalRequested=").append(totalRequested);
+    sb.append(", container count='")
+        .append(containers == null ? 0 : containers.size())
+        .append('\'');
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/ContainerInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/ContainerInformation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/ContainerInformation.java
new file mode 100644
index 0000000..6991340
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/ContainerInformation.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.api.types;
+
+import org.apache.hadoop.registry.client.binding.JsonSerDeser;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+/**
+ * Serializable version of component instance data
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class ContainerInformation {
+  
+  public String containerId;
+  public String component;
+  public String appVersion;
+  public Boolean released;
+  public int state;
+  public Integer exitCode;
+  public String diagnostics;
+  public long createTime;
+  public long startTime;
+
+  public String host;
+  public String hostURL;
+  public String placement;
+  /**
+   * What is the tail output from the executed process (or [] if not started
+   * or the log cannot be picked up
+   */
+  public String[] output;
+
+  @Override
+  public String toString() {
+    JsonSerDeser<ContainerInformation> serDeser =
+        new JsonSerDeser<>(
+            ContainerInformation.class);
+    return serDeser.toString(this);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/NodeEntryInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/NodeEntryInformation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/NodeEntryInformation.java
new file mode 100644
index 0000000..8424be2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/NodeEntryInformation.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.api.types;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+/**
+ * Serialized node entry information. Must be kept in sync with the protobuf equivalent.
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class NodeEntryInformation {
+
+  /** incrementing counter of instances that failed */
+  public int failed;
+
+  /** Counter of "failed recently" events. */
+  public int failedRecently;
+
+  /** timestamp of last use */
+  public long lastUsed;
+
+  /** Number of live nodes. */
+  public int live;
+
+  /** incrementing counter of instances that have been pre-empted. */
+  public int preempted;
+
+  /** Priority */
+  public int priority;
+
+  /** instance explicitly requested on this node */
+  public int requested;
+
+  /** number of containers being released off this node */
+  public int releasing;
+
+  /** incrementing counter of instances that failed to start */
+  public int startFailed;
+
+  /** number of starting instances */
+  public int starting;
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "NodeEntryInformation{");
+    sb.append("priority=").append(priority);
+    sb.append(", live=").append(live);
+    sb.append(", requested=").append(requested);
+    sb.append(", releasing=").append(releasing);
+    sb.append(", starting=").append(starting);
+    sb.append(", failed=").append(failed);
+    sb.append(", failedRecently=").append(failedRecently);
+    sb.append(", startFailed=").append(startFailed);
+    sb.append(", preempted=").append(preempted);
+    sb.append(", lastUsed=").append(lastUsed);
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/NodeInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/NodeInformation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/NodeInformation.java
new file mode 100644
index 0000000..4fe5b4c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/NodeInformation.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.api.types;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Serialized node information. Must be kept in sync with the protobuf equivalent.
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class NodeInformation {
+
+  public String hostname;
+  public String state;
+  public String labels;
+  public String rackName;
+  public String httpAddress;
+  public String healthReport;
+  public long lastUpdated;
+  public Map<String, NodeEntryInformation> entries = new HashMap<>();
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+      "NodeInformation{");
+    sb.append("hostname='").append(hostname).append('\'');
+    sb.append(", state='").append(state).append('\'');
+    sb.append(", labels='").append(labels).append('\'');
+    sb.append(", rackName='").append(rackName).append('\'');
+    sb.append(", httpAddress='").append(httpAddress).append('\'');
+    sb.append(", healthReport='").append(healthReport).append('\'');
+    sb.append(", lastUpdated=").append(lastUpdated);
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/NodeInformationList.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/NodeInformationList.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/NodeInformationList.java
new file mode 100644
index 0000000..741523e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/NodeInformationList.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.api.types;
+
+import org.apache.slider.core.persist.JsonSerDeser;
+
+import java.util.ArrayList;
+import java.util.Collection;
+
+public class NodeInformationList extends ArrayList<NodeInformation> {
+  public NodeInformationList() {
+  }
+
+  public NodeInformationList(Collection<? extends NodeInformation> c) {
+    super(c);
+  }
+
+  public NodeInformationList(int initialCapacity) {
+    super(initialCapacity);
+  }
+
+  public static JsonSerDeser<NodeInformationList> createSerializer() {
+    return new JsonSerDeser<>(NodeInformationList.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/PingInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/PingInformation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/PingInformation.java
new file mode 100644
index 0000000..223edca
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/PingInformation.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.api.types;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+/**
+ * Serialized information to/from Ping operations
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class PingInformation {
+  public long time;
+  public String text;
+  public String verb;
+  public String body;
+
+  @Override
+  public String toString() {
+    
+    final StringBuilder sb =
+        new StringBuilder("PingResource{");
+    sb.append("time=").append(time);
+    sb.append(", verb=").append(verb);
+    sb.append(", text='").append(text).append('\'');
+    sb.append(", body='").append(body).append('\'');
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/RoleStatistics.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/RoleStatistics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/RoleStatistics.java
new file mode 100644
index 0000000..c926600
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/RoleStatistics.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.api.types;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+
+/**
+ * Simple role statistics for state views; can be generated by RoleStatus
+ * instances, and aggregated for summary information.
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class RoleStatistics {
+  public long activeAA  = 0L;
+  public long actual = 0L;
+  public long completed = 0L;
+  public long desired = 0L;
+  public long failed = 0L;
+  public long failedRecently = 0L;
+  public long limitsExceeded = 0L;
+  public long nodeFailed = 0L;
+  public long preempted = 0L;
+  public long releasing = 0L;
+  public long requested = 0L;
+  public long started = 0L;
+  public long startFailed = 0L;
+  public long totalRequested = 0L;
+
+  /**
+   * Add another statistics instance
+   * @param that the other value
+   * @return this entry
+   */
+  public RoleStatistics add(final RoleStatistics that) {
+    activeAA += that.activeAA;
+    actual += that.actual;
+    completed += that.completed;
+    desired += that.desired;
+    failed += that.failed;
+    failedRecently += that.failedRecently;
+    limitsExceeded += that.limitsExceeded;
+    nodeFailed += that.nodeFailed;
+    preempted += that.preempted;
+    releasing += that.releasing;
+    requested += that.requested;
+    started += that.started;
+    startFailed += that.totalRequested;
+    totalRequested += that.totalRequested;
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/SliderInstanceDescription.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/SliderInstanceDescription.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/SliderInstanceDescription.java
new file mode 100644
index 0000000..3b95f80
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/types/SliderInstanceDescription.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.api.types;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+
+/**
+ * Description of a slider instance
+ */
+public class SliderInstanceDescription {
+
+  public final String name;
+  public final Path path;
+  public final ApplicationReport applicationReport;
+
+  public SliderInstanceDescription(String name,
+      Path path,
+      ApplicationReport applicationReport) {
+    this.name = name;
+    this.path = path;
+    this.applicationReport = applicationReport;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder("SliderInstanceDescription{");
+    sb.append("name='").append(name).append('\'');
+    sb.append(", path=").append(path);
+    sb.append(", applicationReport: ")
+      .append(applicationReport == null
+              ? "null"
+              : (" id " + applicationReport.getApplicationId()));
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ClientRegistryBinder.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ClientRegistryBinder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ClientRegistryBinder.java
new file mode 100644
index 0000000..da37d11
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ClientRegistryBinder.java
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.client;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.registry.client.api.RegistryConstants;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
+import org.apache.hadoop.registry.client.binding.RegistryTypeUtils;
+import org.apache.hadoop.registry.client.exceptions.InvalidRecordException;
+import org.apache.hadoop.registry.client.impl.zk.RegistryInternalConstants;
+import org.apache.hadoop.registry.client.types.Endpoint;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.registry.client.binding.RegistryPathUtils.encodeForRegistry;
+import static org.apache.hadoop.registry.client.binding.RegistryUtils.convertUsername;
+import static org.apache.hadoop.registry.client.binding.RegistryUtils.getCurrentUsernameUnencoded;
+import static org.apache.hadoop.registry.client.binding.RegistryUtils.servicePath;
+
+/**
+ * Generic code to get the URLs for clients via the registry
+ */
+public class ClientRegistryBinder {
+  private static final Logger log =
+      LoggerFactory.getLogger(ClientRegistryBinder.class);
+
+  private final RegistryOperations operations;
+
+  public ClientRegistryBinder(RegistryOperations operations) {
+    this.operations = operations;
+  }
+
+  /**
+   * Buld the user path -switches to the system path if the user is "".
+   * It also cross-converts the username to ascii via punycode
+   * @param username username or ""
+   * @return the path to the user
+   */
+  public static String homePathForUser(String username) {
+    Preconditions.checkArgument(username != null, "null user");
+
+    // catch recursion
+    if (username.startsWith(RegistryConstants.PATH_USERS)) {
+      return username;
+    }
+
+    if (username.isEmpty()) {
+      return RegistryConstants.PATH_SYSTEM_SERVICES;
+    }
+
+    // convert username to registry name
+    String convertedName = convertUsername(username);
+
+    return RegistryPathUtils.join(RegistryConstants.PATH_USERS,
+        encodeForRegistry(convertedName));
+  }
+
+  /**
+   * Get the current username, before any encoding has been applied.
+   * @return the current user from the kerberos identity, falling back
+   * to the user and/or env variables.
+   */
+  public static String currentUsernameUnencoded() {
+    String env_hadoop_username = System.getenv(
+        RegistryInternalConstants.HADOOP_USER_NAME);
+    return getCurrentUsernameUnencoded(env_hadoop_username);
+  }
+
+  /**
+   * Qualify a user.
+   * <ol>
+   *   <li> <code>"~"</code> maps to user home path home</li>
+   *   <li> <code>"~user"</code> maps to <code>/users/$user</code></li>
+   *   <li> <code>"/"</code> maps to <code>/services/</code></li>
+   * </ol>
+   * @param user the username
+   * @return the base path
+   */
+  public static String qualifyUser(String user) {
+    // qualify the user
+    String t = user.trim();
+    if (t.startsWith("/")) {
+      // already resolved
+      return t;
+    } else if (t.equals("~")) {
+      // self
+      return currentUsernameUnencoded();
+    } else if (t.startsWith("~")) {
+      // another user
+      // convert username to registry name
+      String convertedName = convertUsername(t.substring(1));
+
+      return RegistryPathUtils.join(RegistryConstants.PATH_USERS,
+          encodeForRegistry(convertedName));
+    } else {
+      return "/" + t;
+    }
+  }
+
+  /**
+   * Look up an external REST API
+   * @param user user which will be qualified as per {@link #qualifyUser(String)}
+   * @param serviceClass service class
+   * @param instance instance name
+   * @param api API
+   * @return the API, or an exception is raised.
+   * @throws IOException
+   */
+  public String lookupExternalRestAPI(String user,
+      String serviceClass,
+      String instance,
+      String api)
+      throws IOException {
+    String qualified = qualifyUser(user);
+    String path = servicePath(qualified, serviceClass, instance);
+    String restAPI = resolveExternalRestAPI(api, path);
+    if (restAPI == null) {
+      throw new PathNotFoundException(path + " API " + api);
+    }
+    return restAPI;
+  }
+
+  /**
+   * Resolve a service record then return an external REST API exported it.
+   *
+   * @param api API to resolve
+   * @param path path of the service record
+   * @return null if the record exists but the API is absent or it has no
+   * REST endpoints.
+   * @throws IOException resolution problems, as covered in
+   * {@link RegistryOperations#resolve(String)}
+   */
+  protected String resolveExternalRestAPI(String api, String path) throws
+      IOException {
+    ServiceRecord record = operations.resolve(path);
+    return lookupRestAPI(record, api, true);
+  }
+
+  /**
+   * Look up an external REST API endpoint
+   * @param record service record
+   * @param api URI of api
+   * @param external flag to indicate this is an external record
+   * @return the first endpoint of the implementation, or null if there
+   * is no entry for the API, implementation or it's the wrong type.
+   */
+  public static String lookupRestAPI(ServiceRecord record,
+      String api, boolean external) throws InvalidRecordException {
+    try {
+      String url = null;
+      Endpoint endpoint = getEndpoint(record, api, external);
+      List<String> addresses =
+          RegistryTypeUtils.retrieveAddressesUriType(endpoint);
+      if (addresses != null && !addresses.isEmpty()) {
+        url = addresses.get(0);
+      }
+      return url;
+    } catch (InvalidRecordException e) {
+      log.debug("looking for API {}", api, e);
+      return null;
+    }
+  }
+
+  /**
+   * Get an endpont by API
+   * @param record service record
+   * @param api API
+   * @param external flag to indicate this is an external record
+   * @return the endpoint or null
+   */
+  public static Endpoint getEndpoint(ServiceRecord record,
+      String api,
+      boolean external) {
+    return external ? record.getExternalEndpoint(api)
+                    : record.getInternalEndpoint(api);
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ClientUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ClientUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ClientUtils.java
new file mode 100644
index 0000000..c3ccb1d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ClientUtils.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.client;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
+import org.apache.hadoop.registry.client.exceptions.NoRecordException;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.NotFoundException;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.core.registry.docstore.ConfigFormat;
+import org.apache.slider.core.registry.docstore.PublishedConfigSet;
+import org.apache.slider.core.registry.docstore.PublishedConfiguration;
+import org.apache.slider.core.registry.docstore.PublishedConfigurationOutputter;
+import org.apache.slider.core.registry.retrieve.RegistryRetriever;
+
+import java.io.File;
+import java.io.IOException;
+
+import static org.apache.hadoop.registry.client.binding.RegistryUtils.currentUser;
+import static org.apache.hadoop.registry.client.binding.RegistryUtils.servicePath;
+
+public class ClientUtils {
+  public static ServiceRecord lookupServiceRecord(RegistryOperations rops,
+      String user, String name) throws IOException, SliderException {
+    return lookupServiceRecord(rops, user, null, name);
+  }
+
+  public static ServiceRecord lookupServiceRecord(RegistryOperations rops,
+      String user, String type, String name) throws IOException,
+      SliderException {
+    if (StringUtils.isEmpty(user)) {
+      user = currentUser();
+    } else {
+      user = RegistryPathUtils.encodeForRegistry(user);
+    }
+    if (StringUtils.isEmpty(type)) {
+      type = SliderKeys.APP_TYPE;
+    }
+
+    String path = servicePath(user, type, name);
+    return resolve(rops, path);
+  }
+
+  public static ServiceRecord resolve(RegistryOperations rops, String path)
+      throws IOException, SliderException {
+    try {
+      return rops.resolve(path);
+    } catch (PathNotFoundException | NoRecordException e) {
+      throw new NotFoundException(e.getPath().toString(), e);
+    }
+  }
+
+  public static PublishedConfiguration getConfigFromRegistry(
+      RegistryOperations rops, Configuration configuration,
+      String configName, String appName, String user, boolean external)
+      throws IOException, SliderException {
+    ServiceRecord instance = lookupServiceRecord(rops, user, appName);
+
+    RegistryRetriever retriever = new RegistryRetriever(configuration, instance);
+    PublishedConfigSet configurations = retriever.getConfigurations(external);
+
+    PublishedConfiguration published = retriever.retrieveConfiguration(
+        configurations, configName, external);
+    return published;
+  }
+
+  public static String saveOrReturnConfig(PublishedConfiguration published,
+      String format, File destPath, String fileName)
+      throws BadCommandArgumentsException, IOException {
+    ConfigFormat configFormat = ConfigFormat.resolve(format);
+    if (configFormat == null) {
+      throw new BadCommandArgumentsException(
+          "Unknown/Unsupported format %s ", format);
+    }
+    PublishedConfigurationOutputter outputter =
+        PublishedConfigurationOutputter.createOutputter(configFormat,
+            published);
+    boolean print = destPath == null;
+    if (!print) {
+      if (destPath.isDirectory()) {
+        // creating it under a directory
+        destPath = new File(destPath, fileName);
+      }
+      outputter.save(destPath);
+      return null;
+    } else {
+      return outputter.asString();
+    }
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[19/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
new file mode 100644
index 0000000..4ffae7c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
@@ -0,0 +1,3212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers.agent;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
+import org.apache.hadoop.registry.client.types.Endpoint;
+import org.apache.hadoop.registry.client.types.ProtocolTypes;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.hadoop.registry.client.types.yarn.PersistencePolicies;
+import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.slider.api.ClusterDescription;
+import org.apache.slider.api.ClusterNode;
+import org.apache.slider.api.InternalKeys;
+import org.apache.slider.api.OptionKeys;
+import org.apache.slider.api.ResourceKeys;
+import org.apache.slider.api.StatusKeys;
+import org.apache.slider.common.SliderExitCodes;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.SliderXmlConfKeys;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.core.exceptions.NoSuchNodeException;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.core.launch.CommandLineBuilder;
+import org.apache.slider.core.launch.ContainerLauncher;
+import org.apache.slider.core.registry.docstore.ConfigFormat;
+import org.apache.slider.core.registry.docstore.ConfigUtils;
+import org.apache.slider.core.registry.docstore.ExportEntry;
+import org.apache.slider.core.registry.docstore.PublishedConfiguration;
+import org.apache.slider.core.registry.docstore.PublishedConfigurationOutputter;
+import org.apache.slider.core.registry.docstore.PublishedExports;
+import org.apache.slider.core.registry.info.CustomRegistryConstants;
+import org.apache.slider.providers.AbstractProviderService;
+import org.apache.slider.providers.MonitorDetail;
+import org.apache.slider.providers.ProviderCore;
+import org.apache.slider.providers.ProviderRole;
+import org.apache.slider.providers.ProviderUtils;
+import org.apache.slider.providers.agent.application.metadata.AbstractComponent;
+import org.apache.slider.providers.agent.application.metadata.Application;
+import org.apache.slider.providers.agent.application.metadata.CommandScript;
+import org.apache.slider.providers.agent.application.metadata.Component;
+import org.apache.slider.providers.agent.application.metadata.ComponentCommand;
+import org.apache.slider.providers.agent.application.metadata.ComponentExport;
+import org.apache.slider.providers.agent.application.metadata.ComponentsInAddonPackage;
+import org.apache.slider.providers.agent.application.metadata.ConfigFile;
+import org.apache.slider.providers.agent.application.metadata.DefaultConfig;
+import org.apache.slider.providers.agent.application.metadata.DockerContainer;
+import org.apache.slider.providers.agent.application.metadata.Export;
+import org.apache.slider.providers.agent.application.metadata.ExportGroup;
+import org.apache.slider.providers.agent.application.metadata.Metainfo;
+import org.apache.slider.providers.agent.application.metadata.OSPackage;
+import org.apache.slider.providers.agent.application.metadata.OSSpecific;
+import org.apache.slider.providers.agent.application.metadata.Package;
+import org.apache.slider.providers.agent.application.metadata.PropertyInfo;
+import org.apache.slider.server.appmaster.actions.ProviderReportedContainerLoss;
+import org.apache.slider.server.appmaster.actions.RegisterComponentInstance;
+import org.apache.slider.server.appmaster.state.ContainerPriority;
+import org.apache.slider.server.appmaster.state.RoleInstance;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+import org.apache.slider.server.appmaster.web.rest.agent.AgentCommandType;
+import org.apache.slider.server.appmaster.web.rest.agent.AgentRestOperations;
+import org.apache.slider.server.appmaster.web.rest.agent.CommandReport;
+import org.apache.slider.server.appmaster.web.rest.agent.ComponentStatus;
+import org.apache.slider.server.appmaster.web.rest.agent.ExecutionCommand;
+import org.apache.slider.server.appmaster.web.rest.agent.HeartBeat;
+import org.apache.slider.server.appmaster.web.rest.agent.HeartBeatResponse;
+import org.apache.slider.server.appmaster.web.rest.agent.Register;
+import org.apache.slider.server.appmaster.web.rest.agent.RegistrationResponse;
+import org.apache.slider.server.appmaster.web.rest.agent.RegistrationStatus;
+import org.apache.slider.server.appmaster.web.rest.agent.StatusCommand;
+import org.apache.slider.server.services.security.CertificateManager;
+import org.apache.slider.server.services.security.SecurityStore;
+import org.apache.slider.server.services.security.StoresGenerator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Scanner;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.SLIDER_PATH_AGENTS;
+
+/**
+ * This class implements the server-side logic for application deployment through Slider application package
+ */
+public class AgentProviderService extends AbstractProviderService implements
+    ProviderCore,
+    AgentKeys,
+    SliderKeys, AgentRestOperations {
+
+
+  protected static final Logger log =
+      LoggerFactory.getLogger(AgentProviderService.class);
+  private static final ProviderUtils providerUtils = new ProviderUtils(log);
+  private static final String LABEL_MAKER = "___";
+  private static final String CONTAINER_ID = "container_id";
+  private static final String GLOBAL_CONFIG_TAG = "global";
+  private static final String LOG_FOLDERS_TAG = "LogFolders";
+  private static final String HOST_FOLDER_FORMAT = "%s:%s";
+  private static final String CONTAINER_LOGS_TAG = "container_log_dirs";
+  private static final String CONTAINER_PWDS_TAG = "container_work_dirs";
+  private static final String COMPONENT_TAG = "component";
+  private static final String APPLICATION_TAG = "application";
+  private static final String COMPONENT_DATA_TAG = "ComponentInstanceData";
+  private static final String SHARED_PORT_TAG = "SHARED";
+  private static final String PER_CONTAINER_TAG = "{PER_CONTAINER}";
+  private static final int MAX_LOG_ENTRIES = 40;
+  private static final int DEFAULT_HEARTBEAT_MONITOR_INTERVAL = 60 * 1000;
+
+  private final Object syncLock = new Object();
+  private final ComponentTagProvider tags = new ComponentTagProvider();
+  private int heartbeatMonitorInterval = 0;
+  private AgentClientProvider clientProvider;
+  private AtomicInteger taskId = new AtomicInteger(0);
+  private volatile Metainfo metaInfo = null;
+  private SliderFileSystem fileSystem = null;
+  private Map<String, DefaultConfig> defaultConfigs = null;
+  private ComponentCommandOrder commandOrder = null;
+  private HeartbeatMonitor monitor;
+  private Boolean canAnyMasterPublish = null;
+  private AgentLaunchParameter agentLaunchParameter = null;
+  private String clusterName = null;
+  private boolean isInUpgradeMode;
+  private Set<String> upgradeContainers = new HashSet<String>();
+  private boolean appStopInitiated;
+
+  private final Map<String, ComponentInstanceState> componentStatuses =
+      new ConcurrentHashMap<String, ComponentInstanceState>();
+  private final Map<String, Map<String, String>> componentInstanceData =
+      new ConcurrentHashMap<String, Map<String, String>>();
+  private final Map<String, Map<String, List<ExportEntry>>> exportGroups =
+      new ConcurrentHashMap<String, Map<String, List<ExportEntry>>>();
+  private final Map<String, Map<String, String>> allocatedPorts =
+      new ConcurrentHashMap<String, Map<String, String>>();
+  private final Map<String, Metainfo> packageMetainfo = 
+      new ConcurrentHashMap<String, Metainfo>();
+
+  private final Map<String, ExportEntry> logFolderExports =
+      Collections.synchronizedMap(new LinkedHashMap<String, ExportEntry>(MAX_LOG_ENTRIES, 0.75f, false) {
+        protected boolean removeEldestEntry(Map.Entry eldest) {
+          return size() > MAX_LOG_ENTRIES;
+        }
+      });
+  private final Map<String, ExportEntry> workFolderExports =
+      Collections.synchronizedMap(new LinkedHashMap<String, ExportEntry>(MAX_LOG_ENTRIES, 0.75f, false) {
+        protected boolean removeEldestEntry(Map.Entry eldest) {
+          return size() > MAX_LOG_ENTRIES;
+        }
+      });
+  private final Map<String, Set<String>> containerExportsMap =
+      new HashMap<String, Set<String>>();
+
+  /**
+   * Create an instance of AgentProviderService
+   */
+  public AgentProviderService() {
+    super("AgentProviderService");
+    setAgentRestOperations(this);
+    setHeartbeatMonitorInterval(DEFAULT_HEARTBEAT_MONITOR_INTERVAL);
+  }
+
+  @Override
+  public String getHumanName() {
+    return "Slider Agent";
+  }
+
+  @Override
+  public List<ProviderRole> getRoles() {
+    return AgentRoles.getRoles();
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    clientProvider = new AgentClientProvider(conf);
+  }
+
+  @Override
+  public Configuration loadProviderConfigurationInformation(File confDir) throws
+      BadCommandArgumentsException,
+      IOException {
+    return new Configuration(false);
+  }
+
+  @Override
+  public void validateInstanceDefinition(AggregateConf instanceDefinition)
+      throws
+      SliderException {
+    clientProvider.validateInstanceDefinition(instanceDefinition, null);
+
+    ConfTreeOperations resources =
+        instanceDefinition.getResourceOperations();
+
+    Set<String> names = resources.getComponentNames();
+    names.remove(SliderKeys.COMPONENT_AM);
+    for (String name : names) {
+      Component componentDef = getMetaInfo().getApplicationComponent(name);
+      if (componentDef == null) {
+        throw new BadConfigException(
+            "Component %s is not a member of application.", name);
+      }
+
+      MapOperations componentConfig = resources.getMandatoryComponent(name);
+      int count =
+          componentConfig.getMandatoryOptionInt(ResourceKeys.COMPONENT_INSTANCES);
+      int definedMinCount = componentDef.getMinInstanceCountInt();
+      int definedMaxCount = componentDef.getMaxInstanceCountInt();
+      if (count < definedMinCount || count > definedMaxCount) {
+        throw new BadConfigException("Component %s, %s value %d out of range. "
+                                     + "Expected minimum is %d and maximum is %d",
+                                     name,
+                                     ResourceKeys.COMPONENT_INSTANCES,
+                                     count,
+                                     definedMinCount,
+                                     definedMaxCount);
+      }
+    }
+  }
+
+  // Reads the metainfo.xml in the application package and loads it
+  private void buildMetainfo(AggregateConf instanceDefinition,
+                             SliderFileSystem fileSystem) throws IOException, SliderException {
+    String appDef = SliderUtils.getApplicationDefinitionPath(instanceDefinition
+        .getAppConfOperations());
+
+    if (metaInfo == null) {
+      synchronized (syncLock) {
+        if (metaInfo == null) {
+          this.fileSystem = fileSystem;
+          readAndSetHeartbeatMonitoringInterval(instanceDefinition);
+          initializeAgentDebugCommands(instanceDefinition);
+
+          metaInfo = getApplicationMetainfo(fileSystem, appDef, false);
+          log.info("Master package metainfo: {}", metaInfo.toString());
+          if (metaInfo == null || metaInfo.getApplication() == null) {
+            log.error("metainfo.xml is unavailable or malformed at {}.", appDef);
+            throw new SliderException(
+                "metainfo.xml is required in app package.");
+          }
+          commandOrder = new ComponentCommandOrder(metaInfo.getApplication().getCommandOrders());
+          defaultConfigs = initializeDefaultConfigs(fileSystem, appDef, metaInfo);
+          monitor = new HeartbeatMonitor(this, getHeartbeatMonitorInterval());
+          monitor.start();
+
+          // build a map from component to metainfo
+          String addonAppDefString = instanceDefinition.getAppConfOperations()
+              .getGlobalOptions().getOption(AgentKeys.ADDONS, null);
+          log.debug("All addon appdefs: {}", addonAppDefString);
+          if (addonAppDefString != null) {
+            Scanner scanner = new Scanner(addonAppDefString).useDelimiter(",");
+            while (scanner.hasNext()) {
+              String addonAppDef = scanner.next();
+              String addonAppDefPath = instanceDefinition
+                  .getAppConfOperations().getGlobalOptions().get(addonAppDef);
+              log.debug("Addon package {} is stored at: {}", addonAppDef
+                  + addonAppDefPath);
+              Metainfo addonMetaInfo = getApplicationMetainfo(fileSystem,
+                  addonAppDefPath, true);
+              addonMetaInfo.validate();
+              packageMetainfo.put(addonMetaInfo.getApplicationPackage()
+                  .getName(), addonMetaInfo);
+            }
+            log.info("Metainfo map for master and addon: {}",
+                packageMetainfo.toString());
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public void initializeApplicationConfiguration(
+      AggregateConf instanceDefinition, SliderFileSystem fileSystem)
+      throws IOException, SliderException {
+    buildMetainfo(instanceDefinition, fileSystem);
+  }
+
+  @Override
+  public void buildContainerLaunchContext(ContainerLauncher launcher,
+                                          AggregateConf instanceDefinition,
+                                          Container container,
+                                          ProviderRole providerRole,
+                                          SliderFileSystem fileSystem,
+                                          Path generatedConfPath,
+                                          MapOperations resourceComponent,
+                                          MapOperations appComponent,
+                                          Path containerTmpDirPath) throws
+      IOException,
+      SliderException {
+    
+    String roleName = providerRole.name;
+    String roleGroup = providerRole.group;
+    String appDef = SliderUtils.getApplicationDefinitionPath(instanceDefinition
+        .getAppConfOperations());
+
+    initializeApplicationConfiguration(instanceDefinition, fileSystem);
+
+    log.info("Build launch context for Agent");
+    log.debug(instanceDefinition.toString());
+    
+    //if we are launching docker based app on yarn, then we need to pass docker image
+    if (isYarnDockerContainer(roleGroup)) {
+      launcher.setYarnDockerMode(true);
+      launcher.setDockerImage(getConfigFromMetaInfo(roleGroup, "image"));
+      launcher.setRunPrivilegedContainer(getConfigFromMetaInfo(roleGroup, "runPriviledgedContainer"));
+      launcher
+          .setYarnContainerMountPoints(getConfigFromMetaInfoWithAppConfigOverriding(
+              roleGroup, "yarn.container.mount.points"));
+    }
+
+    // Set the environment
+    launcher.putEnv(SliderUtils.buildEnvMap(appComponent,
+        getStandardTokenMap(getAmState().getAppConfSnapshot(), roleName, roleGroup)));
+
+    String workDir = ApplicationConstants.Environment.PWD.$();
+    launcher.setEnv("AGENT_WORK_ROOT", workDir);
+    log.info("AGENT_WORK_ROOT set to {}", workDir);
+    String logDir = ApplicationConstants.LOG_DIR_EXPANSION_VAR;
+    launcher.setEnv("AGENT_LOG_ROOT", logDir);
+    log.info("AGENT_LOG_ROOT set to {}", logDir);
+    if (System.getenv(HADOOP_USER_NAME) != null) {
+      launcher.setEnv(HADOOP_USER_NAME, System.getenv(HADOOP_USER_NAME));
+    }
+    // for 2-Way SSL
+    launcher.setEnv(SLIDER_PASSPHRASE, instanceDefinition.getPassphrase());
+    //add english env
+    launcher.setEnv("LANG", "en_US.UTF-8");
+    launcher.setEnv("LC_ALL", "en_US.UTF-8");
+    launcher.setEnv("LANGUAGE", "en_US.UTF-8");
+
+    //local resources
+
+    // TODO: Should agent need to support App Home
+    String scriptPath = new File(AgentKeys.AGENT_MAIN_SCRIPT_ROOT, AgentKeys.AGENT_MAIN_SCRIPT).getPath();
+    String appHome = instanceDefinition.getAppConfOperations().
+        getGlobalOptions().get(AgentKeys.PACKAGE_PATH);
+    if (SliderUtils.isSet(appHome)) {
+      scriptPath = new File(appHome, AgentKeys.AGENT_MAIN_SCRIPT).getPath();
+    }
+
+    // set PYTHONPATH
+    List<String> pythonPaths = new ArrayList<String>();
+    pythonPaths.add(AgentKeys.AGENT_MAIN_SCRIPT_ROOT);
+    pythonPaths.add(AgentKeys.AGENT_JINJA2_ROOT);
+    String pythonPath = StringUtils.join(File.pathSeparator, pythonPaths);
+    launcher.setEnv(PYTHONPATH, pythonPath);
+    log.info("PYTHONPATH set to {}", pythonPath);
+
+    Path agentImagePath = null;
+    String agentImage = instanceDefinition.getInternalOperations().
+        get(InternalKeys.INTERNAL_APPLICATION_IMAGE_PATH);
+    if (SliderUtils.isUnset(agentImage)) {
+      agentImagePath =
+          new Path(new Path(new Path(instanceDefinition.getInternalOperations().get(InternalKeys.INTERNAL_TMP_DIR),
+                                     container.getId().getApplicationAttemptId().getApplicationId().toString()),
+                            AgentKeys.PROVIDER_AGENT),
+                   SliderKeys.AGENT_TAR);
+    } else {
+       agentImagePath = new Path(agentImage);
+    }
+
+    if (fileSystem.getFileSystem().exists(agentImagePath)) {
+      LocalResource agentImageRes = fileSystem.createAmResource(agentImagePath, LocalResourceType.ARCHIVE);
+      launcher.addLocalResource(AgentKeys.AGENT_INSTALL_DIR, agentImageRes);
+    } else {
+      String msg =
+          String.format("Required agent image slider-agent.tar.gz is unavailable at %s", agentImagePath.toString());
+      MapOperations compOps = appComponent;
+      boolean relaxVerificationForTest = compOps != null ? Boolean.valueOf(compOps.
+          getOptionBool(AgentKeys.TEST_RELAX_VERIFICATION, false)) : false;
+      log.error(msg);
+
+      if (!relaxVerificationForTest) {
+        throw new SliderException(SliderExitCodes.EXIT_DEPLOYMENT_FAILED, msg);
+      }
+    }
+
+    log.info("Using {} for agent.", scriptPath);
+    LocalResource appDefRes = fileSystem.createAmResource(
+        fileSystem.getFileSystem().resolvePath(new Path(appDef)),
+        LocalResourceType.ARCHIVE);
+    launcher.addLocalResource(AgentKeys.APP_DEFINITION_DIR, appDefRes);
+
+    for (Package pkg : getMetaInfo().getApplication().getPackages()) {
+      Path pkgPath = fileSystem.buildResourcePath(pkg.getName());
+      if (!fileSystem.isFile(pkgPath)) {
+        pkgPath = fileSystem.buildResourcePath(getClusterName(),
+            pkg.getName());
+      }
+      if (!fileSystem.isFile(pkgPath)) {
+        throw new IOException("Package doesn't exist as a resource: " +
+            pkg.getName());
+      }
+      log.info("Adding resource {}", pkg.getName());
+      LocalResourceType type = LocalResourceType.FILE;
+      if ("archive".equals(pkg.getType())) {
+        type = LocalResourceType.ARCHIVE;
+      }
+      LocalResource packageResource = fileSystem.createAmResource(
+          pkgPath, type);
+      launcher.addLocalResource(AgentKeys.APP_PACKAGES_DIR, packageResource);
+    }
+
+    String agentConf = instanceDefinition.getAppConfOperations().
+        getGlobalOptions().getOption(AgentKeys.AGENT_CONF, "");
+    if (SliderUtils.isSet(agentConf)) {
+      LocalResource agentConfRes = fileSystem.createAmResource(fileSystem
+                                                                   .getFileSystem().resolvePath(new Path(agentConf)),
+                                                               LocalResourceType.FILE);
+      launcher.addLocalResource(AgentKeys.AGENT_CONFIG_FILE, agentConfRes);
+    }
+
+    String agentVer = instanceDefinition.getAppConfOperations().
+        getGlobalOptions().getOption(AgentKeys.AGENT_VERSION, null);
+    if (agentVer != null) {
+      LocalResource agentVerRes = fileSystem.createAmResource(
+          fileSystem.getFileSystem().resolvePath(new Path(agentVer)),
+          LocalResourceType.FILE);
+      launcher.addLocalResource(AgentKeys.AGENT_VERSION_FILE, agentVerRes);
+    }
+
+    if (SliderUtils.isHadoopClusterSecure(getConfig())) {
+      localizeServiceKeytabs(launcher, instanceDefinition, fileSystem);
+    }
+
+    MapOperations amComponent = instanceDefinition.
+        getAppConfOperations().getComponent(SliderKeys.COMPONENT_AM);
+    boolean twoWayEnabled = amComponent != null ? Boolean.valueOf(amComponent.
+        getOptionBool(AgentKeys.KEY_AGENT_TWO_WAY_SSL_ENABLED, false)) : false;
+    if (twoWayEnabled) {
+      localizeContainerSSLResources(launcher, container, fileSystem);
+    }
+
+    MapOperations compOps = appComponent;
+    if (areStoresRequested(compOps)) {
+      localizeContainerSecurityStores(launcher, container, roleName, fileSystem,
+                                      instanceDefinition, compOps);
+    }
+
+    //add the configuration resources
+    launcher.addLocalResources(fileSystem.submitDirectory(
+        generatedConfPath,
+        SliderKeys.PROPAGATED_CONF_DIR_NAME));
+
+    if (appComponent.getOptionBool(AgentKeys.AM_CONFIG_GENERATION, false)) {
+      // build and localize configuration files
+      Map<String, Map<String, String>> configurations =
+          buildCommandConfigurations(instanceDefinition.getAppConfOperations(),
+              container.getId().toString(), roleName, roleGroup);
+      localizeConfigFiles(launcher, roleName, roleGroup, getMetaInfo(),
+          configurations, launcher.getEnv(), fileSystem);
+    }
+
+    String label = getContainerLabel(container, roleName, roleGroup);
+    CommandLineBuilder operation = new CommandLineBuilder();
+
+    String pythonExec = instanceDefinition.getAppConfOperations()
+        .getGlobalOptions().getOption(SliderXmlConfKeys.PYTHON_EXECUTABLE_PATH,
+                                      AgentKeys.PYTHON_EXE);
+
+    operation.add(pythonExec);
+
+    operation.add(scriptPath);
+    operation.add(ARG_LABEL, label);
+    operation.add(ARG_ZOOKEEPER_QUORUM);
+    operation.add(getClusterOptionPropertyValue(OptionKeys.ZOOKEEPER_QUORUM));
+    operation.add(ARG_ZOOKEEPER_REGISTRY_PATH);
+    operation.add(getZkRegistryPath());
+
+    String debugCmd = agentLaunchParameter.getNextLaunchParameter(roleGroup);
+    if (SliderUtils.isSet(debugCmd)) {
+      operation.add(ARG_DEBUG);
+      operation.add(debugCmd);
+    }
+
+    operation.add("> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/"
+        + AgentKeys.AGENT_OUT_FILE + " 2>&1");
+
+    launcher.addCommand(operation.build());
+
+    // localize addon package
+    String addonAppDefString = instanceDefinition.getAppConfOperations()
+        .getGlobalOptions().getOption(AgentKeys.ADDONS, null);
+    log.debug("All addon appdefs: {}", addonAppDefString);
+    if (addonAppDefString != null) {
+      Scanner scanner = new Scanner(addonAppDefString).useDelimiter(",");
+      while (scanner.hasNext()) {
+        String addonAppDef = scanner.next();
+        String addonAppDefPath = instanceDefinition
+            .getAppConfOperations().getGlobalOptions().get(addonAppDef);
+        log.debug("Addon package {} is stored at: {}", addonAppDef, addonAppDefPath);
+        LocalResource addonPkgRes = fileSystem.createAmResource(
+            fileSystem.getFileSystem().resolvePath(new Path(addonAppDefPath)),
+            LocalResourceType.ARCHIVE);
+        launcher.addLocalResource(AgentKeys.ADDON_DEFINITION_DIR + "/" + addonAppDef, addonPkgRes);
+      }
+      log.debug("Metainfo map for master and addon: {}",
+          packageMetainfo.toString());
+    }    
+
+    // Additional files to localize in addition to the application def
+    String appResourcesString = instanceDefinition.getAppConfOperations()
+        .getGlobalOptions().getOption(AgentKeys.APP_RESOURCES, null);
+    log.info("Configuration value for extra resources to localize: {}", appResourcesString);
+    if (null != appResourcesString) {
+      try (Scanner scanner = new Scanner(appResourcesString).useDelimiter(",")) {
+        while (scanner.hasNext()) {
+          String resource = scanner.next();
+          Path resourcePath = new Path(resource);
+          LocalResource extraResource = fileSystem.createAmResource(
+              fileSystem.getFileSystem().resolvePath(resourcePath),
+              LocalResourceType.FILE);
+          String destination = AgentKeys.APP_RESOURCES_DIR + "/" + resourcePath.getName();
+          log.info("Localizing {} to {}", resourcePath, destination);
+          // TODO Can we try harder to avoid collisions?
+          launcher.addLocalResource(destination, extraResource);
+        }
+      }
+    }
+
+    // initialize addon pkg states for all componentInstanceStatus
+    Map<String, State> pkgStatuses = new TreeMap<>();
+    for (Metainfo appPkg : packageMetainfo.values()) {
+      // check each component of that addon to see if they apply to this
+      // component 'role'
+      for (ComponentsInAddonPackage comp : appPkg.getApplicationPackage()
+          .getComponents()) {
+        log.debug("Current component: {} component in metainfo: {}", roleName,
+            comp.getName());
+        if (comp.getName().equals(roleGroup)
+            || comp.getName().equals(AgentKeys.ADDON_FOR_ALL_COMPONENTS)) {
+          pkgStatuses.put(appPkg.getApplicationPackage().getName(), State.INIT);
+        }
+      }
+    }
+    log.debug("For component: {} pkg status map: {}", roleName,
+        pkgStatuses.toString());
+    
+    // initialize the component instance state
+    getComponentStatuses().put(label,
+                               new ComponentInstanceState(
+                                   roleName,
+                                   container.getId(),
+                                   getClusterInfoPropertyValue(OptionKeys.APPLICATION_NAME),
+                                   pkgStatuses));
+  }
+
+  private void localizeContainerSecurityStores(ContainerLauncher launcher,
+                                               Container container,
+                                               String role,
+                                               SliderFileSystem fileSystem,
+                                               AggregateConf instanceDefinition,
+                                               MapOperations compOps)
+      throws SliderException, IOException {
+    // generate and localize security stores
+    SecurityStore[] stores = generateSecurityStores(container, role,
+                                                    instanceDefinition, compOps);
+    for (SecurityStore store : stores) {
+      LocalResource keystoreResource = fileSystem.createAmResource(
+          uploadSecurityResource(store.getFile(), fileSystem), LocalResourceType.FILE);
+      launcher.addLocalResource(String.format("secstores/%s-%s.p12",
+                                              store.getType(), role),
+                                keystoreResource);
+    }
+  }
+
+  private SecurityStore[] generateSecurityStores(Container container,
+                                                 String role,
+                                                 AggregateConf instanceDefinition,
+                                                 MapOperations compOps)
+      throws SliderException, IOException {
+    return StoresGenerator.generateSecurityStores(container.getNodeId().getHost(),
+                                           container.getId().toString(), role,
+                                           instanceDefinition, compOps);
+  }
+
+  private boolean areStoresRequested(MapOperations compOps) {
+    return compOps != null ? compOps.
+        getOptionBool(SliderKeys.COMP_STORES_REQUIRED_KEY, false) : false;
+  }
+
+  private void localizeContainerSSLResources(ContainerLauncher launcher,
+                                             Container container,
+                                             SliderFileSystem fileSystem)
+      throws SliderException {
+    try {
+      // localize server cert
+      Path certsDir = fileSystem.buildClusterSecurityDirPath(getClusterName());
+      LocalResource certResource = fileSystem.createAmResource(
+          new Path(certsDir, SliderKeys.CRT_FILE_NAME),
+            LocalResourceType.FILE);
+      launcher.addLocalResource(AgentKeys.CERT_FILE_LOCALIZATION_PATH,
+                                certResource);
+
+      // generate and localize agent cert
+      CertificateManager certMgr = new CertificateManager();
+      String hostname = container.getNodeId().getHost();
+      String containerId = container.getId().toString();
+      certMgr.generateContainerCertificate(hostname, containerId);
+      LocalResource agentCertResource = fileSystem.createAmResource(
+          uploadSecurityResource(
+            CertificateManager.getAgentCertficateFilePath(containerId),
+            fileSystem), LocalResourceType.FILE);
+      // still using hostname as file name on the agent side, but the files
+      // do end up under the specific container's file space
+      launcher.addLocalResource(AgentKeys.INFRA_RUN_SECURITY_DIR + hostname +
+                                ".crt", agentCertResource);
+      LocalResource agentKeyResource = fileSystem.createAmResource(
+          uploadSecurityResource(
+              CertificateManager.getAgentKeyFilePath(containerId), fileSystem),
+            LocalResourceType.FILE);
+      launcher.addLocalResource(AgentKeys.INFRA_RUN_SECURITY_DIR + hostname +
+                                ".key", agentKeyResource);
+
+    } catch (Exception e) {
+      throw new SliderException(SliderExitCodes.EXIT_DEPLOYMENT_FAILED, e,
+          "Unable to localize certificates.  Two-way SSL cannot be enabled");
+    }
+  }
+
+  private Path uploadSecurityResource(File resource, SliderFileSystem fileSystem)
+      throws IOException {
+    Path certsDir = fileSystem.buildClusterSecurityDirPath(getClusterName());
+    return uploadResource(resource, fileSystem, certsDir);
+  }
+
+  private Path uploadResource(File resource, SliderFileSystem fileSystem,
+      String roleName) throws IOException {
+    Path dir;
+    if (roleName == null) {
+      dir = fileSystem.buildClusterResourcePath(getClusterName());
+    } else {
+      dir = fileSystem.buildClusterResourcePath(getClusterName(), roleName);
+    }
+    return uploadResource(resource, fileSystem, dir);
+  }
+
+  private static synchronized Path uploadResource(File resource,
+      SliderFileSystem fileSystem, Path parentDir) throws IOException {
+    if (!fileSystem.getFileSystem().exists(parentDir)) {
+      fileSystem.getFileSystem().mkdirs(parentDir,
+        new FsPermission(FsAction.ALL, FsAction.NONE, FsAction.NONE));
+    }
+    Path destPath = new Path(parentDir, resource.getName());
+    if (!fileSystem.getFileSystem().exists(destPath)) {
+      FSDataOutputStream os = null;
+      try {
+        os = fileSystem.getFileSystem().create(destPath);
+        byte[] contents = FileUtils.readFileToByteArray(resource);
+        os.write(contents, 0, contents.length);
+        os.flush();
+      } finally {
+        IOUtils.closeStream(os);
+      }
+      log.info("Uploaded {} to localization path {}", resource, destPath);
+    } else {
+      log.info("Resource {} already existed at localization path {}", resource,
+          destPath);
+    }
+
+    while (!fileSystem.getFileSystem().exists(destPath)) {
+      try {
+        Thread.sleep(500);
+      } catch (InterruptedException e) {
+        // ignore
+      }
+    }
+
+    fileSystem.getFileSystem().setPermission(destPath,
+      new FsPermission(FsAction.READ, FsAction.NONE, FsAction.NONE));
+
+    return destPath;
+  }
+
+  private void localizeServiceKeytabs(ContainerLauncher launcher,
+                                      AggregateConf instanceDefinition,
+                                      SliderFileSystem fileSystem)
+      throws IOException {
+    String keytabPathOnHost = instanceDefinition.getAppConfOperations()
+        .getComponent(SliderKeys.COMPONENT_AM).get(
+            SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH);
+    if (SliderUtils.isUnset(keytabPathOnHost)) {
+      String amKeytabName = instanceDefinition.getAppConfOperations()
+          .getComponent(SliderKeys.COMPONENT_AM).get(
+              SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME);
+      String keytabDir = instanceDefinition.getAppConfOperations()
+          .getComponent(SliderKeys.COMPONENT_AM).get(
+              SliderXmlConfKeys.KEY_HDFS_KEYTAB_DIR);
+      // we need to localize the keytab files in the directory
+      Path keytabDirPath = fileSystem.buildKeytabPath(keytabDir, null,
+                                                      getClusterName());
+      boolean serviceKeytabsDeployed = false;
+      if (fileSystem.getFileSystem().exists(keytabDirPath)) {
+        FileStatus[] keytabs = fileSystem.getFileSystem().listStatus(keytabDirPath);
+        LocalResource keytabRes;
+        for (FileStatus keytab : keytabs) {
+          if (!amKeytabName.equals(keytab.getPath().getName())
+              && keytab.getPath().getName().endsWith(".keytab")) {
+            serviceKeytabsDeployed = true;
+            log.info("Localizing keytab {}", keytab.getPath().getName());
+            keytabRes = fileSystem.createAmResource(keytab.getPath(),
+              LocalResourceType.FILE);
+            launcher.addLocalResource(SliderKeys.KEYTAB_DIR + "/" +
+                                    keytab.getPath().getName(),
+                                    keytabRes);
+          }
+        }
+      }
+      if (!serviceKeytabsDeployed) {
+        log.warn("No service keytabs for the application have been localized.  "
+                 + "If the application requires keytabs for secure operation, "
+                 + "please ensure that the required keytabs have been uploaded "
+                 + "to the folder {}", keytabDirPath);
+      }
+    }
+  }
+
+  private void createConfigFile(SliderFileSystem fileSystem, File file,
+      ConfigFile configFile, Map<String, String> config)
+      throws IOException {
+    ConfigFormat configFormat = ConfigFormat.resolve(configFile.getType());
+    log.info("Writing {} file {}", configFormat, file);
+
+    ConfigUtils.prepConfigForTemplateOutputter(configFormat, config,
+        fileSystem, getClusterName(), file.getName());
+    PublishedConfiguration publishedConfiguration =
+        new PublishedConfiguration(configFile.getDictionaryName(),
+            config.entrySet());
+    PublishedConfigurationOutputter configurationOutputter =
+      PublishedConfigurationOutputter.createOutputter(configFormat,
+          publishedConfiguration);
+    configurationOutputter.save(file);
+  }
+
+  @VisibleForTesting
+  protected void localizeConfigFiles(ContainerLauncher launcher,
+                                     String roleName, String roleGroup,
+                                     Metainfo metainfo,
+                                     Map<String, Map<String, String>> configs,
+                                     MapOperations env,
+                                     SliderFileSystem fileSystem)
+      throws IOException {
+    for (ConfigFile configFile : metainfo.getComponentConfigFiles(roleGroup)) {
+      Map<String, String> config = ConfigUtils.replacePropsInConfig(
+          configs.get(configFile.getDictionaryName()), env.options);
+      String fileName = ConfigUtils.replaceProps(config,
+          configFile.getFileName());
+      File localFile = new File(SliderKeys.RESOURCE_DIR);
+      if (!localFile.exists()) {
+        localFile.mkdir();
+      }
+      localFile = new File(localFile, new File(fileName).getName());
+
+      String folder = null;
+      if ("true".equals(config.get(PER_COMPONENT))) {
+        folder = roleName;
+      } else if ("true".equals(config.get(PER_GROUP))) {
+        folder = roleGroup;
+      }
+
+      log.info("Localizing {} configs to config file {} (destination {}) " +
+          "based on {} configs", config.size(), localFile, fileName,
+          configFile.getDictionaryName());
+      createConfigFile(fileSystem, localFile, configFile, config);
+      Path destPath = uploadResource(localFile, fileSystem, folder);
+      LocalResource configResource = fileSystem.createAmResource(destPath,
+          LocalResourceType.FILE);
+
+      File destFile = new File(fileName);
+      if (destFile.isAbsolute()) {
+        launcher.addLocalResource(
+            SliderKeys.RESOURCE_DIR + "/" + destFile.getName(),
+            configResource, fileName);
+      } else {
+        launcher.addLocalResource(AgentKeys.APP_CONF_DIR + "/" + fileName,
+            configResource);
+      }
+    }
+  }
+
+  /**
+   * build the zookeeper registry path.
+   * 
+   * @return the path the service registered at
+   * @throws NullPointerException if the service has not yet registered
+   */
+  private String getZkRegistryPath() {
+    Preconditions.checkNotNull(yarnRegistry, "Yarn registry not bound");
+    String path = yarnRegistry.getAbsoluteSelfRegistrationPath();
+    Preconditions.checkNotNull(path, "Service record path not defined");
+    return path;
+  }
+
+  @Override
+  public void rebuildContainerDetails(List<Container> liveContainers,
+                                      String applicationId, Map<Integer, ProviderRole> providerRoleMap) {
+    for (Container container : liveContainers) {
+      // get the role name and label
+      ProviderRole role = providerRoleMap.get(ContainerPriority
+                                                  .extractRole(container));
+      if (role != null) {
+        String roleName = role.name;
+        String label = getContainerLabel(container, roleName, role.group);
+        log.info("Rebuilding in-memory: container {} in role {} in cluster {}",
+                 container.getId(), roleName, applicationId);
+        getComponentStatuses().put(label,
+            new ComponentInstanceState(roleName, container.getId(),
+                                       applicationId));
+      } else {
+        log.warn("Role not found for container {} in cluster {}",
+                 container.getId(), applicationId);
+      }
+    }
+  }
+
+  @Override
+  public boolean isSupportedRole(String role) {
+    return true;
+  }
+
+  /**
+   * Handle registration calls from the agents
+   *
+   * @param registration registration entry
+   *
+   * @return response
+   */
+  @Override
+  public RegistrationResponse handleRegistration(Register registration) {
+    log.info("Handling registration: {}", registration);
+    RegistrationResponse response = new RegistrationResponse();
+    String label = registration.getLabel();
+    String pkg = registration.getPkg();
+    State agentState = registration.getActualState();
+    String appVersion = registration.getAppVersion();
+
+    log.info("label: {} pkg: {}", label, pkg);
+
+    if (getComponentStatuses().containsKey(label)) {
+      response.setResponseStatus(RegistrationStatus.OK);
+      ComponentInstanceState componentStatus = getComponentStatuses().get(label);
+      componentStatus.heartbeat(System.currentTimeMillis());
+      updateComponentStatusWithAgentState(componentStatus, agentState);
+
+      String roleName = getRoleName(label);
+      String roleGroup = getRoleGroup(label);
+      String containerId = getContainerId(label);
+
+      if (SliderUtils.isSet(registration.getTags())) {
+        tags.recordAssignedTag(roleName, containerId, registration.getTags());
+      } else {
+        response.setTags(tags.getTag(roleName, containerId));
+      }
+
+      String hostFqdn = registration.getPublicHostname();
+      Map<String, String> ports = registration.getAllocatedPorts();
+      if (ports != null && !ports.isEmpty()) {
+        processAllocatedPorts(hostFqdn, roleName, roleGroup, containerId, ports);
+      }
+
+      Map<String, String> folders = registration.getLogFolders();
+      if (folders != null && !folders.isEmpty()) {
+        publishFolderPaths(folders, containerId, roleName, hostFqdn);
+      }
+
+      // Set app version if empty. It gets unset during upgrade - why?
+      checkAndSetContainerAppVersion(containerId, appVersion);
+    } else {
+      response.setResponseStatus(RegistrationStatus.FAILED);
+      response.setLog("Label not recognized.");
+      log.warn("Received registration request from unknown label {}", label);
+    }
+    log.info("Registration response: {}", response);
+    return response;
+  }
+
+  // Checks if app version is empty. Sets it to the version as reported by the
+  // container during registration phase.
+  private void checkAndSetContainerAppVersion(String containerId,
+      String appVersion) {
+    StateAccessForProviders amState = getAmState();
+    try {
+      RoleInstance role = amState.getOwnedContainer(containerId);
+      if (role != null) {
+        String currentAppVersion = role.appVersion;
+        log.debug("Container = {}, app version current = {} new = {}",
+            containerId, currentAppVersion, appVersion);
+        if (currentAppVersion == null
+            || currentAppVersion.equals(APP_VERSION_UNKNOWN)) {
+          amState.getOwnedContainer(containerId).appVersion = appVersion;
+        }
+      }
+    } catch (NoSuchNodeException e) {
+      // ignore - there is nothing to do if we don't find a container
+      log.warn("Owned container {} not found - {}", containerId, e);
+    }
+  }
+
+  /**
+   * Handle heartbeat response from agents
+   *
+   * @param heartBeat incoming heartbeat from Agent
+   *
+   * @return response to send back
+   */
+  @Override
+  public HeartBeatResponse handleHeartBeat(HeartBeat heartBeat) {
+    log.debug("Handling heartbeat: {}", heartBeat);
+    HeartBeatResponse response = new HeartBeatResponse();
+    long id = heartBeat.getResponseId();
+    response.setResponseId(id + 1L);
+
+    String label = heartBeat.getHostname();
+    String pkg = heartBeat.getPackage();
+
+    log.debug("package received: " + pkg);
+    
+    String roleName = getRoleName(label);
+    String roleGroup = getRoleGroup(label);
+    String containerId = getContainerId(label);
+    boolean doUpgrade = false;
+    if (isInUpgradeMode && upgradeContainers.contains(containerId)) {
+      doUpgrade = true;
+    }
+
+    StateAccessForProviders accessor = getAmState();
+    CommandScript cmdScript = getScriptPathForMasterPackage(roleGroup);
+    List<ComponentCommand> commands = getMetaInfo().getApplicationComponent(roleGroup).getCommands();
+
+    if (!isDockerContainer(roleGroup) && !isYarnDockerContainer(roleGroup)
+        && (cmdScript == null || cmdScript.getScript() == null)
+        && commands.size() == 0) {
+      log.error(
+          "role.script is unavailable for {}. Commands will not be sent.",
+          roleName);
+      return response;
+    }
+
+    String scriptPath = null;
+    long timeout = 600L;
+    if (cmdScript != null) {
+      scriptPath = cmdScript.getScript();
+      timeout = cmdScript.getTimeout();
+    }
+
+    if (timeout == 0L) {
+      timeout = 600L;
+    }
+
+    if (!getComponentStatuses().containsKey(label)) {
+      // container is completed but still heart-beating, send terminate signal
+      log.info(
+          "Sending terminate signal to completed container (still heartbeating): {}",
+          label);
+      response.setTerminateAgent(true);
+      return response;
+    }
+
+    List<ComponentStatus> statuses = heartBeat.getComponentStatus();
+    if (statuses != null && !statuses.isEmpty()) {
+      log.info("status from agent: " + statuses.toString());
+      try {
+        for(ComponentStatus status : statuses){
+          RoleInstance role = null;
+          if(status.getIp() != null && !status.getIp().isEmpty()){
+            role = amState.getOwnedContainer(containerId);
+            role.ip = status.getIp();
+          }
+          if(status.getHostname() != null && !status.getHostname().isEmpty()){
+            role = amState.getOwnedContainer(containerId);
+            role.hostname = status.getHostname();
+          }
+          if (role != null) {
+            // create an updated service record (including hostname and ip) and publish...
+            ServiceRecord record = new ServiceRecord();
+            record.set(YarnRegistryAttributes.YARN_ID, containerId);
+            record.description = roleName;
+            record.set(YarnRegistryAttributes.YARN_PERSISTENCE,
+                       PersistencePolicies.CONTAINER);
+            // TODO:  switch record attributes to use constants from YarnRegistryAttributes
+            // when it's been updated.
+            if (role.ip != null) {
+              record.set("yarn:ip", role.ip);
+            }
+            if (role.hostname != null) {
+              record.set("yarn:hostname", role.hostname);
+            }
+            yarnRegistry.putComponent(
+                RegistryPathUtils.encodeYarnID(containerId), record);
+
+          }
+        }
+
+
+      } catch (NoSuchNodeException e) {
+        // ignore - there is nothing to do if we don't find a container
+        log.warn("Owned container {} not found - {}", containerId, e);
+      } catch (IOException e) {
+        log.warn("Error updating container {} service record in registry",
+                 containerId, e);
+      }
+    }
+
+    Boolean isMaster = isMaster(roleGroup);
+    ComponentInstanceState componentStatus = getComponentStatuses().get(label);
+    componentStatus.heartbeat(System.currentTimeMillis());
+    if (doUpgrade) {
+      switch (componentStatus.getState()) {
+      case STARTED:
+        componentStatus.setTargetState(State.UPGRADED);
+        break;
+      case UPGRADED:
+        componentStatus.setTargetState(State.STOPPED);
+        break;
+      case STOPPED:
+        componentStatus.setTargetState(State.TERMINATING);
+        break;
+      default:
+        break;
+      }
+      log.info("Current state = {} target state {}",
+          componentStatus.getState(), componentStatus.getTargetState());
+    }
+
+    if (appStopInitiated && !componentStatus.isStopInitiated()) {
+      log.info("Stop initiated for label {}", label);
+      componentStatus.setTargetState(State.STOPPED);
+      componentStatus.setStopInitiated(true);
+    }
+
+    publishConfigAndExportGroups(heartBeat, componentStatus, roleGroup);
+    CommandResult result = null;
+    List<CommandReport> reports = heartBeat.getReports();
+    if (SliderUtils.isNotEmpty(reports)) {
+      CommandReport report = reports.get(0);
+      Map<String, String> ports = report.getAllocatedPorts();
+      if (SliderUtils.isNotEmpty(ports)) {
+        processAllocatedPorts(heartBeat.getFqdn(), roleName, roleGroup, containerId, ports);
+      }
+      result = CommandResult.getCommandResult(report.getStatus());
+      Command command = Command.getCommand(report.getRoleCommand());
+      componentStatus.applyCommandResult(result, command, pkg);
+      log.info("Component operation. Status: {}; new container state: {};"
+          + " new component state: {}", result,
+          componentStatus.getContainerState(), componentStatus.getState());
+
+      if (command == Command.INSTALL && SliderUtils.isNotEmpty(report.getFolders())) {
+        publishFolderPaths(report.getFolders(), containerId, roleName, heartBeat.getFqdn());
+      }
+    }
+
+    int waitForCount = accessor.getInstanceDefinitionSnapshot().
+        getAppConfOperations().getComponentOptInt(roleGroup, AgentKeys.WAIT_HEARTBEAT, 0);
+
+    if (id < waitForCount) {
+      log.info("Waiting until heartbeat count {}. Current val: {}", waitForCount, id);
+      getComponentStatuses().put(label, componentStatus);
+      return response;
+    }
+
+    Command command = componentStatus.getNextCommand(doUpgrade);
+    try {
+      if (Command.NOP != command) {
+        log.debug("For comp {} pkg {} issuing {}", roleName,
+            componentStatus.getNextPkgToInstall(), command.toString());
+        if (command == Command.INSTALL) {
+          log.info("Installing {} on {}.", roleName, containerId);
+          if (isDockerContainer(roleGroup) || isYarnDockerContainer(roleGroup)){
+            addInstallDockerCommand(roleName, roleGroup, containerId,
+                response, null, timeout);
+          } else if (scriptPath != null) {
+            addInstallCommand(roleName, roleGroup, containerId, response,
+                scriptPath, null, timeout, null);
+          } else {
+            // commands
+            ComponentCommand installCmd = null;
+            for (ComponentCommand compCmd : commands) {
+              if (compCmd.getName().equals("INSTALL")) {
+                installCmd = compCmd;
+              }
+            }
+            addInstallCommand(roleName, roleGroup, containerId, response, null,
+                installCmd, timeout, null);
+          }
+          componentStatus.commandIssued(command);
+        } else if (command == Command.INSTALL_ADDON) {
+          String nextPkgToInstall = componentStatus.getNextPkgToInstall();
+          // retrieve scriptPath or command of that package for the component
+          for (ComponentsInAddonPackage comp : packageMetainfo
+              .get(nextPkgToInstall).getApplicationPackage().getComponents()) {
+            // given nextPkgToInstall and roleName is determined, the if below
+            // should only execute once per heartbeat
+            log.debug("Addon component: {} pkg: {} script: {}", comp.getName(),
+                nextPkgToInstall, comp.getCommandScript().getScript());
+            if (comp.getName().equals(roleGroup)
+                || comp.getName().equals(AgentKeys.ADDON_FOR_ALL_COMPONENTS)) {
+              scriptPath = comp.getCommandScript().getScript();
+              if (scriptPath != null) {
+                addInstallCommand(roleName, roleGroup, containerId, response,
+                    scriptPath, null, timeout, nextPkgToInstall);
+              } else {
+                ComponentCommand installCmd = null;
+                for (ComponentCommand compCmd : comp.getCommands()) {
+                  if (compCmd.getName().equals("INSTALL")) {
+                    installCmd = compCmd;
+                  }
+                }
+                addInstallCommand(roleName, roleGroup, containerId, response,
+                    null, installCmd, timeout, nextPkgToInstall);
+              }
+            }
+          }
+          componentStatus.commandIssued(command);
+        } else if (command == Command.START) {
+          // check against dependencies
+          boolean canExecute = commandOrder.canExecute(roleGroup, command, getComponentStatuses().values());
+          if (canExecute) {
+            log.info("Starting {} on {}.", roleName, containerId);
+            if (isDockerContainer(roleGroup) || isYarnDockerContainer(roleGroup)){
+              addStartDockerCommand(roleName, roleGroup, containerId,
+                  response, null, timeout, false);
+            } else if (scriptPath != null) {
+              addStartCommand(roleName,
+                              roleGroup,
+                              containerId,
+                              response,
+                              scriptPath,
+                              null,
+                              null,
+                              timeout,
+                              isMarkedAutoRestart(roleGroup));
+            } else {
+              ComponentCommand startCmd = null;
+              for (ComponentCommand compCmd : commands) {
+                if (compCmd.getName().equals("START")) {
+                  startCmd = compCmd;
+                }
+              }
+              ComponentCommand stopCmd = null;
+              for (ComponentCommand compCmd : commands) {
+                if (compCmd.getName().equals("STOP")) {
+                  stopCmd = compCmd;
+                }
+              }
+              addStartCommand(roleName, roleGroup, containerId, response, null,
+                  startCmd, stopCmd, timeout, false);
+            }
+            componentStatus.commandIssued(command);
+          } else {
+            log.info("Start of {} on {} delayed as dependencies have not started.", roleName, containerId);
+          }
+        } else if (command == Command.UPGRADE) {
+          addUpgradeCommand(roleName, roleGroup, containerId, response,
+              scriptPath, timeout);
+          componentStatus.commandIssued(command, true);
+        } else if (command == Command.STOP) {
+          log.info("Stop command being sent to container with id {}",
+              containerId);
+          addStopCommand(roleName, roleGroup, containerId, response, scriptPath,
+              timeout, doUpgrade);
+          componentStatus.commandIssued(command);
+        } else if (command == Command.TERMINATE) {
+          log.info("A formal terminate command is being sent to container {}"
+              + " in state {}", label, componentStatus.getState());
+          response.setTerminateAgent(true);
+        }
+      }
+
+      // if there is no outstanding command then retrieve config
+      if (isMaster && componentStatus.getState() == State.STARTED
+          && command == Command.NOP) {
+        if (!componentStatus.getConfigReported()) {
+          log.info("Requesting applied config for {} on {}.", roleName, containerId);
+          if (isDockerContainer(roleGroup) || isYarnDockerContainer(roleGroup)){
+            addGetConfigDockerCommand(roleName, roleGroup, containerId, response);
+          } else {
+            addGetConfigCommand(roleName, roleGroup, containerId, response);
+          }
+        }
+      }
+      
+      // if restart is required then signal
+      response.setRestartEnabled(false);
+      if (componentStatus.getState() == State.STARTED
+          && command == Command.NOP && isMarkedAutoRestart(roleGroup)) {
+        response.setRestartEnabled(true);
+      }
+
+      //If INSTALL_FAILED and no INSTALL is scheduled let the agent fail
+      if (componentStatus.getState() == State.INSTALL_FAILED
+         && command == Command.NOP) {
+        log.warn("Sending terminate signal to container that failed installation: {}", label);
+        response.setTerminateAgent(true);
+      }
+
+    } catch (SliderException e) {
+      log.warn("Component instance failed operation.", e);
+      componentStatus.applyCommandResult(CommandResult.FAILED, command, null);
+    }
+
+    log.debug("Heartbeat response: " + response);
+    return response;
+  }
+
+  private boolean isDockerContainer(String roleGroup) {
+    String type = getMetaInfo().getApplicationComponent(roleGroup).getType();
+    if (SliderUtils.isSet(type)) {
+      return type.toLowerCase().equals(SliderUtils.DOCKER) || type.toLowerCase().equals(SliderUtils.DOCKER_YARN);
+    }
+    return false;
+  }
+
+  private boolean isYarnDockerContainer(String roleGroup) {
+    String type = getMetaInfo().getApplicationComponent(roleGroup).getType();
+    if (SliderUtils.isSet(type)) {
+      return type.toLowerCase().equals(SliderUtils.DOCKER_YARN);
+    }
+    return false;
+  }
+
+  protected void processAllocatedPorts(String fqdn,
+                                       String roleName,
+                                       String roleGroup,
+                                       String containerId,
+                                       Map<String, String> ports) {
+    RoleInstance instance;
+    try {
+      instance = getAmState().getOwnedContainer(containerId);
+    } catch (NoSuchNodeException e) {
+      log.warn("Failed to locate instance of container {}", containerId, e);
+      instance = null;
+    }
+    for (Map.Entry<String, String> port : ports.entrySet()) {
+      String portname = port.getKey();
+      String portNo = port.getValue();
+      log.info("Recording allocated port for {} as {}", portname, portNo);
+
+      // add the allocated ports to the global list as well as per container list
+      // per container allocation will over-write each other in the global
+      this.getAllocatedPorts().put(portname, portNo);
+      this.getAllocatedPorts(containerId).put(portname, portNo);
+      if (instance != null) {
+        try {
+          // if the returned value is not a single port number then there are no
+          // meaningful way for Slider to use it during export
+          // No need to error out as it may not be the responsibility of the component
+          // to allocate port or the component may need an array of ports
+          instance.registerPortEndpoint(Integer.valueOf(portNo), portname);
+        } catch (NumberFormatException e) {
+          log.warn("Failed to parse {}", portNo, e);
+        }
+      }
+    }
+
+    processAndPublishComponentSpecificData(ports, containerId, fqdn, roleGroup);
+    processAndPublishComponentSpecificExports(ports, containerId, fqdn, roleName, roleGroup);
+
+    // and update registration entries
+    if (instance != null) {
+      queueAccess.put(new RegisterComponentInstance(instance.getId(),
+          roleName, roleGroup, 0, TimeUnit.MILLISECONDS));
+    }
+  }
+
+  private void updateComponentStatusWithAgentState(
+      ComponentInstanceState componentStatus, State agentState) {
+    if (agentState != null) {
+      componentStatus.setState(agentState);
+    }
+  }
+
+  @Override
+  public Map<String, MonitorDetail> buildMonitorDetails(ClusterDescription clusterDesc) {
+    Map<String, MonitorDetail> details = super.buildMonitorDetails(clusterDesc);
+    buildRoleHostDetails(details);
+    return details;
+  }
+
+  @Override
+  public void applyInitialRegistryDefinitions(URL amWebURI,
+      URL agentOpsURI,
+      URL agentStatusURI,
+      ServiceRecord serviceRecord)
+    throws IOException {
+    super.applyInitialRegistryDefinitions(amWebURI,
+                                          agentOpsURI,
+                                          agentStatusURI,
+                                          serviceRecord);
+
+    try {
+      URL restURL = new URL(agentOpsURI, SLIDER_PATH_AGENTS);
+      URL agentStatusURL = new URL(agentStatusURI, SLIDER_PATH_AGENTS);
+
+      serviceRecord.addInternalEndpoint(
+          new Endpoint(CustomRegistryConstants.AGENT_SECURE_REST_API,
+                       ProtocolTypes.PROTOCOL_REST,
+                       restURL.toURI()));
+      serviceRecord.addInternalEndpoint(
+          new Endpoint(CustomRegistryConstants.AGENT_ONEWAY_REST_API,
+                       ProtocolTypes.PROTOCOL_REST,
+                       agentStatusURL.toURI()));
+    } catch (URISyntaxException e) {
+      throw new IOException(e);
+    }
+
+    // identify client component
+    Component client = null;
+    for (Component component : getMetaInfo().getApplication().getComponents()) {
+      if (component.getCategory().equals("CLIENT")) {
+        client = component;
+        break;
+      }
+    }
+    if (client == null) {
+      log.info("No client component specified, not publishing client configs");
+      return;
+    }
+
+    // register AM-generated client configs
+    ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
+    MapOperations clientOperations = appConf.getOrAddComponent(client.getName());
+    appConf.resolve();
+    if (!clientOperations.getOptionBool(AgentKeys.AM_CONFIG_GENERATION,
+        false)) {
+      log.info("AM config generation is false, not publishing client configs");
+      return;
+    }
+
+    // build and localize configuration files
+    Map<String, Map<String, String>> configurations = new TreeMap<String, Map<String, String>>();
+    Map<String, String> tokens = null;
+    try {
+      tokens = getStandardTokenMap(appConf, client.getName(), client.getName());
+    } catch (SliderException e) {
+      throw new IOException(e);
+    }
+
+    for (ConfigFile configFile : getMetaInfo()
+        .getComponentConfigFiles(client.getName())) {
+      addNamedConfiguration(configFile.getDictionaryName(),
+          appConf.getGlobalOptions().options, configurations, tokens, null,
+          client.getName());
+      if (appConf.getComponent(client.getName()) != null) {
+        addNamedConfiguration(configFile.getDictionaryName(),
+            appConf.getComponent(client.getName()).options, configurations,
+            tokens, null, client.getName());
+      }
+    }
+
+    //do a final replacement of re-used configs
+    dereferenceAllConfigs(configurations);
+
+    for (ConfigFile configFile : getMetaInfo()
+        .getComponentConfigFiles(client.getName())) {
+      ConfigFormat configFormat = ConfigFormat.resolve(configFile.getType());
+
+      Map<String, String> config = configurations.get(configFile.getDictionaryName());
+      ConfigUtils.prepConfigForTemplateOutputter(configFormat, config,
+          fileSystem, getClusterName(),
+          new File(configFile.getFileName()).getName());
+      PublishedConfiguration publishedConfiguration =
+          new PublishedConfiguration(configFile.getDictionaryName(),
+              config.entrySet());
+      getAmState().getPublishedSliderConfigurations().put(
+          configFile.getDictionaryName(), publishedConfiguration);
+      log.info("Publishing AM configuration {}", configFile.getDictionaryName());
+    }
+  }
+
+  @Override
+  public void notifyContainerCompleted(ContainerId containerId) {
+    // containers get allocated and free'ed without being assigned to any
+    // component - so many of the data structures may not be initialized
+    if (containerId != null) {
+      String containerIdStr = containerId.toString();
+      if (getComponentInstanceData().containsKey(containerIdStr)) {
+        getComponentInstanceData().remove(containerIdStr);
+        log.info("Removing container specific data for {}", containerIdStr);
+        publishComponentInstanceData();
+      }
+
+      if (this.allocatedPorts.containsKey(containerIdStr)) {
+        Map<String, String> portsByContainerId = getAllocatedPorts(containerIdStr);
+        this.allocatedPorts.remove(containerIdStr);
+        // free up the allocations from global as well
+        // if multiple containers allocate global ports then last one
+        // wins and similarly first one removes it - its not supported anyway
+        for(String portName : portsByContainerId.keySet()) {
+          getAllocatedPorts().remove(portName);
+        }
+
+      }
+
+      String componentName = null;
+      synchronized (this.componentStatuses) {
+        for (String label : getComponentStatuses().keySet()) {
+          if (label.startsWith(containerIdStr)) {
+            componentName = getRoleName(label);
+            log.info("Removing component status for label {}", label);
+            getComponentStatuses().remove(label);
+          }
+        }
+      }
+
+      tags.releaseTag(componentName, containerIdStr);
+
+      synchronized (this.containerExportsMap) {
+        Set<String> containerExportSets = containerExportsMap.get(containerIdStr);
+        if (containerExportSets != null) {
+          for (String containerExportStr : containerExportSets) {
+            String[] parts = containerExportStr.split(":");
+            Map<String, List<ExportEntry>> exportGroup = getCurrentExports(parts[0]);
+            List<ExportEntry> exports = exportGroup.get(parts[1]);
+            List<ExportEntry> exportToRemove = new ArrayList<ExportEntry>();
+            for (ExportEntry export : exports) {
+              if (containerIdStr.equals(export.getContainerId())) {
+                exportToRemove.add(export);
+              }
+            }
+            exports.removeAll(exportToRemove);
+          }
+          log.info("Removing container exports for {}", containerIdStr);
+          containerExportsMap.remove(containerIdStr);
+        }
+      }
+    }
+  }
+
+  /**
+   * Reads and sets the heartbeat monitoring interval. If bad value is provided then log it and set to default.
+   *
+   * @param instanceDefinition
+   */
+  private void readAndSetHeartbeatMonitoringInterval(AggregateConf instanceDefinition) {
+    String hbMonitorInterval = instanceDefinition.getAppConfOperations().
+        getGlobalOptions().getOption(AgentKeys.HEARTBEAT_MONITOR_INTERVAL,
+                                     Integer.toString(DEFAULT_HEARTBEAT_MONITOR_INTERVAL));
+    try {
+      setHeartbeatMonitorInterval(Integer.parseInt(hbMonitorInterval));
+    } catch (NumberFormatException e) {
+      log.warn(
+          "Bad value {} for {}. Defaulting to ",
+          hbMonitorInterval,
+          HEARTBEAT_MONITOR_INTERVAL,
+          DEFAULT_HEARTBEAT_MONITOR_INTERVAL);
+    }
+  }
+
+  /**
+   * Reads and sets the heartbeat monitoring interval. If bad value is provided then log it and set to default.
+   *
+   * @param instanceDefinition
+   */
+  private void initializeAgentDebugCommands(AggregateConf instanceDefinition) {
+    String launchParameterStr = instanceDefinition.getAppConfOperations().
+        getGlobalOptions().getOption(AgentKeys.AGENT_INSTANCE_DEBUG_DATA, "");
+    agentLaunchParameter = new AgentLaunchParameter(launchParameterStr);
+  }
+
+  @VisibleForTesting
+  protected Map<String, ExportEntry> getLogFolderExports() {
+    return logFolderExports;
+  }
+
+  @VisibleForTesting
+  protected Map<String, ExportEntry> getWorkFolderExports() {
+    return workFolderExports;
+  }
+
+  @VisibleForTesting
+  protected Metainfo getMetaInfo() {
+    return this.metaInfo;
+  }
+
+  @VisibleForTesting
+  protected Map<String, ComponentInstanceState> getComponentStatuses() {
+    return componentStatuses;
+  }
+
+  @VisibleForTesting
+  protected Metainfo getApplicationMetainfo(SliderFileSystem fileSystem,
+      String appDef, boolean addonPackage) throws IOException,
+      BadConfigException {
+    return AgentUtils.getApplicationMetainfo(fileSystem, appDef, addonPackage);
+  }
+
+  @VisibleForTesting
+  protected Metainfo getApplicationMetainfo(SliderFileSystem fileSystem,
+      String appDef) throws IOException, BadConfigException {
+    return getApplicationMetainfo(fileSystem, appDef, false);
+  }
+
+  @VisibleForTesting
+  protected void setHeartbeatMonitorInterval(int heartbeatMonitorInterval) {
+    this.heartbeatMonitorInterval = heartbeatMonitorInterval;
+  }
+
+  public void setInUpgradeMode(boolean inUpgradeMode) {
+    this.isInUpgradeMode = inUpgradeMode;
+  }
+
+  public void addUpgradeContainers(Set<String> upgradeContainers) {
+    this.upgradeContainers.addAll(upgradeContainers);
+  }
+
+  public void setAppStopInitiated(boolean appStopInitiated) {
+    this.appStopInitiated = appStopInitiated;
+  }
+
+  /**
+   * Read all default configs
+   *
+   * @param fileSystem fs
+   * @param appDef app default path
+   * @param metainfo metadata
+   *
+   * @return configuration maps
+   * 
+   * @throws IOException
+   */
+  protected Map<String, DefaultConfig> initializeDefaultConfigs(SliderFileSystem fileSystem,
+                                                                String appDef, Metainfo metainfo) throws IOException {
+    Map<String, DefaultConfig> defaultConfigMap = new HashMap<>();
+    if (SliderUtils.isNotEmpty(metainfo.getApplication().getConfigFiles())) {
+      for (ConfigFile configFile : metainfo.getApplication().getConfigFiles()) {
+        DefaultConfig config = null;
+        try {
+          config = AgentUtils.getDefaultConfig(fileSystem, appDef, configFile.getDictionaryName() + ".xml");
+        } catch (IOException e) {
+          log.warn("Default config file not found. Only the config as input during create will be applied for {}",
+                   configFile.getDictionaryName());
+        }
+        if (config != null) {
+          defaultConfigMap.put(configFile.getDictionaryName(), config);
+        }
+      }
+    }
+
+    return defaultConfigMap;
+  }
+
+  protected Map<String, DefaultConfig> getDefaultConfigs() {
+    return defaultConfigs;
+  }
+
+  private int getHeartbeatMonitorInterval() {
+    return this.heartbeatMonitorInterval;
+  }
+
+  private String getClusterName() {
+    if (SliderUtils.isUnset(clusterName)) {
+      clusterName = getAmState().getInternalsSnapshot().get(OptionKeys.APPLICATION_NAME);
+    }
+    return clusterName;
+  }
+
+  /**
+   * Publish a named property bag that may contain name-value pairs for app configurations such as hbase-site
+   *
+   * @param name
+   * @param description
+   * @param entries
+   */
+  protected void publishApplicationInstanceData(String name, String description,
+                                                Iterable<Map.Entry<String, String>> entries) {
+    PublishedConfiguration pubconf = new PublishedConfiguration();
+    pubconf.description = description;
+    pubconf.putValues(entries);
+    log.info("publishing {}", pubconf);
+    getAmState().getPublishedSliderConfigurations().put(name, pubconf);
+  }
+
+  /**
+   * Get a list of all hosts for all role/container per role
+   *
+   * @return the map of role->node
+   */
+  protected Map<String, Map<String, ClusterNode>> getRoleClusterNodeMapping() {
+    return amState.getRoleClusterNodeMapping();
+  }
+
+  private String getContainerLabel(Container container, String role, String group) {
+    if (role.equals(group)) {
+      return container.getId().toString() + LABEL_MAKER + role;
+    } else {
+      return container.getId().toString() + LABEL_MAKER + role + LABEL_MAKER +
+          group;
+    }
+  }
+
+  protected String getClusterInfoPropertyValue(String name) {
+    StateAccessForProviders accessor = getAmState();
+    assert accessor.isApplicationLive();
+    ClusterDescription description = accessor.getClusterStatus();
+    return description.getInfo(name);
+  }
+
+  protected String getClusterOptionPropertyValue(String name)
+      throws BadConfigException {
+    StateAccessForProviders accessor = getAmState();
+    assert accessor.isApplicationLive();
+    ClusterDescription description = accessor.getClusterStatus();
+    return description.getMandatoryOption(name);
+  }
+
+  /**
+   * Lost heartbeat from the container - release it and ask for a replacement (async operation)
+   *
+   * @param label
+   * @param containerId
+   */
+  protected void lostContainer(
+      String label,
+      ContainerId containerId) {
+    getComponentStatuses().remove(label);
+    getQueueAccess().put(new ProviderReportedContainerLoss(containerId));
+  }
+
+  /**
+   * Build the provider status, can be empty
+   *
+   * @return the provider status - map of entries to add to the info section
+   */
+  public Map<String, String> buildProviderStatus() {
+    Map<String, String> stats = new HashMap<String, String>();
+    return stats;
+  }
+
+
+  /**
+   * Format the folder locations and publish in the registry service
+   *
+   * @param folders
+   * @param containerId
+   * @param hostFqdn
+   * @param componentName
+   */
+  protected void publishFolderPaths(
+      Map<String, String> folders, String containerId, String componentName, String hostFqdn) {
+    Date now = new Date();
+    for (Map.Entry<String, String> entry : folders.entrySet()) {
+      ExportEntry exportEntry = new ExportEntry();
+      exportEntry.setValue(String.format(HOST_FOLDER_FORMAT, hostFqdn, entry.getValue()));
+      exportEntry.setContainerId(containerId);
+      exportEntry.setLevel(COMPONENT_TAG);
+      exportEntry.setTag(componentName);
+      exportEntry.setUpdatedTime(now.toString());
+      if (entry.getKey().equals("AGENT_LOG_ROOT")) {
+        synchronized (logFolderExports) {
+          getLogFolderExports().put(containerId, exportEntry);
+        }
+      } else {
+        synchronized (workFolderExports) {
+          getWorkFolderExports().put(containerId, exportEntry);
+        }
+      }
+      log.info("Updating log and pwd folders for container {}", containerId);
+    }
+
+    PublishedExports exports = new PublishedExports(CONTAINER_LOGS_TAG);
+    exports.setUpdated(now.getTime());
+    synchronized (logFolderExports) {
+      updateExportsFromList(exports, getLogFolderExports());
+    }
+    getAmState().getPublishedExportsSet().put(CONTAINER_LOGS_TAG, exports);
+
+    exports = new PublishedExports(CONTAINER_PWDS_TAG);
+    exports.setUpdated(now.getTime());
+    synchronized (workFolderExports) {
+      updateExportsFromList(exports, getWorkFolderExports());
+    }
+    getAmState().getPublishedExportsSet().put(CONTAINER_PWDS_TAG, exports);
+  }
+
+  /**
+   * Update the export data from the map
+   * @param exports
+   * @param folderExports
+   */
+  private void updateExportsFromList(PublishedExports exports, Map<String, ExportEntry> folderExports) {
+    Map<String, List<ExportEntry>> perComponentList = new HashMap<String, List<ExportEntry>>();
+    for(Map.Entry<String, ExportEntry> logEntry : folderExports.entrySet())
+    {
+      String componentName = logEntry.getValue().getTag();
+      if (!perComponentList.containsKey(componentName)) {
+        perComponentList.put(componentName, new ArrayList<ExportEntry>());
+      }
+      perComponentList.get(componentName).add(logEntry.getValue());
+    }
+    exports.putValues(perComponentList.entrySet());
+  }
+
+
+  /**
+   * Process return status for component instances
+   *
+   * @param heartBeat
+   * @param componentStatus
+   */
+  protected void publishConfigAndExportGroups(HeartBeat heartBeat,
+      ComponentInstanceState componentStatus, String componentGroup) {
+    List<ComponentStatus> statuses = heartBeat.getComponentStatus();
+    if (statuses != null && !statuses.isEmpty()) {
+      log.info("Processing {} status reports.", statuses.size());
+      for (ComponentStatus status : statuses) {
+        log.info("Status report: {}", status.toString());
+
+        if (status.getConfigs() != null) {
+          Application application = getMetaInfo().getApplication();
+
+          if ((!canAnyMasterPublishConfig() || canPublishConfig(componentGroup)) &&
+              !getAmState().getAppConfSnapshot().getComponentOptBool(
+                  componentGroup, AgentKeys.AM_CONFIG_GENERATION, false)) {
+            // If no Master can explicitly publish then publish if its a master
+            // Otherwise, wait till the master that can publish is ready
+
+            Set<String> exportedConfigs = new HashSet();
+            String exportedConfigsStr = application.getExportedConfigs();
+            boolean exportedAllConfigs = exportedConfigsStr == null || exportedConfigsStr.isEmpty();
+            if (!exportedAllConfigs) {
+              for (String exportedConfig : exportedConfigsStr.split(",")) {
+                if (exportedConfig.trim().length() > 0) {
+                  exportedConfigs.add(exportedConfig.trim());
+                }
+              }
+            }
+
+            for (String key : status.getConfigs().keySet()) {
+              if ((!exportedAllConfigs && exportedConfigs.contains(key)) ||
+                  exportedAllConfigs) {
+                Map<String, String> configs = status.getConfigs().get(key);
+                publishApplicationInstanceData(key, key, configs.entrySet());
+              }
+            }
+          }
+
+          List<ExportGroup> appExportGroups = application.getExportGroups();
+          boolean hasExportGroups = SliderUtils.isNotEmpty(appExportGroups);
+
+          Set<String> appExports = new HashSet();
+          String appExportsStr = getApplicationComponent(componentGroup).getAppExports();
+          if (SliderUtils.isSet(appExportsStr)) {
+            for (String appExport : appExportsStr.split(",")) {
+              if (!appExport.trim().isEmpty()) {
+                appExports.add(appExport.trim());
+              }
+            }
+          }
+
+          if (hasExportGroups && !appExports.isEmpty()) {
+            String configKeyFormat = "${site.%s.%s}";
+            String hostKeyFormat = "${%s_HOST}";
+
+            // publish export groups if any
+            Map<String, String> replaceTokens = new HashMap<String, String>();
+            for (Map.Entry<String, Map<String, ClusterNode>> entry : getRoleClusterNodeMapping().entrySet()) {
+              String hostName = getHostsList(entry.getValue().values(), true).iterator().next();
+              replaceTokens.put(String.format(hostKeyFormat, entry.getKey().toUpperCase(Locale.ENGLISH)), hostName);
+            }
+
+            for (String key : status.getConfigs().keySet()) {
+              Map<String, String> configs = status.getConfigs().get(key);
+              for (String configKey : configs.keySet()) {
+                String lookupKey = String.format(configKeyFormat, key, configKey);
+                replaceTokens.put(lookupKey, configs.get(configKey));
+              }
+            }
+
+            Set<String> modifiedGroups = new HashSet<String>();
+            for (ExportGroup exportGroup : appExportGroups) {
+              List<Export> exports = exportGroup.getExports();
+              if (SliderUtils.isNotEmpty(exports)) {
+                String exportGroupName = exportGroup.getName();
+                ConcurrentHashMap<String, List<ExportEntry>> map =
+                    (ConcurrentHashMap<String, List<ExportEntry>>)getCurrentExports(exportGroupName);
+                for (Export export : exports) {
+                  if (canBeExported(exportGroupName, export.getName(), appExports)) {
+                    String value = export.getValue();
+                    // replace host names
+                    for (String token : replaceTokens.keySet()) {
+                      if (value.contains(token)) {
+                        value = value.replace(token, replaceTokens.get(token));
+                      }
+                    }
+                    ExportEntry entry = new ExportEntry();
+                    entry.setLevel(APPLICATION_TAG);
+                    entry.setValue(value);
+                    entry.setUpdatedTime(new Date().toString());
+                    // over-write, app exports are singletons
+                    map.put(export.getName(), new ArrayList(Arrays.asList(entry)));
+                    log.info("Preparing to publish. Key {} and Value {}", export.getName(), value);
+                  }
+                }
+                modifiedGroups.add(exportGroupName);
+              }
+            }
+            publishModifiedExportGroups(modifiedGroups);
+          }
+
+          log.info("Received and processed config for {}", heartBeat.getHostname());
+          componentStatus.setConfigReported(true);
+
+        }
+      }
+    }
+  }
+
+  private boolean canBeExported(String exportGroupName, String name, Set<String> appExports) {
+    return appExports.contains(String.format("%s-%s", exportGroupName, name));
+  }
+
+  protected Map<String, List<ExportEntry>> getCurrentExports(String groupName) {
+    if (!this.exportGroups.containsKey(groupName)) {
+      synchronized (this.exportGroups) {
+        if (!this.exportGroups.containsKey(groupName)) {
+          this.exportGroups.put(groupName, new ConcurrentHashMap<String, List<ExportEntry>>());
+        }
+      }
+    }
+
+    return this.exportGroups.get(groupName);
+  }
+
+  private void publishModifiedExportGroups(Set<String> modifiedGroups) {
+    for (String groupName : modifiedGroups) {
+      Map<String, List<ExportEntry>> entries = this.exportGroups.get(groupName);
+
+      // Publish in old format for the time being
+      Map<String, String> simpleEntries = new HashMap<String, String>();
+      for (Map.Entry<String, List<ExportEntry>> entry : entries.entrySet()) {
+        List<ExportEntry> exports = entry.getValue();
+        if (SliderUtils.isNotEmpty(exports)) {
+          // there is no support for multiple exports per name - so extract only the first one
+          simpleEntries.put(entry.getKey(), entry.getValue().get(0).getValue());
+        }
+      }
+      if (!getAmState().getAppConfSnapshot().getComponentOptBool(
+          groupName, AgentKeys.AM_CONFIG_GENERATION, false)) {
+        publishApplicationInstanceData(groupName, groupName,
+            simpleEntries.entrySet());
+      }
+
+      PublishedExports exports = new PublishedExports(groupName);
+      exports.setUpdated(new Date().getTime());
+      exports.putValues(entries.entrySet());
+      getAmState().getPublishedExportsSet().put(groupName, exports);
+    }
+  }
+
+  /** Publish component instance specific data if the component demands it */
+  protected void processAndPublishComponentSpecificData(Map<String, String> ports,
+                                                        String containerId,
+                                                        String hostFqdn,
+                                                        String componentGroup) {
+    String portVarFormat = "${site.%s}";
+    String hostNamePattern = "${THIS_HOST}";
+    Map<String, String> toPublish = new HashMap<String, String>();
+
+    Application application = getMetaInfo().getApplication();
+    for (Component component : application.getComponents()) {
+      if (component.getName().equals(componentGroup)) {
+        if (component.getComponentExports().size() > 0) {
+
+          for (ComponentExport export : component.getComponentExports()) {
+            String templateToExport = export.getValue();
+            for (String portName : ports.keySet()) {
+              boolean publishData = false;
+              String portValPattern = String.format(portVarFormat, portName);
+              if (templateToExport.contains(portValPattern)) {
+                templateToExport = templateToExport.replace(portValPattern, ports.get(portName));
+                publishData = true;
+              }
+              if (templateToExport.contains(hostNamePattern)) {
+                templateToExport = templateToExport.replace(hostNamePattern, hostFqdn);
+                publishData = true;
+              }
+              if (publishData) {
+                toPublish.put(export.getName(), templateToExport);
+                log.info("Publishing {} for name {} and container {}",
+                         templateToExport, export.getName(), containerId);
+              }
+            }
+          }
+        }
+      }
+    }
+
+    if (toPublish.size() > 0) {
+      Map<String, String> perContainerData = null;
+      if (!getComponentInstanceData().containsKey(containerId)) {
+        perContainerData = new ConcurrentHashMap<String, String>();
+      } else {
+        perContainerData = getComponentInstanceData().get(containerId);
+      }
+      perContainerData.putAll(toPublish);
+      getComponentInstanceData().put(containerId, perContainerData);
+      publishComponentInstanceData();
+    }
+  }
+
+  /** Publish component instance specific data if the component demands it */
+  protected void processAndPublishComponentSpecificExports(Map<String, String> ports,
+                                                           String containerId,
+                                                           String hostFqdn,
+                                                           String compName,
+                                                           String compGroup) {
+    String portVarFormat = "${site.%s}";
+    String hostNamePattern = "${" + compGroup + "_HOST}";
+
+    List<ExportGroup> appExportGroups = getMetaInfo().getApplication().getExportGroups();
+    Component component = getMetaInfo().getApplicationComponent(compGroup);
+    if (component != null && SliderUtils.isSet(component.getCompExports())
+        && SliderUtils.isNotEmpty(appExportGroups)) {
+
+      Set<String> compExports = new HashSet();
+      String compExportsStr = component.getCompExports();
+      for (String compExport : compExportsStr.split(",")) {
+        if (!compExport.trim().isEmpty()) {
+          compExports.add(compExport.trim());
+        }
+      }
+
+      Date now = new Date();
+      Set<String> modifiedGroups = new HashSet<String>();
+      for (ExportGroup exportGroup : appExportGroups) {
+        List<Export> exports = exportGroup.getExports();
+        if (SliderUtils.isNotEmpty(exports)) {
+          String exportGroupName = exportGroup.getName();
+          ConcurrentHashMap<String, List<ExportEntry>> map =
+              (ConcurrentHashMap<String, List<ExportEntry>>) getCurrentExports(exportGroupName);
+          for (Export export : exports) {
+            if (canBeExported(exportGroupName, export.getName(), compExports)) {
+              log.info("Attempting to publish {} of group {} for component type {}",
+                       export.getName(), exportGroupName, compName);
+              String templateToExport = export.getValue();
+              for (String portName : ports.keySet()) {
+                boolean publishData = false;
+                String portValPattern = String.format(portVarFormat, portName);
+                if (templateToExport.contains(portValPattern)) {
+                  templateToExport = templateToExport.replace(portValPattern, ports.get(portName));
+                  publishData = true;
+                }
+                if (templateToExport.contains(hostNamePattern)) {
+                  templateToExport = templateToExport.replace(hostNamePattern, hostFqdn);
+                  publishData = true;
+                }
+                if (publishData) {
+                  ExportEntry entryToAdd = new ExportEntry();
+                  entryToAdd.setLevel(COMPONENT_TAG);
+                  entryToAdd.setValue(templateToExport);
+                  entryToAdd.setUpdatedTime(now.toString());
+                  entryToAdd.setContainerId(containerId);
+                  entryToAdd.setTag(tags.getTag(compName, containerId));
+
+                  List<ExportEntry> existingList =
+                      map.putIfAbsent(export.getName(), new CopyOnWriteA

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[28/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ComponentArgsDelegate.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ComponentArgsDelegate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ComponentArgsDelegate.java
new file mode 100644
index 0000000..5140059
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ComponentArgsDelegate.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+public class ComponentArgsDelegate extends AbstractArgsDelegate {
+
+  /**
+   * This is a listing of the roles to create
+   */
+  @Parameter(names = {ARG_COMPONENT,  ARG_COMPONENT_SHORT, ARG_ROLE},
+             arity = 2,
+             description = "--component <name> <count> e.g. +1 incr by 1, -2 decr by 2, and 3 makes final count 3",
+             splitter = DontSplitArguments.class)
+  public List<String> componentTuples = new ArrayList<>(0);
+
+
+  /**
+   * Get the role mapping (may be empty, but never null)
+   * @return role mapping
+   * @throws BadCommandArgumentsException parse problem
+   */
+  public Map<String, String> getComponentMap() throws BadCommandArgumentsException {
+    return convertTupleListToMap("component", componentTuples);
+  }
+
+  public List<String> getComponentTuples() {
+    return componentTuples;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/DontSplitArguments.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/DontSplitArguments.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/DontSplitArguments.java
new file mode 100644
index 0000000..0344305
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/DontSplitArguments.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.converters.IParameterSplitter;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class DontSplitArguments implements IParameterSplitter {
+
+  @Override
+  public List<String> split(String value) {
+    List<String> list = new ArrayList<>(1);
+    list.add(value);
+    return list;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsAccessor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsAccessor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsAccessor.java
new file mode 100644
index 0000000..7524053
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsAccessor.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import java.io.File;
+
+/**
+ * Launch args for create and start and anything else that can start something
+ */
+public interface LaunchArgsAccessor extends WaitTimeAccessor {
+  String getRmAddress();
+
+  File getOutputFile();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsDelegate.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsDelegate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsDelegate.java
new file mode 100644
index 0000000..bc7e94c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/LaunchArgsDelegate.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+
+import java.io.File;
+
+/**
+ * Any launch-time args
+ */
+public class LaunchArgsDelegate extends WaitArgsDelegate implements
+                                                         LaunchArgsAccessor {
+
+
+  //TODO: do we need this?
+  @Parameter(names = ARG_RESOURCE_MANAGER,
+             description = "Resource manager hostname:port ",
+             required = false)
+  private String rmAddress;
+
+  @Override
+  public String getRmAddress() {
+    return rmAddress;
+  }
+
+  @Parameter(names = {ARG_OUTPUT, ARG_OUTPUT_SHORT},
+      description = "output file for any application report")
+  public File outputFile;
+
+  @Override
+  public File getOutputFile() {
+    return outputFile;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/PathArgumentConverter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/PathArgumentConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/PathArgumentConverter.java
new file mode 100644
index 0000000..ccb526c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/PathArgumentConverter.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.converters.BaseConverter;
+import org.apache.hadoop.fs.Path;
+
+public class PathArgumentConverter extends BaseConverter<Path> {
+
+  public PathArgumentConverter(String optionName) {
+    super(optionName);
+  }
+
+  @Override
+  public Path convert(String value) {
+    return new Path(value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/SliderAMArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/SliderAMArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/SliderAMArgs.java
new file mode 100644
index 0000000..f9516d1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/SliderAMArgs.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+
+package org.apache.slider.common.params;
+
+/**
+ * Parameters sent by the Client to the AM
+ */
+public class SliderAMArgs extends CommonArgs {
+
+  SliderAMCreateAction createAction = new SliderAMCreateAction();
+
+  public SliderAMArgs(String[] args) {
+    super(args);
+  }
+
+  @Override
+  protected void addActionArguments() {
+    addActions(createAction);
+  }
+
+  public String getImage() {
+    return createAction.image;
+  }
+
+  /**
+   * This is the URI in the FS to the Slider cluster; the conf file (and any
+   * other cluster-specifics) can be picked up here
+   */
+  public String getSliderClusterURI() {
+    return createAction.sliderClusterURI;
+  }
+
+  /**
+   * Am binding is simple: there is only one action
+   */
+  @Override
+  public void applyAction() {
+    bindCoreAction(createAction);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/SliderAMCreateAction.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/SliderAMCreateAction.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/SliderAMCreateAction.java
new file mode 100644
index 0000000..197c22b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/SliderAMCreateAction.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.beust.jcommander.ParametersDelegate;
+
+import java.io.File;
+
+
+@Parameters(commandNames = {SliderActions.ACTION_CREATE},
+            commandDescription = SliderActions.DESCRIBE_ACTION_CREATE)
+
+public class SliderAMCreateAction extends AbstractActionArgs implements
+                                                           LaunchArgsAccessor {
+
+
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_CREATE;
+  }
+
+  @Parameter(names = ARG_IMAGE, description = "image", required = false)
+  public String image;
+
+  /**
+   * This is the URI in the FS to the Slider cluster; the conf file (and any
+   * other cluster-specifics) can be picked up here
+   */
+  @Parameter(names = ARG_CLUSTER_URI,
+             description = "URI to the Slider cluster", required = true)
+  public String sliderClusterURI;
+
+  @ParametersDelegate
+  LaunchArgsDelegate launchArgs = new LaunchArgsDelegate();
+
+  @Override
+  public String getRmAddress() {
+    return launchArgs.getRmAddress();
+  }
+
+  @Override
+  public int getWaittime() {
+    return launchArgs.getWaittime();
+  }
+
+  @Override
+  public void setWaittime(int waittime) {
+    launchArgs.setWaittime(waittime);
+  }
+
+  @Override
+  public File getOutputFile() {
+    return launchArgs.getOutputFile();
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/SliderActions.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/SliderActions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/SliderActions.java
new file mode 100644
index 0000000..204ad9a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/SliderActions.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+/**
+ * Actions.
+ * Only some of these are supported by specific Slider Services; they
+ * are listed here to ensure the names are consistent
+ */
+public interface SliderActions {
+  String ACTION_AM_SUICIDE = "am-suicide";
+  String ACTION_BUILD = "build";
+  String ACTION_CLIENT = "client";
+  String ACTION_CREATE = "create";
+  String ACTION_DIAGNOSTICS = "diagnostics";
+  String ACTION_DEPENDENCY = "dependency";
+  String ACTION_UPDATE = "update";
+  String ACTION_UPGRADE = "upgrade";
+  String ACTION_DESTROY = "destroy";
+  String ACTION_ECHO = "echo";
+  String ACTION_EXISTS = "exists";
+  String ACTION_FLEX = "flex";
+  String ACTION_FREEZE = "stop";
+  String ACTION_HELP = "help";
+  String ACTION_INSTALL_KEYTAB = "install-keytab";
+  String ACTION_INSTALL_PACKAGE = "install-package";
+  String ACTION_KDIAG = "kdiag";
+  String ACTION_KEYTAB = "keytab";
+  String ACTION_KILL_CONTAINER = "kill-container";
+  String ACTION_LIST = "list";
+  String ACTION_LOOKUP = "lookup";
+  String ACTION_NODES = "nodes";
+  String ACTION_PACKAGE = "package";
+  String ACTION_PREFLIGHT = "preflight";
+  String ACTION_RECONFIGURE = "reconfigure";
+  String ACTION_REGISTRY = "registry";
+  String ACTION_RESOLVE = "resolve";
+  String ACTION_RESOURCE = "resource";
+  String ACTION_STATUS = "status";
+  String ACTION_THAW = "start";
+  String ACTION_TOKENS = "tokens";
+
+  String ACTION_VERSION = "version";
+  String DESCRIBE_ACTION_AM_SUICIDE =
+      "Tell the Slider Application Master to simulate a process failure by terminating itself";
+  String DESCRIBE_ACTION_BUILD =
+    "Build a Slider cluster specification, but do not start it";
+  String DESCRIBE_ACTION_CREATE =
+      "Create a live Slider application";
+  String DESCRIBE_ACTION_DEPENDENCY =
+      "Slider AM and agent dependency (libraries) management";
+  String DESCRIBE_ACTION_UPDATE =
+      "Update template for a Slider application";
+  String DESCRIBE_ACTION_UPGRADE =
+      "Rolling upgrade/downgrade the application to a newer/previous version";
+  String DESCRIBE_ACTION_DESTROY =
+        "Destroy a stopped Slider application";
+  String DESCRIBE_ACTION_EXISTS =
+            "Probe for an application running";
+  String DESCRIBE_ACTION_FLEX = "Flex a Slider application";
+  String DESCRIBE_ACTION_FREEZE =
+              "Stop a running application";
+  String DESCRIBE_ACTION_GETCONF =
+                "Get the configuration of an application";
+  String DESCRIBE_ACTION_KDIAG = "Diagnose Kerberos problems";
+  String DESCRIBE_ACTION_KILL_CONTAINER =
+    "Kill a container in the application";
+  String DESCRIBE_ACTION_HELP = "Print help information";
+  String DESCRIBE_ACTION_LIST =
+                  "List running Slider applications";
+  String DESCRIBE_ACTION_LOOKUP =
+                  "look up a YARN application";
+  String DESCRIBE_ACTION_NODES = "List the node information for the YARN cluster or a running application";
+  String DESCRIBE_ACTION_MONITOR =
+                    "Monitor a running application";
+  String DESCRIBE_ACTION_REGISTRY =
+                      "Query the registry of a YARN application";
+  String DESCRIBE_ACTION_RESOLVE =
+                      "Resolve or list records in the YARN registry";
+  String DESCRIBE_ACTION_STATUS =
+                      "Get the status of an application";
+  String DESCRIBE_ACTION_THAW =
+                        "Start a stopped application";
+  String DESCRIBE_ACTION_VERSION =
+                        "Print the Slider version information";
+  String DESCRIBE_ACTION_INSTALL_PACKAGE = "Install application package." +
+  		" Deprecated, use '" + ACTION_PACKAGE + " " + ClientArgs.ARG_INSTALL + "'.";
+  String DESCRIBE_ACTION_PACKAGE = "Install/list/delete application packages and list app instances that use the packages";
+  String DESCRIBE_ACTION_CLIENT = "Install the application client in the specified directory or obtain a client keystore or truststore";
+  String DESCRIBE_ACTION_INSTALL_KEYTAB = "Install the Kerberos keytab." +
+  		" Deprecated, use '" + ACTION_KEYTAB + " " + ClientArgs.ARG_INSTALL + "'.";
+  String DESCRIBE_ACTION_KEYTAB = "Manage a Kerberos keytab file (install, delete, list) in the sub-folder 'keytabs' of the user's Slider base directory";
+  String DESCRIBE_ACTION_DIAGNOSTIC = "Diagnose the configuration of the running slider application and slider client";
+  String DESCRIBE_ACTION_RESOURCE = "Manage a file (install, delete, list) in the 'resources' sub-folder of the user's Slider base directory";
+
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/URIArgumentConverter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/URIArgumentConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/URIArgumentConverter.java
new file mode 100644
index 0000000..b0d1ebf
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/URIArgumentConverter.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.converters.BaseConverter;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+
+public class URIArgumentConverter extends BaseConverter<URI> {
+
+  public URIArgumentConverter(String optionName) {
+    super(optionName);
+  }
+
+  @Override
+  public URI convert(String value) {
+    try {
+      return new URI(value);
+    } catch (URISyntaxException e) {
+      throw new RuntimeException("Cannot make a URI from " + value);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/URLArgumentConverter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/URLArgumentConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/URLArgumentConverter.java
new file mode 100644
index 0000000..8894309
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/URLArgumentConverter.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.converters.BaseConverter;
+
+import java.net.MalformedURLException;
+import java.net.URL;
+
+public class URLArgumentConverter extends BaseConverter<URL> {
+  public URLArgumentConverter(String optionName) {
+    super(optionName);
+  }
+
+  @Override
+  public URL convert(String value) {
+    try {
+      return new URL(value);
+    } catch (MalformedURLException e) {
+      throw new RuntimeException("Cannot make a URL from " + value);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/WaitArgsDelegate.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/WaitArgsDelegate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/WaitArgsDelegate.java
new file mode 100644
index 0000000..1c27c01
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/WaitArgsDelegate.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+
+public class WaitArgsDelegate extends AbstractArgsDelegate implements
+                                                           WaitTimeAccessor {
+
+
+  //--wait [timeout]
+  @Parameter(names = {ARG_WAIT},
+             description = "time to wait for an action to complete")
+  public int waittime = 0;
+
+
+  @Override
+  public int getWaittime() {
+    return waittime;
+  }
+
+  @Override
+  public void setWaittime(int waittime) {
+    this.waittime = waittime;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/WaitTimeAccessor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/WaitTimeAccessor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/WaitTimeAccessor.java
new file mode 100644
index 0000000..13d4d5a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/WaitTimeAccessor.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+public interface WaitTimeAccessor {
+  int getWaittime();
+  void setWaittime(int waittime);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/Comparators.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/Comparators.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/Comparators.java
new file mode 100644
index 0000000..6380d0c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/Comparators.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.tools;
+
+import java.io.Serializable;
+import java.util.Comparator;
+
+/**
+ * Some general comparators
+ */
+public class Comparators {
+
+  public static class LongComparator implements Comparator<Long>, Serializable {
+    @Override
+    public int compare(Long o1, Long o2) {
+      long result = o1 - o2;
+      // need to comparisons with a diff greater than integer size
+      if (result < 0 ) {
+        return -1;
+      } else if (result > 0) {
+        return 1;
+      }
+      return 0;
+    }
+  }
+public static class InvertedLongComparator implements Comparator<Long>, Serializable {
+  private static final LongComparator inner = new LongComparator();
+    @Override
+    public int compare(Long o1, Long o2) {
+      return -inner.compare(o1, o2);
+    }
+  }
+
+
+  /**
+   * Little template class to reverse any comparitor
+   * @param <CompareType> the type that is being compared
+   */
+  public static class ComparatorReverser<CompareType> implements Comparator<CompareType>,
+      Serializable {
+
+    final Comparator<CompareType> instance;
+
+    public ComparatorReverser(Comparator<CompareType> instance) {
+      this.instance = instance;
+    }
+
+    @Override
+    public int compare(CompareType first, CompareType second) {
+      return instance.compare(second, first);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/ConfigHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/ConfigHelper.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/ConfigHelper.java
new file mode 100644
index 0000000..9db241d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/ConfigHelper.java
@@ -0,0 +1,658 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.tools;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.registry.client.api.RegistryConstants;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.SliderXmlConfKeys;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.w3c.dom.Document;
+import org.xml.sax.SAXException;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.StringWriter;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * Methods to aid in config, both in the Configuration class and
+ * with other parts of setting up Slider-initated processes.
+ * 
+ * Some of the methods take an argument of a map iterable for their sources; this allows
+ * the same method
+ */
+public class ConfigHelper {
+  private static final Logger log = LoggerFactory.getLogger(ConfigHelper.class);
+
+  private static AtomicBoolean sliderResourceInjected =
+      new AtomicBoolean(false);
+  private static AtomicBoolean sliderResourceInjectionAttempted =
+      new AtomicBoolean(false);
+  
+  /**
+   * Dump the (sorted) configuration
+   * @param conf config
+   * @return the sorted keyset
+   */
+  public static Set<String> dumpConf(Configuration conf) {
+    Set<String> keys = sortedConfigKeys(conf);
+    for (String key : keys) {
+      log.info("{}={}", key, conf.get(key));
+    }
+    return keys;
+  }
+
+  /**
+   * Take a configuration and return a sorted set
+   * @param conf config
+   * @return the sorted keyset
+
+   */
+  public static Set<String> sortedConfigKeys(Iterable<Map.Entry<String, String>> conf) {
+    TreeSet<String> sorted = new TreeSet<String>();
+    for (Map.Entry<String, String> entry : conf) {
+      sorted.add(entry.getKey());
+    }
+    return sorted;
+  }
+
+  /**
+   * Set an entire map full of values
+   *
+   * @param config config to patch
+   * @param map map of data
+   * @param origin origin data
+   */
+  public static void addConfigMap(Configuration config,
+                                  Map<String, String> map,
+                                  String origin) throws BadConfigException {
+    addConfigMap(config, map.entrySet(), origin);
+  }
+  
+  /**
+   * Set an entire map full of values
+   *
+   * @param config config to patch
+   * @param map map of data
+   * @param origin origin data
+   */
+  public static void addConfigMap(Configuration config,
+                                  Iterable<Map.Entry<String, String>> map,
+                                  String origin) throws BadConfigException {
+    for (Map.Entry<String, String> mapEntry : map) {
+      String key = mapEntry.getKey();
+      String value = mapEntry.getValue();
+      if (value == null) {
+        throw new BadConfigException("Null value for property " + key);
+      }
+      config.set(key, value, origin);
+    }
+  }
+
+
+  /**
+   * Save a config file in a destination directory on a given filesystem
+   * @param systemConf system conf used for creating filesystems
+   * @param confToSave config to save
+   * @param confdir the directory path where the file is to go
+   * @param filename the filename
+   * @return the destination path where the file was saved
+   * @throws IOException IO problems
+   */
+  public static Path saveConfig(Configuration systemConf,
+                                Configuration confToSave,
+                                Path confdir,
+                                String filename) throws IOException {
+    FileSystem fs = FileSystem.get(confdir.toUri(), systemConf);
+    Path destPath = new Path(confdir, filename);
+    saveConfig(fs, destPath, confToSave);
+    return destPath;
+  }
+
+  /**
+   * Save a config
+   * @param fs filesystem
+   * @param destPath dest to save
+   * @param confToSave  config to save
+   * @throws IOException IO problems
+   */
+  public static void saveConfig(FileSystem fs,
+                                Path destPath,
+                                Configuration confToSave) throws
+                                                              IOException {
+    FSDataOutputStream fos = fs.create(destPath);
+    try {
+      confToSave.writeXml(fos);
+    } finally {
+      IOUtils.closeStream(fos);
+    }
+  }
+
+  /**
+   * Convert to an XML string
+   * @param conf configuration
+   * @return conf
+   * @throws IOException
+   */
+  public static String toXml(Configuration conf) throws IOException {
+    StringWriter writer = new StringWriter();
+    conf.writeXml(writer);
+    return writer.toString();
+  }
+  
+  /**
+   * This will load and parse a configuration to an XML document
+   * @param fs filesystem
+   * @param path path
+   * @return an XML document
+   * @throws IOException IO failure
+   */
+  public Document parseConfiguration(FileSystem fs,
+                                     Path path) throws
+                                                IOException {
+
+
+    byte[] data = loadBytes(fs, path);
+    //this is here to track down a parse issue
+    //related to configurations
+    String s = new String(data, 0, data.length);
+    log.debug("XML resource {} is \"{}\"", path, s);
+/* JDK7
+    try (ByteArrayInputStream in = new ByteArrayInputStream(data)) {
+      Document document = parseConfigXML(in);
+      return document;
+    } catch (ParserConfigurationException | SAXException e) {
+      throw new IOException(e);
+    }
+*/
+    ByteArrayInputStream in= null;
+    try {
+      in = new ByteArrayInputStream(data);
+      Document document = parseConfigXML(in);
+      return document;
+    } catch (ParserConfigurationException e) {
+      throw new IOException(e);
+    } catch (SAXException e) {
+      throw new IOException(e);
+    } finally {
+      IOUtils.closeStream(in);
+    }
+  }
+
+  public static byte[] loadBytes(FileSystem fs, Path path) throws IOException {
+    int len = (int) fs.getLength(path);
+    byte[] data = new byte[len];
+    /* JDK7
+    try(FSDataInputStream in = fs.open(path)) {
+      in.readFully(0, data);
+    }
+*/
+    FSDataInputStream in = null;
+    in = fs.open(path);
+    try {
+      in.readFully(0, data);
+    } finally {
+      IOUtils.closeStream(in);
+    }
+    return data;
+  }
+
+  /**
+   * Load a configuration from ANY FS path. The normal Configuration
+   * loader only works with file:// URIs
+   * @param fs filesystem
+   * @param path path
+   * @return a loaded resource
+   * @throws IOException
+   */
+  public static Configuration loadConfiguration(FileSystem fs,
+                                                Path path) throws IOException {
+    byte[] data = loadBytes(fs, path);
+
+    ByteArrayInputStream in2;
+
+    in2 = new ByteArrayInputStream(data);
+    Configuration conf1 = new Configuration(false);
+    conf1.addResource(in2);
+    //now clone it while dropping all its sources
+    Configuration conf2   = new Configuration(false);
+    String src = path.toString();
+    for (Map.Entry<String, String> entry : conf1) {
+      String key = entry.getKey();
+      String value = entry.getValue();
+      conf2.set(key, value, src);
+    }
+    return conf2;
+  }
+
+
+  /**
+   * Generate a config file in a destination directory on the local filesystem
+   * @param confdir the directory path where the file is to go
+   * @param filename the filename
+   * @return the destination path
+   */
+  public static File saveConfig(Configuration generatingConf,
+                                    File confdir,
+                                    String filename) throws IOException {
+
+
+    File destPath = new File(confdir, filename);
+    OutputStream fos = new FileOutputStream(destPath);
+    try {
+      generatingConf.writeXml(fos);
+    } finally {
+      IOUtils.closeStream(fos);
+    }
+    return destPath;
+  }
+
+  /**
+   * Parse an XML Hadoop configuration into an XML document. x-include
+   * is supported, but as the location isn't passed in, relative
+   * URIs are out.
+   * @param in instream
+   * @return a document
+   * @throws ParserConfigurationException parser feature problems
+   * @throws IOException IO problems
+   * @throws SAXException XML is invalid
+   */
+  public static Document parseConfigXML(InputStream in) throws
+                                               ParserConfigurationException,
+                                               IOException,
+                                               SAXException {
+    DocumentBuilderFactory docBuilderFactory
+      = DocumentBuilderFactory.newInstance();
+    //ignore all comments inside the xml file
+    docBuilderFactory.setIgnoringComments(true);
+
+    //allow includes in the xml file
+    docBuilderFactory.setNamespaceAware(true);
+    docBuilderFactory.setXIncludeAware(true);
+    DocumentBuilder builder = docBuilderFactory.newDocumentBuilder();
+    return builder.parse(in);
+  }
+
+  /**
+   * Load a Hadoop configuration from a local file.
+   * @param file file to load
+   * @return a configuration which hasn't actually had the load triggered
+   * yet.
+   * @throws FileNotFoundException file is not there
+   * @throws IOException any other IO problem
+   */
+  public static Configuration loadConfFromFile(File file) throws
+                                                          IOException {
+    return loadConfFromFile(file, false);
+  }
+
+  /**
+   *
+   * Load a Hadoop configuration from a local file.
+   * @param file file to load
+   * @param loadDefaults flag to indicate if the defaults should be loaded yet
+   * @return a configuration which hasn't actually had the load triggered
+   * yet.
+   * @throws FileNotFoundException file is not there
+   * @throws IOException any other IO problem
+   */
+  public static Configuration loadConfFromFile(File file,
+      boolean loadDefaults) throws IOException {
+    if (!file.exists()) {
+      throw new FileNotFoundException("File not found :"
+                                          + file.getAbsoluteFile());
+    }
+    Configuration conf = new Configuration(loadDefaults);
+    try {
+      conf.addResource(file.toURI().toURL());
+    } catch (MalformedURLException e) {
+      // should never happen...
+      throw new IOException(
+        "File " + file.toURI() + " doesn't have a valid URL");
+    }
+    return conf;
+  }
+
+  /**
+   * Add a configuration from a file to an existing configuration
+   * @param conf existing configuration
+   * @param file file to load
+   * @param overwrite flag to indicate new values should overwrite the predecessor
+   * @return the merged configuration
+   * @throws IOException
+   */
+  public static Configuration addConfigurationFile(Configuration conf,
+      File file, boolean overwrite)
+      throws IOException {
+    Configuration c2 = loadConfFromFile(file, false);
+    mergeConfigurations(conf, c2, file.getAbsolutePath(), overwrite);
+    return conf;
+  }
+
+  /**
+   * Add the system env variables with the given prefix (by convention, env.)
+   * @param conf existing configuration
+   * @param prefix prefix
+   */
+  public static void addEnvironmentVariables(Configuration conf, String prefix) {
+    Map<String, String> env = System.getenv();
+    for (Map.Entry<String, String> entry : env.entrySet()) {
+      conf.set(prefix + entry.getKey(),entry.getValue(), "env");
+    }
+  }
+  
+  /**
+   * looks for the config under $confdir/$templateFilename; if not there
+   * loads it from /conf/templateFile.
+   * The property {@link SliderKeys#KEY_TEMPLATE_ORIGIN} is set to the
+   * origin to help debug what's happening
+   * @param systemConf system conf
+   * @param confdir conf dir in FS
+   * @param templateFilename filename in the confdir
+   * @param fallbackResource resource to fall back on
+   * @return loaded conf
+   * @throws IOException IO problems
+   */
+  public static Configuration loadTemplateConfiguration(Configuration systemConf,
+                                                        Path confdir,
+                                                        String templateFilename,
+                                                        String fallbackResource) throws
+                                                                         IOException {
+    FileSystem fs = FileSystem.get(confdir.toUri(), systemConf);
+
+    Path templatePath = new Path(confdir, templateFilename);
+    return loadTemplateConfiguration(fs, templatePath, fallbackResource);
+  }
+
+  /**
+   * looks for the config under $confdir/$templateFilename; if not there
+   * loads it from /conf/templateFile.
+   * The property {@link SliderKeys#KEY_TEMPLATE_ORIGIN} is set to the
+   * origin to help debug what's happening.
+   * @param fs Filesystem
+   * @param templatePath HDFS path for template
+   * @param fallbackResource resource to fall back on, or "" for no fallback
+   * @return loaded conf
+   * @throws IOException IO problems
+   * @throws FileNotFoundException if the path doesn't have a file and there
+   * was no fallback.
+   */
+  public static Configuration loadTemplateConfiguration(FileSystem fs,
+                                                        Path templatePath,
+                                                        String fallbackResource)
+      throws IOException {
+    Configuration conf;
+    String origin;
+    if (fs.exists(templatePath)) {
+      log.debug("Loading template configuration {}", templatePath);
+      conf = loadConfiguration(fs, templatePath);
+      origin = templatePath.toString();
+    } else {
+      if (fallbackResource.isEmpty()) {
+        throw new FileNotFoundException("No config file found at " + templatePath);
+      }
+      log.debug("Template {} not found" +
+                " -reverting to classpath resource {}", templatePath, fallbackResource);
+      conf = new Configuration(false);
+      conf.addResource(fallbackResource);
+      origin = "Resource " + fallbackResource;
+    }
+    //force a get
+    conf.get(SliderXmlConfKeys.KEY_TEMPLATE_ORIGIN);
+    //now set the origin
+    conf.set(SliderXmlConfKeys.KEY_TEMPLATE_ORIGIN, origin);
+    return conf;
+  }
+
+
+  /**
+   * For testing: dump a configuration
+   * @param conf configuration
+   * @return listing in key=value style
+   */
+  public static String dumpConfigToString(Configuration conf) {
+    Set<String> sorted = sortedConfigKeys(conf);
+
+    StringBuilder builder = new StringBuilder();
+    for (String key : sorted) {
+
+      builder.append(key)
+             .append("=")
+             .append(conf.get(key))
+             .append("\n");
+    }
+    return builder.toString();
+  }
+
+  /**
+   * Merge in one configuration above another
+   * @param base base config
+   * @param merge one to merge. This MUST be a non-default-load config to avoid
+   * merge origin confusion
+   * @param origin description of the origin for the put operation
+   * @param overwrite flag to indicate new values should overwrite the predecessor
+   * @return the base with the merged values
+   */
+  public static Configuration mergeConfigurations(Configuration base,
+      Iterable<Map.Entry<String, String>> merge,
+      String origin,
+      boolean overwrite) {
+    for (Map.Entry<String, String> entry : merge) {
+      String key = entry.getKey();
+      if (overwrite || base.get(key) == null) {
+        base.set(key, entry.getValue(), origin);
+      }
+    }
+    return base;
+  }
+
+  /**
+   * Register a resource as a default resource.
+   * Do not attempt to use this unless you understand that the
+   * order in which default resources are loaded affects the outcome,
+   * and that subclasses of Configuration often register new default
+   * resources
+   * @param resource the resource name
+   * @return the URL or null
+   */
+  public static URL registerDefaultResource(String resource) {
+    URL resURL = getResourceUrl(resource);
+    if (resURL != null) {
+      Configuration.addDefaultResource(resource);
+    }
+    return resURL;
+  }
+
+  /**
+   * Load a configuration from a resource on this classpath.
+   * If the resource is not found, an empty configuration is returned
+   * @param resource the resource name
+   * @return the loaded configuration.
+   */
+  public static Configuration loadFromResource(String resource) {
+    Configuration conf = new Configuration(false);
+    URL resURL = getResourceUrl(resource);
+    if (resURL != null) {
+      log.debug("loaded resources from {}", resURL);
+      conf.addResource(resource);
+    } else{
+      log.debug("failed to find {} on the classpath", resource);
+    }
+    return conf;
+    
+  }
+
+  /**
+   * Get the URL to a resource, null if not on the CP
+   * @param resource resource to look for
+   * @return the URL or null
+   */
+  public static URL getResourceUrl(String resource) {
+    return ConfigHelper.class.getClassLoader()
+                                  .getResource(resource);
+  }
+
+  /**
+   * Load a resource that must be on the classpath
+   * @param resource the resource name
+   * @return the loaded configuration
+   * @throws FileNotFoundException if the resource is missing
+   */
+  public static Configuration loadMandatoryResource(String resource)
+      throws FileNotFoundException {
+    Configuration conf = new Configuration(false);
+    URL resURL = getResourceUrl(resource);
+    if (resURL != null) {
+      log.debug("loaded resources from {}", resURL);
+      conf.addResource(resource);
+    } else {
+      throw new FileNotFoundException(resource);
+    }
+    return conf;
+  }
+
+  /**
+   * Propagate a property from a source to a dest config, with a best-effort
+   * attempt at propagating the origin.
+   * If the 
+   * @param dest destination
+   * @param src source
+   * @param key key to try to copy
+   * @return true if the key was found and propagated
+   */
+  public static boolean propagate(Configuration dest,
+                                  Configuration src,
+                                  String key) {
+    String val = src.get(key);
+    if (val != null) {
+      String[] origin = src.getPropertySources(key);
+      if (origin != null && origin.length > 0) {
+        dest.set(key, val, origin[0]);
+      } else {
+        dest.set(key, val);
+        return true;
+      }
+    }
+    return false;
+  }
+
+
+  /**
+   * Take a configuration, return a hash map
+   * @param conf conf
+   * @return hash map
+   */
+  public static Map<String, String> buildMapFromConfiguration(Configuration conf) {
+    Map<String, String> map = new HashMap<String, String>();
+    return SliderUtils.mergeEntries(map, conf);
+  }
+
+  /**
+   * This goes through the keyset of one configuration and retrieves each value
+   * from a value source -a different or the same configuration. This triggers
+   * the property resolution process of the value, resolving any variables against
+   * in-config or inherited configurations
+   * @param keysource source of keys
+   * @param valuesource the source of values
+   * @return a new configuration where <code>foreach key in keysource, get(key)==valuesource.get(key)</code>
+   */
+  public static Configuration resolveConfiguration(
+      Iterable<Map.Entry<String, String>> keysource,
+      Configuration valuesource) {
+    Configuration result = new Configuration(false);
+    for (Map.Entry<String, String> entry : keysource) {
+      String key = entry.getKey();
+      String value = valuesource.get(key);
+      Preconditions.checkState(value != null,
+          "no reference for \"%s\" in values", key);
+      result.set(key, value);
+    }
+    return result;
+  }
+
+  /**
+   * Register anything we consider deprecated
+   */
+  public static void registerDeprecatedConfigItems() {
+    Configuration.addDeprecation(
+        SliderXmlConfKeys.REGISTRY_ZK_QUORUM,
+        RegistryConstants.KEY_REGISTRY_ZK_QUORUM);
+    Configuration.addDeprecation(
+        SliderXmlConfKeys.REGISTRY_PATH,
+        RegistryConstants.KEY_REGISTRY_ZK_ROOT);
+    
+  }
+
+  /**
+   * Load a configuration with the {@link SliderKeys#SLIDER_XML} resource
+   * included
+   * @return a configuration instance
+   */
+  public static Configuration loadSliderConfiguration() {
+    Configuration conf = new Configuration();
+    conf.addResource(SliderKeys.SLIDER_XML);
+    return conf;
+  }
+
+  /**
+   * Inject the {@link SliderKeys#SLIDER_XML} resource
+   * into the configuration resources <i>of all configurations</i>.
+   * <p>
+   *   This operation is idempotent.
+   * <p>
+   * If the resource is not on the classpath, downgrades, rather than
+   * fails.
+   * @return true if the resource was found and loaded.
+   */
+  public static synchronized boolean injectSliderXMLResource() {
+    if (sliderResourceInjectionAttempted.getAndSet(true)) {
+      return sliderResourceInjected.get();
+    }
+    URL resourceUrl = getResourceUrl(SliderKeys.SLIDER_XML);
+    if (resourceUrl != null) {
+      Configuration.addDefaultResource(SliderKeys.SLIDER_XML);
+      sliderResourceInjected.set(true);
+    }
+    return sliderResourceInjected.get();
+  }
+  
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[54/76] [abbrv] hadoop git commit: YARN-5778. Add .keep file for yarn native services AM web app. Contributed by Billie Rinaldi

Posted by ji...@apache.org.
YARN-5778. Add .keep file for yarn native services AM web app. Contributed by Billie Rinaldi


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

Branch: refs/heads/yarn-native-services
Commit: 9c4a2d6ceb2b7181b3946b3b430fd86712dad2e3
Parents: 7ec43e6
Author: Gour Saha <go...@apache.org>
Authored: Tue Oct 25 10:06:16 2016 -0700
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 .../src/main/resources/webapps/slideram/.keep                        | 0
 1 file changed, 0 insertions(+), 0 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c4a2d6c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/webapps/slideram/.keep
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/webapps/slideram/.keep b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/webapps/slideram/.keep
new file mode 100644
index 0000000..e69de29


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[56/76] [abbrv] hadoop git commit: YARN-5690. Integrate native services modules into maven build. Contributed by Billie Rinaldi

Posted by ji...@apache.org.
YARN-5690. Integrate native services modules into maven build. Contributed by Billie Rinaldi


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

Branch: refs/heads/yarn-native-services
Commit: 00c38659e43d6b99de43212470ed7a4c8d45d48a
Parents: 432974a
Author: Gour Saha <go...@apache.org>
Authored: Thu Oct 27 08:50:36 2016 -0700
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 .../resources/assemblies/hadoop-yarn-dist.xml   | 26 ++++++++
 .../assemblies/hadoop-yarn-services-api.xml     | 36 +++++++++++
 .../assemblies/hadoop-yarn-slider-dist.xml      | 30 +++++++++
 hadoop-project/pom.xml                          |  6 ++
 hadoop-yarn-project/hadoop-yarn/bin/yarn        | 30 +++++++++
 .../hadoop-yarn-services-api/pom.xml            | 44 +++++++++++--
 .../conf/slideram-log4j.properties              | 68 ++++++++++++++++++++
 .../hadoop-yarn-slider-core/pom.xml             | 38 +++++++++++
 .../org/apache/slider/client/SliderClient.java  | 29 +++++++--
 .../org/apache/slider/common/SliderKeys.java    |  2 +-
 .../apache/slider/common/tools/SliderUtils.java | 48 ++++++++++----
 .../providers/agent/AgentClientProvider.java    |  3 +-
 .../slideram/SliderAMClientProvider.java        | 15 +++--
 .../TestPublishedConfigurationOutputter.java    | 10 ++-
 14 files changed, 343 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/00c38659/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml
----------------------------------------------------------------------
diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml
index c3f459c..552087c 100644
--- a/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml
+++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml
@@ -86,6 +86,32 @@
       </includes>
     </fileSet>
     <fileSet>
+      <directory>hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/target</directory>
+      <outputDirectory>/share/hadoop/${hadoop.component}/sources</outputDirectory>
+      <includes>
+        <include>*-sources.jar</include>
+      </includes>
+    </fileSet>
+    <fileSet>
+      <directory>hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/conf</directory>
+      <outputDirectory>etc/hadoop</outputDirectory>
+    </fileSet>
+    <fileSet>
+      <directory>hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/target/hadoop-yarn-slider-core-${project.version}</directory>
+      <outputDirectory>/share/hadoop/${hadoop.component}/lib/slider</outputDirectory>
+    </fileSet>
+    <fileSet>
+      <directory>hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/target</directory>
+      <outputDirectory>/share/hadoop/${hadoop.component}/sources</outputDirectory>
+      <includes>
+        <include>*-sources.jar</include>
+      </includes>
+    </fileSet>
+    <fileSet>
+      <directory>hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/target/hadoop-yarn-services-api-${project.version}</directory>
+      <outputDirectory>/share/hadoop/${hadoop.component}/lib/services-api</outputDirectory>
+    </fileSet>
+    <fileSet>
       <directory>hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/target</directory>
       <outputDirectory>/share/hadoop/${hadoop.component}/sources</outputDirectory>
       <includes>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00c38659/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-services-api.xml
----------------------------------------------------------------------
diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-services-api.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-services-api.xml
new file mode 100644
index 0000000..589f724
--- /dev/null
+++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-services-api.xml
@@ -0,0 +1,36 @@
+<!--
+   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.01
+
+   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.
+-->
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0"
+          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+          xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0 http://maven.apache.org/xsd/assembly-1.1.0.xsd">
+  <id>hadoop-yarn-services-api-dist</id>
+  <formats>
+    <format>dir</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+  <dependencySets>
+    <dependencySet>
+      <useProjectArtifact>false</useProjectArtifact>
+      <includes>
+        <include>com.fasterxml.jackson.jaxrs:jackson-jaxrs-base</include>
+        <include>com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider</include>
+        <include>com.fasterxml.jackson.module:jackson-module-jaxb-annotations</include>
+        <include>io.swagger:swagger-annotations</include>
+      </includes>
+    </dependencySet>
+  </dependencySets>
+</assembly>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00c38659/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-slider-dist.xml
----------------------------------------------------------------------
diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-slider-dist.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-slider-dist.xml
new file mode 100644
index 0000000..5de45a9
--- /dev/null
+++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-slider-dist.xml
@@ -0,0 +1,30 @@
+<!--
+   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.01
+
+   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.
+-->
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0"
+          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+          xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0 http://maven.apache.org/xsd/assembly-1.1.0.xsd">
+  <id>hadoop-yarn-slider-dist</id>
+  <formats>
+    <format>dir</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+  <dependencySets>
+    <dependencySet>
+      <useProjectArtifact>false</useProjectArtifact>
+    </dependencySet>
+  </dependencySets>
+</assembly>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00c38659/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index a2b6e25..96b3d86 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -343,6 +343,12 @@
 
       <dependency>
         <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-yarn-slider-core</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
          <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
         <version>${project.version}</version>
         <type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00c38659/hadoop-yarn-project/hadoop-yarn/bin/yarn
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/bin/yarn b/hadoop-yarn-project/hadoop-yarn/bin/yarn
index 804fd1a..26d54b8 100755
--- a/hadoop-yarn-project/hadoop-yarn/bin/yarn
+++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn
@@ -47,7 +47,9 @@ function hadoop_usage
   hadoop_add_subcommand "resourcemanager" "run the ResourceManager"
   hadoop_add_subcommand "rmadmin" "admin tools"
   hadoop_add_subcommand "scmadmin" "SharedCacheManager admin tools"
+  hadoop_add_subcommand "services-api" "run slider services api"
   hadoop_add_subcommand "sharedcachemanager" "run the SharedCacheManager daemon"
+  hadoop_add_subcommand "slider" "run a slider app"
   hadoop_add_subcommand "timelinereader" "run the timeline reader server"
   hadoop_add_subcommand "timelineserver" "run the timeline server"
   hadoop_add_subcommand "top" "view cluster information"
@@ -142,10 +144,38 @@ function yarncmd_case
     scmadmin)
       HADOOP_CLASSNAME='org.apache.hadoop.yarn.client.SCMAdmin'
     ;;
+    services-api)
+      HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
+      hadoop_add_classpath "${HADOOP_YARN_HOME}/${YARN_LIB_JARS_DIR}/slider"'/*'
+      hadoop_add_classpath "${HADOOP_YARN_HOME}/${YARN_LIB_JARS_DIR}/services-api"'/*'
+      HADOOP_CLASSNAME='org.apache.hadoop.yarn.services.webapp.ApplicationApiWebApp'
+      hadoop_debug "Append YARN_CLIENT_OPTS onto HADOOP_OPTS"
+      HADOOP_OPTS="${HADOOP_OPTS} ${YARN_CLIENT_OPTS} \
+-Dslider.libdir=${HADOOP_YARN_HOME}/${YARN_DIR},\
+${HADOOP_YARN_HOME}/${YARN_LIB_JARS_DIR},\
+${HADOOP_YARN_HOME}/${YARN_LIB_JARS_DIR}/slider,\
+${HADOOP_HDFS_HOME}/${HDFS_DIR},\
+${HADOOP_HDFS_HOME}/${HDFS_LIB_JARS_DIR},\
+${HADOOP_COMMON_HOME}/${HADOOP_COMMON_DIR},\
+${HADOOP_COMMON_HOME}/${HADOOP_COMMON_LIB_JARS_DIR}"
+    ;;
     sharedcachemanager)
       HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
       HADOOP_CLASSNAME='org.apache.hadoop.yarn.server.sharedcachemanager.SharedCacheManager'
     ;;
+    slider)
+      hadoop_add_classpath "${HADOOP_YARN_HOME}/${YARN_LIB_JARS_DIR}/slider"'/*'
+      HADOOP_CLASSNAME='org.apache.slider.Slider'
+      hadoop_debug "Append YARN_CLIENT_OPTS onto HADOOP_OPTS"
+      HADOOP_OPTS="${HADOOP_OPTS} ${YARN_CLIENT_OPTS} \
+-Dslider.libdir=${HADOOP_YARN_HOME}/${YARN_DIR},\
+${HADOOP_YARN_HOME}/${YARN_LIB_JARS_DIR},\
+${HADOOP_YARN_HOME}/${YARN_LIB_JARS_DIR}/slider,\
+${HADOOP_HDFS_HOME}/${HDFS_DIR},\
+${HADOOP_HDFS_HOME}/${HDFS_LIB_JARS_DIR},\
+${HADOOP_COMMON_HOME}/${HADOOP_COMMON_DIR},\
+${HADOOP_COMMON_HOME}/${HADOOP_COMMON_LIB_JARS_DIR}"
+    ;;
     timelinereader)
       HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
       HADOOP_CLASSNAME='org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderServer'

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00c38659/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/pom.xml
index 78b7855..c198f28 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/pom.xml
@@ -159,12 +159,6 @@
     <dependency>
       <groupId>com.fasterxml.jackson.jaxrs</groupId>
       <artifactId>jackson-jaxrs-json-provider</artifactId>
-      <exclusions>
-        <exclusion>
-          <groupId>com.fasterxml.jackson.jaxrs</groupId>
-          <artifactId>jackson-jaxrs-base</artifactId>
-        </exclusion>
-      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.mockito</groupId>
@@ -192,6 +186,44 @@
   </dependencies>
 
   <profiles>
+    <profile>
+      <id>dist</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-assembly-plugin</artifactId>
+            <dependencies>
+              <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-assemblies</artifactId>
+                <version>${project.version}</version>
+              </dependency>
+            </dependencies>
+            <executions>
+              <execution>
+                <id>dist</id>
+                <phase>prepare-package</phase>
+                <goals>
+                  <goal>single</goal>
+                </goals>
+                <configuration>
+                  <appendAssemblyId>false</appendAssemblyId>
+                  <attach>false</attach>
+                  <finalName>${project.artifactId}-${project.version}</finalName>
+                  <descriptorRefs>
+                    <descriptorRef>hadoop-yarn-services-api</descriptorRef>
+                  </descriptorRefs>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
 
     <profile>
       <id>rat</id>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00c38659/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/conf/slideram-log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/conf/slideram-log4j.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/conf/slideram-log4j.properties
new file mode 100644
index 0000000..333859e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/conf/slideram-log4j.properties
@@ -0,0 +1,68 @@
+# 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.
+#
+
+# This is the log4j configuration for Slider Application Master
+
+# Log rotation based on size (100KB) with a max of 10 backup files
+log4j.rootLogger=INFO, amlog
+log4j.threshhold=ALL
+log4j.appender.amlog=org.apache.log4j.RollingFileAppender
+log4j.appender.amlog.layout=org.apache.log4j.PatternLayout
+log4j.appender.amlog.File=${LOG_DIR}/slider.log
+log4j.appender.amlog.MaxFileSize=1MB
+log4j.appender.amlog.MaxBackupIndex=10
+
+# log layout skips stack-trace creation operations by avoiding line numbers and method
+log4j.appender.amlog.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} - %m%n
+
+# debug edition is much more expensive
+#log4j.appender.amlog.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n
+
+# configure stderr
+# set the conversion pattern of stderr
+# Print the date in ISO 8601 format
+log4j.appender.stderr=org.apache.log4j.ConsoleAppender
+log4j.appender.stderr.Target=System.err
+log4j.appender.stderr.layout=org.apache.log4j.PatternLayout
+log4j.appender.stderr.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} - %m%n
+
+log4j.appender.subprocess=org.apache.log4j.ConsoleAppender
+log4j.appender.subprocess.layout=org.apache.log4j.PatternLayout
+log4j.appender.subprocess.layout.ConversionPattern=[%c{1}]: %m%n
+#log4j.logger.org.apache.slider.yarn.appmaster.SliderAppMasterer.master=INFO,subprocess
+
+# for debugging Slider
+#log4j.logger.org.apache.slider=DEBUG
+
+# uncomment to debug service lifecycle issues
+#log4j.logger.org.apache.hadoop.yarn.service.launcher=DEBUG
+#log4j.logger.org.apache.hadoop.yarn.service=DEBUG
+
+# uncomment for YARN operations
+#log4j.logger.org.apache.hadoop.yarn.client=DEBUG
+
+# uncomment this to debug security problems
+#log4j.logger.org.apache.hadoop.security=DEBUG
+
+#crank back on some noise
+log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR
+log4j.logger.org.apache.hadoop.hdfs=WARN
+log4j.logger.org.apache.hadoop.hdfs.shortcircuit=ERROR
+
+log4j.logger.org.apache.zookeeper=WARN
+log4j.logger.org.apache.curator.framework.state=ERROR
+log4j.logger.org.apache.curator.framework.imps=WARN

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00c38659/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
index d778f44..a2c67c0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
@@ -332,6 +332,44 @@
 
   <profiles>
     <profile>
+      <id>dist</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-assembly-plugin</artifactId>
+            <dependencies>
+              <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-assemblies</artifactId>
+                <version>${project.version}</version>
+              </dependency>
+            </dependencies>
+            <executions>
+              <execution>
+                <id>dist</id>
+                <phase>prepare-package</phase>
+                <goals>
+                  <goal>single</goal>
+                </goals>
+                <configuration>
+                  <appendAssemblyId>false</appendAssemblyId>
+                  <attach>false</attach>
+                  <finalName>${project.artifactId}-${project.version}</finalName>
+                  <descriptorRefs>
+                    <descriptorRef>hadoop-yarn-slider-dist</descriptorRef>
+                  </descriptorRefs>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+    <profile>
       <id>compile-protobuf</id>
       <build>
         <plugins>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00c38659/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
index 4b546cd..d1f88c5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
@@ -2162,6 +2162,24 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
           new File(confDir, SliderKeys.LOG4J_SERVER_PROP_FILENAME);
       hasServerLog4jProperties = log4jserver.isFile();
     }
+    if (!hasServerLog4jProperties) {
+      // check for log4j properties in hadoop conf dir
+      String hadoopConfDir = System.getenv(ApplicationConstants.Environment
+          .HADOOP_CONF_DIR.name());
+      if (hadoopConfDir != null) {
+        File localFile = new File(hadoopConfDir, SliderKeys
+            .LOG4J_SERVER_PROP_FILENAME);
+        if (localFile.exists()) {
+          Path localFilePath = createLocalPath(localFile);
+          remoteConfPath = new Path(clusterDirectory,
+              SliderKeys.SUBMITTED_CONF_DIR);
+          Path remoteFilePath = new Path(remoteConfPath, SliderKeys
+              .LOG4J_SERVER_PROP_FILENAME);
+          copy(config, localFilePath, remoteFilePath);
+          hasServerLog4jProperties = true;
+        }
+      }
+    }
     // the assumption here is that minimr cluster => this is a test run
     // and the classpath can look after itself
 
@@ -2300,7 +2318,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     // enable asserts
     commandLine.enableJavaAssertions();
     
-    // if the conf dir has a log4j-server.properties, switch to that
+    // if the conf dir has a slideram-log4j.properties, switch to that
     if (hasServerLog4jProperties) {
       commandLine.sysprop(SYSPROP_LOG4J_CONFIGURATION, LOG4J_SERVER_PROP_FILENAME);
       commandLine.sysprop(SYSPROP_LOG_DIR, ApplicationConstants.LOG_DIR_EXPANSION_VAR);
@@ -4471,14 +4489,13 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
       return EXIT_SUCCESS;
     }
     
-    String libDir = System.getProperty(SliderKeys.PROPERTY_LIB_DIR);
-    if (isSet(libDir)) {
-      File srcFolder = new File(libDir);
+    String[] libDirs = SliderUtils.getLibDirs();
+    if (libDirs.length > 0) {
       File tempLibTarGzipFile = File.createTempFile(
           SliderKeys.SLIDER_DEPENDENCY_TAR_GZ_FILE_NAME + "_",
           SliderKeys.SLIDER_DEPENDENCY_TAR_GZ_FILE_EXT);
-      // copy all jars except slider-core-<version>.jar
-      tarGzipFolder(srcFolder, tempLibTarGzipFile, createJarFilter());
+      // copy all jars
+      tarGzipFolder(libDirs, tempLibTarGzipFile, createJarFilter());
 
       log.info("Uploading dependency for AM (version {}) from {} to {}",
           version, tempLibTarGzipFile.toURI(), dependencyLibTarGzip.toUri());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00c38659/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
index 1484ee3..3d25d33 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
@@ -182,7 +182,7 @@ public interface SliderKeys extends SliderXmlConfKeys {
   /**
    * Slider AM log4j file name : {@value}
    */
-  String LOG4J_SERVER_PROP_FILENAME = "log4j-server.properties";
+  String LOG4J_SERVER_PROP_FILENAME = "slideram-log4j.properties";
 
   /**
    * Standard log4j file name  : {@value}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00c38659/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
index f773982..713cd02 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
@@ -25,6 +25,7 @@ import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream;
 import org.apache.commons.compress.archivers.zip.ZipArchiveEntry;
 import org.apache.commons.compress.archivers.zip.ZipArchiveInputStream;
 import org.apache.commons.io.output.ByteArrayOutputStream;
+import org.apache.commons.lang.ArrayUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -1399,6 +1400,22 @@ public final class SliderUtils {
     }
   }
 
+  public static String getLibDir() {
+    String[] libDirs = getLibDirs();
+    if (libDirs == null || libDirs.length == 0) {
+      return null;
+    }
+    return libDirs[0];
+  }
+
+  public static String[] getLibDirs() {
+    String libDirStr = System.getProperty(SliderKeys.PROPERTY_LIB_DIR);
+    if (isUnset(libDirStr)) {
+      return ArrayUtils.EMPTY_STRING_ARRAY;
+    }
+    return StringUtils.split(libDirStr, ',');
+  }
+
   /**
    * Submit a JAR containing a specific class and map it
    * @param providerResources provider map to build up
@@ -1962,31 +1979,34 @@ public final class SliderUtils {
   /**
    * Given a source folder create a tar.gz file
    * 
-   * @param srcFolder
+   * @param libDirs
    * @param tarGzipFile
    * 
    * @throws IOException
    */
-  public static void tarGzipFolder(File srcFolder, File tarGzipFile,
+  public static void tarGzipFolder(String[] libDirs, File tarGzipFile,
       FilenameFilter filter) throws IOException {
-    log.info("Tar-gzipping folder {} to {}", srcFolder.getAbsolutePath(),
+    log.info("Tar-gzipping folders {} to {}", libDirs,
         tarGzipFile.getAbsolutePath());
-    List<String> files = new ArrayList<>();
-    generateFileList(files, srcFolder, srcFolder, true, filter);
 
     try(TarArchiveOutputStream taos =
             new TarArchiveOutputStream(new GZIPOutputStream(
         new BufferedOutputStream(new FileOutputStream(tarGzipFile))))) {
-      for (String file : files) {
-        File srcFile = new File(srcFolder, file);
-        TarArchiveEntry tarEntry = new TarArchiveEntry(
-            srcFile, file);
-        taos.putArchiveEntry(tarEntry);
-        try(FileInputStream in = new FileInputStream(srcFile)) {
-          org.apache.commons.io.IOUtils.copy(in, taos);
+      for (String libDir : libDirs) {
+        File srcFolder = new File(libDir);
+        List<String> files = new ArrayList<>();
+        generateFileList(files, srcFolder, srcFolder, true, filter);
+        for (String file : files) {
+          File srcFile = new File(srcFolder, file);
+          TarArchiveEntry tarEntry = new TarArchiveEntry(
+              srcFile, file);
+          taos.putArchiveEntry(tarEntry);
+          try(FileInputStream in = new FileInputStream(srcFile)) {
+            org.apache.commons.io.IOUtils.copy(in, taos);
+          }
+          taos.flush();
+          taos.closeArchiveEntry();
         }
-        taos.flush();
-        taos.closeArchiveEntry();
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00c38659/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
index fdc5be1..7ca469f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
@@ -562,8 +562,7 @@ public class AgentClientProvider extends AbstractClientProvider
   }
 
   private void expandAgentTar(File agentPkgDir) throws IOException {
-    String libDirProp =
-        System.getProperty(PROPERTY_LIB_DIR);
+    String libDirProp = SliderUtils.getLibDir();
     File tarFile = new File(libDirProp, AGENT_TAR);
     expandTar(tarFile, agentPkgDir);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00c38659/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMClientProvider.java
index e5430f2..b58d3aa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMClientProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMClientProvider.java
@@ -184,19 +184,20 @@ public class SliderAMClientProvider extends AbstractClientProvider
         libdir,
         miniClusterTestRun);
 
-    String libDirProp =
-        System.getProperty(SliderKeys.PROPERTY_LIB_DIR);
     log.info("Loading all dependencies for AM.");
     // If slider.tar.gz is available in hdfs use it, else upload all jars
     Path dependencyLibTarGzip = fileSystem.getDependencyTarGzip();
     if (fileSystem.isFile(dependencyLibTarGzip)) {
       SliderUtils.putAmTarGzipAndUpdate(providerResources, fileSystem);
     } else {
-      ProviderUtils.addAllDependencyJars(providerResources,
-                                         fileSystem,
-                                         tempPath,
-                                         libdir,
-                                         libDirProp);
+      for (String libDirProp : SliderUtils.getLibDirs()) {
+        ProviderUtils.addAllDependencyJars(providerResources,
+                                           fileSystem,
+                                           tempPath,
+                                           libdir,
+                                           libDirProp);
+
+      }
     }
     addKeytabResourceIfNecessary(fileSystem,
                                  instanceDescription,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00c38659/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/registry/docstore/TestPublishedConfigurationOutputter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/registry/docstore/TestPublishedConfigurationOutputter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/registry/docstore/TestPublishedConfigurationOutputter.java
index 3706182..56b84e0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/registry/docstore/TestPublishedConfigurationOutputter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/registry/docstore/TestPublishedConfigurationOutputter.java
@@ -82,16 +82,14 @@ public class TestPublishedConfigurationOutputter {
 
     String output = configurationOutputter.asString().replaceAll("( |\\r|\\n)",
         "");
-    assert output.contains(
-        "<configuration><property><name>key1</name><value>val1</value><source/></property></configuration>");
+    assert output.contains("<name>key1</name><value>val1</value>");
 
     File file = tmpDir.newFile();
     configurationOutputter.save(file);
 
     assert FileUtils.readFileToString(file, Charsets.UTF_8)
         .replaceAll("( |\\r|\\n)", "")
-        .contains(
-            "<configuration><property><name>key1</name><value>val1</value><source/></property></configuration>");
+        .contains("<name>key1</name><value>val1</value>");
   }
 
   @Test
@@ -103,14 +101,14 @@ public class TestPublishedConfigurationOutputter {
 
     String output = configurationOutputter.asString().replaceAll("( |\\r|\\n)",
         "");
-    assert output.contains("<configuration><property><name>key1</name><value>val1</value><source/></property></configuration>");
+    assert output.contains("<name>key1</name><value>val1</value>");
 
     File file = tmpDir.newFile();
     configurationOutputter.save(file);
 
     assert FileUtils.readFileToString(file, Charsets.UTF_8)
         .replaceAll("( |\\r|\\n)", "")
-        .contains( "<configuration><property><name>key1</name><value>val1</value><source/></property></configuration>");
+        .contains("<name>key1</name><value>val1</value>");
   }
 
   @Test


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[50/76] [abbrv] hadoop git commit: YARN-5689. Update native services REST API to use agentless docker provider. Contributed by Billie Rinaldi & Gour Saha

Posted by ji...@apache.org.
YARN-5689. Update native services REST API to use agentless docker provider. Contributed by Billie Rinaldi & Gour Saha


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

Branch: refs/heads/yarn-native-services
Commit: 5b7289317c388359d379e5b346dcfe86096531f7
Parents: fba18cb
Author: Jian He <ji...@apache.org>
Authored: Thu Oct 13 11:34:58 2016 -0700
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 .../api/impl/ApplicationApiService.java         | 251 ++++++-------------
 .../yarn/services/utils/RestApiConstants.java   |   3 -
 .../api/impl/TestApplicationApiService.java     |   6 +-
 3 files changed, 79 insertions(+), 181 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5b728931/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
index 9645696..0a62629 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
@@ -50,7 +50,6 @@ import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response.Status;
 
-import org.apache.commons.collections.MapUtils;
 import org.apache.commons.lang.SerializationUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -68,10 +67,12 @@ import org.apache.hadoop.yarn.services.resource.Configuration;
 import org.apache.hadoop.yarn.services.resource.Container;
 import org.apache.hadoop.yarn.services.resource.ContainerState;
 import org.apache.hadoop.yarn.services.resource.Resource;
+import org.apache.slider.api.OptionKeys;
 import org.apache.slider.api.ResourceKeys;
 import org.apache.slider.api.StateValues;
 import org.apache.slider.client.SliderClient;
 import org.apache.slider.common.SliderExitCodes;
+import org.apache.slider.common.SliderKeys;
 import org.apache.slider.common.params.ActionCreateArgs;
 import org.apache.slider.common.params.ActionFlexArgs;
 import org.apache.slider.common.params.ActionFreezeArgs;
@@ -88,12 +89,11 @@ import org.apache.slider.core.exceptions.NotFoundException;
 import org.apache.slider.core.exceptions.SliderException;
 import org.apache.slider.core.exceptions.UnknownApplicationInstanceException;
 import org.apache.slider.core.registry.docstore.ConfigFormat;
+import org.apache.slider.providers.docker.DockerKeys;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.gson.GsonBuilder;
-import com.google.gson.JsonArray;
 import com.google.gson.JsonElement;
 import com.google.gson.JsonNull;
 import com.google.gson.JsonObject;
@@ -211,7 +211,8 @@ public class ApplicationApiService implements ApplicationApi {
           application.setConfiguration(new Configuration());
         }
         addPropertyToConfiguration(application.getConfiguration(),
-            PROPERTY_COMPONENT_TYPE, COMPONENT_TYPE_EXTERNAL);
+            SliderKeys.COMPONENT_TYPE_KEY,
+            SliderKeys.COMPONENT_TYPE_EXTERNAL_APP);
       }
       // resource
       validateApplicationResource(application.getResource(), null, application
@@ -249,7 +250,8 @@ public class ApplicationApiService implements ApplicationApi {
             comp.setConfiguration(new Configuration());
           }
           addPropertyToConfiguration(comp.getConfiguration(),
-              PROPERTY_COMPONENT_TYPE, COMPONENT_TYPE_EXTERNAL);
+              SliderKeys.COMPONENT_TYPE_KEY,
+              SliderKeys.COMPONENT_TYPE_EXTERNAL_APP);
           compNameArtifactIdMap.put(comp.getName(), comp.getArtifact().getId());
           comp.setName(comp.getArtifact().getId());
         }
@@ -339,9 +341,9 @@ public class ApplicationApiService implements ApplicationApi {
     final ActionCreateArgs createArgs = new ActionCreateArgs();
     addAppConfOptions(createArgs, application, compNameArtifactIdMap);
     addResourceOptions(createArgs, application);
-    String metainfoJson = getMetainfoJson(application, compNameArtifactIdMap);
 
-    createArgs.appMetaInfoJson = metainfoJson;
+    createArgs.provider = DockerKeys.PROVIDER_DOCKER;
+
     if (queueName != null && queueName.trim().length() > 0) {
       createArgs.queue = queueName.trim();
     }
@@ -388,17 +390,76 @@ public class ApplicationApiService implements ApplicationApi {
     //    }
 
     Set<String> uniqueGlobalPropertyCache = new HashSet<>();
-    if (application.getConfiguration() != null
-        && application.getConfiguration().getProperties() != null) {
-      for (Map.Entry<String, String> propEntry : application.getConfiguration()
-          .getProperties().entrySet()) {
+    if (application.getConfiguration() != null) {
+      if (application.getConfiguration().getProperties() != null) {
+        for (Map.Entry<String, String> propEntry : application
+            .getConfiguration().getProperties().entrySet()) {
+          addOptionsIfNotPresent(appOptions, uniqueGlobalPropertyCache,
+              propEntry.getKey(), propEntry.getValue());
+        }
+      }
+      List<ConfigFile> configFiles = application.getConfiguration().getFiles();
+      if (configFiles != null && !configFiles.isEmpty()) {
         addOptionsIfNotPresent(appOptions, uniqueGlobalPropertyCache,
-            propEntry.getKey(), propEntry.getValue());
+            SliderKeys.AM_CONFIG_GENERATION, "true");
+        for (ConfigFile configFile : configFiles) {
+          addOptionsIfNotPresent(appOptions, uniqueGlobalPropertyCache,
+              OptionKeys.CONF_FILE_PREFIX + configFile.getSrcFile() +
+                  OptionKeys.NAME_SUFFIX, configFile.getDestFile());
+          addOptionsIfNotPresent(appOptions, uniqueGlobalPropertyCache,
+              OptionKeys.CONF_FILE_PREFIX + configFile.getSrcFile() +
+                  OptionKeys.TYPE_SUFFIX, configFile.getType().toString());
+        }
       }
     }
     if (application.getComponents() != null) {
+
+      Map<String, String> appQuicklinks = application.getQuicklinks();
+      if (appQuicklinks != null) {
+        for (Map.Entry<String, String> quicklink : appQuicklinks.entrySet()) {
+          addOptionsIfNotPresent(appOptions, uniqueGlobalPropertyCache,
+              OptionKeys.EXPORT_PREFIX + quicklink.getKey(),
+              quicklink.getValue());
+        }
+      }
+
       Map<String, String> placeholders = new HashMap<>();
+      placeholders.put(PLACEHOLDER_APP_NAME, application.getName());
       for (Component comp : application.getComponents()) {
+        placeholders.put(PLACEHOLDER_APP_COMPONENT_NAME, comp.getName());
+        if (comp.getArtifact().getType() == Artifact.TypeEnum.DOCKER) {
+          appCompOptionTriples.addAll(Arrays.asList(comp.getName(),
+              DockerKeys.DOCKER_IMAGE, comp.getArtifact().getId() == null ?
+              application.getArtifact().getId() : comp.getArtifact().getId()));
+          appCompOptionTriples.addAll(Arrays.asList(comp.getName(),
+              DockerKeys.DOCKER_START_COMMAND, comp.getLaunchCommand() == null ?
+              replacePlaceholders(application.getLaunchCommand(), placeholders)
+              : replacePlaceholders(comp.getLaunchCommand(), placeholders)));
+          appCompOptionTriples.addAll(Arrays.asList(comp.getName(),
+              DockerKeys.DOCKER_NETWORK, DockerKeys.DEFAULT_DOCKER_NETWORK));
+          if (comp.getRunPrivilegedContainer() != null) {
+            appCompOptionTriples.addAll(Arrays.asList(comp.getName(),
+                DockerKeys.DOCKER_USE_PRIVILEGED,
+                comp.getRunPrivilegedContainer().toString()));
+          }
+        }
+
+        if (comp.getConfiguration() != null) {
+          List<ConfigFile> configFiles = comp.getConfiguration().getFiles();
+          if (configFiles != null && !configFiles.isEmpty()) {
+            appCompOptionTriples.addAll(Arrays.asList(comp.getName(),
+                SliderKeys.AM_CONFIG_GENERATION, "true"));
+            for (ConfigFile configFile : configFiles) {
+              appCompOptionTriples.addAll(Arrays.asList(comp.getName(),
+                  OptionKeys.CONF_FILE_PREFIX + configFile.getSrcFile() +
+                      OptionKeys.NAME_SUFFIX, configFile.getDestFile()));
+              appCompOptionTriples.addAll(Arrays.asList(comp.getName(),
+                  OptionKeys.CONF_FILE_PREFIX + configFile.getSrcFile() +
+                  OptionKeys.TYPE_SUFFIX, configFile.getType().toString()));
+            }
+          }
+        }
+
         if (Boolean.TRUE.equals(comp.getUniqueComponentSupport())) {
           for (int i = 1; i <= comp.getNumberOfContainers(); i++) {
             placeholders.put(PLACEHOLDER_COMPONENT_ID, Integer.toString(i));
@@ -526,8 +587,8 @@ public class ApplicationApiService implements ApplicationApi {
     if (application.getComponents() != null) {
       int priority = 1;
       for (Component comp : application.getComponents()) {
-        if (hasPropertyWithValue(comp, PROPERTY_COMPONENT_TYPE,
-            COMPONENT_TYPE_EXTERNAL)) {
+        if (hasPropertyWithValue(comp, SliderKeys.COMPONENT_TYPE_KEY,
+            SliderKeys.COMPONENT_TYPE_EXTERNAL_APP)) {
           continue;
         }
         if (Boolean.TRUE.equals(comp.getUniqueComponentSupport())) {
@@ -595,168 +656,6 @@ public class ApplicationApiService implements ApplicationApi {
     return resCompOptTriples;
   }
 
-  private String getMetainfoJson(Application application,
-      Map<String, String> compNameArtifactIdMap) throws SliderException,
-      IOException {
-    JsonObject rootObj = new JsonObject();
-    rootObj.addProperty("schemaVersion", METAINFO_SCHEMA_VERSION);
-    JsonObject applicationObj = new JsonObject();
-    rootObj.add("application", applicationObj);
-    applicationObj.addProperty("name", application.getName().toUpperCase());
-    JsonArray componentsArray = new JsonArray();
-    applicationObj.add("components", componentsArray);
-    JsonArray commandOrdersArray = new JsonArray();
-    applicationObj.add("commandOrders", commandOrdersArray);
-
-    JsonArray exportGroupsArray = new JsonArray();
-    applicationObj.add("exportGroups", exportGroupsArray);
-    // Use only one export group
-    JsonObject exportGroup = new JsonObject();
-    exportGroup.addProperty("name", EXPORT_GROUP_NAME);
-    exportGroupsArray.add(exportGroup);
-    JsonArray exportsArray = new JsonArray();
-    exportGroup.add("exports", exportsArray);
-
-    if (application.getComponents() != null) {
-
-      // Set exports at application level
-      Map<String, String> appQuicklinks = application.getQuicklinks();
-      Map<String, String> placeholders = new HashMap<>();
-      placeholders.put(PLACEHOLDER_APP_NAME, application.getName());
-      if (appQuicklinks != null) {
-        for (Map.Entry<String, String> quicklink : appQuicklinks.entrySet()) {
-          JsonObject export = new JsonObject();
-          export.addProperty("name", quicklink.getKey());
-          export.addProperty("value",
-              replacePlaceholders(quicklink.getValue(), placeholders));
-          exportsArray.add(export);
-        }
-      }
-
-      for (Component comp : application.getComponents()) {
-        JsonObject compObject = null;
-        if (!hasPropertyWithValue(comp, PROPERTY_COMPONENT_TYPE,
-            COMPONENT_TYPE_EXTERNAL)) {
-          if (Boolean.TRUE.equals(comp.getUniqueComponentSupport())) {
-            for (int i = 1; i <= comp.getNumberOfContainers(); i++) {
-              // we also need the capability to specify ports and mount points
-              // sometime
-              compObject = createMetainfoComponent(comp, application,
-                  comp.getName() + i);
-              componentsArray.add(compObject);
-            }
-          } else {
-            compObject = createMetainfoComponent(comp, application,
-                comp.getName());
-            componentsArray.add(compObject);
-          }
-        }
-
-        // Translate dependencies into command orders
-        List<String> dependencies = comp.getDependencies();
-        if (dependencies != null && !dependencies.isEmpty()) {
-          JsonObject commandOrder = new JsonObject();
-          commandOrder.addProperty("command", comp.getName()
-              + COMMAND_ORDER_SUFFIX_START);
-          for (String dependency : dependencies) {
-            // If APPLICATION type artifact then map component name dependencies
-            // to artifact id
-            if (comp.getArtifact().getType() == Artifact.TypeEnum.APPLICATION) {
-              dependency = compNameArtifactIdMap.get(dependency);
-            }
-            commandOrder.addProperty("requires", dependency
-                + COMMAND_ORDER_SUFFIX_STARTED);
-          }
-          commandOrdersArray.add(commandOrder);
-        }
-
-        // Quicklinks need to be added as appExports and componentExports at the
-        // component level
-        List<String> compQuicklinks = comp.getQuicklinks();
-        if (compQuicklinks != null && !compQuicklinks.isEmpty()) {
-          if (MapUtils.isEmpty(appQuicklinks)) {
-            throw new SliderException(ERROR_QUICKLINKS_FOR_COMP_INVALID);
-          }
-          List<String> appExports = new ArrayList<>();
-          JsonArray compExportsArray = new JsonArray();
-          compObject.add("componentExports", compExportsArray);
-
-          for (String quicklink : compQuicklinks) {
-            appExports.add(EXPORT_GROUP_NAME + "-" + quicklink);
-
-            JsonObject compExport = new JsonObject();
-            compExport.addProperty("name", quicklink);
-            compExport.addProperty("value", appQuicklinks.get(quicklink));
-            compExportsArray.add(compExport);
-          }
-          compObject.addProperty("appExports",
-              StringUtils.join(appExports, ","));
-          // specify that there are published configs for this component
-          compObject.addProperty("publishConfig", "true");
-        }
-      }
-    }
-
-    String jsonString = new GsonBuilder().setPrettyPrinting().create()
-        .toJson(rootObj);
-    logger.info("Metainfo = \n{}", jsonString);
-    return jsonString;
-  }
-
-  private JsonObject createMetainfoComponent(Component comp,
-      Application application, String compName) {
-    JsonObject compObj = new JsonObject();
-    compObj.addProperty("name", compName);
-    // below is diff for each type
-    if (comp.getArtifact() != null && comp.getArtifact().getType() != null
-        && comp.getArtifact().getType() == Artifact.TypeEnum.DOCKER) {
-      compObj.addProperty("type", COMPONENT_TYPE_YARN_DOCKER);
-      JsonArray dockerContainerArray = new JsonArray();
-      compObj.add("dockerContainers", dockerContainerArray);
-      JsonObject dockerContainerObj = new JsonObject();
-      dockerContainerArray.add(dockerContainerObj);
-      dockerContainerObj.addProperty("name", compName.toLowerCase());
-      // if image not specified, then use global value
-      dockerContainerObj.addProperty("image",
-          comp.getArtifact().getId() == null ? application.getArtifact()
-              .getId() : comp.getArtifact().getId());
-      // If launch command not specified, then use global value. Resolve all
-      // placeholders.
-      Map<String, String> placeholders = new HashMap<>();
-      placeholders.put(PLACEHOLDER_APP_NAME, application.getName());
-      placeholders.put(PLACEHOLDER_APP_COMPONENT_NAME, compName);
-      dockerContainerObj.addProperty(
-          "startCommand",
-          comp.getLaunchCommand() == null ? replacePlaceholders(
-              application.getLaunchCommand(), placeholders)
-              : replacePlaceholders(comp.getLaunchCommand(), placeholders));
-      dockerContainerObj.addProperty("network", DEFAULT_NETWORK);
-      dockerContainerObj.addProperty("commandPath", DEFAULT_COMMAND_PATH);
-      // TODO: What to do with privContainer ?
-      dockerContainerObj.addProperty("runPrivilegedContainer",
-          comp.getRunPrivilegedContainer());
-      if (comp.getConfiguration() != null) {
-        List<ConfigFile> configFiles = comp.getConfiguration().getFiles();
-        if (configFiles != null && !configFiles.isEmpty()) {
-          JsonArray configFileArray = new JsonArray();
-          for (ConfigFile configFile : configFiles) {
-            JsonObject configFileObj = new JsonObject();
-            configFileObj.addProperty("type", configFile.getType().toString());
-            configFileObj.addProperty("fileName", configFile.getDestFile());
-            // TODO: add all properties which should include dictionaryName
-            configFileObj.addProperty("dictionaryName",
-                configFile.getSrcFile());
-            configFileArray.add(configFileObj);
-          }
-          dockerContainerObj.add("configFiles", configFileArray);
-        }
-      }
-      // we also need to specify artifact_management_service sometime
-    }
-    // we also need the capability to specify ports and mount points sometime
-    return compObj;
-  }
-
   private static UserGroupInformation getSliderUser() {
     if (SLIDER_USER != null) {
       return SLIDER_USER;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5b728931/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiConstants.java
index 4c16546..23b7ad4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiConstants.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiConstants.java
@@ -49,11 +49,8 @@ public interface RestApiConstants {
   Integer GET_APPLICATIONS_THREAD_POOL_SIZE = 200;
 
   String PROPERTY_PYTHON_PATH = "python.path";
-  String PROPERTY_COMPONENT_TYPE = "site.global.component_type";
   String PROPERTY_DNS_DEPENDENCY = "site.global.dns.dependency";
 
-  String COMPONENT_TYPE_EXTERNAL = "external";
-
   String COMMAND_ORDER_SUFFIX_START = "-START";
   String COMMAND_ORDER_SUFFIX_STARTED = "-STARTED";
   String EXPORT_GROUP_NAME = "QuickLinks";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5b728931/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/services/api/impl/TestApplicationApiService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/services/api/impl/TestApplicationApiService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/services/api/impl/TestApplicationApiService.java
index a03ab69..fdf1419 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/services/api/impl/TestApplicationApiService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/services/api/impl/TestApplicationApiService.java
@@ -26,6 +26,7 @@ import java.util.Map;
 import org.apache.hadoop.yarn.services.resource.Application;
 import org.apache.hadoop.yarn.services.resource.Artifact;
 import org.apache.hadoop.yarn.services.resource.Resource;
+import org.apache.slider.common.SliderKeys;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -124,8 +125,9 @@ public class TestApplicationApiService {
         DEFAULT_COMPONENT_NAME);
     Assert.assertEquals(app.getLifetime(), DEFAULT_UNLIMITED_LIFETIME);
     Assert.assertEquals("Property not set",
-        app.getConfiguration().getProperties().get(PROPERTY_COMPONENT_TYPE),
-        COMPONENT_TYPE_EXTERNAL);
+        app.getConfiguration().getProperties()
+            .get(SliderKeys.COMPONENT_TYPE_KEY),
+        SliderKeys.COMPONENT_TYPE_EXTERNAL_APP);
 
     // unset artifact type, default component and no of containers to test other
     // validation logic


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[22/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/YarnAppListClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/YarnAppListClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/YarnAppListClient.java
new file mode 100644
index 0000000..1bdfb9c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/YarnAppListClient.java
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.registry;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.slider.client.SliderYarnClientImpl;
+import org.apache.slider.api.types.SliderInstanceDescription;
+import org.apache.slider.common.tools.CoreFileSystem;
+import org.apache.slider.common.tools.SliderUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Client code for interacting with a list of service instances.
+ * The initial logic just enumerates service instances in the YARN RM
+ */
+public class YarnAppListClient {
+
+  private final SliderYarnClientImpl yarnClient;
+  private final String username;
+  private final Configuration conf;
+  private static final Logger log =
+      LoggerFactory.getLogger(YarnAppListClient.class);
+
+  public YarnAppListClient(SliderYarnClientImpl yarnClient,
+      String username,
+      Configuration conf) {
+
+    Preconditions.checkArgument(yarnClient != null,
+        "yarn client is null: is app inited?");
+    Preconditions.checkArgument(username != null,
+        "username is null");
+    Preconditions.checkArgument(conf != null,
+        "conf parameter is null");
+    this.yarnClient = yarnClient;
+    this.username = username;
+    this.conf = conf;
+  }
+
+  /**
+   * find all live instances of a specific app -if there is more than one 
+   * in the cluster, this returns them all. State should be running or earlier
+   * in the lifecycle
+   * @param appname application name
+   * @return the list of all matching application instances
+   */
+  public List<ApplicationReport> findAllLiveInstances(String appname)
+    throws YarnException, IOException {
+    return yarnClient.findAllLiveInstances(username, appname);
+  }
+
+
+  /**
+   * Find an instance of a application belong to the current user
+   * @param appname application name
+   * @return the app report or null if none is found
+   * @throws YarnException YARN issues
+   * @throws IOException IO problems
+   */
+  public ApplicationReport findInstance(String appname) throws
+                                                        YarnException,
+                                                        IOException {
+    List<ApplicationReport> instances = listInstances(null);
+    return yarnClient.findClusterInInstanceList(instances, appname);
+  }
+
+  /**
+   * List instances belonging to the specific user
+   * @return a possibly empty list of AMs
+   */
+  public List<ApplicationReport> listInstances()
+      throws YarnException, IOException {
+    return listInstances(null);
+  }
+
+  /**
+   * List instances belonging to a specific user
+   * @return a possibly empty list of AMs
+   * @param user user if not the default. null means default, "" means all users, 
+   * otherwise it is the name of a user
+   */
+  public List<ApplicationReport> listInstances(String user)
+      throws YarnException, IOException {
+    String listUser = user == null ? username : user;
+    return yarnClient.listDeployedInstances(listUser);
+  }
+
+  /**
+   * Enumerate slider instances for the current user, and the
+   * most recent app report, where available.
+   * @param listOnlyInState boolean to indicate that the instances should
+   * only include those in a YARN state
+   * <code> minAppState &lt;= currentState &lt;= maxAppState </code>
+   * 
+   * @param minAppState minimum application state to include in enumeration.
+   * @param maxAppState maximum application state to include
+   * @return a map of application instance name to description
+   * @throws IOException Any IO problem
+   * @throws YarnException YARN problems
+   */
+  public Map<String, SliderInstanceDescription> enumSliderInstances(
+      boolean listOnlyInState,
+      YarnApplicationState minAppState,
+      YarnApplicationState maxAppState)
+      throws IOException, YarnException {
+
+    CoreFileSystem sliderFileSystem = new CoreFileSystem(conf);
+    Preconditions.checkArgument(!listOnlyInState || minAppState != null,
+        "null minAppState when listOnlyInState set");
+    Preconditions.checkArgument(!listOnlyInState || maxAppState != null,
+        "null maxAppState when listOnlyInState set");
+    if (!listOnlyInState) {
+      // if there's not filtering, ask for the entire range of states
+      minAppState = YarnApplicationState.NEW;
+      maxAppState = YarnApplicationState.KILLED;
+    }
+    // get the complete list of persistent instances
+    Map<String, Path> persistentInstances =
+        sliderFileSystem.listPersistentInstances();
+    Map<String, SliderInstanceDescription> descriptions =
+        new HashMap<String, SliderInstanceDescription>(persistentInstances.size());
+
+    if (persistentInstances.isEmpty()) {
+      // an empty listing is a success if no cluster was named
+      log.debug("No application instances found");
+      return descriptions;
+    }
+
+    // enum those the RM knows about
+    List<ApplicationReport> rmInstances = listInstances();
+    SliderUtils.sortApplicationsByMostRecent(rmInstances);
+    Map<String, ApplicationReport> reportMap =
+        SliderUtils.buildApplicationReportMap(rmInstances, minAppState,
+            maxAppState);
+    log.debug("Persisted {} deployed {} filtered[{}-{}] & de-duped to {}",
+        persistentInstances.size(),
+        rmInstances.size(),
+        minAppState, maxAppState,
+        reportMap.size());
+
+    // at this point there is a list of all persistent instances, and
+    // a (possibly filtered) list of application reports
+
+    for (Map.Entry<String, Path> entry : persistentInstances.entrySet()) {
+      // loop through the persistent values
+      String name = entry.getKey();
+
+      // look up any report from the (possibly filtered) report set
+      ApplicationReport report = reportMap.get(name);
+      if (!listOnlyInState || report != null) {
+        // if the enum wants to filter in state, only add it if there is
+        // a report in that range. Otherwise: include all values
+        SliderInstanceDescription sid = new SliderInstanceDescription(
+            name, entry.getValue(), report);
+        descriptions.put(name, sid);
+      }
+    }
+
+    return descriptions;
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigFormat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigFormat.java
new file mode 100644
index 0000000..ddab606
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigFormat.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.registry.docstore;
+
+public enum ConfigFormat {
+
+  JSON("json"),
+  PROPERTIES("properties"),
+  XML("xml"),
+  HADOOP_XML("hadoop-xml"),
+  ENV("env"),
+  TEMPLATE("template"),
+  YAML("yaml"),
+  ;
+  ConfigFormat(String suffix) {
+    this.suffix = suffix;
+  }
+
+  private final String suffix;
+
+  public String getSuffix() {
+    return suffix;
+  }
+
+
+  @Override
+  public String toString() {
+    return suffix;
+  }
+
+  /**
+   * Get a matching format or null
+   * @param type
+   * @return the format
+   */
+  public static ConfigFormat resolve(String type) {
+    for (ConfigFormat format: values()) {
+      if (format.getSuffix().equals(type)) {
+        return format;
+      }
+    }
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigUtils.java
new file mode 100644
index 0000000..2e1615b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigUtils.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.core.registry.docstore;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.slider.common.tools.SliderFileSystem;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class ConfigUtils {
+  public static final String TEMPLATE_FILE = "template.file";
+
+  public static String replaceProps(Map<String, String> config, String content) {
+    Map<String, String> tokens = new HashMap<>();
+    for (Entry<String, String> entry : config.entrySet()) {
+      tokens.put("${" + entry.getKey() + "}", entry.getValue());
+      tokens.put("{{" + entry.getKey() + "}}", entry.getValue());
+    }
+    String value = content;
+    for (Map.Entry<String,String> token : tokens.entrySet()) {
+      value = value.replaceAll(Pattern.quote(token.getKey()),
+          Matcher.quoteReplacement(token.getValue()));
+    }
+    return value;
+  }
+
+  public static Map<String, String> replacePropsInConfig(
+      Map<String, String> config, Map<String, String> env) {
+    Map<String, String> tokens = new HashMap<>();
+    for (Entry<String, String> entry : env.entrySet()) {
+      tokens.put("${" + entry.getKey() + "}", entry.getValue());
+    }
+    Map<String, String> newConfig = new HashMap<>();
+    for (Entry<String, String> entry : config.entrySet()) {
+      String value = entry.getValue();
+      for (Map.Entry<String,String> token : tokens.entrySet()) {
+        value = value.replaceAll(Pattern.quote(token.getKey()),
+            Matcher.quoteReplacement(token.getValue()));
+      }
+      newConfig.put(entry.getKey(), entry.getValue());
+    }
+    return newConfig;
+  }
+
+  public static void prepConfigForTemplateOutputter(ConfigFormat configFormat,
+      Map<String, String> config, SliderFileSystem fileSystem,
+      String clusterName, String fileName) throws IOException {
+    if (!configFormat.equals(ConfigFormat.TEMPLATE)) {
+      return;
+    }
+    Path templateFile = null;
+    if (config.containsKey(TEMPLATE_FILE)) {
+      templateFile = fileSystem.buildResourcePath(config.get(TEMPLATE_FILE));
+      if (!fileSystem.isFile(templateFile)) {
+        templateFile = fileSystem.buildResourcePath(clusterName,
+            config.get(TEMPLATE_FILE));
+      }
+      if (!fileSystem.isFile(templateFile)) {
+        throw new IOException("config specified template file " + config
+            .get(TEMPLATE_FILE) + " but " + templateFile + " doesn't exist");
+      }
+    }
+    if (templateFile == null && fileName != null) {
+      templateFile = fileSystem.buildResourcePath(fileName);
+      if (!fileSystem.isFile(templateFile)) {
+        templateFile = fileSystem.buildResourcePath(clusterName,
+            fileName);
+      }
+    }
+    if (fileSystem.isFile(templateFile)) {
+      config.put("content", fileSystem.cat(templateFile));
+    } else {
+      config.put("content", "");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigurationResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigurationResolver.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigurationResolver.java
new file mode 100644
index 0000000..88bac77
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigurationResolver.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.registry.docstore;
+
+public class ConfigurationResolver {
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ExportEntry.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ExportEntry.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ExportEntry.java
new file mode 100644
index 0000000..4bcf6c1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ExportEntry.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.registry.docstore;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+/**
+ * JSON-serializable description of a published key-val configuration.
+ *
+ * The values themselves are not serialized in the external view; they have to be served up by the far end
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class ExportEntry {
+
+  /**
+   * The value of the export
+   */
+  private String value;
+  /**
+   * The container id of the container that is responsible for the export
+   */
+  private String containerId;
+  /**
+   * Tag associated with the container - its usually an identifier different than container id
+   * that allows a soft serial id to all containers of a component - e.g. 1, 2, 3, ...
+   */
+  private String tag;
+  /**
+   * An export can be at the level of a component or an application
+   */
+  private String level;
+  /**
+   * The time when the export was updated
+   */
+  private String updatedTime;
+  /**
+   * The time when the export expires
+   */
+  private String validUntil;
+
+  public ExportEntry() {
+  }
+
+  public String getValue() {
+    return value;
+  }
+
+  public void setValue(String value) {
+    this.value = value;
+  }
+
+  public String getContainerId() {
+    return containerId;
+  }
+
+  public void setContainerId(String containerId) {
+    this.containerId = containerId;
+  }
+
+  public String getTag() {
+    return tag;
+  }
+
+  public void setTag(String tag) {
+    this.tag = tag;
+  }
+
+  public String getLevel() {
+    return level;
+  }
+
+  public void setLevel(String level) {
+    this.level = level;
+  }
+  public String getUpdatedTime() {
+    return updatedTime;
+  }
+
+  public void setUpdatedTime(String updatedTime) {
+    this.updatedTime = updatedTime;
+  }
+
+  public String getValidUntil() {
+    return validUntil;
+  }
+
+  public void setValidUntil(String validUntil) {
+    this.validUntil = validUntil;
+  }
+
+  @Override
+  public String toString() {
+    return new StringBuilder("ExportEntry{").
+        append("value='").append(value).append("',").
+        append("containerId='").append(containerId).append("',").
+        append("tag='").append(tag).append("',").
+        append("level='").append(level).append("'").
+        append("updatedTime='").append(updatedTime).append("'").
+        append("validUntil='").append(validUntil).append("'").
+        append(" }").toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigSet.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigSet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigSet.java
new file mode 100644
index 0000000..edc129e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigSet.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.registry.docstore;
+
+import org.apache.slider.server.appmaster.web.rest.RestPaths;
+import org.apache.slider.server.services.utility.PatternValidator;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.HashMap;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * Represents a set of configurations for an application, component, etc.
+ * Json serialisable; accessors are synchronized
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class PublishedConfigSet {
+
+  private static final PatternValidator validator = new PatternValidator(
+      RestPaths.PUBLISHED_CONFIGURATION_REGEXP);
+  
+  public Map<String, PublishedConfiguration> configurations =
+      new HashMap<>();
+
+  public PublishedConfigSet() {
+  }
+
+  /**
+   * Put a name -it will be converted to lower case before insertion.
+   * Any existing entry will be overwritten (that includes an entry
+   * with a different case in the original name)
+   * @param name name of entry
+   * @param conf configuration
+   * @throws IllegalArgumentException if not a valid name
+   */
+  public void put(String name, PublishedConfiguration conf) {
+    String name1 = name.toLowerCase(Locale.ENGLISH);
+    validateName(name1);
+    configurations.put(name1, conf);
+  }
+
+  /**
+   * Validate the name -restricting it to the set defined in 
+   * {@link RestPaths#PUBLISHED_CONFIGURATION_REGEXP}
+   * @param name name to validate
+   * @throws IllegalArgumentException if not a valid name
+   */
+  public static void validateName(String name) {
+    validator.validate(name);
+    
+  }
+
+  public PublishedConfiguration get(String name) {
+    return configurations.get(name);
+  }
+  
+  public boolean contains(String name) {
+    return configurations.containsKey(name);
+  }
+  
+  public int size() {
+    return configurations.size();
+  }
+  
+  public Set<String> keys() {
+    TreeSet<String> keys = new TreeSet<>();
+    keys.addAll(configurations.keySet());
+    return keys;
+  }
+
+  public PublishedConfigSet shallowCopy() {
+    PublishedConfigSet that = new PublishedConfigSet();
+    for (Map.Entry<String, PublishedConfiguration> entry :
+        configurations.entrySet()) {
+      that.put(entry.getKey(), entry.getValue().shallowCopy());
+    }
+    return that;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfiguration.java
new file mode 100644
index 0000000..50b522f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfiguration.java
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.registry.docstore;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.slider.common.tools.ConfigHelper;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.SerializationConfig;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.io.IOException;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+/**
+ * JSON-serializable description of a published key-val configuration.
+ * 
+ * The values themselves are not serialized in the external view; they have
+ * to be served up by the far end
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class PublishedConfiguration {
+
+  public String description;
+  public long updated;
+  
+  public String updatedTime;
+
+  public Map<String, String> entries = new HashMap<>();
+
+  public PublishedConfiguration() {
+  }
+
+  /**
+   * build an empty published configuration 
+   * @param description configuration description
+   */
+  public PublishedConfiguration(String description) {
+    this.description = description;
+  }
+
+  /**
+   * Build a configuration from the entries
+   * @param description configuration description
+   * @param entries entries to put
+   */
+  public PublishedConfiguration(String description,
+      Iterable<Map.Entry<String, String>> entries) {
+    this.description = description;
+    putValues(entries);
+  }
+
+  /**
+   * Build a published configuration, using the keys from keysource,
+   * but resolving the values from the value source, via Configuration.get()
+   * @param description configuration description
+   * @param keysource source of keys
+   * @param valuesource source of values
+   */
+  public PublishedConfiguration(String description,
+      Iterable<Map.Entry<String, String>> keysource,
+      Configuration valuesource) {
+    this.description = description;
+    putValues(ConfigHelper.resolveConfiguration(keysource, valuesource));
+  }
+
+  
+  /**
+   * Is the configuration empty. This means either that it has not
+   * been given any values, or it is stripped down copy set down over the
+   * wire.
+   * @return true if it is empty
+   */
+  public boolean isEmpty() {
+    return entries.isEmpty();
+  }
+
+
+  public void setUpdated(long updated) {
+    this.updated = updated;
+    this.updatedTime = new Date(updated).toString();
+  }
+
+  public long getUpdated() {
+    return updated;
+  }
+
+  /**
+   * Set the values from an iterable (this includes a Hadoop Configuration
+   * and Java properties object).
+   * Any existing value set is discarded
+   * @param entries entries to put
+   */
+  public void putValues(Iterable<Map.Entry<String, String>> entries) {
+    this.entries = new HashMap<String, String>();
+    for (Map.Entry<String, String> entry : entries) {
+      this.entries.put(entry.getKey(), entry.getValue());
+    }
+    
+  }
+
+  /**
+   * Convert to Hadoop XML
+   * @return the configuration as a Hadoop Configuratin
+   */
+  public Configuration asConfiguration() {
+    Configuration conf = new Configuration(false);
+    try {
+      ConfigHelper.addConfigMap(conf, entries, "");
+    } catch (BadConfigException e) {
+      // triggered on a null value; switch to a runtime (and discard the stack)
+      throw new RuntimeException(e.toString());
+    }
+    return conf;
+  }
+  
+  public String asConfigurationXML() throws IOException {
+    return ConfigHelper.toXml(asConfiguration());
+  }
+
+  /**
+   * Convert values to properties
+   * @return a property file
+   */
+  public Properties asProperties() {
+    Properties props = new Properties();
+    props.putAll(entries);
+    return props;
+  }
+
+  /**
+   * Return the values as json string
+   * @return the JSON representation
+   * @throws IOException marshalling failure
+   */
+  public String asJson() throws IOException {
+    ObjectMapper mapper = new ObjectMapper();
+    mapper.configure(SerializationConfig.Feature.INDENT_OUTPUT, true);
+    String json = mapper.writeValueAsString(entries);
+    return json;
+  }
+
+
+  /**
+   * This makes a copy without the nested content -so is suitable
+   * for returning as part of the list of a parent's values
+   * @return the copy
+   */
+  public PublishedConfiguration shallowCopy() {
+    PublishedConfiguration that = new PublishedConfiguration();
+    that.description = this.description;
+    that.updated = this.updated;
+    that.updatedTime = this.updatedTime;
+    return that;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder("PublishedConfiguration{");
+    sb.append("description='").append(description).append('\'');
+    sb.append(" entries = ").append(entries.size());
+    sb.append('}');
+    return sb.toString();
+  }
+
+  /**
+   * Create an outputter for a given format
+   * @param format format to use
+   * @return an instance of output
+   */
+  public PublishedConfigurationOutputter createOutputter(ConfigFormat format) {
+    return PublishedConfigurationOutputter.createOutputter(format, this);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigurationOutputter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigurationOutputter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigurationOutputter.java
new file mode 100644
index 0000000..9bdcfcb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigurationOutputter.java
@@ -0,0 +1,210 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.registry.docstore;
+
+import com.google.common.base.Charsets;
+import com.google.common.base.Preconditions;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.slider.common.tools.ConfigHelper;
+import org.yaml.snakeyaml.DumperOptions;
+import org.yaml.snakeyaml.DumperOptions.FlowStyle;
+import org.yaml.snakeyaml.Yaml;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.StringWriter;
+import java.util.Properties;
+
+/**
+ * Output a published configuration
+ */
+public abstract class PublishedConfigurationOutputter {
+
+  protected final PublishedConfiguration owner;
+
+  protected PublishedConfigurationOutputter(PublishedConfiguration owner) {
+    this.owner = owner;
+  }
+
+  /**
+   * Save the config to a destination file, in the format of this outputter
+   * @param dest destination file
+   * @throws IOException
+   */
+/* JDK7
+  public void save(File dest) throws IOException {
+    try(FileOutputStream out = new FileOutputStream(dest)) {
+      save(out);
+      out.close();
+    }
+  }
+*/
+  public void save(File dest) throws IOException {
+    FileUtils.writeStringToFile(dest, asString(), Charsets.UTF_8);
+  }
+
+  /**
+   * Save the content. The default saves the asString() value
+   * to the output stream
+   * @param out output stream
+   * @throws IOException
+   */
+  public void save(OutputStream out) throws IOException {
+    IOUtils.write(asString(), out, Charsets.UTF_8);
+  }
+  /**
+   * Convert to a string
+   * @return the string form
+   * @throws IOException
+   */
+  public abstract String asString() throws IOException;
+
+  /**
+   * Create an outputter for the chosen format
+   * @param format format enumeration
+   * @param owner owning config
+   * @return the outputter
+   */
+
+  public static PublishedConfigurationOutputter createOutputter(ConfigFormat format,
+      PublishedConfiguration owner) {
+    Preconditions.checkNotNull(owner);
+    switch (format) {
+      case XML:
+      case HADOOP_XML:
+        return new XmlOutputter(owner);
+      case PROPERTIES:
+        return new PropertiesOutputter(owner);
+      case JSON:
+        return new JsonOutputter(owner);
+      case ENV:
+        return new EnvOutputter(owner);
+      case TEMPLATE:
+        return new TemplateOutputter(owner);
+      case YAML:
+        return new YamlOutputter(owner);
+      default:
+        throw new RuntimeException("Unsupported format :" + format);
+    }
+  }
+
+  public static class XmlOutputter extends PublishedConfigurationOutputter {
+
+
+    private final Configuration configuration;
+
+    public XmlOutputter(PublishedConfiguration owner) {
+      super(owner);
+      configuration = owner.asConfiguration();
+    }
+
+    @Override
+    public void save(OutputStream out) throws IOException {
+      configuration.writeXml(out);
+    }
+
+    @Override
+    public String asString() throws IOException {
+      return ConfigHelper.toXml(configuration);
+    }
+
+    public Configuration getConfiguration() {
+      return configuration;
+    }
+  }
+
+  public static class PropertiesOutputter extends PublishedConfigurationOutputter {
+
+    private final Properties properties;
+
+    public PropertiesOutputter(PublishedConfiguration owner) {
+      super(owner);
+      properties = owner.asProperties();
+    }
+
+    @Override
+    public void save(OutputStream out) throws IOException {
+      properties.store(out, "");
+    }
+
+
+    public String asString() throws IOException {
+      StringWriter sw = new StringWriter();
+      properties.store(sw, "");
+      return sw.toString();
+    }
+  }
+
+
+  public static class JsonOutputter extends PublishedConfigurationOutputter {
+
+    public JsonOutputter(PublishedConfiguration owner) {
+      super(owner);
+    }
+
+    @Override
+    public String asString() throws IOException {
+      return owner.asJson();
+    }
+  }
+
+
+  public static class EnvOutputter extends PublishedConfigurationOutputter {
+
+    public EnvOutputter(PublishedConfiguration owner) {
+      super(owner);
+    }
+
+    @Override
+    public String asString() throws IOException {
+      if (!owner.entries.containsKey("content")) {
+        throw new IOException("Configuration has no content field and cannot " +
+            "be retrieved as type 'env'");
+      }
+      String content = owner.entries.get("content");
+      return ConfigUtils.replaceProps(owner.entries, content);
+    }
+  }
+
+  public static class TemplateOutputter extends EnvOutputter {
+    public TemplateOutputter(PublishedConfiguration owner) {
+      super(owner);
+    }
+  }
+
+  public static class YamlOutputter extends PublishedConfigurationOutputter {
+
+    private final Yaml yaml;
+
+    public YamlOutputter(PublishedConfiguration owner) {
+      super(owner);
+      DumperOptions options = new DumperOptions();
+      options.setDefaultFlowStyle(FlowStyle.BLOCK);
+      yaml = new Yaml(options);
+    }
+
+    public String asString() throws IOException {
+      return yaml.dump(owner.entries);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedExports.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedExports.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedExports.java
new file mode 100644
index 0000000..0759b4e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedExports.java
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.registry.docstore;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.SerializationConfig;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.io.IOException;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * JSON-serializable description of a published key-val configuration.
+ *
+ * The values themselves are not serialized in the external view; they have to be served up by the far end
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class PublishedExports {
+
+  public String description;
+  public long updated;
+  public String updatedTime;
+  public Map<String, List<ExportEntry>> entries = new HashMap<>();
+
+  public PublishedExports() {
+  }
+
+  /**
+   * build an empty published configuration
+   *
+   * @param description configuration description
+   */
+  public PublishedExports(String description) {
+    this.description = description;
+  }
+
+  /**
+   * Build a configuration from the entries
+   *
+   * @param description configuration description
+   * @param entries     entries to put
+   */
+  public PublishedExports(String description,
+                          Iterable<Map.Entry<String, List<ExportEntry>>> entries) {
+    this.description = description;
+    putValues(entries);
+  }
+
+  /**
+   * Is the configuration empty. This means either that it has not been given any values,
+   * or it is stripped down copy
+   * set down over the wire.
+   *
+   * @return true if it is empty
+   */
+  public boolean isEmpty() {
+    return entries.isEmpty();
+  }
+
+  public long getUpdated() {
+    return updated;
+  }
+
+  public void setUpdated(long updated) {
+    this.updated = updated;
+    this.updatedTime = new Date(updated).toString();
+  }
+
+  /**
+   * Set the values from an iterable (this includes a Hadoop Configuration and Java properties
+   * object). Any existing value set is discarded
+   *
+   * @param entries entries to put
+   */
+  public void putValues(Iterable<Map.Entry<String, List<ExportEntry>>> entries) {
+    this.entries = new HashMap<String, List<ExportEntry>>();
+    for (Map.Entry<String, List<ExportEntry>> entry : entries) {
+      this.entries.put(entry.getKey(), entry.getValue());
+    }
+  }
+
+  /**
+   * Return the values as json string
+   *
+   * @return the JSON form
+   *
+   * @throws IOException mapping problems
+   */
+  public String asJson() throws IOException {
+    ObjectMapper mapper = new ObjectMapper();
+    mapper.configure(SerializationConfig.Feature.INDENT_OUTPUT, true);
+    String json = mapper.writeValueAsString(entries);
+    return json;
+  }
+
+  /**
+   * This makes a copy without the nested content -so is suitable for returning as part of the list of a parent's
+   * values
+   *
+   * @return the copy
+   */
+  public PublishedExports shallowCopy() {
+    PublishedExports that = new PublishedExports();
+    that.description = this.description;
+    that.updated = this.updated;
+    that.updatedTime = this.updatedTime;
+    return that;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder("PublishedConfiguration{");
+    sb.append("description='").append(description).append('\'');
+    sb.append(" entries = ").append(entries.size());
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedExportsOutputter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedExportsOutputter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedExportsOutputter.java
new file mode 100644
index 0000000..67cb094
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedExportsOutputter.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.registry.docstore;
+
+import com.google.common.base.Charsets;
+import com.google.common.base.Preconditions;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+
+/** Output a published configuration */
+public abstract class PublishedExportsOutputter {
+
+  protected final PublishedExports exports;
+
+  protected PublishedExportsOutputter(PublishedExports exports) {
+    this.exports = exports;
+  }
+
+  /**
+   * Create an outputter for the chosen format
+   *
+   * @param format  format enumeration
+   * @param exports owning config
+   * @return the outputter
+   */
+
+  public static PublishedExportsOutputter createOutputter(ConfigFormat format,
+                                                         PublishedExports exports) {
+    Preconditions.checkNotNull(exports);
+    switch (format) {
+      case JSON:
+        return new JsonOutputter(exports);
+      default:
+        throw new RuntimeException("Unsupported format :" + format);
+    }
+  }
+
+  public void save(File dest) throws IOException {
+    FileOutputStream out = null;
+    try {
+      out = new FileOutputStream(dest);
+      save(out);
+      out.close();
+    } finally {
+      org.apache.hadoop.io.IOUtils.closeStream(out);
+    }
+  }
+
+  /**
+   * Save the content. The default saves the asString() value to the output stream
+   *
+   * @param out output stream
+   * @throws IOException
+   */
+  public void save(OutputStream out) throws IOException {
+    IOUtils.write(asString(), out, Charsets.UTF_8);
+  }
+
+  /**
+   * Convert to a string
+   *
+   * @return the string form
+   * @throws IOException
+   */
+  public abstract String asString() throws IOException;
+
+  public static class JsonOutputter extends PublishedExportsOutputter {
+
+    public JsonOutputter(PublishedExports exports) {
+      super(exports);
+    }
+
+    @Override
+    public void save(File dest) throws IOException {
+      FileUtils.writeStringToFile(dest, asString(), Charsets.UTF_8);
+    }
+
+    @Override
+    public String asString() throws IOException {
+      return exports.asJson();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedExportsSet.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedExportsSet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedExportsSet.java
new file mode 100644
index 0000000..339d3d6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedExportsSet.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.registry.docstore;
+
+import org.apache.slider.server.appmaster.web.rest.RestPaths;
+import org.apache.slider.server.services.utility.PatternValidator;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.HashMap;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * Represents a set of configurations for an application, component, etc.
+ * Json serialisable; accessors are synchronized
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class PublishedExportsSet {
+
+  private static final PatternValidator validator = new PatternValidator(
+      RestPaths.PUBLISHED_CONFIGURATION_REGEXP);
+  
+  public Map<String, PublishedExports> exports = new HashMap<>();
+
+  public PublishedExportsSet() {
+  }
+
+  /**
+   * Put a name -it will be converted to lower case before insertion.
+   * Any existing entry will be overwritten (that includes an entry
+   * with a different case in the original name)
+   * @param name name of entry
+   * @param export published export
+   * @throws IllegalArgumentException if not a valid name
+   */
+  public void put(String name, PublishedExports export) {
+    String name1 = name.toLowerCase(Locale.ENGLISH);
+    validateName(name1);
+    exports.put(name1, export);
+  }
+
+  /**
+   * Validate the name -restricting it to the set defined in 
+   * {@link RestPaths#PUBLISHED_CONFIGURATION_REGEXP}
+   * @param name name to validate
+   * @throws IllegalArgumentException if not a valid name
+   */
+  public static void validateName(String name) {
+    validator.validate(name);
+    
+  }
+
+  public PublishedExports get(String name) {
+    return exports.get(name);
+  }
+  
+  public boolean contains(String name) {
+    return exports.containsKey(name);
+  }
+  
+  public int size() {
+    return exports.size();
+  }
+  
+  public Set<String> keys() {
+    TreeSet<String> keys = new TreeSet<>();
+    keys.addAll(exports.keySet());
+    return keys;
+  }
+
+  public PublishedExportsSet shallowCopy() {
+    PublishedExportsSet that = new PublishedExportsSet();
+    for (Map.Entry<String, PublishedExports> entry : exports.entrySet()) {
+      that.put(entry.getKey(), entry.getValue().shallowCopy());
+    }
+    return that;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/UriMap.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/UriMap.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/UriMap.java
new file mode 100644
index 0000000..120966f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/UriMap.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.registry.docstore;
+
+import org.codehaus.jackson.annotate.JsonIgnore;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.HashMap;
+import java.util.Map;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class UriMap {
+
+  public Map<String, String> uris = new HashMap<>();
+  
+  @JsonIgnore
+  public void put(String key, String value) {
+    uris.put(key, value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/info/CustomRegistryConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/info/CustomRegistryConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/info/CustomRegistryConstants.java
new file mode 100644
index 0000000..13ad5c5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/info/CustomRegistryConstants.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.registry.info;
+
+/**
+ * These are constants unique to the Slider AM
+ */
+public class CustomRegistryConstants {
+
+  public static final String MANAGEMENT_REST_API =
+      "classpath:org.apache.slider.management";
+  
+  public static final String REGISTRY_REST_API =
+      "classpath:org.apache.slider.registry";
+  
+  public static final String PUBLISHER_REST_API =
+      "classpath:org.apache.slider.publisher";
+
+  public static final String PUBLISHER_CONFIGURATIONS_API =
+      "classpath:org.apache.slider.publisher.configurations";
+
+  public static final String PUBLISHER_EXPORTS_API =
+      "classpath:org.apache.slider.publisher.exports";
+
+  public static final String PUBLISHER_DOCUMENTS_API =
+      "classpath:org.apache.slider.publisher.documents";
+
+  public static final String AGENT_SECURE_REST_API =
+      "classpath:org.apache.slider.agents.secure";
+
+  public static final String AGENT_ONEWAY_REST_API =
+      "classpath:org.apache.slider.agents.oneway";
+
+  public static final String AM_IPC_PROTOCOL =
+      "classpath:org.apache.slider.appmaster.ipc";
+
+  public static final String AM_REST_BASE =
+      "classpath:org.apache.slider.client.rest";
+
+  public static final String WEB_UI = "http://";
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/retrieve/AMWebClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/retrieve/AMWebClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/retrieve/AMWebClient.java
new file mode 100644
index 0000000..40fa217
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/retrieve/AMWebClient.java
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.registry.retrieve;
+
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.GenericType;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.client.config.ClientConfig;
+import com.sun.jersey.api.client.config.DefaultClientConfig;
+import com.sun.jersey.api.json.JSONConfiguration;
+import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
+import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.slider.client.rest.BaseRestClient;
+import org.apache.slider.core.restclient.HttpVerb;
+import org.apache.slider.core.restclient.UgiJerseyBinding;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MediaType;
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URI;
+import java.net.URL;
+
+/**
+ * Class to retrieve artifacts from the AM's web site. This sets up
+ * the redirection and security logic properly
+ */
+public class AMWebClient {
+
+
+  private final BaseRestClient restClient;
+  private static final Logger
+      log = LoggerFactory.getLogger(AMWebClient.class);
+
+
+  public AMWebClient(Configuration conf) {
+    UgiJerseyBinding binding = new UgiJerseyBinding(conf);
+
+    restClient = new BaseRestClient(binding.createJerseyClient());
+
+  }
+
+
+  private static URLConnectionClientHandler getUrlConnectionClientHandler() {
+    return new URLConnectionClientHandler(new HttpURLConnectionFactory() {
+      @Override
+      public HttpURLConnection getHttpURLConnection(URL url)
+          throws IOException {
+        HttpURLConnection connection = (HttpURLConnection) url.openConnection();
+        if (connection.getResponseCode() == HttpURLConnection.HTTP_MOVED_TEMP) {
+          // is a redirect - are we changing schemes?
+          String redirectLocation = connection.getHeaderField(HttpHeaders.LOCATION);
+          String originalScheme = url.getProtocol();
+          String redirectScheme = URI.create(redirectLocation).getScheme();
+          if (!originalScheme.equals(redirectScheme)) {
+            // need to fake it out by doing redirect ourselves
+            log.info("Protocol change during redirect. Redirecting {} to URL {}",
+                     url, redirectLocation);
+            URL redirectURL = new URL(redirectLocation);
+            connection = (HttpURLConnection) redirectURL.openConnection();
+          }
+        }
+        if (connection instanceof HttpsURLConnection) {
+          log.debug("Attempting to configure HTTPS connection using client "
+                    + "configuration");
+          final SSLFactory factory;
+          final SSLSocketFactory sf;
+          final HostnameVerifier hv;
+
+          try {
+            HttpsURLConnection c = (HttpsURLConnection) connection;
+            factory = new SSLFactory(SSLFactory.Mode.CLIENT, new Configuration());
+            factory.init();
+            sf = factory.createSSLSocketFactory();
+            hv = factory.getHostnameVerifier();
+            c.setSSLSocketFactory(sf);
+            c.setHostnameVerifier(hv);
+          } catch (Exception e) {
+            log.info("Unable to configure HTTPS connection from "
+                     + "configuration.  Using JDK properties.");
+          }
+
+        }
+        return connection;
+      }
+    });
+  }
+
+  public WebResource resource(String url) {
+    return restClient.resource(url);
+  }
+
+  public BaseRestClient getRestClient() {
+    return restClient;
+  }
+
+  /**
+   * Execute the operation. Failures are raised as IOException subclasses
+   * @param method method to execute
+   * @param resource resource to work against
+   * @param c class to build
+   * @param <T> type expected
+   * @return an instance of the type T
+   * @throws IOException on any failure
+   */
+  public <T> T exec(HttpVerb method, WebResource resource, Class<T> c) throws IOException {
+    return restClient.exec(method, resource, c);
+  }
+
+  /**
+   * Execute the operation. Failures are raised as IOException subclasses
+   * @param method method to execute
+   * @param resource resource to work against
+   * @param t type to work with
+   * @param <T> type expected
+   * @return an instance of the type T
+   * @throws IOException on any failure
+   */
+  public <T> T exec(HttpVerb method, WebResource resource, GenericType<T> t)
+      throws IOException {
+    return restClient.exec(method, resource, t);
+  }
+
+  /**
+   * Execute the  GET operation. Failures are raised as IOException subclasses
+   * @param resource resource to work against
+   * @param c class to build
+   * @param <T> type expected
+   * @return an instance of the type T
+   * @throws IOException on any failure
+   */
+  public <T> T get(WebResource resource, Class<T> c) throws IOException {
+    return restClient.get(resource, c);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java
new file mode 100644
index 0000000..b0eddb8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/retrieve/RegistryRetriever.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.registry.retrieve;
+
+import com.beust.jcommander.Strings;
+import com.sun.jersey.api.client.UniformInterfaceException;
+import com.sun.jersey.api.client.WebResource;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.registry.client.exceptions.RegistryIOException;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import static org.apache.slider.client.ClientRegistryBinder.*;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.exceptions.ExceptionConverter;
+import org.apache.slider.core.registry.docstore.PublishedConfigSet;
+import org.apache.slider.core.registry.docstore.PublishedConfiguration;
+import org.apache.slider.core.registry.docstore.PublishedExports;
+import org.apache.slider.core.registry.docstore.PublishedExportsSet;
+import static org.apache.slider.core.registry.info.CustomRegistryConstants.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+/**
+ * Registry retriever. 
+ * This hides the HTTP operations that take place to
+ * get the actual content
+ */
+public class RegistryRetriever extends AMWebClient {
+  private static final Logger log = LoggerFactory.getLogger(RegistryRetriever.class);
+
+  private final String externalConfigurationURL;
+  private final String internalConfigurationURL;
+  private final String externalExportsURL;
+  private final String internalExportsURL;
+
+  /**
+   * Retrieve from a service by locating the
+   * exported {@link CustomRegistryConstants.PUBLISHER_CONFIGURATIONS_API}
+   * and working off it.
+   *
+   * @param conf configuration to work from
+   * @param record service record
+   * @throws RegistryIOException the address type of the endpoint does
+   * not match that expected (i.e. not a list of URLs), missing endpoint...
+   */
+  public RegistryRetriever(Configuration conf, ServiceRecord record) throws RegistryIOException {
+    super(conf);
+    externalConfigurationURL = lookupRestAPI(record,
+        PUBLISHER_CONFIGURATIONS_API, true);
+    internalConfigurationURL = lookupRestAPI(record,
+        PUBLISHER_CONFIGURATIONS_API, false);
+    externalExportsURL = lookupRestAPI(record,
+        PUBLISHER_EXPORTS_API, true);
+    internalExportsURL = lookupRestAPI(record,
+        PUBLISHER_EXPORTS_API, false);
+  }
+
+  /**
+   * Does a bonded registry retriever have a configuration?
+   * @param external flag to indicate that it is the external entries to fetch
+   * @return true if there is a URL to the configurations defined
+   */
+  public boolean hasConfigurations(boolean external) {
+    return !Strings.isStringEmpty(
+        external ? externalConfigurationURL : internalConfigurationURL);
+  }
+  
+  /**
+   * Get the configurations of the registry
+   * @param external flag to indicate that it is the external entries to fetch
+   * @return the configuration sets
+   */
+  public PublishedConfigSet getConfigurations(boolean external) throws
+      FileNotFoundException, IOException {
+
+    String confURL = getConfigurationURL(external);
+      WebResource webResource = resource(confURL);
+    return get(webResource, PublishedConfigSet.class);
+  }
+
+  protected String getConfigurationURL(boolean external) throws FileNotFoundException {
+    String confURL = external ? externalConfigurationURL: internalConfigurationURL;
+    if (Strings.isStringEmpty(confURL)) {
+      throw new FileNotFoundException("No configuration URL");
+    }
+    return confURL;
+  }
+
+  protected String getExportURL(boolean external) throws FileNotFoundException {
+    String confURL = external ? externalExportsURL: internalExportsURL;
+    if (Strings.isStringEmpty(confURL)) {
+      throw new FileNotFoundException("No configuration URL");
+    }
+    return confURL;
+  }
+
+  /**
+   * Get the configurations of the registry
+   * @param external flag to indicate that it is the external entries to fetch
+   * @return the configuration sets
+   */
+  public PublishedExportsSet getExports(boolean external) throws
+      FileNotFoundException, IOException {
+
+    String exportsUrl = getExportURL(external);
+    WebResource webResource = resource(exportsUrl);
+    return get(webResource, PublishedExportsSet.class);
+  }
+
+
+  /**
+   * Get a complete configuration, with all values
+   * @param configSet config set to ask for
+   * @param name name of the configuration
+   * @param external flag to indicate that it is an external configuration
+   * @return the retrieved config
+   * @throws IOException IO problems
+   */
+  public PublishedConfiguration retrieveConfiguration(PublishedConfigSet configSet,
+      String name,
+      boolean external) throws IOException {
+    String confURL = getConfigurationURL(external);
+    if (!configSet.contains(name)) {
+      throw new FileNotFoundException("Unknown configuration " + name);
+    }
+    confURL = SliderUtils.appendToURL(confURL, name);
+    WebResource webResource = resource(confURL);
+    return get(webResource, PublishedConfiguration.class);
+  }
+
+  /**
+   * Get a complete export, with all values
+   * @param exportSet
+   * @param name name of the configuration
+   * @param external flag to indicate that it is an external configuration
+   * @return the retrieved config
+   * @throws IOException IO problems
+   */
+  public PublishedExports retrieveExports(PublishedExportsSet exportSet,
+                                                      String name,
+                                                      boolean external) throws IOException {
+    if (!exportSet.contains(name)) {
+      throw new FileNotFoundException("Unknown export " + name);
+    }
+    String exportsURL = getExportURL(external);
+    exportsURL = SliderUtils.appendToURL(exportsURL, name);
+    return get(resource(exportsURL), PublishedExports.class);
+ }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder("RegistryRetriever{");
+    sb.append("externalConfigurationURL='")
+      .append(externalConfigurationURL)
+      .append('\'');
+    sb.append(", internalConfigurationURL='")
+      .append(internalConfigurationURL)
+      .append('\'');
+    sb.append(", externalExportsURL='").append(externalExportsURL).append('\'');
+    sb.append(", internalExportsURL='").append(internalExportsURL).append('\'');
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/HttpOperationResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/HttpOperationResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/HttpOperationResponse.java
new file mode 100644
index 0000000..0266223
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/HttpOperationResponse.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.restclient;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A response for use as a return value from operations
+ */
+public class HttpOperationResponse {
+  
+  public int responseCode;
+  public long lastModified;
+  public String contentType;
+  public byte[] data;
+  public Map<String, List<String>> headers;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/HttpVerb.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/HttpVerb.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/HttpVerb.java
new file mode 100644
index 0000000..c040345
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/HttpVerb.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.restclient;
+
+/**
+ * Http verbs with details on what they support in terms of submit and
+ * response bodies.
+ * <p>
+ * Those verbs which do support bodies in the response MAY NOT return it;
+ * if the response code is 204 then the answer is "no body", but the operation
+ * is considered a success.
+ */
+public enum HttpVerb {
+  GET("GET", false, true),
+  POST("POST", true, true),
+  PUT("PUT", true, true),
+  DELETE("DELETE", false, true),
+  HEAD("HEAD", false, false);
+  
+  private final String verb;
+  private final boolean hasUploadBody;
+  private final boolean hasResponseBody;
+
+  HttpVerb(String verb, boolean hasUploadBody, boolean hasResponseBody) {
+    this.verb = verb;
+    this.hasUploadBody = hasUploadBody;
+    this.hasResponseBody = hasResponseBody;
+  }
+
+  public String getVerb() {
+    return verb;
+  }
+
+  public boolean hasUploadBody() {
+    return hasUploadBody;
+  }
+
+  public boolean hasResponseBody() {
+    return hasResponseBody;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/SliderURLConnectionFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/SliderURLConnectionFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/SliderURLConnectionFactory.java
new file mode 100644
index 0000000..e453f52
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/SliderURLConnectionFactory.java
@@ -0,0 +1,176 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.restclient;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.web.KerberosUgiAuthenticator;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.net.URLConnection;
+import java.security.GeneralSecurityException;
+
+/**
+ * Factory for URL connections; used behind the scenes in the Jersey integration.
+ * <p>
+ * Derived from the WebHDFS implementation.
+ */
+public class SliderURLConnectionFactory {
+  private static final Logger log =
+      LoggerFactory.getLogger(SliderURLConnectionFactory.class);
+
+  /**
+   * Timeout for socket connects and reads
+   */
+  public final static int DEFAULT_SOCKET_TIMEOUT = 60 * 1000; // 1 minute
+  private final ConnectionConfigurator connConfigurator;
+
+  private static final ConnectionConfigurator DEFAULT_CONFIGURATOR = new BasicConfigurator();
+
+  /**
+   * Construct a new URLConnectionFactory based on the configuration. It will
+   * try to load SSL certificates when it is specified.
+   */
+  public static SliderURLConnectionFactory newInstance(Configuration conf) {
+    ConnectionConfigurator conn;
+    try {
+      conn = newSslConnConfigurator(DEFAULT_SOCKET_TIMEOUT, conf);
+    } catch (Exception e) {
+      log.debug("Cannot load customized SSL configuration.", e);
+      conn = DEFAULT_CONFIGURATOR;
+    }
+    return new SliderURLConnectionFactory(conn);
+  }
+
+  private SliderURLConnectionFactory(ConnectionConfigurator connConfigurator) {
+    this.connConfigurator = connConfigurator;
+  }
+
+  /**
+   * Create a new ConnectionConfigurator for SSL connections
+   */
+  private static ConnectionConfigurator newSslConnConfigurator(final int timeout,
+      Configuration conf) throws IOException, GeneralSecurityException {
+    final SSLFactory factory;
+    final SSLSocketFactory sf;
+    final HostnameVerifier hv;
+
+    factory = new SSLFactory(SSLFactory.Mode.CLIENT, conf);
+    factory.init();
+    sf = factory.createSSLSocketFactory();
+    hv = factory.getHostnameVerifier();
+
+    return new ConnectionConfigurator() {
+      @Override
+      public HttpURLConnection configure(HttpURLConnection conn)
+          throws IOException {
+        if (conn instanceof HttpsURLConnection) {
+          HttpsURLConnection c = (HttpsURLConnection) conn;
+          c.setSSLSocketFactory(sf);
+          c.setHostnameVerifier(hv);
+        }
+        SliderURLConnectionFactory.setupConnection(conn, timeout);
+        return conn;
+      }
+    };
+  }
+
+  /**
+   * Opens a url with read and connect timeouts
+   *
+   * @param url
+   *          to open
+   * @return URLConnection
+   * @throws IOException
+   */
+  public URLConnection openConnection(URL url) throws IOException {
+    try {
+      return openConnection(url, false);
+    } catch (AuthenticationException e) {
+      // Unreachable
+      return null;
+    }
+  }
+
+  /**
+   * Opens a url with read and connect timeouts
+   *
+   * @param url
+   *          URL to open
+   * @param isSpnego
+   *          whether the url should be authenticated via SPNEGO
+   * @return URLConnection
+   * @throws IOException
+   * @throws AuthenticationException
+   */
+  public URLConnection openConnection(URL url, boolean isSpnego)
+      throws IOException, AuthenticationException {
+    if (isSpnego) {
+        log.debug("open AuthenticatedURL connection {}", url);
+      UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
+      final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
+      return new AuthenticatedURL(new KerberosUgiAuthenticator(),
+          connConfigurator).openConnection(url, authToken);
+    } else {
+      log.debug("open URL connection {}", url);
+      URLConnection connection = url.openConnection();
+      if (connection instanceof HttpURLConnection) {
+        connConfigurator.configure((HttpURLConnection) connection);
+      }
+      return connection;
+    }
+  }
+
+  /**
+   * Sets connection parameters on the given URLConnection
+   * 
+   * @param connection
+   *          URLConnection to set
+   * @param socketTimeout
+   *          the connection and read timeout of the connection.
+   */
+  private static void setupConnection(URLConnection connection, int socketTimeout) {
+    connection.setConnectTimeout(socketTimeout);
+    connection.setReadTimeout(socketTimeout);
+    connection.setUseCaches(false);
+    if (connection instanceof HttpURLConnection) {
+      ((HttpURLConnection) connection).setInstanceFollowRedirects(true);
+    }
+  }
+
+  private static class BasicConfigurator implements ConnectionConfigurator {
+    @Override
+    public HttpURLConnection configure(HttpURLConnection conn)
+        throws IOException {
+      SliderURLConnectionFactory.setupConnection(conn, DEFAULT_SOCKET_TIMEOUT);
+      return conn;
+    }
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[05/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/ComponentResource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/ComponentResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/ComponentResource.java
new file mode 100644
index 0000000..a44448e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/ComponentResource.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.management.resources;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import javax.ws.rs.core.UriBuilder;
+import java.util.Map;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class ComponentResource {
+  private final Map<String, String> props;
+  private String href;
+
+  public ComponentResource() {
+    this(null, null, null, null);
+  }
+
+  public ComponentResource(String name,
+                           Map<String, String> props,
+                           UriBuilder uriBuilder,
+                           Map<String, Object> pathElems) {
+    this.props = props;
+  }
+
+  public Map<String, String> getProps() {
+    return props;
+  }
+
+  public String getHref() {
+    return href;
+  }
+
+  public void setHref(String href) {
+    this.href = href;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/ConfTreeResource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/ConfTreeResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/ConfTreeResource.java
new file mode 100644
index 0000000..407bab6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/ConfTreeResource.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.management.resources;
+
+import org.apache.slider.core.conf.ConfTree;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import javax.ws.rs.core.UriBuilder;
+import java.util.Map;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class ConfTreeResource {
+
+  private final String href;
+  private final Map<String, Object> metadata;
+  private final Map<String, String> global;
+  private final Map<String, Map<String, String>> components;
+
+  public ConfTreeResource() {
+    this(null, null);
+  }
+
+  public ConfTreeResource(ConfTree confTree,
+                          UriBuilder uriBuilder) {
+    if (uriBuilder != null && confTree != null) {
+      metadata = confTree.metadata;
+      global = confTree.global;
+      components = confTree.components;
+      this.href = uriBuilder.build().toASCIIString();
+    } else {
+      this.href = null;
+      this.metadata = null;
+      this.global = null;
+      this.components = null;
+    }
+  }
+
+  public Map<String, Object> getMetadata() {
+    return metadata;
+  }
+
+  public Map<String, String> getGlobal() {
+    return global;
+  }
+
+  public Map<String, Map<String, String>> getComponents() {
+    return components;
+  }
+
+  public String getHref() {
+    return href;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/ResourceFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/ResourceFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/ResourceFactory.java
new file mode 100644
index 0000000..9876412
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/ResourceFactory.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.management.resources;
+
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTree;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import javax.ws.rs.core.UriBuilder;
+import java.util.Map;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class ResourceFactory {
+
+  public static AggregateConfResource createAggregateConfResource(AggregateConf conf,
+                                                                  UriBuilder uriBuilder) {
+    return new AggregateConfResource(conf, uriBuilder);
+  }
+
+  public static ConfTreeResource createConfTreeResource(ConfTree confTree,
+                                                        UriBuilder uriBuilder) {
+    return new ConfTreeResource(confTree, uriBuilder);
+  }
+
+  public static ComponentResource createComponentResource(String name,
+                                                          Map<String, String> props,
+                                                          UriBuilder uriBuilder,
+                                                          Map<String, Object> pathElems) {
+    return new ComponentResource(name, props, uriBuilder, pathElems);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/publisher/PublisherResource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/publisher/PublisherResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/publisher/PublisherResource.java
new file mode 100644
index 0000000..c727581
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/publisher/PublisherResource.java
@@ -0,0 +1,273 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest.publisher;
+
+import org.apache.hadoop.yarn.webapp.NotFoundException;
+import org.apache.slider.core.registry.docstore.ConfigFormat;
+import org.apache.slider.core.registry.docstore.PublishedConfigSet;
+import org.apache.slider.core.registry.docstore.PublishedConfiguration;
+import org.apache.slider.core.registry.docstore.PublishedConfigurationOutputter;
+import org.apache.slider.core.registry.docstore.PublishedExports;
+import org.apache.slider.core.registry.docstore.PublishedExportsSet;
+import org.apache.slider.core.registry.docstore.UriMap;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.apache.slider.server.appmaster.web.rest.AbstractSliderResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.UriInfo;
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static  org.apache.slider.server.appmaster.web.rest.RestPaths.*;
+
+/**
+ * This publishes configuration sets
+ */
+public class PublisherResource extends AbstractSliderResource {
+  protected static final Logger log =
+      LoggerFactory.getLogger(PublisherResource.class);
+  public static final String EXPORTS_NAME = "exports";
+  public static final String EXPORTS_RESOURCES_PATH = "/" + EXPORTS_NAME;
+  public static final String EXPORT_RESOURCE_PATH = EXPORTS_RESOURCES_PATH + "/{exportname}" ;
+  public static final String SET_NAME =
+      "{setname: " + PUBLISHED_CONFIGURATION_SET_REGEXP + "}";
+  public static final String SETNAME = "setname";
+  public static final String CLASSPATH = "/classpath";
+  public static final String CONFIG = "config";
+  
+  public static final String SETNAME_PATTERN = 
+      "{"+ SETNAME+": " + PUBLISHED_CONFIGURATION_SET_REGEXP + "}";
+  private static final String CONFIG_PATTERN =
+      SETNAME_PATTERN + "/{"+ CONFIG +": " + PUBLISHED_CONFIGURATION_REGEXP + "}";
+  private final StateAccessForProviders appState;
+
+  public PublisherResource(WebAppApi slider) {
+    super(slider);
+    appState = slider.getAppState();
+  }
+
+  private void init(HttpServletResponse res, UriInfo uriInfo) {
+    res.setContentType(null);
+    log.debug(uriInfo.getRequestUri().toString());
+  }
+ 
+  /**
+   * Get a named config set 
+   * @param setname name of the config set
+   * @return the config set
+   * @throws NotFoundException if there was no matching set
+   */
+  private PublishedConfigSet getConfigSet(String setname) {
+    PublishedConfigSet configSet =
+        appState.getPublishedConfigSet(setname);
+    if (configSet == null) {
+      throw new NotFoundException("Not found: " + setname);
+    }
+    return configSet;
+  }
+
+  @GET
+  @Path("/")
+  @Produces({MediaType.APPLICATION_JSON})
+  public UriMap enumConfigSets(
+      @Context UriInfo uriInfo,
+      @Context HttpServletResponse res) {
+    init(res, uriInfo);
+    String baseURL = uriInfo.getRequestUri().toString();
+    if (!baseURL.endsWith("/")) {
+      baseURL += "/";
+    }
+    UriMap uriMap = new UriMap();
+    for (String name : appState.listConfigSets()) {
+      uriMap.put(name, baseURL + name);
+      log.info("registering config set {} at {}", name, baseURL);
+    }
+    uriMap.put(EXPORTS_NAME, baseURL + EXPORTS_NAME);
+    return uriMap;
+  }
+
+  @GET
+  @Path(CLASSPATH)
+  @Produces({MediaType.APPLICATION_JSON})
+  public Set<URL> getAMClassPath() {
+    URL[] urls = ((URLClassLoader) getClass().getClassLoader()).getURLs();
+    return new LinkedHashSet<URL>(Arrays.asList(urls));
+  }
+
+  @GET
+  @Path(EXPORTS_RESOURCES_PATH)
+  @Produces({MediaType.APPLICATION_JSON})
+  public PublishedExportsSet gePublishedExports() {
+
+    return appState.getPublishedExportsSet();
+  }
+
+  @GET
+  @Path(EXPORT_RESOURCE_PATH)
+  @Produces({MediaType.APPLICATION_JSON})
+  public PublishedExports getAMExports2(@PathParam("exportname") String exportname,
+                              @Context UriInfo uriInfo,
+                              @Context HttpServletResponse res) {
+    init(res, uriInfo);
+    PublishedExportsSet set = appState.getPublishedExportsSet();
+    return set.get(exportname);
+  }
+
+  @GET
+  @Path("/"+ SETNAME_PATTERN)
+  @Produces({MediaType.APPLICATION_JSON})
+  public PublishedConfigSet getPublishedConfiguration(
+      @PathParam(SETNAME) String setname,
+      @Context UriInfo uriInfo,
+      @Context HttpServletResponse res) {
+    init(res, uriInfo);
+
+    logRequest(uriInfo);
+    PublishedConfigSet publishedConfigSet = getConfigSet(setname);
+    log.debug("Number of configurations: {}", publishedConfigSet.size());
+    return publishedConfigSet.shallowCopy();
+  }
+
+  private void logRequest(UriInfo uriInfo) {
+    log.info(uriInfo.getRequestUri().toString());
+  }
+
+  @GET
+  @Path("/" + CONFIG_PATTERN)
+  @Produces({MediaType.APPLICATION_JSON})
+  public PublishedConfiguration getConfigurationInstance(
+      @PathParam(SETNAME) String setname,
+      @PathParam(CONFIG) String config,
+      @Context UriInfo uriInfo,
+      @Context HttpServletResponse res) {
+    init(res, uriInfo);
+
+    PublishedConfiguration publishedConfig =
+        getPublishedConfiguration(setname, config);
+    if (publishedConfig == null) {
+      log.info("Configuration {} not found", config);
+      throw new NotFoundException("Not found: " + uriInfo.getAbsolutePath());
+    }
+    return publishedConfig;
+  }
+
+  /**
+   * Get a configuration
+   * @param setname name of the config set
+   * @param config config
+   * @return null if there was a config, but not a set
+   * @throws NotFoundException if there was no matching set
+   */
+  public PublishedConfiguration getPublishedConfiguration(String setname,
+      String config) {
+    return getConfigSet(setname).get(config);
+  }
+
+  @GET
+  @Path("/" + CONFIG_PATTERN + ".json")
+  @Produces({MediaType.APPLICATION_JSON})
+  public String getConfigurationContentJson(
+      @PathParam(SETNAME) String setname,
+
+      @PathParam(CONFIG) String config,
+      @Context UriInfo uriInfo,
+      @Context HttpServletResponse res) throws IOException {
+    return getStringRepresentation(setname, config, uriInfo, res,
+        ConfigFormat.JSON);
+  }
+
+  @GET
+  @Path("/" + CONFIG_PATTERN + ".xml")
+  @Produces({MediaType.APPLICATION_XML})
+  public String getConfigurationContentXML(
+      @PathParam(SETNAME) String setname,
+      @PathParam(CONFIG) String config,
+      @Context UriInfo uriInfo,
+      @Context HttpServletResponse res) throws IOException {
+    return getStringRepresentation(setname, config, uriInfo, res,
+        ConfigFormat.XML);
+  }
+  
+  @GET
+  @Path("/" + CONFIG_PATTERN + ".properties")
+  @Produces({MediaType.APPLICATION_XML})
+  public String getConfigurationContentProperties(
+      @PathParam(SETNAME) String setname,
+
+      @PathParam(CONFIG) String config,
+      @Context UriInfo uriInfo,
+      @Context HttpServletResponse res) throws IOException {
+
+    return getStringRepresentation(setname, config, uriInfo, res,
+        ConfigFormat.PROPERTIES);
+  }
+
+  public String getStringRepresentation(String setname,
+      String config,
+      UriInfo uriInfo,
+      HttpServletResponse res, ConfigFormat format) throws IOException {
+    // delegate (including init)
+    PublishedConfiguration publishedConfig =
+        getConfigurationInstance(setname, config, uriInfo, res);
+    PublishedConfigurationOutputter outputter =
+        publishedConfig.createOutputter(format);
+    return outputter.asString();
+  }
+
+  @GET
+  @Path("/" + CONFIG_PATTERN +"/{propertyName}")
+  @Produces({MediaType.APPLICATION_JSON})
+  public Map<String,String> getConfigurationProperty(
+      @PathParam(SETNAME) String setname,
+      @PathParam(CONFIG) String config,
+      @PathParam("propertyName") String propertyName,
+      @Context UriInfo uriInfo,
+      @Context HttpServletResponse res) {
+    PublishedConfiguration publishedConfig =
+        getConfigurationInstance(setname, config, uriInfo, res);
+    String propVal = publishedConfig.entries.get(propertyName);
+    if (propVal == null) {
+      log.debug("Configuration property {} not found in configuration {}",
+          propertyName, config);
+      throw new NotFoundException("Property not found: " + propertyName);
+    }
+    Map<String, String> rtnVal = new HashMap<>();
+    rtnVal.put(propertyName, propVal);
+
+    return rtnVal;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/registry/PathEntryResource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/registry/PathEntryResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/registry/PathEntryResource.java
new file mode 100644
index 0000000..efb09a8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/registry/PathEntryResource.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest.registry;
+
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.List;
+
+/**
+ * Representation of a path entry
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class PathEntryResource {
+
+  /**
+   * Child nodes: as the short path to each element
+   */
+  public List<String> nodes;
+
+  /**
+   * Service record: if null \u2014there is no resolvable service
+   * record at this node.
+   */
+  public ServiceRecord service;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/registry/RegistryResource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/registry/RegistryResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/registry/RegistryResource.java
new file mode 100644
index 0000000..c824848
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/registry/RegistryResource.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest.registry;
+
+import com.google.inject.Singleton;
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.registry.client.exceptions.AuthenticationFailedException;
+import org.apache.hadoop.registry.client.exceptions.InvalidRecordException;
+import org.apache.hadoop.registry.client.exceptions.NoPathPermissionsException;
+import org.apache.hadoop.registry.client.exceptions.NoRecordException;
+import org.apache.hadoop.yarn.webapp.ForbiddenException;
+import org.apache.hadoop.yarn.webapp.NotFoundException;
+import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.apache.slider.server.appmaster.web.rest.AbstractSliderResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.UriInfo;
+import java.io.IOException;
+
+/**
+ * This is the read-only view of the YARN registry.
+ * 
+ * Model:
+ * <ol>
+ *   <li>a tree of nodes</li>
+ *   <li>Default view is of children + record</li>
+ * </ol>
+ * 
+ */
+@Singleton
+public class RegistryResource extends AbstractSliderResource {
+  protected static final Logger log =
+      LoggerFactory.getLogger(RegistryResource.class);
+  public static final String SERVICE_PATH =
+      "/{path:.*}";
+
+  private final RegistryOperations registry;
+
+  /**
+   * Construct an instance bonded to a registry
+   * @param slider slider API
+   */
+  public RegistryResource(WebAppApi slider) {
+    super(slider);
+    this.registry = slider.getRegistryOperations();
+  }
+
+  
+  /**
+   * Internal init code, per request
+   * @param request incoming request 
+   * @param uriInfo URI details
+   */
+  private void init(HttpServletRequest request, UriInfo uriInfo) {
+    log.debug(uriInfo.getRequestUri().toString());
+  }
+
+  @GET
+  @Produces({MediaType.APPLICATION_JSON})
+  public PathEntryResource getRoot(@Context HttpServletRequest request,
+      @Context UriInfo uriInfo) {
+    return lookup("/", request, uriInfo);
+  }
+
+//   {path:.*}
+
+  @Path(SERVICE_PATH)
+  @GET
+  @Produces({MediaType.APPLICATION_JSON})
+  public PathEntryResource lookup(
+      @PathParam("path") String path,
+      @Context HttpServletRequest request,
+      @Context UriInfo uriInfo) {
+      init(request, uriInfo);
+      return resolvePath(path);
+  }
+
+  /**
+   * Do the actual processing of requests to responses; can be directly
+   * invoked for testing.
+   * @param path path to query
+   * @return the entry
+   * @throws WebApplicationException on any failure.
+   */
+  public PathEntryResource resolvePath(String path) throws
+      WebApplicationException {
+    try {
+      PathEntryResource pathEntry =
+          fromRegistry(path);
+      if (log.isDebugEnabled()) {
+        log.debug("Resolved:\n{}", pathEntry);
+      }
+      return pathEntry;
+   
+    } catch (Exception e) {
+      throw buildException(path, e);
+    }
+  }
+
+
+  /**
+   * Build from the registry, filling up the children and service records.
+   * If there is no service record at the end of the path, that entry is 
+   * null
+   * @param path path to query
+   * @return the built up record
+   * @throws IOException problems
+   *
+   */
+  private PathEntryResource fromRegistry(String path) throws IOException {
+    PathEntryResource entry = new PathEntryResource();
+    try {
+      entry.service = registry.resolve(path);
+    } catch (NoRecordException e) {
+      // ignoring
+      log.debug("No record at {}", path);
+    } catch (InvalidRecordException e) {
+      // swallowing this exception, the sign of "no entry present"
+      // "nothing parseable"
+        log.warn("Failed to resolve {}: {}", path, e, e);
+    }
+    entry.nodes = registry.list(path);
+    return entry;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ClusterSpecificationBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ClusterSpecificationBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ClusterSpecificationBlock.java
new file mode 100644
index 0000000..2f02f27
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ClusterSpecificationBlock.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.view;
+
+import com.google.inject.Inject;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
+import org.apache.slider.server.appmaster.web.WebAppApi;
+
+/**
+ * 
+ */
+public class ClusterSpecificationBlock extends SliderHamletBlock {
+
+  @Inject
+  public ClusterSpecificationBlock(WebAppApi slider) {
+    super(slider);
+  }
+
+  @Override
+  protected void render(Block html) {
+    doRender(html);
+  }
+
+  // An extra method to make testing easier since you can't make an instance of Block
+  protected void doRender(Hamlet html) {
+    html.
+      div("cluster_json").
+        h2("JSON Cluster Specification").
+        pre().
+          _(getJson())._()._();
+  }
+
+  /**
+   * Get the JSON, catching any exceptions and returning error text instead
+   * @return
+   */
+  private String getJson() {
+    return appState.getClusterStatus().toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ContainerStatsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ContainerStatsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ContainerStatsBlock.java
new file mode 100644
index 0000000..56285c2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/ContainerStatsBlock.java
@@ -0,0 +1,282 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.view;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Maps;
+import com.google.inject.Inject;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TR;
+import org.apache.slider.api.ClusterDescription;
+import org.apache.slider.api.ClusterNode;
+import org.apache.slider.api.types.ComponentInformation;
+import org.apache.slider.server.appmaster.state.RoleInstance;
+import org.apache.slider.server.appmaster.web.WebAppApi;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * 
+ */
+public class ContainerStatsBlock extends SliderHamletBlock {
+
+  private static final String EVEN = "even", ODD = "odd", BOLD = "bold", SCHEME = "http://", PATH = "/node/container/";
+
+  // Some functions that help transform the data into an object we can use to abstract presentation specifics
+  protected static final Function<Entry<String,Integer>,Entry<TableContent,Integer>> stringIntPairFunc = toTableContentFunction();
+  protected static final Function<Entry<String,Long>,Entry<TableContent,Long>> stringLongPairFunc = toTableContentFunction();
+  protected static final Function<Entry<String,String>,Entry<TableContent,String>> stringStringPairFunc = toTableContentFunction();
+
+  @Inject
+  public ContainerStatsBlock(WebAppApi slider) {
+    super(slider);
+  }
+
+  /**
+   * Sort a collection of ClusterNodes by name
+   */
+  protected static class ClusterNodeNameComparator implements Comparator<ClusterNode>,
+      Serializable {
+
+    @Override
+    public int compare(ClusterNode node1, ClusterNode node2) {
+      if (null == node1 && null != node2) {
+        return -1;
+      } else if (null != node1 && null == node2) {
+        return 1;
+      } else if (null == node1) {
+        return 0;
+      }
+
+      final String name1 = node1.name, name2 = node2.name;
+      if (null == name1 && null != name2) {
+        return -1;
+      } else if (null != name1 && null == name2) {
+        return 1;
+      } else if (null == name1) {
+        return 0;
+      }
+
+      return name1.compareTo(name2);
+    }
+
+  }
+
+  @Override
+  protected void render(Block html) {
+    final Map<String,RoleInstance> containerInstances = getContainerInstances(
+        appState.cloneOwnedContainerList());
+
+    Map<String, Map<String, ClusterNode>> clusterNodeMap =
+        appState.getRoleClusterNodeMapping();
+    Map<String, ComponentInformation> componentInfoMap = appState.getComponentInfoSnapshot();
+
+    for (Entry<String, Map<String, ClusterNode>> entry : clusterNodeMap.entrySet()) {
+      final String name = entry.getKey();
+      Map<String, ClusterNode> clusterNodesInRole = entry.getValue();
+      //final RoleStatus roleStatus = entry.getValue();
+
+      DIV<Hamlet> div = html.div("role-info ui-widget-content ui-corner-all");
+
+      List<ClusterNode> nodesInRole =
+          new ArrayList<>(clusterNodesInRole.values());
+
+      div.h2(BOLD, StringUtils.capitalize(name));
+
+      // Generate the details on this role
+      ComponentInformation componentInfo = componentInfoMap.get(name);
+      if (componentInfo != null) {
+        Iterable<Entry<String,Integer>> stats = componentInfo.buildStatistics().entrySet();
+        generateRoleDetails(div,"role-stats-wrap", "Specifications", 
+            Iterables.transform(stats, stringIntPairFunc));
+      }
+
+      // Sort the ClusterNodes by their name (containerid)
+      Collections.sort(nodesInRole, new ClusterNodeNameComparator());
+
+      // Generate the containers running this role
+      generateRoleDetails(div, "role-stats-containers", "Containers",
+          Iterables.transform(nodesInRole, new Function<ClusterNode,Entry<TableContent,String>>() {
+
+            @Override
+            public Entry<TableContent,String> apply(ClusterNode input) {
+              final String containerId = input.name;
+              
+              if (containerInstances.containsKey(containerId)) {
+                RoleInstance roleInst = containerInstances.get(containerId);
+                if (roleInst.container.getNodeHttpAddress() != null) {
+                  return Maps.<TableContent,String> immutableEntry(
+                    new TableAnchorContent(containerId,
+                        buildNodeUrlForContainer(roleInst.container.getNodeHttpAddress(), containerId)), null);
+                }
+              }
+              return Maps.immutableEntry(new TableContent(input.name), null);
+            }
+
+          }));
+
+      ClusterDescription desc = appState.getClusterStatus();
+      Map<String, String> options = desc.getRole(name);
+      Iterable<Entry<TableContent, String>> tableContent;
+      
+      // Generate the pairs of data in the expected form
+      if (null != options) {
+        tableContent = Iterables.transform(options.entrySet(), stringStringPairFunc);
+      } else {
+        // Or catch that we have no options and provide "empty"
+        tableContent = Collections.emptySet();
+      }
+      
+      // Generate the options used by this role
+      generateRoleDetails(div, "role-options-wrap", "Role Options", tableContent);
+
+      // Close the div for this role
+      div._();
+    }
+  }
+
+  protected static <T> Function<Entry<String,T>,Entry<TableContent,T>> toTableContentFunction() {
+    return new Function<Entry<String,T>,Entry<TableContent,T>>() {
+      @Override
+      public Entry<TableContent,T> apply(Entry<String,T> input) {
+        return Maps.immutableEntry(new TableContent(input.getKey()), input.getValue());
+      }
+    };
+  }
+
+  protected Map<String,RoleInstance> getContainerInstances(List<RoleInstance> roleInstances) {
+    Map<String,RoleInstance> map = Maps.newHashMapWithExpectedSize(roleInstances.size());
+    for (RoleInstance roleInstance : roleInstances) {
+      // UUID is the containerId
+      map.put(roleInstance.id, roleInstance);
+    }
+    return map;
+  }
+
+  /**
+   * Given a div, a name for this data, and some pairs of data, generate a nice HTML table. If contents is empty (of size zero), then a mesage will be printed
+   * that there were no items instead of an empty table.
+   *
+   */
+  protected <T1 extends TableContent,T2> void generateRoleDetails(DIV<Hamlet> parent, String divSelector, String detailsName, Iterable<Entry<T1,T2>> contents) {
+    final DIV<DIV<Hamlet>> div = parent.div(divSelector).h3(BOLD, detailsName);
+
+    int offset = 0;
+    TABLE<DIV<DIV<Hamlet>>> table = null;
+    TBODY<TABLE<DIV<DIV<Hamlet>>>> tbody = null;
+    for (Entry<T1,T2> content : contents) {
+      if (null == table) {
+        table = div.table("ui-widget-content ui-corner-bottom");
+        tbody = table.tbody();
+      }
+      
+      TR<TBODY<TABLE<DIV<DIV<Hamlet>>>>> row = tbody.tr(offset % 2 == 0 ? EVEN : ODD);
+      
+      // Defer to the implementation of the TableContent for what the cell should contain
+      content.getKey().printCell(row);
+
+      // Only add the second column if the element is non-null
+      // This also lets us avoid making a second method if we're only making a one-column table
+      if (null != content.getValue()) {
+        row.td(content.getValue().toString());
+      }
+
+      row._();
+
+      offset++;
+    }
+
+    // If we made a table, close it out
+    if (null != table) {
+      tbody._()._();
+    } else {
+      // Otherwise, throw in a nice "no content" message
+      div.p("no-table-contents")._("None")._();
+    }
+    
+    // Close out the initial div
+    div._();
+  }
+
+  /**
+   * Build a URL from the address:port and container ID directly to the NodeManager service
+   * @param nodeAddress
+   * @param containerId
+   * @return
+   */
+  protected String buildNodeUrlForContainer(String nodeAddress, String containerId) {
+    StringBuilder sb = new StringBuilder(SCHEME.length() + nodeAddress.length() + PATH.length() + containerId.length());
+
+    sb.append(SCHEME).append(nodeAddress).append(PATH).append(containerId);
+
+    return sb.toString();
+  }
+
+  /**
+   * Creates a table cell with the provided String as content.
+   */
+  protected static class TableContent {
+    private String cell;
+
+    public TableContent(String cell) {
+      this.cell = cell;
+    }
+
+    public String getCell() {
+      return cell;
+    }
+
+    /**
+     * Adds a td to the given tr. The tr is not closed 
+     * @param tableRow
+     */
+    public void printCell(TR<?> tableRow) {
+      tableRow.td(this.cell);
+    }
+  }
+
+  /**
+   * Creates a table cell with an anchor to the given URL with the provided String as content.
+   */
+  protected static class TableAnchorContent extends TableContent {
+    private String anchorUrl;
+
+    public TableAnchorContent(String cell, String anchorUrl) {
+      super(cell);
+      this.anchorUrl = anchorUrl;
+    }
+
+    /* (non-javadoc)
+     * @see org.apache.slider.server.appmaster.web.view.ContainerStatsBlock$TableContent#printCell()
+     */
+    @Override
+    public void printCell(TR<?> tableRow) {
+      tableRow.td().a(anchorUrl, getCell())._();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/IndexBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/IndexBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/IndexBlock.java
new file mode 100644
index 0000000..c3b9b6f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/IndexBlock.java
@@ -0,0 +1,305 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.view;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.inject.Inject;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.LI;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.UL;
+import org.apache.slider.api.ClusterDescription;
+import org.apache.slider.api.StatusKeys;
+import org.apache.slider.api.types.ApplicationLivenessInformation;
+import org.apache.slider.api.types.RoleStatistics;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.registry.docstore.ExportEntry;
+import org.apache.slider.core.registry.docstore.PublishedExports;
+import org.apache.slider.core.registry.docstore.PublishedExportsSet;
+import org.apache.slider.providers.MonitorDetail;
+import org.apache.slider.providers.ProviderService;
+import org.apache.slider.server.appmaster.state.RoleStatus;
+import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.LIVE_COMPONENTS;
+
+/**
+ * The main content on the Slider AM web page
+ */
+public class IndexBlock extends SliderHamletBlock {
+  private static final Logger log = LoggerFactory.getLogger(IndexBlock.class);
+
+  /**
+   * Message printed when application is at full size.
+   *
+   * {@value}
+   */
+  public static final String ALL_CONTAINERS_ALLOCATED = "all containers allocated";
+
+  @Inject
+  public IndexBlock(WebAppApi slider) {
+    super(slider);
+  }
+
+  @Override
+  protected void render(Block html) {
+    doIndex(html, getProviderName());
+  }
+
+  // An extra method to make testing easier since you can't make an instance of Block
+  @VisibleForTesting
+  protected void doIndex(Hamlet html, String providerName) {
+    ClusterDescription clusterStatus = appState.getClusterStatus();
+    String name = clusterStatus.name;
+    if (name != null && (name.startsWith(" ") || name.endsWith(" "))) {
+      name = "'" + name + "'";
+    } 
+    DIV<Hamlet> div = html.div("general_info")
+                          .h1("index_header",
+                              "Application: " + name);
+
+    ApplicationLivenessInformation liveness =
+        appState.getApplicationLivenessInformation();
+    String livestatus = liveness.allRequestsSatisfied
+        ? ALL_CONTAINERS_ALLOCATED
+        : String.format("Awaiting %d containers", liveness.requestsOutstanding);
+    Hamlet.TABLE<DIV<Hamlet>> table1 = div.table();
+    table1.tr()
+          .td("Status")
+          .td(livestatus)
+          ._();
+    table1.tr()
+          .td("Total number of containers")
+          .td(Integer.toString(appState.getNumOwnedContainers()))
+          ._();
+    table1.tr()
+          .td("Create time: ")
+          .td(getInfoAvoidingNulls(StatusKeys.INFO_CREATE_TIME_HUMAN))
+          ._();
+    table1.tr()
+          .td("Running since: ")
+          .td(getInfoAvoidingNulls(StatusKeys.INFO_LIVE_TIME_HUMAN))
+          ._();
+    table1.tr()
+          .td("Time last flexed: ")
+          .td(getInfoAvoidingNulls(StatusKeys.INFO_FLEX_TIME_HUMAN))
+          ._();
+    table1.tr()
+          .td("Application storage path: ")
+          .td(clusterStatus.dataPath)
+          ._();
+    table1.tr()
+          .td("Application configuration path: ")
+          .td(clusterStatus.originConfigurationPath)
+          ._();
+    table1._();
+    div._();
+    div = null;
+
+    DIV<Hamlet> containers = html.div("container_instances")
+      .h3("Component Instances");
+
+    int aaRoleWithNoSuitableLocations = 0;
+    int aaRoleWithOpenRequest = 0;
+    int roleWithOpenRequest = 0;
+
+    Hamlet.TABLE<DIV<Hamlet>> table = containers.table();
+    Hamlet.TR<Hamlet.THEAD<Hamlet.TABLE<DIV<Hamlet>>>> header = table.thead().tr();
+    trb(header, "Component");
+    trb(header, "Desired");
+    trb(header, "Actual");
+    trb(header, "Outstanding Requests");
+    trb(header, "Failed");
+    trb(header, "Failed to start");
+    trb(header, "Placement");
+    header._()._();  // tr & thead
+
+    List<RoleStatus> roleStatuses = appState.cloneRoleStatusList();
+    Collections.sort(roleStatuses, new RoleStatus.CompareByName());
+    for (RoleStatus status : roleStatuses) {
+      String roleName = status.getName();
+      String nameUrl = apiPath(LIVE_COMPONENTS) + "/" + roleName;
+      String aatext;
+      if (status.isAntiAffinePlacement()) {
+        boolean aaRequestOutstanding = status.isAARequestOutstanding();
+        int pending = (int)status.getPendingAntiAffineRequests();
+        aatext = buildAADetails(aaRequestOutstanding, pending);
+        if (SliderUtils.isSet(status.getLabelExpression())) {
+          aatext += " (label: " + status.getLabelExpression() + ")";
+        }
+        if (pending > 0 && !aaRequestOutstanding) {
+          aaRoleWithNoSuitableLocations ++;
+        } else if (aaRequestOutstanding) {
+          aaRoleWithOpenRequest++;
+        }
+      } else {
+        if (SliderUtils.isSet(status.getLabelExpression())) {
+          aatext = "label: " + status.getLabelExpression();
+        } else {
+          aatext = "";
+        }
+        if (status.getRequested() > 0) {
+          roleWithOpenRequest ++;
+        }
+      }
+      table.tr()
+        .td().a(nameUrl, roleName)._()
+        .td(String.format("%d", status.getDesired()))
+        .td(String.format("%d", status.getActual()))
+        .td(String.format("%d", status.getRequested()))
+        .td(String.format("%d", status.getFailed()))
+        .td(String.format("%d", status.getStartFailed()))
+        .td(aatext)
+        ._();
+    }
+
+    // empty row for some more spacing
+    table.tr()._();
+    // close table
+    table._();
+
+    containers._();
+    containers = null;
+
+    // some spacing
+    html.div()._();
+    html.div()._();
+
+    DIV<Hamlet> diagnostics = html.div("diagnostics");
+
+    List<String> statusEntries = new ArrayList<>(0);
+    if (roleWithOpenRequest > 0) {
+      statusEntries.add(String.format("%d %s with requests unsatisfiable by cluster",
+          roleWithOpenRequest, plural(roleWithOpenRequest, "component")));
+    }
+    if (aaRoleWithNoSuitableLocations > 0) {
+      statusEntries.add(String.format("%d anti-affinity %s no suitable nodes in the cluster",
+        aaRoleWithNoSuitableLocations,
+        plural(aaRoleWithNoSuitableLocations, "component has", "components have")));
+    }
+    if (aaRoleWithOpenRequest > 0) {
+      statusEntries.add(String.format("%d anti-affinity %s with requests unsatisfiable by cluster",
+        aaRoleWithOpenRequest,
+        plural(aaRoleWithOpenRequest, "component has", "components have")));
+
+    }
+    if (!statusEntries.isEmpty()) {
+      diagnostics.h3("Diagnostics");
+      Hamlet.TABLE<DIV<Hamlet>> diagnosticsTable = diagnostics.table();
+      for (String entry : statusEntries) {
+        diagnosticsTable.tr().td(entry)._();
+      }
+      diagnosticsTable._();
+    }
+    diagnostics._();
+
+    DIV<Hamlet> provider_info = html.div("provider_info");
+    provider_info.h3(providerName + " information");
+    UL<Hamlet> ul = html.ul();
+    addProviderServiceOptions(providerService, ul, clusterStatus);
+    ul._();
+    provider_info._();
+
+    DIV<Hamlet> exports = html.div("exports");
+    exports.h3("Exports");
+    ul = html.ul();
+    enumeratePublishedExports(appState.getPublishedExportsSet(), ul);
+    ul._();
+    exports._();
+  }
+
+  @VisibleForTesting
+  String buildAADetails(boolean outstanding, int pending) {
+    return String.format("Anti-affinity:%s %d pending %s",
+      (outstanding ? " 1 active request and" : ""),
+      pending, plural(pending, "request"));
+  }
+
+  private String plural(int n, String singular) {
+    return plural(n, singular, singular + "s");
+  }
+  private String plural(int n, String singular, String plural) {
+    return n == 1 ? singular : plural;
+  }
+
+  private void trb(Hamlet.TR tr,
+      String text) {
+    tr.td().b(text)._();
+  }
+
+  private String getProviderName() {
+    return providerService.getHumanName();
+  }
+
+  private String getInfoAvoidingNulls(String key) {
+    String createTime = appState.getClusterStatus().getInfo(key);
+
+    return null == createTime ? "N/A" : createTime;
+  }
+
+  protected void addProviderServiceOptions(ProviderService provider,
+      UL ul, ClusterDescription clusterStatus) {
+    Map<String, MonitorDetail> details = provider.buildMonitorDetails(
+        clusterStatus);
+    if (null == details) {
+      return;
+    }
+    // Loop over each entry, placing the text in the UL, adding an anchor when the URL is non-null/empty
+    for (Entry<String, MonitorDetail> entry : details.entrySet()) {
+      MonitorDetail detail = entry.getValue();
+      if (SliderUtils.isSet(detail.getValue()) ) {
+        LI item = ul.li();
+        item.span().$class("bold")._(entry.getKey())._();
+        item._(" - ");
+        if (detail.isUrl()) {
+          // Render an anchor if the value is a URL
+          item.a(detail.getValue(), detail.getValue())._();
+        } else {
+          item._(detail.getValue())._();
+        }
+      } else {
+        ul.li(entry.getKey());
+      }
+    }
+  }
+
+  protected void enumeratePublishedExports(PublishedExportsSet exports, UL<Hamlet> ul) {
+    for(String key : exports.keys()) {
+      PublishedExports export = exports.get(key);
+      LI<UL<Hamlet>> item = ul.li();
+      item.span().$class("bold")._(export.description)._();
+      UL sublist = item.ul();
+      for (Entry<String, List<ExportEntry>> entry : export.entries.entrySet()) {
+        LI sublistItem = sublist.li()._(entry.getKey());
+        for (ExportEntry exportEntry : entry.getValue()) {
+          sublistItem._(exportEntry.getValue());
+        }
+        sublistItem._();
+      }
+      sublist._();
+      item._();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/NavBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/NavBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/NavBlock.java
new file mode 100644
index 0000000..069d386
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/NavBlock.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.view;
+
+import com.google.inject.Inject;
+import org.apache.slider.server.appmaster.web.WebAppApi;
+
+import static org.apache.slider.server.appmaster.web.SliderAMWebApp.*;
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.*;
+
+/**
+ * 
+ */
+public class NavBlock extends SliderHamletBlock {
+
+  @Inject
+  public NavBlock(WebAppApi slider) {
+    super(slider);
+  }
+
+  @Override
+  protected void render(Block html) {
+    html.
+      div("#nav").
+        h3("Slider").
+        ul().
+          li().a(this.prefix(), "Overview")._().
+          li().a(relPath(CONTAINER_STATS), "Statistics")._().
+          li().a(relPath(CLUSTER_SPEC), "Specification")._().
+          li().a(rootPath(SYSTEM_METRICS_JSON), "Metrics")._().
+          li().a(rootPath(SYSTEM_HEALTHCHECK), "Health")._().
+          li().a(rootPath(SYSTEM_THREADS), "Threads")._().
+        _()
+    .h3("REST API"). 
+        ul().
+          li().a(apiPath(MODEL_DESIRED), "Specified")._().
+          li().a(apiPath(MODEL_RESOLVED), "Resolved")._().
+          li().a(apiPath(LIVE_RESOURCES), "Resources")._().
+          li().a(apiPath(LIVE_COMPONENTS), "Components")._().
+          li().a(apiPath(LIVE_CONTAINERS), "Containers")._().
+          li().a(apiPath(LIVE_NODES), "Nodes")._().
+          li().a(apiPath(LIVE_STATISTICS), "Statistics")._().
+          li().a(apiPath(LIVE_LIVENESS), "Liveness")._()
+        ._()
+      ._();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/SliderHamletBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/SliderHamletBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/SliderHamletBlock.java
new file mode 100644
index 0000000..82d7c8f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/view/SliderHamletBlock.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.view;
+
+import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
+import org.apache.slider.providers.ProviderService;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.apache.slider.server.appmaster.web.rest.RestPaths;
+
+import static org.apache.hadoop.yarn.util.StringHelper.ujoin;
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.SLIDER_PATH_APPLICATION;
+
+/**
+ * Anything we want to share across slider hamlet blocks
+ */
+public abstract class SliderHamletBlock extends HtmlBlock  {
+
+  protected final StateAccessForProviders appState;
+  protected final ProviderService providerService;
+  protected final RestPaths restPaths = new RestPaths();
+  
+  public SliderHamletBlock(WebAppApi slider) {
+    this.appState = slider.getAppState();
+    this.providerService = slider.getProviderService();
+  }
+
+  protected String rootPath(String absolutePath) {
+    return root_url(absolutePath);
+  }
+
+  protected String relPath(String... args) {
+    return ujoin(this.prefix(), args);
+  }
+
+  protected String apiPath(String api) {
+    return root_url(SLIDER_PATH_APPLICATION,  api);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/LoadedRoleHistory.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/LoadedRoleHistory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/LoadedRoleHistory.java
new file mode 100644
index 0000000..77408a5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/LoadedRoleHistory.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.avro;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.slider.common.tools.SliderUtils;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * The role history
+ */
+public class LoadedRoleHistory {
+
+  private RoleHistoryHeader header;
+
+  private Path path;
+
+  public final Map<String, Integer> roleMap = new HashMap<>();
+
+  public final List<NodeEntryRecord> records = new ArrayList<>();
+
+  /**
+   * Add a record
+   * @param record
+   */
+  public void add(NodeEntryRecord record) {
+    records.add(record);
+  }
+
+  /**
+   * Number of loaded records
+   * @return
+   */
+  public int size() {
+    return records.size();
+  }
+
+  public RoleHistoryHeader getHeader() {
+    return header;
+  }
+
+  public void setHeader(RoleHistoryHeader header) {
+    this.header = header;
+  }
+
+  public Path getPath() {
+    return path;
+  }
+
+  public void setPath(Path path) {
+    this.path = path;
+  }
+
+  public void buildMapping(Map<CharSequence, Integer> source) {
+    roleMap.clear();
+    for (Map.Entry<CharSequence, Integer> entry : source.entrySet()) {
+      roleMap.put(SliderUtils.sequenceToString(entry.getKey()),
+          entry.getValue());
+    }
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+      "LoadedRoleHistory{");
+    sb.append("path=").append(path);
+    sb.append("; number of roles=").append(roleMap.size());
+    sb.append("; size=").append(size());
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/NewerFilesFirst.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/NewerFilesFirst.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/NewerFilesFirst.java
new file mode 100644
index 0000000..2e049cb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/NewerFilesFirst.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.avro;
+
+import org.apache.hadoop.fs.Path;
+
+import java.io.Serializable;
+import java.util.Comparator;
+
+/**
+ * Compare two filenames by name; the more recent one comes first
+ */
+public class NewerFilesFirst implements Comparator<Path>, Serializable {
+
+  /**
+   * Takes the ordering of path names from the normal string comparison
+   * and negates it, so that names that come after other names in 
+   * the string sort come before here
+   * @param o1 leftmost 
+   * @param o2 rightmost
+   * @return positive if o1 &gt; o2 
+   */
+  @Override
+  public int compare(Path o1, Path o2) {
+    return (o2.getName().compareTo(o1.getName()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/OlderFilesFirst.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/OlderFilesFirst.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/OlderFilesFirst.java
new file mode 100644
index 0000000..407aaa6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/OlderFilesFirst.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.avro;
+
+import org.apache.hadoop.fs.Path;
+
+import java.io.Serializable;
+import java.util.Comparator;
+
+/**
+ * Compare two filenames by name; the older ones comes first
+ */
+public class OlderFilesFirst implements Comparator<Path>, Serializable {
+
+  /**
+   * Takes the ordering of path names from the normal string comparison
+   * and negates it, so that names that come after other names in 
+   * the string sort come before here
+   * @param o1 leftmost 
+   * @param o2 rightmost
+   * @return positive if o1 &gt; o2 
+   */
+  @Override
+  public int compare(Path o1, Path o2) {
+    return (o1.getName().compareTo(o2.getName()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/RoleHistoryWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/RoleHistoryWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/RoleHistoryWriter.java
new file mode 100644
index 0000000..49d8fb2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/avro/RoleHistoryWriter.java
@@ -0,0 +1,449 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.avro;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.avro.AvroTypeException;
+import org.apache.avro.Schema;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.io.Decoder;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.io.Encoder;
+import org.apache.avro.io.EncoderFactory;
+import org.apache.avro.specific.SpecificDatumReader;
+import org.apache.avro.specific.SpecificDatumWriter;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.GlobFilter;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.server.appmaster.state.NodeEntry;
+import org.apache.slider.server.appmaster.state.NodeInstance;
+import org.apache.slider.server.appmaster.state.RoleHistory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.EOFException;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Locale;
+import java.util.Map;
+
+/**
+ * Write out the role history to an output stream.
+ */
+public class RoleHistoryWriter {
+  protected static final Logger log =
+    LoggerFactory.getLogger(RoleHistoryWriter.class);
+
+  /**
+   * Although Avro is designed to handle some changes, we still keep a version
+   * marker in the file to catch changes that are fundamentally incompatible
+   * at the semantic level -changes that require either a different
+   * parser or get rejected outright.
+   */
+  public static final int ROLE_HISTORY_VERSION = 0x01;
+  
+  /**
+   * Write out the history.
+   * This does not update the history's dirty/savetime fields
+   *
+   * @param out outstream
+   * @param history history
+   * @param savetime time in millis for the save time to go in as a record
+   * @return no of records written
+   * @throws IOException IO failures
+   */
+  public long write(OutputStream out, RoleHistory history, long savetime)
+    throws IOException {
+    try {
+      DatumWriter<RoleHistoryRecord> writer =
+        new SpecificDatumWriter<>(RoleHistoryRecord.class);
+
+      RoleHistoryRecord record = createHeaderRecord(savetime, history);
+      int roles = history.getRoleSize();
+      Schema schema = record.getSchema();
+      Encoder encoder = EncoderFactory.get().jsonEncoder(schema, out);
+      writer.write(record, encoder);
+      // now write the rolemap record
+      writer.write(createRolemapRecord(history), encoder);
+      long count = 0;
+      //now for every role history entry, write out its record
+      Collection<NodeInstance> instances = history.cloneNodemap().values();
+      for (NodeInstance instance : instances) {
+        for (int role = 0; role < roles; role++) {
+          NodeEntry nodeEntry = instance.get(role);
+
+          if (nodeEntry != null) {
+            NodeEntryRecord ner = build(nodeEntry, role, instance.hostname);
+            record = new RoleHistoryRecord(ner);
+            writer.write(record, encoder);
+            count++;
+          }
+        }
+      }
+      // footer
+      RoleHistoryFooter footer = new RoleHistoryFooter();
+      footer.setCount(count);
+      writer.write(new RoleHistoryRecord(footer), encoder);
+      encoder.flush();
+      return count;
+    } finally {
+      out.close();
+    }
+  }
+
+  /**
+   * Create the header record
+   * @param savetime time of save
+   * @param history history
+   * @return a record to place at the head of the file
+   */
+  private RoleHistoryRecord createHeaderRecord(long savetime, RoleHistory history) {
+    RoleHistoryHeader header = new RoleHistoryHeader();
+    header.setVersion(ROLE_HISTORY_VERSION);
+    header.setSaved(savetime);
+    header.setSavedx(Long.toHexString(savetime));
+    header.setSavedate(SliderUtils.toGMTString(savetime));
+    header.setRoles(history.getRoleSize());
+    return new RoleHistoryRecord(header);
+  }
+
+  /**
+   * Create the rolemap record
+   * @param history history
+   * @return a record to insert into the file
+   */
+  private RoleHistoryRecord createRolemapRecord(RoleHistory history) {
+    RoleHistoryMapping entry = new RoleHistoryMapping();
+    Map<CharSequence, Integer> mapping = history.buildMappingForHistoryFile();
+    entry.setRolemap(mapping);
+    return new RoleHistoryRecord(entry);
+  }
+
+  /**
+   * Write the history information to a file
+   *
+   * @param fs filesystem
+   * @param path path
+   * @param overwrite overwrite flag
+   * @param history history
+   * @param savetime time in millis for the save time to go in as a record
+   * @return no of records written
+   * @throws IOException IO failures
+   */
+  public long write(FileSystem fs,
+      Path path,
+      boolean overwrite,
+      RoleHistory history,
+      long savetime)
+      throws IOException {
+    FSDataOutputStream out = fs.create(path, overwrite);
+    return write(out, history, savetime);
+  }
+
+
+  /**
+   * Create the filename for a history file
+   * @param time time value
+   * @return a filename such that later filenames sort later in the directory
+   */
+  public Path createHistoryFilename(Path historyPath, long time) {
+    String filename = String.format(Locale.ENGLISH,
+                                    SliderKeys.HISTORY_FILENAME_CREATION_PATTERN,
+                                    time);
+    Path path = new Path(historyPath, filename);
+    return path;
+  }
+
+  /**
+   * Build a {@link NodeEntryRecord} from a node entry; include whether
+   * the node is in use and when it was last used.
+   * @param entry entry count
+   * @param role role index
+   * @param hostname name
+   * @return the record
+   */
+  private NodeEntryRecord build(NodeEntry entry, int role, String hostname) {
+    NodeEntryRecord record = new NodeEntryRecord(
+      hostname, role, entry.getLive() > 0, entry.getLastUsed()
+    );
+    return record;
+  }
+
+  /**
+   * Read a history, returning one that is ready to have its onThaw() 
+   * method called
+   * @param in input source
+   * @return no. of entries read
+   * @throws IOException problems
+   */
+  public LoadedRoleHistory read(InputStream in) throws
+                                                       IOException,
+                                                       BadConfigException {
+    try {
+      LoadedRoleHistory loadedRoleHistory = new LoadedRoleHistory();
+      DatumReader<RoleHistoryRecord> reader =
+        new SpecificDatumReader<>(RoleHistoryRecord.class);
+      Decoder decoder =
+        DecoderFactory.get().jsonDecoder(RoleHistoryRecord.getClassSchema(),
+            in);
+
+      //read header : no entry -> EOF
+      RoleHistoryRecord record = reader.read(null, decoder);
+      if (record == null) {
+        throw new IOException("Role History Header not found at start of file.");
+      }
+      Object entry = record.getEntry();
+      if (!(entry instanceof RoleHistoryHeader)) {
+        throw new IOException("Role History Header not found at start of file");
+      }
+      RoleHistoryHeader header = (RoleHistoryHeader) entry;
+      if (header.getVersion() != ROLE_HISTORY_VERSION) {
+        throw new IOException(
+          String.format("Can't read role file version %04x -need %04x",
+          header.getVersion(),
+          ROLE_HISTORY_VERSION));
+      }
+      loadedRoleHistory.setHeader(header);
+      RoleHistoryFooter footer = null;
+      int records = 0;
+      //go through reading data
+      try {
+        while (footer == null) {
+          record = reader.read(null, decoder);
+          if (record == null) {
+            throw new IOException("Null record after " + records + " records");
+          }
+          entry = record.getEntry();
+
+          if (entry instanceof RoleHistoryHeader) {
+            throw new IOException("Duplicate Role History Header found");
+          } else if (entry instanceof RoleHistoryMapping) {
+            // role history mapping entry
+            if (!loadedRoleHistory.roleMap.isEmpty()) {
+              // duplicate role maps are viewed as something to warn over, rather than fail
+              log.warn("Duplicate role map; ignoring");
+            } else {
+              RoleHistoryMapping historyMapping = (RoleHistoryMapping) entry;
+              loadedRoleHistory.buildMapping(historyMapping.getRolemap());
+            }
+          } else if (entry instanceof NodeEntryRecord) {
+            // normal record
+            records++;
+            NodeEntryRecord nodeEntryRecord = (NodeEntryRecord) entry;
+            loadedRoleHistory.add(nodeEntryRecord);
+          } else if (entry instanceof RoleHistoryFooter) {
+            //tail end of the file
+            footer = (RoleHistoryFooter) entry;
+          } else {
+            // this is to handle future versions, such as when rolling back
+            // from a later version of slider
+            log.warn("Discarding unknown record {}", entry);
+          }
+        }
+      } catch (EOFException e) {
+        EOFException ex = new EOFException(
+          "End of file reached after " + records + " records");
+        ex.initCause(e);
+        throw ex;
+      }
+      // at this point there should be no data left.
+      // check by reading and expecting a -1
+      if (in.read() > 0) {
+        // footer is in stream before the last record
+        throw new EOFException(
+          "File footer reached before end of file -after " + records +
+          " records");
+      }
+      if (records != footer.getCount()) {
+        log.warn("mismatch between no of records saved {} and number read {}",
+                 footer.getCount(), records);
+      }
+      return loadedRoleHistory;
+    } finally {
+      in.close();
+    }
+
+  }
+
+  /**
+   * Read a role history from a path in a filesystem
+   * @param fs filesystem
+   * @param path path to the file
+   * @return the records read
+   * @throws IOException any problem
+   */
+  public LoadedRoleHistory read(FileSystem fs, Path path)
+      throws IOException, BadConfigException {
+    FSDataInputStream instream = fs.open(path);
+    return read(instream);
+  }
+
+  /**
+   * Read from a resource in the classpath -used for testing
+   * @param resource resource
+   * @return the records read
+   * @throws IOException any problem
+   */
+  public LoadedRoleHistory read(String resource)
+      throws IOException, BadConfigException {
+
+    return read(this.getClass().getClassLoader().getResourceAsStream(resource));
+  }
+
+
+  /**
+   * Find all history entries in a dir. The dir is created if it is
+   * not already defined.
+   * 
+   * The scan uses the match pattern {@link SliderKeys#HISTORY_FILENAME_MATCH_PATTERN}
+   * while dropping empty files and directories which match the pattern.
+   * The list is then sorted with a comparator that sorts on filename,
+   * relying on the filename of newer created files being later than the old ones.
+   * 
+   * 
+   *
+   * @param fs filesystem
+   * @param dir dir to scan
+   * @param includeEmptyFiles should empty files be included in the result?
+   * @return a possibly empty list
+   * @throws IOException IO problems
+   * @throws FileNotFoundException if the target dir is actually a path
+   */
+  public List<Path> findAllHistoryEntries(FileSystem fs,
+                                          Path dir,
+                                          boolean includeEmptyFiles) throws IOException {
+    assert fs != null;
+    assert dir != null;
+    if (!fs.exists(dir)) {
+      fs.mkdirs(dir);
+    } else if (!fs.isDirectory(dir)) {
+      throw new FileNotFoundException("Not a directory " + dir.toString());
+    }
+    
+    PathFilter filter = new GlobFilter(SliderKeys.HISTORY_FILENAME_GLOB_PATTERN);
+    FileStatus[] stats = fs.listStatus(dir, filter);
+    List<Path> paths = new ArrayList<Path>(stats.length);
+    for (FileStatus stat : stats) {
+      log.debug("Possible entry: {}", stat.toString());
+      if (stat.isFile() && (includeEmptyFiles || stat.getLen() > 0)) {
+        paths.add(stat.getPath());
+      }
+    }
+    sortHistoryPaths(paths);
+    return paths;
+  }
+
+  @VisibleForTesting
+  public static void sortHistoryPaths(List<Path> paths) {
+    Collections.sort(paths, new NewerFilesFirst());
+  }
+  
+  /**
+   * Iterate through the paths until one can be loaded
+   * @param paths paths to load
+   * @return the loaded history including the path -or null if all failed to load
+   */
+  public LoadedRoleHistory attemptToReadHistory(FileSystem fileSystem,
+      List<Path> paths)
+      throws BadConfigException {
+    ListIterator<Path> pathIterator = paths.listIterator();
+    boolean success = false;
+    LoadedRoleHistory history = null;
+    while (!success && pathIterator.hasNext()) {
+      Path path = pathIterator.next();
+      try {
+        history = read(fileSystem, path);
+        //success
+        success = true;
+        history.setPath(path);
+      } catch (IOException e) {
+        log.info("Failed to read {}", path, e);
+      } catch (AvroTypeException e) {
+        log.warn("Failed to parse {}", path, e);
+      } catch (Exception e) {
+        // low level event logged @ warn level
+        log.warn("Exception while reading {}", path, e);
+      }
+    }
+    return history;
+  }
+
+  /**
+   * Try to load the history from a directory -a failure to load a specific
+   * file is downgraded to a log and the next older path attempted instead
+   * @param fs filesystem
+   * @param dir dir to load from
+   * @return the history loaded, including the path
+   * @throws IOException if indexing the history directory fails. 
+   */
+  public LoadedRoleHistory loadFromHistoryDir(FileSystem fs, Path dir)
+      throws IOException, BadConfigException {
+    assert fs != null: "null filesystem";
+    List<Path> entries = findAllHistoryEntries(fs, dir, false);
+    return attemptToReadHistory(fs, entries);
+  }
+
+  /**
+   * Delete all old history entries older than the one we want to keep. This
+   * uses the filename ordering to determine age, not timestamps
+   * @param fileSystem filesystem
+   * @param keep path to keep -used in thresholding the files
+   * @return the number of files deleted
+   * @throws FileNotFoundException if the path to keep is not present (safety
+   * check to stop the entire dir being purged)
+   * @throws IOException IO problems
+   */
+  public int purgeOlderHistoryEntries(FileSystem fileSystem, Path keep)
+      throws IOException { assert fileSystem != null : "null filesystem";
+    if (!fileSystem.exists(keep)) {
+      throw new FileNotFoundException(keep.toString());
+    }
+    Path dir = keep.getParent();
+    log.debug("Purging entries in {} up to {}", dir, keep);
+    List<Path> paths = findAllHistoryEntries(fileSystem, dir, true);
+    Collections.sort(paths, new OlderFilesFirst());
+    int deleteCount = 0;
+    for (Path path : paths) {
+      if (path.equals(keep)) {
+        break;
+      } else {
+        log.debug("Deleting {}", path);
+        deleteCount++;
+        fileSystem.delete(path, false);
+      }
+    }
+    return deleteCount;
+  }
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[12/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/RpcBinder.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/RpcBinder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/RpcBinder.java
new file mode 100644
index 0000000..dd4785d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/RpcBinder.java
@@ -0,0 +1,310 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.rpc;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.BlockingService;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtocolProxy;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcEngine;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.SecretManager;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.slider.api.SliderClusterProtocol;
+import org.apache.slider.common.SliderExitCodes;
+import org.apache.slider.common.tools.Duration;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.exceptions.BadClusterStateException;
+import org.apache.slider.core.exceptions.ErrorStrings;
+import org.apache.slider.core.exceptions.ServiceNotReadyException;
+import org.apache.slider.core.exceptions.SliderException;
+
+import static org.apache.slider.common.SliderXmlConfKeys.*; 
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.security.PrivilegedExceptionAction;
+
+public class RpcBinder {
+  protected static final Logger log =
+    LoggerFactory.getLogger(RpcBinder.class);
+
+  /**
+   * Create a protobuf server bonded to the specific socket address
+   * @param addr address to listen to; 0.0.0.0 as hostname acceptable
+   * @param conf config
+   * @param secretManager token secret handler
+   * @param numHandlers threads to service requests
+   * @param blockingService service to handle
+   * @param portRangeConfig range of ports
+   * @return the IPC server itself
+   * @throws IOException
+   */
+  public static Server createProtobufServer(InetSocketAddress addr,
+                                            Configuration conf,
+                                            SecretManager<? extends TokenIdentifier> secretManager,
+                                            int numHandlers,
+                                            BlockingService blockingService,
+                                            String portRangeConfig) throws
+                                                      IOException {
+    Class<SliderClusterProtocolPB> sliderClusterAPIClass = registerSliderAPI(
+        conf);
+    RPC.Server server = new RPC.Builder(conf).setProtocol(sliderClusterAPIClass)
+                                             .setInstance(blockingService)
+                                             .setBindAddress(addr.getAddress()
+                                                 .getCanonicalHostName())
+                                             .setPort(addr.getPort())
+                                             .setNumHandlers(numHandlers)
+                                             .setVerbose(false)
+                                             .setSecretManager(secretManager)
+                                             .setPortRangeConfig(
+                                               portRangeConfig)
+                                             .build();
+    log.debug(
+      "Adding protocol " + sliderClusterAPIClass.getCanonicalName() + " to the server");
+    server.addProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER, sliderClusterAPIClass,
+                       blockingService);
+    return server;
+  }
+
+  /**
+   * Add the protobuf engine to the configuration. Harmless and inexpensive
+   * if repeated.
+   * @param conf configuration to patch
+   * @return the protocol class
+   */
+  public static Class<SliderClusterProtocolPB> registerSliderAPI(
+      Configuration conf) {
+    Class<SliderClusterProtocolPB> sliderClusterAPIClass =
+      SliderClusterProtocolPB.class;
+    RPC.setProtocolEngine(conf, sliderClusterAPIClass, ProtobufRpcEngine.class);
+    
+    //quick sanity check here
+    assert verifyBondedToProtobuf(conf, sliderClusterAPIClass);
+    
+    return sliderClusterAPIClass;
+  }
+
+  /**
+   * Verify that the conf is set up for protobuf transport of Slider RPC
+   * @param conf configuration
+   * @param sliderClusterAPIClass class for the API
+   * @return true if the RPC engine is protocol buffers
+   */
+  public static boolean verifyBondedToProtobuf(Configuration conf,
+                                                Class<SliderClusterProtocolPB> sliderClusterAPIClass) {
+    return conf.getClass("rpc.engine." + sliderClusterAPIClass.getName(),
+                         RpcEngine.class) .equals(ProtobufRpcEngine.class);
+  }
+
+
+  /**
+   * Connect to a server. May include setting up retry policies
+   * @param addr
+   * @param currentUser
+   * @param conf
+   * @param rpcTimeout
+   * @return
+   * @throws IOException
+   */
+  public static SliderClusterProtocol connectToServer(InetSocketAddress addr,
+                                                    UserGroupInformation currentUser,
+                                                    Configuration conf,
+                                                    int rpcTimeout) throws IOException {
+    Class<SliderClusterProtocolPB> sliderClusterAPIClass =
+        registerSliderAPI(conf);
+
+    final RetryPolicy retryPolicy = RetryPolicies.TRY_ONCE_THEN_FAIL;
+    log.debug("Connecting to Slider AM at {}", addr);
+    ProtocolProxy<SliderClusterProtocolPB> protoProxy =
+        RPC.getProtocolProxy(sliderClusterAPIClass,
+            1,
+            addr,
+            currentUser,
+            conf,
+            NetUtils.getDefaultSocketFactory(conf),
+            rpcTimeout,
+            retryPolicy);
+    SliderClusterProtocolPB endpoint = protoProxy.getProxy();
+    return new SliderClusterProtocolProxy(endpoint, addr);
+  }
+
+
+  /**
+   * This loops for a limited period trying to get the Proxy -
+   * by doing so it handles AM failover
+   * @param conf configuration to patch and use
+   * @param rmClient client of the resource manager
+   * @param application application to work with
+   * @param connectTimeout timeout for the whole proxy operation to timeout
+   * (milliseconds). Use 0 to indicate "do not attempt to wait" -fail fast.
+   * @param rpcTimeout timeout for RPCs to block during communications
+   * @return the proxy
+   * @throws IOException IO problems
+   * @throws YarnException Slider-generated exceptions related to the binding
+   * failing. This can include the application finishing or timeouts
+   * @throws InterruptedException if a sleep operation waiting for
+   * the cluster to respond is interrupted.
+   */
+  @SuppressWarnings("NestedAssignment")
+  public static SliderClusterProtocol getProxy(final Configuration conf,
+                                      final ApplicationClientProtocol rmClient,
+                                      ApplicationReport application,
+                                      final int connectTimeout,
+                                      final int rpcTimeout)
+      throws IOException, YarnException, InterruptedException {
+    ApplicationId appId;
+    appId = application.getApplicationId();
+    Duration timeout = new Duration(connectTimeout);
+    timeout.start();
+    Exception exception = null;
+    YarnApplicationState state = null;
+    try {
+      while (application != null &&
+             (state = application.getYarnApplicationState()).equals(
+                 YarnApplicationState.RUNNING)) {
+  
+        try {
+          return getProxy(conf, application, rpcTimeout);
+        } catch (IOException e) {
+          if (connectTimeout <= 0 || timeout.getLimitExceeded()) {
+            throw e;
+          }
+          exception = e;
+        } catch (YarnException e) {
+          if (connectTimeout <= 0 || timeout.getLimitExceeded()) {
+            throw e;
+          }
+          exception = e;
+        }
+        //at this point: app failed to work
+        log.debug("Could not connect to {}. Waiting for getting the latest AM address...",
+                  appId);
+        Thread.sleep(1000);
+        //or get the app report
+        application =
+          rmClient.getApplicationReport(
+              GetApplicationReportRequest.newInstance(appId)).getApplicationReport();
+      }
+      //get here if the app is no longer running. Raise a specific
+      //exception but init it with the previous failure
+      throw new BadClusterStateException(
+                              exception,
+                              ErrorStrings.E_FINISHED_APPLICATION, appId, state );
+    } finally {
+      timeout.close();
+    }
+  }
+
+  /**
+   * Get a proxy from the application report
+   * @param conf config to use
+   * @param application app report
+   * @param rpcTimeout timeout in RPC operations
+   * @return the proxy
+   * @throws IOException
+   * @throws SliderException
+   * @throws InterruptedException
+   */
+  public static SliderClusterProtocol getProxy(final Configuration conf,
+      final ApplicationReport application,
+      final int rpcTimeout)
+      throws IOException, SliderException, InterruptedException {
+
+    String host = application.getHost();
+    int port = application.getRpcPort();
+    org.apache.hadoop.yarn.api.records.Token clientToAMToken =
+        application.getClientToAMToken();
+    return createProxy(conf, host, port, clientToAMToken, rpcTimeout);
+  }
+
+  /**
+   *
+   * @param conf config to use
+   * @param host hosname
+   * @param port port
+   * @param clientToAMToken auth token: only used in a secure cluster.
+   * converted via {@link ConverterUtils#convertFromYarn(org.apache.hadoop.yarn.api.records.Token, InetSocketAddress)}
+   * @param rpcTimeout timeout in RPC operations
+   * @return the proxy
+   * @throws SliderException
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public static SliderClusterProtocol createProxy(final Configuration conf,
+      String host,
+      int port,
+      org.apache.hadoop.yarn.api.records.Token clientToAMToken,
+      final int rpcTimeout) throws
+      SliderException,
+      IOException,
+      InterruptedException {
+    String address = host + ":" + port;
+    if (SliderUtils.isUnset(host) || 0 == port) {
+      throw new SliderException(SliderExitCodes.EXIT_CONNECTIVITY_PROBLEM,
+                              "Slider instance "
+                              + " isn't providing a valid address for the" +
+                              " Slider RPC protocol: " + address);
+    }
+
+    UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
+    final UserGroupInformation newUgi = UserGroupInformation.createRemoteUser(
+      currentUser.getUserName());
+    final InetSocketAddress serviceAddr =
+        NetUtils.createSocketAddrForHost(host, port);
+    SliderClusterProtocol realProxy;
+
+    log.debug("Connecting to {}", serviceAddr);
+    if (UserGroupInformation.isSecurityEnabled()) {
+      Preconditions.checkArgument(clientToAMToken != null,
+          "Null clientToAMToken");
+      Token<ClientToAMTokenIdentifier> token =
+        ConverterUtils.convertFromYarn(clientToAMToken, serviceAddr);
+      newUgi.addToken(token);
+      realProxy =
+        newUgi.doAs(new PrivilegedExceptionAction<SliderClusterProtocol>() {
+          @Override
+          public SliderClusterProtocol run() throws IOException {
+            return connectToServer(serviceAddr, newUgi, conf, rpcTimeout);
+          }
+        });
+    } else {
+      realProxy = connectToServer(serviceAddr, newUgi, conf, rpcTimeout);
+    }
+    return realProxy;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderAMPolicyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderAMPolicyProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderAMPolicyProvider.java
new file mode 100644
index 0000000..a40078a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderAMPolicyProvider.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.rpc;
+
+import org.apache.hadoop.security.authorize.PolicyProvider;
+import org.apache.hadoop.security.authorize.Service;
+import org.apache.slider.common.SliderXmlConfKeys;
+
+/**
+ * {@link PolicyProvider} for Slider protocols.
+ */
+
+public class SliderAMPolicyProvider extends PolicyProvider {
+  
+  private static final Service[] services = 
+      new Service[] {
+    new Service(SliderXmlConfKeys.KEY_PROTOCOL_ACL, SliderClusterProtocolPB.class)
+  };
+
+  @SuppressWarnings("ReturnOfCollectionOrArrayField")
+  @Override
+  public Service[] getServices() {
+    return services;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderClusterProtocolPB.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderClusterProtocolPB.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderClusterProtocolPB.java
new file mode 100644
index 0000000..7d237de
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderClusterProtocolPB.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.rpc;
+
+import org.apache.slider.api.SliderClusterProtocol;
+import org.apache.slider.api.proto.SliderClusterAPI;
+
+public interface SliderClusterProtocolPB extends SliderClusterAPI.SliderClusterProtocolPB.BlockingInterface{
+
+  long versionID = SliderClusterProtocol.versionID;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderClusterProtocolPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderClusterProtocolPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderClusterProtocolPBImpl.java
new file mode 100644
index 0000000..f0d9063
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderClusterProtocolPBImpl.java
@@ -0,0 +1,318 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.rpc;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+import org.apache.slider.api.SliderClusterProtocol;
+import org.apache.slider.api.proto.Messages;
+
+import java.io.IOException;
+
+/**
+ * Server-side Relay from Protobuf to internal RPC.
+ *
+ */
+public class SliderClusterProtocolPBImpl implements SliderClusterProtocolPB {
+
+  private SliderClusterProtocol real;
+
+  public SliderClusterProtocolPBImpl(SliderClusterProtocol real) {
+    this.real = real;
+  }
+
+  private ServiceException wrap(Exception e) {
+    if (e instanceof ServiceException) {
+      return (ServiceException) e;
+    }
+    return new ServiceException(e);
+  }
+
+  public long getProtocolVersion(String protocol, long clientVersion)
+      throws IOException {
+    return SliderClusterProtocol.versionID;
+  }
+
+  @Override
+  public Messages.StopClusterResponseProto stopCluster(RpcController controller,
+       Messages.StopClusterRequestProto request) throws ServiceException {
+    try {
+      return real.stopCluster(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+
+  @Override
+  public Messages.UpgradeContainersResponseProto upgradeContainers(RpcController controller,
+        Messages.UpgradeContainersRequestProto request) throws ServiceException {
+    try {
+      return real.upgradeContainers(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+
+  @Override
+  public Messages.FlexClusterResponseProto flexCluster(RpcController controller,
+      Messages.FlexClusterRequestProto request) throws ServiceException {
+    try {
+      return real.flexCluster(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+
+  @Override
+  public Messages.GetJSONClusterStatusResponseProto getJSONClusterStatus(
+    RpcController controller,
+    Messages.GetJSONClusterStatusRequestProto request) throws ServiceException {
+    try {
+      return real.getJSONClusterStatus(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+
+
+  @Override
+  public Messages.GetInstanceDefinitionResponseProto getInstanceDefinition(
+    RpcController controller,
+    Messages.GetInstanceDefinitionRequestProto request)
+      throws ServiceException {
+    try {
+      return real.getInstanceDefinition(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+
+  @Override
+  public Messages.ListNodeUUIDsByRoleResponseProto listNodeUUIDsByRole(
+    RpcController controller,
+    Messages.ListNodeUUIDsByRoleRequestProto request) throws ServiceException {
+    try {
+      return real.listNodeUUIDsByRole(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+
+  @Override
+  public Messages.GetNodeResponseProto getNode(RpcController controller,
+      Messages.GetNodeRequestProto request) throws ServiceException {
+    try {
+      return real.getNode(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+
+  @Override
+  public Messages.GetClusterNodesResponseProto getClusterNodes(RpcController controller,
+      Messages.GetClusterNodesRequestProto request) throws ServiceException {
+    try {
+      return real.getClusterNodes(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+
+  @Override
+  public Messages.EchoResponseProto echo(RpcController controller,
+      Messages.EchoRequestProto request) throws ServiceException {
+    try {
+      return real.echo(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+
+  @Override
+  public Messages.KillContainerResponseProto killContainer(RpcController controller,
+      Messages.KillContainerRequestProto request) throws ServiceException {
+    try {
+      return real.killContainer(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+  
+  @Override
+  public Messages.AMSuicideResponseProto amSuicide(RpcController controller,
+      Messages.AMSuicideRequestProto request) throws ServiceException {
+    try {
+      return real.amSuicide(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+
+  @Override
+  public Messages.ApplicationLivenessInformationProto getLivenessInformation(
+      RpcController controller,
+      Messages.GetApplicationLivenessRequestProto request) throws ServiceException {
+    try {
+      return real.getLivenessInformation(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+  
+  @Override
+  public Messages.GetLiveContainersResponseProto getLiveContainers(RpcController controller,
+      Messages.GetLiveContainersRequestProto request) throws ServiceException {
+    try {
+      return real.getLiveContainers(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+
+  @Override
+  public Messages.ContainerInformationProto getLiveContainer(RpcController controller,
+      Messages.GetLiveContainerRequestProto request) throws ServiceException {
+    try {
+      return real.getLiveContainer(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+
+  @Override
+  public Messages.GetLiveComponentsResponseProto getLiveComponents(RpcController controller,
+      Messages.GetLiveComponentsRequestProto request) throws ServiceException {
+    try {
+      return real.getLiveComponents(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+
+  @Override
+  public Messages.ComponentInformationProto getLiveComponent(RpcController controller,
+      Messages.GetLiveComponentRequestProto request) throws ServiceException {
+    try {
+      return real.getLiveComponent(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+
+  @Override
+  public Messages.GetLiveNodesResponseProto getLiveNodes(RpcController controller,
+      Messages.GetLiveNodesRequestProto request) throws ServiceException {
+    try {
+      return real.getLiveNodes(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+
+  @Override
+  public Messages.NodeInformationProto getLiveNode(RpcController controller,
+      Messages.GetLiveNodeRequestProto request) throws ServiceException {
+    try {
+      return real.getLiveNode(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+
+  @Override
+  public Messages.WrappedJsonProto getModelDesired(RpcController controller,
+      Messages.EmptyPayloadProto request) throws ServiceException {
+    try {
+      return real.getModelDesired(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+
+  @Override
+  public Messages.WrappedJsonProto getModelDesiredAppconf(RpcController controller,
+      Messages.EmptyPayloadProto request) throws ServiceException {
+    try {
+      return real.getModelDesiredAppconf(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }  }
+
+  @Override
+  public Messages.WrappedJsonProto getModelDesiredResources(RpcController controller,
+      Messages.EmptyPayloadProto request) throws ServiceException {
+    try {
+      return real.getModelDesiredResources(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+
+  @Override
+  public Messages.WrappedJsonProto getModelResolved(RpcController controller,
+      Messages.EmptyPayloadProto request) throws ServiceException {
+    try {
+      return real.getModelResolved(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+
+  @Override
+  public Messages.WrappedJsonProto getModelResolvedAppconf(RpcController controller,
+      Messages.EmptyPayloadProto request) throws ServiceException {
+    try {
+      return real.getModelResolvedAppconf(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+
+  @Override
+  public Messages.WrappedJsonProto getModelResolvedResources(RpcController controller,
+      Messages.EmptyPayloadProto request) throws ServiceException {
+    try {
+      return real.getModelResolvedResources(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+
+  @Override
+  public Messages.WrappedJsonProto getLiveResources(RpcController controller,
+      Messages.EmptyPayloadProto request) throws ServiceException {
+    try {
+      return real.getLiveResources(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+
+  @Override
+  public Messages.GetCertificateStoreResponseProto getClientCertificateStore(
+      RpcController controller,
+      Messages.GetCertificateStoreRequestProto request)
+      throws ServiceException {
+    try {
+      return real.getClientCertificateStore(request);
+    } catch (Exception e) {
+      throw wrap(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderClusterProtocolProxy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderClusterProtocolProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderClusterProtocolProxy.java
new file mode 100644
index 0000000..b230816
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderClusterProtocolProxy.java
@@ -0,0 +1,358 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.rpc;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.ipc.ProtobufHelper;
+import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.slider.api.SliderClusterProtocol;
+import org.apache.slider.api.proto.Messages;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+public class SliderClusterProtocolProxy implements SliderClusterProtocol {
+
+  private static final RpcController NULL_CONTROLLER = null;
+  private final SliderClusterProtocolPB endpoint;
+  private final InetSocketAddress address;
+
+  public SliderClusterProtocolProxy(SliderClusterProtocolPB endpoint,
+      InetSocketAddress address) {
+    Preconditions.checkArgument(endpoint != null, "null endpoint");
+    Preconditions.checkNotNull(address != null, "null address");
+    this.endpoint = endpoint;
+    this.address = address;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder("SliderClusterProtocolProxy{");
+    sb.append("address=").append(address);
+    sb.append('}');
+    return sb.toString();
+  }
+
+  @Override
+  public ProtocolSignature getProtocolSignature(String protocol,
+      long clientVersion,
+      int clientMethodsHash)
+      throws IOException {
+    if (!protocol.equals(RPC.getProtocolName(SliderClusterProtocolPB.class))) {
+      throw new IOException("Serverside implements " +
+                            RPC.getProtocolName(SliderClusterProtocolPB.class) +
+                            ". The following requested protocol is unknown: " +
+                            protocol);
+    }
+
+    return ProtocolSignature.getProtocolSignature(clientMethodsHash,
+        RPC.getProtocolVersion(
+            SliderClusterProtocol.class),
+        SliderClusterProtocol.class);
+  }
+
+  @Override
+  public long getProtocolVersion(String protocol, long clientVersion)
+      throws IOException {
+    return SliderClusterProtocol.versionID;
+  }
+  
+  private IOException convert(ServiceException se) {
+    IOException ioe = ProtobufHelper.getRemoteException(se);
+    if (ioe instanceof RemoteException) {
+      RemoteException remoteException = (RemoteException) ioe;
+      return remoteException.unwrapRemoteException();
+    }
+    return ioe;
+  }
+  
+  @Override
+  public Messages.StopClusterResponseProto stopCluster(Messages.StopClusterRequestProto request) throws
+                                                                                                 IOException,
+                                                                                                 YarnException {
+    try {
+      return endpoint.stopCluster(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public Messages.UpgradeContainersResponseProto upgradeContainers(
+      Messages.UpgradeContainersRequestProto request) throws IOException,
+      YarnException {
+    try {
+      return endpoint.upgradeContainers(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public Messages.FlexClusterResponseProto flexCluster(Messages.FlexClusterRequestProto request)
+      throws IOException {
+    try {
+      return endpoint.flexCluster(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public Messages.GetJSONClusterStatusResponseProto getJSONClusterStatus(
+    Messages.GetJSONClusterStatusRequestProto request) throws
+                                                       IOException,
+                                                       YarnException {
+    try {
+      return endpoint.getJSONClusterStatus(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+
+  @Override
+  public Messages.GetInstanceDefinitionResponseProto getInstanceDefinition(
+    Messages.GetInstanceDefinitionRequestProto request) throws
+                                                        IOException,
+                                                        YarnException {
+    try {
+      return endpoint.getInstanceDefinition(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public Messages.ListNodeUUIDsByRoleResponseProto listNodeUUIDsByRole(Messages.ListNodeUUIDsByRoleRequestProto request) throws
+                                                                                                                         IOException,
+                                                                                                                         YarnException {
+    try {
+      return endpoint.listNodeUUIDsByRole(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public Messages.GetNodeResponseProto getNode(Messages.GetNodeRequestProto request) throws
+                                                                                     IOException,
+                                                                                     YarnException {
+    try {
+      return endpoint.getNode(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public Messages.GetClusterNodesResponseProto getClusterNodes(Messages.GetClusterNodesRequestProto request) throws
+                                                                                                             IOException,
+                                                                                                             YarnException {
+    try {
+      return endpoint.getClusterNodes(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+
+  @Override
+  public Messages.EchoResponseProto echo(Messages.EchoRequestProto request) throws
+                                                                                                             IOException,
+                                                                                                             YarnException {
+    try {
+      return endpoint.echo(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+
+  @Override
+  public Messages.KillContainerResponseProto killContainer(Messages.KillContainerRequestProto request) throws
+                                                                                                             IOException,
+                                                                                                             YarnException {
+    try {
+      return endpoint.killContainer(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public Messages.AMSuicideResponseProto amSuicide(Messages.AMSuicideRequestProto request) throws
+                                                                                           IOException {
+    try {
+      return endpoint.amSuicide(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public Messages.ApplicationLivenessInformationProto getLivenessInformation(
+      Messages.GetApplicationLivenessRequestProto request) throws IOException {
+    try {
+      return endpoint.getLivenessInformation(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public Messages.GetLiveContainersResponseProto getLiveContainers(Messages.GetLiveContainersRequestProto request) throws
+      IOException {
+    try {
+      return endpoint.getLiveContainers(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public Messages.ContainerInformationProto getLiveContainer(Messages.GetLiveContainerRequestProto request) throws
+      IOException {
+    try {
+      return endpoint.getLiveContainer(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public Messages.GetLiveComponentsResponseProto getLiveComponents(Messages.GetLiveComponentsRequestProto request) throws
+      IOException {
+    try {
+      return endpoint.getLiveComponents(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public Messages.ComponentInformationProto getLiveComponent(Messages.GetLiveComponentRequestProto request) throws
+      IOException {
+    try {
+      return endpoint.getLiveComponent(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public Messages.GetLiveNodesResponseProto getLiveNodes(Messages.GetLiveNodesRequestProto request)
+      throws IOException {
+    try {
+      return endpoint.getLiveNodes(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public Messages.NodeInformationProto getLiveNode(Messages.GetLiveNodeRequestProto request)
+      throws IOException {
+    try {
+      return endpoint.getLiveNode(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public Messages.WrappedJsonProto getModelDesired(Messages.EmptyPayloadProto request) throws IOException {
+    try {
+      return endpoint.getModelDesired(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public Messages.WrappedJsonProto getModelDesiredAppconf(Messages.EmptyPayloadProto request) throws IOException {
+    try {
+      return endpoint.getModelDesiredAppconf(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public Messages.WrappedJsonProto getModelDesiredResources(Messages.EmptyPayloadProto request) throws IOException {
+    try {
+      return endpoint.getModelDesiredResources(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public Messages.WrappedJsonProto getModelResolved(Messages.EmptyPayloadProto request) throws IOException {
+    try {
+      return endpoint.getModelResolved(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public Messages.WrappedJsonProto getModelResolvedAppconf(Messages.EmptyPayloadProto request) throws IOException {
+    try {
+      return endpoint.getModelResolvedAppconf(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public Messages.WrappedJsonProto getModelResolvedResources(Messages.EmptyPayloadProto request) throws IOException {
+    try {
+      return endpoint.getModelResolvedResources(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public Messages.WrappedJsonProto getLiveResources(Messages.EmptyPayloadProto request) throws IOException {
+    try {
+      return endpoint.getLiveResources(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+
+  }
+
+  @Override
+  public Messages.GetCertificateStoreResponseProto getClientCertificateStore(Messages.GetCertificateStoreRequestProto request) throws
+      IOException {
+    try {
+      return endpoint.getClientCertificateStore(NULL_CONTROLLER, request);
+    } catch (ServiceException e) {
+      throw convert(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderIPCService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderIPCService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderIPCService.java
new file mode 100644
index 0000000..fda23aa
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderIPCService.java
@@ -0,0 +1,551 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.rpc;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.slider.api.ClusterDescription;
+import org.apache.slider.api.SliderClusterProtocol;
+import org.apache.slider.api.proto.Messages;
+import org.apache.slider.api.types.ApplicationLivenessInformation;
+import org.apache.slider.api.types.ComponentInformation;
+import org.apache.slider.api.types.ContainerInformation;
+import org.apache.slider.api.types.NodeInformation;
+import org.apache.slider.api.types.NodeInformationList;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTree;
+import org.apache.slider.core.exceptions.ServiceNotReadyException;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.core.main.LauncherExitCodes;
+import org.apache.slider.core.persist.AggregateConfSerDeser;
+import org.apache.slider.core.persist.ConfTreeSerDeser;
+import org.apache.slider.server.appmaster.AppMasterActionOperations;
+import org.apache.slider.server.appmaster.actions.ActionFlexCluster;
+import org.apache.slider.server.appmaster.actions.ActionHalt;
+import org.apache.slider.server.appmaster.actions.ActionKillContainer;
+import org.apache.slider.server.appmaster.actions.ActionStopSlider;
+import org.apache.slider.server.appmaster.actions.ActionUpgradeContainers;
+import org.apache.slider.server.appmaster.actions.AsyncAction;
+import org.apache.slider.server.appmaster.actions.QueueAccess;
+import org.apache.slider.server.appmaster.management.MetricsAndMonitoring;
+import org.apache.slider.server.appmaster.state.RoleInstance;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+import org.apache.slider.server.appmaster.web.rest.application.resources.ContentCache;
+import org.apache.slider.server.services.security.CertificateManager;
+import org.apache.slider.server.services.security.SecurityStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.slider.api.proto.RestTypeMarshalling.marshall;
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.LIVE_COMPONENTS;
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.LIVE_CONTAINERS;
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.LIVE_NODES;
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.LIVE_RESOURCES;
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.MODEL_DESIRED;
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.MODEL_DESIRED_APPCONF;
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.MODEL_DESIRED_RESOURCES;
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.MODEL_RESOLVED;
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.MODEL_RESOLVED_APPCONF;
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.MODEL_RESOLVED_RESOURCES;
+
+/**
+ * Implement the {@link SliderClusterProtocol}.
+ */
+@SuppressWarnings("unchecked")
+
+public class SliderIPCService extends AbstractService
+    implements SliderClusterProtocol {
+
+  protected static final Logger log =
+      LoggerFactory.getLogger(SliderIPCService.class);
+
+  private final QueueAccess actionQueues;
+  private final StateAccessForProviders state;
+  private final MetricsAndMonitoring metricsAndMonitoring;
+  private final AppMasterActionOperations amOperations;
+  private final ContentCache cache;
+  private final CertificateManager certificateManager;
+
+  /**
+   * This is the prefix used for metrics
+   */
+  public static final String METRICS_PREFIX =
+      "org.apache.slider.api.SliderIPCService.";
+
+  /**
+   * Constructor
+   * @param amOperations access to any AM operations
+   * @param state state view
+   * @param actionQueues queues for actions
+   * @param metricsAndMonitoring metrics
+   * @param cache
+   */
+  public SliderIPCService(AppMasterActionOperations amOperations,
+      CertificateManager certificateManager,
+      StateAccessForProviders state,
+      QueueAccess actionQueues,
+      MetricsAndMonitoring metricsAndMonitoring,
+      ContentCache cache) {
+    super("SliderIPCService");
+    Preconditions.checkArgument(amOperations != null, "null amOperations");
+    Preconditions.checkArgument(state != null, "null appState");
+    Preconditions.checkArgument(actionQueues != null, "null actionQueues");
+    Preconditions.checkArgument(metricsAndMonitoring != null,
+        "null metricsAndMonitoring");
+    Preconditions.checkArgument(cache != null, "null cache");
+    this.state = state;
+    this.actionQueues = actionQueues;
+    this.metricsAndMonitoring = metricsAndMonitoring;
+    this.amOperations = amOperations;
+    this.cache = cache;
+    this.certificateManager = certificateManager;
+  }
+
+  @Override   //SliderClusterProtocol
+  public ProtocolSignature getProtocolSignature(String protocol,
+      long clientVersion,
+      int clientMethodsHash) throws IOException {
+    return ProtocolSignature.getProtocolSignature(
+        this, protocol, clientVersion, clientMethodsHash);
+  }
+
+
+  @Override   //SliderClusterProtocol
+  public long getProtocolVersion(String protocol, long clientVersion)
+      throws IOException {
+    return SliderClusterProtocol.versionID;
+  }
+
+  /**
+   * General actions to perform on a slider RPC call coming in
+   * @param operation operation to log
+   * @throws IOException problems
+   * @throws ServiceNotReadyException if the RPC service is constructed
+   * but not fully initialized
+   */
+  protected void onRpcCall(String operation) throws IOException {
+    log.debug("Received call to {}", operation);
+    metricsAndMonitoring.markMeterAndCounter(METRICS_PREFIX + operation);
+  }
+
+  /**
+   * Schedule an action
+   * @param action for delayed execution
+   */
+  public void schedule(AsyncAction action) {
+    actionQueues.schedule(action);
+  }
+
+  /**
+   * Queue an action for immediate execution in the executor thread
+   * @param action action to execute
+   */
+  public void queue(AsyncAction action) {
+    actionQueues.put(action);
+  }
+
+  @Override //SliderClusterProtocol
+  public Messages.StopClusterResponseProto stopCluster(Messages.StopClusterRequestProto request)
+      throws IOException, YarnException {
+    onRpcCall("stop");
+    String message = request.getMessage();
+    if (message == null) {
+      message = "application stopped by client";
+    }
+    ActionStopSlider stopSlider =
+        new ActionStopSlider(message,
+            1000, TimeUnit.MILLISECONDS,
+            LauncherExitCodes.EXIT_SUCCESS,
+            FinalApplicationStatus.SUCCEEDED,
+            message);
+    log.info("SliderAppMasterApi.stopCluster: {}", stopSlider);
+    schedule(stopSlider);
+    return Messages.StopClusterResponseProto.getDefaultInstance();
+  }
+
+  @Override //SliderClusterProtocol
+  public Messages.UpgradeContainersResponseProto upgradeContainers(
+      Messages.UpgradeContainersRequestProto request) throws IOException,
+      YarnException {
+    onRpcCall("upgrade");
+    String message = request.getMessage();
+    if (message == null) {
+      message = "application containers upgraded by client";
+    }
+    ActionUpgradeContainers upgradeContainers =
+        new ActionUpgradeContainers(
+            "Upgrade containers",
+            1000, TimeUnit.MILLISECONDS,
+            LauncherExitCodes.EXIT_SUCCESS,
+            FinalApplicationStatus.SUCCEEDED,
+            request.getContainerList(),
+            request.getComponentList(),
+            message);
+    log.info("SliderAppMasterApi.upgradeContainers: {}", upgradeContainers);
+    schedule(upgradeContainers);
+    return Messages.UpgradeContainersResponseProto.getDefaultInstance();
+  }
+
+  @Override //SliderClusterProtocol
+  public Messages.FlexClusterResponseProto flexCluster(Messages.FlexClusterRequestProto request)
+      throws IOException {
+    onRpcCall("flex");
+    String payload = request.getClusterSpec();
+    ConfTreeSerDeser confTreeSerDeser = new ConfTreeSerDeser();
+    ConfTree updatedResources = confTreeSerDeser.fromJson(payload);
+    schedule(new ActionFlexCluster("flex", 1, TimeUnit.MILLISECONDS,
+        updatedResources));
+    return Messages.FlexClusterResponseProto.newBuilder().setResponse(
+        true).build();
+  }
+
+  @Override //SliderClusterProtocol
+  public Messages.GetJSONClusterStatusResponseProto getJSONClusterStatus(
+      Messages.GetJSONClusterStatusRequestProto request)
+      throws IOException, YarnException {
+    onRpcCall("getstatus");
+    String result;
+    //quick update
+    //query and json-ify
+    ClusterDescription cd = state.refreshClusterStatus();
+    result = cd.toJsonString();
+    String stat = result;
+    return Messages.GetJSONClusterStatusResponseProto.newBuilder()
+                                                     .setClusterSpec(stat)
+                                                     .build();
+  }
+
+  @Override
+  public Messages.GetInstanceDefinitionResponseProto getInstanceDefinition(
+      Messages.GetInstanceDefinitionRequestProto request)
+      throws IOException, YarnException {
+
+    onRpcCall("getinstancedefinition");
+    String internal;
+    String resources;
+    String app;
+    AggregateConf instanceDefinition =
+        state.getInstanceDefinitionSnapshot();
+    internal = instanceDefinition.getInternal().toJson();
+    resources = instanceDefinition.getResources().toJson();
+    app = instanceDefinition.getAppConf().toJson();
+    assert internal != null;
+    assert resources != null;
+    assert app != null;
+    log.debug("Generating getInstanceDefinition Response");
+    Messages.GetInstanceDefinitionResponseProto.Builder builder =
+        Messages.GetInstanceDefinitionResponseProto.newBuilder();
+    builder.setInternal(internal);
+    builder.setResources(resources);
+    builder.setApplication(app);
+    return builder.build();
+  }
+
+  @Override //SliderClusterProtocol
+  public Messages.ListNodeUUIDsByRoleResponseProto listNodeUUIDsByRole(Messages.ListNodeUUIDsByRoleRequestProto request)
+      throws IOException, YarnException {
+    onRpcCall("listnodes)");
+    String role = request.getRole();
+    Messages.ListNodeUUIDsByRoleResponseProto.Builder builder =
+        Messages.ListNodeUUIDsByRoleResponseProto.newBuilder();
+    List<RoleInstance> nodes = state.enumLiveInstancesInRole(role);
+    for (RoleInstance node : nodes) {
+      builder.addUuid(node.id);
+    }
+    return builder.build();
+  }
+
+  @Override //SliderClusterProtocol
+  public Messages.GetNodeResponseProto getNode(Messages.GetNodeRequestProto request)
+      throws IOException, YarnException {
+    onRpcCall("getnode");
+    RoleInstance instance = state.getLiveInstanceByContainerID(
+        request.getUuid());
+    return Messages.GetNodeResponseProto.newBuilder()
+                                        .setClusterNode(instance.toProtobuf())
+                                        .build();
+  }
+
+  @Override //SliderClusterProtocol
+  public Messages.GetClusterNodesResponseProto getClusterNodes(
+      Messages.GetClusterNodesRequestProto request)
+      throws IOException, YarnException {
+    onRpcCall("getclusternodes");
+    List<RoleInstance>
+        clusterNodes = state.getLiveInstancesByContainerIDs(
+        request.getUuidList());
+
+    Messages.GetClusterNodesResponseProto.Builder builder =
+        Messages.GetClusterNodesResponseProto.newBuilder();
+    for (RoleInstance node : clusterNodes) {
+      builder.addClusterNode(node.toProtobuf());
+    }
+    //at this point: a possibly empty list of nodes
+    return builder.build();
+  }
+
+  @Override
+  public Messages.EchoResponseProto echo(Messages.EchoRequestProto request)
+      throws IOException, YarnException {
+    onRpcCall("echo");
+    Messages.EchoResponseProto.Builder builder =
+        Messages.EchoResponseProto.newBuilder();
+    String text = request.getText();
+    log.info("Echo request size ={}", text.length());
+    log.info(text);
+    //now return it
+    builder.setText(text);
+    return builder.build();
+  }
+
+  @Override
+  public Messages.KillContainerResponseProto killContainer(Messages.KillContainerRequestProto request)
+      throws IOException, YarnException {
+    onRpcCall("killcontainer");
+    String containerID = request.getId();
+    log.info("Kill Container {}", containerID);
+    //throws NoSuchNodeException if it is missing
+    RoleInstance instance =
+        state.getLiveInstanceByContainerID(containerID);
+    queue(new ActionKillContainer(instance.getId(), 0, TimeUnit.MILLISECONDS,
+        amOperations));
+    Messages.KillContainerResponseProto.Builder builder =
+        Messages.KillContainerResponseProto.newBuilder();
+    builder.setSuccess(true);
+    return builder.build();
+  }
+
+
+  @Override
+  public Messages.AMSuicideResponseProto amSuicide(
+      Messages.AMSuicideRequestProto request)
+      throws IOException {
+    onRpcCall("amsuicide");
+    int signal = request.getSignal();
+    String text = request.getText();
+    if (text == null) {
+      text = "";
+    }
+    int delay = request.getDelay();
+    log.info("AM Suicide with signal {}, message {} delay = {}", signal, text,
+        delay);
+    ActionHalt action = new ActionHalt(signal, text, delay,
+        TimeUnit.MILLISECONDS);
+    schedule(action);
+    return Messages.AMSuicideResponseProto.getDefaultInstance();
+  }
+
+  @Override
+  public Messages.ApplicationLivenessInformationProto getLivenessInformation(
+      Messages.GetApplicationLivenessRequestProto request) throws IOException {
+    ApplicationLivenessInformation info =
+        state.getApplicationLivenessInformation();
+    return marshall(info);
+  }
+
+  @Override
+  public Messages.GetLiveContainersResponseProto getLiveContainers(
+      Messages.GetLiveContainersRequestProto request)
+      throws IOException {
+    Map<String, ContainerInformation> infoMap =
+        (Map<String, ContainerInformation>) cache.lookupWithIOE(LIVE_CONTAINERS);
+    Messages.GetLiveContainersResponseProto.Builder builder =
+        Messages.GetLiveContainersResponseProto.newBuilder();
+
+    for (Map.Entry<String, ContainerInformation> entry : infoMap.entrySet()) {
+      builder.addNames(entry.getKey());
+      builder.addContainers(marshall(entry.getValue()));
+    }
+    return builder.build();
+  }
+
+  @Override
+  public Messages.ContainerInformationProto getLiveContainer(Messages.GetLiveContainerRequestProto request)
+      throws IOException {
+    String containerId = request.getContainerId();
+    RoleInstance id = state.getLiveInstanceByContainerID(containerId);
+    ContainerInformation containerInformation = id.serialize();
+    return marshall(containerInformation);
+  }
+
+  @Override
+  public Messages.GetLiveComponentsResponseProto getLiveComponents(Messages.GetLiveComponentsRequestProto request)
+      throws IOException {
+    Map<String, ComponentInformation> infoMap =
+        (Map<String, ComponentInformation>) cache.lookupWithIOE(LIVE_COMPONENTS);
+    Messages.GetLiveComponentsResponseProto.Builder builder =
+        Messages.GetLiveComponentsResponseProto.newBuilder();
+
+    for (Map.Entry<String, ComponentInformation> entry : infoMap.entrySet()) {
+      builder.addNames(entry.getKey());
+      builder.addComponents(marshall(entry.getValue()));
+    }
+    return builder.build();
+  }
+
+
+  @Override
+  public Messages.ComponentInformationProto getLiveComponent(Messages.GetLiveComponentRequestProto request)
+      throws IOException {
+    String name = request.getName();
+    try {
+      return marshall(state.getComponentInformation(name));
+    } catch (YarnRuntimeException e) {
+      throw new FileNotFoundException("Unknown component: " + name);
+    }
+  }
+
+  @Override
+  public Messages.GetLiveNodesResponseProto getLiveNodes(Messages.GetLiveNodesRequestProto request)
+      throws IOException {
+    NodeInformationList info = (NodeInformationList) cache.lookupWithIOE(LIVE_NODES);
+    Messages.GetLiveNodesResponseProto.Builder builder =
+        Messages.GetLiveNodesResponseProto.newBuilder();
+
+    for (NodeInformation nodeInformation : info) {
+      builder.addNodes(marshall(nodeInformation));
+    }
+    return builder.build();
+  }
+
+
+  @Override
+  public Messages.NodeInformationProto getLiveNode(Messages.GetLiveNodeRequestProto request)
+      throws IOException {
+    String name = request.getName();
+    NodeInformation nodeInformation = state.getNodeInformation(name);
+    if (nodeInformation != null) {
+      return marshall(nodeInformation);
+    } else {
+      throw new FileNotFoundException("Unknown host: " + name);
+    }
+  }
+
+  @Override
+  public Messages.WrappedJsonProto getModelDesired(Messages.EmptyPayloadProto request) throws IOException {
+    return lookupAggregateConf(MODEL_DESIRED);
+  }
+
+  @Override
+  public Messages.WrappedJsonProto getModelDesiredAppconf(Messages.EmptyPayloadProto request) throws IOException {
+    return lookupConfTree(MODEL_DESIRED_APPCONF);
+  }
+
+  @Override
+  public Messages.WrappedJsonProto getModelDesiredResources(Messages.EmptyPayloadProto request) throws IOException {
+    return lookupConfTree(MODEL_DESIRED_RESOURCES);
+  }
+
+  @Override
+  public Messages.WrappedJsonProto getModelResolved(Messages.EmptyPayloadProto request) throws IOException {
+    return lookupAggregateConf(MODEL_RESOLVED);
+  }
+
+  @Override
+  public Messages.WrappedJsonProto getModelResolvedAppconf(Messages.EmptyPayloadProto request) throws IOException {
+    return lookupConfTree(MODEL_RESOLVED_APPCONF);
+  }
+
+  @Override
+  public Messages.WrappedJsonProto getModelResolvedResources(Messages.EmptyPayloadProto request) throws IOException {
+    return lookupConfTree(MODEL_RESOLVED_RESOURCES);
+  }
+
+  @Override
+  public Messages.WrappedJsonProto getLiveResources(Messages.EmptyPayloadProto request) throws IOException {
+    return lookupConfTree(LIVE_RESOURCES);
+  }
+
+  /**
+   * Helper method; look up an aggregate configuration in the cache from
+   * a key, or raise an exception
+   * @param key key to resolve
+   * @return the configuration
+   * @throws IOException on a failure
+   */
+
+  protected Messages.WrappedJsonProto lookupAggregateConf(String key) throws
+      IOException {
+    AggregateConf aggregateConf = (AggregateConf) cache.lookupWithIOE(key);
+    String json = AggregateConfSerDeser.toString(aggregateConf);
+    return wrap(json);
+  }
+
+  /**
+   * Helper method; look up an conf tree in the cache from
+   * a key, or raise an exception
+   * @param key key to resolve
+   * @return the configuration
+   * @throws IOException on a failure
+   */
+  protected Messages.WrappedJsonProto lookupConfTree(String key) throws
+      IOException {
+    ConfTree conf = (ConfTree) cache.lookupWithIOE(key);
+    String json = ConfTreeSerDeser.toString(conf);
+    return wrap(json);
+  }
+
+  private Messages.WrappedJsonProto wrap(String json) {
+    Messages.WrappedJsonProto.Builder builder =
+        Messages.WrappedJsonProto.newBuilder();
+    builder.setJson(json);
+    return builder.build();
+  }
+
+  @Override
+  public Messages.GetCertificateStoreResponseProto getClientCertificateStore(Messages.GetCertificateStoreRequestProto request) throws
+      IOException {
+    String hostname = request.getHostname();
+    String clientId = request.getRequesterId();
+    String password = request.getPassword();
+    String type = request.getType();
+
+    SecurityStore store = null;
+    try {
+      if ( SecurityStore.StoreType.keystore.equals(
+          SecurityStore.StoreType.valueOf(type))) {
+        store = certificateManager.generateContainerKeystore(hostname,
+                                                             clientId,
+                                                             null,
+                                                             password);
+      } else if (SecurityStore.StoreType.truststore.equals(
+          SecurityStore.StoreType.valueOf(type))) {
+        store = certificateManager.generateContainerTruststore(clientId,
+                                                               null,
+                                                               password);
+
+      } else {
+        throw new IOException("Illegal store type");
+      }
+    } catch (SliderException e) {
+      throw new IOException(e);
+    }
+    return marshall(store);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderRPCSecurityInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderRPCSecurityInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderRPCSecurityInfo.java
new file mode 100644
index 0000000..4fd4910
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/rpc/SliderRPCSecurityInfo.java
@@ -0,0 +1,87 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.slider.server.appmaster.rpc;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.KerberosInfo;
+import org.apache.hadoop.security.SecurityInfo;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.security.token.TokenInfo;
+import org.apache.hadoop.security.token.TokenSelector;
+import org.apache.hadoop.yarn.security.client.ClientToAMTokenSelector;
+import org.apache.slider.common.SliderXmlConfKeys;
+
+import java.lang.annotation.Annotation;
+
+/**
+ * This is where security information goes.
+ * It is referred to in the <code>META-INF/services/org.apache.hadoop.security.SecurityInfo</code>
+ * resource of this JAR, which is used to find the binding info
+ */
+public class SliderRPCSecurityInfo extends SecurityInfo {
+
+  @Override
+  public KerberosInfo getKerberosInfo(Class<?> protocol, Configuration conf) {
+    if (!protocol.equals(SliderClusterProtocolPB.class)) {
+      return null;
+    }
+    return new KerberosInfo() {
+
+      @Override
+      public Class<? extends Annotation> annotationType() {
+        return null;
+      }
+
+      @Override
+      public String serverPrincipal() {
+        return SliderXmlConfKeys.KEY_KERBEROS_PRINCIPAL;
+      }
+
+      @Override
+      public String clientPrincipal() {
+        return null;
+      }
+    };
+  }
+
+  @Override
+  public TokenInfo getTokenInfo(Class<?> protocol, Configuration conf) {
+    if (!protocol.equals(SliderClusterProtocolPB.class)) {
+      return null;
+    }
+    return new TokenInfo() {
+
+      @Override
+      public Class<? extends Annotation> annotationType() {
+        return null;
+      }
+
+      @Override
+      public Class<? extends TokenSelector<? extends TokenIdentifier>>
+          value() {
+        return ClientToAMTokenSelector.class;
+      }
+
+      @Override
+      public String toString() {
+        return "SliderClusterProtocolPB token info";
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/security/SecurityConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/security/SecurityConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/security/SecurityConfiguration.java
new file mode 100644
index 0000000..9a89c39
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/security/SecurityConfiguration.java
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.security;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import static org.apache.slider.core.main.LauncherExitCodes.EXIT_UNAUTHORIZED;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.SliderXmlConfKeys;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.exceptions.SliderException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+
+/**
+ * Class keeping code security information
+ */
+public class SecurityConfiguration {
+
+  protected static final Logger log =
+      LoggerFactory.getLogger(SecurityConfiguration.class);
+  private final Configuration configuration;
+  private final AggregateConf instanceDefinition;
+  private String clusterName;
+
+  public SecurityConfiguration(Configuration configuration,
+                               AggregateConf instanceDefinition,
+                               String clusterName) throws SliderException {
+    Preconditions.checkNotNull(configuration);
+    Preconditions.checkNotNull(instanceDefinition);
+    Preconditions.checkNotNull(clusterName);
+    this.configuration = configuration;
+    this.instanceDefinition = instanceDefinition;
+    this.clusterName = clusterName;
+    validate();
+  }
+
+  private void validate() throws SliderException {
+    if (isSecurityEnabled()) {
+      String principal = instanceDefinition.getAppConfOperations()
+          .getComponent(SliderKeys.COMPONENT_AM).get(SliderXmlConfKeys.KEY_KEYTAB_PRINCIPAL);
+      if(SliderUtils.isUnset(principal)) {
+        // if no login identity is available, fail
+        UserGroupInformation loginUser = null;
+        try {
+          loginUser = getLoginUser();
+        } catch (IOException e) {
+          throw new SliderException(EXIT_UNAUTHORIZED, e,
+                                    "No principal configured for the application and "
+                                    + "exception raised during retrieval of login user. "
+                                    + "Unable to proceed with application "
+                                    + "initialization.  Please ensure a value "
+                                    + "for %s exists in the application "
+                                    + "configuration or the login issue is addressed",
+                                    SliderXmlConfKeys.KEY_KEYTAB_PRINCIPAL);
+        }
+        if (loginUser == null) {
+          throw new SliderException(EXIT_UNAUTHORIZED,
+                                    "No principal configured for the application "
+                                    + "and no login user found. "
+                                    + "Unable to proceed with application "
+                                    + "initialization.  Please ensure a value "
+                                    + "for %s exists in the application "
+                                    + "configuration or the login issue is addressed",
+                                    SliderXmlConfKeys.KEY_KEYTAB_PRINCIPAL);
+        }
+      }
+      // ensure that either local or distributed keytab mechanism is enabled,
+      // but not both
+      String keytabFullPath = instanceDefinition.getAppConfOperations()
+          .getComponent(SliderKeys.COMPONENT_AM)
+          .get(SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH);
+      String keytabName = instanceDefinition.getAppConfOperations()
+          .getComponent(SliderKeys.COMPONENT_AM)
+          .get(SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME);
+      if (SliderUtils.isSet(keytabFullPath) && SliderUtils.isSet(keytabName)) {
+        throw new SliderException(EXIT_UNAUTHORIZED,
+                                  "Both a keytab on the cluster host (%s) and a"
+                                  + " keytab to be retrieved from HDFS (%s) are"
+                                  + " specified.  Please configure only one keytab"
+                                  + " retrieval mechanism.",
+                                  SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH,
+                                  SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME);
+
+      }
+    }
+  }
+
+  protected UserGroupInformation getLoginUser() throws IOException {
+    return UserGroupInformation.getLoginUser();
+  }
+
+  public boolean isSecurityEnabled () {
+    return SliderUtils.isHadoopClusterSecure(configuration);
+  }
+
+  public String getPrincipal () throws IOException {
+    String principal = instanceDefinition.getAppConfOperations()
+        .getComponent(SliderKeys.COMPONENT_AM).get(SliderXmlConfKeys.KEY_KEYTAB_PRINCIPAL);
+    if (SliderUtils.isUnset(principal)) {
+      principal = UserGroupInformation.getLoginUser().getShortUserName();
+      log.info("No principal set in the slider configuration.  Will use AM login"
+               + " identity {} to attempt keytab-based login", principal);
+    }
+
+    return principal;
+  }
+
+  public boolean isKeytabProvided() {
+    boolean keytabProvided = instanceDefinition.getAppConfOperations()
+                    .getComponent(SliderKeys.COMPONENT_AM)
+                    .get(SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH) != null ||
+                instanceDefinition.getAppConfOperations()
+                    .getComponent(SliderKeys.COMPONENT_AM).
+                    get(SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME) != null;
+    return keytabProvided;
+
+  }
+
+  public File getKeytabFile(AggregateConf instanceDefinition)
+      throws SliderException, IOException {
+    String keytabFullPath = instanceDefinition.getAppConfOperations()
+        .getComponent(SliderKeys.COMPONENT_AM)
+        .get(SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH);
+    File localKeytabFile;
+    if (SliderUtils.isUnset(keytabFullPath)) {
+      // get the keytab
+      String keytabName = instanceDefinition.getAppConfOperations()
+          .getComponent(SliderKeys.COMPONENT_AM).
+              get(SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME);
+      log.info("No host keytab file path specified. Will attempt to retrieve"
+               + " keytab file {} as a local resource for the container",
+               keytabName);
+      // download keytab to local, protected directory
+      localKeytabFile = new File(SliderKeys.KEYTAB_DIR, keytabName);
+    } else {
+      log.info("Using host keytab file {} for login", keytabFullPath);
+      localKeytabFile = new File(keytabFullPath);
+    }
+    return localKeytabFile;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AbstractClusterServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AbstractClusterServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AbstractClusterServices.java
new file mode 100644
index 0000000..54f384b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AbstractClusterServices.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
+
+/**
+ * Cluster services offered by the YARN infrastructure.
+ */
+public abstract class AbstractClusterServices {
+
+  private final DefaultResourceCalculator
+      defaultResourceCalculator = new DefaultResourceCalculator();
+
+  /**
+   * Create a resource for requests
+   * @return a resource which can be built up.
+   */
+  public abstract Resource newResource();
+
+  public abstract Resource newResource(int memory, int cores);
+
+  /**
+   * Normalise memory, CPU and other resources according to the YARN AM-supplied
+   * values and the resource calculator in use (currently hard-coded to the
+   * {@link DefaultResourceCalculator}.
+   * Those resources which aren't normalized (currently: CPU) are left
+   * as is.
+   * @param resource resource requirements of a role
+   * @param minR minimum values of this queue
+   * @param maxR max values of this queue
+   * @return a normalized value.
+   */
+  public Resource normalize(Resource resource, Resource minR, Resource maxR) {
+    Preconditions.checkArgument(resource != null, "null resource");
+    Preconditions.checkArgument(minR != null, "null minR");
+    Preconditions.checkArgument(maxR != null, "null maxR");
+
+    Resource normalize = defaultResourceCalculator.normalize(resource, minR,
+        maxR, minR);
+    return newResource(normalize.getMemory(), resource.getVirtualCores());
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[47/76] [abbrv] hadoop git commit: YARN-5610. Initial code for native services REST API. Contributed by Gour Saha

Posted by ji...@apache.org.
YARN-5610. Initial code for native services REST API. Contributed by Gour Saha


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

Branch: refs/heads/yarn-native-services
Commit: 019adbc868ec2cd448f5776ae3a19d301a4bc27c
Parents: cc7b200
Author: Jian He <ji...@apache.org>
Authored: Tue Oct 11 11:36:57 2016 -0700
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 hadoop-project/pom.xml                          |   20 +
 .../dev-support/findbugs-exclude.xml            |   20 +
 .../hadoop-yarn-services-api/pom.xml            |  225 +++
 .../yarn/services/api/ApplicationApi.java       |   38 +
 .../api/impl/ApplicationApiService.java         | 1527 ++++++++++++++++++
 .../yarn/services/resource/Application.java     |  452 ++++++
 .../services/resource/ApplicationState.java     |   25 +
 .../services/resource/ApplicationStatus.java    |  147 ++
 .../hadoop/yarn/services/resource/Artifact.java |  155 ++
 .../yarn/services/resource/BaseResource.java    |   48 +
 .../yarn/services/resource/Component.java       |  377 +++++
 .../yarn/services/resource/ConfigFile.java      |  190 +++
 .../yarn/services/resource/Configuration.java   |  147 ++
 .../yarn/services/resource/Container.java       |  256 +++
 .../yarn/services/resource/ContainerState.java  |   25 +
 .../hadoop/yarn/services/resource/Error.java    |  125 ++
 .../yarn/services/resource/PlacementPolicy.java |   97 ++
 .../yarn/services/resource/ReadinessCheck.java  |  161 ++
 .../hadoop/yarn/services/resource/Resource.java |  149 ++
 .../yarn/services/utils/RestApiConstants.java   |   66 +
 .../services/utils/RestApiErrorMessages.java    |   79 +
 .../services/webapp/ApplicationApiWebApp.java   |  127 ++
 .../src/main/resources/log4j-server.properties  |   76 +
 .../resources/webapps/services-rest-api/app     |   16 +
 .../src/main/scripts/run_rest_service.sh        |   28 +
 .../src/main/webapp/WEB-INF/web.xml             |   36 +
 .../api/impl/TestApplicationApiService.java     |  232 +++
 .../hadoop-yarn-applications/pom.xml            |    2 +-
 28 files changed, 4845 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 1ae60ed..a2b6e25 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -126,6 +126,9 @@
     <native-maven-plugin.version>1.0-alpha-8</native-maven-plugin.version>
     <surefire.fork.timeout>900</surefire.fork.timeout>
     <aws-java-sdk.version>1.11.45</aws-java-sdk.version>
+
+    <swagger-annotations-version>1.5.4</swagger-annotations-version>
+    <maven-doxia-module-markdown.version>1.4</maven-doxia-module-markdown.version>
   </properties>
 
   <dependencyManagement>
@@ -1235,6 +1238,23 @@
           <artifactId>kerb-simplekdc</artifactId>
           <version>1.0.0-RC2</version>
         </dependency>
+
+      <dependency>
+        <groupId>io.swagger</groupId>
+        <artifactId>swagger-annotations</artifactId>
+        <version>${swagger-annotations-version}</version>
+      </dependency>
+      <dependency>
+        <groupId>com.fasterxml.jackson.jaxrs</groupId>
+        <artifactId>jackson-jaxrs-json-provider</artifactId>
+        <version>${jackson2.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.maven.doxia</groupId>
+        <artifactId>doxia-module-markdown</artifactId>
+        <version>${maven-doxia-module-markdown.version}</version>
+      </dependency>
+
     </dependencies>
   </dependencyManagement>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/dev-support/findbugs-exclude.xml
new file mode 100644
index 0000000..b89146a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/dev-support/findbugs-exclude.xml
@@ -0,0 +1,20 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+   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.
+-->
+<FindBugsFilter>
+
+</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/pom.xml
new file mode 100644
index 0000000..78b7855
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/pom.xml
@@ -0,0 +1,225 @@
+<!--
+   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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-yarn-applications</artifactId>
+    <version>3.0.0-alpha2-SNAPSHOT</version>
+  </parent>
+  <groupId>org.apache.hadoop</groupId>
+  <artifactId>hadoop-yarn-services-api</artifactId>
+  <name>Apache Hadoop YARN Services API</name>
+  <version>3.0.0-alpha2-SNAPSHOT</version>
+  <packaging>jar</packaging>
+  <description>Hadoop YARN REST APIs for services</description>
+
+  <properties>
+    <test.failIfNoTests>false</test.failIfNoTests>
+    <powermock.version>1.6.5</powermock.version>
+  </properties>
+
+  <build>
+
+    <!-- resources are filtered for dynamic updates. This gets build info in-->
+    <resources>
+      <resource>
+        <directory>src/main/resources</directory>
+        <filtering>true</filtering>
+      </resource>
+      <resource>
+        <directory>src/main/scripts/</directory>
+        <filtering>true</filtering>
+      </resource>
+    </resources>
+
+    <plugins>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>${maven-jar-plugin.version}</version>
+        <!-- The configuration of the plugin -->
+        <configuration>
+          <!-- Configuration of the archiver -->
+          <archive>
+            <manifestEntries>
+              <mode>development</mode>
+              <url>${project.url}</url>
+            </manifestEntries>
+            <!-- Manifest specific configuration -->
+            <manifest>
+            </manifest>
+          </archive>
+          <excludes>
+            <exclude>**/run_rest_service.sh</exclude>
+          </excludes>
+        </configuration>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <version>${maven-surefire-plugin.version}</version>
+        <configuration>
+          <reuseForks>${test.reuseForks}</reuseForks>
+          <forkMode>${test.forkMode}</forkMode>
+          <forkCount>1</forkCount>
+          <forkedProcessTimeoutInSeconds>${test.forkedProcessTimeoutInSeconds}
+          </forkedProcessTimeoutInSeconds>
+          <threadCount>1</threadCount>
+          <argLine>${test.argLine}</argLine>
+          <failIfNoTests>${test.failIfNoTests}</failIfNoTests>
+          <redirectTestOutputToFile>${build.redirect.test.output.to.file}</redirectTestOutputToFile>
+          <environmentVariables>
+            <PATH>${test.env.path}</PATH>
+          </environmentVariables>
+          <systemPropertyVariables>
+            <java.net.preferIPv4Stack>true</java.net.preferIPv4Stack>
+            <java.awt.headless>true</java.awt.headless>
+          </systemPropertyVariables>
+          <includes>
+            <include>**/Test*.java</include>
+          </includes>
+          <excludes>
+            <exclude>**/Test*$*.java</exclude>
+          </excludes>
+        </configuration>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-site-plugin</artifactId>
+        <version>${maven-site-plugin.version}</version>
+
+        <dependencies>
+          <dependency>
+            <groupId>org.apache.maven.doxia</groupId>
+            <artifactId>doxia-module-markdown</artifactId>
+            <version>${maven-doxia-module-markdown.version}</version>
+          </dependency>
+        </dependencies>
+      </plugin>
+    </plugins>
+  </build>
+
+  <reporting>
+  </reporting>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-slider-core</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.swagger</groupId>
+      <artifactId>swagger-annotations</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.jaxrs</groupId>
+      <artifactId>jackson-jaxrs-json-provider</artifactId>
+      <exclusions>
+        <exclusion>
+          <groupId>com.fasterxml.jackson.jaxrs</groupId>
+          <artifactId>jackson-jaxrs-base</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.powermock</groupId>
+      <artifactId>powermock-module-junit4</artifactId>
+      <version>${powermock.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.powermock</groupId>
+      <artifactId>powermock-api-easymock</artifactId>
+      <version>${powermock.version}</version>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.easymock</groupId>
+          <artifactId>easymock</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+  </dependencies>
+
+  <profiles>
+
+    <profile>
+      <id>rat</id>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.rat</groupId>
+            <artifactId>apache-rat-plugin</artifactId>
+            <version>${apache-rat-plugin.version}</version>
+            <executions>
+              <execution>
+                <id>check-licenses</id>
+                <goals>
+                  <goal>check</goal>
+                </goals>
+              </execution>
+            </executions>
+            <configuration>
+              <excludes>
+                <exclude>**/*.json</exclude>
+                <exclude>**/THIRD-PARTY.properties</exclude>
+              </excludes>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+
+  </profiles>
+
+</project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/ApplicationApi.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/ApplicationApi.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/ApplicationApi.java
new file mode 100644
index 0000000..654413c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/ApplicationApi.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.services.api;
+
+import javax.ws.rs.core.Response;
+
+import org.apache.hadoop.yarn.services.resource.Application;
+
+/**
+ * Apache Hadoop YARN Services REST API interface.
+ *
+ */
+public interface ApplicationApi {
+  Response createApplication(Application application);
+
+  Response getApplications(String state);
+
+  Response getApplication(String appName);
+
+  Response deleteApplication(String appName);
+
+  Response updateApplication(String appName, Application updateAppData);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
new file mode 100644
index 0000000..9645696
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
@@ -0,0 +1,1527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.services.api.impl;
+
+import static org.apache.hadoop.yarn.services.utils.RestApiConstants.*;
+import static org.apache.hadoop.yarn.services.utils.RestApiErrorMessages.*;
+
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+import javax.inject.Singleton;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.Response.Status;
+
+import org.apache.commons.collections.MapUtils;
+import org.apache.commons.lang.SerializationUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.services.api.ApplicationApi;
+import org.apache.hadoop.yarn.services.resource.Application;
+import org.apache.hadoop.yarn.services.resource.ApplicationState;
+import org.apache.hadoop.yarn.services.resource.ApplicationStatus;
+import org.apache.hadoop.yarn.services.resource.Artifact;
+import org.apache.hadoop.yarn.services.resource.Component;
+import org.apache.hadoop.yarn.services.resource.ConfigFile;
+import org.apache.hadoop.yarn.services.resource.Configuration;
+import org.apache.hadoop.yarn.services.resource.Container;
+import org.apache.hadoop.yarn.services.resource.ContainerState;
+import org.apache.hadoop.yarn.services.resource.Resource;
+import org.apache.slider.api.ResourceKeys;
+import org.apache.slider.api.StateValues;
+import org.apache.slider.client.SliderClient;
+import org.apache.slider.common.SliderExitCodes;
+import org.apache.slider.common.params.ActionCreateArgs;
+import org.apache.slider.common.params.ActionFlexArgs;
+import org.apache.slider.common.params.ActionFreezeArgs;
+import org.apache.slider.common.params.ActionListArgs;
+import org.apache.slider.common.params.ActionRegistryArgs;
+import org.apache.slider.common.params.ActionStatusArgs;
+import org.apache.slider.common.params.ActionThawArgs;
+import org.apache.slider.common.params.ComponentArgsDelegate;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.common.tools.SliderVersionInfo;
+import org.apache.slider.core.buildutils.BuildHelper;
+import org.apache.slider.core.exceptions.BadClusterStateException;
+import org.apache.slider.core.exceptions.NotFoundException;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.core.exceptions.UnknownApplicationInstanceException;
+import org.apache.slider.core.registry.docstore.ConfigFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonArray;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonNull;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParser;
+
+@Singleton
+@Path(APPLICATIONS_API_RESOURCE_PATH)
+@Consumes({ MediaType.APPLICATION_JSON })
+@Produces({ MediaType.APPLICATION_JSON })
+public class ApplicationApiService implements ApplicationApi {
+  private static final Logger logger = LoggerFactory
+      .getLogger(ApplicationApiService.class);
+  private static org.apache.hadoop.conf.Configuration SLIDER_CONFIG;
+  private static UserGroupInformation SLIDER_USER;
+  private static SliderClient SLIDER_CLIENT;
+
+  static {
+    init();
+  }
+
+  // initialize all the common resources - order is important
+  protected static void init() {
+    SLIDER_CONFIG = getSliderClientConfiguration();
+    SLIDER_USER = getSliderUser();
+    SLIDER_CLIENT = createSliderClient();
+  }
+
+  @GET
+  @Path("/slider-version")
+  @Consumes({ MediaType.APPLICATION_JSON })
+  @Produces({ MediaType.APPLICATION_JSON })
+  public Response getSliderVersion() {
+    logger.info("GET: getSliderVersion");
+
+    Map<String, Object> metadata = new HashMap<>();
+    BuildHelper.addBuildMetadata(metadata, "org.apache.hadoop.yarn.services");
+    String sliderVersion = metadata.toString();
+    logger.info("Slider version = {}", sliderVersion);
+    String hadoopVersion = SliderVersionInfo.getHadoopVersionString();
+    logger.info("Hadoop version = {}", hadoopVersion);
+    return Response.ok(
+        "{ \"slider_version\": \"" + sliderVersion
+            + "\", \"hadoop_version\": \"" + hadoopVersion + "\"}").build();
+  }
+
+  @POST
+  @Consumes({ MediaType.APPLICATION_JSON })
+  @Produces({ MediaType.APPLICATION_JSON })
+  public Response createApplication(Application application) {
+    logger.info("POST: createApplication for app = {}", application);
+    ApplicationStatus applicationStatus = new ApplicationStatus();
+
+    Map<String, String> compNameArtifactIdMap = new HashMap<>();
+    // post payload validation
+    try {
+      validateApplicationPostPayload(application, compNameArtifactIdMap);
+    } catch (IllegalArgumentException e) {
+      applicationStatus.setDiagnostics(e.getMessage());
+      return Response.status(Status.BAD_REQUEST).entity(applicationStatus)
+          .build();
+    }
+    String applicationId = null;
+    try {
+      applicationId = createSliderApp(application, compNameArtifactIdMap);
+      applicationStatus.setState(ApplicationState.ACCEPTED);
+    } catch (SliderException se) {
+      logger.error("Create application failed", se);
+      if (se.getExitCode() == SliderExitCodes.EXIT_APPLICATION_IN_USE) {
+        applicationStatus.setDiagnostics(ERROR_APPLICATION_IN_USE);
+        return Response.status(Status.BAD_REQUEST).entity(applicationStatus)
+            .build();
+      } else {
+        applicationStatus.setDiagnostics(se.getMessage());
+      }
+    } catch (Exception e) {
+      logger.error("Create application failed", e);
+      applicationStatus.setDiagnostics(e.getMessage());
+    }
+
+    if (StringUtils.isNotEmpty(applicationId)) {
+      applicationStatus.setUri(CONTEXT_ROOT + APPLICATIONS_API_RESOURCE_PATH
+          + "/" + application.getName());
+      // 202 = ACCEPTED
+      return Response.status(HTTP_STATUS_CODE_ACCEPTED)
+          .entity(applicationStatus).build();
+    } else {
+      return Response.status(Status.INTERNAL_SERVER_ERROR)
+          .entity(applicationStatus).build();
+    }
+  }
+
+  @VisibleForTesting
+  protected void validateApplicationPostPayload(Application application,
+      Map<String, String> compNameArtifactIdMap) {
+    if (StringUtils.isEmpty(application.getName())) {
+      throw new IllegalArgumentException(ERROR_APPLICATION_NAME_INVALID);
+    }
+    if (!SliderUtils.isClusternameValid(application.getName())) {
+      throw new IllegalArgumentException(ERROR_APPLICATION_NAME_INVALID_FORMAT);
+    }
+
+    // If the application has no components do top-level checks
+    if (application.getComponents() == null) {
+      // artifact
+      if (application.getArtifact() == null) {
+        throw new IllegalArgumentException(ERROR_ARTIFACT_INVALID);
+      }
+      if (StringUtils.isEmpty(application.getArtifact().getId())) {
+        throw new IllegalArgumentException(ERROR_ARTIFACT_ID_INVALID);
+      }
+
+      // If artifact is of type APPLICATION, add a slider specific property
+      if (application.getArtifact().getType() == Artifact.TypeEnum.APPLICATION) {
+        if (application.getConfiguration() == null) {
+          application.setConfiguration(new Configuration());
+        }
+        addPropertyToConfiguration(application.getConfiguration(),
+            PROPERTY_COMPONENT_TYPE, COMPONENT_TYPE_EXTERNAL);
+      }
+      // resource
+      validateApplicationResource(application.getResource(), null, application
+          .getArtifact().getType());
+
+      // container size
+      if (application.getNumberOfContainers() == null) {
+        throw new IllegalArgumentException(ERROR_CONTAINERS_COUNT_INVALID);
+      }
+    } else {
+      // If the application has components, then run checks for each component.
+      // Let global values take effect if component level values are not
+      // provided.
+      Artifact globalArtifact = application.getArtifact();
+      Resource globalResource = application.getResource();
+      Long globalNumberOfContainers = application.getNumberOfContainers();
+      for (Component comp : application.getComponents()) {
+        // artifact
+        if (comp.getArtifact() == null) {
+          comp.setArtifact(globalArtifact);
+        }
+        // If still null raise validation exception
+        if (comp.getArtifact() == null) {
+          throw new IllegalArgumentException(String.format(
+              ERROR_ARTIFACT_FOR_COMP_INVALID, comp.getName()));
+        }
+        if (StringUtils.isEmpty(comp.getArtifact().getId())) {
+          throw new IllegalArgumentException(String.format(
+              ERROR_ARTIFACT_ID_FOR_COMP_INVALID, comp.getName()));
+        }
+
+        // If artifact is of type APPLICATION, add a slider specific property
+        if (comp.getArtifact().getType() == Artifact.TypeEnum.APPLICATION) {
+          if (comp.getConfiguration() == null) {
+            comp.setConfiguration(new Configuration());
+          }
+          addPropertyToConfiguration(comp.getConfiguration(),
+              PROPERTY_COMPONENT_TYPE, COMPONENT_TYPE_EXTERNAL);
+          compNameArtifactIdMap.put(comp.getName(), comp.getArtifact().getId());
+          comp.setName(comp.getArtifact().getId());
+        }
+
+        // resource
+        if (comp.getResource() == null) {
+          comp.setResource(globalResource);
+        }
+        validateApplicationResource(comp.getResource(), comp, comp
+            .getArtifact().getType());
+
+        // container count
+        if (comp.getNumberOfContainers() == null) {
+          comp.setNumberOfContainers(globalNumberOfContainers);
+        }
+        if (comp.getNumberOfContainers() == null) {
+          throw new IllegalArgumentException(String.format(
+              ERROR_CONTAINERS_COUNT_FOR_COMP_INVALID, comp.getName()));
+        }
+      }
+    }
+
+    // If it is a simple app with no components, then create a default component
+    if (application.getComponents() == null) {
+      application.setComponents(getDefaultComponentAsList());
+    }
+
+    // Application lifetime if not specified, is set to unlimited lifetime
+    if (application.getLifetime() == null) {
+      application.setLifetime(DEFAULT_UNLIMITED_LIFETIME);
+    }
+  }
+
+  private void validateApplicationResource(Resource resource, Component comp,
+      Artifact.TypeEnum artifactType) {
+    // Only apps/components of type APPLICATION can skip resource requirement
+    if (resource == null && artifactType == Artifact.TypeEnum.APPLICATION) {
+      return;
+    }
+    if (resource == null) {
+      throw new IllegalArgumentException(comp == null ? ERROR_RESOURCE_INVALID
+          : String.format(ERROR_RESOURCE_FOR_COMP_INVALID, comp.getName()));
+    }
+    // One and only one of profile OR cpus & memory can be specified. Specifying
+    // both raises validation error.
+    if (StringUtils.isNotEmpty(resource.getProfile())
+        && (resource.getCpus() != null
+            || StringUtils.isNotEmpty(resource.getMemory()))) {
+      throw new IllegalArgumentException(
+          comp == null ? ERROR_RESOURCE_PROFILE_MULTIPLE_VALUES_NOT_SUPPORTED
+              : String.format(
+                  ERROR_RESOURCE_PROFILE_MULTIPLE_VALUES_FOR_COMP_NOT_SUPPORTED,
+                  comp.getName()));
+    }
+    // Currently resource profile is not supported yet, so we will raise
+    // validation error if only resource profile is specified
+    if (StringUtils.isNotEmpty(resource.getProfile())) {
+      throw new IllegalArgumentException(
+          ERROR_RESOURCE_PROFILE_NOT_SUPPORTED_YET);
+    }
+
+    String memory = resource.getMemory();
+    Integer cpus = resource.getCpus();
+    if (StringUtils.isEmpty(memory)) {
+      throw new IllegalArgumentException(
+          comp == null ? ERROR_RESOURCE_MEMORY_INVALID : String.format(
+              ERROR_RESOURCE_MEMORY_FOR_COMP_INVALID, comp.getName()));
+    }
+    if (cpus == null) {
+      throw new IllegalArgumentException(
+          comp == null ? ERROR_RESOURCE_CPUS_INVALID : String.format(
+              ERROR_RESOURCE_CPUS_FOR_COMP_INVALID, comp.getName()));
+    }
+    if (cpus <= 0) {
+      throw new IllegalArgumentException(
+          comp == null ? ERROR_RESOURCE_CPUS_INVALID_RANGE : String.format(
+              ERROR_RESOURCE_CPUS_FOR_COMP_INVALID_RANGE, comp.getName()));
+    }
+  }
+
+  private String createSliderApp(Application application,
+      Map<String, String> compNameArtifactIdMap) throws IOException,
+      YarnException, InterruptedException {
+    final String appName = application.getName();
+    final String queueName = application.getQueue();
+
+    final ActionCreateArgs createArgs = new ActionCreateArgs();
+    addAppConfOptions(createArgs, application, compNameArtifactIdMap);
+    addResourceOptions(createArgs, application);
+    String metainfoJson = getMetainfoJson(application, compNameArtifactIdMap);
+
+    createArgs.appMetaInfoJson = metainfoJson;
+    if (queueName != null && queueName.trim().length() > 0) {
+      createArgs.queue = queueName.trim();
+    }
+
+    return invokeSliderClientRunnable(new SliderClientContextRunnable<String>() {
+      @Override
+      public String run(SliderClient sliderClient) throws YarnException,
+          IOException, InterruptedException {
+        sliderClient.actionCreate(appName, createArgs);
+        ApplicationId applicationId = sliderClient.applicationId;
+        if (applicationId != null) {
+          return applicationId.toString();
+          // return getApplicationIdString(applicationId);
+        }
+        return null;
+      }
+    });
+  }
+
+  private void addAppConfOptions(ActionCreateArgs createArgs,
+      Application application, Map<String, String> compNameArtifactIdMap) throws IOException {
+    List<String> appCompOptionTriples = createArgs.optionsDelegate.compOptTriples; // TODO: optionTuples instead of compOptTriples
+    logger.info("Initial appCompOptionTriples = {}",
+        Arrays.toString(appCompOptionTriples.toArray()));
+    List<String> appOptions = createArgs.optionsDelegate.optionTuples;
+    logger.info("Initial appOptions = {}",
+        Arrays.toString(appOptions.toArray()));
+    // TODO: Set Slider-AM memory and vcores here
+    //    appCompOptionTriples.addAll(Arrays.asList(SLIDER_APPMASTER_COMPONENT_NAME,
+    //        "", ""));
+
+    // Global configuration - for override purpose
+    // TODO: add it to yaml
+    Configuration globalConfig = null;
+    //    Configuration globalConfig = (Configuration) SerializationUtils
+    //        .clone(application.getConfiguration());
+
+    // TODO: Add the below into globalConfig
+    //    if (application.getConfigurations() != null) {
+    //      for (Entry<String, String> entry : application.getConfigurations()
+    //          .entrySet()) {
+    //        globalConf.addProperty(entry.getKey(), entry.getValue());
+    //      }
+    //    }
+
+    Set<String> uniqueGlobalPropertyCache = new HashSet<>();
+    if (application.getConfiguration() != null
+        && application.getConfiguration().getProperties() != null) {
+      for (Map.Entry<String, String> propEntry : application.getConfiguration()
+          .getProperties().entrySet()) {
+        addOptionsIfNotPresent(appOptions, uniqueGlobalPropertyCache,
+            propEntry.getKey(), propEntry.getValue());
+      }
+    }
+    if (application.getComponents() != null) {
+      Map<String, String> placeholders = new HashMap<>();
+      for (Component comp : application.getComponents()) {
+        if (Boolean.TRUE.equals(comp.getUniqueComponentSupport())) {
+          for (int i = 1; i <= comp.getNumberOfContainers(); i++) {
+            placeholders.put(PLACEHOLDER_COMPONENT_ID, Integer.toString(i));
+            appCompOptionTriples.addAll(createAppConfigComponent(
+                comp.getName() + i, comp, comp.getName() + i, globalConfig,
+                placeholders, compNameArtifactIdMap));
+          }
+        } else {
+          appCompOptionTriples.addAll(createAppConfigComponent(comp.getName(),
+              comp, comp.getName(), globalConfig, null, compNameArtifactIdMap));
+        }
+      }
+    }
+
+    logger.info("Updated appCompOptionTriples = {}",
+        Arrays.toString(appCompOptionTriples.toArray()));
+    logger.info("Updated appOptions = {}",
+        Arrays.toString(appOptions.toArray()));
+  }
+
+  private void addOptionsIfNotPresent(List<String> options,
+      Set<String> uniqueGlobalPropertyCache, String key, String value) {
+    if (uniqueGlobalPropertyCache == null) {
+      options.addAll(Arrays.asList(key, value));
+    } else if (!uniqueGlobalPropertyCache.contains(key)) {
+      options.addAll(Arrays.asList(key, value));
+      uniqueGlobalPropertyCache.add(key);
+    }
+  }
+
+  private void addPropertyToConfiguration(Configuration conf, String key,
+      String value) {
+    if (conf == null) {
+      return;
+    }
+    if (conf.getProperties() == null) {
+      conf.setProperties(new HashMap<String, String>());
+    }
+    conf.getProperties().put(key, value);
+  }
+
+  private List<String> createAppConfigComponent(String compName,
+      Component component, String configPrefix, Configuration globalConf,
+      Map<String, String> placeholders,
+      Map<String, String> compNameArtifactIdMap) {
+    List<String> appConfOptTriples = new ArrayList<>();
+
+    if (component.getConfiguration() != null
+        && component.getConfiguration().getProperties() != null) {
+      for (Map.Entry<String, String> propEntry : component.getConfiguration()
+          .getProperties().entrySet()) {
+        appConfOptTriples.addAll(Arrays.asList(compName, propEntry.getKey(),
+            replacePlaceholders(propEntry.getValue(), placeholders)));
+      }
+    }
+
+    // If artifact is of type APPLICATION, then in the POST JSON there will
+    // be no component definition for that artifact. Hence it's corresponding id
+    // field is added. Every external APPLICATION has a unique id field.
+    List<String> convertedDeps = new ArrayList<>();
+    for (String dep : component.getDependencies()) {
+      if (compNameArtifactIdMap.containsKey(dep)) {
+        convertedDeps.add(compNameArtifactIdMap.get(dep));
+      } else {
+        convertedDeps.add(dep);
+      }
+    }
+    // If the DNS dependency property is set to true for a component, it means
+    // that it is ensured that DNS entry has been added for all the containers
+    // of this component, before moving on to the next component in the DAG.
+    if (hasPropertyWithValue(component, PROPERTY_DNS_DEPENDENCY, "true")) {
+      if (component.getArtifact().getType() == Artifact.TypeEnum.APPLICATION) {
+        convertedDeps.add(component.getArtifact().getId());
+      } else {
+        convertedDeps.add(compName);
+      }
+    }
+    if (convertedDeps.size() > 0) {
+      appConfOptTriples.addAll(Arrays.asList(compName, "requires",
+          StringUtils.join(convertedDeps, ",")));
+    }
+    return appConfOptTriples;
+  }
+
+  private String replacePlaceholders(String value,
+      Map<String, String> placeholders) {
+    if (StringUtils.isEmpty(value) || placeholders == null) {
+      return value;
+    }
+    for (Map.Entry<String, String> placeholder : placeholders.entrySet()) {
+      value = value.replaceAll(Pattern.quote(placeholder.getKey()),
+          placeholder.getValue());
+    }
+    return value;
+  }
+
+  private List<String> createAppConfigGlobal(Component component,
+      Configuration globalConf, Set<String> uniqueGlobalPropertyCache) {
+    List<String> appOptions = new ArrayList<>();
+    if (component.getConfiguration() != null
+        && component.getConfiguration().getProperties() != null) {
+      for (Map.Entry<String, String> propEntry : component.getConfiguration()
+          .getProperties().entrySet()) {
+        addOptionsIfNotPresent(appOptions, uniqueGlobalPropertyCache,
+            propEntry.getKey(), propEntry.getValue());
+      }
+    }
+    return appOptions;
+  }
+
+  private void addResourceOptions(ActionCreateArgs createArgs,
+      Application application) throws IOException {
+    List<String> resCompOptionTriples = createArgs.optionsDelegate.resCompOptTriples;
+    logger.info("Initial resCompOptTriples = {}",
+        Arrays.toString(resCompOptionTriples.toArray()));
+    // TODO: Add any Slider AM resource specific props here like jvm.heapsize
+    //    resCompOptionTriples.addAll(Arrays.asList(SLIDER_APPMASTER_COMPONENT_NAME,
+    //        "", ""));
+
+    // Global resource - for override purpose
+    Resource globalResource = (Resource) SerializationUtils.clone(application
+        .getResource());
+    // Priority seeded with 1, expecting every new component will increase it by
+    // 1 making it ready for the next component to use.
+    if (application.getComponents() != null) {
+      int priority = 1;
+      for (Component comp : application.getComponents()) {
+        if (hasPropertyWithValue(comp, PROPERTY_COMPONENT_TYPE,
+            COMPONENT_TYPE_EXTERNAL)) {
+          continue;
+        }
+        if (Boolean.TRUE.equals(comp.getUniqueComponentSupport())) {
+          for (int i = 1; i <= comp.getNumberOfContainers(); i++) {
+            resCompOptionTriples.addAll(createResourcesComponent(comp.getName()
+                + i, comp, priority, 1, globalResource));
+            priority++;
+          }
+        } else {
+          resCompOptionTriples.addAll(createResourcesComponent(comp.getName(),
+              comp, priority, comp.getNumberOfContainers(), globalResource));
+          priority++;
+        }
+      }
+    }
+
+    logger.info("Updated resCompOptTriples = {}",
+        Arrays.toString(resCompOptionTriples.toArray()));
+  }
+
+  private boolean hasPropertyWithValue(Component comp, String key, String value) {
+    if (comp == null || key == null) {
+      return false;
+    }
+    if (comp.getConfiguration() == null
+        || comp.getConfiguration().getProperties() == null) {
+      return false;
+    }
+    Map<String, String> props = comp.getConfiguration().getProperties();
+    if (props.containsKey(key)) {
+      if (value == null) {
+        return props.get(key) == null;
+      } else {
+        if (value.equals(props.get(key))) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  private List<String> createResourcesComponent(String compName,
+      Component component, int priority, long numInstances,
+      Resource globalResource) {
+    String memory = component.getResource() == null ? globalResource
+        .getMemory() : component.getResource().getMemory();
+    Integer cpus = component.getResource() == null ? globalResource.getCpus()
+        : component.getResource().getCpus();
+
+    List<String> resCompOptTriples = new ArrayList<String>();
+    resCompOptTriples.addAll(Arrays.asList(compName,
+        ResourceKeys.COMPONENT_PRIORITY, Integer.toString(priority)));
+    resCompOptTriples.addAll(Arrays.asList(compName,
+        ResourceKeys.COMPONENT_INSTANCES, Long.toString(numInstances)));
+    resCompOptTriples.addAll(Arrays.asList(compName, ResourceKeys.YARN_MEMORY,
+        memory));
+    resCompOptTriples.addAll(Arrays.asList(compName, ResourceKeys.YARN_CORES,
+        cpus.toString()));
+    if (component.getPlacementPolicy() != null) {
+      resCompOptTriples.addAll(Arrays.asList(compName,
+          ResourceKeys.COMPONENT_PLACEMENT_POLICY,
+          component.getPlacementPolicy().getLabel()));
+    }
+
+    return resCompOptTriples;
+  }
+
+  private String getMetainfoJson(Application application,
+      Map<String, String> compNameArtifactIdMap) throws SliderException,
+      IOException {
+    JsonObject rootObj = new JsonObject();
+    rootObj.addProperty("schemaVersion", METAINFO_SCHEMA_VERSION);
+    JsonObject applicationObj = new JsonObject();
+    rootObj.add("application", applicationObj);
+    applicationObj.addProperty("name", application.getName().toUpperCase());
+    JsonArray componentsArray = new JsonArray();
+    applicationObj.add("components", componentsArray);
+    JsonArray commandOrdersArray = new JsonArray();
+    applicationObj.add("commandOrders", commandOrdersArray);
+
+    JsonArray exportGroupsArray = new JsonArray();
+    applicationObj.add("exportGroups", exportGroupsArray);
+    // Use only one export group
+    JsonObject exportGroup = new JsonObject();
+    exportGroup.addProperty("name", EXPORT_GROUP_NAME);
+    exportGroupsArray.add(exportGroup);
+    JsonArray exportsArray = new JsonArray();
+    exportGroup.add("exports", exportsArray);
+
+    if (application.getComponents() != null) {
+
+      // Set exports at application level
+      Map<String, String> appQuicklinks = application.getQuicklinks();
+      Map<String, String> placeholders = new HashMap<>();
+      placeholders.put(PLACEHOLDER_APP_NAME, application.getName());
+      if (appQuicklinks != null) {
+        for (Map.Entry<String, String> quicklink : appQuicklinks.entrySet()) {
+          JsonObject export = new JsonObject();
+          export.addProperty("name", quicklink.getKey());
+          export.addProperty("value",
+              replacePlaceholders(quicklink.getValue(), placeholders));
+          exportsArray.add(export);
+        }
+      }
+
+      for (Component comp : application.getComponents()) {
+        JsonObject compObject = null;
+        if (!hasPropertyWithValue(comp, PROPERTY_COMPONENT_TYPE,
+            COMPONENT_TYPE_EXTERNAL)) {
+          if (Boolean.TRUE.equals(comp.getUniqueComponentSupport())) {
+            for (int i = 1; i <= comp.getNumberOfContainers(); i++) {
+              // we also need the capability to specify ports and mount points
+              // sometime
+              compObject = createMetainfoComponent(comp, application,
+                  comp.getName() + i);
+              componentsArray.add(compObject);
+            }
+          } else {
+            compObject = createMetainfoComponent(comp, application,
+                comp.getName());
+            componentsArray.add(compObject);
+          }
+        }
+
+        // Translate dependencies into command orders
+        List<String> dependencies = comp.getDependencies();
+        if (dependencies != null && !dependencies.isEmpty()) {
+          JsonObject commandOrder = new JsonObject();
+          commandOrder.addProperty("command", comp.getName()
+              + COMMAND_ORDER_SUFFIX_START);
+          for (String dependency : dependencies) {
+            // If APPLICATION type artifact then map component name dependencies
+            // to artifact id
+            if (comp.getArtifact().getType() == Artifact.TypeEnum.APPLICATION) {
+              dependency = compNameArtifactIdMap.get(dependency);
+            }
+            commandOrder.addProperty("requires", dependency
+                + COMMAND_ORDER_SUFFIX_STARTED);
+          }
+          commandOrdersArray.add(commandOrder);
+        }
+
+        // Quicklinks need to be added as appExports and componentExports at the
+        // component level
+        List<String> compQuicklinks = comp.getQuicklinks();
+        if (compQuicklinks != null && !compQuicklinks.isEmpty()) {
+          if (MapUtils.isEmpty(appQuicklinks)) {
+            throw new SliderException(ERROR_QUICKLINKS_FOR_COMP_INVALID);
+          }
+          List<String> appExports = new ArrayList<>();
+          JsonArray compExportsArray = new JsonArray();
+          compObject.add("componentExports", compExportsArray);
+
+          for (String quicklink : compQuicklinks) {
+            appExports.add(EXPORT_GROUP_NAME + "-" + quicklink);
+
+            JsonObject compExport = new JsonObject();
+            compExport.addProperty("name", quicklink);
+            compExport.addProperty("value", appQuicklinks.get(quicklink));
+            compExportsArray.add(compExport);
+          }
+          compObject.addProperty("appExports",
+              StringUtils.join(appExports, ","));
+          // specify that there are published configs for this component
+          compObject.addProperty("publishConfig", "true");
+        }
+      }
+    }
+
+    String jsonString = new GsonBuilder().setPrettyPrinting().create()
+        .toJson(rootObj);
+    logger.info("Metainfo = \n{}", jsonString);
+    return jsonString;
+  }
+
+  private JsonObject createMetainfoComponent(Component comp,
+      Application application, String compName) {
+    JsonObject compObj = new JsonObject();
+    compObj.addProperty("name", compName);
+    // below is diff for each type
+    if (comp.getArtifact() != null && comp.getArtifact().getType() != null
+        && comp.getArtifact().getType() == Artifact.TypeEnum.DOCKER) {
+      compObj.addProperty("type", COMPONENT_TYPE_YARN_DOCKER);
+      JsonArray dockerContainerArray = new JsonArray();
+      compObj.add("dockerContainers", dockerContainerArray);
+      JsonObject dockerContainerObj = new JsonObject();
+      dockerContainerArray.add(dockerContainerObj);
+      dockerContainerObj.addProperty("name", compName.toLowerCase());
+      // if image not specified, then use global value
+      dockerContainerObj.addProperty("image",
+          comp.getArtifact().getId() == null ? application.getArtifact()
+              .getId() : comp.getArtifact().getId());
+      // If launch command not specified, then use global value. Resolve all
+      // placeholders.
+      Map<String, String> placeholders = new HashMap<>();
+      placeholders.put(PLACEHOLDER_APP_NAME, application.getName());
+      placeholders.put(PLACEHOLDER_APP_COMPONENT_NAME, compName);
+      dockerContainerObj.addProperty(
+          "startCommand",
+          comp.getLaunchCommand() == null ? replacePlaceholders(
+              application.getLaunchCommand(), placeholders)
+              : replacePlaceholders(comp.getLaunchCommand(), placeholders));
+      dockerContainerObj.addProperty("network", DEFAULT_NETWORK);
+      dockerContainerObj.addProperty("commandPath", DEFAULT_COMMAND_PATH);
+      // TODO: What to do with privContainer ?
+      dockerContainerObj.addProperty("runPrivilegedContainer",
+          comp.getRunPrivilegedContainer());
+      if (comp.getConfiguration() != null) {
+        List<ConfigFile> configFiles = comp.getConfiguration().getFiles();
+        if (configFiles != null && !configFiles.isEmpty()) {
+          JsonArray configFileArray = new JsonArray();
+          for (ConfigFile configFile : configFiles) {
+            JsonObject configFileObj = new JsonObject();
+            configFileObj.addProperty("type", configFile.getType().toString());
+            configFileObj.addProperty("fileName", configFile.getDestFile());
+            // TODO: add all properties which should include dictionaryName
+            configFileObj.addProperty("dictionaryName",
+                configFile.getSrcFile());
+            configFileArray.add(configFileObj);
+          }
+          dockerContainerObj.add("configFiles", configFileArray);
+        }
+      }
+      // we also need to specify artifact_management_service sometime
+    }
+    // we also need the capability to specify ports and mount points sometime
+    return compObj;
+  }
+
+  private static UserGroupInformation getSliderUser() {
+    if (SLIDER_USER != null) {
+      return SLIDER_USER;
+    }
+    UserGroupInformation sliderUser = null;
+    UserGroupInformation.setConfiguration(SLIDER_CONFIG);
+    String loggedInUser = getUserToRunAs();
+    try {
+      sliderUser = UserGroupInformation.getBestUGI(null, loggedInUser);
+      // TODO: Once plugged into RM process we should remove the previous call
+      // and replace it with getCurrentUser as commented below.
+      // sliderUser = UserGroupInformation.getCurrentUser();
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to create UGI (slider user)", e);
+    }
+    return sliderUser;
+  }
+
+  private <T> T invokeSliderClientRunnable(
+      final SliderClientContextRunnable<T> runnable)
+      throws IOException, InterruptedException, YarnException {
+    try {
+      T value = SLIDER_USER.doAs(new PrivilegedExceptionAction<T>() {
+        @Override
+        public T run() throws Exception {
+          return runnable.run(SLIDER_CLIENT);
+        }
+      });
+      return value;
+    } catch (UndeclaredThrowableException e) {
+      Throwable cause = e.getCause();
+      if (cause instanceof YarnException) {
+        YarnException ye = (YarnException) cause;
+        throw ye;
+      }
+      throw e;
+    }
+  }
+
+  protected static SliderClient createSliderClient() {
+    if (SLIDER_CLIENT != null) {
+      return SLIDER_CLIENT;
+    }
+    org.apache.hadoop.conf.Configuration sliderClientConfiguration = SLIDER_CONFIG;
+    SliderClient client = new SliderClient() {
+      @Override
+      public void init(org.apache.hadoop.conf.Configuration conf) {
+        super.init(conf);
+        try {
+          initHadoopBinding();
+        } catch (SliderException e) {
+          throw new RuntimeException(
+              "Unable to automatically init Hadoop binding", e);
+        } catch (IOException e) {
+          throw new RuntimeException(
+              "Unable to automatically init Hadoop binding", e);
+        }
+      }
+    };
+    try {
+      logger
+          .debug("Slider Client configuration: {}", sliderClientConfiguration);
+      sliderClientConfiguration = client.bindArgs(sliderClientConfiguration,
+          new String[] { "help" });
+      client.init(sliderClientConfiguration);
+      client.start();
+    } catch (Exception e) {
+      logger.error("Unable to create SliderClient", e);
+      throw new RuntimeException(e.getMessage(), e);
+    }
+    return client;
+  }
+
+  private static String getUserToRunAs() {
+    String user = System.getenv(PROPERTY_APP_RUNAS_USER);
+    if (StringUtils.isEmpty(user)) {
+      user = "root";
+    }
+    return user;
+  }
+
+  private static org.apache.hadoop.conf.Configuration getSliderClientConfiguration() {
+    if (SLIDER_CONFIG != null) {
+      return SLIDER_CONFIG;
+    }
+    YarnConfiguration yarnConfig = new YarnConfiguration();
+    logger.info("prop yarn.resourcemanager.address = {}",
+        yarnConfig.get("yarn.resourcemanager.address"));
+
+    return yarnConfig;
+  }
+
+  private interface SliderClientContextRunnable<T> {
+    T run(SliderClient sliderClient)
+        throws YarnException, IOException, InterruptedException;
+  }
+
+  @GET
+  @Consumes({ MediaType.APPLICATION_JSON })
+  @Produces({ MediaType.APPLICATION_JSON })
+  public Response getApplications(@QueryParam("state") String state) {
+    logger.info("GET: getApplications with param state = {}", state);
+
+    // Get all applications in a specific state - lighter projection. For full
+    // detail, call getApplication on a specific app.
+    Set<String> applications;
+    try {
+      if (StringUtils.isNotEmpty(state)) {
+        ApplicationStatus appStatus = new ApplicationStatus();
+        try {
+          ApplicationState.valueOf(state);
+        } catch (IllegalArgumentException e) {
+          appStatus.setDiagnostics("Invalid value for param state - " + state);
+          return Response.status(Status.BAD_REQUEST).entity(appStatus).build();
+        }
+        applications = getSliderApplications(state);
+      } else {
+        applications = getSliderApplications(true);
+      }
+    } catch (Exception e) {
+      logger.error("Get applications failed", e);
+      return Response.status(Status.INTERNAL_SERVER_ERROR).build();
+    }
+
+    Set<Application> apps = new HashSet<Application>();
+    if (applications.size() > 0) {
+      try {
+        for (String app : applications) {
+          Application application = new Application();
+          // TODO: Need to get lifetime, launch-time and privileged container
+          // status from YARN
+          application.setLifetime(null);
+          application.setLaunchTime(new Date());
+          application.setName(app);
+          // Containers not required, setting to null to avoid empty list
+          application.setContainers(null);
+          apps.add(application);
+        }
+      } catch (Exception e) {
+        logger.error("Get applications failed", e);
+        return Response.status(Status.INTERNAL_SERVER_ERROR).build();
+      }
+    }
+
+    return Response.ok().entity(apps).build();
+  }
+
+  @GET
+  @Path("/{app_name}")
+  @Consumes({ MediaType.APPLICATION_JSON })
+  @Produces({ MediaType.APPLICATION_JSON })
+  public Response getApplication(@PathParam("app_name") String appName) {
+    logger.info("GET: getApplication for appName = {}", appName);
+
+    // app name validation
+    if (!SliderUtils.isClusternameValid(appName)) {
+      ApplicationStatus applicationStatus = new ApplicationStatus();
+      applicationStatus.setDiagnostics("Invalid application name");
+      applicationStatus.setCode(ERROR_CODE_APP_NAME_INVALID);
+      return Response.status(Status.NOT_FOUND).entity(applicationStatus)
+          .build();
+    }
+
+    // Check if app exists
+    try {
+      int livenessCheck = getSliderList(appName);
+      if (livenessCheck < 0) {
+        logger.info("Application not running");
+        ApplicationStatus applicationStatus = new ApplicationStatus();
+        applicationStatus.setDiagnostics(ERROR_APPLICATION_NOT_RUNNING);
+        applicationStatus.setCode(ERROR_CODE_APP_IS_NOT_RUNNING);
+        return Response.status(Status.NOT_FOUND).entity(applicationStatus)
+            .build();
+      }
+    } catch (UnknownApplicationInstanceException e) {
+      logger.error("Get application failed, application not found", e);
+      ApplicationStatus applicationStatus = new ApplicationStatus();
+      applicationStatus.setDiagnostics(ERROR_APPLICATION_DOES_NOT_EXIST);
+      applicationStatus.setCode(ERROR_CODE_APP_DOES_NOT_EXIST);
+      return Response.status(Status.NOT_FOUND).entity(applicationStatus)
+          .build();
+    } catch (Exception e) {
+      logger.error("Get application failed, application not running", e);
+      ApplicationStatus applicationStatus = new ApplicationStatus();
+      applicationStatus.setDiagnostics(ERROR_APPLICATION_NOT_RUNNING);
+      applicationStatus.setCode(ERROR_CODE_APP_IS_NOT_RUNNING);
+      return Response.status(Status.NOT_FOUND).entity(applicationStatus)
+          .build();
+    }
+
+    Application app = new Application();
+    app.setName(appName);
+    app.setUri(CONTEXT_ROOT + APPLICATIONS_API_RESOURCE_PATH + "/"
+        + appName);
+    // TODO: add status
+    app.setState(ApplicationState.ACCEPTED);
+    JsonObject appStatus = null;
+    JsonObject appRegistryDocker = null;
+    JsonObject appRegistryQuicklinks = null;
+    try {
+      appStatus = getSliderApplicationStatus(appName);
+      appRegistryDocker = getSliderApplicationRegistry(appName, "docker");
+      appRegistryQuicklinks = getSliderApplicationRegistry(appName,
+          "quicklinks");
+      return populateAppData(app, appStatus, appRegistryDocker,
+          appRegistryQuicklinks);
+    } catch (BadClusterStateException | NotFoundException e) {
+      logger.error(
+          "Get application failed, application not in running state yet", e);
+      ApplicationStatus applicationStatus = new ApplicationStatus();
+      applicationStatus.setDiagnostics("Application not running yet");
+      applicationStatus.setCode(ERROR_CODE_APP_SUBMITTED_BUT_NOT_RUNNING_YET);
+      return Response.status(Status.NOT_FOUND).entity(applicationStatus)
+          .build();
+    } catch (Exception e) {
+      logger.error("Get application failed", e);
+      ApplicationStatus applicationStatus = new ApplicationStatus();
+      applicationStatus.setDiagnostics("Failed to retrieve application: "
+          + e.getMessage());
+      return Response.status(Status.INTERNAL_SERVER_ERROR)
+          .entity(applicationStatus).build();
+    }
+  }
+
+  private Response populateAppData(Application app, JsonObject appStatus,
+      JsonObject appRegistryDocker, JsonObject appRegistryQuicklinks) {
+    String appName = jsonGetAsString(appStatus, "name");
+    Long totalNumberOfRunningContainers = 0L;
+    Long totalExpectedNumberOfRunningContainers = 0L;
+    Long totalNumberOfIpAssignedContainers = 0L;
+
+    // info
+    JsonObject applicationInfo = jsonGetAsObject(appStatus, "info");
+    if (applicationInfo != null) {
+      String applicationId = jsonGetAsString(applicationInfo, "info.am.app.id");
+      if (applicationId != null) {
+        app.setId(applicationId);
+      }
+    }
+
+    // state
+    String appState = jsonGetAsString(appStatus, "state");
+    switch (Integer.parseInt(appState)) {
+      case StateValues.STATE_LIVE:
+        app.setState(ApplicationState.STARTED);
+        break;
+      case StateValues.STATE_CREATED:
+      case StateValues.STATE_INCOMPLETE:
+      case StateValues.STATE_SUBMITTED:
+        app.setState(ApplicationState.ACCEPTED);
+        return Response.ok(app).build();
+      case StateValues.STATE_DESTROYED:
+      case StateValues.STATE_STOPPED:
+        app.setState(ApplicationState.STOPPED);
+        return Response.ok(app).build();
+      default:
+        break;
+    }
+
+    // start time
+    app.setLaunchTime(appStatus.get("createTime") == null ? null
+        : new Date(appStatus.get("createTime").getAsLong()));
+
+    // lifetime - set it to unlimited for now
+    // TODO: Once YARN-3813 and YARN-4205 are available - get it from YARN
+    app.setLifetime(DEFAULT_UNLIMITED_LIFETIME);
+
+    // Quicklinks
+    Map<String, String> appQuicklinks = new HashMap<>();
+    for (Map.Entry<String, JsonElement> quicklink : appRegistryQuicklinks
+        .entrySet()) {
+      appQuicklinks.put(quicklink.getKey(), quicklink.getValue() == null ? null
+          : quicklink.getValue().getAsString());
+    }
+    if (!appQuicklinks.isEmpty()) {
+      app.setQuicklinks(appQuicklinks);
+    }
+
+    ArrayList<String> componentNames = new ArrayList<>();
+
+    // status.live
+    JsonObject applicationStatus = jsonGetAsObject(appStatus, "status");
+    // roles
+    JsonObject applicationRoles = jsonGetAsObject(appStatus, "roles");
+    // statistics
+    JsonObject applicationStatistics = jsonGetAsObject(appStatus, "statistics");
+    if (applicationRoles == null) {
+      // initialize to empty object to avoid too many null checks
+      applicationRoles = new JsonObject();
+    }
+    if (applicationStatus != null) {
+      JsonObject applicationLive = jsonGetAsObject(applicationStatus, "live");
+      if (applicationLive != null) {
+        for (Entry<String, JsonElement> entry : applicationLive.entrySet()) {
+          if (entry.getKey().equals(SLIDER_APPMASTER_COMPONENT_NAME)) {
+            continue;
+          }
+          componentNames.add(entry.getKey());
+          JsonObject componentRole = applicationRoles.get(entry.getKey()) == null ? new JsonObject()
+              : applicationRoles.get(entry.getKey()).getAsJsonObject();
+          JsonObject liveContainers = entry.getValue().getAsJsonObject();
+          if (liveContainers != null) {
+            for (Map.Entry<String, JsonElement> liveContainerEntry : liveContainers
+                .entrySet()) {
+              String containerId = liveContainerEntry.getKey();
+              Container container = new Container();
+              container.setId(containerId);
+              JsonObject liveContainer = (JsonObject) liveContainerEntry
+                  .getValue();
+              container
+                  .setLaunchTime(liveContainer.get("startTime") == null ? null
+                      : new Date(liveContainer.get("startTime").getAsLong()));
+              container
+                  .setComponentName(jsonGetAsString(liveContainer, "role"));
+              container.setIp(jsonGetAsString(liveContainer, "ip"));
+              // If ip is non-null increment count
+              if (container.getIp() != null) {
+                totalNumberOfIpAssignedContainers++;
+              }
+              container.setHostname(jsonGetAsString(liveContainer, "hostname"));
+              container.setState(ContainerState.INIT);
+              if (StringUtils.isNotEmpty(container.getIp())
+                  && StringUtils.isNotEmpty(container.getHostname())) {
+                container.setState(ContainerState.READY);
+              }
+              container.setBareHost(jsonGetAsString(liveContainer, "host"));
+              container.setUri(CONTEXT_ROOT + APPLICATIONS_API_RESOURCE_PATH
+                  + "/" + appName + CONTAINERS_API_RESOURCE_PATH + "/"
+                  + containerId);
+              Resource resource = new Resource();
+              resource.setCpus(jsonGetAsInt(componentRole, "yarn.vcores"));
+              resource.setMemory(jsonGetAsString(componentRole, "yarn.memory"));
+              container.setResource(resource);
+              // TODO: add container property - for response only?
+              app.addContainer(container);
+            }
+          }
+        }
+      }
+    }
+
+    // application info
+    if (applicationRoles != null && !componentNames.isEmpty()) {
+      JsonObject applicationRole = jsonGetAsObject(applicationRoles,
+          componentNames.get(0));
+      if (applicationRole != null) {
+        Artifact artifact = new Artifact();
+        // how to get artifact id - docker image name??
+        artifact.setId(null);
+      }
+    }
+
+    // actual and expected number of containers
+    if (applicationStatistics != null) {
+      for (Entry<String, JsonElement> entry : applicationStatistics.entrySet()) {
+        if (entry.getKey().equals(SLIDER_APPMASTER_COMPONENT_NAME)) {
+          continue;
+        }
+        JsonObject containerStats = (JsonObject) entry.getValue();
+        totalNumberOfRunningContainers += jsonGetAsInt(containerStats,
+            "containers.live");
+        totalExpectedNumberOfRunningContainers += jsonGetAsInt(containerStats,
+            "containers.desired");
+      }
+      app.setNumberOfContainers(totalExpectedNumberOfRunningContainers);
+      app.setNumberOfRunningContainers(totalNumberOfRunningContainers);
+    }
+
+    // If all containers of the app has IP assigned, then according to the REST
+    // API it is considered to be READY. Note, application readiness from
+    // end-users point of view, is out of scope of the REST API. Also, this
+    // readiness has nothing to do with readiness-check defined at the component
+    // level (which is used for dependency resolution of component DAG).
+    if (totalNumberOfIpAssignedContainers == totalExpectedNumberOfRunningContainers) {
+      app.setState(ApplicationState.READY);
+    }
+    logger.info("Application = {}", app);
+    return Response.ok(app).build();
+  }
+
+  private String jsonGetAsString(JsonObject object, String key) {
+    return object.get(key) == null ? null : object.get(key).getAsString();
+  }
+
+  private Integer jsonGetAsInt(JsonObject object, String key) {
+    return object.get(key) == null ? null
+        : object.get(key).isJsonNull() ? null : object.get(key).getAsInt();
+  }
+
+  private JsonObject jsonGetAsObject(JsonObject object, String key) {
+    return object.get(key) == null ? null : object.get(key).getAsJsonObject();
+  }
+
+  private JsonObject getSliderApplicationStatus(final String appName)
+      throws IOException, YarnException, InterruptedException {
+    final File appStatusOutputFile = File.createTempFile("status_", ".json");
+    final ActionStatusArgs statusArgs = new ActionStatusArgs();
+    statusArgs.output = appStatusOutputFile.getAbsolutePath();
+
+    return invokeSliderClientRunnable(new SliderClientContextRunnable<JsonObject>() {
+      @Override
+      public JsonObject run(SliderClient sliderClient) throws YarnException,
+          IOException, InterruptedException {
+        sliderClient.actionStatus(appName, statusArgs);
+        JsonParser parser = new JsonParser();
+        FileReader reader = null;
+        JsonElement statusElement = null;
+        try {
+          reader = new FileReader(appStatusOutputFile);
+          statusElement = parser.parse(reader);
+        } finally {
+          if (reader != null) {
+            reader.close();
+          }
+          appStatusOutputFile.delete();
+        }
+        return (statusElement == null || statusElement instanceof JsonNull) ?
+            new JsonObject() : (JsonObject) statusElement;
+      }
+    });
+  }
+
+  private JsonObject getSliderApplicationRegistry(final String appName,
+      final String registryName) throws IOException, YarnException,
+      InterruptedException {
+    final File appRegistryOutputFile = File
+        .createTempFile("registry_", ".json");
+    final ActionRegistryArgs registryArgs = new ActionRegistryArgs();
+    registryArgs.out = appRegistryOutputFile;
+    registryArgs.name = appName;
+    registryArgs.getConf = registryName;
+    registryArgs.format = ConfigFormat.JSON.toString();
+
+    return invokeSliderClientRunnable(new SliderClientContextRunnable<JsonObject>() {
+      @Override
+      public JsonObject run(SliderClient sliderClient) throws YarnException,
+          IOException, InterruptedException {
+        sliderClient.actionRegistry(registryArgs);
+        JsonParser parser = new JsonParser();
+        FileReader reader = null;
+        JsonElement registryElement = null;
+        try {
+          reader = new FileReader(appRegistryOutputFile);
+          registryElement = parser.parse(reader);
+        } catch (Throwable t) {
+          logger.error("Error reading file {}", appRegistryOutputFile);
+        } finally {
+          if (reader != null) {
+            reader.close();
+          }
+          appRegistryOutputFile.delete();
+        }
+        return (registryElement == null || registryElement instanceof JsonNull) ?
+            new JsonObject() : (JsonObject) registryElement;
+      }
+    });
+  }
+
+  private Integer getSliderList(final String appName)
+      throws IOException, YarnException, InterruptedException {
+    return getSliderList(appName, true);
+  }
+
+  private Integer getSliderList(final String appName, final boolean liveOnly)
+      throws IOException, YarnException, InterruptedException {
+    return invokeSliderClientRunnable(new SliderClientContextRunnable<Integer>() {
+      @Override
+      public Integer run(SliderClient sliderClient) throws YarnException,
+          IOException, InterruptedException {
+        int status = 0;
+        if (liveOnly) {
+          status = sliderClient.actionList(appName);
+        } else {
+          ActionListArgs listArgs = new ActionListArgs();
+          status = sliderClient.actionList(appName, listArgs);
+        }
+        return status;
+      }
+    });
+  }
+
+  private Set<String> getSliderApplications(final String state)
+      throws IOException, YarnException, InterruptedException {
+    return getSliderApplications(false, state);
+  }
+
+  private Set<String> getSliderApplications(final boolean liveOnly)
+      throws IOException, YarnException, InterruptedException {
+    return getSliderApplications(liveOnly, null);
+  }
+
+  private Set<String> getSliderApplications(final boolean liveOnly,
+      final String state) throws IOException, YarnException,
+      InterruptedException {
+    return invokeSliderClientRunnable(new SliderClientContextRunnable<Set<String>>() {
+      @Override
+      public Set<String> run(SliderClient sliderClient) throws YarnException,
+          IOException, InterruptedException {
+        Set<String> apps;
+        ActionListArgs listArgs = new ActionListArgs();
+        if (liveOnly) {
+          apps = sliderClient.getApplicationList(null);
+        } else if (StringUtils.isNotEmpty(state)) {
+          listArgs.state = state;
+          apps = sliderClient.getApplicationList(null, listArgs);
+        } else {
+          apps = sliderClient.getApplicationList(null, listArgs);
+        }
+        return apps;
+      }
+    });
+  }
+
+  @DELETE
+  @Path("/{app_name}")
+  @Consumes({ MediaType.APPLICATION_JSON })
+  @Produces({ MediaType.APPLICATION_JSON })
+  public Response deleteApplication(@PathParam("app_name") String appName) {
+    logger.info("DELETE: deleteApplication for appName = {}", appName);
+
+    try {
+      Response stopResponse = stopSliderApplication(appName);
+      if (stopResponse.getStatus() == Status.INTERNAL_SERVER_ERROR
+          .getStatusCode()) {
+        return Response.status(Status.NOT_FOUND).build();
+      }
+    } catch (UnknownApplicationInstanceException e) {
+      logger.error("Application does not exist", e);
+      return Response.status(Status.NOT_FOUND).build();
+    } catch (Exception e) {
+      logger.error("Delete application failed", e);
+      return Response.status(Status.INTERNAL_SERVER_ERROR).build();
+    }
+
+    // Although slider client stop returns immediately, it usually takes a
+    // little longer for it to stop from YARN point of view. Slider destroy
+    // fails if the application is not completely stopped. Hence the need to
+    // call destroy in a controlled loop few times (only if exit code is
+    // EXIT_APPLICATION_IN_USE), before giving up.
+    boolean keepTrying = true;
+    int maxDeleteAttempt = 5;
+    int deleteAttempt = 0;
+    while (keepTrying && deleteAttempt < maxDeleteAttempt) {
+      try {
+        destroySliderApplication(appName);
+        keepTrying = false;
+      } catch (SliderException e) {
+        logger.error("Delete application threw exception", e);
+        if (e.getExitCode() == SliderExitCodes.EXIT_APPLICATION_IN_USE) {
+          deleteAttempt++;
+          try {
+            Thread.sleep(500);
+          } catch (InterruptedException e1) {
+          }
+        } else {
+          return Response.status(Status.INTERNAL_SERVER_ERROR).build();
+        }
+      } catch (Exception e) {
+        logger.error("Delete application failed", e);
+        return Response.status(Status.INTERNAL_SERVER_ERROR).build();
+      }
+    }
+    return Response.status(Status.NO_CONTENT).build();
+  }
+
+  private Response stopSliderApplication(final String appName)
+      throws IOException, YarnException, InterruptedException {
+    return invokeSliderClientRunnable(new SliderClientContextRunnable<Response>() {
+      @Override
+      public Response run(SliderClient sliderClient) throws YarnException,
+          IOException, InterruptedException {
+        ActionFreezeArgs freezeArgs = new ActionFreezeArgs();
+        int returnCode = sliderClient.actionFreeze(appName, freezeArgs);
+        if (returnCode == 0) {
+          logger.info("Successfully stopped application {}", appName);
+          return Response.status(Status.NO_CONTENT).build();
+        } else {
+          logger.error("Stop of application {} failed with return code ",
+              appName, returnCode);
+          ApplicationStatus applicationStatus = new ApplicationStatus();
+          applicationStatus.setDiagnostics("Stop of application " + appName
+              + " failed");
+          return Response.status(Status.INTERNAL_SERVER_ERROR)
+              .entity(applicationStatus).build();
+        }
+      }
+    });
+  }
+
+  private Response startSliderApplication(final String appName)
+      throws IOException, YarnException, InterruptedException {
+    return invokeSliderClientRunnable(new SliderClientContextRunnable<Response>() {
+      @Override
+      public Response run(SliderClient sliderClient) throws YarnException,
+          IOException, InterruptedException {
+        ActionThawArgs thawArgs = new ActionThawArgs();
+        int returnCode = sliderClient.actionThaw(appName, thawArgs);
+        if (returnCode == 0) {
+          logger.info("Successfully started application {}", appName);
+          ApplicationStatus applicationStatus = new ApplicationStatus();
+          applicationStatus.setState(ApplicationState.ACCEPTED);
+          applicationStatus.setUri(CONTEXT_ROOT
+              + APPLICATIONS_API_RESOURCE_PATH + "/" + appName);
+          // 202 = ACCEPTED
+          return Response.status(HTTP_STATUS_CODE_ACCEPTED)
+              .entity(applicationStatus).build();
+        } else {
+          logger.error("Start of application {} failed with returnCode ",
+              appName, returnCode);
+          ApplicationStatus applicationStatus = new ApplicationStatus();
+          applicationStatus.setDiagnostics("Start of application " + appName
+              + " failed");
+          return Response.status(Status.INTERNAL_SERVER_ERROR)
+              .entity(applicationStatus).build();
+        }
+      }
+    });
+  }
+
+  private Void destroySliderApplication(final String appName)
+      throws IOException, YarnException, InterruptedException {
+    return invokeSliderClientRunnable(new SliderClientContextRunnable<Void>() {
+      @Override
+      public Void run(SliderClient sliderClient) throws YarnException,
+          IOException, InterruptedException {
+        sliderClient.actionDestroy(appName);
+        return null;
+      }
+    });
+  }
+
+  @PUT
+  @Path("/{app_name}")
+  @Consumes({ MediaType.APPLICATION_JSON })
+  @Produces({ MediaType.APPLICATION_JSON })
+  public Response updateApplication(@PathParam("app_name") String appName,
+      Application updateAppData) {
+    logger.info("PUT: updateApplication for app = {} with data = {}", appName,
+        updateAppData);
+
+    // Ignore the app name provided in updateAppData and always use appName
+    // path param
+    updateAppData.setName(appName);
+
+    // Adding support for stop and start
+    // For STOP the app should be running. If already stopped then this
+    // operation will be a no-op. For START it should be in stopped state.
+    // If already running then this operation will be a no-op.
+
+    // Check if app exists in any state
+    try {
+      int appsFound = getSliderList(appName, false);
+      if (appsFound < 0) {
+        return Response.status(Status.NOT_FOUND).build();
+      }
+    } catch (Exception e) {
+      logger.error("Update application failed", e);
+      return Response.status(Status.NOT_FOUND).build();
+    }
+
+    // If a STOP is requested
+    if (updateAppData.getState() != null
+        && updateAppData.getState() == ApplicationState.STOPPED) {
+      try {
+        int livenessCheck = getSliderList(appName);
+        if (livenessCheck == 0) {
+          return stopSliderApplication(appName);
+        } else {
+          logger.info("Application {} is already stopped", appName);
+          ApplicationStatus applicationStatus = new ApplicationStatus();
+          applicationStatus.setDiagnostics("Application " + appName
+              + " is already stopped");
+          return Response.status(Status.BAD_REQUEST).entity(applicationStatus)
+              .build();
+        }
+      } catch (Exception e) {
+        logger.error("Stop application failed", e);
+        return Response.status(Status.INTERNAL_SERVER_ERROR).build();
+      }
+    }
+
+    // If a START is requested
+    if (updateAppData.getState() != null
+        && updateAppData.getState() == ApplicationState.STARTED) {
+      try {
+        int livenessCheck = getSliderList(appName);
+        if (livenessCheck != 0) {
+          return startSliderApplication(appName);
+        } else {
+          logger.info("Application {} is already running", appName);
+          ApplicationStatus applicationStatus = new ApplicationStatus();
+          applicationStatus.setDiagnostics("Application " + appName
+              + " is already running");
+          applicationStatus.setUri(CONTEXT_ROOT
+              + APPLICATIONS_API_RESOURCE_PATH + "/" + appName);
+          return Response.status(Status.BAD_REQUEST).entity(applicationStatus)
+              .build();
+        }
+      } catch (Exception e) {
+        logger.error("Start application failed", e);
+        return Response.status(Status.INTERNAL_SERVER_ERROR).build();
+      }
+    }
+
+    // If no of instances specified then treat it as a flex
+    if (updateAppData.getNumberOfContainers() != null
+        && updateAppData.getComponents() == null) {
+      updateAppData.setComponents(getDefaultComponentAsList());
+    }
+
+    // At this point if there are components then it is a flex
+    if (updateAppData.getComponents() != null) {
+      try {
+        int livenessCheck = getSliderList(appName);
+        if (livenessCheck == 0) {
+          flexSliderApplication(appName, updateAppData);
+        }
+        return Response.status(Status.NO_CONTENT).build();
+      } catch (Exception e) {
+        logger.error("Update application failed", e);
+        return Response.status(Status.INTERNAL_SERVER_ERROR).build();
+      }
+    }
+
+    // If new lifetime value specified then update it
+    if (updateAppData.getLifetime() != null) {
+      // TODO: Once YARN-3813 and YARN-4205 are available
+    }
+
+    // If nothing happens consider it a no-op
+    return Response.status(Status.NO_CONTENT).build();
+  }
+
+  private List<Component> getDefaultComponentAsList() {
+    Component comp = new Component();
+    comp.setName(DEFAULT_COMPONENT_NAME);
+    List<Component> comps = new ArrayList<>();
+    comps.add(comp);
+    return comps;
+  }
+
+  private Void flexSliderApplication(final String appName,
+      final Application updateAppData) throws IOException, YarnException,
+      InterruptedException {
+    return invokeSliderClientRunnable(new SliderClientContextRunnable<Void>() {
+      @Override
+      public Void run(SliderClient sliderClient) throws YarnException,
+          IOException, InterruptedException {
+        ActionFlexArgs flexArgs = new ActionFlexArgs();
+        ComponentArgsDelegate compDelegate = new ComponentArgsDelegate();
+        Long globalNumberOfContainers = updateAppData.getNumberOfContainers();
+        for (Component comp : updateAppData.getComponents()) {
+          Long noOfContainers = comp.getNumberOfContainers() == null
+              ? globalNumberOfContainers : comp.getNumberOfContainers();
+          if (noOfContainers != null) {
+            compDelegate.componentTuples.addAll(
+                Arrays.asList(comp.getName(), String.valueOf(noOfContainers)));
+          }
+        }
+        if (!compDelegate.componentTuples.isEmpty()) {
+          flexArgs.componentDelegate = compDelegate;
+          sliderClient.actionFlex(appName, flexArgs);
+        }
+        return null;
+      }
+    });
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[03/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/CertificateManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/CertificateManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/CertificateManager.java
new file mode 100644
index 0000000..e436ae9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/CertificateManager.java
@@ -0,0 +1,495 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.services.security;
+
+import com.google.inject.Singleton;
+import org.apache.commons.io.FileUtils;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.exceptions.SliderException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.nio.charset.Charset;
+import java.text.MessageFormat;
+
+@Singleton
+public class CertificateManager {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CertificateManager.class);
+
+  private static final String GEN_SRVR_KEY = "openssl genrsa -des3 " +
+      "-passout pass:{0} -out {1}" + File.separator + "{2} 4096 ";
+  private static final String GEN_SRVR_REQ = "openssl req -passin pass:{0} " +
+      "-new -key {1}" + File.separator + "{2} -out {1}" + File.separator +
+      "{5} -config {1}" + File.separator + "ca.config " +
+      "-subj {6} -batch";
+  private static final String SIGN_SRVR_CRT = "openssl ca -create_serial " +
+    "-out {1}" + File.separator + "{3} -days 365 -keyfile {1}" + File.separator
+    + "{2} -key {0} -selfsign -extensions jdk7_ca -config {1}" + File.separator
+    + "ca.config -batch -infiles {1}" + File.separator + "{5}";
+  private static final String EXPRT_KSTR = "openssl pkcs12 -export" +
+      " -in {2}" + File.separator + "{4} -inkey {2}" + File.separator +
+      "{3} -certfile {2}" + File.separator + "{4} -out {2}" + File.separator +
+      "{5} -password pass:{1} -passin pass:{0} \n";
+  private static final String REVOKE_AGENT_CRT = "openssl ca " +
+      "-config {0}" + File.separator + "ca.config -keyfile {0}" +
+      File.separator + "{4} -revoke {0}" + File.separator + "{2} -batch " +
+      "-passin pass:{3} -cert {0}" + File.separator + "{5}";
+  private static final String SIGN_AGENT_CRT = "openssl ca -config " +
+      "{0}" + File.separator + "ca.config -in {0}" + File.separator +
+      "{1} -out {0}" + File.separator + "{2} -batch -passin pass:{3} " +
+      "-keyfile {0}" + File.separator + "{4} -cert {0}" + File.separator + "{5}";
+  private static final String GEN_AGENT_KEY="openssl req -new -newkey " +
+      "rsa:1024 -nodes -keyout {0}" + File.separator +
+      "{2}.key -subj {1} -out {0}" + File.separator + "{2}.csr " +
+      "-config {3}" + File.separator + "ca.config ";
+  private String passphrase;
+  private String applicationName;
+
+
+  public void initialize(MapOperations compOperations) throws SliderException {
+    String hostname = null;
+    try {
+      hostname = InetAddress.getLocalHost().getCanonicalHostName();
+    } catch (UnknownHostException e) {
+      hostname = "localhost";
+    }
+    this.initialize(compOperations, hostname, null, null);
+  }
+
+  /**
+    * Verify that root certificate exists, generate it otherwise.
+    */
+  public void initialize(MapOperations compOperations,
+                         String hostname, String containerId,
+                         String appName) throws SliderException {
+    SecurityUtils.initializeSecurityParameters(compOperations);
+
+    LOG.info("Initialization of root certificate");
+    boolean certExists = isCertExists();
+    LOG.info("Certificate exists:" + certExists);
+
+    this.applicationName = appName;
+
+    if (!certExists) {
+      generateAMKeystore(hostname, containerId);
+    }
+
+  }
+
+  /**
+   * Checks root certificate state.
+   * @return "true" if certificate exists
+   */
+  private boolean isCertExists() {
+
+    String srvrKstrDir = SecurityUtils.getSecurityDir();
+    String srvrCrtName = SliderKeys.CRT_FILE_NAME;
+    File certFile = new File(srvrKstrDir + File.separator + srvrCrtName);
+    LOG.debug("srvrKstrDir = " + srvrKstrDir);
+    LOG.debug("srvrCrtName = " + srvrCrtName);
+    LOG.debug("certFile = " + certFile.getAbsolutePath());
+
+    return certFile.exists();
+  }
+
+  public void setPassphrase(String passphrase) {
+    this.passphrase = passphrase;
+  }
+
+  class StreamConsumer extends Thread
+  {
+    InputStream is;
+    boolean logOutput;
+
+    StreamConsumer(InputStream is, boolean logOutput)
+    {
+      this.is = is;
+      this.logOutput = logOutput;
+    }
+
+    StreamConsumer(InputStream is)
+    {
+      this(is, false);
+    }
+
+    public void run()
+    {
+      try
+      {
+        InputStreamReader isr = new InputStreamReader(is,
+                                                      Charset.forName("UTF8"));
+        BufferedReader br = new BufferedReader(isr);
+        String line;
+        while ( (line = br.readLine()) != null)
+          if (logOutput) {
+            LOG.info(line);
+          }
+      } catch (IOException e)
+      {
+        LOG.error("Error during processing of process stream", e);
+      }
+    }
+  }
+
+
+  /**
+   * Runs os command
+   *
+   * @return command execution exit code
+   */
+  private int runCommand(String command) throws SliderException {
+    int exitCode = -1;
+    String line = null;
+    Process process = null;
+    BufferedReader br= null;
+    try {
+      process = Runtime.getRuntime().exec(command);
+      StreamConsumer outputConsumer =
+          new StreamConsumer(process.getInputStream(), true);
+      StreamConsumer errorConsumer =
+          new StreamConsumer(process.getErrorStream(), true);
+
+      outputConsumer.start();
+      errorConsumer.start();
+
+      try {
+        process.waitFor();
+        SecurityUtils.logOpenSslExitCode(command, process.exitValue());
+        exitCode = process.exitValue();
+        if (exitCode != 0) {
+          throw new SliderException(exitCode, "Error running command %s", command);
+        }
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }
+    } catch (IOException e) {
+      e.printStackTrace();
+    } finally {
+      if (br != null) {
+        try {
+          br.close();
+        } catch (IOException ioe) {
+          ioe.printStackTrace();
+        }
+      }
+    }
+
+    return exitCode;//some exception occurred
+
+  }
+
+  public synchronized void generateContainerCertificate(String hostname,
+                                                        String identifier) {
+    LOG.info("Generation of certificate for {}", hostname);
+
+    String srvrKstrDir = SecurityUtils.getSecurityDir();
+    Object[] scriptArgs = {srvrKstrDir, getSubjectDN(hostname, identifier,
+        this.applicationName), identifier, SecurityUtils.getSecurityDir()};
+
+    try {
+      String command = MessageFormat.format(GEN_AGENT_KEY, scriptArgs);
+      runCommand(command);
+
+      signAgentCertificate(identifier);
+
+    } catch (SliderException e) {
+      LOG.error("Error generating the agent certificate", e);
+    }
+  }
+
+  public synchronized SecurityStore generateContainerKeystore(String hostname,
+                                                              String requesterId,
+                                                              String role,
+                                                              String keystorePass)
+      throws SliderException {
+    LOG.info("Generation of container keystore for container {} on {}",
+             requesterId, hostname);
+
+    generateContainerCertificate(hostname, requesterId);
+
+    // come up with correct args to invoke keystore command
+    String srvrCrtPass = SecurityUtils.getKeystorePass();
+    String srvrKstrDir = SecurityUtils.getSecurityDir();
+    String containerCrtName = requesterId + ".crt";
+    String containerKeyName = requesterId + ".key";
+    String kstrName = getKeystoreFileName(requesterId, role);
+
+    Object[] scriptArgs = {srvrCrtPass, keystorePass, srvrKstrDir,
+        containerKeyName, containerCrtName, kstrName};
+
+    String command = MessageFormat.format(EXPRT_KSTR, scriptArgs);
+    runCommand(command);
+
+    return new SecurityStore(new File(srvrKstrDir, kstrName),
+                             SecurityStore.StoreType.keystore);
+  }
+
+  private static String getKeystoreFileName(String containerId,
+                                            String role) {
+    return String.format("keystore-%s-%s.p12", containerId,
+                         role != null ? role : "");
+  }
+
+  private void generateAMKeystore(String hostname, String containerId)
+      throws SliderException {
+    LOG.info("Generation of server certificate");
+
+    String srvrKstrDir = SecurityUtils.getSecurityDir();
+    String srvrCrtName = SliderKeys.CRT_FILE_NAME;
+    String srvrCsrName = SliderKeys.CSR_FILE_NAME;
+    String srvrKeyName = SliderKeys.KEY_FILE_NAME;
+    String kstrName = SliderKeys.KEYSTORE_FILE_NAME;
+    String srvrCrtPass = SecurityUtils.getKeystorePass();
+
+    Object[] scriptArgs = {srvrCrtPass, srvrKstrDir, srvrKeyName,
+        srvrCrtName, kstrName, srvrCsrName, getSubjectDN(hostname, containerId,
+        this.applicationName)};
+
+    String command = MessageFormat.format(GEN_SRVR_KEY, scriptArgs);
+    runCommand(command);
+
+    command = MessageFormat.format(GEN_SRVR_REQ, scriptArgs);
+    runCommand(command);
+
+    command = MessageFormat.format(SIGN_SRVR_CRT, scriptArgs);
+    runCommand(command);
+
+    Object[] keystoreArgs = {srvrCrtPass, srvrCrtPass, srvrKstrDir, srvrKeyName,
+        srvrCrtName, kstrName, srvrCsrName};
+    command = MessageFormat.format(EXPRT_KSTR, keystoreArgs);
+    runCommand(command);
+  }
+
+  public SecurityStore generateContainerTruststore(String containerId,
+                                                   String role,
+                                                   String truststorePass)
+      throws SliderException {
+
+    String srvrKstrDir = SecurityUtils.getSecurityDir();
+    String srvrCrtName = SliderKeys.CRT_FILE_NAME;
+    String srvrCsrName = SliderKeys.CSR_FILE_NAME;
+    String srvrKeyName = SliderKeys.KEY_FILE_NAME;
+    String kstrName = getTruststoreFileName(role, containerId);
+    String srvrCrtPass = SecurityUtils.getKeystorePass();
+
+    Object[] scriptArgs = {srvrCrtPass, truststorePass, srvrKstrDir, srvrKeyName,
+        srvrCrtName, kstrName, srvrCsrName};
+
+    String command = MessageFormat.format(EXPRT_KSTR, scriptArgs);
+    runCommand(command);
+
+    return new SecurityStore(new File(srvrKstrDir, kstrName),
+                             SecurityStore.StoreType.truststore);
+  }
+
+  private static String getTruststoreFileName(String role, String containerId) {
+    return String.format("truststore-%s-%s.p12", containerId,
+                         role != null ? role : "");
+  }
+
+  /**
+   * Returns server certificate content
+   * @return string with server certificate content
+   */
+  public String getServerCert() {
+    File certFile = getServerCertficateFilePath();
+    String srvrCrtContent = null;
+    try {
+      srvrCrtContent = FileUtils.readFileToString(certFile);
+    } catch (IOException e) {
+      LOG.error(e.getMessage());
+    }
+    return srvrCrtContent;
+  }
+
+  public static File getServerCertficateFilePath() {
+    return new File(String.format("%s%s%s",
+                                  SecurityUtils.getSecurityDir(),
+                                  File.separator,
+                                  SliderKeys.CRT_FILE_NAME));
+  }
+
+  public static File getAgentCertficateFilePath(String containerId) {
+    return new File(String.format("%s%s%s.crt",
+                                  SecurityUtils.getSecurityDir(),
+                                  File.separator,
+                                  containerId));
+  }
+
+  public static File getContainerKeystoreFilePath(String containerId,
+                                                  String role) {
+    return new File(SecurityUtils.getSecurityDir(), getKeystoreFileName(
+        containerId,
+        role
+    ));
+  }
+
+  public static File getContainerTruststoreFilePath(String role,
+                                                    String containerId) {
+    return new File(SecurityUtils.getSecurityDir(),
+                    getTruststoreFileName(role, containerId));
+  }
+
+  public static File getAgentKeyFilePath(String containerId) {
+    return new File(String.format("%s%s%s.key",
+                                  SecurityUtils.getSecurityDir(),
+                                  File.separator,
+                                  containerId));
+  }
+
+  /**
+   * Signs agent certificate
+   * Adds agent certificate to server keystore
+   * @return string with agent signed certificate content
+   */
+  public synchronized SignCertResponse signAgentCrt(String agentHostname,
+                                                    String agentCrtReqContent,
+                                                    String passphraseAgent) {
+    SignCertResponse response = new SignCertResponse();
+    LOG.info("Signing of agent certificate");
+    LOG.info("Verifying passphrase");
+
+    if (!this.passphrase.equals(passphraseAgent.trim())) {
+      LOG.warn("Incorrect passphrase from the agent");
+      response.setResult(SignCertResponse.ERROR_STATUS);
+      response.setMessage("Incorrect passphrase from the agent");
+      return response;
+    }
+
+    String srvrKstrDir = SecurityUtils.getSecurityDir();
+    String srvrCrtPass = SecurityUtils.getKeystorePass();
+    String srvrCrtName = SliderKeys.CRT_FILE_NAME;
+    String srvrKeyName = SliderKeys.KEY_FILE_NAME;
+    String agentCrtReqName = agentHostname + ".csr";
+    String agentCrtName = agentHostname + ".crt";
+
+    Object[] scriptArgs = {srvrKstrDir, agentCrtReqName, agentCrtName,
+        srvrCrtPass, srvrKeyName, srvrCrtName};
+
+    //Revoke previous agent certificate if exists
+    File agentCrtFile = new File(srvrKstrDir + File.separator + agentCrtName);
+
+    String command = null;
+    if (agentCrtFile.exists()) {
+      LOG.info("Revoking of " + agentHostname + " certificate.");
+      command = MessageFormat.format(REVOKE_AGENT_CRT, scriptArgs);
+      try {
+        runCommand(command);
+      } catch (SliderException e) {
+        int commandExitCode = e.getExitCode();
+        response.setResult(SignCertResponse.ERROR_STATUS);
+        response.setMessage(
+            SecurityUtils.getOpenSslCommandResult(command, commandExitCode));
+        return response;
+      }
+    }
+
+    File agentCrtReqFile = new File(srvrKstrDir + File.separator +
+        agentCrtReqName);
+    try {
+      FileUtils.writeStringToFile(agentCrtReqFile, agentCrtReqContent);
+    } catch (IOException e1) {
+      // TODO Auto-generated catch block
+      e1.printStackTrace();
+    }
+
+    command = MessageFormat.format(SIGN_AGENT_CRT, scriptArgs);
+
+    LOG.debug(SecurityUtils.hideOpenSslPassword(command));
+    try {
+      runCommand(command);
+    } catch (SliderException e) {
+      int commandExitCode = e.getExitCode();
+      response.setResult(SignCertResponse.ERROR_STATUS);
+      response.setMessage(
+          SecurityUtils.getOpenSslCommandResult(command, commandExitCode));
+      return response;
+    }
+
+    String agentCrtContent = "";
+    try {
+      agentCrtContent = FileUtils.readFileToString(agentCrtFile);
+    } catch (IOException e) {
+      e.printStackTrace();
+      LOG.error("Error reading signed agent certificate");
+      response.setResult(SignCertResponse.ERROR_STATUS);
+      response.setMessage("Error reading signed agent certificate");
+      return response;
+    }
+    response.setResult(SignCertResponse.OK_STATUS);
+    response.setSignedCa(agentCrtContent);
+    //LOG.info(ShellCommandUtil.getOpenSslCommandResult(command, commandExitCode));
+    return response;
+  }
+
+  private String signAgentCertificate (String containerId)
+      throws SliderException {
+    String srvrKstrDir = SecurityUtils.getSecurityDir();
+    String srvrCrtPass = SecurityUtils.getKeystorePass();
+    String srvrCrtName = SliderKeys.CRT_FILE_NAME;
+    String srvrKeyName = SliderKeys.KEY_FILE_NAME;
+    String agentCrtReqName = containerId + ".csr";
+    String agentCrtName = containerId + ".crt";
+
+    // server certificate must exist already
+    if (!(new File(srvrKstrDir, srvrCrtName).exists())) {
+      throw new SliderException("CA certificate not generated");
+    }
+
+    Object[] scriptArgs = {srvrKstrDir, agentCrtReqName, agentCrtName,
+        srvrCrtPass, srvrKeyName, srvrCrtName};
+
+    //Revoke previous agent certificate if exists
+    File agentCrtFile = new File(srvrKstrDir + File.separator + agentCrtName);
+
+    String command;
+    if (agentCrtFile.exists()) {
+      LOG.info("Revoking of " + containerId + " certificate.");
+      command = MessageFormat.format(REVOKE_AGENT_CRT, scriptArgs);
+      runCommand(command);
+    }
+
+    command = MessageFormat.format(SIGN_AGENT_CRT, scriptArgs);
+
+    LOG.debug(SecurityUtils.hideOpenSslPassword(command));
+    runCommand(command);
+
+    return agentCrtName;
+
+  }
+
+  private String getSubjectDN(String hostname, String containerId,
+                              String appName) {
+    return String.format("/CN=%s%s%s",
+                         hostname,
+                         containerId != null ? "/OU=" + containerId : "",
+                         appName != null ? "/OU=" + appName : "");
+
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/KeystoreGenerator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/KeystoreGenerator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/KeystoreGenerator.java
new file mode 100644
index 0000000..e2339d5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/KeystoreGenerator.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.services.security;
+
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.exceptions.SliderException;
+
+import java.io.File;
+import java.io.IOException;
+
+/**
+ *
+ */
+public class KeystoreGenerator extends AbstractSecurityStoreGenerator {
+
+
+  public KeystoreGenerator(CertificateManager certificateMgr) {
+    super(certificateMgr);
+  }
+
+  @Override
+  public SecurityStore generate(String hostname, String containerId,
+                                AggregateConf instanceDefinition,
+                                MapOperations compOps, String role)
+      throws SliderException, IOException {
+    SecurityStore keystore = null;
+    String password = getStorePassword(
+        instanceDefinition.getAppConf().credentials, compOps, role);
+    if (password != null) {
+      keystore =
+          certificateMgr.generateContainerKeystore(hostname, containerId, role,
+                                                   password);
+    }
+    return keystore;
+  }
+
+  @Override
+  String getPassword(MapOperations compOps) {
+    return compOps.get(
+        compOps.get(SliderKeys.COMP_KEYSTORE_PASSWORD_PROPERTY_KEY));
+  }
+
+  @Override
+  String getAlias(MapOperations compOps) {
+    return compOps.getOption(SliderKeys.COMP_KEYSTORE_PASSWORD_ALIAS_KEY,
+                             SliderKeys.COMP_KEYSTORE_PASSWORD_ALIAS_DEFAULT);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/SecurityStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/SecurityStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/SecurityStore.java
new file mode 100644
index 0000000..fc54267
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/SecurityStore.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.services.security;
+
+import java.io.File;
+
+/**
+ *
+ */
+public class SecurityStore {
+  private File file;
+
+  public enum StoreType {truststore, keystore}
+
+  private StoreType type;
+
+  public String getType() {
+    return type.name();
+  }
+
+  public File getFile() {
+    return file;
+  }
+
+  public SecurityStore(File file,
+                       StoreType type) {
+
+    this.file = file;
+    this.type = type;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    SecurityStore that = (SecurityStore) o;
+
+    if (file != null ? !file.equals(that.file) : that.file != null)
+      return false;
+    if (type != that.type) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = file != null ? file.hashCode() : 0;
+    result = 31 * result + (type != null ? type.hashCode() : 0);
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/SecurityStoreGenerator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/SecurityStoreGenerator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/SecurityStoreGenerator.java
new file mode 100644
index 0000000..a814988
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/SecurityStoreGenerator.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.services.security;
+
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.exceptions.SliderException;
+
+import java.io.File;
+import java.io.IOException;
+
+/**
+ *
+ */
+public interface SecurityStoreGenerator {
+
+  SecurityStore generate(String hostname,
+                         String containerId,
+                         AggregateConf instanceDefinition,
+                         MapOperations compOps,
+                         String role)
+      throws SliderException, IOException;
+
+  boolean isStoreRequested(MapOperations compOps);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/SecurityUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/SecurityUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/SecurityUtils.java
new file mode 100644
index 0000000..e82ad84
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/SecurityUtils.java
@@ -0,0 +1,256 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.services.security;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.SliderXmlConfKeys;
+import org.apache.slider.core.conf.MapOperations;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+//import java.nio.file.Files;
+//import java.nio.file.Path;
+//import java.nio.file.Paths;
+//import java.nio.file.attribute.PosixFilePermission;
+//import java.nio.file.attribute.PosixFilePermissions;
+
+
+/**
+ *
+ */
+public class SecurityUtils {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SecurityUtils.class);
+
+  private static String CA_CONFIG_CONTENTS =  "HOME            = .\n"
+                                            + "RANDFILE        = $ENV::HOME/.rnd\n\n"
+                                            + "[ ca ]\n"
+                                            + "default_ca             = CA_CLIENT\n"
+                                            + "[ CA_CLIENT ]\n"
+                                            + "dir                    = ${SEC_DIR}/db\n"
+                                            + "certs                  = $dir/certs\n"
+                                            + "new_certs_dir          = $dir/newcerts\n"
+                                            + "\n"
+                                            + "database               = $dir/index.txt\n"
+                                            + "serial                 = $dir/serial\n"
+                                            + "default_days           = 365    \n"
+                                            + "\n"
+                                            + "default_crl_days       = 7  \n"
+                                            + "default_md             = sha256 \n"
+                                            + "\n"
+                                            + "policy                 = policy_anything \n"
+                                            + "\n"
+                                            + "[ policy_anything ]\n"
+                                            + "countryName            = optional\n"
+                                            + "stateOrProvinceName    = optional\n"
+                                            + "localityName           = optional\n"
+                                            + "organizationName       = optional\n"
+                                            + "organizationalUnitName = optional\n"
+                                            + "commonName             = optional\n"
+                                            + "emailAddress           = optional\n"
+                                            + "\n"
+                                            + "[req]\n"
+                                            + "distinguished_name     = req_distinguished_name\n"
+                                            + "\n"
+                                            + "[ req_distinguished_name ]\n"
+                                            + "\n"
+                                            + "[ jdk7_ca ]\n"
+                                            + "subjectKeyIdentifier = hash\n"
+                                            + "authorityKeyIdentifier = keyid:always,issuer:always\n"
+                                            + "basicConstraints = CA:true\n";
+
+  private static final String PASS_TOKEN = "pass:";
+  private static String keystorePass;
+  private static String securityDir;
+
+  public static void logOpenSslExitCode(String command, int exitCode) {
+    if (exitCode == 0) {
+      LOG.info(getOpenSslCommandResult(command, exitCode));
+    } else {
+      LOG.warn(getOpenSslCommandResult(command, exitCode));
+    }
+
+  }
+
+  public static String hideOpenSslPassword(String command){
+    int start = command.indexOf(PASS_TOKEN);
+    while (start >= 0) {
+      start += PASS_TOKEN.length();
+      CharSequence cs = command.subSequence(start, command.indexOf(" ", start));
+      command = command.replace(cs, "****");
+      start = command.indexOf(PASS_TOKEN, start + 1);
+    }
+    return command;
+  }
+
+  public static String getOpenSslCommandResult(String command, int exitCode) {
+    return new StringBuilder().append("Command ")
+        .append(hideOpenSslPassword(command))
+        .append(" was finished with exit code: ")
+        .append(exitCode).append(" - ")
+        .append(getOpenSslExitCodeDescription(exitCode)).toString();
+  }
+
+  private static String getOpenSslExitCodeDescription(int exitCode) {
+    switch (exitCode) {
+      case 0: {
+        return "the operation was completed successfully.";
+      }
+      case 1: {
+        return "an error occurred parsing the command options.";
+      }
+      case 2: {
+        return "one of the input files could not be read.";
+      }
+      case 3: {
+        return "an error occurred creating the PKCS#7 file or when reading the MIME message.";
+      }
+      case 4: {
+        return "an error occurred decrypting or verifying the message.";
+      }
+      case 5: {
+        return "the message was verified correctly but an error occurred writing out the signers certificates.";
+      }
+      default:
+        return "unsupported code";
+    }
+  }
+
+  public static void writeCaConfigFile(String path) throws IOException {
+    String contents = CA_CONFIG_CONTENTS.replace("${SEC_DIR}", path);
+    FileUtils.writeStringToFile(new File(path, "ca.config"), contents);
+  }
+
+  public static String getKeystorePass() {
+    return keystorePass;
+  }
+
+  public static String getSecurityDir() {
+    return securityDir;
+  }
+
+  public static void    initializeSecurityParameters(MapOperations configMap) {
+    initializeSecurityParameters(configMap, false);
+  }
+
+  public static void initializeSecurityParameters(MapOperations configMap,
+                                                boolean persistPassword) {
+    String keyStoreLocation = configMap.getOption(
+        SliderXmlConfKeys.KEY_KEYSTORE_LOCATION, getDefaultKeystoreLocation());
+    if (keyStoreLocation == null) {
+      LOG.error(SliderXmlConfKeys.KEY_KEYSTORE_LOCATION
+          + " is not specified. Unable to initialize security params.");
+      return;
+    }
+    File secDirFile = new File(keyStoreLocation).getParentFile();
+    if (!secDirFile.exists()) {
+      // create entire required directory structure
+      File dbDir = new File(secDirFile, "db");
+      File newCertsDir = new File(dbDir, "newcerts");
+      newCertsDir.mkdirs();
+      RawLocalFileSystem fileSystem = null;
+      try {
+        fileSystem = new RawLocalFileSystem();
+        FsPermission permissions = new FsPermission(FsAction.ALL, FsAction.NONE,
+                                                    FsAction.NONE);
+        fileSystem.setPermission(new Path(dbDir.getAbsolutePath()),
+                                 permissions);
+        fileSystem.setPermission(new Path(dbDir.getAbsolutePath()), permissions);
+        fileSystem.setPermission(new Path(newCertsDir.getAbsolutePath()),
+                                 permissions);
+        File indexFile = new File(dbDir, "index.txt");
+        indexFile.createNewFile();
+        SecurityUtils.writeCaConfigFile(secDirFile.getAbsolutePath().replace('\\', '/'));
+
+      } catch (IOException e) {
+        LOG.error("Unable to create SSL configuration directories/files", e);
+      } finally {
+        if (fileSystem != null) {
+          try {
+            fileSystem.close();
+          } catch (IOException e) {
+            LOG.warn("Unable to close fileSystem", e);
+          }
+        }
+      }
+      // need to create the password
+    }
+    keystorePass = getKeystorePassword(secDirFile, persistPassword);
+    securityDir = secDirFile.getAbsolutePath();
+  }
+
+  private static String getKeystorePassword(File secDirFile,
+                                            boolean persistPassword) {
+    File passFile = new File(secDirFile, SliderKeys.CRT_PASS_FILE_NAME);
+    String password = null;
+    if (!passFile.exists()) {
+      LOG.info("Generating keystore password");
+      password = RandomStringUtils.randomAlphanumeric(
+          Integer.valueOf(SliderKeys.PASS_LEN));
+      if (persistPassword) {
+        try {
+          FileUtils.writeStringToFile(passFile, password);
+          passFile.setWritable(true);
+          passFile.setReadable(true);
+        } catch (IOException e) {
+          e.printStackTrace();
+          throw new RuntimeException(
+              "Error creating certificate password file");
+        }
+      }
+    } else {
+      LOG.info("Reading password from existing file");
+      try {
+        password = FileUtils.readFileToString(passFile);
+        password = password.replaceAll("\\p{Cntrl}", "");
+      } catch (IOException e) {
+        e.printStackTrace();
+      }
+    }
+
+    return password;
+  }
+
+  private static String getDefaultKeystoreLocation() {
+    File workDir = null;
+    try {
+      workDir =  new File(FileUtils.getTempDirectory().getAbsolutePath()
+                          + "/sec" + System.currentTimeMillis());
+      if (!workDir.mkdirs()) {
+        throw new IOException("Unable to create temporary security directory");
+      }
+    } catch (IOException e) {
+      LOG.warn("Unable to create security directory");
+      return null;
+    }
+
+    return new StringBuilder().append(workDir.getAbsolutePath())
+        .append(File.separator)
+        .append(SliderKeys.SECURITY_DIR)
+        .append(File.separator)
+        .append(SliderKeys.KEYSTORE_FILE_NAME).toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/SignCertResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/SignCertResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/SignCertResponse.java
new file mode 100644
index 0000000..8437d88
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/SignCertResponse.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.security;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlType;
+
+/**
+ *
+ * Sign certificate response data model.
+ *
+ */
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+@XmlType(name = "", propOrder = {})
+public class SignCertResponse {
+	
+  public static final String ERROR_STATUS = "ERROR";
+  public static final String OK_STATUS = "OK";
+
+  @XmlElement
+  private String result;
+  @XmlElement
+  private String signedCa;
+  @XmlElement
+  private String message;
+
+  public String getResult() {
+    return result;
+  }
+  public void setResult(String result) {
+    this.result = result;
+  }
+  public String getSignedCa() {
+    return signedCa;
+  }
+  public void setSignedCa(String signedCa) {
+    this.signedCa = signedCa;
+  }
+
+  public String getMessage() {
+    return message;
+  }
+  public void setMessage(String message) {
+    this.message = message;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/SignMessage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/SignMessage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/SignMessage.java
new file mode 100644
index 0000000..4bccb87
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/SignMessage.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.security;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlType;
+
+/**
+ *
+ * Sign certificate request data model.
+ *
+ */
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+@XmlType(name = "", propOrder = {})
+public class SignMessage {
+
+  @XmlElement
+  private String csr;
+  @XmlElement
+  private String passphrase;
+  public String getCsr() {
+    return csr;
+  }
+  public void setCsr(String csr) {
+    this.csr = csr;
+  }
+  public String getPassphrase() {
+    return passphrase;
+  }
+  public void setPassphrase(String passphrase) {
+    this.passphrase = passphrase;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/StoresGenerator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/StoresGenerator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/StoresGenerator.java
new file mode 100644
index 0000000..226250f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/StoresGenerator.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.services.security;
+
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.exceptions.SliderException;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ *
+ */
+public class StoresGenerator {
+
+  static CertificateManager certMgr = new CertificateManager();
+  private static SecurityStoreGenerator[] GENERATORS = {
+      new KeystoreGenerator(certMgr), new TruststoreGenerator(certMgr)
+  };
+
+  public static SecurityStore[] generateSecurityStores(String hostname,
+                                                       String containerId,
+                                                       String role,
+                                                       AggregateConf instanceDefinition,
+                                                       MapOperations compOps)
+      throws SliderException, IOException {
+    //discover which stores need generation based on the passwords configured
+    List<SecurityStore> files = new ArrayList<SecurityStore>();
+    for (SecurityStoreGenerator generator : GENERATORS) {
+      if (generator.isStoreRequested(compOps)) {
+        SecurityStore store = generator.generate(hostname,
+                                                 containerId,
+                                                 instanceDefinition,
+                                                 compOps,
+                                                 role);
+        if (store != null) {
+          files.add(store);
+        }
+      }
+    }
+
+    if (files.isEmpty()) {
+      throw new SliderException("Security stores were requested but none were "
+                                + "generated. Check the AM logs and ensure "
+                                + "passwords are configured for the components "
+                                + "requiring the stores.");
+    }
+    return files.toArray(new SecurityStore[files.size()]);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/TruststoreGenerator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/TruststoreGenerator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/TruststoreGenerator.java
new file mode 100644
index 0000000..d16dcbd
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/TruststoreGenerator.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.services.security;
+
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.exceptions.SliderException;
+
+import java.io.IOException;
+
+/**
+ *
+ */
+public class TruststoreGenerator extends AbstractSecurityStoreGenerator {
+
+
+  public TruststoreGenerator(CertificateManager certificateMgr) {
+    super(certificateMgr);
+  }
+
+  @Override
+  public SecurityStore generate(String hostname, String containerId,
+                                AggregateConf instanceDefinition,
+                                MapOperations compOps, String role)
+      throws SliderException, IOException {
+    SecurityStore truststore = null;
+    String password = getStorePassword(
+        instanceDefinition.getAppConf().credentials, compOps, role);
+    if (password != null) {
+      truststore = certificateMgr.generateContainerTruststore(containerId,
+                                                              role, password);
+    }
+    return truststore;
+  }
+
+  @Override
+  String getPassword(MapOperations compOps) {
+    return compOps.get(
+        compOps.get(SliderKeys.COMP_TRUSTSTORE_PASSWORD_PROPERTY_KEY));
+  }
+
+  @Override
+  String getAlias(MapOperations compOps) {
+    return compOps.getOption(SliderKeys.COMP_TRUSTSTORE_PASSWORD_ALIAS_KEY,
+                             SliderKeys.COMP_TRUSTSTORE_PASSWORD_ALIAS_DEFAULT);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/AbstractSliderLaunchedService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/AbstractSliderLaunchedService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/AbstractSliderLaunchedService.java
new file mode 100644
index 0000000..1622309
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/AbstractSliderLaunchedService.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.utility;
+
+import org.apache.hadoop.registry.client.api.RegistryConstants;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.registry.client.api.RegistryOperationsFactory;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.slider.common.tools.ConfigHelper;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.core.zk.ZookeeperUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Base service for the standard slider client/server services
+ */
+public abstract class AbstractSliderLaunchedService extends
+    LaunchedWorkflowCompositeService {
+  private static final Logger log =
+    LoggerFactory.getLogger(AbstractSliderLaunchedService.class);
+
+  protected AbstractSliderLaunchedService(String name) {
+    super(name);
+    // make sure all the yarn configs get loaded
+    YarnConfiguration conf = new YarnConfiguration();
+    ConfigHelper.registerDeprecatedConfigItems();
+  }
+
+  /**
+   * look up the registry quorum from the config
+   * @return the quorum string
+   * @throws BadConfigException if it is not there or invalid
+   */
+  public String lookupZKQuorum() throws BadConfigException {
+ 
+    String registryQuorum = getConfig().get(RegistryConstants.KEY_REGISTRY_ZK_QUORUM);
+    
+    // though if neither is set: trouble
+    if (SliderUtils.isUnset(registryQuorum)) {
+      throw new BadConfigException(
+          "No Zookeeper quorum provided in the"
+          + " configuration property " + RegistryConstants.KEY_REGISTRY_ZK_QUORUM
+      );
+    }
+    ZookeeperUtils.splitToHostsAndPortsStrictly(registryQuorum);
+    return registryQuorum;
+  }
+
+  /**
+   * Create, adopt ,and start the YARN registration service
+   * @return the registry operations service, already deployed as a child
+   * of the AbstractSliderLaunchedService instance.
+   */
+  public RegistryOperations startRegistryOperationsService()
+      throws BadConfigException {
+
+    // push back the slider registry entry if needed
+    String quorum = lookupZKQuorum();
+    RegistryOperations registryWriterService =
+        createRegistryOperationsInstance();
+    deployChildService(registryWriterService);
+    return registryWriterService;
+  }
+
+  /**
+   * Create the registry operations instance. This is to allow
+   * subclasses to instantiate a subclass service
+   * @return an instance to match to the lifecycle of this service
+   */
+  protected RegistryOperations createRegistryOperationsInstance() {
+    return RegistryOperationsFactory.createInstance("YarnRegistry", getConfig());
+  }
+
+  /**
+   * Utility method to require an argument to be set (non null, non-empty)
+   * @param argname argument name
+   * @param value value
+   * @throws BadCommandArgumentsException if the condition is not met
+   */
+  protected static void requireArgumentSet(String argname, String value)
+      throws BadCommandArgumentsException {
+    require(isSet(value), "Required argument %s missing", argname );
+  }
+
+  /**
+   * Require a condition to hold; throw {@link BadCommandArgumentsException} if not.
+   * The exception text is the formatted message.
+   * @param condition condition
+   * @param message string to format
+   * @param args list of arguments to format.
+   * @throws BadCommandArgumentsException
+   */
+  protected static void require(boolean condition, String message,
+      Object... args)
+      throws BadCommandArgumentsException {
+    if (!condition) {
+      throw new BadCommandArgumentsException(message, args);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/EndOfServiceWaiter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/EndOfServiceWaiter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/EndOfServiceWaiter.java
new file mode 100644
index 0000000..40ceab8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/EndOfServiceWaiter.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.utility;
+
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.service.ServiceStateChangeListener;
+
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * Wait for a service to stop.
+ * 
+ * WARNING: the notification may come in as soon as the service enters
+ * the stopped state: it may take some time for the actual stop operation
+ * to complete.
+ */
+public class EndOfServiceWaiter implements ServiceStateChangeListener {
+
+  private final AtomicBoolean finished = new AtomicBoolean(false);
+  private final String name;
+  private Service service;
+
+  /**
+   * Wait for a service; use the service name as this instance's name
+   * @param service service
+   */
+  public EndOfServiceWaiter(Service service) {
+    this(service.getName(), service);
+  }
+
+
+  /**
+   * Wait for a service
+   * @param name name for messages
+   * @param service service
+   */
+  public EndOfServiceWaiter(String name, Service service) {
+    this.name = name;
+    this.service = service;
+    service.registerServiceListener(this);
+  }
+
+  public synchronized void waitForServiceToStop(long timeout) throws
+      InterruptedException, TimeoutException {
+    service.waitForServiceToStop(timeout);
+    if (!finished.get()) {
+      wait(timeout);
+      if (!finished.get()) {
+        throw new TimeoutException(name
+                                   + " did not finish after " + timeout +
+                                   " milliseconds");
+      }
+    }
+  }
+
+  /**
+   * Wait for service state change callbacks; notify self if the service has
+   * now stopped
+   * @param service service
+   */
+  @Override
+  public synchronized void stateChanged(Service service) {
+    if (service.isInState(Service.STATE.STOPPED)) {
+      finished.set(true);
+      notify();
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/LaunchedWorkflowCompositeService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/LaunchedWorkflowCompositeService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/LaunchedWorkflowCompositeService.java
new file mode 100644
index 0000000..bcd1969
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/LaunchedWorkflowCompositeService.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.utility;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.Service;
+import org.apache.slider.core.main.LauncherExitCodes;
+import org.apache.slider.core.main.RunService;
+import org.apache.slider.server.services.workflow.WorkflowCompositeService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This is a workflow compositoe service which can be launched from the CLI
+ * ... catches the arguments and implements a stub runService operation.
+ */
+public class LaunchedWorkflowCompositeService extends WorkflowCompositeService
+    implements RunService {
+  private static final Logger log = LoggerFactory.getLogger(
+      LaunchedWorkflowCompositeService.class);
+  private String[] argv;
+  
+  public LaunchedWorkflowCompositeService(String name) {
+    super(name);
+  }
+
+  public LaunchedWorkflowCompositeService(String name, Service... children) {
+    super(name, children);
+  }
+
+  /**
+   * Implementation of set-ness, groovy definition of true/false for a string
+   * @param s
+   * @return true iff the string is non-null and non-empty
+   */
+  protected static boolean isUnset(String s) {
+    return StringUtils.isEmpty(s);
+  }
+
+  protected static boolean isSet(String s) {
+    return StringUtils.isNotEmpty(s);
+  }
+
+  protected String[] getArgv() {
+    return argv;
+  }
+
+  /**
+   * Pre-init argument binding
+   * @param config the initial configuration build up by the
+   * service launcher.
+   * @param args argument list list of arguments passed to the command line
+   * after any launcher-specific commands have been stripped.
+   * @return the configuration
+   * @throws Exception
+   */
+  @Override
+  public Configuration bindArgs(Configuration config, String... args) throws
+                                                                      Exception {
+    this.argv = args;
+    if (log.isDebugEnabled()) {
+      log.debug("Binding {} Arguments:", args.length);
+
+      StringBuilder builder = new StringBuilder();
+      for (String arg : args) {
+        builder.append('"').append(arg).append("\" ");
+      }
+      log.debug(builder.toString());
+    }
+    return config;
+  }
+
+  @Override
+  public int runService() throws Throwable {
+    return LauncherExitCodes.EXIT_SUCCESS;
+  }
+
+  @Override
+  public synchronized void addService(Service service) {
+    Preconditions.checkArgument(service != null, "null service argument");
+    super.addService(service);
+  }
+
+  /**
+   * Run a child service -initing and starting it if this
+   * service has already passed those parts of its own lifecycle
+   * @param service the service to start
+   */
+  protected boolean deployChildService(Service service) {
+    service.init(getConfig());
+    addService(service);
+    if (isInState(STATE.STARTED)) {
+      service.start();
+      return true;
+    }
+    return false;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/PatternValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/PatternValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/PatternValidator.java
new file mode 100644
index 0000000..6ab9de6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/PatternValidator.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.utility;
+
+import org.apache.slider.server.appmaster.web.rest.RestPaths;
+
+import java.util.regex.Pattern;
+
+/**
+ * Utility class to validate strings against a predefined pattern.
+ */
+public class PatternValidator {
+
+  public static final String E_INVALID_NAME =
+      "Invalid name %s does not match the pattern pattern %s ";
+  private final Pattern valid;
+  private final String pattern;
+
+  public PatternValidator(String pattern) {
+    this.pattern = pattern;
+    valid = Pattern.compile(pattern);
+  }
+
+  /**
+   * Validate the name -restricting it to the set defined in 
+   * {@link RestPaths#PUBLISHED_CONFIGURATION_REGEXP}
+   * @param name name to validate
+   * @throws IllegalArgumentException if not a valid name
+   */
+  public void validate(String name) {
+    if (!matches(name)) {
+      throw new IllegalArgumentException(
+          String.format(E_INVALID_NAME, name, pattern));
+    }
+  }
+
+  /**
+   * Query to see if the pattern matches
+   * @param name name to validate
+   * @return true if the string matches the pattern
+   */
+  public boolean matches(String name) {
+    return valid.matcher(name).matches();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/WebAppService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/WebAppService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/WebAppService.java
new file mode 100644
index 0000000..ebfcb99
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/utility/WebAppService.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.utility;
+
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.webapp.WebApp;
+
+/**
+ * Contains a webapp reference and stops it in teardown if non-null
+ * <p>
+ * It does not start the application.
+ * Access to the field is not synchronized across threads; it is the
+ * responsibility of the caller.
+ */
+public class WebAppService<T extends WebApp> extends AbstractService {
+
+  private volatile T webApp;
+
+  public WebAppService(String name) {
+    super(name);
+  }
+
+  public WebAppService(String name, T app) {
+    super(name);
+    webApp = app;
+  }
+
+  public T getWebApp() {
+    return webApp;
+  }
+
+  public void setWebApp(T webApp) {
+    this.webApp = webApp;
+  }
+
+
+  @Override
+  protected void serviceStart() throws Exception {
+
+  }
+
+  /**
+   * Stop operation stops the webapp; sets the reference to null
+   * @throws Exception
+   */
+  @Override
+  protected void serviceStop() throws Exception {
+    if (webApp != null) {
+      webApp.stop();
+      webApp = null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ClosingService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ClosingService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ClosingService.java
new file mode 100644
index 0000000..8b711aa
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ClosingService.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+import org.apache.hadoop.service.AbstractService;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * Service that closes the closeable supplied during shutdown, if not null.
+ * 
+ * As the Service interface itself extends Closeable, this service
+ * can be used to shut down other services if desired.
+ */
+public class ClosingService<C extends Closeable> extends AbstractService {
+
+  private C closeable;
+
+  public ClosingService(String name) {
+    super(name);
+  }
+
+  /**
+   * Construct an instance of the service
+   * @param name service name
+   * @param closeable closeable to close (may be null)
+   */
+  public ClosingService(String name,
+      C closeable) {
+    super(name);
+    this.closeable = closeable;
+  }
+
+  /**
+   * Construct an instance of the service, using the default name
+   * @param closeable closeable to close (may be null)
+   */
+  public ClosingService(C closeable) {
+    this("ClosingService", closeable);
+  }
+
+
+  /**
+   * Get the closeable
+   * @return the closeable
+   */
+  public synchronized C getCloseable() {
+    return closeable;
+  }
+
+  /**
+   * Set or update the closeable.
+   * @param closeable
+   */
+  public synchronized void setCloseable(C closeable) {
+    this.closeable = closeable;
+  }
+
+  /**
+   * Stop routine will close the closeable -if not null - and set the
+   * reference to null afterwards
+   * This operation does raise any exception on the close, though it does
+   * record it
+   */
+  @Override
+  protected void serviceStop() {
+    C target = getCloseable();
+    if (target != null) {
+      try {
+        target.close();
+      } catch (IOException ioe) {
+        noteFailure(ioe);
+      }
+      setCloseable(null);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ForkedProcessService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ForkedProcessService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ForkedProcessService.java
new file mode 100644
index 0000000..352be49
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ForkedProcessService.java
@@ -0,0 +1,301 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+import org.apache.hadoop.service.ServiceStateException;
+import org.apache.slider.core.main.ServiceLaunchException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Service wrapper for an external program that is launched and can/will terminate.
+ * This service is notified when the subprocess terminates, and stops itself 
+ * and converts a non-zero exit code into a failure exception.
+ * 
+ * <p>
+ * Key Features:
+ * <ol>
+ *   <li>The property {@link #executionTimeout} can be set to set a limit
+ *   on the duration of a process</li>
+ *   <li>Output is streamed to the output logger provided</li>.
+ *   <li>The most recent lines of output are saved to a linked list</li>.
+ *   <li>A synchronous callback, {@link LongLivedProcessLifecycleEvent}, is raised on the start
+ *   and finish of a process.</li>
+ * </ol>
+ *
+ * Usage:
+ * <p></p>
+ * The service can be built in the constructor, {@link #ForkedProcessService(String, Map, List)},
+ * or have its simple constructor used to instantiate the service, then the 
+ * {@link #build(Map, List)} command used to define the environment variables
+ * and list of commands to execute. One of these two options MUST be exercised
+ * before calling the services's {@link #start()} method.
+ * <p></p>
+ * The forked process is executed in the service's {@link #serviceStart()} method;
+ * if still running when the service is stopped, {@link #serviceStop()} will
+ * attempt to stop it.
+ * <p></p>
+ * 
+ * The service delegates process execution to {@link LongLivedProcess},
+ * receiving callbacks via the {@link LongLivedProcessLifecycleEvent}.
+ * When the service receives a callback notifying that the process has completed,
+ * it calls its {@link #stop()} method. If the error code was non-zero, 
+ * the service is logged as having failed.
+ */
+public class ForkedProcessService
+    extends WorkflowExecutorService<ExecutorService>
+    implements LongLivedProcessLifecycleEvent, Runnable {
+
+  /**
+   * Log for the forked master process
+   */
+  private static final Logger LOG =
+    LoggerFactory.getLogger(ForkedProcessService.class);
+
+  private final AtomicBoolean processTerminated = new AtomicBoolean(false);
+  private boolean processStarted = false;
+  private LongLivedProcess process;
+  private int executionTimeout = -1;
+  private int timeoutCode = 1;
+  /** 
+  log to log to; defaults to this service log
+   */
+  private Logger processLog = LOG;
+  
+  /**
+   * Exit code set when the spawned process exits
+   */
+  private AtomicInteger exitCode = new AtomicInteger(0);
+
+  /**
+   * Create an instance of the service
+   * @param name a name
+   */
+  public ForkedProcessService(String name) {
+    super(name);
+  }
+
+  /**
+   * Create an instance of the service,  set up the process
+   * @param name a name
+   * @param commandList list of commands is inserted on the front
+   * @param env environment variables above those generated by
+   * @throws IOException IO problems
+   */
+  public ForkedProcessService(String name,
+      Map<String, String> env,
+      List<String> commandList) throws IOException {
+    super(name);
+    build(env, commandList);
+  }
+
+  @Override //AbstractService
+  protected void serviceStart() throws Exception {
+    if (process == null) {
+      throw new ServiceStateException("Process not yet configured");
+    }
+    //now spawn the process -expect updates via callbacks
+    process.start();
+  }
+
+  @Override //AbstractService
+  protected void serviceStop() throws Exception {
+    completed();
+    stopForkedProcess();
+  }
+
+  private void stopForkedProcess() {
+    if (process != null) {
+      process.stop();
+    }
+  }
+
+  /**
+   * Set the process log. This may be null for "do not log"
+   * @param processLog process log
+   */
+  public void setProcessLog(Logger processLog) {
+    this.processLog = processLog;
+    process.setProcessLog(processLog);
+  }
+
+  /**
+   * Set the timeout by which time a process must have finished -or -1 for forever
+   * @param timeout timeout in milliseconds
+   */
+  public void setTimeout(int timeout, int code) {
+    this.executionTimeout = timeout;
+    this.timeoutCode = code;
+  }
+
+  /**
+   * Build the process to execute when the service is started
+   * @param commandList list of commands is inserted on the front
+   * @param env environment variables above those generated by
+   * @throws IOException IO problems
+   */
+  public void build(Map<String, String> env,
+                    List<String> commandList)
+      throws IOException {
+    assert process == null;
+
+    process = new LongLivedProcess(getName(), processLog, commandList);
+    process.setLifecycleCallback(this);
+    //set the env variable mapping
+    process.putEnvMap(env);
+  }
+
+  @Override // notification from executed process
+  public synchronized void onProcessStarted(LongLivedProcess process) {
+    LOG.debug("Process has started");
+    processStarted = true;
+    if (executionTimeout > 0) {
+      setExecutor(ServiceThreadFactory.singleThreadExecutor(getName(), true));
+      execute(this);
+    }
+  }
+
+  @Override  // notification from executed process
+  public void onProcessExited(LongLivedProcess process,
+      int uncorrected,
+      int code) {
+    try {
+      synchronized (this) {
+        completed();
+        //note whether or not the service had already stopped
+        LOG.debug("Process has exited with exit code {}", code);
+        if (code != 0) {
+          reportFailure(code, getName() + " failed with code " + code);
+        }
+      }
+    } finally {
+      stop();
+    }
+  }
+
+  private void reportFailure(int code, String text) {
+    //error
+    ServiceLaunchException execEx = new ServiceLaunchException(code, text);
+    LOG.debug("Noting failure", execEx);
+    noteFailure(execEx);
+  }
+
+  /**
+   * handle timeout response by escalating it to a failure
+   */
+  @Override
+  public void run() {
+    try {
+      synchronized (processTerminated) {
+        if (!processTerminated.get()) {
+          processTerminated.wait(executionTimeout);
+        }
+      }
+
+    } catch (InterruptedException e) {
+      //assume signalled; exit
+    }
+    //check the status; if the marker isn't true, bail
+    if (!processTerminated.getAndSet(true)) {
+      LOG.info("process timeout: reporting error code {}", timeoutCode);
+
+      //timeout
+      if (isInState(STATE.STARTED)) {
+        //trigger a failure
+        stopForkedProcess();
+      }
+      reportFailure(timeoutCode, getName() + ": timeout after " + executionTimeout
+                   + " millis: exit code =" + timeoutCode);
+    }
+  }
+
+  /**
+   * Note the process as having completed.
+   * The process marked as terminated
+   * -and anything synchronized on <code>processTerminated</code>
+   * is notified
+   */
+  protected void completed() {
+    processTerminated.set(true);
+    synchronized (processTerminated) {
+      processTerminated.notify();
+    }
+  }
+
+  public boolean isProcessTerminated() {
+    return processTerminated.get();
+  }
+
+  public synchronized boolean isProcessStarted() {
+    return processStarted;
+  }
+
+  /**
+   * Is a process running: between started and terminated
+   * @return true if the process is up.
+   */
+  public synchronized boolean isProcessRunning() {
+    return processStarted && !isProcessTerminated();
+  }
+
+
+  public Integer getExitCode() {
+    return process.getExitCode();
+  }
+  
+  public int getExitCodeSignCorrected() {
+    Integer exitCode = process.getExitCodeSignCorrected();
+    if (exitCode == null) return -1;
+    return exitCode;
+  }
+
+  /**
+   * Get the recent output from the process, or [] if not defined
+   * @return a possibly empty list
+   */
+  public List<String> getRecentOutput() {
+    return process != null
+           ? process.getRecentOutput()
+           : new LinkedList<String>();
+  }
+
+  /**
+   * Get the recent output from the process, or [] if not defined
+   *
+   * @param finalOutput flag to indicate "wait for the final output of the process"
+   * @param duration the duration, in ms, 
+   * to wait for recent output to become non-empty
+   * @return a possibly empty list
+   */
+  public List<String> getRecentOutput(boolean finalOutput, int duration) {
+    if (process == null) {
+      return new LinkedList<>();
+    }
+    return process.getRecentOutput(finalOutput, duration);
+  }
+  
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[70/76] [abbrv] hadoop git commit: YARN-5796. Convert enums values in service code to upper case and special handling of an error. Contributed by Gour Saha

Posted by ji...@apache.org.
YARN-5796. Convert enums values in service code to upper case and special handling of an error. Contributed by Gour Saha


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

Branch: refs/heads/yarn-native-services
Commit: c3f32f4c7309fde0bc352ef5ae9571be5d52e30d
Parents: 00c3865
Author: Jian He <ji...@apache.org>
Authored: Tue Nov 1 11:00:11 2016 -0700
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 .../yarn/services/api/impl/ApplicationApiService.java     |  4 ++++
 .../apache/hadoop/yarn/services/resource/Artifact.java    |  2 +-
 .../apache/hadoop/yarn/services/resource/ConfigFile.java  |  4 ++--
 .../hadoop/yarn/services/resource/ReadinessCheck.java     |  2 +-
 .../hadoop/yarn/services/utils/RestApiErrorMessages.java  | 10 +++++++---
 5 files changed, 15 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3f32f4c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
index cf43ac2..37bd134 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
@@ -172,6 +172,10 @@ public class ApplicationApiService implements ApplicationApi {
         applicationStatus.setDiagnostics(ERROR_APPLICATION_IN_USE);
         return Response.status(Status.BAD_REQUEST).entity(applicationStatus)
             .build();
+      } else if (se.getExitCode() == SliderExitCodes.EXIT_INSTANCE_EXISTS) {
+        applicationStatus.setDiagnostics(ERROR_APPLICATION_INSTANCE_EXISTS);
+        return Response.status(Status.BAD_REQUEST).entity(applicationStatus)
+            .build();
       } else {
         applicationStatus.setDiagnostics(se.getMessage());
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3f32f4c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Artifact.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Artifact.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Artifact.java
index 9ac2bc7..87fcf89 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Artifact.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Artifact.java
@@ -40,7 +40,7 @@ public class Artifact implements Serializable {
   private String id = null;
 
   public enum TypeEnum {
-    DOCKER("docker"), TARBALL("tarball"), APPLICATION("application");
+    DOCKER("DOCKER"), TARBALL("TARBALL"), APPLICATION("APPLICATION");
 
     private String value;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3f32f4c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ConfigFile.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ConfigFile.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ConfigFile.java
index 3ced153..01d976f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ConfigFile.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ConfigFile.java
@@ -43,8 +43,8 @@ public class ConfigFile implements Serializable {
   private static final long serialVersionUID = -7009402089417704612L;
 
   public enum TypeEnum {
-    XML("xml"), PROPERTIES("properties"), JSON("json"), YAML("yaml"), TEMPLATE(
-        "template"), ENV("env"), HADOOP_XML("hadoop_xml");
+    XML("XML"), PROPERTIES("PROPERTIES"), JSON("JSON"), YAML("YAML"), TEMPLATE(
+        "TEMPLATE"), ENV("ENV"), HADOOP_XML("HADOOP_XML");
 
     private String value;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3f32f4c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ReadinessCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ReadinessCheck.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ReadinessCheck.java
index f549746..26cd39a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ReadinessCheck.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ReadinessCheck.java
@@ -39,7 +39,7 @@ public class ReadinessCheck implements Serializable {
   private static final long serialVersionUID = -3836839816887186801L;
 
   public enum TypeEnum {
-    HTTP("http");
+    HTTP("HTTP");
 
     private String value;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3f32f4c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiErrorMessages.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiErrorMessages.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiErrorMessages.java
index 685f85a..2d739a4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiErrorMessages.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiErrorMessages.java
@@ -26,6 +26,11 @@ public interface RestApiErrorMessages {
 
   String ERROR_APPLICATION_NOT_RUNNING = "Application not running";
   String ERROR_APPLICATION_DOES_NOT_EXIST = "Application not found";
+  String ERROR_APPLICATION_IN_USE = "Application already exists in started"
+      + " state";
+  String ERROR_APPLICATION_INSTANCE_EXISTS = "Application already exists in"
+      + " stopped/failed state (either restart with PUT or destroy with DELETE"
+      + " before creating a new one)";
 
   String ERROR_SUFFIX_FOR_COMPONENT =
       " for component %s (nor at the global level)";
@@ -66,7 +71,6 @@ public interface RestApiErrorMessages {
   String ERROR_RESOURCE_PROFILE_NOT_SUPPORTED_YET =
       "Resource profile is not " + "supported yet. Please specify cpus/memory.";
 
-  String ERROR_APPLICATION_IN_USE = "Application name is already in use";
   String ERROR_NULL_ARTIFACT_ID =
       "Artifact Id can not be null if artifact type is none";
   String ERROR_ABSENT_NUM_OF_INSTANCE =
@@ -74,6 +78,6 @@ public interface RestApiErrorMessages {
   String ERROR_ABSENT_LAUNCH_COMMAND =
       "launch command should appear if type is slider-zip or none";
 
-  String ERROR_QUICKLINKS_FOR_COMP_INVALID =
-      "Quicklinks specified at component level, needs corresponding values set at application level";
+  String ERROR_QUICKLINKS_FOR_COMP_INVALID = "Quicklinks specified at"
+      + " component level, needs corresponding values set at application level";
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[23/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/RunService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/RunService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/RunService.java
new file mode 100644
index 0000000..c3a1d0e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/RunService.java
@@ -0,0 +1,62 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.core.main;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.Service;
+
+/**
+ * An interface which services can implement to have their
+ * execution managed by the ServiceLauncher.
+ * The command line options will be passed down before the 
+ * {@link Service#init(Configuration)} operation is invoked via an
+ * invocation of {@link RunService#bindArgs(Configuration, String...)}
+ * After the service has been successfully started via {@link Service#start()}
+ * the {@link RunService#runService()} method is called to execute the 
+ * service. When this method returns, the service launcher will exit, using
+ * the return code from the method as its exit option.
+ */
+public interface RunService extends Service {
+
+  /**
+   * Propagate the command line arguments.
+   * This method is called before {@link Service#init(Configuration)};
+   * the configuration that is returned from this operation
+   * is the one that is passed on to the init operation.
+   * This permits implemenations to change the configuration before
+   * the init operation.n
+   * 
+   *
+   * @param config the initial configuration build up by the
+   * service launcher.
+   * @param args argument list list of arguments passed to the command line
+   * after any launcher-specific commands have been stripped.
+   * @return the configuration to init the service with. This MUST NOT be null.
+   * Recommended: pass down the config parameter with any changes
+   * @throws Exception any problem
+   */
+  Configuration bindArgs(Configuration config, String... args) throws Exception;
+  
+  /**
+   * Run a service. This called after {@link Service#start()}
+   * @return the exit code
+   * @throws Throwable any exception to report
+   */
+  int runService() throws Throwable ;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ServiceLaunchException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ServiceLaunchException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ServiceLaunchException.java
new file mode 100644
index 0000000..27813b7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ServiceLaunchException.java
@@ -0,0 +1,73 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.core.main;
+
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * A service launch exception that includes an exit code;
+ * when caught by the ServiceLauncher, it will convert that
+ * into a process exit code.
+ */
+public class ServiceLaunchException extends YarnException
+  implements ExitCodeProvider, LauncherExitCodes {
+
+  private final int exitCode;
+
+  /**
+   * Create an exception with the specific exit code
+   * @param exitCode exit code
+   * @param cause cause of the exception
+   */
+  public ServiceLaunchException(int exitCode, Throwable cause) {
+    super(cause);
+    this.exitCode = exitCode;
+  }
+
+  /**
+   * Create an exception with the specific exit code and text
+   * @param exitCode exit code
+   * @param message message to use in exception
+   */
+  public ServiceLaunchException(int exitCode, String message) {
+    super(message);
+    this.exitCode = exitCode;
+  }
+
+  /**
+   * Create an exception with the specific exit code, text and cause
+   * @param exitCode exit code
+   * @param message message to use in exception
+   * @param cause cause of the exception
+   */
+  public ServiceLaunchException(int exitCode, String message, Throwable cause) {
+    super(message, cause);
+    this.exitCode = exitCode;
+  }
+
+  /**
+   * Get the exit code
+   * @return the exit code
+   */
+  @Override
+  public int getExitCode() {
+    return exitCode;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ServiceLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ServiceLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ServiceLauncher.java
new file mode 100644
index 0000000..f192ec8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ServiceLauncher.java
@@ -0,0 +1,642 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.main;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.util.ExitUtil;
+import org.apache.hadoop.util.ShutdownHookManager;
+import org.apache.hadoop.util.VersionInfo;
+import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.net.MalformedURLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * A class to launch any service by name.
+ * 
+ * It's designed to be subclassed for custom entry points.
+ * 
+ * 
+ * Workflow
+ * <ol>
+ *   <li>An instance of the class is created</li>
+ *   <li>If it implements RunService, it is given the binding args off the CLI</li>
+ *   <li>Its service.init() and service.start() methods are called.</li>
+ *   <li>If it implements RunService, runService() is called and its return
+ *   code used as the exit code.</li>
+ *   <li>Otherwise: it waits for the service to stop, assuming in its start() method
+ *   it begins work</li>
+ *   <li>If an exception returned an exit code, that becomes the exit code of the
+ *   command.</li>
+ * </ol>
+ * Error and warning messages are logged to stderr. Why? If the classpath
+ * is wrong & logger configurations not on it, then no error messages by
+ * the started app will be seen and the caller is left trying to debug
+ * using exit codes. 
+ * 
+ */
+@SuppressWarnings("UseOfSystemOutOrSystemErr")
+public class ServiceLauncher<S extends Service>
+  implements LauncherExitCodes, IrqHandler.Interrupted,
+    Thread.UncaughtExceptionHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      ServiceLauncher.class);
+
+  protected static final int PRIORITY = 30;
+
+  public static final String NAME = "ServiceLauncher";
+
+  /**
+   * Name of the "--conf" argument. 
+   */
+  public static final String ARG_CONF = "--conf";
+
+  public static final String USAGE_MESSAGE =
+      "Usage: " + NAME + " classname [" + ARG_CONF +
+      "<conf file>] <service arguments> | ";
+  static final int SHUTDOWN_TIME_ON_INTERRUPT = 30 * 1000;
+
+  private volatile S service;
+  private int serviceExitCode;
+  @SuppressWarnings("MismatchedQueryAndUpdateOfCollection")
+  private final List<IrqHandler> interruptHandlers = new ArrayList<>(1);
+  private Configuration configuration;
+  private String serviceClassName;
+  private static AtomicBoolean signalAlreadyReceived = new AtomicBoolean(false);
+
+
+  /**
+   * Create an instance of the launcher
+   * @param serviceClassName classname of the service
+   */
+  public ServiceLauncher(String serviceClassName) {
+    this.serviceClassName = serviceClassName;
+  }
+
+  /**
+   * Get the service. Null until and unless
+   * {@link #launchService(Configuration, String[], boolean)} has completed
+   * @return the service
+   */
+  public S getService() {
+    return service;
+  }
+
+  /**
+   * Get the configuration constructed from the command line arguments
+   * @return the configuration used to create the service
+   */
+  public Configuration getConfiguration() {
+    return configuration;
+  }
+
+  /**
+   * The exit code from a successful service execution
+   * @return the exit code. 
+   */
+  public int getServiceExitCode() {
+    return serviceExitCode;
+  }
+
+  @Override
+  public String toString() {
+    return "ServiceLauncher for " + serviceClassName;
+  }
+
+  /**
+   * Launch the service, by creating it, initing it, starting it and then
+   * maybe running it. {@link RunService#bindArgs(Configuration, String...)} is invoked
+   * on the service between creation and init.
+   *
+   * All exceptions that occur are propagated upwards.
+   *
+   * If the method returns a status code, it means that it got as far starting
+   * the service, and if it implements {@link RunService}, that the 
+   * method {@link RunService#runService()} has completed. 
+   *
+   * At this point, the service is returned by {@link #getService()}.
+   *
+   * @param conf configuration
+   * @param processedArgs arguments after the configuration parameters
+   * have been stripped out.
+   * @param addProcessHooks should process failure handlers be added to
+   * terminate this service on shutdown. Tests should set this to false.
+   * @throws ClassNotFoundException classname not on the classpath
+   * @throws IllegalAccessException not allowed at the class
+   * @throws InstantiationException not allowed to instantiate it
+   * @throws InterruptedException thread interrupted
+   * @throws Throwable any other failure
+   */
+  public int launchService(Configuration conf,
+      String[] processedArgs,
+      boolean addProcessHooks)
+    throws Throwable {
+
+    instantiateService(conf);
+
+    // add any process shutdown hooks
+    if (addProcessHooks) {
+      ServiceShutdownHook shutdownHook = new ServiceShutdownHook(service);
+      ShutdownHookManager.get().addShutdownHook(shutdownHook, PRIORITY);
+    }
+    RunService runService = null;
+
+    if (service instanceof RunService) {
+      //if its a runService, pass in the conf and arguments before init)
+      runService = (RunService) service;
+      configuration = runService.bindArgs(configuration, processedArgs);
+      Preconditions.checkNotNull(configuration,
+          "null configuration returned by bindArgs()");
+    }
+
+    //some class constructors init; here this is picked up on.
+    if (!service.isInState(Service.STATE.INITED)) {
+      service.init(configuration);
+    }
+    service.start();
+    int exitCode = EXIT_SUCCESS;
+    if (runService != null) {
+      //assume that runnable services are meant to run from here
+      exitCode = runService.runService();
+      LOG.debug("Service exited with exit code {}", exitCode);
+
+    } else {
+      //run the service until it stops or an interrupt happens on a different thread.
+      LOG.debug("waiting for service threads to terminate");
+      service.waitForServiceToStop(0);
+    }
+    //exit
+    serviceExitCode = exitCode;
+    return serviceExitCode;
+  }
+
+  /**
+   * Instantiate the service defined in <code>serviceClassName</code>
+   * . Sets the <code>configuration</code> field
+   * to the configuration, and <code>service</code> to the service.
+   *
+   * @param conf configuration to use
+   * @throws ClassNotFoundException classname not on the classpath
+   * @throws IllegalAccessException not allowed at the class
+   * @throws InstantiationException not allowed to instantiate it
+   */
+  @SuppressWarnings("unchecked")
+  public Service instantiateService(Configuration conf)
+      throws ClassNotFoundException, InstantiationException, IllegalAccessException,
+      ExitUtil.ExitException, NoSuchMethodException, InvocationTargetException {
+    Preconditions.checkArgument(conf != null, "null conf");
+    configuration = conf;
+
+    //Instantiate the class -this requires the service to have a public
+    // zero-argument constructor
+    Class<?> serviceClass =
+        this.getClass().getClassLoader().loadClass(serviceClassName);
+    Object instance = serviceClass.getConstructor().newInstance();
+    if (!(instance instanceof Service)) {
+      //not a service
+      throw new ExitUtil.ExitException(EXIT_COMMAND_ARGUMENT_ERROR,
+          "Not a Service class: " + serviceClassName);
+    }
+
+    service = (S) instance;
+    return service;
+  }
+
+  /**
+   * Register this class as the handler for the control-C interrupt.
+   * Can be overridden for testing.
+   */
+  protected void registerInterruptHandler() {
+    try {
+      interruptHandlers.add(new IrqHandler(IrqHandler.CONTROL_C, this));
+      interruptHandlers.add(new IrqHandler(IrqHandler.SIGTERM, this));
+    } catch (IOException e) {
+      error("Signal handler setup failed : {}" + e, e);
+    }
+  }
+
+  /**
+   * The service has been interrupted -try to shut down the service.
+   * Give the service time to do this before the exit operation is called 
+   * @param interruptData the interrupted data.
+   */
+  @Override
+  public void interrupted(IrqHandler.InterruptData interruptData) {
+    String message = "Service interrupted by " + interruptData.toString();
+    warn(message);
+    if (!signalAlreadyReceived.compareAndSet(false, true)) {
+      warn("Repeated interrupt: escalating to a JVM halt");
+      // signal already received. On a second request to a hard JVM
+      // halt and so bypass any blocking shutdown hooks.
+      ExitUtil.halt(EXIT_INTERRUPTED, message);
+    }
+    int shutdownTimeMillis = SHUTDOWN_TIME_ON_INTERRUPT;
+    //start an async shutdown thread with a timeout
+    ServiceForcedShutdown forcedShutdown =
+      new ServiceForcedShutdown(shutdownTimeMillis);
+    Thread thread = new Thread(forcedShutdown);
+    thread.setDaemon(true);
+    thread.start();
+    //wait for that thread to finish
+    try {
+      thread.join(shutdownTimeMillis);
+    } catch (InterruptedException ignored) {
+      //ignored
+    }
+    if (!forcedShutdown.isServiceStopped()) {
+      warn("Service did not shut down in time");
+    }
+    exit(EXIT_INTERRUPTED, message);
+  }
+
+  /**
+   * Uncaught exception handler.
+   * If an error is raised: shutdown
+   * The state of the system is unknown at this point -attempting
+   * a clean shutdown is dangerous. Instead: exit
+   * @param thread thread that failed
+   * @param exception exception
+   */
+  @Override
+  public void uncaughtException(Thread thread, Throwable exception) {
+    if (ShutdownHookManager.get().isShutdownInProgress()) {
+      LOG.error("Thread {} threw an error during shutdown: {}.",
+          thread.toString(),
+          exception,
+          exception);
+    } else if (exception instanceof Error) {
+      try {
+        LOG.error("Thread {} threw an error: {}. Shutting down",
+            thread.toString(),
+            exception,
+            exception);
+      } catch (Throwable err) {
+        // We don't want to not exit because of an issue with logging
+      }
+      if (exception instanceof OutOfMemoryError) {
+        // After catching an OOM java says it is undefined behavior, so don't
+        // even try to clean up or we can get stuck on shutdown.
+        try {
+          System.err.println("Halting due to Out Of Memory Error...");
+        } catch (Throwable err) {
+          // Again we don't want to exit because of logging issues.
+        }
+        ExitUtil.halt(EXIT_EXCEPTION_THROWN);
+      } else {
+        // error other than OutOfMemory
+        exit(convertToExitException(exception));
+      }
+    } else {
+      // simple exception in a thread. There's a policy decision here:
+      // terminate the service vs. keep going after a thread has failed
+      LOG.error("Thread {} threw an exception: {}",
+          thread.toString(),
+          exception,
+          exception);
+    }
+  }
+
+  /**
+   * Print a warning: currently this goes to stderr
+   * @param text
+   */
+  protected void warn(String text) {
+    System.err.println(text);
+  }
+
+  /**
+   * Report an error. The message is printed to stderr; the exception
+   * is logged via the current logger.
+   * @param message message for the user
+   * @param thrown the exception thrown
+   */
+  protected void error(String message, Throwable thrown) {
+    String text = "Exception: " + message;
+    warn(text);
+    LOG.error(text, thrown);
+  }
+
+  /**
+   * Exit the code.
+   * This is method can be overridden for testing, throwing an 
+   * exception instead. Any subclassed method MUST raise an 
+   * <code>ExitUtil.ExitException</code> instance.
+   * The service launcher code assumes that after this method is invoked,
+   * no other code in the same method is called.
+   * @param exitCode code to exit
+   */
+  protected void exit(int exitCode, String message) {
+    ExitUtil.terminate(exitCode, message);
+  }
+
+  /**
+   * Exit off an exception. This can be subclassed for testing
+   * @param ee exit exception
+   */
+  protected void exit(ExitUtil.ExitException ee) {
+    ExitUtil.terminate(ee.status, ee);
+  }
+
+  /**
+   * Get the service name via {@link Service#getName()}.
+   * If the service is not instantiated, the classname is returned instead.
+   * @return the service name
+   */
+  public String getServiceName() {
+    Service s = service;
+    String name = null;
+    if (s != null) {
+      try {
+        name = s.getName();
+      } catch (Exception ignored) {
+        // ignored
+      }
+    }
+    if (name != null) {
+      return "service " + name;
+    } else {
+      return "service classname " + serviceClassName;
+    }
+  }
+
+  /**
+   * Parse the command line, building a configuration from it, then
+   * launch the service and wait for it to finish. finally, exit
+   * passing the status code to the #exit(int) method.
+   * @param args arguments to the service. arg[0] is 
+   * assumed to be the service classname and is automatically
+   */
+  public void launchServiceAndExit(List<String> args) {
+
+    registerInterruptHandler();
+    //Currently the config just the default
+    Configuration conf = new Configuration();
+    String[] processedArgs = extractConfigurationArgs(conf, args);
+    ExitUtil.ExitException ee = launchServiceRobustly(conf, processedArgs);
+    System.out.flush();
+    System.err.flush();
+    exit(ee);
+  }
+
+  /**
+   * Extract the configuration arguments and apply them to the configuration,
+   * building an array of processed arguments to hand down to the service.
+   *
+   * @param conf configuration to update
+   * @param args main arguments. args[0] is assumed to be the service
+   * classname and is skipped
+   * @return the processed list.
+   */
+  public static String[] extractConfigurationArgs(Configuration conf,
+                                                  List<String> args) {
+
+    //convert args to a list
+    int argCount = args.size();
+    if (argCount <= 1 ) {
+      return new String[0];
+    }
+    List<String> argsList = new ArrayList<String>(argCount);
+    ListIterator<String> arguments = args.listIterator();
+    //skip that first entry
+    arguments.next();
+    while (arguments.hasNext()) {
+      String arg = arguments.next();
+      if (arg.equals(ARG_CONF)) {
+        //the argument is a --conf file tuple: extract the path and load
+        //it in as a configuration resource.
+
+        //increment the loop iterator
+        if (!arguments.hasNext()) {
+          //overshot the end of the file
+          exitWithMessage(EXIT_COMMAND_ARGUMENT_ERROR,
+              ARG_CONF + ": missing configuration file after ");
+        }
+        File file = new File(arguments.next());
+        if (!file.exists()) {
+          exitWithMessage(EXIT_COMMAND_ARGUMENT_ERROR,
+              ARG_CONF + ": configuration file not found: " + file);
+        }
+        try {
+          conf.addResource(file.toURI().toURL());
+        } catch (MalformedURLException e) {
+          LOG.debug("File {} cannot be converted to URL", file, e);
+          exitWithMessage(EXIT_COMMAND_ARGUMENT_ERROR,
+              ARG_CONF + ": configuration file path invalid: " + file);
+        }
+      } else {
+        argsList.add(arg);
+      }
+    }
+    String[] processedArgs = new String[argsList.size()];
+    argsList.toArray(processedArgs);
+    return processedArgs;
+  }
+
+  /**
+   * Launch a service catching all exceptions and downgrading them to exit codes
+   * after logging.
+   * @param conf configuration to use
+   * @param processedArgs command line after the launcher-specific arguments have
+   * been stripped out
+   * @return an exit exception, which will have a status code of 0 if it worked
+   */
+  public ExitUtil.ExitException launchServiceRobustly(Configuration conf,
+                                   String[] processedArgs) {
+    ExitUtil.ExitException exitException;
+    try {
+      int exitCode = launchService(conf, processedArgs, true);
+      if (service != null) {
+        Throwable failure = service.getFailureCause();
+        if (failure != null) {
+          //the service exited with a failure.
+          //check what state it is in
+          Service.STATE failureState = service.getFailureState();
+          if (failureState == Service.STATE.STOPPED) {
+            //the failure occurred during shutdown, not important enough to bother
+            //the user as it may just scare them
+            LOG.debug("Failure during shutdown:{} ", failure, failure);
+          } else {
+            //throw it for the catch handlers to deal with
+            throw failure;
+          }
+        }
+      }
+      exitException = new ExitUtil.ExitException(exitCode,
+                                     "In " + serviceClassName);
+      // either the service succeeded, or an error raised during shutdown, 
+      // which we don't worry that much about
+    } catch (ExitUtil.ExitException ee) {
+      exitException = ee;
+    } catch (Throwable thrown) {
+      exitException = convertToExitException(thrown);
+    }
+    return exitException;
+  }
+
+  /**
+   * Convert the exception to one that can be handed off to ExitUtils;
+   * if it is of the write type it is passed throw as is. If not, a 
+   * new exception with the exit code {@link #EXIT_EXCEPTION_THROWN}
+   * is created, with the argument <code>thrown</code> as the inner cause
+   * @param thrown the exception thrown
+   * @return an exception to terminate the process with
+   */
+  protected ExitUtil.ExitException convertToExitException(Throwable thrown) {
+    ExitUtil.ExitException exitException;
+    int exitCode;
+    String message = thrown.getMessage();
+    if (message == null) {
+      message = thrown.toString();
+    }
+    if (thrown instanceof ExitCodeProvider) {
+      exitCode = ((ExitCodeProvider) thrown).getExitCode();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("While running {}: {}", getServiceName(), message, thrown);
+      }
+      LOG.error(message);
+    } else {
+      // not any of the service launcher exceptions -assume something worse
+      error(message, thrown);
+      exitCode = EXIT_EXCEPTION_THROWN;
+    }
+    exitException = new ExitUtil.ExitException(exitCode, message);
+    exitException.initCause(thrown);
+    return exitException;
+  }
+
+
+  /**
+   * Build a log message for starting up and shutting down. 
+   * This was grabbed from the ToolRunner code.
+   * @param classname the class of the server
+   * @param args arguments
+   */
+  public static String startupShutdownMessage(String classname,
+                                              List<String> args) {
+    final String hostname = NetUtils.getHostname();
+
+    return toStartupShutdownString("STARTUP_MSG: ", new String[]{
+      "Starting " + classname,
+      "  host = " + hostname,
+      "  args = " + args,
+      "  version = " + VersionInfo.getVersion(),
+      "  classpath = " + System.getProperty("java.class.path"),
+      "  build = " + VersionInfo.getUrl() + " -r "
+      + VersionInfo.getRevision()
+      + "; compiled by '" + VersionInfo.getUser()
+      + "' on " + VersionInfo.getDate(),
+      "  java = " + System.getProperty("java.version")
+    });
+  }
+
+  /**
+   * Exit with a printed message
+   * @param status status code
+   * @param message message
+   */
+  private static void exitWithMessage(int status, String message) {
+    System.err.println(message);
+    ExitUtil.terminate(status);
+  }
+
+  private static String toStartupShutdownString(String prefix, String[] msg) {
+    StringBuilder b = new StringBuilder(prefix);
+    b.append("\n/************************************************************");
+    for (String s : msg) {
+      b.append("\n").append(prefix).append(s);
+    }
+    b.append("\n************************************************************/");
+    return b.toString();
+  }
+
+  /**
+   * forced shutdown runnable.
+   */
+  protected class ServiceForcedShutdown implements Runnable {
+
+    private final int shutdownTimeMillis;
+    private boolean serviceStopped;
+
+    public ServiceForcedShutdown(int shutdownTimeoutMillis) {
+      this.shutdownTimeMillis = shutdownTimeoutMillis;
+    }
+
+    @Override
+    public void run() {
+      if (service != null) {
+        service.stop();
+        serviceStopped = service.waitForServiceToStop(shutdownTimeMillis);
+      } else {
+        serviceStopped = true;
+      }
+    }
+
+    private boolean isServiceStopped() {
+      return serviceStopped;
+    }
+  }
+
+  /**
+   * The real main function, which takes the arguments as a list
+   * arg 0 must be the service classname
+   * @param argsList the list of arguments
+   */
+  public static void serviceMain(List<String> argsList) {
+    if (argsList.isEmpty()) {
+      exitWithMessage(EXIT_USAGE, USAGE_MESSAGE);
+    } else {
+      String serviceClassName = argsList.get(0);
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(startupShutdownMessage(serviceClassName, argsList));
+        StringBuilder builder = new StringBuilder();
+        for (String arg : argsList) {
+          builder.append('"').append(arg).append("\" ");
+        }
+        LOG.debug(builder.toString());
+      }
+      Thread.setDefaultUncaughtExceptionHandler(
+        new YarnUncaughtExceptionHandler());
+
+      ServiceLauncher serviceLauncher = new ServiceLauncher<>(serviceClassName);
+      serviceLauncher.launchServiceAndExit(argsList);
+    }
+  }
+
+  /**
+   * This is the main entry point for the service launcher.
+   * @param args command line arguments.
+   */
+  public static void main(String[] args) {
+    List<String> argsList = Arrays.asList(args);
+    serviceMain(argsList);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ServiceShutdownHook.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ServiceShutdownHook.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ServiceShutdownHook.java
new file mode 100644
index 0000000..de55789
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ServiceShutdownHook.java
@@ -0,0 +1,80 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.core.main;
+
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.util.ShutdownHookManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.ref.WeakReference;
+
+/**
+ * JVM Shutdown hook for Service which will stop the
+ * Service gracefully in case of JVM shutdown.
+ * This hook uses a weak reference to the service, so
+ * does not cause services to be retained after they have
+ * been stopped and deferenced elsewhere.
+ */
+public class ServiceShutdownHook implements Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      ServiceShutdownHook.class);
+
+  private final WeakReference<Service> serviceRef;
+  private Runnable hook;
+
+  public ServiceShutdownHook(Service service) {
+    serviceRef = new WeakReference<>(service);
+  }
+
+  public void register(int priority) {
+    unregister();
+    hook = this;
+    ShutdownHookManager.get().addShutdownHook(hook, priority);
+  }
+
+  public synchronized void unregister() {
+    if (hook != null) {
+      try {
+        ShutdownHookManager.get().removeShutdownHook(hook);
+      } catch (IllegalStateException e) {
+        LOG.info("Failed to unregister shutdown hook: {}", e, e);
+      }
+      hook = null;
+    }
+  }
+
+  @Override
+  public void run() {
+    Service service;
+    synchronized (this) {
+      service = serviceRef.get();
+      serviceRef.clear();
+    }
+    if (service == null) {
+      return;
+    }
+    try {
+      // Stop the  Service
+      service.stop();
+    } catch (Throwable t) {
+      LOG.info("Error stopping {}", service.getName(), t);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/AggregateConfSerDeser.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/AggregateConfSerDeser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/AggregateConfSerDeser.java
new file mode 100644
index 0000000..90537b6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/AggregateConfSerDeser.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.persist;
+
+import org.apache.slider.core.conf.AggregateConf;
+import org.codehaus.jackson.JsonGenerationException;
+import org.codehaus.jackson.JsonParseException;
+import org.codehaus.jackson.map.JsonMappingException;
+
+import java.io.IOException;
+
+/**
+ * Conf tree to JSON binding
+ */
+public class AggregateConfSerDeser extends JsonSerDeser<AggregateConf> {
+  public AggregateConfSerDeser() {
+    super(AggregateConf.class);
+  }
+
+
+  private static final AggregateConfSerDeser
+      staticinstance = new AggregateConfSerDeser();
+
+  /**
+   * Convert a tree instance to a JSON string -sync access to a shared ser/deser
+   * object instance
+   * @param instance object to convert
+   * @return a JSON string description
+   * @throws JsonParseException parse problems
+   * @throws JsonMappingException O/J mapping problems
+   */
+  public static String toString(AggregateConf instance) throws IOException,
+                                                          JsonGenerationException,
+                                                          JsonMappingException {
+    synchronized (staticinstance) {
+      return staticinstance.toJson(instance);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/AppDefinitionPersister.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/AppDefinitionPersister.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/AppDefinitionPersister.java
new file mode 100644
index 0000000..7fb3158
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/AppDefinitionPersister.java
@@ -0,0 +1,260 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.persist;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.io.Files;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.fs.Path;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.params.AbstractClusterBuildingActionArgs;
+import org.apache.slider.common.params.Arguments;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.providers.agent.AgentKeys;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Class to prepare and persist app and add-on definitions.
+ *
+ * In this case, the app definition and add-on definitions are auto-inferred from the user input rather than explicit
+ * inclusion of application package in the config.
+ *
+ * Processing an app definition involves one or more of the following: - modify appConfig - package definition into a
+ * temporary folder - upload to HDFS
+ *
+ * This class keeps track of all the required operations and allows them to be invoked by build operation
+ */
+public class AppDefinitionPersister {
+  private static final Logger log =
+      LoggerFactory.getLogger(AppDefinitionPersister.class);
+
+  private final SliderFileSystem sliderFileSystem;
+  private List<AppDefinition> appDefinitions;
+
+  public AppDefinitionPersister(SliderFileSystem sliderFileSystem) {
+    this.sliderFileSystem = sliderFileSystem;
+    appDefinitions = new ArrayList<>();
+  }
+
+
+  /**
+   * Process the application package or folder by copying it to the cluster path
+   *
+   * @param appDefinition details of application package
+   *
+   * @throws BadConfigException
+   * @throws IOException
+   */
+  private void persistDefinitionPackageOrFolder(AppDefinition appDefinition)
+      throws BadConfigException, IOException {
+    if (!appDefinition.appDefPkgOrFolder.canRead()) {
+      throw new BadConfigException("Pkg/Folder cannot be accessed - "
+                                   + appDefinition.appDefPkgOrFolder.getAbsolutePath());
+    }
+
+    File src = appDefinition.appDefPkgOrFolder;
+    String targetName = appDefinition.pkgName;
+    log.debug("Package name: " + targetName);
+    if (appDefinition.appDefPkgOrFolder.isDirectory()) {
+      log.info("Processing app package/folder {} for {}",
+               appDefinition.appDefPkgOrFolder.getAbsolutePath(),
+               appDefinition.pkgName);
+      File tmpDir = Files.createTempDir();
+      File zipFile = new File(tmpDir.getCanonicalPath(), File.separator + appDefinition.pkgName);
+      SliderUtils.zipFolder(appDefinition.appDefPkgOrFolder, zipFile);
+      src = zipFile;
+    }
+
+    sliderFileSystem.getFileSystem().copyFromLocalFile(
+        false,
+        false,
+        new Path(src.toURI()),
+        new Path(appDefinition.targetFolderInFs, targetName));
+  }
+
+  public void persistPackages() throws BadConfigException, IOException {
+    for (AppDefinition appDefinition : appDefinitions) {
+      persistDefinitionPackageOrFolder(appDefinition);
+    }
+  }
+
+  public void processSuppliedDefinitions(String clustername,
+                                         AbstractClusterBuildingActionArgs buildInfo,
+                                         ConfTreeOperations appConf)
+      throws BadConfigException, IOException, BadCommandArgumentsException {
+    // if metainfo is provided add to the app instance
+    if (buildInfo.appMetaInfo != null || buildInfo.appMetaInfoJson != null) {
+      if (buildInfo.appMetaInfo != null && buildInfo.appMetaInfoJson != null) {
+        throw new BadConfigException("Both %s and %s cannot be specified",
+            Arguments.ARG_METAINFO, Arguments.ARG_METAINFO_JSON);
+      }
+
+      // Now we know that only one of either file or JSON is used
+      boolean isFileUsed = buildInfo.appMetaInfo != null ? true : false;
+      String argUsed = isFileUsed ? Arguments.ARG_METAINFO
+          : Arguments.ARG_METAINFO_JSON;
+
+      if (buildInfo.appDef != null) {
+        throw new BadConfigException("Both %s and %s cannot be specified",
+            argUsed, Arguments.ARG_APPDEF);
+      }
+      if (SliderUtils.isSet(appConf.getGlobalOptions().get(AgentKeys.APP_DEF))) {
+        throw new BadConfigException(
+            "%s cannot not be set if %s is specified in the cmd line ",
+            AgentKeys.APP_DEF, argUsed);
+      }
+
+      if (isFileUsed) {
+        if (!buildInfo.appMetaInfo.canRead() || !buildInfo.appMetaInfo.isFile()) {
+          throw new BadConfigException(
+              "Path specified with %s either cannot be read or is not a file",
+              Arguments.ARG_METAINFO);
+        }
+      } else {
+        if (StringUtils.isEmpty(buildInfo.appMetaInfoJson.trim())) {
+          throw new BadConfigException("Empty string specified with %s",
+              Arguments.ARG_METAINFO_JSON);
+        }
+      }
+
+      File tempDir = Files.createTempDir();
+      File pkgSrcDir = new File(tempDir, "default");
+      pkgSrcDir.mkdirs();
+      File destMetaInfo = new File(pkgSrcDir, "metainfo.json");
+      if (isFileUsed) {
+        if (buildInfo.appMetaInfo.getName().endsWith(".xml")) {
+          Files.copy(buildInfo.appMetaInfo, new File(pkgSrcDir, "metainfo.xml"));
+        } else {
+          Files.copy(buildInfo.appMetaInfo, destMetaInfo);
+        }
+      } else {
+        Files.write(
+            buildInfo.appMetaInfoJson.getBytes(Charset.forName("UTF-8")),
+            destMetaInfo);
+      }
+
+      Path appDirPath = sliderFileSystem.buildAppDefDirPath(clustername);
+      log.info("Using default app def path {}", appDirPath.toString());
+
+      appDefinitions.add(new AppDefinition(appDirPath, pkgSrcDir,
+          SliderKeys.DEFAULT_APP_PKG));
+      Path appDefPath = new Path(appDirPath, SliderKeys.DEFAULT_APP_PKG);
+      appConf.getGlobalOptions().set(AgentKeys.APP_DEF, appDefPath);
+      log.info("Setting app package to {}.", appDefPath);
+    }
+
+    if (buildInfo.appDef != null) {
+      if (SliderUtils.isSet(appConf.getGlobalOptions().get(AgentKeys.APP_DEF))) {
+        throw new BadConfigException("application.def must not be set if --appdef is provided.");
+      }
+
+      if (!buildInfo.appDef.exists()) {
+        throw new BadConfigException("--appdef is not a valid path.");
+      }
+
+      Path appDirPath = sliderFileSystem.buildAppDefDirPath(clustername);
+      appDefinitions.add(new AppDefinition(appDirPath, buildInfo.appDef, SliderKeys.DEFAULT_APP_PKG));
+      Path appDefPath = new Path(appDirPath, SliderKeys.DEFAULT_APP_PKG);
+      appConf.getGlobalOptions().set(AgentKeys.APP_DEF, appDefPath);
+      log.info("Setting app package to {}.", appDefPath);
+    }
+
+    if (buildInfo.addonDelegate.getAddonMap().size() > 0) {
+      if (SliderUtils.isUnset(appConf.getGlobalOptions().get(AgentKeys.APP_DEF))) {
+        throw new BadConfigException("addon package can only be specified if main app package is specified.");
+      }
+
+      List<String> addons = new ArrayList<String>();
+      Map<String, String> addonMap = buildInfo.addonDelegate.getAddonMap();
+      for (String key : addonMap.keySet()) {
+        File defPath = new File(addonMap.get(key));
+        if (SliderUtils.isUnset(addonMap.get(key))) {
+          throw new BadConfigException("Invalid path for addon package " + key);
+        }
+
+        if (!defPath.exists()) {
+          throw new BadConfigException("addon folder or package path is not valid.");
+        }
+
+        Path addonPath = sliderFileSystem.buildAddonDirPath(clustername, key);
+        String addonPkgName = "addon_" + key + ".zip";
+
+        log.debug(
+            "addonMap.get(key): {} addonPath: {} defPath: {} addonPkgName: {}",
+            addonMap.get(key), addonPath, defPath, addonPkgName);
+
+        appDefinitions.add(new AppDefinition(addonPath, defPath, addonPkgName));
+        String addOnKey = AgentKeys.ADDON_PREFIX + key;
+        Path addonPkgPath = new Path(addonPath, addonPkgName);
+        log.info("Setting addon package {} to {}.", addOnKey, addonPkgPath);
+        appConf.getGlobalOptions().set(addOnKey, addonPkgPath);
+        addons.add(addOnKey);
+      }
+
+      String existingList = appConf.getGlobalOptions().get(AgentKeys.ADDONS);
+      if (SliderUtils.isUnset(existingList)) {
+        existingList = "";
+      }
+      appConf.getGlobalOptions().set(AgentKeys.ADDONS, existingList + StringUtils.join(addons, ","));
+    }
+  }
+
+
+  @VisibleForTesting
+  public List<AppDefinitionPersister.AppDefinition> getAppDefinitions() {
+    return appDefinitions;
+  }
+
+  // Helper class to hold details for the app and addon packages
+  public class AppDefinition {
+    // The target folder where the package will be stored
+    public Path targetFolderInFs;
+    // The on disk location of the app def package or folder
+    public File appDefPkgOrFolder;
+    // Package name
+    public String pkgName;
+
+    public AppDefinition(Path targetFolderInFs, File appDefPkgOrFolder, String pkgName) {
+      this.targetFolderInFs = targetFolderInFs;
+      this.appDefPkgOrFolder = appDefPkgOrFolder;
+      this.pkgName = pkgName;
+    }
+
+    @Override
+    public String toString() {
+      return new StringBuilder().append("targetFolderInFs").append(" : ").append(targetFolderInFs.toString())
+          .append(", ")
+          .append("appDefPkgOrFolder").append(" : ").append(appDefPkgOrFolder.toString())
+          .append(", ")
+          .append("pkgName").append(" : ").append(pkgName).toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/ApplicationReportSerDeser.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/ApplicationReportSerDeser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/ApplicationReportSerDeser.java
new file mode 100644
index 0000000..a8c72ce
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/ApplicationReportSerDeser.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.persist;
+
+import org.apache.slider.core.launch.SerializedApplicationReport;
+import org.codehaus.jackson.JsonGenerationException;
+import org.codehaus.jackson.JsonParseException;
+import org.codehaus.jackson.map.JsonMappingException;
+
+import java.io.IOException;
+
+/**
+ * Persistence of {@link SerializedApplicationReport}
+ * 
+ */
+public class ApplicationReportSerDeser
+    extends JsonSerDeser<SerializedApplicationReport> {
+  public ApplicationReportSerDeser() {
+    super(SerializedApplicationReport.class);
+  }
+
+
+  private static final ApplicationReportSerDeser
+      staticinstance = new ApplicationReportSerDeser();
+
+  /**
+   * Convert an instance to a JSON string -sync access to a shared ser/deser
+   * object instance
+   * @param instance object to convert
+   * @return a JSON string description
+   * @throws JsonParseException parse problems
+   * @throws JsonMappingException O/J mapping problems
+   */
+  public static String toString(SerializedApplicationReport instance)
+      throws IOException, JsonGenerationException, JsonMappingException {
+    synchronized (staticinstance) {
+      return staticinstance.toJson(instance);
+    }
+  }
+ 
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/ConfPersister.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/ConfPersister.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/ConfPersister.java
new file mode 100644
index 0000000..9759205
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/ConfPersister.java
@@ -0,0 +1,286 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.persist;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.slider.common.tools.CoreFileSystem;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.exceptions.SliderException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Date;
+
+/**
+ * Class to implement persistence of a configuration.
+ *
+ * This code contains the logic to acquire and release locks.
+ * # writelock MUST be acquired exclusively for writes. This is done
+ * by creating the file with no overwrite
+ * # shared readlock MUST be acquired for reads. This is done by creating the readlock
+ * file with overwrite forbidden -but treating a failure as a sign that
+ * the lock exists, and therefore the operation can continue.
+ * # releaselock is only released if the client created it.
+ * # after acquiring either lock, client must check for the alternate lock
+ * existing. If it is, release lock and fail.
+ * 
+ * There's one small race here: multiple readers; first reader releases lock
+ * while second is in use. 
+ * 
+ * Strict Fix: client checks for readlock after read completed.
+ * If it is not there, problem: fail. But this massively increases the risk of
+ * false negatives.
+ * 
+ * This isn't 100% perfect, because of the condition where the owner releases
+ * a lock, a writer grabs its lock & writes to it, the reader gets slightly
+ * contaminated data:
+ * own-share-delete-write-own-release(shared)-delete
+ * 
+ * We are assuming that the rate of change is low enough that this is rare, and
+ * of limited damage.
+ * 
+ * ONCE A CLUSTER IS RUNNING, ONLY THE AM MAY PERSIST UPDATES VIA ITS APIs
+ * 
+ * That is: outside the AM, a writelock MUST only be acquired after verifying there is no
+ * running application.
+ */
+public class ConfPersister {
+  private static final Logger log =
+    LoggerFactory.getLogger(ConfPersister.class);
+
+
+  private final ConfTreeSerDeser confTreeSerDeser = new ConfTreeSerDeser();
+
+  private final CoreFileSystem coreFS;
+  private final FileSystem fileSystem;
+  private final Path persistDir;
+  private final Path internal, resources, app_conf;
+  private final Path writelock, readlock;
+
+  public ConfPersister(CoreFileSystem coreFS, Path persistDir) {
+    this.coreFS = coreFS;
+    this.persistDir = persistDir;
+    internal = new Path(persistDir, Filenames.INTERNAL);
+    resources = new Path(persistDir, Filenames.RESOURCES);
+    app_conf = new Path(persistDir, Filenames.APPCONF);
+    writelock = new Path(persistDir, Filenames.WRITELOCK);
+    readlock = new Path(persistDir, Filenames.READLOCK);
+    fileSystem = coreFS.getFileSystem();
+  }
+
+  /**
+   * Get the target directory
+   * @return the directory for persistence
+   */
+  public Path getPersistDir() {
+    return persistDir;
+  }
+
+  /**
+   * Make the persistent directory
+   * @throws IOException IO failure
+   */
+  public void mkPersistDir() throws IOException {
+    coreFS.getFileSystem().mkdirs(persistDir);
+  }
+  
+  @Override
+  public String toString() {
+    return "Persister to " + persistDir;
+  }
+
+  /**
+   * Acquire the writelock
+   * @throws IOException IO
+   * @throws LockAcquireFailedException
+   */
+  @VisibleForTesting
+  void acquireWritelock() throws IOException,
+                                 LockAcquireFailedException {
+    mkPersistDir();
+    long now = System.currentTimeMillis();
+    try {
+      coreFS.cat(writelock, false, new Date(now).toGMTString());
+    } catch (FileAlreadyExistsException e) {
+      // filesystems should raise this (HDFS does)
+      throw new LockAcquireFailedException(writelock);
+    } catch (IOException e) {
+      // some filesystems throw a generic IOE
+      throw new LockAcquireFailedException(writelock, e);
+    }
+    //here the lock is acquired, but verify there is no readlock
+    boolean lockFailure;
+    try {
+      lockFailure = readLockExists();
+    } catch (IOException e) {
+      lockFailure = true;
+    }
+    if (lockFailure) {
+      releaseWritelock();
+      throw new LockAcquireFailedException(readlock);
+    }
+  }
+
+  @VisibleForTesting
+  boolean readLockExists() throws IOException {
+    return fileSystem.exists(readlock);
+  }
+
+  /**
+   * Release the writelock if it is present.
+   * IOExceptions are logged
+   */
+  @VisibleForTesting
+  boolean releaseWritelock() {
+    try {
+      return fileSystem.delete(writelock, false);
+    } catch (IOException e) {
+      log.warn("IOException releasing writelock {} ", writelock, e);
+    }
+    return false;
+  }
+  
+  /**
+   * Acquire the writelock
+   * @throws IOException IO
+   * @throws LockAcquireFailedException
+   * @throws FileNotFoundException if the target dir does not exist.
+   */
+  @VisibleForTesting
+  boolean acquireReadLock() throws FileNotFoundException,
+                                  IOException,
+                                  LockAcquireFailedException {
+    if (!coreFS.getFileSystem().exists(persistDir)) {
+      // the dir is not there, so the data is not there, so there
+      // is nothing to read
+      throw new FileNotFoundException(persistDir.toString());
+    }
+    long now = System.currentTimeMillis();
+    boolean owner;
+    try {
+      coreFS.cat(readlock, false, new Date(now).toGMTString());
+      owner = true;
+    } catch (IOException e) {
+      owner = false;
+    }
+    //here the lock is acquired, but verify there is no readlock
+    boolean lockFailure;
+    try {
+      lockFailure = writelockExists();
+    } catch (IOException e) {
+      lockFailure = true;
+    }
+    if (lockFailure) {
+      releaseReadlock(owner);
+      throw new LockAcquireFailedException(writelock);
+    }
+    return owner;
+  }
+
+  @VisibleForTesting
+  boolean writelockExists() throws IOException {
+    return fileSystem.exists(writelock);
+  }
+
+  /**
+   * Release the writelock if it is present.
+   * IOExceptions are downgraded to failures
+   * @return true if the lock was present and then released  
+   */
+  @VisibleForTesting
+  boolean releaseReadlock(boolean owner) {
+    if (owner) {
+      try {
+        return fileSystem.delete(readlock, false);
+      } catch (IOException e) {
+        log.warn("IOException releasing writelock {} ", readlock, e);
+      }
+    }
+    return false;
+  }
+
+  private void saveConf(AggregateConf conf) throws IOException {
+    confTreeSerDeser.save(fileSystem, internal, conf.getInternal(), true);
+    confTreeSerDeser.save(fileSystem, resources, conf.getResources(), true);
+    confTreeSerDeser.save(fileSystem, app_conf, conf.getAppConf(), true);
+  }
+
+  private void loadConf(AggregateConf conf) throws IOException {
+    conf.setInternal(confTreeSerDeser.load(fileSystem, internal));
+    conf.setResources(confTreeSerDeser.load(fileSystem, resources));
+    conf.setAppConf(confTreeSerDeser.load(fileSystem, app_conf));
+  }
+
+
+  private void maybeExecLockHeldAction(LockHeldAction action) throws
+      IOException,
+      SliderException {
+    if (action != null) {
+      action.execute();
+    }
+  }
+  
+  /**
+   * Save the configuration
+   * @param conf configuration to fill in
+   * @param action
+   * @throws IOException IO problems
+   * @throws LockAcquireFailedException the lock could not be acquired
+   */
+  public void save(AggregateConf conf, LockHeldAction action) throws
+      IOException,
+      SliderException,
+      LockAcquireFailedException {
+    acquireWritelock();
+    try {
+      saveConf(conf);
+      maybeExecLockHeldAction(action);
+    } finally {
+      releaseWritelock();
+    }
+  }
+
+  /**
+   * Load the configuration. If a lock failure is raised, the 
+   * contents of the configuration MAY have changed -lock race conditions
+   * are looked for on exit
+   * @param conf configuration to fill in
+   * @throws IOException IO problems
+   * @throws LockAcquireFailedException the lock could not be acquired
+   */
+  public void load(AggregateConf conf) throws
+      FileNotFoundException,
+      IOException,
+      SliderException,
+      LockAcquireFailedException {
+    boolean owner = acquireReadLock();
+    try {
+      loadConf(conf);
+    } finally {
+      releaseReadlock(owner);
+    }
+  }
+  
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/ConfTreeSerDeser.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/ConfTreeSerDeser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/ConfTreeSerDeser.java
new file mode 100644
index 0000000..8271ef1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/ConfTreeSerDeser.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.persist;
+
+import org.apache.slider.core.conf.ConfTree;
+import org.codehaus.jackson.JsonGenerationException;
+import org.codehaus.jackson.JsonParseException;
+import org.codehaus.jackson.map.JsonMappingException;
+
+import java.io.IOException;
+
+/**
+ * Conf tree to JSON binding
+ */
+public class ConfTreeSerDeser extends JsonSerDeser<ConfTree> {
+  public ConfTreeSerDeser() {
+    super(ConfTree.class);
+  }
+
+
+  private static final ConfTreeSerDeser staticinstance = new ConfTreeSerDeser();
+
+  /**
+   * Convert a tree instance to a JSON string -sync access to a shared ser/deser
+   * object instance
+   * @param instance object to convert
+   * @return a JSON string description
+   * @throws JsonParseException parse problems
+   * @throws JsonMappingException O/J mapping problems
+   */
+  public static String toString(ConfTree instance) throws IOException,
+                                                          JsonGenerationException,
+                                                          JsonMappingException {
+    synchronized (staticinstance) {
+      return staticinstance.toJson(instance);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/Filenames.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/Filenames.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/Filenames.java
new file mode 100644
index 0000000..06ecc51
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/Filenames.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.persist;
+
+public interface Filenames {
+
+  String RESOURCES = "resources.json";
+  String APPCONF = "app_config.json";
+  String INTERNAL = "internal.json";
+  String WRITELOCK = "writelock";
+  String READLOCK = "readlock";
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/InstancePaths.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/InstancePaths.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/InstancePaths.java
new file mode 100644
index 0000000..3505ac3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/InstancePaths.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.persist;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.slider.common.SliderKeys;
+
+/**
+ * Build up all the paths of an instance relative to the supplied instance
+ * directory.
+ */
+public class InstancePaths {
+
+  public final Path instanceDir;
+  public final Path snapshotConfPath;
+  public final Path generatedConfPath;
+  public final Path historyPath;
+  public final Path dataPath;
+  public final Path tmpPath;
+  public final Path tmpPathAM;
+  public final Path appDefPath;
+  public final Path addonsPath;
+
+  public InstancePaths(Path instanceDir) {
+    this.instanceDir = instanceDir;
+    snapshotConfPath =
+      new Path(instanceDir, SliderKeys.SNAPSHOT_CONF_DIR_NAME);
+    generatedConfPath =
+      new Path(instanceDir, SliderKeys.GENERATED_CONF_DIR_NAME);
+    historyPath = new Path(instanceDir, SliderKeys.HISTORY_DIR_NAME);
+    dataPath = new Path(instanceDir, SliderKeys.DATA_DIR_NAME);
+    tmpPath = new Path(instanceDir, SliderKeys.TMP_DIR_PREFIX);
+    tmpPathAM = new Path(tmpPath, SliderKeys.AM_DIR_PREFIX);
+    appDefPath = new Path(tmpPath, SliderKeys.APP_DEF_DIR);
+    addonsPath = new Path(tmpPath, SliderKeys.ADDONS_DIR);
+  }
+
+  @Override
+  public String toString() {
+    return "instance at " + instanceDir;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/JsonSerDeser.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/JsonSerDeser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/JsonSerDeser.java
new file mode 100644
index 0000000..4f60c06
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/JsonSerDeser.java
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.persist;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.codehaus.jackson.JsonGenerationException;
+import org.codehaus.jackson.JsonParseException;
+import org.codehaus.jackson.map.DeserializationConfig;
+import org.codehaus.jackson.map.JsonMappingException;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.SerializationConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * Support for marshalling objects to and from JSON.
+ * This class is NOT thread safe; it constructs an object mapper
+ * as an instance field.
+ * @param <T>
+ */
+public class JsonSerDeser<T> {
+
+  private static final Logger log = LoggerFactory.getLogger(JsonSerDeser.class);
+  private static final String UTF_8 = "UTF-8";
+
+  private final Class<T> classType;
+  private final ObjectMapper mapper;
+
+  /**
+   * Create an instance bound to a specific type
+   * @param classType class type
+   */
+  public JsonSerDeser(Class<T> classType) {
+    this.classType = classType;
+    this.mapper = new ObjectMapper();
+    mapper.configure(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+  }
+
+  /**
+   * Convert from JSON
+   * @param json input
+   * @return the parsed JSON
+   * @throws IOException IO
+   * @throws JsonMappingException failure to map from the JSON to this class
+   */
+  public T fromJson(String json)
+    throws IOException, JsonParseException, JsonMappingException {
+    try {
+      return mapper.readValue(json, classType);
+    } catch (IOException e) {
+      log.error("Exception while parsing json : " + e + "\n" + json, e);
+      throw e;
+    }
+  }
+
+  /**
+   * Convert from a JSON file
+   * @param jsonFile input file
+   * @return the parsed JSON
+   * @throws IOException IO problems
+   * @throws JsonMappingException failure to map from the JSON to this class
+   */
+  public T fromFile(File jsonFile)
+    throws IOException, JsonParseException, JsonMappingException {
+    File absoluteFile = jsonFile.getAbsoluteFile();
+    try {
+      return mapper.readValue(absoluteFile, classType);
+    } catch (IOException e) {
+      log.error("Exception while parsing json file {}", absoluteFile, e);
+      throw e;
+    }
+  }
+
+  /**
+   * Convert from a JSON file
+   * @param resource input file
+   * @return the parsed JSON
+   * @throws IOException IO problems
+   * @throws JsonMappingException failure to map from the JSON to this class
+   */
+ public T fromResource(String resource)
+    throws IOException, JsonParseException, JsonMappingException {
+    try(InputStream resStream = this.getClass().getResourceAsStream(resource)) {
+      if (resStream == null) {
+        throw new FileNotFoundException(resource);
+      }
+      return (T) (mapper.readValue(resStream, classType));
+    } catch (IOException e) {
+      log.error("Exception while parsing json resource {}", resource, e);
+      throw e;
+    }
+  }
+
+  /**
+   * Convert from an input stream, closing the stream afterwards.
+   * @param stream
+   * @return the parsed JSON
+   * @throws IOException IO problems
+   */
+  public T fromStream(InputStream stream) throws IOException {
+    try {
+      return (T) (mapper.readValue(stream, classType));
+    } catch (IOException e) {
+      log.error("Exception while parsing json input stream", e);
+      throw e;
+    } finally {
+      IOUtils.closeStream(stream);
+    }
+  }
+
+  /**
+   * clone by converting to JSON and back again.
+   * This is much less efficient than any Java clone process.
+   * @param instance instance to duplicate
+   * @return a new instance
+   * @throws IOException problems.
+   */
+  public T fromInstance(T instance) throws IOException {
+    return fromJson(toJson(instance));
+  }
+
+  /**
+   * Deserialize from a byte array
+   * @param b
+   * @return the deserialized value
+   * @throws IOException parse problems
+   */
+  public T fromBytes(byte[] b) throws IOException {
+    String json = new String(b, 0, b.length, UTF_8);
+    return fromJson(json);
+  }
+  
+  /**
+   * Load from a Hadoop filesystem
+   * @param fs filesystem
+   * @param path path
+   * @return a loaded CD
+   * @throws IOException IO problems
+   * @throws JsonParseException parse problems
+   * @throws JsonMappingException O/J mapping problems
+   */
+  public T load(FileSystem fs, Path path)
+    throws IOException, JsonParseException, JsonMappingException {
+    FileStatus status = fs.getFileStatus(path);
+    long len = status.getLen();
+    byte[] b = new byte[(int) len];
+    FSDataInputStream dataInputStream = fs.open(path);
+    int count = dataInputStream.read(b);
+    if (count != len) {
+      throw new EOFException("Read of " + path +" finished prematurely");
+    }
+    return fromBytes(b);
+  }
+
+
+  /**
+   * Save to a hadoop filesystem
+   * @param fs filesystem
+   * @param path path
+   * @param instance instance to save
+   * @param overwrite should any existing file be overwritten
+   * @throws IOException IO exception
+   */
+  public void save(FileSystem fs, Path path, T instance,
+                   boolean overwrite) throws
+                                      IOException {
+    FSDataOutputStream dataOutputStream = fs.create(path, overwrite);
+    writeJsonAsBytes(instance, dataOutputStream);
+  }
+
+  /**
+   * Save an instance to a file
+   * @param instance instance to save
+   * @param file file
+   * @throws IOException
+   */
+  public void save(T instance, File file) throws
+      IOException {
+    writeJsonAsBytes(instance, new FileOutputStream(file.getAbsoluteFile()));
+  }
+
+  /**
+   * Write the json as bytes -then close the file
+   * @param dataOutputStream an outout stream that will always be closed
+   * @throws IOException on any failure
+   */
+  private void writeJsonAsBytes(T instance,
+      OutputStream dataOutputStream) throws IOException {
+    try {
+      String json = toJson(instance);
+      byte[] b = json.getBytes(UTF_8);
+      dataOutputStream.write(b);
+      dataOutputStream.flush();
+      dataOutputStream.close();
+    } finally {
+      IOUtils.closeStream(dataOutputStream);
+    }
+  }
+
+  /**
+   * Convert an object to a JSON string
+   * @param instance instance to convert
+   * @return a JSON string description
+   * @throws JsonParseException parse problems
+   * @throws JsonMappingException O/J mapping problems
+   */
+  public String toJson(T instance) throws IOException,
+                                               JsonGenerationException,
+                                               JsonMappingException {
+    mapper.configure(SerializationConfig.Feature.INDENT_OUTPUT, true);
+    return mapper.writeValueAsString(instance);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/LockAcquireFailedException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/LockAcquireFailedException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/LockAcquireFailedException.java
new file mode 100644
index 0000000..da58520
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/LockAcquireFailedException.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.persist;
+
+import org.apache.hadoop.fs.Path;
+
+public class LockAcquireFailedException extends Exception {
+  
+  private final Path path;
+
+  public LockAcquireFailedException(Path path) {
+    super("Failed to acquire lock " +path);
+    this.path = path;
+  }
+
+  public LockAcquireFailedException(Path path, Throwable cause) {
+    super("Failed to acquire lock " + path, cause);
+    this.path = path;
+  }
+
+  public Path getPath() {
+    return path;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/LockHeldAction.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/LockHeldAction.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/LockHeldAction.java
new file mode 100644
index 0000000..6659687
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/LockHeldAction.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.persist;
+
+import org.apache.slider.core.exceptions.SliderException;
+
+import java.io.IOException;
+
+/**
+ * Optional action to add while the lock is held; this is needed to execute
+ * some other persistent operations within the scope at the same lock
+ * without inserting too much code into the persister
+ */
+public interface LockHeldAction {
+
+  /**
+   * Execute the action
+   * @throws IOException on any failure
+   */
+  public void execute() throws IOException, SliderException;
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/PersistKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/PersistKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/PersistKeys.java
new file mode 100644
index 0000000..1964459
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/persist/PersistKeys.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.persist;
+
+public class PersistKeys {
+
+  public static final String SCHEMA =
+    "http://example.org/specification/v2.0.0";
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/SliderRegistryUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/SliderRegistryUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/SliderRegistryUtils.java
new file mode 100644
index 0000000..37b36ea
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/SliderRegistryUtils.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.registry;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.registry.client.binding.RegistryUtils;
+import org.apache.slider.common.SliderKeys;
+
+/**
+ * Miscellaneous methods to assist slider registry work
+ * 
+ */
+public class SliderRegistryUtils {
+
+
+  /**
+   * Get the registry path for an instance under the user's home node
+   * @param instanceName application instance
+   * @return a path to the registry location for this application instance.
+   */
+  public static String registryPathForInstance(String instanceName) {
+    return RegistryUtils.servicePath(
+        RegistryUtils.currentUser(), SliderKeys.APP_TYPE, instanceName
+    );
+  }
+
+  /**
+   * Process a path expanding it if needed.
+   * Validation is delegated to later as the core registry will need
+   * to do that anyway
+   * @param path path
+   * @return a path maybe with some expansion
+   */
+  public static String resolvePath(String path) {
+    Preconditions.checkArgument(path!=null, "null path");
+    Preconditions.checkArgument(!path.isEmpty(), "empty path");
+    String newpath = path;
+    if (path.startsWith("~/")) {
+      // add user expansion
+      newpath = RegistryUtils.homePathForCurrentUser() + path.substring(1);
+    } else if (path.equals("~")) {
+      newpath = RegistryUtils.homePathForCurrentUser();
+    }
+    return newpath;
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[64/76] [abbrv] hadoop git commit: YARN-5505. Create an agent-less docker provider in the native-services framework. Contributed by Billie Rinaldi

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb61fe3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
index 8203cf0..fdc5be1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
@@ -82,6 +82,8 @@ import java.util.Set;
 import java.util.zip.ZipEntry;
 import java.util.zip.ZipInputStream;
 
+import static org.apache.slider.common.tools.SliderUtils.getApplicationDefinitionPath;
+
 /** This class implements  the client-side aspects of the agent deployer */
 public class AgentClientProvider extends AbstractClientProvider
     implements AgentKeys, SliderKeys {
@@ -132,13 +134,13 @@ public class AgentClientProvider extends AbstractClientProvider
     sliderFileSystem.verifyFileExists(appDefPath);
 
     String agentConf = instanceDefinition.getAppConfOperations().
-        getGlobalOptions().getOption(AgentKeys.AGENT_CONF, "");
+        getGlobalOptions().getOption(AGENT_CONF, "");
     if (StringUtils.isNotEmpty(agentConf)) {
       sliderFileSystem.verifyFileExists(new Path(agentConf));
     }
 
     String appHome = instanceDefinition.getAppConfOperations().
-        getGlobalOptions().get(AgentKeys.PACKAGE_PATH);
+        getGlobalOptions().get(PACKAGE_PATH);
     if (SliderUtils.isUnset(appHome)) {
       String agentImage = instanceDefinition.getInternalOperations().
           get(InternalKeys.INTERNAL_APPLICATION_IMAGE_PATH);
@@ -173,7 +175,7 @@ public class AgentClientProvider extends AbstractClientProvider
     }
 
     Set<String> names = resources.getComponentNames();
-    names.remove(SliderKeys.COMPONENT_AM);
+    names.remove(COMPONENT_AM);
     Map<Integer, String> priorityMap = new HashMap<Integer, String>();
 
     for (String name : names) {
@@ -271,7 +273,7 @@ public class AgentClientProvider extends AbstractClientProvider
     String agentImage = instanceDefinition.getInternalOperations().
         get(InternalKeys.INTERNAL_APPLICATION_IMAGE_PATH);
     if (SliderUtils.isUnset(agentImage)) {
-      Path agentPath = new Path(tempPath.getParent(), AgentKeys.PROVIDER_AGENT);
+      Path agentPath = new Path(tempPath.getParent(), PROVIDER_AGENT);
       log.info("Automatically uploading the agent tarball at {}", agentPath);
       fileSystem.getFileSystem().mkdirs(agentPath);
       if (ProviderUtils.addAgentTar(this, AGENT_TAR, fileSystem, agentPath)) {
@@ -284,6 +286,12 @@ public class AgentClientProvider extends AbstractClientProvider
 
   @Override
   public Set<String> getApplicationTags(SliderFileSystem fileSystem,
+      ConfTreeOperations appConf) throws SliderException {
+    return getApplicationTags(fileSystem,
+        getApplicationDefinitionPath(appConf));
+  }
+
+  public Set<String> getApplicationTags(SliderFileSystem fileSystem,
                                         String appDef) throws SliderException {
     Set<String> tags;
     Metainfo metaInfo = getMetainfo(fileSystem, appDef);
@@ -437,19 +445,19 @@ public class AgentClientProvider extends AbstractClientProvider
         if (config != null) {
           try {
             clientRoot = config.getJSONObject("global")
-                .getString(AgentKeys.APP_CLIENT_ROOT);
+                .getString(APP_CLIENT_ROOT);
           } catch (JSONException e) {
             log.info("Couldn't read {} from provided client config, falling " +
-                "back on default", AgentKeys.APP_CLIENT_ROOT);
+                "back on default", APP_CLIENT_ROOT);
           }
         }
         if (clientRoot == null && defaultConfig != null) {
           try {
             clientRoot = defaultConfig.getJSONObject("global")
-                .getString(AgentKeys.APP_CLIENT_ROOT);
+                .getString(APP_CLIENT_ROOT);
           } catch (JSONException e) {
             log.info("Couldn't read {} from default client config, using {}",
-                AgentKeys.APP_CLIENT_ROOT, clientInstallPath);
+                APP_CLIENT_ROOT, clientInstallPath);
           }
         }
         if (clientRoot == null) {
@@ -500,7 +508,7 @@ public class AgentClientProvider extends AbstractClientProvider
     try {
       String clientScriptPath = appPkgDir.getAbsolutePath() + File.separator + "package" +
                                 File.separator + clientScript;
-      List<String> command = Arrays.asList(AgentKeys.PYTHON_EXE,
+      List<String> command = Arrays.asList(PYTHON_EXE,
                "-S",
                clientScriptPath,
                "INSTALL",
@@ -510,12 +518,12 @@ public class AgentClientProvider extends AbstractClientProvider
                "DEBUG");
       ProcessBuilder pb = new ProcessBuilder(command);
       log.info("Command: " + StringUtils.join(pb.command(), " "));
-      pb.environment().put(SliderKeys.PYTHONPATH,
+      pb.environment().put(PYTHONPATH,
                            agentPkgDir.getAbsolutePath()
                            + File.separator + "slider-agent" + File.pathSeparator
                            + agentPkgDir.getAbsolutePath()
                            + File.separator + "slider-agent/jinja2");
-      log.info("{}={}", SliderKeys.PYTHONPATH, pb.environment().get(SliderKeys.PYTHONPATH));
+      log.info("{}={}", PYTHONPATH, pb.environment().get(PYTHONPATH));
 
       Process proc = pb.start();
       InputStream stderr = proc.getErrorStream();
@@ -555,8 +563,8 @@ public class AgentClientProvider extends AbstractClientProvider
 
   private void expandAgentTar(File agentPkgDir) throws IOException {
     String libDirProp =
-        System.getProperty(SliderKeys.PROPERTY_LIB_DIR);
-    File tarFile = new File(libDirProp, SliderKeys.AGENT_TAR);
+        System.getProperty(PROPERTY_LIB_DIR);
+    File tarFile = new File(libDirProp, AGENT_TAR);
     expandTar(tarFile, agentPkgDir);
   }
 
@@ -620,7 +628,7 @@ public class AgentClientProvider extends AbstractClientProvider
                                       String name) throws SliderException {
     try {
       JSONObject pkgList = new JSONObject();
-      pkgList.put(AgentKeys.PACKAGE_LIST,
+      pkgList.put(PACKAGE_LIST,
                   AgentProviderService.getPackageListFromApplication(metainfo.getApplication()));
       JSONObject obj = new JSONObject();
       obj.put("hostLevelParams", pkgList);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb61fe3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java
index 9ea984c..c4228e4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java
@@ -23,6 +23,7 @@ package org.apache.slider.providers.agent;
  */
 public interface AgentKeys {
 
+  String AGENT_TAR = "slider-agent.tar.gz";
   String PROVIDER_AGENT = "agent";
   String ROLE_NODE = "echo";
 
@@ -76,23 +77,13 @@ public interface AgentKeys {
   String AGENT_CONF = "agent.conf";
   String ADDON_FOR_ALL_COMPONENTS = "ALL";
 
-  String APP_RESOURCES = "application.resources";
-  String APP_RESOURCES_DIR = "app/resources";
-
-  String APP_CONF_DIR = "app/conf";
-
   String AGENT_INSTALL_DIR = "infra/agent";
   String APP_DEFINITION_DIR = "app/definition";
   String ADDON_DEFINITION_DIR = "addon/definition";
   String AGENT_CONFIG_FILE = "infra/conf/agent.ini";
   String AGENT_VERSION_FILE = "infra/version";
-  String APP_PACKAGES_DIR = "app/packages";
-  String PER_COMPONENT = "per.component";
-  String PER_GROUP = "per.group";
 
-  String JAVA_HOME = "java_home";
   String PACKAGE_LIST = "package_list";
-  String SYSTEM_CONFIGS = "system_configs";
   String WAIT_HEARTBEAT = "wait.heartbeat";
   String PYTHON_EXE = "python";
   String CREATE_DEF_ZK_NODE = "create.default.zookeeper.node";
@@ -104,7 +95,6 @@ public interface AgentKeys {
   String CERT_FILE_LOCALIZATION_PATH = INFRA_RUN_SECURITY_DIR + "ca.crt";
   String KEY_CONTAINER_LAUNCH_DELAY = "container.launch.delay.sec";
   String TEST_RELAX_VERIFICATION = "test.relax.validation";
-  String AM_CONFIG_GENERATION = "am.config.generation";
 
   String DEFAULT_METAINFO_MAP_KEY = "DEFAULT_KEY";
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb61fe3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
index 2ab5c6f..499812e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
@@ -21,20 +21,11 @@ package org.apache.slider.providers.agent;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
-import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
 import org.apache.hadoop.registry.client.types.Endpoint;
 import org.apache.hadoop.registry.client.types.ProtocolTypes;
 import org.apache.hadoop.registry.client.types.ServiceRecord;
-import org.apache.hadoop.registry.client.types.yarn.PersistencePolicies;
-import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.records.Container;
@@ -55,7 +46,6 @@ import org.apache.slider.common.tools.SliderUtils;
 import org.apache.slider.core.conf.AggregateConf;
 import org.apache.slider.core.conf.ConfTreeOperations;
 import org.apache.slider.core.conf.MapOperations;
-import org.apache.slider.core.exceptions.BadCommandArgumentsException;
 import org.apache.slider.core.exceptions.BadConfigException;
 import org.apache.slider.core.exceptions.NoSuchNodeException;
 import org.apache.slider.core.exceptions.SliderException;
@@ -65,7 +55,6 @@ import org.apache.slider.core.registry.docstore.ConfigFormat;
 import org.apache.slider.core.registry.docstore.ConfigUtils;
 import org.apache.slider.core.registry.docstore.ExportEntry;
 import org.apache.slider.core.registry.docstore.PublishedConfiguration;
-import org.apache.slider.core.registry.docstore.PublishedConfigurationOutputter;
 import org.apache.slider.core.registry.docstore.PublishedExports;
 import org.apache.slider.core.registry.info.CustomRegistryConstants;
 import org.apache.slider.providers.AbstractProviderService;
@@ -107,20 +96,15 @@ import org.apache.slider.server.appmaster.web.rest.agent.Register;
 import org.apache.slider.server.appmaster.web.rest.agent.RegistrationResponse;
 import org.apache.slider.server.appmaster.web.rest.agent.RegistrationStatus;
 import org.apache.slider.server.appmaster.web.rest.agent.StatusCommand;
-import org.apache.slider.server.services.security.CertificateManager;
-import org.apache.slider.server.services.security.SecurityStore;
-import org.apache.slider.server.services.security.StoresGenerator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
-import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.Date;
 import java.util.HashMap;
@@ -157,10 +141,6 @@ public class AgentProviderService extends AbstractProviderService implements
   private static final String LABEL_MAKER = "___";
   private static final String CONTAINER_ID = "container_id";
   private static final String GLOBAL_CONFIG_TAG = "global";
-  private static final String LOG_FOLDERS_TAG = "LogFolders";
-  private static final String HOST_FOLDER_FORMAT = "%s:%s";
-  private static final String CONTAINER_LOGS_TAG = "container_log_dirs";
-  private static final String CONTAINER_PWDS_TAG = "container_work_dirs";
   private static final String COMPONENT_TAG = "component";
   private static final String APPLICATION_TAG = "application";
   private static final String COMPONENT_DATA_TAG = "ComponentInstanceData";
@@ -249,13 +229,6 @@ public class AgentProviderService extends AbstractProviderService implements
   }
 
   @Override
-  public Configuration loadProviderConfigurationInformation(File confDir) throws
-      BadCommandArgumentsException,
-      IOException {
-    return new Configuration(false);
-  }
-
-  @Override
   public void validateInstanceDefinition(AggregateConf instanceDefinition)
       throws
       SliderException {
@@ -265,7 +238,7 @@ public class AgentProviderService extends AbstractProviderService implements
         instanceDefinition.getResourceOperations();
 
     Set<String> names = resources.getComponentNames();
-    names.remove(SliderKeys.COMPONENT_AM);
+    names.remove(COMPONENT_AM);
     for (String name : names) {
       Component componentDef = getApplicationComponent(name);
       if (componentDef == null) {
@@ -350,9 +323,9 @@ public class AgentProviderService extends AbstractProviderService implements
 
           // build a map from component to metainfo
           String addonAppDefString = instanceDefinition.getAppConfOperations()
-              .getGlobalOptions().getOption(AgentKeys.ADDONS, null);
+              .getGlobalOptions().getOption(ADDONS, null);
           if (component != null) {
-            addonAppDefString = component.getOption(AgentKeys.ADDONS, addonAppDefString);
+            addonAppDefString = component.getOption(ADDONS, addonAppDefString);
           }
           log.debug("All addon appdefs: {}", addonAppDefString);
           if (addonAppDefString != null) {
@@ -415,6 +388,7 @@ public class AgentProviderService extends AbstractProviderService implements
     if (isYarnDockerContainer(roleGroup)) {
       launcher.setYarnDockerMode(true);
       launcher.setDockerImage(getConfigFromMetaInfo(roleGroup, "image"));
+      launcher.setDockerNetwork(getConfigFromMetaInfo(roleGroup, "network"));
       launcher.setRunPrivilegedContainer(getConfigFromMetaInfo(roleGroup, "runPriviledgedContainer"));
       launcher
           .setYarnContainerMountPoints(getConfigFromMetaInfoWithAppConfigOverriding(
@@ -423,7 +397,9 @@ public class AgentProviderService extends AbstractProviderService implements
 
     // Set the environment
     launcher.putEnv(SliderUtils.buildEnvMap(appComponent,
-        getStandardTokenMap(getAmState().getAppConfSnapshot(), roleName, roleGroup)));
+        providerUtils.getStandardTokenMap(getAmState().getAppConfSnapshot(),
+            getAmState().getInternalsSnapshot(), roleName, roleGroup,
+            getClusterName())));
 
     String workDir = ApplicationConstants.Environment.PWD.$();
     launcher.setEnv("AGENT_WORK_ROOT", workDir);
@@ -444,17 +420,17 @@ public class AgentProviderService extends AbstractProviderService implements
     //local resources
 
     // TODO: Should agent need to support App Home
-    String scriptPath = new File(AgentKeys.AGENT_MAIN_SCRIPT_ROOT, AgentKeys.AGENT_MAIN_SCRIPT).getPath();
+    String scriptPath = new File(AGENT_MAIN_SCRIPT_ROOT, AGENT_MAIN_SCRIPT).getPath();
     String appHome = instanceDefinition.getAppConfOperations().
-        getGlobalOptions().get(AgentKeys.PACKAGE_PATH);
+        getGlobalOptions().get(PACKAGE_PATH);
     if (SliderUtils.isSet(appHome)) {
-      scriptPath = new File(appHome, AgentKeys.AGENT_MAIN_SCRIPT).getPath();
+      scriptPath = new File(appHome, AGENT_MAIN_SCRIPT).getPath();
     }
 
     // set PYTHONPATH
     List<String> pythonPaths = new ArrayList<String>();
-    pythonPaths.add(AgentKeys.AGENT_MAIN_SCRIPT_ROOT);
-    pythonPaths.add(AgentKeys.AGENT_JINJA2_ROOT);
+    pythonPaths.add(AGENT_MAIN_SCRIPT_ROOT);
+    pythonPaths.add(AGENT_JINJA2_ROOT);
     String pythonPath = StringUtils.join(File.pathSeparator, pythonPaths);
     launcher.setEnv(PYTHONPATH, pythonPath);
     log.info("PYTHONPATH set to {}", pythonPath);
@@ -466,21 +442,21 @@ public class AgentProviderService extends AbstractProviderService implements
       agentImagePath =
           new Path(new Path(new Path(instanceDefinition.getInternalOperations().get(InternalKeys.INTERNAL_TMP_DIR),
                                      container.getId().getApplicationAttemptId().getApplicationId().toString()),
-                            AgentKeys.PROVIDER_AGENT),
-                   SliderKeys.AGENT_TAR);
+                            PROVIDER_AGENT),
+                   AGENT_TAR);
     } else {
        agentImagePath = new Path(agentImage);
     }
 
     if (fileSystem.getFileSystem().exists(agentImagePath)) {
       LocalResource agentImageRes = fileSystem.createAmResource(agentImagePath, LocalResourceType.ARCHIVE);
-      launcher.addLocalResource(AgentKeys.AGENT_INSTALL_DIR, agentImageRes);
+      launcher.addLocalResource(AGENT_INSTALL_DIR, agentImageRes);
     } else {
       String msg =
           String.format("Required agent image slider-agent.tar.gz is unavailable at %s", agentImagePath.toString());
       MapOperations compOps = appComponent;
       boolean relaxVerificationForTest = compOps != null ? Boolean.valueOf(compOps.
-          getOptionBool(AgentKeys.TEST_RELAX_VERIFICATION, false)) : false;
+          getOptionBool(TEST_RELAX_VERIFICATION, false)) : false;
       log.error(msg);
 
       if (!relaxVerificationForTest) {
@@ -492,7 +468,7 @@ public class AgentProviderService extends AbstractProviderService implements
     LocalResource appDefRes = fileSystem.createAmResource(
         fileSystem.getFileSystem().resolvePath(new Path(appDef)),
         LocalResourceType.ARCHIVE);
-    launcher.addLocalResource(AgentKeys.APP_DEFINITION_DIR, appDefRes);
+    launcher.addLocalResource(APP_DEFINITION_DIR, appDefRes);
 
     for (Package pkg : getMetaInfo(roleGroup).getApplication().getPackages()) {
       Path pkgPath = fileSystem.buildResourcePath(pkg.getName());
@@ -511,57 +487,61 @@ public class AgentProviderService extends AbstractProviderService implements
       }
       LocalResource packageResource = fileSystem.createAmResource(
           pkgPath, type);
-      launcher.addLocalResource(AgentKeys.APP_PACKAGES_DIR, packageResource);
+      launcher.addLocalResource(APP_PACKAGES_DIR, packageResource);
     }
 
     String agentConf = instanceDefinition.getAppConfOperations().
-        getGlobalOptions().getOption(AgentKeys.AGENT_CONF, "");
+        getGlobalOptions().getOption(AGENT_CONF, "");
     if (SliderUtils.isSet(agentConf)) {
       LocalResource agentConfRes = fileSystem.createAmResource(fileSystem
                                                                    .getFileSystem().resolvePath(new Path(agentConf)),
                                                                LocalResourceType.FILE);
-      launcher.addLocalResource(AgentKeys.AGENT_CONFIG_FILE, agentConfRes);
+      launcher.addLocalResource(AGENT_CONFIG_FILE, agentConfRes);
     }
 
     String agentVer = instanceDefinition.getAppConfOperations().
-        getGlobalOptions().getOption(AgentKeys.AGENT_VERSION, null);
+        getGlobalOptions().getOption(AGENT_VERSION, null);
     if (agentVer != null) {
       LocalResource agentVerRes = fileSystem.createAmResource(
           fileSystem.getFileSystem().resolvePath(new Path(agentVer)),
           LocalResourceType.FILE);
-      launcher.addLocalResource(AgentKeys.AGENT_VERSION_FILE, agentVerRes);
+      launcher.addLocalResource(AGENT_VERSION_FILE, agentVerRes);
     }
 
     if (SliderUtils.isHadoopClusterSecure(getConfig())) {
-      localizeServiceKeytabs(launcher, instanceDefinition, fileSystem);
+      providerUtils.localizeServiceKeytabs(launcher, instanceDefinition,
+          fileSystem, getClusterName());
     }
 
     MapOperations amComponent = instanceDefinition.
-        getAppConfOperations().getComponent(SliderKeys.COMPONENT_AM);
-    boolean twoWayEnabled = amComponent != null ? Boolean.valueOf(amComponent.
-        getOptionBool(AgentKeys.KEY_AGENT_TWO_WAY_SSL_ENABLED, false)) : false;
-    if (twoWayEnabled) {
-      localizeContainerSSLResources(launcher, container, fileSystem);
+        getAppConfOperations().getComponent(COMPONENT_AM);
+    if (providerUtils.hasTwoWaySSLEnabled(amComponent)) {
+      providerUtils.localizeContainerSSLResources(launcher, container,
+          fileSystem, getClusterName());
     }
 
-    MapOperations compOps = appComponent;
-    if (areStoresRequested(compOps)) {
-      localizeContainerSecurityStores(launcher, container, roleName, fileSystem,
-                                      instanceDefinition, compOps);
+    if (providerUtils.areStoresRequested(appComponent)) {
+      providerUtils.localizeContainerSecurityStores(launcher, container,
+          roleName, fileSystem, instanceDefinition, appComponent,
+          getClusterName());
     }
 
     //add the configuration resources
     launcher.addLocalResources(fileSystem.submitDirectory(
         generatedConfPath,
-        SliderKeys.PROPAGATED_CONF_DIR_NAME));
+        PROPAGATED_CONF_DIR_NAME));
 
-    if (appComponent.getOptionBool(AgentKeys.AM_CONFIG_GENERATION, false)) {
+    if (appComponent.getOptionBool(AM_CONFIG_GENERATION, false)) {
       // build and localize configuration files
       Map<String, Map<String, String>> configurations =
           buildCommandConfigurations(instanceDefinition.getAppConfOperations(),
+              instanceDefinition.getInternalOperations(),
               container.getId().toString(), roleName, roleGroup);
-      localizeConfigFiles(launcher, roleName, roleGroup, getMetaInfo(roleGroup),
-          configurations, launcher.getEnv(), fileSystem);
+      for (ConfigFile configFile : getMetaInfo(roleGroup)
+          .getComponentConfigFiles(roleGroup)) {
+        localizeConfigFile(launcher, roleName, roleGroup, configFile,
+            configurations, launcher.getEnv(), fileSystem);
+      }
     }
 
     String label = getContainerLabel(container, roleName, roleGroup);
@@ -569,7 +549,7 @@ public class AgentProviderService extends AbstractProviderService implements
 
     String pythonExec = instanceDefinition.getAppConfOperations()
         .getGlobalOptions().getOption(SliderXmlConfKeys.PYTHON_EXECUTABLE_PATH,
-                                      AgentKeys.PYTHON_EXE);
+                                      PYTHON_EXE);
 
     operation.add(pythonExec);
 
@@ -587,13 +567,13 @@ public class AgentProviderService extends AbstractProviderService implements
     }
 
     operation.add("> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/"
-        + AgentKeys.AGENT_OUT_FILE + " 2>&1");
+        + AGENT_OUT_FILE + " 2>&1");
 
     launcher.addCommand(operation.build());
 
     // localize addon package
     String addonAppDefString = instanceDefinition.getAppConfOperations()
-        .getGlobalOptions().getOption(AgentKeys.ADDONS, null);
+        .getGlobalOptions().getOption(ADDONS, null);
     log.debug("All addon appdefs: {}", addonAppDefString);
     if (addonAppDefString != null) {
       Scanner scanner = new Scanner(addonAppDefString).useDelimiter(",");
@@ -605,7 +585,7 @@ public class AgentProviderService extends AbstractProviderService implements
         LocalResource addonPkgRes = fileSystem.createAmResource(
             fileSystem.getFileSystem().resolvePath(new Path(addonAppDefPath)),
             LocalResourceType.ARCHIVE);
-        launcher.addLocalResource(AgentKeys.ADDON_DEFINITION_DIR + "/" + addonAppDef, addonPkgRes);
+        launcher.addLocalResource(ADDON_DEFINITION_DIR + "/" + addonAppDef, addonPkgRes);
       }
       log.debug("Metainfo map for master and addon: {}",
           packageMetainfo.toString());
@@ -613,7 +593,7 @@ public class AgentProviderService extends AbstractProviderService implements
 
     // Additional files to localize in addition to the application def
     String appResourcesString = instanceDefinition.getAppConfOperations()
-        .getGlobalOptions().getOption(AgentKeys.APP_RESOURCES, null);
+        .getGlobalOptions().getOption(APP_RESOURCES, null);
     log.info("Configuration value for extra resources to localize: {}", appResourcesString);
     if (null != appResourcesString) {
       try (Scanner scanner = new Scanner(appResourcesString).useDelimiter(",")) {
@@ -623,7 +603,7 @@ public class AgentProviderService extends AbstractProviderService implements
           LocalResource extraResource = fileSystem.createAmResource(
               fileSystem.getFileSystem().resolvePath(resourcePath),
               LocalResourceType.FILE);
-          String destination = AgentKeys.APP_RESOURCES_DIR + "/" + resourcePath.getName();
+          String destination = APP_RESOURCES_DIR + "/" + resourcePath.getName();
           log.info("Localizing {} to {}", resourcePath, destination);
           // TODO Can we try harder to avoid collisions?
           launcher.addLocalResource(destination, extraResource);
@@ -641,7 +621,7 @@ public class AgentProviderService extends AbstractProviderService implements
         log.debug("Current component: {} component in metainfo: {}", roleName,
             comp.getName());
         if (comp.getName().equals(roleGroup)
-            || comp.getName().equals(AgentKeys.ADDON_FOR_ALL_COMPONENTS)) {
+            || comp.getName().equals(ADDON_FOR_ALL_COMPONENTS)) {
           pkgStatuses.put(appPkg.getApplicationPackage().getName(), State.INIT);
         }
       }
@@ -658,253 +638,18 @@ public class AgentProviderService extends AbstractProviderService implements
                                    pkgStatuses));
   }
 
-  private void localizeContainerSecurityStores(ContainerLauncher launcher,
-                                               Container container,
-                                               String role,
-                                               SliderFileSystem fileSystem,
-                                               AggregateConf instanceDefinition,
-                                               MapOperations compOps)
-      throws SliderException, IOException {
-    // substitute CLUSTER_NAME into credentials
-    Map<String,List<String>> newcred = new HashMap<>();
-    for (Entry<String,List<String>> entry : instanceDefinition.getAppConf().credentials.entrySet()) {
-      List<String> resultList = new ArrayList<>();
-      for (String v : entry.getValue()) {
-        resultList.add(v.replaceAll(Pattern.quote("${CLUSTER_NAME}"),
-            clusterName).replaceAll(Pattern.quote("${CLUSTER}"),
-            clusterName));
-      }
-      newcred.put(entry.getKey().replaceAll(Pattern.quote("${CLUSTER_NAME}"),
-          clusterName).replaceAll(Pattern.quote("${CLUSTER}"),
-          clusterName),
-          resultList);
-    }
-    instanceDefinition.getAppConf().credentials = newcred;
-
-    // generate and localize security stores
-    SecurityStore[] stores = generateSecurityStores(container, role,
-                                                    instanceDefinition, compOps);
-    for (SecurityStore store : stores) {
-      LocalResource keystoreResource = fileSystem.createAmResource(
-          uploadSecurityResource(store.getFile(), fileSystem), LocalResourceType.FILE);
-      launcher.addLocalResource(String.format("secstores/%s-%s.p12",
-                                              store.getType(), role),
-                                keystoreResource);
-    }
-  }
-
-  private SecurityStore[] generateSecurityStores(Container container,
-                                                 String role,
-                                                 AggregateConf instanceDefinition,
-                                                 MapOperations compOps)
-      throws SliderException, IOException {
-    return StoresGenerator.generateSecurityStores(container.getNodeId().getHost(),
-                                           container.getId().toString(), role,
-                                           instanceDefinition, compOps);
-  }
-
-  private boolean areStoresRequested(MapOperations compOps) {
-    return compOps != null ? compOps.
-        getOptionBool(SliderKeys.COMP_STORES_REQUIRED_KEY, false) : false;
-  }
-
-  private void localizeContainerSSLResources(ContainerLauncher launcher,
-                                             Container container,
-                                             SliderFileSystem fileSystem)
-      throws SliderException {
-    try {
-      // localize server cert
-      Path certsDir = fileSystem.buildClusterSecurityDirPath(getClusterName());
-      LocalResource certResource = fileSystem.createAmResource(
-          new Path(certsDir, SliderKeys.CRT_FILE_NAME),
-            LocalResourceType.FILE);
-      launcher.addLocalResource(AgentKeys.CERT_FILE_LOCALIZATION_PATH,
-                                certResource);
-
-      // generate and localize agent cert
-      CertificateManager certMgr = new CertificateManager();
-      String hostname = container.getNodeId().getHost();
-      String containerId = container.getId().toString();
-      certMgr.generateContainerCertificate(hostname, containerId);
-      LocalResource agentCertResource = fileSystem.createAmResource(
-          uploadSecurityResource(
-            CertificateManager.getAgentCertficateFilePath(containerId),
-            fileSystem), LocalResourceType.FILE);
-      // still using hostname as file name on the agent side, but the files
-      // do end up under the specific container's file space
-      launcher.addLocalResource(AgentKeys.INFRA_RUN_SECURITY_DIR + hostname +
-                                ".crt", agentCertResource);
-      LocalResource agentKeyResource = fileSystem.createAmResource(
-          uploadSecurityResource(
-              CertificateManager.getAgentKeyFilePath(containerId), fileSystem),
-            LocalResourceType.FILE);
-      launcher.addLocalResource(AgentKeys.INFRA_RUN_SECURITY_DIR + hostname +
-                                ".key", agentKeyResource);
-
-    } catch (Exception e) {
-      throw new SliderException(SliderExitCodes.EXIT_DEPLOYMENT_FAILED, e,
-          "Unable to localize certificates.  Two-way SSL cannot be enabled");
-    }
-  }
-
-  private Path uploadSecurityResource(File resource, SliderFileSystem fileSystem)
-      throws IOException {
-    Path certsDir = fileSystem.buildClusterSecurityDirPath(getClusterName());
-    return uploadResource(resource, fileSystem, certsDir);
-  }
-
-  private Path uploadResource(File resource, SliderFileSystem fileSystem,
-      String roleName) throws IOException {
-    Path dir;
-    if (roleName == null) {
-      dir = fileSystem.buildClusterResourcePath(getClusterName());
-    } else {
-      dir = fileSystem.buildClusterResourcePath(getClusterName(), roleName);
-    }
-    return uploadResource(resource, fileSystem, dir);
-  }
-
-  private static synchronized Path uploadResource(File resource,
-      SliderFileSystem fileSystem, Path parentDir) throws IOException {
-    if (!fileSystem.getFileSystem().exists(parentDir)) {
-      fileSystem.getFileSystem().mkdirs(parentDir,
-        new FsPermission(FsAction.ALL, FsAction.NONE, FsAction.NONE));
-    }
-    Path destPath = new Path(parentDir, resource.getName());
-    if (!fileSystem.getFileSystem().exists(destPath)) {
-      FSDataOutputStream os = null;
-      try {
-        os = fileSystem.getFileSystem().create(destPath);
-        byte[] contents = FileUtils.readFileToByteArray(resource);
-        os.write(contents, 0, contents.length);
-        os.flush();
-      } finally {
-        IOUtils.closeStream(os);
-      }
-      log.info("Uploaded {} to localization path {}", resource, destPath);
-    } else {
-      log.info("Resource {} already existed at localization path {}", resource,
-          destPath);
-    }
-
-    while (!fileSystem.getFileSystem().exists(destPath)) {
-      try {
-        Thread.sleep(500);
-      } catch (InterruptedException e) {
-        // ignore
-      }
-    }
-
-    fileSystem.getFileSystem().setPermission(destPath,
-      new FsPermission(FsAction.READ, FsAction.NONE, FsAction.NONE));
-
-    return destPath;
-  }
-
-  private void localizeServiceKeytabs(ContainerLauncher launcher,
-                                      AggregateConf instanceDefinition,
-                                      SliderFileSystem fileSystem)
-      throws IOException {
-    String keytabPathOnHost = instanceDefinition.getAppConfOperations()
-        .getComponent(SliderKeys.COMPONENT_AM).get(
-            SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH);
-    if (SliderUtils.isUnset(keytabPathOnHost)) {
-      String amKeytabName = instanceDefinition.getAppConfOperations()
-          .getComponent(SliderKeys.COMPONENT_AM).get(
-              SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME);
-      String keytabDir = instanceDefinition.getAppConfOperations()
-          .getComponent(SliderKeys.COMPONENT_AM).get(
-              SliderXmlConfKeys.KEY_HDFS_KEYTAB_DIR);
-      // we need to localize the keytab files in the directory
-      Path keytabDirPath = fileSystem.buildKeytabPath(keytabDir, null,
-                                                      getClusterName());
-      boolean serviceKeytabsDeployed = false;
-      if (fileSystem.getFileSystem().exists(keytabDirPath)) {
-        FileStatus[] keytabs = fileSystem.getFileSystem().listStatus(keytabDirPath);
-        LocalResource keytabRes;
-        for (FileStatus keytab : keytabs) {
-          if (!amKeytabName.equals(keytab.getPath().getName())
-              && keytab.getPath().getName().endsWith(".keytab")) {
-            serviceKeytabsDeployed = true;
-            log.info("Localizing keytab {}", keytab.getPath().getName());
-            keytabRes = fileSystem.createAmResource(keytab.getPath(),
-              LocalResourceType.FILE);
-            launcher.addLocalResource(SliderKeys.KEYTAB_DIR + "/" +
-                                    keytab.getPath().getName(),
-                                    keytabRes);
-          }
-        }
-      }
-      if (!serviceKeytabsDeployed) {
-        log.warn("No service keytabs for the application have been localized.  "
-                 + "If the application requires keytabs for secure operation, "
-                 + "please ensure that the required keytabs have been uploaded "
-                 + "to the folder {}", keytabDirPath);
-      }
-    }
-  }
-
-  private void createConfigFile(SliderFileSystem fileSystem, File file,
-      ConfigFile configFile, Map<String, String> config)
-      throws IOException {
-    ConfigFormat configFormat = ConfigFormat.resolve(configFile.getType());
-    log.info("Writing {} file {}", configFormat, file);
-
-    ConfigUtils.prepConfigForTemplateOutputter(configFormat, config,
-        fileSystem, getClusterName(), file.getName());
-    PublishedConfiguration publishedConfiguration =
-        new PublishedConfiguration(configFile.getDictionaryName(),
-            config.entrySet());
-    PublishedConfigurationOutputter configurationOutputter =
-      PublishedConfigurationOutputter.createOutputter(configFormat,
-          publishedConfiguration);
-    configurationOutputter.save(file);
-  }
-
   @VisibleForTesting
-  protected void localizeConfigFiles(ContainerLauncher launcher,
+  protected void localizeConfigFile(ContainerLauncher launcher,
                                      String roleName, String roleGroup,
-                                     Metainfo metainfo,
+                                     ConfigFile configFile,
                                      Map<String, Map<String, String>> configs,
                                      MapOperations env,
                                      SliderFileSystem fileSystem)
       throws IOException {
-    for (ConfigFile configFile : metainfo.getComponentConfigFiles(roleGroup)) {
-      Map<String, String> config = ConfigUtils.replacePropsInConfig(
-          configs.get(configFile.getDictionaryName()), env.options);
-      String fileName = ConfigUtils.replaceProps(config,
-          configFile.getFileName());
-      File localFile = new File(SliderKeys.RESOURCE_DIR);
-      if (!localFile.exists()) {
-        localFile.mkdir();
-      }
-      localFile = new File(localFile, new File(fileName).getName());
-
-      String folder = null;
-      if ("true".equals(config.get(PER_COMPONENT))) {
-        folder = roleName;
-      } else if ("true".equals(config.get(PER_GROUP))) {
-        folder = roleGroup;
-      }
-
-      log.info("Localizing {} configs to config file {} (destination {}) " +
-          "based on {} configs", config.size(), localFile, fileName,
-          configFile.getDictionaryName());
-      createConfigFile(fileSystem, localFile, configFile, config);
-      Path destPath = uploadResource(localFile, fileSystem, folder);
-      LocalResource configResource = fileSystem.createAmResource(destPath,
-          LocalResourceType.FILE);
-
-      File destFile = new File(fileName);
-      if (destFile.isAbsolute()) {
-        launcher.addLocalResource(
-            SliderKeys.RESOURCE_DIR + "/" + destFile.getName(),
-            configResource, fileName);
-      } else {
-        launcher.addLocalResource(AgentKeys.APP_CONF_DIR + "/" + fileName,
-            configResource);
-      }
-    }
+    ConfigFormat configFormat = ConfigFormat.resolve(configFile.getType());
+    providerUtils.localizeConfigFile(launcher, roleName, roleGroup,
+        configFile.getDictionaryName(), configFormat, configFile.getFileName(),
+        configs, env, fileSystem, getClusterName());
   }
 
   /**
@@ -1053,7 +798,6 @@ public class AgentProviderService extends AbstractProviderService implements
       doUpgrade = true;
     }
 
-    StateAccessForProviders accessor = getAmState();
     CommandScript cmdScript = getScriptPathForMasterPackage(roleGroup);
     List<ComponentCommand> commands = getApplicationComponent(roleGroup).getCommands();
 
@@ -1089,45 +833,10 @@ public class AgentProviderService extends AbstractProviderService implements
     List<ComponentStatus> statuses = heartBeat.getComponentStatus();
     if (statuses != null && !statuses.isEmpty()) {
       log.info("status from agent: " + statuses.toString());
-      try {
-        for(ComponentStatus status : statuses){
-          RoleInstance role = null;
-          if(status.getIp() != null && !status.getIp().isEmpty()){
-            role = amState.getOwnedContainer(containerId);
-            role.ip = status.getIp();
-          }
-          if(status.getHostname() != null && !status.getHostname().isEmpty()){
-            role = amState.getOwnedContainer(containerId);
-            role.hostname = status.getHostname();
-          }
-          if (role != null) {
-            // create an updated service record (including hostname and ip) and publish...
-            ServiceRecord record = new ServiceRecord();
-            record.set(YarnRegistryAttributes.YARN_ID, containerId);
-            record.description = roleName;
-            record.set(YarnRegistryAttributes.YARN_PERSISTENCE,
-                       PersistencePolicies.CONTAINER);
-            // TODO:  switch record attributes to use constants from YarnRegistryAttributes
-            // when it's been updated.
-            if (role.ip != null) {
-              record.set("yarn:ip", role.ip);
-            }
-            if (role.hostname != null) {
-              record.set("yarn:hostname", role.hostname);
-            }
-            yarnRegistry.putComponent(
-                RegistryPathUtils.encodeYarnID(containerId), record);
-
-          }
-        }
-
-
-      } catch (NoSuchNodeException e) {
-        // ignore - there is nothing to do if we don't find a container
-        log.warn("Owned container {} not found - {}", containerId, e);
-      } catch (IOException e) {
-        log.warn("Error updating container {} service record in registry",
-                 containerId, e);
+      for(ComponentStatus status : statuses){
+        providerUtils.updateServiceRecord(getAmState(), yarnRegistry,
+            containerId, roleName, Collections.singletonList(status.getIp()),
+            status.getHostname());
       }
     }
 
@@ -1179,8 +888,8 @@ public class AgentProviderService extends AbstractProviderService implements
       }
     }
 
-    int waitForCount = accessor.getInstanceDefinitionSnapshot().
-        getAppConfOperations().getComponentOptInt(roleGroup, AgentKeys.WAIT_HEARTBEAT, 0);
+    int waitForCount = getAmState().getInstanceDefinitionSnapshot().
+        getAppConfOperations().getComponentOptInt(roleGroup, WAIT_HEARTBEAT, 0);
 
     if (id < waitForCount) {
       log.info("Waiting until heartbeat count {}. Current val: {}", waitForCount, id);
@@ -1223,7 +932,7 @@ public class AgentProviderService extends AbstractProviderService implements
             log.debug("Addon component: {} pkg: {} script: {}", comp.getName(),
                 nextPkgToInstall, comp.getCommandScript().getScript());
             if (comp.getName().equals(roleGroup)
-                || comp.getName().equals(AgentKeys.ADDON_FOR_ALL_COMPONENTS)) {
+                || comp.getName().equals(ADDON_FOR_ALL_COMPONENTS)) {
               scriptPath = comp.getCommandScript().getScript();
               if (scriptPath != null) {
                 addInstallCommand(roleName, roleGroup, containerId, response,
@@ -1406,15 +1115,12 @@ public class AgentProviderService extends AbstractProviderService implements
     return details;
   }
 
-  @Override
   public void applyInitialRegistryDefinitions(URL amWebURI,
       URL agentOpsURI,
       URL agentStatusURI,
       ServiceRecord serviceRecord)
     throws IOException {
     super.applyInitialRegistryDefinitions(amWebURI,
-                                          agentOpsURI,
-                                          agentStatusURI,
                                           serviceRecord);
 
     try {
@@ -1450,20 +1156,17 @@ public class AgentProviderService extends AbstractProviderService implements
     ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
     MapOperations clientOperations = appConf.getOrAddComponent(client.getName());
     appConf.resolve();
-    if (!clientOperations.getOptionBool(AgentKeys.AM_CONFIG_GENERATION,
+    if (!clientOperations.getOptionBool(AM_CONFIG_GENERATION,
         false)) {
       log.info("AM config generation is false, not publishing client configs");
       return;
     }
 
     // build and localize configuration files
-    Map<String, Map<String, String>> configurations = new TreeMap<String, Map<String, String>>();
-    Map<String, String> tokens = null;
-    try {
-      tokens = getStandardTokenMap(appConf, client.getName(), client.getName());
-    } catch (SliderException e) {
-      throw new IOException(e);
-    }
+    Map<String, Map<String, String>> configurations = new TreeMap<>();
+    Map<String, String> tokens = providerUtils.getStandardTokenMap(appConf,
+        getAmState().getInternalsSnapshot(), client.getName(),
+        client.getName(), getClusterName());
 
     for (ConfigFile configFile : getMetaInfo().getComponentConfigFiles(client.getName())) {
       addNamedConfiguration(configFile.getDictionaryName(),
@@ -1561,7 +1264,7 @@ public class AgentProviderService extends AbstractProviderService implements
    */
   private void readAndSetHeartbeatMonitoringInterval(AggregateConf instanceDefinition) {
     String hbMonitorInterval = instanceDefinition.getAppConfOperations().
-        getGlobalOptions().getOption(AgentKeys.HEARTBEAT_MONITOR_INTERVAL,
+        getGlobalOptions().getOption(HEARTBEAT_MONITOR_INTERVAL,
                                      Integer.toString(DEFAULT_HEARTBEAT_MONITOR_INTERVAL));
     try {
       setHeartbeatMonitorInterval(Integer.parseInt(hbMonitorInterval));
@@ -1581,7 +1284,7 @@ public class AgentProviderService extends AbstractProviderService implements
    */
   private void initializeAgentDebugCommands(AggregateConf instanceDefinition) {
     String launchParameterStr = instanceDefinition.getAppConfOperations().
-        getGlobalOptions().getOption(AgentKeys.AGENT_INSTANCE_DEBUG_DATA, "");
+        getGlobalOptions().getOption(AGENT_INSTANCE_DEBUG_DATA, "");
     agentLaunchParameter = new AgentLaunchParameter(launchParameterStr);
   }
 
@@ -1699,20 +1402,11 @@ public class AgentProviderService extends AbstractProviderService implements
     return clusterName;
   }
 
-  /**
-   * Publish a named property bag that may contain name-value pairs for app configurations such as hbase-site
-   *
-   * @param name
-   * @param description
-   * @param entries
-   */
+  @VisibleForTesting
   protected void publishApplicationInstanceData(String name, String description,
                                                 Iterable<Map.Entry<String, String>> entries) {
-    PublishedConfiguration pubconf = new PublishedConfiguration();
-    pubconf.description = description;
-    pubconf.putValues(entries);
-    log.info("publishing {}", pubconf);
-    getAmState().getPublishedSliderConfigurations().put(name, pubconf);
+    providerUtils.publishApplicationInstanceData(name, description, entries,
+        getAmState());
   }
 
   /**
@@ -1771,72 +1465,14 @@ public class AgentProviderService extends AbstractProviderService implements
     return stats;
   }
 
-
-  /**
-   * Format the folder locations and publish in the registry service
-   *
-   * @param folders
-   * @param containerId
-   * @param hostFqdn
-   * @param componentName
-   */
+  @VisibleForTesting
   protected void publishFolderPaths(
       Map<String, String> folders, String containerId, String componentName, String hostFqdn) {
-    Date now = new Date();
-    for (Map.Entry<String, String> entry : folders.entrySet()) {
-      ExportEntry exportEntry = new ExportEntry();
-      exportEntry.setValue(String.format(HOST_FOLDER_FORMAT, hostFqdn, entry.getValue()));
-      exportEntry.setContainerId(containerId);
-      exportEntry.setLevel(COMPONENT_TAG);
-      exportEntry.setTag(componentName);
-      exportEntry.setUpdatedTime(now.toString());
-      if (entry.getKey().equals("AGENT_LOG_ROOT")) {
-        synchronized (logFolderExports) {
-          getLogFolderExports().put(containerId, exportEntry);
-        }
-      } else {
-        synchronized (workFolderExports) {
-          getWorkFolderExports().put(containerId, exportEntry);
-        }
-      }
-      log.info("Updating log and pwd folders for container {}", containerId);
-    }
-
-    PublishedExports exports = new PublishedExports(CONTAINER_LOGS_TAG);
-    exports.setUpdated(now.getTime());
-    synchronized (logFolderExports) {
-      updateExportsFromList(exports, getLogFolderExports());
-    }
-    getAmState().getPublishedExportsSet().put(CONTAINER_LOGS_TAG, exports);
-
-    exports = new PublishedExports(CONTAINER_PWDS_TAG);
-    exports.setUpdated(now.getTime());
-    synchronized (workFolderExports) {
-      updateExportsFromList(exports, getWorkFolderExports());
-    }
-    getAmState().getPublishedExportsSet().put(CONTAINER_PWDS_TAG, exports);
+    providerUtils.publishFolderPaths(folders, containerId, componentName, hostFqdn,
+        getAmState(), getLogFolderExports(), getWorkFolderExports());
   }
 
   /**
-   * Update the export data from the map
-   * @param exports
-   * @param folderExports
-   */
-  private void updateExportsFromList(PublishedExports exports, Map<String, ExportEntry> folderExports) {
-    Map<String, List<ExportEntry>> perComponentList = new HashMap<String, List<ExportEntry>>();
-    for(Map.Entry<String, ExportEntry> logEntry : folderExports.entrySet())
-    {
-      String componentName = logEntry.getValue().getTag();
-      if (!perComponentList.containsKey(componentName)) {
-        perComponentList.put(componentName, new ArrayList<ExportEntry>());
-      }
-      perComponentList.get(componentName).add(logEntry.getValue());
-    }
-    exports.putValues(perComponentList.entrySet());
-  }
-
-
-  /**
    * Process return status for component instances
    *
    * @param heartBeat
@@ -1855,7 +1491,7 @@ public class AgentProviderService extends AbstractProviderService implements
 
           if ((!canAnyMasterPublishConfig(componentGroup) || canPublishConfig(componentGroup)) &&
               !getAmState().getAppConfSnapshot().getComponentOptBool(
-                  componentGroup, AgentKeys.AM_CONFIG_GENERATION, false)) {
+                  componentGroup, AM_CONFIG_GENERATION, false)) {
             // If no Master can explicitly publish then publish if its a master
             // Otherwise, wait till the master that can publish is ready
 
@@ -1899,7 +1535,8 @@ public class AgentProviderService extends AbstractProviderService implements
             // publish export groups if any
             Map<String, String> replaceTokens = new HashMap<String, String>();
             for (Map.Entry<String, Map<String, ClusterNode>> entry : getRoleClusterNodeMapping().entrySet()) {
-              String hostName = getHostsList(entry.getValue().values(), true).iterator().next();
+              String hostName = providerUtils.getHostsList(
+                  entry.getValue().values(), true).iterator().next();
               replaceTokens.put(String.format(hostKeyFormat, entry.getKey().toUpperCase(Locale.ENGLISH)), hostName);
             }
 
@@ -1967,28 +1604,24 @@ public class AgentProviderService extends AbstractProviderService implements
   }
 
   private void publishModifiedExportGroups(Set<String> modifiedGroups) {
-    for (String groupName : modifiedGroups) {
-      Map<String, List<ExportEntry>> entries = this.exportGroups.get(groupName);
-
+    for (String roleGroup : modifiedGroups) {
+      Map<String, List<ExportEntry>> entries = this.exportGroups.get(roleGroup);
       // Publish in old format for the time being
       Map<String, String> simpleEntries = new HashMap<String, String>();
-      for (Map.Entry<String, List<ExportEntry>> entry : entries.entrySet()) {
+      for (Entry<String, List<ExportEntry>> entry : entries.entrySet()) {
         List<ExportEntry> exports = entry.getValue();
         if (SliderUtils.isNotEmpty(exports)) {
           // there is no support for multiple exports per name - so extract only the first one
           simpleEntries.put(entry.getKey(), entry.getValue().get(0).getValue());
         }
       }
-      if (!getAmState().getAppConfSnapshot().getComponentOptBool(
-          groupName, AgentKeys.AM_CONFIG_GENERATION, false)) {
-        publishApplicationInstanceData(groupName, groupName,
-            simpleEntries.entrySet());
-      }
+      publishApplicationInstanceData(roleGroup, roleGroup,
+          simpleEntries.entrySet());
 
-      PublishedExports exports = new PublishedExports(groupName);
+      PublishedExports exports = new PublishedExports(roleGroup);
       exports.setUpdated(new Date().getTime());
       exports.putValues(entries.entrySet());
-      getAmState().getPublishedExportsSet().put(groupName, exports);
+      getAmState().getPublishedExportsSet().put(roleGroup, exports);
     }
   }
 
@@ -2310,7 +1943,8 @@ public class AgentProviderService extends AbstractProviderService implements
     cmd.setHostLevelParams(hostLevelParams);
 
     Map<String, Map<String, String>> configurations =
-        buildCommandConfigurations(appConf, containerId, roleName, roleGroup);
+        buildCommandConfigurations(appConf, getAmState().getInternalsSnapshot(),
+            containerId, roleName, roleGroup);
     cmd.setConfigurations(configurations);
     Map<String, Map<String, String>> componentConfigurations = buildComponentConfigurations(appConf);
     cmd.setComponentConfigurations(componentConfigurations);
@@ -2359,7 +1993,8 @@ public class AgentProviderService extends AbstractProviderService implements
     cmd.setHostLevelParams(hostLevelParams);
 
     Map<String, Map<String, String>> configurations = buildCommandConfigurations(
-        appConf, containerId, roleName, roleGroup);
+        appConf, getAmState().getInternalsSnapshot(), containerId, roleName,
+        roleGroup);
     cmd.setConfigurations(configurations);
     Map<String, Map<String, String>> componentConfigurations = buildComponentConfigurations(appConf);
     cmd.setComponentConfigurations(componentConfigurations);
@@ -2522,7 +2157,9 @@ public class AgentProviderService extends AbstractProviderService implements
 
     cmd.setCommandParams(commandParametersSet(scriptPath, timeout, false));
 
-    Map<String, Map<String, String>> configurations = buildCommandConfigurations(appConf, containerId, roleName, roleGroup);
+    Map<String, Map<String, String>> configurations =
+        buildCommandConfigurations(appConf, getAmState().getInternalsSnapshot(),
+            containerId, roleName, roleGroup);
 
     cmd.setConfigurations(configurations);
 
@@ -2557,7 +2194,8 @@ public class AgentProviderService extends AbstractProviderService implements
     cmd.setCommandParams(setCommandParameters(scriptPath, timeout, false));
 
     Map<String, Map<String, String>> configurations = buildCommandConfigurations(
-        appConf, containerId, roleName, roleGroup);
+        appConf, getAmState().getInternalsSnapshot(), containerId, roleName,
+        roleGroup);
     Map<String, String> dockerConfig = new HashMap<String, String>();
     String statusCommand = getConfigFromMetaInfoWithAppConfigOverriding(roleGroup, "statusCommand");
     if (statusCommand == null) {
@@ -2598,7 +2236,8 @@ public class AgentProviderService extends AbstractProviderService implements
 
     ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
     Map<String, Map<String, String>> configurations = buildCommandConfigurations(
-        appConf, containerId, roleName, roleGroup);
+        appConf, getAmState().getInternalsSnapshot(), containerId, roleName,
+        roleGroup);
     Map<String, String> dockerConfig = new HashMap<String, String>();
     String statusCommand = getConfigFromMetaInfoWithAppConfigOverriding(roleGroup, "statusCommand");
     if (statusCommand == null) {
@@ -2690,7 +2329,8 @@ public class AgentProviderService extends AbstractProviderService implements
     cmd.setCommandParams(setCommandParameters(startCommand, timeout, true));
     
     Map<String, Map<String, String>> configurations = buildCommandConfigurations(
-        appConf, containerId, roleName, roleGroup);
+        appConf, getAmState().getInternalsSnapshot(), containerId, roleName,
+        roleGroup);
     Map<String, Map<String, String>> componentConfigurations = buildComponentConfigurations(appConf);
     cmd.setComponentConfigurations(componentConfigurations);
     
@@ -2794,13 +2434,6 @@ public class AgentProviderService extends AbstractProviderService implements
           result = container.getNetwork();
         }
         break;
-      case "useNetworkScript":
-        if (container.getUseNetworkScript() == null || container.getUseNetworkScript().isEmpty()) {
-          result = "yes";
-        } else {
-          result = container.getUseNetworkScript();
-        }
-        break;
       case "statusCommand":
         result = container.getStatusCommand();
         break;
@@ -2903,7 +2536,9 @@ public class AgentProviderService extends AbstractProviderService implements
     cmd.setRoleParams(roleParams);
     cmd.getRoleParams().put("auto_restart", Boolean.toString(isMarkedAutoRestart));
 
-    Map<String, Map<String, String>> configurations = buildCommandConfigurations(appConf, containerId, roleName, roleGroup);
+    Map<String, Map<String, String>> configurations =
+        buildCommandConfigurations(appConf, internalsConf, containerId,
+            roleName, roleGroup);
     cmd.setConfigurations(configurations);
     Map<String, Map<String, String>> componentConfigurations = buildComponentConfigurations(appConf);
     cmd.setComponentConfigurations(componentConfigurations);
@@ -2957,7 +2592,7 @@ public class AgentProviderService extends AbstractProviderService implements
 
 
     Map<String, Map<String, String>> configurationsStop = buildCommandConfigurations(
-        appConf, containerId, roleName, roleGroup);
+        appConf, internalsConf, containerId, roleName, roleGroup);
     cmdStop.setConfigurations(configurationsStop);
     response.addExecutionCommand(cmdStop);
   }
@@ -2989,7 +2624,7 @@ public class AgentProviderService extends AbstractProviderService implements
     cmd.setCommandParams(commandParametersSet(scriptPath, timeout, true));
 
     Map<String, Map<String, String>> configurations = buildCommandConfigurations(
-        appConf, containerId, roleName, roleGroup);
+        appConf, internalsConf, containerId, roleName, roleGroup);
     cmd.setConfigurations(configurations);
     response.addExecutionCommand(cmd);
   }
@@ -3023,7 +2658,7 @@ public class AgentProviderService extends AbstractProviderService implements
     cmdStop.setCommandParams(commandParametersSet(scriptPath, timeout, true));
 
     Map<String, Map<String, String>> configurationsStop = buildCommandConfigurations(
-        appConf, containerId, roleName, roleGroup);
+        appConf, internalsConf, containerId, roleName, roleGroup);
     cmdStop.setConfigurations(configurationsStop);
     response.addExecutionCommand(cmdStop);
   }
@@ -3062,12 +2697,13 @@ public class AgentProviderService extends AbstractProviderService implements
   }
 
   private Map<String, Map<String, String>> buildCommandConfigurations(
-      ConfTreeOperations appConf, String containerId, String roleName, String roleGroup)
+      ConfTreeOperations appConf, ConfTreeOperations internalsConf,
+      String containerId, String roleName, String roleGroup)
       throws SliderException {
 
-    Map<String, Map<String, String>> configurations =
-        new TreeMap<String, Map<String, String>>();
-    Map<String, String> tokens = getStandardTokenMap(appConf, roleName, roleGroup);
+    Map<String, Map<String, String>> configurations = new TreeMap<>();
+    Map<String, String> tokens = providerUtils.getStandardTokenMap(appConf,
+        internalsConf, roleName, roleGroup, getClusterName());
     tokens.put("${CONTAINER_ID}", containerId);
 
     Set<String> configs = new HashSet<String>();
@@ -3090,111 +2726,16 @@ public class AgentProviderService extends AbstractProviderService implements
     return configurations;
   }
 
+  @VisibleForTesting
   protected void dereferenceAllConfigs(Map<String, Map<String, String>> configurations) {
-    Map<String, String> allConfigs = new HashMap<String, String>();
-    String lookupFormat = "${@//site/%s/%s}";
-    for (String configType : configurations.keySet()) {
-      Map<String, String> configBucket = configurations.get(configType);
-      for (String configName : configBucket.keySet()) {
-        allConfigs.put(String.format(lookupFormat, configType, configName), configBucket.get(configName));
-      }
-    }
-
-    boolean finished = false;
-    while (!finished) {
-      finished = true;
-      for (Map.Entry<String, String> entry : allConfigs.entrySet()) {
-        String configValue = entry.getValue();
-        for (Map.Entry<String, String> lookUpEntry : allConfigs.entrySet()) {
-          String lookUpValue = lookUpEntry.getValue();
-          if (lookUpValue.contains("${@//site/")) {
-            continue;
-          }
-          String lookUpKey = lookUpEntry.getKey();
-          if (configValue != null && configValue.contains(lookUpKey)) {
-            configValue = configValue.replace(lookUpKey, lookUpValue);
-          }
-        }
-        if (!configValue.equals(entry.getValue())) {
-          finished = false;
-          allConfigs.put(entry.getKey(), configValue);
-        }
-      }
-    }
-
-    for (String configType : configurations.keySet()) {
-      Map<String, String> configBucket = configurations.get(configType);
-      for (Map.Entry<String, String> entry: configBucket.entrySet()) {
-        String configName = entry.getKey();
-        String configValue = entry.getValue();
-        for (Map.Entry<String, String> lookUpEntry : allConfigs.entrySet()) {
-          String lookUpValue = lookUpEntry.getValue();
-          if (lookUpValue.contains("${@//site/")) {
-            continue;
-          }
-          String lookUpKey = lookUpEntry.getKey();
-          if (configValue != null && configValue.contains(lookUpKey)) {
-            configValue = configValue.replace(lookUpKey, lookUpValue);
-          }
-        }
-        configBucket.put(configName, configValue);
-      }
-    }
-  }
-
-  private Map<String, String> getStandardTokenMap(ConfTreeOperations appConf,
-      String componentName, String componentGroup) throws SliderException {
-    Map<String, String> tokens = new HashMap<String, String>();
-    String nnuri = appConf.get("site.fs.defaultFS");
-    tokens.put("${NN_URI}", nnuri);
-    tokens.put("${NN_HOST}", URI.create(nnuri).getHost());
-    tokens.put("${ZK_HOST}", appConf.get(OptionKeys.ZOOKEEPER_HOSTS));
-    tokens.put("${DEFAULT_ZK_PATH}", appConf.get(OptionKeys.ZOOKEEPER_PATH));
-    String prefix = appConf.getComponentOpt(componentGroup, ROLE_PREFIX,
-        null);
-    String dataDirSuffix = "";
-    if (prefix == null) {
-      prefix = "";
-    } else {
-      dataDirSuffix = "_" + SliderUtils.trimPrefix(prefix);
-    }
-    tokens.put("${DEFAULT_DATA_DIR}", getAmState()
-        .getInternalsSnapshot()
-        .getGlobalOptions()
-        .getMandatoryOption(InternalKeys.INTERNAL_DATA_DIR_PATH) + dataDirSuffix);
-    tokens.put("${JAVA_HOME}", appConf.get(AgentKeys.JAVA_HOME));
-    tokens.put("${COMPONENT_NAME}", componentName);
-    tokens.put("${COMPONENT_NAME.lc}", componentName.toLowerCase());
-    tokens.put("${COMPONENT_PREFIX}", prefix);
-    tokens.put("${COMPONENT_PREFIX.lc}", prefix.toLowerCase());
-    if (!componentName.equals(componentGroup) && componentName.startsWith(componentGroup)) {
-      tokens.put("${COMPONENT_ID}", componentName.substring(componentGroup.length()));
-    }
-    tokens.put("${CLUSTER_NAME}", getClusterName());
-    tokens.put("${CLUSTER_NAME.lc}", getClusterName().toLowerCase());
-    tokens.put("${APP_NAME}", getClusterName());
-    tokens.put("${APP_NAME.lc}", getClusterName().toLowerCase());
-    tokens.put("${APP_COMPONENT_NAME}", componentName);
-    tokens.put("${APP_COMPONENT_NAME.lc}", componentName.toLowerCase());
-    return tokens;
+    providerUtils.dereferenceAllConfigs(configurations);
   }
 
   @VisibleForTesting
-  protected List<String> getSystemConfigurationsRequested(ConfTreeOperations appConf) {
-    List<String> configList = new ArrayList<String>();
-
-    String configTypes = appConf.get(AgentKeys.SYSTEM_CONFIGS);
-    if (configTypes != null && configTypes.length() > 0) {
-      String[] configs = configTypes.split(",");
-      for (String config : configs) {
-        configList.add(config.trim());
-      }
-    }
-
-    return new ArrayList<String>(new HashSet<String>(configList));
+  protected Set<String> getSystemConfigurationsRequested(ConfTreeOperations appConf) {
+    return providerUtils.getSystemConfigurationsRequested(appConf);
   }
 
-
   @VisibleForTesting
   protected List<String> getApplicationConfigurationTypes(String roleGroup) {
     List<String> configList = new ArrayList<String>();
@@ -3275,30 +2816,17 @@ public class AgentProviderService extends AbstractProviderService implements
     configurations.put(configName, config);
   }
 
+  @VisibleForTesting
   protected void addRoleRelatedTokens(Map<String, String> tokens) {
-    for (Map.Entry<String, Map<String, ClusterNode>> entry : getRoleClusterNodeMapping().entrySet()) {
-      String tokenName = entry.getKey().toUpperCase(Locale.ENGLISH) + "_HOST";
-      String hosts = StringUtils.join(",", getHostsList(entry.getValue().values(), true));
-      tokens.put("${" + tokenName + "}", hosts);
-    }
-  }
-
-  private Iterable<String> getHostsList(Collection<ClusterNode> values,
-                                        boolean hostOnly) {
-    List<String> hosts = new ArrayList<String>();
-    for (ClusterNode cn : values) {
-      hosts.add(hostOnly ? cn.host : cn.host + "/" + cn.name);
-    }
-
-    return hosts;
+    providerUtils.addRoleRelatedTokens(tokens, getAmState());
   }
 
   private void addDefaultGlobalConfig(Map<String, String> config, String containerId, String roleName) {
     config.put("app_log_dir", "${AGENT_LOG_ROOT}");
     config.put("app_pid_dir", "${AGENT_WORK_ROOT}/app/run");
     config.put("app_install_dir", "${AGENT_WORK_ROOT}/app/install");
-    config.put("app_conf_dir", "${AGENT_WORK_ROOT}/" + AgentKeys.APP_CONF_DIR);
-    config.put("app_input_conf_dir", "${AGENT_WORK_ROOT}/" + SliderKeys.PROPAGATED_CONF_DIR_NAME);
+    config.put("app_conf_dir", "${AGENT_WORK_ROOT}/" + APP_CONF_DIR);
+    config.put("app_input_conf_dir", "${AGENT_WORK_ROOT}/" + PROPAGATED_CONF_DIR_NAME);
     config.put("app_container_id", containerId);
     config.put("app_container_tag", tags.getTag(roleName, containerId));
 
@@ -3315,7 +2843,8 @@ public class AgentProviderService extends AbstractProviderService implements
     for (Map.Entry<String, Map<String, ClusterNode>> entry :
         getRoleClusterNodeMapping().entrySet()) {
       details.put(entry.getKey() + " Host(s)/Container(s)",
-                  new MonitorDetail(getHostsList(entry.getValue().values(), false).toString(), false));
+                  new MonitorDetail(providerUtils.getHostsList(
+                      entry.getValue().values(), false).toString(), false));
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb61fe3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerClientProvider.java
new file mode 100644
index 0000000..13473e5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerClientProvider.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.docker;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.providers.AbstractClientProvider;
+import org.apache.slider.providers.ProviderRole;
+import org.apache.slider.providers.ProviderUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.slider.providers.docker.DockerKeys.DOCKER_IMAGE;
+
+public class DockerClientProvider extends AbstractClientProvider
+    implements SliderKeys {
+
+  protected static final Logger log =
+      LoggerFactory.getLogger(DockerClientProvider.class);
+  private static final ProviderUtils providerUtils = new ProviderUtils(log);
+  protected static final String NAME = "docker";
+
+  public DockerClientProvider(Configuration conf) {
+    super(conf);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+
+  @Override
+  public List<ProviderRole> getRoles() {
+    return Collections.emptyList();
+  }
+
+  @Override
+  public void validateInstanceDefinition(AggregateConf instanceDefinition,
+      SliderFileSystem fs) throws SliderException {
+    super.validateInstanceDefinition(instanceDefinition, fs);
+
+    ConfTreeOperations appConf = instanceDefinition.getAppConfOperations();
+    ConfTreeOperations resources = instanceDefinition.getResourceOperations();
+
+    for (String roleGroup : resources.getComponentNames()) {
+      if (roleGroup.equals(COMPONENT_AM)) {
+        continue;
+      }
+      if (appConf.getComponentOpt(roleGroup, DOCKER_IMAGE, null) == null &&
+          appConf.getGlobalOptions().get(DOCKER_IMAGE) == null) {
+        throw new BadConfigException("Property " + DOCKER_IMAGE + " not " +
+            "specified for " + roleGroup);
+      }
+
+      providerUtils.getPackages(roleGroup, appConf);
+
+      if (appConf.getComponentOptBool(roleGroup, AM_CONFIG_GENERATION, false)) {
+        // build and localize configuration files
+        Map<String, Map<String, String>> configurations =
+            providerUtils.buildConfigurations(appConf, appConf, null, roleGroup,
+                roleGroup, null);
+        try {
+          providerUtils.localizeConfigFiles(null, roleGroup, roleGroup, appConf,
+              configurations, null, fs, null);
+        } catch (IOException e) {
+          throw new BadConfigException(e.toString());
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb61fe3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerKeys.java
new file mode 100644
index 0000000..40b73a2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerKeys.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.docker;
+
+public interface DockerKeys {
+  String PROVIDER_DOCKER = "docker";
+  String DOCKER_PREFIX = "docker.";
+  String DOCKER_IMAGE = DOCKER_PREFIX + "image";
+  String DOCKER_NETWORK = DOCKER_PREFIX + "network";
+  String DOCKER_USE_PRIVILEGED = DOCKER_PREFIX + "usePrivileged";
+  String DOCKER_START_COMMAND = DOCKER_PREFIX + "startCommand";
+
+  String DEFAULT_DOCKER_NETWORK = "bridge";
+
+  String OUT_FILE = "stdout.txt";
+  String ERR_FILE = "stderr.txt";
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb61fe3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerProviderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerProviderFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerProviderFactory.java
new file mode 100644
index 0000000..5d2592f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerProviderFactory.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.docker;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.slider.providers.AbstractClientProvider;
+import org.apache.slider.providers.ProviderService;
+import org.apache.slider.providers.SliderProviderFactory;
+
+public class DockerProviderFactory extends SliderProviderFactory {
+
+  public DockerProviderFactory() {
+  }
+
+  public DockerProviderFactory(Configuration conf) {
+    super(conf);
+  }
+
+  @Override
+  public AbstractClientProvider createClientProvider() {
+    return new DockerClientProvider(getConf());
+  }
+
+  @Override
+  public ProviderService createServerProvider() {
+    return new DockerProviderService();
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[75/76] [abbrv] hadoop git commit: Addendum patch for YARN-5610. Contributed by Gour Saha

Posted by ji...@apache.org.
Addendum patch for YARN-5610. Contributed by Gour Saha


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

Branch: refs/heads/yarn-native-services
Commit: fba18cbe97bfe524ee007a435db32bb3057716c7
Parents: 69c9724
Author: Jian He <ji...@apache.org>
Authored: Wed Oct 12 13:33:09 2016 -0700
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 .../yarn/services/resource/Application.java     | 44 ++++++++++----------
 .../services/resource/ApplicationState.java     |  5 +++
 .../services/resource/ApplicationStatus.java    |  8 ++--
 .../hadoop/yarn/services/resource/Artifact.java |  4 +-
 .../yarn/services/resource/Component.java       | 16 +++----
 .../yarn/services/resource/Container.java       | 15 ++++---
 .../yarn/services/resource/ReadinessCheck.java  |  6 +--
 7 files changed, 54 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fba18cbe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Application.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Application.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Application.java
index cfcae95..719bf95 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Application.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Application.java
@@ -48,8 +48,8 @@ import com.fasterxml.jackson.annotation.JsonPropertyOrder;
 public class Application extends BaseResource {
   private static final long serialVersionUID = -4491694636566094885L;
 
-  private String id = null;
   private String name = null;
+  private String id = null;
   private Artifact artifact = null;
   private Resource resource = null;
   private String launchCommand = null;
@@ -63,25 +63,7 @@ public class Application extends BaseResource {
   private List<Container> containers = new ArrayList<>();
   private ApplicationState state = null;
   private Map<String, String> quicklinks = null;
-  private String queue;
-
-  /**
-   * A unique application id.
-   **/
-  public Application id(String id) {
-    this.id = id;
-    return this;
-  }
-
-  @ApiModelProperty(example = "null", required = true, value = "A unique application id.")
-  @JsonProperty("id")
-  public String getId() {
-    return id;
-  }
-
-  public void setId(String id) {
-    this.id = id;
-  }
+  private String queue = null;
 
   /**
    * A unique application name.
@@ -102,6 +84,24 @@ public class Application extends BaseResource {
   }
 
   /**
+   * A unique application id.
+   **/
+  public Application id(String id) {
+    this.id = id;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "A unique application id.")
+  @JsonProperty("id")
+  public String getId() {
+    return id;
+  }
+
+  public void setId(String id) {
+    this.id = id;
+  }
+
+  /**
    * Artifact of single-component applications. Mandatory if components
    * attribute is not specified.
    **/
@@ -423,8 +423,8 @@ public class Application extends BaseResource {
     sb.append("    numberOfRunningContainers: ")
         .append(toIndentedString(numberOfRunningContainers)).append("\n");
     sb.append("    lifetime: ").append(toIndentedString(lifetime)).append("\n");
-    sb.append("    placementPolicy: ")
-        .append(toIndentedString(placementPolicy)).append("\n");
+    sb.append("    placementPolicy: ").append(toIndentedString(placementPolicy))
+        .append("\n");
     sb.append("    components: ").append(toIndentedString(components))
         .append("\n");
     sb.append("    configuration: ").append(toIndentedString(configuration))

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fba18cbe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationState.java
index ae96e8a..7f90a9e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationState.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationState.java
@@ -17,9 +17,14 @@
 
 package org.apache.hadoop.yarn.services.resource;
 
+import io.swagger.annotations.ApiModel;
+
 /**
  * The current state of an application.
  **/
+
+@ApiModel(description = "The current state of an application.")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
 public enum ApplicationState {
   ACCEPTED, STARTED, READY, STOPPED, FAILED;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fba18cbe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationStatus.java
index 0166b48..ed826d8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationStatus.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationStatus.java
@@ -22,7 +22,6 @@ import io.swagger.annotations.ApiModelProperty;
 
 import java.util.Objects;
 
-import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
 
 import com.fasterxml.jackson.annotation.JsonInclude;
@@ -33,7 +32,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
  * GET API.
  **/
 
-@ApiModel(description = "The current status of a submitted application, returned as a response to the GET api")
+@ApiModel(description = "The current status of a submitted application, returned as a response to the GET API.")
 @javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
 @XmlRootElement
 @JsonInclude(JsonInclude.Include.NON_NULL)
@@ -60,20 +59,19 @@ public class ApplicationStatus extends BaseResource {
     return diagnostics;
   }
 
-  @XmlElement(name = "diagnostics")
   public void setDiagnostics(String diagnostics) {
     this.diagnostics = diagnostics;
   }
 
   /**
-   * Application state
+   * Application state.
    **/
   public ApplicationStatus state(ApplicationState state) {
     this.state = state;
     return this;
   }
 
-  @ApiModelProperty(example = "null", value = "Application state")
+  @ApiModelProperty(example = "null", value = "Application state.")
   @JsonProperty("state")
   public ApplicationState getState() {
     return state;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fba18cbe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Artifact.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Artifact.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Artifact.java
index aee4d11..af0ad12 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Artifact.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Artifact.java
@@ -76,14 +76,14 @@ public class Artifact {
   }
 
   /**
-   * Artifact type, like docker, tarball, etc. (optional)
+   * Artifact type, like docker, tarball, etc. (optional).
    **/
   public Artifact type(TypeEnum type) {
     this.type = type;
     return this;
   }
 
-  @ApiModelProperty(example = "null", value = "Artifact type, like docker, tarball, etc. (optional)")
+  @ApiModelProperty(example = "null", value = "Artifact type, like docker, tarball, etc. (optional).")
   @JsonProperty("type")
   public TypeEnum getType() {
     return type;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fba18cbe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Component.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Component.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Component.java
index 3ff6945..1246aa8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Component.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Component.java
@@ -202,8 +202,11 @@ public class Component {
    * dictates the initial number of components created. Component names
    * typically differ with a trailing id, but assumptions should not be made on
    * that, as the algorithm can change at any time. Configurations section will
-   * be able to use placeholders like ${APP_COMPONENT_NAME} to get its component
-   * name at runtime, and thereby differing in value at runtime. The best part
+   * be able to use placeholders like ${USER}, ${CLUSTER_NAME} and
+   * ${COMPONENT_NAME} to be replaced at runtime with user the app is submitted
+   * as, application name and application component name respectively. Launch
+   * command can use placeholders like ${APP_COMPONENT_NAME} and ${APP_NAME} to
+   * get its component name and app name respectively at runtime. The best part
    * of this feature is that when the component is flexed up, entirely new
    * components (with new trailing ids) are created.
    **/
@@ -212,7 +215,7 @@ public class Component {
     return this;
   }
 
-  @ApiModelProperty(example = "null", value = "Certain applications need to define multiple components using the same artifact and resource profile, differing only in configurations. In such cases, this field helps app owners to avoid creating multiple component definitions with repeated information. The number_of_containers field dictates the initial number of components created. Component names typically differ with a trailing id, but assumptions should not be made on that, as the algorithm can change at any time. Configurations section will be able to use placeholders like ${APP_COMPONENT_NAME} to get its component name at runtime, and thereby differing in value at runtime. The best part of this feature is that when the component is flexed up, entirely new components (with new trailing ids) are created.")
+  @ApiModelProperty(example = "null", value = "Certain applications need to define multiple components using the same artifact and resource profile, differing only in configurations. In such cases, this field helps app owners to avoid creating multiple component definitions with repeated information. The number_of_containers field dictates the initial number of components created. Component names typically differ with a trailing id, but assumptions should not be made on that, as the algorithm can change at any time. Configurations section will be able to use placeholders like ${USER}, ${CLUSTER_NAME} and ${COMPONENT_NAME} to be replaced at runtime with user the app is submitted as, application name and application component name respectively. Launch command can use placeholders like ${APP_COMPONENT_NAME} and ${APP_NAME} to get its component name and app name respectively at runtime. The best part of this feature is that when the component is flexed up, entirely new components (with 
 new trailing ids) are created.")
   @JsonProperty("unique_component_support")
   public Boolean getUniqueComponentSupport() {
     return uniqueComponentSupport;
@@ -316,8 +319,7 @@ public class Component {
         && Objects.equals(this.artifact, component.artifact)
         && Objects.equals(this.launchCommand, component.launchCommand)
         && Objects.equals(this.resource, component.resource)
-        && Objects
-            .equals(this.numberOfContainers, component.numberOfContainers)
+        && Objects.equals(this.numberOfContainers, component.numberOfContainers)
         && Objects.equals(this.uniqueComponentSupport,
             component.uniqueComponentSupport)
         && Objects.equals(this.runPrivilegedContainer,
@@ -354,8 +356,8 @@ public class Component {
         .append(toIndentedString(uniqueComponentSupport)).append("\n");
     sb.append("    runPrivilegedContainer: ")
         .append(toIndentedString(runPrivilegedContainer)).append("\n");
-    sb.append("    placementPolicy: ")
-        .append(toIndentedString(placementPolicy)).append("\n");
+    sb.append("    placementPolicy: ").append(toIndentedString(placementPolicy))
+        .append("\n");
     sb.append("    configuration: ").append(toIndentedString(configuration))
         .append("\n");
     sb.append("    quicklinks: ").append(toIndentedString(quicklinks))

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fba18cbe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Container.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Container.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Container.java
index 2faf6f2..4e40102 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Container.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Container.java
@@ -29,6 +29,10 @@ import javax.xml.bind.annotation.XmlRootElement;
 import com.fasterxml.jackson.annotation.JsonInclude;
 import com.fasterxml.jackson.annotation.JsonProperty;
 
+/**
+ * An instance of a running application container.
+ **/
+
 @ApiModel(description = "An instance of a running application container")
 @javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
 @XmlRootElement
@@ -47,14 +51,14 @@ public class Container extends BaseResource {
 
   /**
    * Unique container id of a running application, e.g.
-   * container_e3751_1458061340047_0008_01_000002
+   * container_e3751_1458061340047_0008_01_000002.
    **/
   public Container id(String id) {
     this.id = id;
     return this;
   }
 
-  @ApiModelProperty(example = "null", value = "Unique container id of a running application, e.g. container_e3751_1458061340047_0008_01_000002")
+  @ApiModelProperty(example = "null", value = "Unique container id of a running application, e.g. container_e3751_1458061340047_0008_01_000002.")
   @JsonProperty("id")
   public String getId() {
     return id;
@@ -65,7 +69,8 @@ public class Container extends BaseResource {
   }
 
   /**
-   * The time when the container was created, e.g. 2016-03-16T01:01:49.000Z. This will most likely be different from cluster launch time.
+   * The time when the container was created, e.g. 2016-03-16T01:01:49.000Z.
+   * This will most likely be different from cluster launch time.
    **/
   public Container launchTime(Date launchTime) {
     this.launchTime = launchTime;
@@ -126,14 +131,14 @@ public class Container extends BaseResource {
 
   /**
    * The bare node or host in which the container is running, e.g.
-   * cn008.example.com
+   * cn008.example.com.
    **/
   public Container bareHost(String bareHost) {
     this.bareHost = bareHost;
     return this;
   }
 
-  @ApiModelProperty(example = "null", value = "The bare node or host in which the container is running, e.g. cn008.example.com")
+  @ApiModelProperty(example = "null", value = "The bare node or host in which the container is running, e.g. cn008.example.com.")
   @JsonProperty("bare_host")
   public String getBareHost() {
     return bareHost;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fba18cbe/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ReadinessCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ReadinessCheck.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ReadinessCheck.java
index 80fdf92..10c951a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ReadinessCheck.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ReadinessCheck.java
@@ -57,15 +57,15 @@ public class ReadinessCheck {
   private Artifact artifact = null;
 
   /**
-   * http (YARN will perform a simple REST call at a regular interval and expect
-   * a 204 No content).
+   * E.g. HTTP (YARN will perform a simple REST call at a regular interval and
+   * expect a 204 No content).
    **/
   public ReadinessCheck type(TypeEnum type) {
     this.type = type;
     return this;
   }
 
-  @ApiModelProperty(example = "null", value = "http (YARN will perform a simple REST call at a regular interval and expect a 204 No content).")
+  @ApiModelProperty(example = "null", value = "E.g. HTTP (YARN will perform a simple REST call at a regular interval and expect a 204 No content).")
   @JsonProperty("type")
   public TypeEnum getType() {
     return type;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[24/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/AbstractLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/AbstractLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/AbstractLauncher.java
new file mode 100644
index 0000000..5a3eb3d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/AbstractLauncher.java
@@ -0,0 +1,528 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.launch;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LogAggregationContext;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.Records;
+import org.apache.slider.api.ResourceKeys;
+import org.apache.slider.api.RoleKeys;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.tools.CoreFileSystem;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.conf.MapOperations;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Method;
+import java.lang.reflect.InvocationTargetException;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * Launcher of applications: base class
+ */
+public abstract class AbstractLauncher extends Configured {
+  private static final Logger log =
+    LoggerFactory.getLogger(AbstractLauncher.class);
+  public static final String CLASSPATH = "CLASSPATH";
+  /**
+   * Filesystem to use for the launch
+   */
+  protected final CoreFileSystem coreFileSystem;
+  /**
+   * Env vars; set up at final launch stage
+   */
+  protected final Map<String, String> envVars = new HashMap<>();
+  protected final MapOperations env = new MapOperations("env", envVars);
+  protected final ContainerLaunchContext containerLaunchContext =
+    Records.newRecord(ContainerLaunchContext.class);
+  protected final List<String> commands = new ArrayList<>(20);
+  protected final Map<String, LocalResource> localResources = new HashMap<>();
+  protected final Map<String, String> mountPaths = new HashMap<>();
+  private final Map<String, ByteBuffer> serviceData = new HashMap<>();
+  // security
+  protected final Credentials credentials;
+  protected LogAggregationContext logAggregationContext;
+  protected boolean yarnDockerMode = false;
+  protected String dockerImage;
+  protected String yarnContainerMountPoints;
+  protected String runPrivilegedContainer;
+
+
+  /**
+   * Create instance.
+   * @param conf configuration
+   * @param coreFileSystem filesystem
+   * @param credentials initial set of credentials -null is permitted
+   */
+  protected AbstractLauncher(Configuration conf,
+      CoreFileSystem coreFileSystem,
+      Credentials credentials) {
+    super(conf);
+    this.coreFileSystem = coreFileSystem;
+    this.credentials = credentials != null ? credentials: new Credentials();
+  }
+
+  /**
+   * Get the container. Until "completed", this isn't valid to launch.
+   * @return the container to launch
+   */
+  public ContainerLaunchContext getContainerLaunchContext() {
+    return containerLaunchContext;
+  }
+  
+  public void setYarnDockerMode(boolean yarnDockerMode){
+    this.yarnDockerMode = yarnDockerMode;
+  }
+
+  /**
+   * Get the env vars to work on
+   * @return env vars
+   */
+  public MapOperations getEnv() {
+    return env;
+  }
+
+  /**
+   * Get the launch commands.
+   * @return the live list of commands 
+   */
+  public List<String> getCommands() {
+    return commands;
+  }
+
+  /**
+   * Get the map of local resources.
+   * @return the live map of local resources.
+   */
+  public Map<String, LocalResource> getLocalResources() {
+    return localResources;
+  }
+
+  public void addLocalResource(String subPath, LocalResource resource) {
+    localResources.put(subPath, resource);
+  }
+
+  public void addLocalResource(String subPath, LocalResource resource, String mountPath) {
+    localResources.put(subPath, resource);
+    mountPaths.put(subPath, mountPath);
+  }
+
+  /**
+   * Add a set of local resources
+   * @param resourceMap map of name:resource to add
+   */
+  public void addLocalResources(Map<String, LocalResource> resourceMap) {
+    localResources.putAll(resourceMap);
+  }
+
+  public Map<String, ByteBuffer> getServiceData() {
+    return serviceData;
+  }
+
+  /**
+   * Accessor to the credentials
+   * @return the credentials associated with this launcher
+   */
+  public Credentials getCredentials() {
+    return credentials;
+  }
+
+  /**
+   * Add a command line. It is converted to a single command before being
+   * added.
+   * @param cmd
+   */
+  public void addCommandLine(CommandLineBuilder cmd) {
+    commands.add(cmd.build());
+  }
+
+  public void addCommand(String cmd) {
+    commands.add(cmd);
+  }
+
+  /**
+   * Add a list of commands. Each element in the list becomes a single command
+   * @param commandList list of commands
+   */
+  public void addCommands(List<String> commandList) {
+    commands.addAll(commandList);
+  }
+
+  /**
+   * Get all commands as a string, separated by ";". This is for diagnostics
+   * @return a string description of the commands
+   */
+  public String getCommandsAsString() {
+    return SliderUtils.join(getCommands(), "; ");
+  }
+
+  /**
+   * Complete the launch context (copy in env vars, etc).
+   * @return the container to launch
+   */
+  public ContainerLaunchContext completeContainerLaunch() throws IOException {
+    
+    String cmdStr = SliderUtils.join(commands, " ", false);
+    log.debug("Completed setting up container command {}", cmdStr);
+    containerLaunchContext.setCommands(commands);
+
+    //env variables
+    if (log.isDebugEnabled()) {
+      log.debug("Environment variables");
+      for (Map.Entry<String, String> envPair : envVars.entrySet()) {
+        log.debug("    \"{}\"=\"{}\"", envPair.getKey(), envPair.getValue());
+      }
+    }    
+    containerLaunchContext.setEnvironment(env);
+
+    //service data
+    if (log.isDebugEnabled()) {
+      log.debug("Service Data size");
+      for (Map.Entry<String, ByteBuffer> entry : serviceData.entrySet()) {
+        log.debug("\"{}\"=> {} bytes of data", entry.getKey(),
+            entry.getValue().array().length);
+      }
+    }
+    containerLaunchContext.setServiceData(serviceData);
+
+    // resources
+    dumpLocalResources();
+    containerLaunchContext.setLocalResources(localResources);
+
+    //tokens
+    log.debug("{} tokens", credentials.numberOfTokens());
+    containerLaunchContext.setTokens(CredentialUtils.marshallCredentials(
+        credentials));
+
+    if(yarnDockerMode){
+      Map<String, String> env = containerLaunchContext.getEnvironment();
+      env.put("YARN_CONTAINER_RUNTIME_TYPE", "docker");
+      env.put("YARN_CONTAINER_RUNTIME_DOCKER_IMAGE", dockerImage);//if yarnDockerMode, then dockerImage is set
+      env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER", runPrivilegedContainer);
+      StringBuilder sb = new StringBuilder();
+      for (Entry<String,String> mount : mountPaths.entrySet()) {
+        if (sb.length() > 0) {
+          sb.append(",");
+        }
+        sb.append(mount.getKey());
+        sb.append(":");
+        sb.append(mount.getValue());
+      }
+      env.put("YARN_CONTAINER_RUNTIME_DOCKER_LOCAL_RESOURCE_MOUNTS", sb.toString());
+      log.info("yarn docker env var has been set {}", containerLaunchContext.getEnvironment().toString());
+    }
+
+    return containerLaunchContext;
+  }
+
+  /**
+   * Dump local resources at debug level
+   */
+  private void dumpLocalResources() {
+    if (log.isDebugEnabled()) {
+      log.debug("{} resources: ", localResources.size());
+      for (Map.Entry<String, LocalResource> entry : localResources.entrySet()) {
+
+        String key = entry.getKey();
+        LocalResource val = entry.getValue();
+        log.debug(key + "=" + SliderUtils.stringify(val.getResource()));
+      }
+    }
+  }
+
+  /**
+   * This is critical for an insecure cluster -it passes
+   * down the username to YARN, and so gives the code running
+   * in containers the rights it needs to work with
+   * data.
+   * @throws IOException problems working with current user
+   */
+  protected void propagateUsernameInInsecureCluster() throws IOException {
+    //insecure cluster: propagate user name via env variable
+    String userName = UserGroupInformation.getCurrentUser().getUserName();
+    env.put(SliderKeys.HADOOP_USER_NAME, userName);
+  }
+
+  /**
+   * Extract any resource requirements from this component's settings.
+   * All fields that are set will override the existing values -if
+   * unset that resource field will be left unchanged.
+   *
+   * Important: the configuration must already be fully resolved 
+   * in order to pick up global options.
+   * @param resource resource to configure
+   * @param map map of options
+   */
+  public void extractResourceRequirements(Resource resource,
+                                          Map<String, String> map) {
+
+    if (map != null) {
+      MapOperations options = new MapOperations("", map);
+      resource.setMemory(options.getOptionInt(ResourceKeys.YARN_MEMORY,
+                                              resource.getMemory()));
+      resource.setVirtualCores(options.getOptionInt(ResourceKeys.YARN_CORES,
+                                                    resource.getVirtualCores()));
+    }
+  }
+
+  /**
+   * Extract the value for option
+   * {@code yarn.resourcemanager.am.retry-count-window-ms}
+   * and set it on the ApplicationSubmissionContext. Use the default value
+   * if option is not set.
+   *
+   * @param submissionContext
+   * @param map
+   */
+  public void extractAmRetryCount(ApplicationSubmissionContext submissionContext,
+                                  Map<String, String> map) {
+
+    if (map != null) {
+      MapOperations options = new MapOperations("", map);
+      long amRetryCountWindow = options.getOptionLong(ResourceKeys
+          .YARN_RESOURCEMANAGER_AM_RETRY_COUNT_WINDOW_MS,
+          ResourceKeys.DEFAULT_AM_RETRY_COUNT_WINDOW_MS);
+      log.info("Setting {} to {}",
+          ResourceKeys.YARN_RESOURCEMANAGER_AM_RETRY_COUNT_WINDOW_MS,
+          amRetryCountWindow);
+      submissionContext.setAttemptFailuresValidityInterval(amRetryCountWindow);
+    }
+  }
+
+  public void extractLogAggregationContext(Map<String, String> map) {
+    if (map != null) {
+      String logPatternSepStr = "\\|";
+      String logPatternJoinStr = "|";
+      MapOperations options = new MapOperations("", map);
+
+      List<String> logIncludePatterns = new ArrayList<>();
+      String includePatternExpression = options.getOption(
+          ResourceKeys.YARN_LOG_INCLUDE_PATTERNS, "").trim();
+      if (!includePatternExpression.isEmpty()) {
+        String[] includePatterns = includePatternExpression
+            .split(logPatternSepStr);
+        for (String includePattern : includePatterns) {
+          String trimmedIncludePattern = includePattern.trim();
+          if (!trimmedIncludePattern.isEmpty()) {
+            logIncludePatterns.add(trimmedIncludePattern);
+          }
+        }
+      }
+      String logIncludePattern = StringUtils.join(logIncludePatterns,
+          logPatternJoinStr);
+      log.info("Log include patterns: {}", logIncludePattern);
+
+      List<String> logExcludePatterns = new ArrayList<>();
+      String excludePatternExpression = options.getOption(
+          ResourceKeys.YARN_LOG_EXCLUDE_PATTERNS, "").trim();
+      if (!excludePatternExpression.isEmpty()) {
+        String[] excludePatterns = excludePatternExpression
+            .split(logPatternSepStr);
+        for (String excludePattern : excludePatterns) {
+          String trimmedExcludePattern = excludePattern.trim();
+          if (!trimmedExcludePattern.isEmpty()) {
+            logExcludePatterns.add(trimmedExcludePattern);
+          }
+        }
+      }
+      String logExcludePattern = StringUtils.join(logExcludePatterns,
+          logPatternJoinStr);
+      log.info("Log exclude patterns: {}", logExcludePattern);
+
+      // SLIDER-810/YARN-3154 - hadoop 2.7.0 onwards a new instance method has
+      // been added for log aggregation for LRS. Existing newInstance method's
+      // behavior has changed and is used for log aggregation only after the
+      // application has finished. This forces Slider users to move to hadoop
+      // 2.7.0+ just for log aggregation, which is not very desirable. So we
+      // decided to use reflection here to find out if the new 2.7.0 newInstance
+      // method is available. If yes, then we use it, so log aggregation will
+      // work in hadoop 2.7.0+ env. If no, then we fallback to the pre-2.7.0
+      // newInstance method, which means log aggregation will work as expected
+      // in hadoop 2.6 as well.
+      // TODO: At some point, say 2-3 Slider releases down, when most users are
+      // running hadoop 2.7.0, we should get rid of the reflection code here.
+      try {
+        Method logAggregationContextMethod = LogAggregationContext.class
+            .getMethod("newInstance", String.class, String.class, String.class,
+                String.class);
+        // Need to set include/exclude patterns appropriately since by default
+        // rolled log aggregation is not done for any files, so defaults are
+        // - include pattern set to ""
+        // - exclude pattern set to "*"
+        // For Slider we want all logs to be uploaded if include/exclude
+        // patterns are left empty by the app owner in resources file
+        if (StringUtils.isEmpty(logIncludePattern)
+            && StringUtils.isEmpty(logExcludePattern)) {
+          logIncludePattern = ".*";
+          logExcludePattern = "";
+        } else if (StringUtils.isEmpty(logIncludePattern)
+            && StringUtils.isNotEmpty(logExcludePattern)) {
+          logIncludePattern = ".*";
+        } else if (StringUtils.isNotEmpty(logIncludePattern)
+            && StringUtils.isEmpty(logExcludePattern)) {
+          logExcludePattern = "";
+        }
+        log.debug("LogAggregationContext newInstance method for rolled logs "
+            + "include/exclude patterns is available");
+        log.info("Modified log include patterns: {}", logIncludePattern);
+        log.info("Modified log exclude patterns: {}", logExcludePattern);
+        logAggregationContext = (LogAggregationContext) logAggregationContextMethod
+            .invoke(null, null, null, logIncludePattern, logExcludePattern);
+      } catch (NoSuchMethodException | SecurityException
+          | IllegalAccessException | IllegalArgumentException
+          | InvocationTargetException e) {
+        log.debug("LogAggregationContext newInstance method for rolled logs "
+            + "include/exclude patterns is not available - fallback to old one");
+        log.debug(e.toString());
+        logAggregationContext = LogAggregationContext.newInstance(
+            logIncludePattern, logExcludePattern);
+      }
+    }
+  }
+
+  /**
+   * Utility method to set up the classpath
+   * @param classpath classpath to use
+   */
+  public void setClasspath(ClasspathConstructor classpath) {
+    setEnv(CLASSPATH, classpath.buildClasspath());
+  }
+
+  /**
+   * Set an environment variable in the launch context
+   * @param var variable name
+   * @param value value (must be non null)
+   */
+  public void setEnv(String var, String value) {
+    Preconditions.checkArgument(var != null, "null variable name");
+    Preconditions.checkArgument(value != null, "null value");
+    env.put(var, value);
+  }
+
+  /**
+   * Set an environment variable if its value is non-null.
+   * @param var variable name
+   * @param value value (may be null)
+   */
+  public void maybeSetEnv(String var, String value) {
+    if (value != null) {
+      setEnv(var, value);
+    }
+  }
+
+  public void putEnv(Map<String, String> map) {
+    env.putAll(map);
+  }
+
+  /**
+   * Important: the configuration must already be fully resolved 
+   * in order to pick up global options
+   * Copy env vars into the launch context.
+   */
+  public boolean copyEnvVars(MapOperations options) {
+    if (options == null) {
+      return false;
+    }
+    for (Map.Entry<String, String> entry : options.entrySet()) {
+      String key = entry.getKey();
+      if (key.startsWith(RoleKeys.ENV_PREFIX)) {
+        key = key.substring(RoleKeys.ENV_PREFIX.length());
+        env.put(key, entry.getValue());
+      }
+    }
+    return true;
+  }
+
+  public String[] dumpEnvToString() {
+
+    List<String> nodeEnv = new ArrayList<>();
+
+    for (Map.Entry<String, String> entry : env.entrySet()) {
+      String envElt = String.format("%s=\"%s\"",
+                                    entry.getKey(),
+                                    entry.getValue());
+      log.debug(envElt);
+      nodeEnv.add(envElt);
+    }
+    String[] envDescription = nodeEnv.toArray(new String[nodeEnv.size()]);
+
+    return envDescription;
+  }
+
+  /**
+   * Submit an entire directory
+   * @param srcDir src path in filesystem
+   * @param destRelativeDir relative path under destination local dir
+   * @throws IOException IO problems
+   */
+  public void submitDirectory(Path srcDir, String destRelativeDir)
+      throws IOException {
+    //add the configuration resources
+    Map<String, LocalResource> confResources;
+    confResources = coreFileSystem.submitDirectory(
+      srcDir,
+      destRelativeDir);
+    addLocalResources(confResources);
+  }
+
+  /**
+   * Return the label expression and if not set null
+   * @param map map to look up
+   * @return extracted label or null
+   */
+  public String extractLabelExpression(Map<String, String> map) {
+    if (map != null) {
+      MapOperations options = new MapOperations("", map);
+      return options.getOption(ResourceKeys.YARN_LABEL_EXPRESSION, null);
+    }
+    return null;
+  }
+
+  public void setDockerImage(String dockerImage) {
+    this.dockerImage = dockerImage;
+  }
+
+  public void setYarnContainerMountPoints(String yarnContainerMountPoints) {
+    this.yarnContainerMountPoints = yarnContainerMountPoints;
+  }
+
+  public void setRunPrivilegedContainer(String runPrivilegedContainer) {
+    this.runPrivilegedContainer = runPrivilegedContainer;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/AppMasterLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/AppMasterLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/AppMasterLauncher.java
new file mode 100644
index 0000000..7190c3a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/AppMasterLauncher.java
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.launch;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.client.api.YarnClientApplication;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.util.Records;
+import org.apache.slider.client.SliderYarnClientImpl;
+import org.apache.slider.common.tools.CoreFileSystem;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Set;
+
+public class AppMasterLauncher extends AbstractLauncher {
+
+
+  private static final Logger log =
+    LoggerFactory.getLogger(AppMasterLauncher.class);
+
+  public final YarnClientApplication application;
+  public final String name;
+  public final String type;
+  public final ApplicationSubmissionContext submissionContext;
+  public final ApplicationId appId;
+  public final boolean secureCluster;
+  private int maxAppAttempts = 0;
+  private boolean keepContainersOverRestarts = true;
+  private String queue = YarnConfiguration.DEFAULT_QUEUE_NAME;
+  private int priority = 1;
+  private final Resource resource = Records.newRecord(Resource.class);
+  private final SliderYarnClientImpl yarnClient;
+  private Long submitTime;
+
+  /**
+   * Build the AM Launcher
+   * @param name app name
+   * @param type application type
+   * @param conf hadoop config
+   * @param fs filesystem binding
+   * @param yarnClient yarn client
+   * @param secureCluster flag to indicate secure cluster
+   * @param options map of options. All values are extracted in this constructor only
+   * @param resourceGlobalOptions global options
+   * @param applicationTags any app tags
+   * @param credentials initial set of credentials
+   * @throws IOException
+   * @throws YarnException
+   */
+  public AppMasterLauncher(String name,
+      String type,
+      Configuration conf,
+      CoreFileSystem fs,
+      SliderYarnClientImpl yarnClient,
+      boolean secureCluster,
+      Map<String, String> options,
+      Map<String, String> resourceGlobalOptions,
+      Set<String> applicationTags,
+      Credentials credentials) throws IOException, YarnException {
+    super(conf, fs, credentials);
+    this.yarnClient = yarnClient;
+    this.application = yarnClient.createApplication();
+    this.name = name;
+    this.type = type;
+    this.secureCluster = secureCluster;
+
+    submissionContext = application.getApplicationSubmissionContext();
+    appId = submissionContext.getApplicationId();
+    // set the application name;
+    submissionContext.setApplicationName(name);
+    // app type used in service enum;
+    submissionContext.setApplicationType(type);
+    if (!applicationTags.isEmpty()) {
+      submissionContext.setApplicationTags(applicationTags);
+    }
+    submissionContext.setNodeLabelExpression(extractLabelExpression(options));
+
+    extractAmRetryCount(submissionContext, resourceGlobalOptions);
+    extractResourceRequirements(resource, options);
+    extractLogAggregationContext(resourceGlobalOptions);
+  }
+
+  public void setMaxAppAttempts(int maxAppAttempts) {
+    this.maxAppAttempts = maxAppAttempts;
+  }
+
+  public void setKeepContainersOverRestarts(boolean keepContainersOverRestarts) {
+    this.keepContainersOverRestarts = keepContainersOverRestarts;
+  }
+
+
+  public Resource getResource() {
+    return resource;
+  }
+
+  public void setMemory(int memory) {
+    resource.setMemory(memory);
+  }
+
+  public void setVirtualCores(int cores) {
+    resource.setVirtualCores(cores);
+  }
+
+  public ApplicationId getApplicationId() {
+    return appId;
+  }
+
+  public int getMaxAppAttempts() {
+    return maxAppAttempts;
+  }
+
+  public boolean isKeepContainersOverRestarts() {
+    return keepContainersOverRestarts;
+  }
+
+  public String getQueue() {
+    return queue;
+  }
+
+  public int getPriority() {
+    return priority;
+  }
+
+  public void setQueue(String queue) {
+    this.queue = queue;
+  }
+
+  public void setPriority(int priority) {
+    this.priority = priority;
+  }
+
+  /**
+   * Complete the launch context (copy in env vars, etc).
+   * @return the container to launch
+   */
+  public ApplicationSubmissionContext completeAppMasterLaunch()
+      throws IOException {
+
+    //queue priority
+    Priority pri = Records.newRecord(Priority.class);
+    pri.setPriority(priority);
+    submissionContext.setPriority(pri);
+
+    // Set the queue to which this application is to be submitted in the RM
+    // Queue for App master
+
+    submissionContext.setQueue(queue);
+
+
+    //container requirements
+    submissionContext.setResource(resource);
+    submissionContext.setLogAggregationContext(logAggregationContext);
+
+    if (keepContainersOverRestarts) {
+      log.debug("Requesting cluster stays running over AM failure");
+      submissionContext.setKeepContainersAcrossApplicationAttempts(true);
+    }
+
+    if (maxAppAttempts > 0) {
+      log.debug("Setting max AM attempts to {}", maxAppAttempts);
+      submissionContext.setMaxAppAttempts(maxAppAttempts);
+    }
+
+    if (secureCluster) {
+      //tokens
+      log.debug("Credentials: {}",
+          CredentialUtils.dumpTokens(getCredentials(), "\n"));
+
+    } else {
+      propagateUsernameInInsecureCluster();
+    }
+    completeContainerLaunch();
+    submissionContext.setAMContainerSpec(containerLaunchContext);
+    return submissionContext;
+  }
+
+  /**
+   * Submit the application. 
+   * @return a launched application representing the submitted application
+   * @throws IOException
+   * @throws YarnException
+   */
+  public LaunchedApplication submitApplication() throws IOException, YarnException {
+    completeAppMasterLaunch();
+    log.info("Submitting application to Resource Manager");
+    ApplicationId applicationId =
+      yarnClient.submitApplication(submissionContext);
+    // implicit success; record the time
+    submitTime = System.currentTimeMillis();
+    return new LaunchedApplication(applicationId, yarnClient);
+  }
+
+  /**
+   * Build a serializable application report. This is a very minimal
+   * report that contains the application Id, name and type \u2014the information
+   * available
+   * @return a data structure which can be persisted
+   */
+  public SerializedApplicationReport createSerializedApplicationReport() {
+    SerializedApplicationReport sar = new SerializedApplicationReport();
+    sar.applicationId = appId.toString();
+    sar.name = name;
+    sar.applicationType = type;
+    sar.queue = queue;
+    sar.submitTime = submitTime;
+    return sar;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java
new file mode 100644
index 0000000..6eb4058
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.launch;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.slider.common.tools.SliderUtils;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * build a classpath -allows for entries to be injected in front of
+ * YARN classpath as well as behind, adds appropriate separators, 
+ * extraction of local classpath, etc.
+ */
+public class ClasspathConstructor {
+
+    public static final String CLASS_PATH_SEPARATOR = ApplicationConstants.CLASS_PATH_SEPARATOR;
+  private final List<String> pathElements = new ArrayList<>();
+
+  public ClasspathConstructor() {
+  }
+
+
+  /**
+   * Get the list of JARs from the YARN settings
+   * @param config configuration
+   */
+  public List<String> yarnApplicationClasspath(Configuration config) {
+    String[] cp = config.getTrimmedStrings(
+      YarnConfiguration.YARN_APPLICATION_CLASSPATH,
+      YarnConfiguration.DEFAULT_YARN_CROSS_PLATFORM_APPLICATION_CLASSPATH);
+    return cp != null ? Arrays.asList(cp) : new ArrayList<String>(0);
+
+  }
+
+
+  @Override
+  public String toString() {
+    return buildClasspath();
+  }
+
+  public String buildClasspath() {
+    return SliderUtils.join(pathElements,
+        CLASS_PATH_SEPARATOR,
+        false);
+  }
+
+  /**
+   * Get a copy of the path list
+   * @return the JARs
+   */
+  public List<String> getPathElements() {
+    return Collections.unmodifiableList(pathElements);
+  }
+
+  /**
+   * Append an entry
+   * @param path path
+   */
+  public void append(String path) {
+    pathElements.add(path);
+  }
+
+  /**
+   * Insert a path at the front of the list. This places it ahead of
+   * the standard YARN artifacts
+   * @param path path to the JAR. Absolute or relative -on the target
+   * system
+   */
+  public void insert(String path) {
+    pathElements.add(0, path);
+  }
+
+  public void appendAll(Collection<String> paths) {
+    pathElements.addAll(paths);
+  }
+
+  public void insertAll(Collection<String> paths) {
+    pathElements.addAll(0, paths);
+  }
+
+
+  public void addLibDir(String pathToLibDir) {
+    append(buildLibDir(pathToLibDir));
+  }
+
+  public void insertLibDir(String pathToLibDir) {
+    insert(buildLibDir(pathToLibDir));
+  }
+
+  public void addClassDirectory(String pathToDir) {
+    append(appendDirectoryTerminator(pathToDir));
+  }
+
+  public void insertClassDirectory(String pathToDir) {
+    insert(buildLibDir(appendDirectoryTerminator(pathToDir)));
+  }
+
+
+  public void addRemoteClasspathEnvVar() {
+    append(ApplicationConstants.Environment.CLASSPATH.$$());
+  }
+
+
+  public void insertRemoteClasspathEnvVar() {
+    append(ApplicationConstants.Environment.CLASSPATH.$$());
+  }
+
+
+  /**
+   * Build a lib dir path
+   * @param pathToLibDir path to the directory; may or may not end with a
+   * trailing space
+   * @return a path to a lib dir that is compatible with the java classpath
+   */
+  public String buildLibDir(String pathToLibDir) {
+    String dir = appendDirectoryTerminator(pathToLibDir);
+    dir += "*";
+    return dir;
+  }
+
+  private String appendDirectoryTerminator(String pathToLibDir) {
+    String dir = pathToLibDir.trim();
+    if (!dir.endsWith("/")) {
+      dir += "/";
+    }
+    return dir;
+  }
+
+  /**
+   * Split a classpath. This uses the local path separator so MUST NOT
+   * be used to work with remote classpaths
+   * @param localpath local path
+   * @return a splite
+   */
+  public Collection<String> splitClasspath(String localpath) {
+    String separator = System.getProperty("path.separator");
+    return StringUtils.getStringCollection(localpath, separator);
+  }
+
+  /**
+   * Get the local JVM classpath split up
+   * @return the list of entries on the JVM classpath env var
+   */
+  public Collection<String> localJVMClasspath() {
+    return splitClasspath(System.getProperty("java.class.path"));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/CommandLineBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/CommandLineBuilder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/CommandLineBuilder.java
new file mode 100644
index 0000000..dbaa981
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/CommandLineBuilder.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.launch;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.slider.common.tools.SliderUtils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Build a single command line to include in the container commands;
+ * Special support for JVM command buildup.
+ */
+public class CommandLineBuilder {
+  protected final List<String> argumentList = new ArrayList<>(20);
+
+
+  /**
+   * Add an entry to the command list
+   * @param args arguments -these will be converted strings
+   */
+  public void add(Object... args) {
+    for (Object arg : args) {
+      argumentList.add(arg.toString());
+    }
+  }
+
+  /**
+   * Get the value at an offset
+   * @param offset offset
+   * @return the value at that point
+   */
+  public String elt(int offset) {
+    return argumentList.get(offset);
+  }
+
+  /**
+   * Get the number of arguments
+   * @return an integer >= 0
+   */
+  public int size() {
+    return argumentList.size();
+  }
+  
+  /**
+   * Append the output and error files to the tail of the command
+   * @param stdout out
+   * @param stderr error. Set this to null to append into stdout
+   */
+  public void addOutAndErrFiles(String stdout, String stderr) {
+    Preconditions.checkNotNull(stdout, "Null output file");
+    Preconditions.checkState(!stdout.isEmpty(), "output filename invalid");
+    // write out the path output
+    argumentList.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/" +
+             stdout);
+    if (stderr != null) {
+      argumentList.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/" +
+               stderr);
+    } else {
+      argumentList.add("2>&1");
+    }
+  }
+
+  /**
+   * This just returns the command line
+   * @see #build()
+   * @return the command line
+   */
+  @Override
+  public String toString() {
+    return build();
+  }
+
+  /**
+   * Build the command line
+   * @return the command line
+   */
+  public String build() {
+    return SliderUtils.join(argumentList, " ");
+  }
+
+  public List<String> getArgumentList() {
+    return argumentList;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/ContainerLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/ContainerLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/ContainerLauncher.java
new file mode 100644
index 0000000..d220928
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/ContainerLauncher.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.launch;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.slider.common.tools.CoreFileSystem;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetSocketAddress;
+
+/**
+ * Code to ease launching of any container
+ */
+public class ContainerLauncher extends AbstractLauncher {
+  private static final Logger log =
+    LoggerFactory.getLogger(ContainerLauncher.class);
+  // Allocated container
+  public final Container container;
+
+  public ContainerLauncher(Configuration conf,
+      CoreFileSystem coreFileSystem,
+      Container container,
+      Credentials credentials) {
+    super(conf, coreFileSystem, credentials);
+    this.container = container;
+  }
+
+  /**
+   * This code is in the dist shell examples -it's been moved here
+   * so that if it is needed, it's still here
+   * @return a remote user with a token to access the container.
+   */
+  public UserGroupInformation setupUGI() {
+    UserGroupInformation user =
+      UserGroupInformation.createRemoteUser(container.getId().toString());
+    String cmIpPortStr = container.getNodeId().getHost() + ":" + container.getNodeId().getPort();
+    final InetSocketAddress cmAddress = NetUtils.createSocketAddr(cmIpPortStr);
+
+    org.apache.hadoop.yarn.api.records.Token containerToken = container.getContainerToken();
+    if (containerToken != null) {
+      Token<ContainerTokenIdentifier> token =
+        ConverterUtils.convertFromYarn(containerToken, cmAddress);
+      user.addToken(token);
+    }
+    return user;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/CredentialUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/CredentialUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/CredentialUtils.java
new file mode 100644
index 0000000..5357cc4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/CredentialUtils.java
@@ -0,0 +1,379 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.launch;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
+import org.apache.hadoop.yarn.client.ClientRMProxy;
+import org.apache.hadoop.yarn.client.api.TimelineClient;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.conf.HAUtil;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.slider.common.SliderXmlConfKeys;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.security.UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION;
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.*;
+
+/**
+ * Utils to work with credentials and tokens.
+ *
+ * Designed to be movable to Hadoop core
+ */
+public final class CredentialUtils {
+
+  public static final String JOB_CREDENTIALS_BINARY
+      = SliderXmlConfKeys.MAPREDUCE_JOB_CREDENTIALS_BINARY;
+
+  private CredentialUtils() {
+  }
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CredentialUtils.class);
+
+  /**
+   * Save credentials to a byte buffer. Returns null if there were no
+   * credentials to save
+   * @param credentials credential set
+   * @return a byte buffer of serialized tokens
+   * @throws IOException if the credentials could not be written to the stream
+   */
+  public static ByteBuffer marshallCredentials(Credentials credentials) throws IOException {
+    ByteBuffer buffer = null;
+    if (!credentials.getAllTokens().isEmpty()) {
+      DataOutputBuffer dob = new DataOutputBuffer();
+      try {
+        credentials.writeTokenStorageToStream(dob);
+      } finally {
+        dob.close();
+      }
+      buffer = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
+    }
+    return buffer;
+  }
+
+  public static File locateEnvCredentials(Map<String, String> env,
+      Configuration conf,
+      StringBuffer sourceTextOut) throws FileNotFoundException {
+    String tokenFilename = env.get(HADOOP_TOKEN_FILE_LOCATION);
+    String source = "environment variable " + HADOOP_TOKEN_FILE_LOCATION;
+    if (tokenFilename == null) {
+      tokenFilename = conf.get(JOB_CREDENTIALS_BINARY);
+      source = "configuration option " + JOB_CREDENTIALS_BINARY;
+    }
+    if (tokenFilename != null) {
+      // use delegation tokens, i.e. from Oozie
+      File file = new File(tokenFilename.trim());
+      String details = String.format(
+          "Token File %s from %s",
+          file,
+          source);
+      if (!file.exists()) {
+        throw new FileNotFoundException("No " + details);
+      }
+      if (!file.isFile() && !file.canRead()) {
+        throw new FileNotFoundException("Cannot read " + details);
+      }
+      sourceTextOut.append(details);
+      return file;
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Load the credentials from the environment. This looks at
+   * the value of {@link UserGroupInformation#HADOOP_TOKEN_FILE_LOCATION}
+   * and attempts to read in the value
+   * @param env environment to resolve the variable from
+   * @param conf configuration use when reading the tokens
+   * @return a set of credentials, or null if the environment did not
+   * specify any
+   * @throws IOException if a location for credentials was defined, but
+   * the credentials could not be loaded.
+   */
+  public static Credentials loadTokensFromEnvironment(Map<String, String> env,
+      Configuration conf)
+      throws IOException {
+    StringBuffer origin = new StringBuffer();
+    File file = locateEnvCredentials(env, conf, origin);
+    if (file != null) {
+      LOG.debug("Using {}", origin);
+      return Credentials.readTokenStorageFile(file, conf);
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Save credentials to a file
+   * @param file file to save to (will be overwritten)
+   * @param credentials credentials to write
+   * @throws IOException
+   */
+  public static void saveTokens(File file,
+      Credentials credentials) throws IOException {
+    try(DataOutputStream daos = new DataOutputStream(
+        new FileOutputStream(file))) {
+      credentials.writeTokenStorageToStream(daos);
+    }
+  }
+
+  /**
+   * Look up and return the resource manager's principal. This method
+   * automatically does the <code>_HOST</code> replacement in the principal and
+   * correctly handles HA resource manager configurations.
+   *
+   * From: YARN-4629
+   * @param conf the {@link Configuration} file from which to read the
+   * principal
+   * @return the resource manager's principal string
+   * @throws IOException thrown if there's an error replacing the host name
+   */
+  public static String getRMPrincipal(Configuration conf) throws IOException {
+    String principal = conf.get(RM_PRINCIPAL, "");
+    String hostname;
+    Preconditions.checkState(!principal.isEmpty(), "Not set: " + RM_PRINCIPAL);
+
+    if (HAUtil.isHAEnabled(conf)) {
+      YarnConfiguration yarnConf = new YarnConfiguration(conf);
+      if (yarnConf.get(RM_HA_ID) == null) {
+        // If RM_HA_ID is not configured, use the first of RM_HA_IDS.
+        // Any valid RM HA ID should work.
+        String[] rmIds = yarnConf.getStrings(RM_HA_IDS);
+        Preconditions.checkState((rmIds != null) && (rmIds.length > 0),
+            "Not set " + RM_HA_IDS);
+        yarnConf.set(RM_HA_ID, rmIds[0]);
+      }
+
+      hostname = yarnConf.getSocketAddr(
+          RM_ADDRESS,
+          DEFAULT_RM_ADDRESS,
+          DEFAULT_RM_PORT).getHostName();
+    } else {
+      hostname = conf.getSocketAddr(
+          RM_ADDRESS,
+          DEFAULT_RM_ADDRESS,
+          DEFAULT_RM_PORT).getHostName();
+    }
+    return SecurityUtil.getServerPrincipal(principal, hostname);
+  }
+
+  /**
+   * Create and add any filesystem delegation tokens with
+   * the RM(s) configured to be able to renew them. Returns null
+   * on an insecure cluster (i.e. harmless)
+   * @param conf configuration
+   * @param fs filesystem
+   * @param credentials credentials to update
+   * @return a list of all added tokens.
+   * @throws IOException
+   */
+  public static Token<?>[] addRMRenewableFSDelegationTokens(Configuration conf,
+      FileSystem fs,
+      Credentials credentials) throws IOException {
+    Preconditions.checkArgument(conf != null);
+    Preconditions.checkArgument(credentials != null);
+    if (UserGroupInformation.isSecurityEnabled()) {
+      return fs.addDelegationTokens(CredentialUtils.getRMPrincipal(conf),
+          credentials);
+    }
+    return null;
+  }
+
+  /**
+   * Add an FS delegation token which can be renewed by the current user
+   * @param fs filesystem
+   * @param credentials credentials to update
+   * @throws IOException problems.
+   */
+  public static void addSelfRenewableFSDelegationTokens(
+      FileSystem fs,
+      Credentials credentials) throws IOException {
+    Preconditions.checkArgument(fs != null);
+    Preconditions.checkArgument(credentials != null);
+    fs.addDelegationTokens(
+        getSelfRenewer(),
+        credentials);
+  }
+
+  public static String getSelfRenewer() throws IOException {
+    return UserGroupInformation.getLoginUser().getShortUserName();
+  }
+
+  /**
+   * Create and add an RM delegation token to the credentials
+   * @param yarnClient Yarn Client
+   * @param credentials to add token to
+   * @return the token which was added
+   * @throws IOException
+   * @throws YarnException
+   */
+  public static Token<TokenIdentifier> addRMDelegationToken(YarnClient yarnClient,
+      Credentials credentials)
+      throws IOException, YarnException {
+    Configuration conf = yarnClient.getConfig();
+    Text rmPrincipal = new Text(CredentialUtils.getRMPrincipal(conf));
+    Text rmDTService = ClientRMProxy.getRMDelegationTokenService(conf);
+    Token<TokenIdentifier> rmDelegationToken =
+        ConverterUtils.convertFromYarn(
+            yarnClient.getRMDelegationToken(rmPrincipal),
+            rmDTService);
+    credentials.addToken(rmDelegationToken.getService(), rmDelegationToken);
+    return rmDelegationToken;
+  }
+
+  public static Token<TimelineDelegationTokenIdentifier> maybeAddTimelineToken(
+      Configuration conf,
+      Credentials credentials)
+      throws IOException, YarnException {
+    if (conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, false)) {
+      LOG.debug("Timeline service enabled -fetching token");
+
+      try(TimelineClient timelineClient = TimelineClient.createTimelineClient()) {
+        timelineClient.init(conf);
+        timelineClient.start();
+        Token<TimelineDelegationTokenIdentifier> token =
+            timelineClient.getDelegationToken(
+                CredentialUtils.getRMPrincipal(conf));
+        credentials.addToken(token.getService(), token);
+        return token;
+      }
+    } else {
+      LOG.debug("Timeline service is disabled");
+      return null;
+    }
+  }
+
+  /**
+   * Filter a list of tokens from a set of credentials
+   * @param credentials credential source (a new credential set os re
+   * @param filter List of tokens to strip out
+   * @return a new, filtered, set of credentials
+   */
+  public static Credentials filterTokens(Credentials credentials,
+      List<Text> filter) {
+    Credentials result = new Credentials(credentials);
+    Iterator<Token<? extends TokenIdentifier>> iter =
+        result.getAllTokens().iterator();
+    while (iter.hasNext()) {
+      Token<? extends TokenIdentifier> token = iter.next();
+      LOG.debug("Token {}", token.getKind());
+      if (filter.contains(token.getKind())) {
+        LOG.debug("Filtering token {}", token.getKind());
+        iter.remove();
+      }
+    }
+    return result;
+  }
+
+  public static String dumpTokens(Credentials credentials, String separator) {
+    ArrayList<Token<? extends TokenIdentifier>> sorted =
+        new ArrayList<>(credentials.getAllTokens());
+    Collections.sort(sorted, new TokenComparator());
+    StringBuilder buffer = new StringBuilder(sorted.size()* 128);
+    for (Token<? extends TokenIdentifier> token : sorted) {
+      buffer.append(tokenToString(token)).append(separator);
+    }
+    return buffer.toString();
+  }
+
+  /**
+   * Create a string for people to look at
+   * @param token token to convert to a string form
+   * @return a printable view of the token
+   */
+  public static String tokenToString(Token<? extends TokenIdentifier> token) {
+    DateFormat df = DateFormat.getDateTimeInstance(
+        DateFormat.SHORT, DateFormat.SHORT);
+    StringBuilder buffer = new StringBuilder(128);
+    buffer.append(token.toString());
+    try {
+      TokenIdentifier ti = token.decodeIdentifier();
+      buffer.append("; ").append(ti);
+      if (ti instanceof AbstractDelegationTokenIdentifier) {
+        // details in human readable form, and compensate for information HDFS DT omits
+        AbstractDelegationTokenIdentifier dt = (AbstractDelegationTokenIdentifier) ti;
+        buffer.append("; Renewer: ").append(dt.getRenewer());
+        buffer.append("; Issued: ")
+            .append(df.format(new Date(dt.getIssueDate())));
+        buffer.append("; Max Date: ")
+            .append(df.format(new Date(dt.getMaxDate())));
+      }
+    } catch (IOException e) {
+      //marshall problem; not ours
+      LOG.debug("Failed to decode {}: {}", token, e, e);
+    }
+    return buffer.toString();
+  }
+
+  /**
+   * Get the expiry time of a token.
+   * @param token token to examine
+   * @return the time in milliseconds after which the token is invalid.
+   * @throws IOException
+   */
+  public static long getTokenExpiryTime(Token token) throws IOException {
+    TokenIdentifier identifier = token.decodeIdentifier();
+    Preconditions.checkState(identifier instanceof AbstractDelegationTokenIdentifier,
+        "Token %s of type: %s has an identifier which cannot be examined: %s",
+        token, token.getClass(), identifier);
+    AbstractDelegationTokenIdentifier id =
+        (AbstractDelegationTokenIdentifier) identifier;
+    return id.getMaxDate();
+  }
+
+  private static class TokenComparator
+      implements Comparator<Token<? extends TokenIdentifier>>, Serializable {
+    @Override
+    public int compare(Token<? extends TokenIdentifier> left,
+        Token<? extends TokenIdentifier> right) {
+      return left.getKind().toString().compareTo(right.getKind().toString());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java
new file mode 100644
index 0000000..b8aa4c6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.launch;
+
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.exceptions.BadConfigException;
+
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ * Command line builder purely for the Java CLI.
+ * Some of the <code>define</code> methods are designed to work with Hadoop tool and
+ * Slider launcher applications.
+ */
+public class JavaCommandLineBuilder extends CommandLineBuilder {
+
+  public JavaCommandLineBuilder() {
+    add(getJavaBinary());
+  }
+
+  /**
+   * Get the java binary. This is called in the constructor so don't try and
+   * do anything other than return a constant.
+   * @return the path to the Java binary
+   */
+  protected String getJavaBinary() {
+    return ApplicationConstants.Environment.JAVA_HOME.$$() + "/bin/java";
+  }
+
+  /**
+   * Set the size of the heap if a non-empty heap is passed in. 
+   * @param heap empty string or something like "128M" ,"1G" etc. The value is
+   * trimmed.
+   */
+  public void setJVMHeap(String heap) {
+    if (SliderUtils.isSet(heap)) {
+      add("-Xmx" + heap.trim());
+    }
+  }
+
+  /**
+   * Turn Java assertions on
+   */
+  public void enableJavaAssertions() {
+    add("-ea");
+    add("-esa");
+  }
+
+  /**
+   * Add a system property definition -must be used before setting the main entry point
+   * @param property
+   * @param value
+   */
+  public void sysprop(String property, String value) {
+    Preconditions.checkArgument(property != null, "null property name");
+    Preconditions.checkArgument(value != null, "null value");
+    add("-D" + property + "=" + value);
+  }
+  
+  public JavaCommandLineBuilder forceIPv4() {
+    sysprop("java.net.preferIPv4Stack", "true");
+    return this;
+  }
+  
+  public JavaCommandLineBuilder headless() {
+    sysprop("java.awt.headless", "true");
+    return this;
+  }
+
+  public boolean addConfOption(Configuration conf, String key) {
+    return defineIfSet(key, conf.get(key));
+  }
+
+  /**
+   * Add a varargs list of configuration parameters \u2014if they are present
+   * @param conf configuration source
+   * @param keys keys
+   */
+  public void addConfOptions(Configuration conf, String... keys) {
+    for (String key : keys) {
+      addConfOption(conf, key);
+    }
+  }
+
+  /**
+   * Add all configuration options which match the prefix
+   * @param conf configuration
+   * @param prefix prefix, e.g {@code "slider."}
+   * @return the number of entries copied
+   */
+  public int addPrefixedConfOptions(Configuration conf, String prefix) {
+    int copied = 0;
+    for (Map.Entry<String, String> entry : conf) {
+      if (entry.getKey().startsWith(prefix)) {
+        define(entry.getKey(), entry.getValue());
+        copied++;
+      }
+    }
+    return copied;
+  }
+
+  /**
+   * Ass a configuration option to the command line of  the application
+   * @param conf configuration
+   * @param key key
+   * @param defVal default value
+   * @return the resolved configuration option
+   * @throws IllegalArgumentException if key is null or the looked up value
+   * is null (that is: the argument is missing and devVal was null.
+   */
+  public String addConfOptionToCLI(Configuration conf,
+      String key,
+      String defVal) {
+    Preconditions.checkArgument(key != null, "null key");
+    String val = conf.get(key, defVal);
+    define(key, val);
+    return val;
+  }
+
+  /**
+   * Add a <code>-D key=val</code> command to the CLI. This is very Hadoop API
+   * @param key key
+   * @param val value
+   * @throws IllegalArgumentException if either argument is null
+   */
+  public void define(String key, String val) {
+    Preconditions.checkArgument(key != null, "null key");
+    Preconditions.checkArgument(val != null, "null value");
+    add("-D", key + "=" + val);
+  }
+
+  /**
+   * Add a <code>-D key=val</code> command to the CLI if <code>val</code>
+   * is not null
+   * @param key key
+   * @param val value
+   */
+  public boolean defineIfSet(String key, String val) {
+    Preconditions.checkArgument(key != null, "null key");
+    if (val != null) {
+      define(key, val);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Add a mandatory config option
+   * @param conf configuration
+   * @param key key
+   * @throws BadConfigException if the key is missing
+   */
+  public void addMandatoryConfOption(Configuration conf,
+      String key) throws BadConfigException {
+    if (!addConfOption(conf, key)) {
+      throw new BadConfigException("Missing configuration option: " + key);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/LaunchedApplication.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/LaunchedApplication.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/LaunchedApplication.java
new file mode 100644
index 0000000..632e3fd
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/LaunchedApplication.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.launch;
+
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.slider.client.SliderYarnClientImpl;
+import org.apache.slider.common.tools.Duration;
+
+import java.io.IOException;
+
+/**
+ * Launched App with logic around it.
+ */
+public class LaunchedApplication {
+
+  protected final ApplicationId applicationId;
+  protected final SliderYarnClientImpl yarnClient;
+
+  public LaunchedApplication(ApplicationId applicationId,
+                             SliderYarnClientImpl yarnClient) {
+    assert applicationId != null;
+    assert yarnClient != null;
+    this.applicationId = applicationId;
+    this.yarnClient = yarnClient;
+  }
+
+  public LaunchedApplication(SliderYarnClientImpl yarnClient,
+                             ApplicationReport report) {
+    this.yarnClient = yarnClient;
+    this.applicationId = report.getApplicationId();
+  }
+
+  public ApplicationId getApplicationId() {
+    return applicationId;
+  }
+
+  /**
+   * Monitor the submitted application for reaching the requested state.
+   * Will also report if the app reaches a later state (failed, killed, etc)
+   * Kill application if duration!= null & time expires. 
+   * @param duration how long to wait -must be more than 0
+   * @param desiredState desired state.
+   * @return the application report -null on a timeout
+   * @throws YarnException
+   * @throws IOException
+   */
+  public ApplicationReport monitorAppToState(YarnApplicationState desiredState, Duration duration)
+    throws YarnException, IOException {
+    return yarnClient.monitorAppToState(applicationId, desiredState, duration);
+  }
+
+  /**
+   * Kill the submitted application by sending a call to the ASM
+   * @throws YarnException
+   * @throws IOException
+   */
+  public boolean forceKill(String reason)
+    throws YarnException, IOException {
+    if (applicationId != null) {
+      yarnClient.killRunningApplication(applicationId, reason);
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * Kill the application
+   * @return the response
+   * @throws YarnException YARN problems
+   * @throws IOException IO problems
+   */
+  public KillApplicationResponse kill(String reason) throws
+                                                     YarnException,
+                                                     IOException {
+    return yarnClient.killRunningApplication(applicationId, reason);
+  }
+
+  /**
+   * Get the application report of this application
+   * @return an application report
+   * @throws YarnException
+   * @throws IOException
+   */
+  public ApplicationReport getApplicationReport()
+    throws YarnException, IOException {
+    return yarnClient.getApplicationReport(applicationId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/RunningApplication.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/RunningApplication.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/RunningApplication.java
new file mode 100644
index 0000000..14c522c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/RunningApplication.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.launch;
+
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.slider.api.SliderClusterProtocol;
+import org.apache.slider.client.SliderYarnClientImpl;
+import org.apache.slider.common.SliderExitCodes;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.server.appmaster.rpc.RpcBinder;
+
+import java.io.IOException;
+
+import static org.apache.slider.common.Constants.CONNECT_TIMEOUT;
+import static org.apache.slider.common.Constants.RPC_TIMEOUT;
+
+/**
+ * A running application built from an app report. This one
+ * can be talked to
+ */
+public class RunningApplication extends LaunchedApplication {
+
+  private final ApplicationReport applicationReport;
+  public RunningApplication(SliderYarnClientImpl yarnClient,
+                            ApplicationReport applicationReport) {
+    super(yarnClient, applicationReport);
+    this.applicationReport = applicationReport;
+  }
+
+  public ApplicationReport getApplicationReport() {
+    return applicationReport;
+  }
+
+
+  /**
+   * Connect to a Slider AM
+   * @param app application report providing the details on the application
+   * @return an instance
+   * @throws YarnException
+   * @throws IOException
+   */
+  public SliderClusterProtocol connect(ApplicationReport app) throws
+                                                             YarnException,
+                                                             IOException {
+
+    try {
+      return RpcBinder.getProxy(yarnClient.getConfig(),
+                                yarnClient.getRmClient(),
+                                app,
+                                CONNECT_TIMEOUT,
+                                RPC_TIMEOUT);
+    } catch (InterruptedException e) {
+      throw new SliderException(SliderExitCodes.EXIT_TIMED_OUT,
+          e,
+          "Interrupted waiting for communications with the Application Master");
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/SerializedApplicationReport.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/SerializedApplicationReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/SerializedApplicationReport.java
new file mode 100644
index 0000000..8e0ef5a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/SerializedApplicationReport.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.launch;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.slider.core.persist.ApplicationReportSerDeser;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.io.IOException;
+
+/**
+ * Serialized form of an application report which can be persisted
+ * and then parsed. It can not be converted back into a
+ * real YARN application report
+ * 
+ * Useful for testing
+ */
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+
+public class SerializedApplicationReport {
+
+  public String applicationId;
+  public String applicationAttemptId;
+  public String name;
+  public String applicationType;
+  public String user;
+  public String queue;
+  public String host;
+  public Integer rpcPort;
+  public String state;
+  public String diagnostics;
+  public String url;
+  /**
+   * This value is non-null only when a report is generated from a submission context.
+   * The YARN {@link ApplicationReport} structure does not propagate this value
+   * from the RM.
+   */
+  public Long submitTime;
+  public Long startTime;
+  public Long finishTime;
+  public String finalStatus;
+  public String origTrackingUrl;
+  public Float progress;
+  
+  public SerializedApplicationReport() {
+  }
+  
+  public SerializedApplicationReport(ApplicationReport report) {
+    this.applicationId = report.getApplicationId().toString();
+    ApplicationAttemptId attemptId = report.getCurrentApplicationAttemptId();
+    this.applicationAttemptId = attemptId != null ? attemptId.toString() : "N/A";
+    this.name = report.getName();
+    this.applicationType = report.getApplicationType();
+    this.user = report.getUser();
+    this.queue = report.getQueue();
+    this.host = report.getHost();
+    this.rpcPort = report.getRpcPort();
+    this.state = report.getYarnApplicationState().toString();
+    this.diagnostics = report.getDiagnostics();
+    this.startTime = report.getStartTime();
+    this.finishTime = report.getFinishTime();
+    FinalApplicationStatus appStatus = report.getFinalApplicationStatus();
+    this.finalStatus = appStatus == null ? "" : appStatus.toString();
+    this.progress = report.getProgress();
+    this.url = report.getTrackingUrl();
+    this.origTrackingUrl= report.getOriginalTrackingUrl();
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return ApplicationReportSerDeser.toString(this);
+    } catch (IOException e) {
+      return super.toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ExitCodeProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ExitCodeProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ExitCodeProvider.java
new file mode 100644
index 0000000..503b9b9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/ExitCodeProvider.java
@@ -0,0 +1,32 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.core.main;
+
+/**
+ * Get the exit code of an exception. Making it an interface allows
+ * us to retrofit exit codes onto existing classes
+ */
+public interface ExitCodeProvider {
+
+  /**
+   * Method to get the exit code
+   * @return the exit code
+   */
+  int  getExitCode();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/IrqHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/IrqHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/IrqHandler.java
new file mode 100644
index 0000000..42442d1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/IrqHandler.java
@@ -0,0 +1,103 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.core.main;
+
+import sun.misc.Signal;
+import sun.misc.SignalHandler;
+
+import java.io.IOException;
+
+/**
+ * This class bundles up all the compiler warnings about abuse of sun.misc
+ * interrupt handling code
+ * into one place.
+ */
+@SuppressWarnings("UseOfSunClasses")
+public final class IrqHandler implements SignalHandler {
+
+  public static final String CONTROL_C = "INT";
+  public static final String SIGTERM = "TERM";
+
+  private final String name;
+  private final Interrupted handler;
+
+  /**
+   * Create an IRQ handler bound to the specific interrupt
+   * @param name signal name
+   * @param handler handler
+   * @throws IOException
+   */
+  public IrqHandler(String name, Interrupted handler) throws IOException {
+    this.handler = handler;
+    this.name = name;
+    try {
+      Signal.handle(new Signal(name), this);
+    } catch (IllegalArgumentException e) {
+      throw new IOException(
+        "Could not set handler for signal \"" + name + "\"."
+        + "This can happen if the JVM has the -Xrs set.",
+        e);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "IrqHandler for signal " + name ;
+  }
+
+  /**
+   * Handler for the JVM API for signal handling
+   * @param signal signal raised
+   */
+//  @Override
+  public void handle(Signal signal) {
+    InterruptData data = new InterruptData(signal.getName(), signal.getNumber());
+    handler.interrupted(data);
+  }
+
+  /**
+   * Interrupt data to pass on.
+   */
+  public static class InterruptData {
+    public final String name;
+    public final int number;
+
+    public InterruptData(String name, int number) {
+      this.name = name;
+      this.number = number;
+    }
+
+    @Override
+    public String toString() {
+      return "signal " + name + '(' + number + ')';
+    }
+  }
+
+  /**
+   * Callback on interruption
+   */
+  public interface Interrupted {
+
+    /**
+     * Handle an interrupt
+     * @param interruptData data
+     */
+    void interrupted(InterruptData interruptData);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/LauncherExitCodes.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/LauncherExitCodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/LauncherExitCodes.java
new file mode 100644
index 0000000..83e89f0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/main/LauncherExitCodes.java
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.core.main;
+
+/*
+ * Common Exit codes
+ * <p>
+ * Exit codes from 64 up are application specific.
+ * <p>
+ * Many of the exit codes are designed to resemble HTTP error codes,
+ * squashed into a single byte. e.g 44 , "not found" is the equivalent
+ * of 404
+ * <pre>
+ *    0-10: general command issues
+ *   30-39: equivalent to the 3XX responses, where those responses are
+ *          considered errors by the application.
+ *   40-49: request-related errors
+ *   50-59: server-side problems. These may be triggered by the request.
+ *   64-  : application specific error codes
+ * </pre>
+ */
+public interface LauncherExitCodes {
+  
+  /**
+   * 0: success
+   */
+  int EXIT_SUCCESS                    =  0;
+
+  /**
+   * -1: generic "false" response. The operation worked but
+   * the result was not true
+   */
+  int EXIT_FALSE                      = -1;
+
+  /**
+   * Exit code when a client requested service termination: {@value}
+   */
+  int EXIT_CLIENT_INITIATED_SHUTDOWN  =  1;
+
+  /**
+   * Exit code when targets could not be launched: {@value}
+   */
+  int EXIT_TASK_LAUNCH_FAILURE        =  2;
+
+  /**
+   * Exit code when a control-C, kill -3, signal was picked up: {@value}
+   */
+  int EXIT_INTERRUPTED                = 3;
+
+  /**
+   * Exit code when a usage message was printed: {@value}
+   */
+  int EXIT_USAGE                      = 4;
+
+  /**
+   * Exit code when something happened but we can't be specific: {@value}
+   */
+  int EXIT_OTHER_FAILURE               = 5;
+
+  /**
+   * Exit code on connectivity problems: {@value}
+   */
+  int EXIT_MOVED                      = 31;
+  
+  /**
+   * found: {@value}.
+   * <p>
+   * This is low value as in HTTP it is normally a success/redirect;
+   * whereas on the command line 0 is the sole success code.
+   * <p>
+   * <code>302 Found</code>
+   */
+  int EXIT_FOUND                      = 32;
+
+  /**
+   * Exit code on a request where the destination has not changed
+   * and (somehow) the command specified that this is an error.
+   * That is, this exit code is somehow different from a "success"
+   * : {@value}
+   * <p>
+   * <code>304 Not Modified </code>
+  */
+  int EXIT_NOT_MODIFIED               = 34;
+
+  /**
+   * Exit code when the command line doesn't parse: {@value}, or
+   * when it is otherwise invalid.
+   * <p>
+   * <code>400 BAD REQUEST</code>
+   */
+  int EXIT_COMMAND_ARGUMENT_ERROR     = 40;
+
+  /**
+   * The request requires user authentication: {@value}
+   * <p>
+   * <code>401 Unauthorized</code>
+   */
+  int EXIT_UNAUTHORIZED               = 41;
+  
+  /**
+   * Forbidden action: {@value}
+   * <p>
+   * <code>403: Forbidden</code>
+   */
+  int EXIT_FORBIDDEN                  = 43;
+  
+  /**
+   * Something was not found: {@value}
+   * <p>
+   * <code>404: NOT FOUND</code>
+   */
+  int EXIT_NOT_FOUND                  = 44;
+
+  /**
+   * The operation is not allowed: {@value}
+   * <p>
+   * <code>405: NOT ALLOWED</code>
+   */
+  int EXIT_OPERATION_NOT_ALLOWED       = 45;
+
+  /**
+   * The command is somehow not acceptable: {@value}
+   * <p>
+   * <code>406: NOT ACCEPTABLE</code>
+   */
+  int EXIT_NOT_ACCEPTABLE            = 46;
+
+  /**
+   * Exit code on connectivity problems: {@value}
+   * <p>
+   * <code>408: Request Timeout</code>
+   */
+  int EXIT_CONNECTIVITY_PROBLEM       = 48;
+
+  /**
+   * The request could not be completed due to a conflict with the current
+   * state of the resource.  {@value}
+   * <p>
+   * <code>409: conflict</code>
+   */
+  int EXIT_CONFLICT                   = 49;
+
+  /**
+   * internal error: {@value}
+   * <p>
+   * <code>500 Internal Server Error</code>
+   */
+  int EXIT_INTERNAL_ERROR             = 50;
+
+  /**
+   * Unimplemented feature: {@value}
+   * <p>
+   * <code>501: Not Implemented</code>
+   */
+  int EXIT_UNIMPLEMENTED              = 51;
+
+  /**
+   * Service Unavailable; it may be available later: {@value}
+   * <p>
+   * <code>503 Service Unavailable</code>
+   */
+  int EXIT_SERVICE_UNAVAILABLE        = 53;
+
+  /**
+   * The application does not support, or refuses to support this version: {@value}.
+   * If raised, this is expected to be raised server-side and likely due
+   * to client/server version incompatibilities.
+   * <p>
+   * <code> 505: Version Not Supported</code>
+   */
+  int EXIT_UNSUPPORTED_VERSION        = 55;
+
+  /**
+   * Exit code when an exception was thrown from the service: {@value}
+   * <p>
+   * <code>5XX</code>
+   */
+  int EXIT_EXCEPTION_THROWN           = 56;
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[30/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderExitCodes.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderExitCodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderExitCodes.java
new file mode 100644
index 0000000..5758f79
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderExitCodes.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.common;
+
+import org.apache.slider.core.main.LauncherExitCodes;
+
+public interface SliderExitCodes extends LauncherExitCodes {
+
+  /**
+   * starting point for exit codes; not an exception itself
+   */
+  int _EXIT_CODE_BASE =           64;
+
+  /**
+   * service entered the failed state: {@value}
+   */
+  int EXIT_YARN_SERVICE_FAILED =  65;
+
+  /**
+   * service was killed: {@value}
+   */
+  int EXIT_YARN_SERVICE_KILLED =  66;
+
+  /**
+   * timeout on monitoring client: {@value}
+   */
+  int EXIT_TIMED_OUT =            67;
+
+  /**
+   * service finished with an error: {@value}
+   */
+  int EXIT_YARN_SERVICE_FINISHED_WITH_ERROR = 68;
+
+  /**
+   * the application instance is unknown: {@value}
+   */
+  int EXIT_UNKNOWN_INSTANCE =     69;
+
+  /**
+   * the application instance is in the wrong state for that operation: {@value}
+   */
+  int EXIT_BAD_STATE =            70;
+
+  /**
+   * A spawned master process failed 
+   */
+  int EXIT_PROCESS_FAILED =       71;
+
+  /**
+   * The instance failed -too many containers were
+   * failing or some other threshold was reached
+   */
+  int EXIT_DEPLOYMENT_FAILED =    72;
+
+  /**
+   * The application is live -and the requested operation
+   * does not work if the cluster is running
+   */
+  int EXIT_APPLICATION_IN_USE =   73;
+
+  /**
+   * There already is an application instance of that name
+   * when an attempt is made to create a new instance
+   */
+  int EXIT_INSTANCE_EXISTS =      75;
+
+  /**
+   * Exit code when the configurations in valid/incomplete: {@value}
+   */
+  int EXIT_BAD_CONFIGURATION =    77;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
new file mode 100644
index 0000000..ba3effc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
@@ -0,0 +1,278 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.common;
+
+
+/**
+ * Keys and various constants for Slider
+ */
+public interface SliderKeys extends SliderXmlConfKeys {
+
+  /**
+   * This is the name of the slider appmaster in configurations :{@value}
+   */
+  String COMPONENT_AM = "slider-appmaster";
+  
+  /**
+   * Slider role is "special":{@value}
+   */
+  int ROLE_AM_PRIORITY_INDEX = 0;
+  
+  
+  /**
+   * The path under which cluster and temp data are stored
+   * {@value}
+   */
+  String SLIDER_BASE_DIRECTORY = ".slider";
+
+  /**
+   * The paths under which Slider AM dependency libraries are stored
+   */
+  String SLIDER_DEPENDENCY_LOCALIZED_DIR_LINK = "slider_dep";
+  String SLIDER_DEPENDENCY_HDP_PARENT_DIR = "/hdp";
+  String SLIDER_DEPENDENCY_DIR = "/apps/%s/slider";
+  String SLIDER_DEPENDENCY_TAR_GZ_FILE_NAME = "slider";
+  String SLIDER_DEPENDENCY_TAR_GZ_FILE_EXT = ".tar.gz";
+  String SLIDER_DEPENDENCY_DIR_PERMISSIONS = "755";
+
+  /**
+   * 
+   */
+  String HDP_VERSION_PROP_NAME = "HDP_VERSION";
+
+  /**
+   *  name of the relative path to expaned an image into:  {@value}.
+   *  The title of this path is to help people understand it when
+   *  they see it in their error messages
+   */
+  String LOCAL_TARBALL_INSTALL_SUBDIR = "expandedarchive";
+
+
+  /**
+   * Application type for YARN  {@value}
+   */
+  String APP_TYPE = "org-apache-slider";
+
+  /**
+   * Key for application version. This must be set in app_config/global {@value}
+   */
+  String APP_VERSION = "site.global.app_version";
+  String APP_VERSION_UNKNOWN = "awaiting heartbeat...";
+
+  /**
+   * Keys for application container specific properties, like release timeout
+   */
+  String APP_CONTAINER_RELEASE_TIMEOUT = "site.global.app_container.release_timeout_secs";
+  int APP_CONTAINER_HEARTBEAT_INTERVAL_SEC = 10; // look for HEARTBEAT_IDDLE_INTERVAL_SEC
+
+  /**
+   * JVM arg to force IPv4  {@value}
+   */
+  String JVM_ENABLE_ASSERTIONS = "-ea";
+  
+  /**
+   * JVM arg enable JVM system/runtime {@value}
+   */
+  String JVM_ENABLE_SYSTEM_ASSERTIONS = "-esa";
+
+  /**
+   * JVM arg to force IPv4  {@value}
+   */
+  String JVM_FORCE_IPV4 = "-Djava.net.preferIPv4Stack=true";
+
+  /**
+   * JVM arg to go headless  {@value}
+   */
+
+  String JVM_JAVA_HEADLESS = "-Djava.awt.headless=true";
+
+  /**
+   * This is the name of the dir/subdir containing
+   * the hbase conf that is propagated via YARN
+   *  {@value}
+   */
+  String PROPAGATED_CONF_DIR_NAME = "propagatedconf";
+  String INFRA_DIR_NAME = "infra";
+  String GENERATED_CONF_DIR_NAME = "generated";
+  String SNAPSHOT_CONF_DIR_NAME = "snapshot";
+  String DATA_DIR_NAME = "database";
+  String HISTORY_DIR_NAME = "history";
+  String HISTORY_FILENAME_SUFFIX = "json";
+  String HISTORY_FILENAME_PREFIX = "rolehistory-";
+  String KEYTAB_DIR = "keytabs";
+  String RESOURCE_DIR = "resources";
+
+  /**
+   * Filename pattern is required to save in strict temporal order.
+   * Important: older files must sort less-than newer files when using
+   * case-sensitive name sort.
+   */
+  String HISTORY_FILENAME_CREATION_PATTERN = HISTORY_FILENAME_PREFIX +"%016x."+
+                                    HISTORY_FILENAME_SUFFIX;
+  /**
+   * The posix regexp used to locate this 
+   */
+  String HISTORY_FILENAME_MATCH_PATTERN = HISTORY_FILENAME_PREFIX +"[0-9a-f]+\\."+
+                                    HISTORY_FILENAME_SUFFIX;
+    /**
+   * The posix regexp used to locate this 
+   */
+  String HISTORY_FILENAME_GLOB_PATTERN = HISTORY_FILENAME_PREFIX +"*."+
+                                    HISTORY_FILENAME_SUFFIX;
+  /**
+   * XML resource listing the standard Slider providers
+   * {@value}
+   */
+  String SLIDER_XML = "org/apache/slider/slider.xml";
+  
+  String CLUSTER_DIRECTORY = "cluster";
+
+  String PACKAGE_DIRECTORY = "package";
+
+  /**
+   * JVM property to define the slider configuration directory;
+   * this is set by the slider script: {@value}
+   */
+  String PROPERTY_CONF_DIR = "slider.confdir";
+
+  /**
+   * JVM property to define the slider lib directory;
+   * this is set by the slider script: {@value}
+   */
+  String PROPERTY_LIB_DIR = "slider.libdir";
+
+  /**
+   * name of generated dir for this conf: {@value}
+   */
+  String SUBMITTED_CONF_DIR = "confdir";
+
+  /**
+   * Slider AM log4j file name : {@value}
+   */
+  String LOG4J_SERVER_PROP_FILENAME = "log4j-server.properties";
+
+  /**
+   * Standard log4j file name  : {@value}
+   */
+  String LOG4J_PROP_FILENAME = "log4j.properties";
+
+  /**
+   * Log4j sysprop to name the resource :{@value}
+   */
+  String SYSPROP_LOG4J_CONFIGURATION = "log4j.configuration";
+
+  /**
+   * sysprop for Slider AM log4j directory :{@value}
+   */
+  String SYSPROP_LOG_DIR = "LOG_DIR";
+
+  /**
+   * name of the Slider client resource
+   * loaded when the service is loaded.
+   */
+  String SLIDER_CLIENT_XML = "slider-client.xml";
+
+  /**
+   * The name of the resource to put on the classpath
+   */
+  String SLIDER_SERVER_XML = "slider-server.xml";
+
+  String TMP_LOGDIR_PREFIX = "/tmp/slider-";
+  String TMP_DIR_PREFIX = "tmp";
+  String AM_DIR_PREFIX = "appmaster";
+
+  /**
+   * Store the default app definition, e.g. metainfo file or content of a folder
+   */
+  String APP_DEF_DIR = "appdef";
+  /**
+   * Store additional app defs - co-processors
+   */
+  String ADDONS_DIR = "addons";
+
+  String SLIDER_JAR = "slider.jar";
+  String JCOMMANDER_JAR = "jcommander.jar";
+  String GSON_JAR = "gson.jar";
+  String AGENT_TAR = "slider-agent.tar.gz";
+  String DEFAULT_APP_PKG = "appPkg.zip";
+
+  String DEFAULT_JVM_HEAP = "256M";
+  int DEFAULT_YARN_MEMORY = 256;
+  String STDOUT_AM = "slider-out.txt";
+  String STDERR_AM = "slider-err.txt";
+  String DEFAULT_GC_OPTS = "";
+
+  String HADOOP_USER_NAME = "HADOOP_USER_NAME";
+  String HADOOP_PROXY_USER = "HADOOP_PROXY_USER";
+  String SLIDER_PASSPHRASE = "SLIDER_PASSPHRASE";
+
+  boolean PROPAGATE_RESOURCE_OPTION = true;
+
+  /**
+   * Security associated keys.
+   */
+  String SECURITY_DIR = "security";
+  String CRT_FILE_NAME = "ca.crt";
+  String CSR_FILE_NAME = "ca.csr";
+  String KEY_FILE_NAME = "ca.key";
+  String KEYSTORE_FILE_NAME = "keystore.p12";
+  String CRT_PASS_FILE_NAME = "pass.txt";
+  String PASS_LEN = "50";
+
+  String COMP_STORES_REQUIRED_KEY =
+      "slider.component.security.stores.required";
+  String COMP_KEYSTORE_PASSWORD_PROPERTY_KEY =
+      "slider.component.keystore.password.property";
+  String COMP_KEYSTORE_PASSWORD_ALIAS_KEY =
+      "slider.component.keystore.credential.alias.property";
+  String COMP_KEYSTORE_PASSWORD_ALIAS_DEFAULT =
+      "component.keystore.credential.alias";
+  String COMP_TRUSTSTORE_PASSWORD_PROPERTY_KEY =
+      "slider.component.truststore.password.property";
+  String COMP_TRUSTSTORE_PASSWORD_ALIAS_KEY =
+      "slider.component.truststore.credential.alias.property";
+  String COMP_TRUSTSTORE_PASSWORD_ALIAS_DEFAULT =
+      "component.truststore.credential.alias";
+
+  /**
+   * Python specific
+   */
+  String PYTHONPATH = "PYTHONPATH";
+
+
+  /**
+   * Name of the AM filter to use: {@value}
+   */
+  String AM_FILTER_NAME =
+      "org.apache.hadoop.yarn.server.webproxy.amfilter.AmFilterInitializer";
+
+  /**
+   * Allowed port range. This MUST be set in app_conf/global.
+   * {@value}
+   */
+  String KEY_ALLOWED_PORT_RANGE = "site.global.slider.allowed.ports";
+
+  /**
+   * env var for custom JVM options.
+   */
+  String SLIDER_JVM_OPTS = "SLIDER_JVM_OPTS";
+
+  String SLIDER_CLASSPATH_EXTRA = "SLIDER_CLASSPATH_EXTRA";
+  String YARN_CONTAINER_PATH = "/node/container/";
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderXMLConfKeysForTesting.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderXMLConfKeysForTesting.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderXMLConfKeysForTesting.java
new file mode 100644
index 0000000..61c828e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderXMLConfKeysForTesting.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common;
+
+/**
+ * Keys shared across tests
+ */
+public interface SliderXMLConfKeysForTesting {
+  
+  String KEY_TEST_THAW_WAIT_TIME = "slider.test.thaw.wait.seconds";
+
+  int DEFAULT_THAW_WAIT_TIME_SECONDS = 60;
+
+
+  String KEY_TEST_FREEZE_WAIT_TIME = "slider.test.freeze.wait.seconds";
+
+  int DEFAULT_TEST_FREEZE_WAIT_TIME_SECONDS = 60;
+
+  String KEY_TEST_TIMEOUT = "slider.test.timeout.seconds";
+
+  int DEFAULT_TEST_TIMEOUT_SECONDS = 30 * 60;
+
+  String KEY_ACCUMULO_LAUNCH_TIME =
+    "slider.test.accumulo.launch.wait.seconds";
+  int DEFAULT_ACCUMULO_LAUNCH_TIME_SECONDS = 60 * 3;
+
+  String KEY_ACCUMULO_GO_LIVE_TIME =
+      "slider.test.accumulo.live.wait.seconds";
+  int DEFAULT_ACCUMULO_LIVE_TIME_SECONDS = 90;
+
+  String KEY_TEST_AGENT_ENABLED = "slider.test.agent.enabled";
+  String KEY_AGENTTESTS_QUEUE_LABELED_DEFINED = "slider.test.agent.labeled.queue.enabled";
+  String KEY_AGENTTESTS_LABELS_RED_BLUE_DEFINED = "slider.test.agent.labels.defined";
+  String KEY_AGENTTESTS_AM_FAILURES_ENABLED = "slider.test.agent.am.failures.enabled";
+
+  int DEFAULT_AGENT_LAUNCH_TIME_SECONDS = 60 * 3;
+
+  String KEY_TEST_AGENT_HOME = "slider.test.agent.home";
+  String KEY_TEST_AGENT_TAR = "slider.test.agent.tar";
+
+  String KEY_TEST_TEARDOWN_KILLALL = "slider.test.teardown.killall";
+  boolean DEFAULT_TEARDOWN_KILLALL = true;
+
+
+  /**
+   * Key for amount of RAM to request
+   */
+  String KEY_TEST_YARN_RAM_REQUEST = "slider.test.yarn.ram";
+  String DEFAULT_YARN_RAM_REQUEST = "192";
+
+  /**
+   * security related keys
+   */
+  String TEST_SECURITY_DIR = "/tmp/work/security";
+
+  /**
+   * Local path to AM keytab: {@value}
+   */
+  String KEY_TEST_AM_KEYTAB = "slider.test.am.keytab.local";
+
+  /**
+   * Is the test cluster windows? Default is: same as the local system.
+   *  {@value}
+   */
+  String KEY_TEST_WINDOWS_CLUSTER = "slider.test.windows.cluster";
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java
new file mode 100644
index 0000000..72dd44f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common;
+
+import org.apache.hadoop.registry.client.api.RegistryConstants;
+
+/**
+ * These are the keys that can be added to <code>conf/slider-client.xml</code>.
+ */
+public interface SliderXmlConfKeys {
+  String PREFIX_PROVIDER = "slider.provider";
+  /**
+   * pattern to identify a provider
+   * {@value}
+   */
+  String KEY_PROVIDER = PREFIX_PROVIDER + ".%s";
+
+  /**
+   * conf option set to point to where the config came from
+   * {@value}
+   */
+  String KEY_TEMPLATE_ORIGIN = "slider.template.origin";
+
+  /**
+   * Original name for the default FS. This is still 
+   * expected by applications deployed
+   */
+  String FS_DEFAULT_NAME_CLASSIC = "fs.default.name";
+
+  /**
+   * Slider principal
+   */
+  String KEY_KERBEROS_PRINCIPAL = "slider.kerberos.principal";
+
+  /**
+   * Name of the property for ACLs for Slider AM.
+   * {@value}
+   */
+  String KEY_PROTOCOL_ACL = "slider.security.protocol.acl";
+
+  /**
+   * Limit on restarts for the AM
+   * {@value}
+   */
+  String KEY_AM_RESTART_LIMIT = "slider.yarn.restart.limit";
+
+  /**
+   * queue name, by default let YARN pick the queue
+   */
+  String KEY_YARN_QUEUE = "slider.yarn.queue";
+  String DEFAULT_YARN_QUEUE = null;
+
+  /**
+   * default priority
+   */
+  String KEY_YARN_QUEUE_PRIORITY = "slider.yarn.queue.priority";
+  int DEFAULT_YARN_QUEUE_PRIORITY = 1;
+
+
+  /**
+   * The slider base path: {@value}
+   * Defaults to HomeDir/.slider
+   */
+  String KEY_SLIDER_BASE_PATH = "slider.base.path";
+
+
+  /**
+   * Option for the permissions for the cluster directory itself: {@value}
+   */
+  String CLUSTER_DIRECTORY_PERMISSIONS =
+    "slider.cluster.directory.permissions";
+
+  /**
+   * Default value for the permissions :{@value}
+   */
+  String DEFAULT_CLUSTER_DIRECTORY_PERMISSIONS = "750";
+
+  /**
+   * 
+   * Option for the permissions for the data directory itself: {@value}
+   */
+  String DATA_DIRECTORY_PERMISSIONS = "slider.data.directory.permissions";
+
+  /**
+   * Default value for the data directory permissions: {@value}
+   */
+  String DEFAULT_DATA_DIRECTORY_PERMISSIONS = "750";
+
+  /**
+   *
+   * Use {@link RegistryConstants#KEY_REGISTRY_ZK_ROOT}
+   *
+   */
+  @Deprecated
+  String REGISTRY_PATH = "slider.registry.path";
+
+  /**
+   * 
+   * @Deprecated use {@link RegistryConstants#KEY_REGISTRY_ZK_QUORUM}
+   * 
+   */
+  @Deprecated
+  String REGISTRY_ZK_QUORUM = "slider.zookeeper.quorum";
+
+
+  String IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH =
+      "ipc.client.fallback-to-simple-auth-allowed";
+  String HADOOP_HTTP_FILTER_INITIALIZERS =
+      "hadoop.http.filter.initializers";
+  String KEY_KEYSTORE_LOCATION = "ssl.server.keystore.location";
+  String KEY_AM_LOGIN_KEYTAB_NAME = "slider.am.login.keytab.name";
+  /** Declare that a keytab must be provided */
+  String KEY_AM_LOGIN_KEYTAB_REQUIRED = "slider.am.login.keytab.required";
+  String KEY_HDFS_KEYTAB_DIR = "slider.hdfs.keytab.dir";
+  String KEY_AM_KEYTAB_LOCAL_PATH = "slider.am.keytab.local.path";
+  String KEY_KEYTAB_PRINCIPAL = "slider.keytab.principal.name";
+  String KEY_SECURITY_ENABLED = "site.global.security_enabled";
+
+  /**
+   * Set to disable server-side checks for python, openssl &c.
+   * This should only be set for testing
+   */
+  String KEY_SLIDER_AM_DEPENDENCY_CHECKS_DISABLED =
+      "slider.am.dependency.checks.disabled";
+
+  /**
+   * The path to the python executable utilized to launch the agent.
+   */
+  String PYTHON_EXECUTABLE_PATH = "agent.python.exec.path";
+
+  /**
+   * Flag to enable the insecure AM filter: {@value}
+   */
+  String X_DEV_INSECURE_WS = "slider.feature.ws.insecure";
+
+  /**
+   * Flag to indicate the insecure AM filter is enabled by default: {@value}.
+   */
+  boolean X_DEV_INSECURE_DEFAULT = false;
+
+
+  /**
+   * Flag to indicate the insecure AM filter is required for
+   * complex REST Verbs: {@value}.
+   * When Slider switches to being Hadoop 2.7+ only, this flag
+   * can be set to false
+   */
+  boolean X_DEV_INSECURE_REQUIRED = true;
+
+  /**
+   *
+   */
+  String KEY_IPC_CLIENT_RETRY_POLICY_ENABLED =
+      "slider.ipc.client.retry.enabled";
+  boolean IPC_CLIENT_RETRY_POLICY_ENABLED_DEFAULT = true;
+  String KEY_IPC_CLIENT_RETRY_POLICY_SPEC =
+      "slider.ipc.client.retry.policy.spec";
+  String IPC_CLIENT_RETRY_POLICY_SPEC_DEFAULT =
+      "10000,6,60000,10"; //t1,n1,t2,n2,... 
+
+  String KEY_AM_LAUNCH_ENV = "slider.am.launch.env";
+
+  /**
+   * From {@code DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY}
+   */
+  String DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY = "dfs.namenode.kerberos.principal";
+
+  String DFS_DATANODE_KERBEROS_PRINCIPAL_KEY = "dfs.datanode.kerberos.principal";
+
+  //Delegation token related keys
+  String DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_KEY
+      = "dfs.namenode.delegation.key.update-interval";
+  long DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT = 24 * 60 * 60 *
+      1000; // 1 day
+  String DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_KEY
+      = "dfs.namenode.delegation.token.renew-interval";
+  long DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_DEFAULT = 24 * 60 * 60 *
+      1000;  // 1 day
+  String DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY
+      = "dfs.namenode.delegation.token.max-lifetime";
+  long DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_DEFAULT = 7 * 24 * 60 * 60 *
+      1000; // 7 days
+  String DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY
+      = "dfs.namenode.delegation.token.always-use"; // for tests
+  boolean DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_DEFAULT = false;
+  String DFS_NAMENODE_KEYTAB_FILE_KEY = "dfs.namenode.keytab.file";
+  String DFS_NAMENODE_DU_RESERVED_KEY = "dfs.namenode.resource.du.reserved";
+
+  String MAPREDUCE_JOB_CREDENTIALS_BINARY = "mapreduce.job.credentials.binary";
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractActionArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractActionArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractActionArgs.java
new file mode 100644
index 0000000..e3cb288
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractActionArgs.java
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import org.apache.hadoop.fs.Path;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.ErrorStrings;
+import org.apache.slider.core.exceptions.UsageException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Base args for all actions
+ */
+public abstract class AbstractActionArgs extends ArgOps implements Arguments {
+  protected static final Logger log =
+    LoggerFactory.getLogger(AbstractActionArgs.class);
+
+
+  protected AbstractActionArgs() {
+  }
+
+  /**
+   * URI/binding to the filesystem
+   */
+  @Parameter(names = {ARG_FILESYSTEM, ARG_FILESYSTEM_LONG},
+             description = "Filesystem Binding")
+  public String filesystemBinding;
+
+  @Parameter(names = {ARG_BASE_PATH},
+             description = "Slider base path on the filesystem",
+             converter =  PathArgumentConverter.class)
+  public Path basePath;
+
+  /**
+   * This is the default parameter
+   */
+  @Parameter
+  public final List<String> parameters = new ArrayList<>();
+
+  /**
+   * get the name: relies on arg 1 being the cluster name in all operations 
+   * @return the name argument, null if there is none
+   */
+  public String getClusterName() {
+    return (parameters.isEmpty()) ? null : parameters.get(0);
+  }
+
+  /**
+   -D name=value
+
+   Define an HBase configuration option which overrides any options in
+   the configuration XML files of the image or in the image configuration
+   directory. The values will be persisted.
+   Configuration options are only passed to the cluster when creating or reconfiguring a cluster.
+
+   */
+
+  @Parameter(names = ARG_DEFINE, arity = 1, description = "Definitions")
+  public final List<String> definitions = new ArrayList<>();
+
+  /**
+   * System properties
+   */
+  @Parameter(names = {ARG_SYSPROP}, arity = 1,
+             description = "system properties in the form name value" +
+                           " These are set after the JVM is started.")
+  public final List<String> sysprops = new ArrayList<>(0);
+
+
+  @Parameter(names = {ARG_MANAGER_SHORT, ARG_MANAGER},
+             description = "Binding (usually hostname:port) of the YARN resource manager")
+  public String manager;
+
+
+  @Parameter(names = ARG_DEBUG, description = "Debug mode")
+  public boolean debug = false;
+
+  @Parameter(names = {ARG_HELP}, description = "Help", help = true)
+  public boolean help = false;
+
+  /**
+   * Get the min #of params expected
+   * @return the min number of params in the {@link #parameters} field
+   */
+  public int getMinParams() {
+    return 1;
+  }
+
+  /**
+   * Get the name of the action
+   * @return the action name
+   */
+  public abstract String getActionName() ;
+
+  /**
+   * Get the max #of params expected
+   * @return the number of params in the {@link #parameters} field;
+   */
+  public int getMaxParams() {
+    return getMinParams();
+  }
+
+  public void validate() throws BadCommandArgumentsException, UsageException {
+    
+    int minArgs = getMinParams();
+    int actionArgSize = parameters.size();
+    if (minArgs > actionArgSize) {
+      throw new BadCommandArgumentsException(
+        ErrorStrings.ERROR_NOT_ENOUGH_ARGUMENTS + getActionName() +
+        " Expected minimum " + minArgs + " but got " + actionArgSize);
+    }
+    int maxArgs = getMaxParams();
+    if (maxArgs == -1) {
+      maxArgs = minArgs;
+    }
+    if (actionArgSize > maxArgs) {
+      String message = String.format("%s for action %s: limit is %d but saw %d: ",
+                                     ErrorStrings.ERROR_TOO_MANY_ARGUMENTS,
+                                     getActionName(), maxArgs,
+                                     actionArgSize);
+      
+      log.error(message);
+      int index = 1;
+      for (String actionArg : parameters) {
+        log.error("[{}] \"{}\"", index++, actionArg);
+        message += " \"" + actionArg + "\" ";
+      }
+      throw new BadCommandArgumentsException(message);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return super.toString() + ": " + getActionName();
+  }
+
+  /**
+   * Override point: 
+   * Flag to indicate that core hadoop API services are needed (HDFS, YARN, etc)
+   * \u2014and that validation of the client state should take place.
+   * 
+   * @return a flag to indicate that the core hadoop services will be needed.
+   */
+  public boolean getHadoopServicesRequired() {
+    return true;
+  }
+
+  /**
+   * Flag to disable secure login.
+   * This MUST only be set if the action is bypassing security or setting
+   * it itself
+   * @return true if login at slider client init time is to be skipped
+   */
+  public boolean disableSecureLogin() {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractArgsDelegate.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractArgsDelegate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractArgsDelegate.java
new file mode 100644
index 0000000..23ba414
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractArgsDelegate.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+
+/**
+ * Base class for all the delegates
+ */
+public class AbstractArgsDelegate extends ArgOps implements Arguments {
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractClusterBuildingActionArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractClusterBuildingActionArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractClusterBuildingActionArgs.java
new file mode 100644
index 0000000..2a5eedc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractClusterBuildingActionArgs.java
@@ -0,0 +1,217 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParametersDelegate;
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.fs.Path;
+import org.apache.slider.core.conf.ConfTree;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.providers.SliderProviderFactory;
+
+import java.io.File;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Abstract Action to build things; shares args across build and
+ * list
+ */
+public abstract class AbstractClusterBuildingActionArgs extends
+    AbstractActionArgs {
+
+  /**
+   * Declare the image configuration directory to use when creating or
+   * reconfiguring a slider cluster. The path must be on a filesystem visible
+   * to all nodes in the YARN cluster. Only one configuration directory can
+   * be specified.
+   */
+  @Parameter(names = ARG_CONFDIR,
+      description = "Path to cluster configuration directory in HDFS",
+      converter = PathArgumentConverter.class)
+  public Path confdir;
+
+  @Parameter(names = ARG_ZKPATH,
+      description = "Zookeeper path for the application")
+  public String appZKPath;
+
+  @Parameter(names = ARG_ZKHOSTS,
+      description = "comma separated list of the Zookeeper hosts")
+  public String zkhosts;
+
+  /**
+   * --image path
+   * the full path to a .tar or .tar.gz path containing an HBase image.
+   */
+  @Parameter(names = ARG_IMAGE,
+      description = "The full path to a .tar or .tar.gz path containing the application",
+      converter = PathArgumentConverter.class)
+  public Path image;
+
+  @Parameter(names = ARG_APP_HOME,
+      description = "Home directory of a pre-installed application")
+  public String appHomeDir;
+
+  @Parameter(names = ARG_PROVIDER,
+      description = "Provider of the specific cluster application")
+  public String provider = SliderProviderFactory.DEFAULT_CLUSTER_TYPE;
+
+  @Parameter(names = {ARG_PACKAGE},
+      description = "URI to a slider package")
+  public String packageURI;
+
+  @Parameter(names = {ARG_RESOURCES},
+      description = "File defining the resources of this instance")
+  public File resources;
+
+  @Parameter(names = {ARG_TEMPLATE},
+      description = "Template application configuration")
+  public File template;
+
+  @Parameter(names = {ARG_METAINFO},
+      description = "Application meta info file")
+  public File appMetaInfo;
+
+  @Parameter(names = {ARG_METAINFO_JSON},
+      description = "Application meta info JSON blob")
+  public String appMetaInfoJson;
+
+  @Parameter(names = {ARG_APPDEF},
+      description = "Application def (folder or a zip package)")
+  public File appDef;
+
+  @Parameter(names = {ARG_QUEUE},
+             description = "Queue to submit the application")
+  public String queue;
+
+  @ParametersDelegate
+  public ComponentArgsDelegate componentDelegate = new ComponentArgsDelegate();
+
+  @ParametersDelegate
+  public AddonArgsDelegate addonDelegate = new AddonArgsDelegate();
+
+
+  @ParametersDelegate
+  public AppAndResouceOptionArgsDelegate optionsDelegate =
+      new AppAndResouceOptionArgsDelegate();
+
+
+  public Map<String, String> getOptionsMap() throws
+      BadCommandArgumentsException {
+    return optionsDelegate.getOptionsMap();
+  }
+
+  /**
+   * Get the role heap mapping (may be empty, but never null)
+   * @return role heap mapping
+   * @throws BadCommandArgumentsException parse problem
+   */
+  public Map<String, Map<String, String>> getCompOptionMap() throws
+      BadCommandArgumentsException {
+    return optionsDelegate.getCompOptionMap();
+  }
+
+
+  public Map<String, String> getResourceOptionsMap() throws
+      BadCommandArgumentsException {
+    return optionsDelegate.getResourceOptionsMap();
+  }
+
+  /**
+   * Get the role heap mapping (may be empty, but never null)
+   * @return role heap mapping
+   * @throws BadCommandArgumentsException parse problem
+   */
+  public Map<String, Map<String, String>> getResourceCompOptionMap() throws
+      BadCommandArgumentsException {
+    return optionsDelegate.getResourceCompOptionMap();
+  }
+
+  @VisibleForTesting
+  public List<String> getComponentTuples() {
+    return componentDelegate.getComponentTuples();
+  }
+
+  /**
+   * Get the role mapping (may be empty, but never null)
+   * @return role mapping
+   * @throws BadCommandArgumentsException parse problem
+   */
+  public Map<String, String> getComponentMap() throws
+      BadCommandArgumentsException {
+    return componentDelegate.getComponentMap();
+  }
+
+  @VisibleForTesting
+  public List<String> getAddonTuples() {
+    return addonDelegate.getAddonTuples();
+  }
+
+  /**
+   * Get the list of addons (may be empty, but never null)
+   */
+  public Map<String, String> getAddonMap() throws
+      BadCommandArgumentsException {
+    return addonDelegate.getAddonMap();
+  }
+
+  public Path getConfdir() {
+    return confdir;
+  }
+
+  public String getAppZKPath() {
+    return appZKPath;
+  }
+
+  public String getZKhosts() {
+    return zkhosts;
+  }
+
+  public Path getImage() {
+    return image;
+  }
+
+  public String getAppHomeDir() {
+    return appHomeDir;
+  }
+
+  public String getProvider() {
+    return provider;
+  }
+
+  public ConfTree buildAppOptionsConfTree() throws
+      BadCommandArgumentsException {
+    return buildConfTree(getOptionsMap());
+  }
+
+  public ConfTree buildResourceOptionsConfTree() throws
+      BadCommandArgumentsException {
+    return buildConfTree(getResourceOptionsMap());
+  }
+
+  protected ConfTree buildConfTree(Map<String, String> optionsMap) throws
+      BadCommandArgumentsException {
+    ConfTree confTree = new ConfTree();
+    ConfTreeOperations ops = new ConfTreeOperations(confTree);
+    confTree.global.putAll(optionsMap);
+    return confTree;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionAMSuicideArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionAMSuicideArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionAMSuicideArgs.java
new file mode 100644
index 0000000..5b4cfdc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionAMSuicideArgs.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+
+@Parameters(commandNames = {SliderActions.ACTION_AM_SUICIDE},
+            commandDescription = SliderActions.DESCRIBE_ACTION_AM_SUICIDE)
+public class ActionAMSuicideArgs extends AbstractActionArgs {
+  
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_AM_SUICIDE;
+  }
+  
+  @Parameter(names = {ARG_MESSAGE},
+             description = "reason for the action")
+  public String message = "";
+  
+  @Parameter(names = {ARG_EXITCODE},
+             description = "exit code")
+  public int exitcode = 1;
+
+  @Parameter(names = {ARG_WAIT},
+             description = "time for AM to wait before exiting")
+  public int waittime = 1000;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionBuildArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionBuildArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionBuildArgs.java
new file mode 100644
index 0000000..1a182d1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionBuildArgs.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameters;
+
+@Parameters(commandNames = {SliderActions.ACTION_BUILD},
+            commandDescription = SliderActions.DESCRIBE_ACTION_BUILD)
+
+public class ActionBuildArgs extends AbstractClusterBuildingActionArgs {
+
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_BUILD;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionClientArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionClientArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionClientArgs.java
new file mode 100644
index 0000000..85d39ea
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionClientArgs.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+
+import java.io.File;
+
+@Parameters(commandNames = {SliderActions.ACTION_CLIENT},
+    commandDescription = SliderActions.DESCRIBE_ACTION_CLIENT)
+
+public class ActionClientArgs extends AbstractActionArgs {
+
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_CLIENT;
+  }
+
+  @Parameter(names = {ARG_INSTALL},
+      description = "Install client")
+  public boolean install;
+
+  @Parameter(names = {ARG_GETCERTSTORE},
+      description = "Get a certificate store")
+  public boolean getCertStore;
+
+  @Parameter(names = {ARG_KEYSTORE},
+      description = "Retrieve keystore to specified location")
+  public File keystore;
+
+  @Parameter(names = {ARG_TRUSTSTORE},
+      description = "Retrieve truststore to specified location")
+  public File truststore;
+
+  @Parameter(names = {ARG_HOSTNAME},
+      description = "(Optional) Specify the hostname to use for generation of keystore certificate")
+  public String hostname;
+
+  @Parameter(names = {ARG_NAME},
+      description = "The name of the application")
+  public String name;
+
+  @Parameter(names = {ARG_PROVIDER},
+      description = "The credential provider in which the password is stored")
+  public String provider;
+
+  @Parameter(names = {ARG_ALIAS},
+      description = "The credential provider alias associated with the password")
+  public String alias;
+
+  @Parameter(names = {ARG_PASSWORD},
+      description = "The certificate store password (alternative to " +
+          "provider/alias; if password is specified, those will be ignored)")
+  public String password;
+
+  @Parameter(names = {ARG_PACKAGE},
+      description = "Path to app package")
+  public String packageURI;
+
+  @Parameter(names = {ARG_DEST},
+      description = "The location where to install the client")
+  public File installLocation;
+
+  @Parameter(names = {ARG_CONFIG},
+      description = "Client configuration")
+  public File clientConfig;
+
+  /**
+   * Get the min #of params expected
+   *
+   * @return the min number of params in the {@link #parameters} field
+   */
+  public int getMinParams() {
+    return 0;
+  }
+
+  @Override
+  public int getMaxParams() {
+    return 1;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionCreateArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionCreateArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionCreateArgs.java
new file mode 100644
index 0000000..e70f30a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionCreateArgs.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameters;
+import com.beust.jcommander.ParametersDelegate;
+
+import java.io.File;
+
+@Parameters(commandNames = {SliderActions.ACTION_CREATE},
+            commandDescription = SliderActions.DESCRIBE_ACTION_CREATE)
+
+public class ActionCreateArgs extends AbstractClusterBuildingActionArgs
+  implements WaitTimeAccessor, LaunchArgsAccessor {
+  
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_CREATE;
+  }
+  
+  @ParametersDelegate
+  LaunchArgsDelegate launchArgs = new LaunchArgsDelegate();
+
+  @Override
+  public File getOutputFile() {
+    return launchArgs.getOutputFile();
+  }
+
+  @Override
+  public String getRmAddress() {
+    return launchArgs.getRmAddress();
+  }
+
+  @Override
+  public int getWaittime() {
+    return launchArgs.getWaittime();
+  }
+
+  @Override
+  public void setWaittime(int waittime) {
+    launchArgs.setWaittime(waittime);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionDependencyArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionDependencyArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionDependencyArgs.java
new file mode 100644
index 0000000..87f9f0dc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionDependencyArgs.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.common.params;
+
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.UsageException;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+
+@Parameters(commandNames = { SliderActions.ACTION_DEPENDENCY }, 
+            commandDescription = SliderActions.DESCRIBE_ACTION_DEPENDENCY)
+public class ActionDependencyArgs extends AbstractActionArgs {
+
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_DEPENDENCY;
+  }
+
+  @Parameter(names = { ARG_UPLOAD }, 
+             description = "Upload AM and agent libraries to HDFS for this client")
+  public boolean upload;
+
+  @Parameter(names = { ARG_OVERWRITE },
+             description = "Overwrite current uploaded dependency libs")
+  public boolean overwrite = false;
+
+  /**
+   * Get the min #of params expected
+   * 
+   * @return the min number of params in the {@link #parameters} field
+   */
+  public int getMinParams() {
+    return 0;
+  }
+
+  @Override
+  public int getMaxParams() {
+    return 1;
+  }
+
+  @Override
+  public void validate() throws BadCommandArgumentsException, UsageException {
+    super.validate();
+
+    if (!upload) {
+      throw new UsageException("Option " + ARG_UPLOAD + " is mandatory");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionDestroyArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionDestroyArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionDestroyArgs.java
new file mode 100644
index 0000000..4a129ab
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionDestroyArgs.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+
+@Parameters(commandNames = {SliderActions.ACTION_DESTROY},
+            commandDescription = SliderActions.DESCRIBE_ACTION_DESTROY)
+
+public class ActionDestroyArgs extends AbstractActionArgs {
+
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_DESTROY;
+  }
+
+  @Parameter(names = {ARG_FORCE},
+             description = "force the operation")
+  public boolean force;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionDiagnosticArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionDiagnosticArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionDiagnosticArgs.java
new file mode 100644
index 0000000..c891873
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionDiagnosticArgs.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+
+@Parameters(
+  commandNames = {SliderActions.ACTION_DIAGNOSTICS},
+  commandDescription = SliderActions.DESCRIBE_ACTION_DIAGNOSTIC)
+public class ActionDiagnosticArgs extends AbstractActionArgs {
+
+    @Override
+    public String getActionName() {
+      return SliderActions.ACTION_DIAGNOSTICS;
+    }
+
+    @Parameter(names = {ARG_NAME}, 
+        description = "the name of the running application")
+    public String name;
+
+	  @Parameter(names = {ARG_CLIENT}, 
+	      description = "print configuration of the slider client")
+	  public boolean client = false;
+
+	  @Parameter(names = {ARG_APPLICATION}, 
+	      description = "print configuration of the running application")
+	  public boolean application;
+
+	  @Parameter(names = {ARG_VERBOSE}, 
+	      description = "print out information in details")
+	  public boolean verbose = false;
+
+	  @Parameter(names = {ARG_YARN}, 
+	      description = "print configuration of the YARN cluster")
+	  public boolean yarn = false;
+
+	  @Parameter(names = {ARG_CREDENTIALS}, 
+	      description = "print credentials of the current user")
+	  public boolean credentials = false;
+
+	  @Parameter(names = {ARG_ALL}, 
+	      description = "print all of the information above")
+	  public boolean all;
+
+	  @Parameter(names = {ARG_LEVEL}, 
+	      description = "diagnose each slider configuration one by one")
+	  public boolean level;
+
+	  /**
+	   * Get the min #of params expected
+	   * @return the min number of params in the {@link #parameters} field
+	   */
+	  @Override
+	  public int getMinParams() {
+	    return 0;
+	  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionEchoArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionEchoArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionEchoArgs.java
new file mode 100644
index 0000000..d05f10b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionEchoArgs.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+
+public class ActionEchoArgs extends AbstractActionArgs {
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_ECHO;
+  }
+
+  @Parameter(names = {ARG_MESSAGE},
+             description = "message to echo")
+  public String message;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionExistsArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionExistsArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionExistsArgs.java
new file mode 100644
index 0000000..dd1c04b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionExistsArgs.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+
+import java.io.File;
+
+@Parameters(commandNames = {SliderActions.ACTION_EXISTS},
+            commandDescription = SliderActions.DESCRIBE_ACTION_EXISTS)
+
+public class ActionExistsArgs extends AbstractActionArgs {
+
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_EXISTS;
+  }
+
+  @Parameter(names = {ARG_LIVE},
+             description = "verify that the application is running")
+  public boolean live;
+  
+  @Parameter(names = {ARG_STATE},
+             description = "verify that the application is in the specific YARN state")
+  public String state = "";
+
+  @Parameter(names = {ARG_OUTPUT, ARG_OUTPUT_SHORT},
+      description = "output file for any application report")
+  public File out;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionFlexArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionFlexArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionFlexArgs.java
new file mode 100644
index 0000000..725973e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionFlexArgs.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameters;
+import com.beust.jcommander.ParametersDelegate;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+
+import java.util.List;
+import java.util.Map;
+
+@Parameters(commandNames = {SliderActions.ACTION_FLEX},
+            commandDescription = SliderActions.DESCRIBE_ACTION_FLEX)
+
+public class ActionFlexArgs extends AbstractActionArgs {
+
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_FLEX;
+  }
+  
+  @ParametersDelegate
+  public ComponentArgsDelegate componentDelegate = new ComponentArgsDelegate();
+
+  /**
+   * Get the component mapping (may be empty, but never null)
+   * @return mapping
+   * @throws BadCommandArgumentsException parse problem
+   */
+  public Map<String, String> getComponentMap() throws BadCommandArgumentsException {
+    return componentDelegate.getComponentMap();
+  }
+
+  public List<String> getComponentTuples() {
+    return componentDelegate.getComponentTuples();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionFreezeArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionFreezeArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionFreezeArgs.java
new file mode 100644
index 0000000..e3085d9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionFreezeArgs.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.beust.jcommander.ParametersDelegate;
+
+@Parameters(commandNames = {SliderActions.ACTION_FREEZE},
+            commandDescription = SliderActions.DESCRIBE_ACTION_FREEZE)
+
+public class ActionFreezeArgs extends AbstractActionArgs implements
+                                                         WaitTimeAccessor {
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_FREEZE;
+  }
+  
+  public static final String FREEZE_COMMAND_ISSUED = "stop command issued";
+  @ParametersDelegate
+  public WaitArgsDelegate waitDelegate = new WaitArgsDelegate();
+
+  @Override
+  public int getWaittime() {
+    return waitDelegate.getWaittime();
+  }
+
+  @Override
+  public void setWaittime(int waittime) {
+    waitDelegate.setWaittime(waittime);
+  }
+
+  @Parameter(names={ARG_MESSAGE},
+             description = "reason for the operation")
+  public String message = FREEZE_COMMAND_ISSUED;
+
+  @Parameter(names = {ARG_FORCE},
+             description = "force the operation")
+  public boolean force;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionHelpArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionHelpArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionHelpArgs.java
new file mode 100644
index 0000000..62773c4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionHelpArgs.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameters;
+
+/**
+ * The Help command
+ */
+@Parameters(commandNames = {SliderActions.ACTION_HELP},
+            commandDescription = SliderActions.DESCRIBE_ACTION_HELP)
+public class ActionHelpArgs extends AbstractActionArgs {
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_HELP;
+  }
+
+  /**
+   * Get the min #of params expected
+   * @return the min number of params in the {@link #parameters} field
+   */
+  @Override
+  public int getMinParams() {
+    return 0;
+  }
+
+  /**
+   * This action does not need hadoop services
+   * @return false
+   */
+  @Override
+  public boolean getHadoopServicesRequired() {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionInstallKeytabArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionInstallKeytabArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionInstallKeytabArgs.java
new file mode 100644
index 0000000..4cfb889
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionInstallKeytabArgs.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+
+@Parameters(commandNames = {SliderActions.ACTION_INSTALL_KEYTAB},
+            commandDescription = SliderActions.DESCRIBE_ACTION_INSTALL_KEYTAB)
+
+public class ActionInstallKeytabArgs extends AbstractActionArgs {
+  
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_INSTALL_KEYTAB;
+  }
+
+  @Parameter(names = {ARG_KEYTAB},
+             description = "Path to keytab on local disk")
+  public String keytabUri;
+
+  @Parameter(names = {ARG_FOLDER},
+             description = "The name of the folder in which to store the keytab")
+  public String folder;
+
+  @Parameter(names = {ARG_OVERWRITE}, description = "Overwrite existing keytab")
+  public boolean overwrite = false;
+
+  /**
+   * Get the min #of params expected
+   * @return the min number of params in the {@link #parameters} field
+   */
+  public int getMinParams() {
+    return 0;
+  }
+
+  @Override
+  public int getMaxParams() {
+    return 3;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionInstallPackageArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionInstallPackageArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionInstallPackageArgs.java
new file mode 100644
index 0000000..646e795
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionInstallPackageArgs.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.beust.jcommander.ParametersDelegate;
+
+@Parameters(commandNames = {SliderActions.ACTION_INSTALL_PACKAGE},
+            commandDescription = SliderActions.DESCRIBE_ACTION_INSTALL_PACKAGE)
+
+public class ActionInstallPackageArgs extends AbstractActionArgs {
+  
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_INSTALL_PACKAGE;
+  }
+
+  @Parameter(names = {ARG_PACKAGE},
+             description = "Path to app package on local disk")
+  public String packageURI;
+
+  @Parameter(names = {ARG_NAME},
+             description = "The type of the package")
+  public String name;
+
+  @Parameter(names = {ARG_REPLACE_PKG}, description = "Overwrite existing package")
+  public boolean replacePkg = false;
+
+  /**
+   * Get the min #of params expected
+   * @return the min number of params in the {@link #parameters} field
+   */
+  public int getMinParams() {
+    return 0;
+  }
+
+  @Override
+  public int getMaxParams() {
+    return 1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionKDiagArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionKDiagArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionKDiagArgs.java
new file mode 100644
index 0000000..9fc9d2e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionKDiagArgs.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.UsageException;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+@Parameters(commandNames = {SliderActions.ACTION_KDIAG},
+            commandDescription = SliderActions.DESCRIBE_ACTION_KDIAG)
+
+public class ActionKDiagArgs extends AbstractActionArgs {
+
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_KDIAG;
+  }
+
+  @Parameter(names = {ARG_SERVICES}, variableArity = true,
+    description =" list of services to check")
+  public List<String> services = new ArrayList<>();
+
+  @Parameter(names = {ARG_OUTPUT, ARG_OUTPUT_SHORT},
+      description = "output file for report")
+  public File out;
+
+  @Parameter(names = {ARG_KEYTAB}, description = "keytab to use")
+  public File keytab;
+
+  @Parameter(names = {ARG_KEYLEN}, description = "minimum key length")
+  public int keylen = 256;
+
+  @Parameter(names = {ARG_PRINCIPAL}, description = "principal to log in from a keytab")
+  public String principal;
+
+  @Parameter(names = {ARG_SECURE}, description = "Is security required")
+  public boolean secure = false;
+
+  @Override
+  public int getMinParams() {
+    return 0;
+  }
+
+  @Override
+  public boolean getHadoopServicesRequired() {
+    return false;
+  }
+
+  @Override
+  public boolean disableSecureLogin() {
+    return true;
+  }
+
+  @Override
+  public void validate() throws BadCommandArgumentsException, UsageException {
+    super.validate();
+    if (keytab != null && SliderUtils.isUnset(principal)) {
+      throw new UsageException("Missing argument " + ARG_PRINCIPAL);
+    }
+    if (keytab == null && SliderUtils.isSet(principal)) {
+      throw new UsageException("Missing argument " + ARG_KEYTAB);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionKeytabArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionKeytabArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionKeytabArgs.java
new file mode 100644
index 0000000..32b1d2b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionKeytabArgs.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+
+@Parameters(commandNames = {SliderActions.ACTION_KEYTAB},
+            commandDescription = SliderActions.DESCRIBE_ACTION_KEYTAB)
+
+public class ActionKeytabArgs extends AbstractActionArgs {
+
+  public ActionKeytabArgs(ActionInstallKeytabArgs installKeytabInfo) {
+    this.install = true;
+    this.overwrite = installKeytabInfo.overwrite;
+    this.keytab = installKeytabInfo.keytabUri;
+    this.folder = installKeytabInfo.folder;
+  }
+
+  public ActionKeytabArgs() {
+    super();
+  }
+
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_INSTALL_KEYTAB;
+  }
+
+  @Parameter(names = {ARG_KEYTABINSTALL},
+             description = "Install the keytab")
+  public boolean install;
+
+  @Parameter(names = {ARG_KEYTABDELETE},
+             description = "Delete the keytab")
+  public boolean delete;
+
+  @Parameter(names = {ARG_KEYTABLIST},
+             description = "List of installed keytabs")
+  public boolean list;
+
+  @Parameter(names = {ARG_KEYTAB},
+             description = "Path or name of the keytab")
+  public String keytab;
+
+  @Parameter(names = {ARG_FOLDER},
+             description = "The name of the folder in which to store the keytab")
+  public String folder;
+
+  @Parameter(names = {ARG_OVERWRITE}, description = "Overwrite existing keytab")
+  public boolean overwrite = false;
+
+  /**
+   * Get the min #of params expected
+   * @return the min number of params in the {@link #parameters} field
+   */
+  public int getMinParams() {
+    return 0;
+  }
+
+  @Override
+  public int getMaxParams() {
+    return 3;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionKillContainerArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionKillContainerArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionKillContainerArgs.java
new file mode 100644
index 0000000..8c18ad8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionKillContainerArgs.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+
+@Parameters(commandNames = {SliderActions.ACTION_KILL_CONTAINER},
+            commandDescription = SliderActions.DESCRIBE_ACTION_KILL_CONTAINER)
+
+public class ActionKillContainerArgs extends AbstractActionArgs {
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_KILL_CONTAINER;
+  }
+
+  @Parameter(names = {ARG_ID},
+             description = "ID of the container")
+  public String id;
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[34/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/proto/SliderClusterAPI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/proto/SliderClusterAPI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/proto/SliderClusterAPI.java
new file mode 100644
index 0000000..081b7fa
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/proto/SliderClusterAPI.java
@@ -0,0 +1,2293 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: SliderClusterProtocol.proto
+
+package org.apache.slider.api.proto;
+
+public final class SliderClusterAPI {
+  private SliderClusterAPI() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  /**
+   * Protobuf service {@code org.apache.slider.api.SliderClusterProtocolPB}
+   *
+   * <pre>
+   **
+   * Protocol used from between Slider Client and AM
+   * </pre>
+   */
+  public static abstract class SliderClusterProtocolPB
+      implements com.google.protobuf.Service {
+    protected SliderClusterProtocolPB() {}
+
+    public interface Interface {
+      /**
+       * <code>rpc stopCluster(.org.apache.slider.api.StopClusterRequestProto) returns (.org.apache.slider.api.StopClusterResponseProto);</code>
+       */
+      public abstract void stopCluster(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.StopClusterRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.StopClusterResponseProto> done);
+
+      /**
+       * <code>rpc upgradeContainers(.org.apache.slider.api.UpgradeContainersRequestProto) returns (.org.apache.slider.api.UpgradeContainersResponseProto);</code>
+       *
+       * <pre>
+       **
+       * Upgrade containers 
+       * </pre>
+       */
+      public abstract void upgradeContainers(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.UpgradeContainersRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.UpgradeContainersResponseProto> done);
+
+      /**
+       * <code>rpc flexCluster(.org.apache.slider.api.FlexClusterRequestProto) returns (.org.apache.slider.api.FlexClusterResponseProto);</code>
+       *
+       * <pre>
+       **
+       * Flex the cluster. 
+       * </pre>
+       */
+      public abstract void flexCluster(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.FlexClusterRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.FlexClusterResponseProto> done);
+
+      /**
+       * <code>rpc getJSONClusterStatus(.org.apache.slider.api.GetJSONClusterStatusRequestProto) returns (.org.apache.slider.api.GetJSONClusterStatusResponseProto);</code>
+       *
+       * <pre>
+       **
+       * Get the current cluster status
+       * </pre>
+       */
+      public abstract void getJSONClusterStatus(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetJSONClusterStatusRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetJSONClusterStatusResponseProto> done);
+
+      /**
+       * <code>rpc getInstanceDefinition(.org.apache.slider.api.GetInstanceDefinitionRequestProto) returns (.org.apache.slider.api.GetInstanceDefinitionResponseProto);</code>
+       *
+       * <pre>
+       **
+       * Get the instance definition
+       * </pre>
+       */
+      public abstract void getInstanceDefinition(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetInstanceDefinitionRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetInstanceDefinitionResponseProto> done);
+
+      /**
+       * <code>rpc listNodeUUIDsByRole(.org.apache.slider.api.ListNodeUUIDsByRoleRequestProto) returns (.org.apache.slider.api.ListNodeUUIDsByRoleResponseProto);</code>
+       *
+       * <pre>
+       **
+       * List all running nodes in a role
+       * </pre>
+       */
+      public abstract void listNodeUUIDsByRole(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleResponseProto> done);
+
+      /**
+       * <code>rpc getNode(.org.apache.slider.api.GetNodeRequestProto) returns (.org.apache.slider.api.GetNodeResponseProto);</code>
+       *
+       * <pre>
+       **
+       * Get the details on a node
+       * </pre>
+       */
+      public abstract void getNode(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetNodeRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetNodeResponseProto> done);
+
+      /**
+       * <code>rpc getClusterNodes(.org.apache.slider.api.GetClusterNodesRequestProto) returns (.org.apache.slider.api.GetClusterNodesResponseProto);</code>
+       *
+       * <pre>
+       **
+       * Get the 
+       * details on a list of nodes.
+       * Unknown nodes are not returned
+       * &lt;i&gt;Important: the order of the results are undefined&lt;/i&gt;
+       * </pre>
+       */
+      public abstract void getClusterNodes(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetClusterNodesRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetClusterNodesResponseProto> done);
+
+      /**
+       * <code>rpc echo(.org.apache.slider.api.EchoRequestProto) returns (.org.apache.slider.api.EchoResponseProto);</code>
+       *
+       * <pre>
+       **
+       * echo some text
+       * </pre>
+       */
+      public abstract void echo(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.EchoRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.EchoResponseProto> done);
+
+      /**
+       * <code>rpc killContainer(.org.apache.slider.api.KillContainerRequestProto) returns (.org.apache.slider.api.KillContainerResponseProto);</code>
+       *
+       * <pre>
+       **
+       * kill a container
+       * </pre>
+       */
+      public abstract void killContainer(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.KillContainerRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.KillContainerResponseProto> done);
+
+      /**
+       * <code>rpc amSuicide(.org.apache.slider.api.AMSuicideRequestProto) returns (.org.apache.slider.api.AMSuicideResponseProto);</code>
+       *
+       * <pre>
+       **
+       * kill the AM
+       * </pre>
+       */
+      public abstract void amSuicide(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.AMSuicideRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.AMSuicideResponseProto> done);
+
+      /**
+       * <code>rpc getLivenessInformation(.org.apache.slider.api.GetApplicationLivenessRequestProto) returns (.org.apache.slider.api.ApplicationLivenessInformationProto);</code>
+       */
+      public abstract void getLivenessInformation(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetApplicationLivenessRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ApplicationLivenessInformationProto> done);
+
+      /**
+       * <code>rpc getLiveContainers(.org.apache.slider.api.GetLiveContainersRequestProto) returns (.org.apache.slider.api.GetLiveContainersResponseProto);</code>
+       */
+      public abstract void getLiveContainers(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetLiveContainersRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveContainersResponseProto> done);
+
+      /**
+       * <code>rpc getLiveContainer(.org.apache.slider.api.GetLiveContainerRequestProto) returns (.org.apache.slider.api.ContainerInformationProto);</code>
+       */
+      public abstract void getLiveContainer(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetLiveContainerRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ContainerInformationProto> done);
+
+      /**
+       * <code>rpc getLiveComponents(.org.apache.slider.api.GetLiveComponentsRequestProto) returns (.org.apache.slider.api.GetLiveComponentsResponseProto);</code>
+       */
+      public abstract void getLiveComponents(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetLiveComponentsRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveComponentsResponseProto> done);
+
+      /**
+       * <code>rpc getLiveComponent(.org.apache.slider.api.GetLiveComponentRequestProto) returns (.org.apache.slider.api.ComponentInformationProto);</code>
+       */
+      public abstract void getLiveComponent(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetLiveComponentRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ComponentInformationProto> done);
+
+      /**
+       * <code>rpc getLiveNodes(.org.apache.slider.api.GetLiveNodesRequestProto) returns (.org.apache.slider.api.GetLiveNodesResponseProto);</code>
+       */
+      public abstract void getLiveNodes(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetLiveNodesRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveNodesResponseProto> done);
+
+      /**
+       * <code>rpc getLiveNode(.org.apache.slider.api.GetLiveNodeRequestProto) returns (.org.apache.slider.api.NodeInformationProto);</code>
+       */
+      public abstract void getLiveNode(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetLiveNodeRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.NodeInformationProto> done);
+
+      /**
+       * <code>rpc getModelDesired(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
+       *
+       * <pre>
+       * AggregateConf getModelDesired()
+       * </pre>
+       */
+      public abstract void getModelDesired(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
+
+      /**
+       * <code>rpc getModelDesiredAppconf(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
+       *
+       * <pre>
+       * ConfTree getModelDesiredAppconf
+       * </pre>
+       */
+      public abstract void getModelDesiredAppconf(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
+
+      /**
+       * <code>rpc getModelDesiredResources(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
+       *
+       * <pre>
+       * ConfTree getModelDesiredResources
+       * </pre>
+       */
+      public abstract void getModelDesiredResources(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
+
+      /**
+       * <code>rpc getModelResolved(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
+       *
+       * <pre>
+       * AggregateConf getModelResolved()
+       * </pre>
+       */
+      public abstract void getModelResolved(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
+
+      /**
+       * <code>rpc getModelResolvedAppconf(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
+       *
+       * <pre>
+       * ConfTree getModelResolvedAppconf
+       * </pre>
+       */
+      public abstract void getModelResolvedAppconf(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
+
+      /**
+       * <code>rpc getModelResolvedResources(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
+       *
+       * <pre>
+       * ConfTree getModelResolvedResources
+       * </pre>
+       */
+      public abstract void getModelResolvedResources(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
+
+      /**
+       * <code>rpc getLiveResources(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
+       *
+       * <pre>
+       * ConfTree getLiveResources
+       * </pre>
+       */
+      public abstract void getLiveResources(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
+
+      /**
+       * <code>rpc getClientCertificateStore(.org.apache.slider.api.GetCertificateStoreRequestProto) returns (.org.apache.slider.api.GetCertificateStoreResponseProto);</code>
+       */
+      public abstract void getClientCertificateStore(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetCertificateStoreRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetCertificateStoreResponseProto> done);
+
+    }
+
+    public static com.google.protobuf.Service newReflectiveService(
+        final Interface impl) {
+      return new SliderClusterProtocolPB() {
+        @java.lang.Override
+        public  void stopCluster(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.StopClusterRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.StopClusterResponseProto> done) {
+          impl.stopCluster(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void upgradeContainers(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.UpgradeContainersRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.UpgradeContainersResponseProto> done) {
+          impl.upgradeContainers(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void flexCluster(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.FlexClusterRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.FlexClusterResponseProto> done) {
+          impl.flexCluster(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void getJSONClusterStatus(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.GetJSONClusterStatusRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetJSONClusterStatusResponseProto> done) {
+          impl.getJSONClusterStatus(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void getInstanceDefinition(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.GetInstanceDefinitionRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetInstanceDefinitionResponseProto> done) {
+          impl.getInstanceDefinition(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void listNodeUUIDsByRole(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleResponseProto> done) {
+          impl.listNodeUUIDsByRole(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void getNode(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.GetNodeRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetNodeResponseProto> done) {
+          impl.getNode(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void getClusterNodes(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.GetClusterNodesRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetClusterNodesResponseProto> done) {
+          impl.getClusterNodes(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void echo(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.EchoRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.EchoResponseProto> done) {
+          impl.echo(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void killContainer(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.KillContainerRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.KillContainerResponseProto> done) {
+          impl.killContainer(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void amSuicide(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.AMSuicideRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.AMSuicideResponseProto> done) {
+          impl.amSuicide(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void getLivenessInformation(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.GetApplicationLivenessRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ApplicationLivenessInformationProto> done) {
+          impl.getLivenessInformation(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void getLiveContainers(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.GetLiveContainersRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveContainersResponseProto> done) {
+          impl.getLiveContainers(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void getLiveContainer(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.GetLiveContainerRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ContainerInformationProto> done) {
+          impl.getLiveContainer(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void getLiveComponents(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.GetLiveComponentsRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveComponentsResponseProto> done) {
+          impl.getLiveComponents(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void getLiveComponent(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.GetLiveComponentRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ComponentInformationProto> done) {
+          impl.getLiveComponent(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void getLiveNodes(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.GetLiveNodesRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveNodesResponseProto> done) {
+          impl.getLiveNodes(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void getLiveNode(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.GetLiveNodeRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.NodeInformationProto> done) {
+          impl.getLiveNode(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void getModelDesired(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
+          impl.getModelDesired(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void getModelDesiredAppconf(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
+          impl.getModelDesiredAppconf(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void getModelDesiredResources(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
+          impl.getModelDesiredResources(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void getModelResolved(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
+          impl.getModelResolved(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void getModelResolvedAppconf(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
+          impl.getModelResolvedAppconf(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void getModelResolvedResources(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
+          impl.getModelResolvedResources(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void getLiveResources(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
+          impl.getLiveResources(controller, request, done);
+        }
+
+        @java.lang.Override
+        public  void getClientCertificateStore(
+            com.google.protobuf.RpcController controller,
+            org.apache.slider.api.proto.Messages.GetCertificateStoreRequestProto request,
+            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetCertificateStoreResponseProto> done) {
+          impl.getClientCertificateStore(controller, request, done);
+        }
+
+      };
+    }
+
+    public static com.google.protobuf.BlockingService
+        newReflectiveBlockingService(final BlockingInterface impl) {
+      return new com.google.protobuf.BlockingService() {
+        public final com.google.protobuf.Descriptors.ServiceDescriptor
+            getDescriptorForType() {
+          return getDescriptor();
+        }
+
+        public final com.google.protobuf.Message callBlockingMethod(
+            com.google.protobuf.Descriptors.MethodDescriptor method,
+            com.google.protobuf.RpcController controller,
+            com.google.protobuf.Message request)
+            throws com.google.protobuf.ServiceException {
+          if (method.getService() != getDescriptor()) {
+            throw new java.lang.IllegalArgumentException(
+              "Service.callBlockingMethod() given method descriptor for " +
+              "wrong service type.");
+          }
+          switch(method.getIndex()) {
+            case 0:
+              return impl.stopCluster(controller, (org.apache.slider.api.proto.Messages.StopClusterRequestProto)request);
+            case 1:
+              return impl.upgradeContainers(controller, (org.apache.slider.api.proto.Messages.UpgradeContainersRequestProto)request);
+            case 2:
+              return impl.flexCluster(controller, (org.apache.slider.api.proto.Messages.FlexClusterRequestProto)request);
+            case 3:
+              return impl.getJSONClusterStatus(controller, (org.apache.slider.api.proto.Messages.GetJSONClusterStatusRequestProto)request);
+            case 4:
+              return impl.getInstanceDefinition(controller, (org.apache.slider.api.proto.Messages.GetInstanceDefinitionRequestProto)request);
+            case 5:
+              return impl.listNodeUUIDsByRole(controller, (org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleRequestProto)request);
+            case 6:
+              return impl.getNode(controller, (org.apache.slider.api.proto.Messages.GetNodeRequestProto)request);
+            case 7:
+              return impl.getClusterNodes(controller, (org.apache.slider.api.proto.Messages.GetClusterNodesRequestProto)request);
+            case 8:
+              return impl.echo(controller, (org.apache.slider.api.proto.Messages.EchoRequestProto)request);
+            case 9:
+              return impl.killContainer(controller, (org.apache.slider.api.proto.Messages.KillContainerRequestProto)request);
+            case 10:
+              return impl.amSuicide(controller, (org.apache.slider.api.proto.Messages.AMSuicideRequestProto)request);
+            case 11:
+              return impl.getLivenessInformation(controller, (org.apache.slider.api.proto.Messages.GetApplicationLivenessRequestProto)request);
+            case 12:
+              return impl.getLiveContainers(controller, (org.apache.slider.api.proto.Messages.GetLiveContainersRequestProto)request);
+            case 13:
+              return impl.getLiveContainer(controller, (org.apache.slider.api.proto.Messages.GetLiveContainerRequestProto)request);
+            case 14:
+              return impl.getLiveComponents(controller, (org.apache.slider.api.proto.Messages.GetLiveComponentsRequestProto)request);
+            case 15:
+              return impl.getLiveComponent(controller, (org.apache.slider.api.proto.Messages.GetLiveComponentRequestProto)request);
+            case 16:
+              return impl.getLiveNodes(controller, (org.apache.slider.api.proto.Messages.GetLiveNodesRequestProto)request);
+            case 17:
+              return impl.getLiveNode(controller, (org.apache.slider.api.proto.Messages.GetLiveNodeRequestProto)request);
+            case 18:
+              return impl.getModelDesired(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request);
+            case 19:
+              return impl.getModelDesiredAppconf(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request);
+            case 20:
+              return impl.getModelDesiredResources(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request);
+            case 21:
+              return impl.getModelResolved(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request);
+            case 22:
+              return impl.getModelResolvedAppconf(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request);
+            case 23:
+              return impl.getModelResolvedResources(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request);
+            case 24:
+              return impl.getLiveResources(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request);
+            case 25:
+              return impl.getClientCertificateStore(controller, (org.apache.slider.api.proto.Messages.GetCertificateStoreRequestProto)request);
+            default:
+              throw new java.lang.AssertionError("Can't get here.");
+          }
+        }
+
+        public final com.google.protobuf.Message
+            getRequestPrototype(
+            com.google.protobuf.Descriptors.MethodDescriptor method) {
+          if (method.getService() != getDescriptor()) {
+            throw new java.lang.IllegalArgumentException(
+              "Service.getRequestPrototype() given method " +
+              "descriptor for wrong service type.");
+          }
+          switch(method.getIndex()) {
+            case 0:
+              return org.apache.slider.api.proto.Messages.StopClusterRequestProto.getDefaultInstance();
+            case 1:
+              return org.apache.slider.api.proto.Messages.UpgradeContainersRequestProto.getDefaultInstance();
+            case 2:
+              return org.apache.slider.api.proto.Messages.FlexClusterRequestProto.getDefaultInstance();
+            case 3:
+              return org.apache.slider.api.proto.Messages.GetJSONClusterStatusRequestProto.getDefaultInstance();
+            case 4:
+              return org.apache.slider.api.proto.Messages.GetInstanceDefinitionRequestProto.getDefaultInstance();
+            case 5:
+              return org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleRequestProto.getDefaultInstance();
+            case 6:
+              return org.apache.slider.api.proto.Messages.GetNodeRequestProto.getDefaultInstance();
+            case 7:
+              return org.apache.slider.api.proto.Messages.GetClusterNodesRequestProto.getDefaultInstance();
+            case 8:
+              return org.apache.slider.api.proto.Messages.EchoRequestProto.getDefaultInstance();
+            case 9:
+              return org.apache.slider.api.proto.Messages.KillContainerRequestProto.getDefaultInstance();
+            case 10:
+              return org.apache.slider.api.proto.Messages.AMSuicideRequestProto.getDefaultInstance();
+            case 11:
+              return org.apache.slider.api.proto.Messages.GetApplicationLivenessRequestProto.getDefaultInstance();
+            case 12:
+              return org.apache.slider.api.proto.Messages.GetLiveContainersRequestProto.getDefaultInstance();
+            case 13:
+              return org.apache.slider.api.proto.Messages.GetLiveContainerRequestProto.getDefaultInstance();
+            case 14:
+              return org.apache.slider.api.proto.Messages.GetLiveComponentsRequestProto.getDefaultInstance();
+            case 15:
+              return org.apache.slider.api.proto.Messages.GetLiveComponentRequestProto.getDefaultInstance();
+            case 16:
+              return org.apache.slider.api.proto.Messages.GetLiveNodesRequestProto.getDefaultInstance();
+            case 17:
+              return org.apache.slider.api.proto.Messages.GetLiveNodeRequestProto.getDefaultInstance();
+            case 18:
+              return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
+            case 19:
+              return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
+            case 20:
+              return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
+            case 21:
+              return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
+            case 22:
+              return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
+            case 23:
+              return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
+            case 24:
+              return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
+            case 25:
+              return org.apache.slider.api.proto.Messages.GetCertificateStoreRequestProto.getDefaultInstance();
+            default:
+              throw new java.lang.AssertionError("Can't get here.");
+          }
+        }
+
+        public final com.google.protobuf.Message
+            getResponsePrototype(
+            com.google.protobuf.Descriptors.MethodDescriptor method) {
+          if (method.getService() != getDescriptor()) {
+            throw new java.lang.IllegalArgumentException(
+              "Service.getResponsePrototype() given method " +
+              "descriptor for wrong service type.");
+          }
+          switch(method.getIndex()) {
+            case 0:
+              return org.apache.slider.api.proto.Messages.StopClusterResponseProto.getDefaultInstance();
+            case 1:
+              return org.apache.slider.api.proto.Messages.UpgradeContainersResponseProto.getDefaultInstance();
+            case 2:
+              return org.apache.slider.api.proto.Messages.FlexClusterResponseProto.getDefaultInstance();
+            case 3:
+              return org.apache.slider.api.proto.Messages.GetJSONClusterStatusResponseProto.getDefaultInstance();
+            case 4:
+              return org.apache.slider.api.proto.Messages.GetInstanceDefinitionResponseProto.getDefaultInstance();
+            case 5:
+              return org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleResponseProto.getDefaultInstance();
+            case 6:
+              return org.apache.slider.api.proto.Messages.GetNodeResponseProto.getDefaultInstance();
+            case 7:
+              return org.apache.slider.api.proto.Messages.GetClusterNodesResponseProto.getDefaultInstance();
+            case 8:
+              return org.apache.slider.api.proto.Messages.EchoResponseProto.getDefaultInstance();
+            case 9:
+              return org.apache.slider.api.proto.Messages.KillContainerResponseProto.getDefaultInstance();
+            case 10:
+              return org.apache.slider.api.proto.Messages.AMSuicideResponseProto.getDefaultInstance();
+            case 11:
+              return org.apache.slider.api.proto.Messages.ApplicationLivenessInformationProto.getDefaultInstance();
+            case 12:
+              return org.apache.slider.api.proto.Messages.GetLiveContainersResponseProto.getDefaultInstance();
+            case 13:
+              return org.apache.slider.api.proto.Messages.ContainerInformationProto.getDefaultInstance();
+            case 14:
+              return org.apache.slider.api.proto.Messages.GetLiveComponentsResponseProto.getDefaultInstance();
+            case 15:
+              return org.apache.slider.api.proto.Messages.ComponentInformationProto.getDefaultInstance();
+            case 16:
+              return org.apache.slider.api.proto.Messages.GetLiveNodesResponseProto.getDefaultInstance();
+            case 17:
+              return org.apache.slider.api.proto.Messages.NodeInformationProto.getDefaultInstance();
+            case 18:
+              return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
+            case 19:
+              return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
+            case 20:
+              return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
+            case 21:
+              return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
+            case 22:
+              return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
+            case 23:
+              return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
+            case 24:
+              return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
+            case 25:
+              return org.apache.slider.api.proto.Messages.GetCertificateStoreResponseProto.getDefaultInstance();
+            default:
+              throw new java.lang.AssertionError("Can't get here.");
+          }
+        }
+
+      };
+    }
+
+    /**
+     * <code>rpc stopCluster(.org.apache.slider.api.StopClusterRequestProto) returns (.org.apache.slider.api.StopClusterResponseProto);</code>
+     */
+    public abstract void stopCluster(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.StopClusterRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.StopClusterResponseProto> done);
+
+    /**
+     * <code>rpc upgradeContainers(.org.apache.slider.api.UpgradeContainersRequestProto) returns (.org.apache.slider.api.UpgradeContainersResponseProto);</code>
+     *
+     * <pre>
+     **
+     * Upgrade containers 
+     * </pre>
+     */
+    public abstract void upgradeContainers(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.UpgradeContainersRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.UpgradeContainersResponseProto> done);
+
+    /**
+     * <code>rpc flexCluster(.org.apache.slider.api.FlexClusterRequestProto) returns (.org.apache.slider.api.FlexClusterResponseProto);</code>
+     *
+     * <pre>
+     **
+     * Flex the cluster. 
+     * </pre>
+     */
+    public abstract void flexCluster(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.FlexClusterRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.FlexClusterResponseProto> done);
+
+    /**
+     * <code>rpc getJSONClusterStatus(.org.apache.slider.api.GetJSONClusterStatusRequestProto) returns (.org.apache.slider.api.GetJSONClusterStatusResponseProto);</code>
+     *
+     * <pre>
+     **
+     * Get the current cluster status
+     * </pre>
+     */
+    public abstract void getJSONClusterStatus(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.GetJSONClusterStatusRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetJSONClusterStatusResponseProto> done);
+
+    /**
+     * <code>rpc getInstanceDefinition(.org.apache.slider.api.GetInstanceDefinitionRequestProto) returns (.org.apache.slider.api.GetInstanceDefinitionResponseProto);</code>
+     *
+     * <pre>
+     **
+     * Get the instance definition
+     * </pre>
+     */
+    public abstract void getInstanceDefinition(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.GetInstanceDefinitionRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetInstanceDefinitionResponseProto> done);
+
+    /**
+     * <code>rpc listNodeUUIDsByRole(.org.apache.slider.api.ListNodeUUIDsByRoleRequestProto) returns (.org.apache.slider.api.ListNodeUUIDsByRoleResponseProto);</code>
+     *
+     * <pre>
+     **
+     * List all running nodes in a role
+     * </pre>
+     */
+    public abstract void listNodeUUIDsByRole(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleResponseProto> done);
+
+    /**
+     * <code>rpc getNode(.org.apache.slider.api.GetNodeRequestProto) returns (.org.apache.slider.api.GetNodeResponseProto);</code>
+     *
+     * <pre>
+     **
+     * Get the details on a node
+     * </pre>
+     */
+    public abstract void getNode(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.GetNodeRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetNodeResponseProto> done);
+
+    /**
+     * <code>rpc getClusterNodes(.org.apache.slider.api.GetClusterNodesRequestProto) returns (.org.apache.slider.api.GetClusterNodesResponseProto);</code>
+     *
+     * <pre>
+     **
+     * Get the 
+     * details on a list of nodes.
+     * Unknown nodes are not returned
+     * &lt;i&gt;Important: the order of the results are undefined&lt;/i&gt;
+     * </pre>
+     */
+    public abstract void getClusterNodes(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.GetClusterNodesRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetClusterNodesResponseProto> done);
+
+    /**
+     * <code>rpc echo(.org.apache.slider.api.EchoRequestProto) returns (.org.apache.slider.api.EchoResponseProto);</code>
+     *
+     * <pre>
+     **
+     * echo some text
+     * </pre>
+     */
+    public abstract void echo(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.EchoRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.EchoResponseProto> done);
+
+    /**
+     * <code>rpc killContainer(.org.apache.slider.api.KillContainerRequestProto) returns (.org.apache.slider.api.KillContainerResponseProto);</code>
+     *
+     * <pre>
+     **
+     * kill a container
+     * </pre>
+     */
+    public abstract void killContainer(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.KillContainerRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.KillContainerResponseProto> done);
+
+    /**
+     * <code>rpc amSuicide(.org.apache.slider.api.AMSuicideRequestProto) returns (.org.apache.slider.api.AMSuicideResponseProto);</code>
+     *
+     * <pre>
+     **
+     * kill the AM
+     * </pre>
+     */
+    public abstract void amSuicide(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.AMSuicideRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.AMSuicideResponseProto> done);
+
+    /**
+     * <code>rpc getLivenessInformation(.org.apache.slider.api.GetApplicationLivenessRequestProto) returns (.org.apache.slider.api.ApplicationLivenessInformationProto);</code>
+     */
+    public abstract void getLivenessInformation(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.GetApplicationLivenessRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ApplicationLivenessInformationProto> done);
+
+    /**
+     * <code>rpc getLiveContainers(.org.apache.slider.api.GetLiveContainersRequestProto) returns (.org.apache.slider.api.GetLiveContainersResponseProto);</code>
+     */
+    public abstract void getLiveContainers(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.GetLiveContainersRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveContainersResponseProto> done);
+
+    /**
+     * <code>rpc getLiveContainer(.org.apache.slider.api.GetLiveContainerRequestProto) returns (.org.apache.slider.api.ContainerInformationProto);</code>
+     */
+    public abstract void getLiveContainer(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.GetLiveContainerRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ContainerInformationProto> done);
+
+    /**
+     * <code>rpc getLiveComponents(.org.apache.slider.api.GetLiveComponentsRequestProto) returns (.org.apache.slider.api.GetLiveComponentsResponseProto);</code>
+     */
+    public abstract void getLiveComponents(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.GetLiveComponentsRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveComponentsResponseProto> done);
+
+    /**
+     * <code>rpc getLiveComponent(.org.apache.slider.api.GetLiveComponentRequestProto) returns (.org.apache.slider.api.ComponentInformationProto);</code>
+     */
+    public abstract void getLiveComponent(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.GetLiveComponentRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ComponentInformationProto> done);
+
+    /**
+     * <code>rpc getLiveNodes(.org.apache.slider.api.GetLiveNodesRequestProto) returns (.org.apache.slider.api.GetLiveNodesResponseProto);</code>
+     */
+    public abstract void getLiveNodes(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.GetLiveNodesRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveNodesResponseProto> done);
+
+    /**
+     * <code>rpc getLiveNode(.org.apache.slider.api.GetLiveNodeRequestProto) returns (.org.apache.slider.api.NodeInformationProto);</code>
+     */
+    public abstract void getLiveNode(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.GetLiveNodeRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.NodeInformationProto> done);
+
+    /**
+     * <code>rpc getModelDesired(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
+     *
+     * <pre>
+     * AggregateConf getModelDesired()
+     * </pre>
+     */
+    public abstract void getModelDesired(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
+
+    /**
+     * <code>rpc getModelDesiredAppconf(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
+     *
+     * <pre>
+     * ConfTree getModelDesiredAppconf
+     * </pre>
+     */
+    public abstract void getModelDesiredAppconf(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
+
+    /**
+     * <code>rpc getModelDesiredResources(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
+     *
+     * <pre>
+     * ConfTree getModelDesiredResources
+     * </pre>
+     */
+    public abstract void getModelDesiredResources(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
+
+    /**
+     * <code>rpc getModelResolved(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
+     *
+     * <pre>
+     * AggregateConf getModelResolved()
+     * </pre>
+     */
+    public abstract void getModelResolved(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
+
+    /**
+     * <code>rpc getModelResolvedAppconf(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
+     *
+     * <pre>
+     * ConfTree getModelResolvedAppconf
+     * </pre>
+     */
+    public abstract void getModelResolvedAppconf(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
+
+    /**
+     * <code>rpc getModelResolvedResources(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
+     *
+     * <pre>
+     * ConfTree getModelResolvedResources
+     * </pre>
+     */
+    public abstract void getModelResolvedResources(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
+
+    /**
+     * <code>rpc getLiveResources(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
+     *
+     * <pre>
+     * ConfTree getLiveResources
+     * </pre>
+     */
+    public abstract void getLiveResources(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
+
+    /**
+     * <code>rpc getClientCertificateStore(.org.apache.slider.api.GetCertificateStoreRequestProto) returns (.org.apache.slider.api.GetCertificateStoreResponseProto);</code>
+     */
+    public abstract void getClientCertificateStore(
+        com.google.protobuf.RpcController controller,
+        org.apache.slider.api.proto.Messages.GetCertificateStoreRequestProto request,
+        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetCertificateStoreResponseProto> done);
+
+    public static final
+        com.google.protobuf.Descriptors.ServiceDescriptor
+        getDescriptor() {
+      return org.apache.slider.api.proto.SliderClusterAPI.getDescriptor().getServices().get(0);
+    }
+    public final com.google.protobuf.Descriptors.ServiceDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+
+    public final void callMethod(
+        com.google.protobuf.Descriptors.MethodDescriptor method,
+        com.google.protobuf.RpcController controller,
+        com.google.protobuf.Message request,
+        com.google.protobuf.RpcCallback<
+          com.google.protobuf.Message> done) {
+      if (method.getService() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "Service.callMethod() given method descriptor for wrong " +
+          "service type.");
+      }
+      switch(method.getIndex()) {
+        case 0:
+          this.stopCluster(controller, (org.apache.slider.api.proto.Messages.StopClusterRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.StopClusterResponseProto>specializeCallback(
+              done));
+          return;
+        case 1:
+          this.upgradeContainers(controller, (org.apache.slider.api.proto.Messages.UpgradeContainersRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.UpgradeContainersResponseProto>specializeCallback(
+              done));
+          return;
+        case 2:
+          this.flexCluster(controller, (org.apache.slider.api.proto.Messages.FlexClusterRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.FlexClusterResponseProto>specializeCallback(
+              done));
+          return;
+        case 3:
+          this.getJSONClusterStatus(controller, (org.apache.slider.api.proto.Messages.GetJSONClusterStatusRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.GetJSONClusterStatusResponseProto>specializeCallback(
+              done));
+          return;
+        case 4:
+          this.getInstanceDefinition(controller, (org.apache.slider.api.proto.Messages.GetInstanceDefinitionRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.GetInstanceDefinitionResponseProto>specializeCallback(
+              done));
+          return;
+        case 5:
+          this.listNodeUUIDsByRole(controller, (org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleResponseProto>specializeCallback(
+              done));
+          return;
+        case 6:
+          this.getNode(controller, (org.apache.slider.api.proto.Messages.GetNodeRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.GetNodeResponseProto>specializeCallback(
+              done));
+          return;
+        case 7:
+          this.getClusterNodes(controller, (org.apache.slider.api.proto.Messages.GetClusterNodesRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.GetClusterNodesResponseProto>specializeCallback(
+              done));
+          return;
+        case 8:
+          this.echo(controller, (org.apache.slider.api.proto.Messages.EchoRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.EchoResponseProto>specializeCallback(
+              done));
+          return;
+        case 9:
+          this.killContainer(controller, (org.apache.slider.api.proto.Messages.KillContainerRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.KillContainerResponseProto>specializeCallback(
+              done));
+          return;
+        case 10:
+          this.amSuicide(controller, (org.apache.slider.api.proto.Messages.AMSuicideRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.AMSuicideResponseProto>specializeCallback(
+              done));
+          return;
+        case 11:
+          this.getLivenessInformation(controller, (org.apache.slider.api.proto.Messages.GetApplicationLivenessRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.ApplicationLivenessInformationProto>specializeCallback(
+              done));
+          return;
+        case 12:
+          this.getLiveContainers(controller, (org.apache.slider.api.proto.Messages.GetLiveContainersRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.GetLiveContainersResponseProto>specializeCallback(
+              done));
+          return;
+        case 13:
+          this.getLiveContainer(controller, (org.apache.slider.api.proto.Messages.GetLiveContainerRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.ContainerInformationProto>specializeCallback(
+              done));
+          return;
+        case 14:
+          this.getLiveComponents(controller, (org.apache.slider.api.proto.Messages.GetLiveComponentsRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.GetLiveComponentsResponseProto>specializeCallback(
+              done));
+          return;
+        case 15:
+          this.getLiveComponent(controller, (org.apache.slider.api.proto.Messages.GetLiveComponentRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.ComponentInformationProto>specializeCallback(
+              done));
+          return;
+        case 16:
+          this.getLiveNodes(controller, (org.apache.slider.api.proto.Messages.GetLiveNodesRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.GetLiveNodesResponseProto>specializeCallback(
+              done));
+          return;
+        case 17:
+          this.getLiveNode(controller, (org.apache.slider.api.proto.Messages.GetLiveNodeRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.NodeInformationProto>specializeCallback(
+              done));
+          return;
+        case 18:
+          this.getModelDesired(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.WrappedJsonProto>specializeCallback(
+              done));
+          return;
+        case 19:
+          this.getModelDesiredAppconf(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.WrappedJsonProto>specializeCallback(
+              done));
+          return;
+        case 20:
+          this.getModelDesiredResources(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.WrappedJsonProto>specializeCallback(
+              done));
+          return;
+        case 21:
+          this.getModelResolved(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.WrappedJsonProto>specializeCallback(
+              done));
+          return;
+        case 22:
+          this.getModelResolvedAppconf(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.WrappedJsonProto>specializeCallback(
+              done));
+          return;
+        case 23:
+          this.getModelResolvedResources(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.WrappedJsonProto>specializeCallback(
+              done));
+          return;
+        case 24:
+          this.getLiveResources(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.WrappedJsonProto>specializeCallback(
+              done));
+          return;
+        case 25:
+          this.getClientCertificateStore(controller, (org.apache.slider.api.proto.Messages.GetCertificateStoreRequestProto)request,
+            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.GetCertificateStoreResponseProto>specializeCallback(
+              done));
+          return;
+        default:
+          throw new java.lang.AssertionError("Can't get here.");
+      }
+    }
+
+    public final com.google.protobuf.Message
+        getRequestPrototype(
+        com.google.protobuf.Descriptors.MethodDescriptor method) {
+      if (method.getService() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "Service.getRequestPrototype() given method " +
+          "descriptor for wrong service type.");
+      }
+      switch(method.getIndex()) {
+        case 0:
+          return org.apache.slider.api.proto.Messages.StopClusterRequestProto.getDefaultInstance();
+        case 1:
+          return org.apache.slider.api.proto.Messages.UpgradeContainersRequestProto.getDefaultInstance();
+        case 2:
+          return org.apache.slider.api.proto.Messages.FlexClusterRequestProto.getDefaultInstance();
+        case 3:
+          return org.apache.slider.api.proto.Messages.GetJSONClusterStatusRequestProto.getDefaultInstance();
+        case 4:
+          return org.apache.slider.api.proto.Messages.GetInstanceDefinitionRequestProto.getDefaultInstance();
+        case 5:
+          return org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleRequestProto.getDefaultInstance();
+        case 6:
+          return org.apache.slider.api.proto.Messages.GetNodeRequestProto.getDefaultInstance();
+        case 7:
+          return org.apache.slider.api.proto.Messages.GetClusterNodesRequestProto.getDefaultInstance();
+        case 8:
+          return org.apache.slider.api.proto.Messages.EchoRequestProto.getDefaultInstance();
+        case 9:
+          return org.apache.slider.api.proto.Messages.KillContainerRequestProto.getDefaultInstance();
+        case 10:
+          return org.apache.slider.api.proto.Messages.AMSuicideRequestProto.getDefaultInstance();
+        case 11:
+          return org.apache.slider.api.proto.Messages.GetApplicationLivenessRequestProto.getDefaultInstance();
+        case 12:
+          return org.apache.slider.api.proto.Messages.GetLiveContainersRequestProto.getDefaultInstance();
+        case 13:
+          return org.apache.slider.api.proto.Messages.GetLiveContainerRequestProto.getDefaultInstance();
+        case 14:
+          return org.apache.slider.api.proto.Messages.GetLiveComponentsRequestProto.getDefaultInstance();
+        case 15:
+          return org.apache.slider.api.proto.Messages.GetLiveComponentRequestProto.getDefaultInstance();
+        case 16:
+          return org.apache.slider.api.proto.Messages.GetLiveNodesRequestProto.getDefaultInstance();
+        case 17:
+          return org.apache.slider.api.proto.Messages.GetLiveNodeRequestProto.getDefaultInstance();
+        case 18:
+          return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
+        case 19:
+          return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
+        case 20:
+          return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
+        case 21:
+          return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
+        case 22:
+          return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
+        case 23:
+          return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
+        case 24:
+          return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
+        case 25:
+          return org.apache.slider.api.proto.Messages.GetCertificateStoreRequestProto.getDefaultInstance();
+        default:
+          throw new java.lang.AssertionError("Can't get here.");
+      }
+    }
+
+    public final com.google.protobuf.Message
+        getResponsePrototype(
+        com.google.protobuf.Descriptors.MethodDescriptor method) {
+      if (method.getService() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "Service.getResponsePrototype() given method " +
+          "descriptor for wrong service type.");
+      }
+      switch(method.getIndex()) {
+        case 0:
+          return org.apache.slider.api.proto.Messages.StopClusterResponseProto.getDefaultInstance();
+        case 1:
+          return org.apache.slider.api.proto.Messages.UpgradeContainersResponseProto.getDefaultInstance();
+        case 2:
+          return org.apache.slider.api.proto.Messages.FlexClusterResponseProto.getDefaultInstance();
+        case 3:
+          return org.apache.slider.api.proto.Messages.GetJSONClusterStatusResponseProto.getDefaultInstance();
+        case 4:
+          return org.apache.slider.api.proto.Messages.GetInstanceDefinitionResponseProto.getDefaultInstance();
+        case 5:
+          return org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleResponseProto.getDefaultInstance();
+        case 6:
+          return org.apache.slider.api.proto.Messages.GetNodeResponseProto.getDefaultInstance();
+        case 7:
+          return org.apache.slider.api.proto.Messages.GetClusterNodesResponseProto.getDefaultInstance();
+        case 8:
+          return org.apache.slider.api.proto.Messages.EchoResponseProto.getDefaultInstance();
+        case 9:
+          return org.apache.slider.api.proto.Messages.KillContainerResponseProto.getDefaultInstance();
+        case 10:
+          return org.apache.slider.api.proto.Messages.AMSuicideResponseProto.getDefaultInstance();
+        case 11:
+          return org.apache.slider.api.proto.Messages.ApplicationLivenessInformationProto.getDefaultInstance();
+        case 12:
+          return org.apache.slider.api.proto.Messages.GetLiveContainersResponseProto.getDefaultInstance();
+        case 13:
+          return org.apache.slider.api.proto.Messages.ContainerInformationProto.getDefaultInstance();
+        case 14:
+          return org.apache.slider.api.proto.Messages.GetLiveComponentsResponseProto.getDefaultInstance();
+        case 15:
+          return org.apache.slider.api.proto.Messages.ComponentInformationProto.getDefaultInstance();
+        case 16:
+          return org.apache.slider.api.proto.Messages.GetLiveNodesResponseProto.getDefaultInstance();
+        case 17:
+          return org.apache.slider.api.proto.Messages.NodeInformationProto.getDefaultInstance();
+        case 18:
+          return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
+        case 19:
+          return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
+        case 20:
+          return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
+        case 21:
+          return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
+        case 22:
+          return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
+        case 23:
+          return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
+        case 24:
+          return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
+        case 25:
+          return org.apache.slider.api.proto.Messages.GetCertificateStoreResponseProto.getDefaultInstance();
+        default:
+          throw new java.lang.AssertionError("Can't get here.");
+      }
+    }
+
+    public static Stub newStub(
+        com.google.protobuf.RpcChannel channel) {
+      return new Stub(channel);
+    }
+
+    public static final class Stub extends org.apache.slider.api.proto.SliderClusterAPI.SliderClusterProtocolPB implements Interface {
+      private Stub(com.google.protobuf.RpcChannel channel) {
+        this.channel = channel;
+      }
+
+      private final com.google.protobuf.RpcChannel channel;
+
+      public com.google.protobuf.RpcChannel getChannel() {
+        return channel;
+      }
+
+      public  void stopCluster(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.StopClusterRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.StopClusterResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(0),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.StopClusterResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.StopClusterResponseProto.class,
+            org.apache.slider.api.proto.Messages.StopClusterResponseProto.getDefaultInstance()));
+      }
+
+      public  void upgradeContainers(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.UpgradeContainersRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.UpgradeContainersResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(1),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.UpgradeContainersResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.UpgradeContainersResponseProto.class,
+            org.apache.slider.api.proto.Messages.UpgradeContainersResponseProto.getDefaultInstance()));
+      }
+
+      public  void flexCluster(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.FlexClusterRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.FlexClusterResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(2),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.FlexClusterResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.FlexClusterResponseProto.class,
+            org.apache.slider.api.proto.Messages.FlexClusterResponseProto.getDefaultInstance()));
+      }
+
+      public  void getJSONClusterStatus(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetJSONClusterStatusRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetJSONClusterStatusResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(3),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.GetJSONClusterStatusResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.GetJSONClusterStatusResponseProto.class,
+            org.apache.slider.api.proto.Messages.GetJSONClusterStatusResponseProto.getDefaultInstance()));
+      }
+
+      public  void getInstanceDefinition(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetInstanceDefinitionRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetInstanceDefinitionResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(4),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.GetInstanceDefinitionResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.GetInstanceDefinitionResponseProto.class,
+            org.apache.slider.api.proto.Messages.GetInstanceDefinitionResponseProto.getDefaultInstance()));
+      }
+
+      public  void listNodeUUIDsByRole(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(5),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleResponseProto.class,
+            org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleResponseProto.getDefaultInstance()));
+      }
+
+      public  void getNode(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetNodeRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetNodeResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(6),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.GetNodeResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.GetNodeResponseProto.class,
+            org.apache.slider.api.proto.Messages.GetNodeResponseProto.getDefaultInstance()));
+      }
+
+      public  void getClusterNodes(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetClusterNodesRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetClusterNodesResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(7),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.GetClusterNodesResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.GetClusterNodesResponseProto.class,
+            org.apache.slider.api.proto.Messages.GetClusterNodesResponseProto.getDefaultInstance()));
+      }
+
+      public  void echo(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.EchoRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.EchoResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(8),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.EchoResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.EchoResponseProto.class,
+            org.apache.slider.api.proto.Messages.EchoResponseProto.getDefaultInstance()));
+      }
+
+      public  void killContainer(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.KillContainerRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.KillContainerResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(9),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.KillContainerResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.KillContainerResponseProto.class,
+            org.apache.slider.api.proto.Messages.KillContainerResponseProto.getDefaultInstance()));
+      }
+
+      public  void amSuicide(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.AMSuicideRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.AMSuicideResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(10),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.AMSuicideResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.AMSuicideResponseProto.class,
+            org.apache.slider.api.proto.Messages.AMSuicideResponseProto.getDefaultInstance()));
+      }
+
+      public  void getLivenessInformation(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetApplicationLivenessRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ApplicationLivenessInformationProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(11),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.ApplicationLivenessInformationProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.ApplicationLivenessInformationProto.class,
+            org.apache.slider.api.proto.Messages.ApplicationLivenessInformationProto.getDefaultInstance()));
+      }
+
+      public  void getLiveContainers(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetLiveContainersRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveContainersResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(12),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.GetLiveContainersResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.GetLiveContainersResponseProto.class,
+            org.apache.slider.api.proto.Messages.GetLiveContainersResponseProto.getDefaultInstance()));
+      }
+
+      public  void getLiveContainer(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetLiveContainerRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ContainerInformationProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(13),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.ContainerInformationProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.ContainerInformationProto.class,
+            org.apache.slider.api.proto.Messages.ContainerInformationProto.getDefaultInstance()));
+      }
+
+      public  void getLiveComponents(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetLiveComponentsRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveComponentsResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(14),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.GetLiveComponentsResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.GetLiveComponentsResponseProto.class,
+            org.apache.slider.api.proto.Messages.GetLiveComponentsResponseProto.getDefaultInstance()));
+      }
+
+      public  void getLiveComponent(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetLiveComponentRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ComponentInformationProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(15),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.ComponentInformationProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.ComponentInformationProto.class,
+            org.apache.slider.api.proto.Messages.ComponentInformationProto.getDefaultInstance()));
+      }
+
+      public  void getLiveNodes(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetLiveNodesRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveNodesResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(16),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.GetLiveNodesResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.GetLiveNodesResponseProto.class,
+            org.apache.slider.api.proto.Messages.GetLiveNodesResponseProto.getDefaultInstance()));
+      }
+
+      public  void getLiveNode(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetLiveNodeRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.NodeInformationProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(17),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.NodeInformationProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.NodeInformationProto.class,
+            org.apache.slider.api.proto.Messages.NodeInformationProto.getDefaultInstance()));
+      }
+
+      public  void getModelDesired(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(18),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.WrappedJsonProto.class,
+            org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance()));
+      }
+
+      public  void getModelDesiredAppconf(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(19),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.WrappedJsonProto.class,
+            org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance()));
+      }
+
+      public  void getModelDesiredResources(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(20),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.WrappedJsonProto.class,
+            org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance()));
+      }
+
+      public  void getModelResolved(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(21),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.WrappedJsonProto.class,
+            org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance()));
+      }
+
+      public  void getModelResolvedAppconf(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(22),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.WrappedJsonProto.class,
+            org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance()));
+      }
+
+      public  void getModelResolvedResources(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(23),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.WrappedJsonProto.class,
+            org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance()));
+      }
+
+      public  void getLiveResources(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(24),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.WrappedJsonProto.class,
+            org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance()));
+      }
+
+      public  void getClientCertificateStore(
+          com.google.protobuf.RpcController controller,
+          org.apache.slider.api.proto.Messages.GetCertificateStoreRequestProto request,
+          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetCertificateStoreResponseProto> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(25),
+          controller,
+          request,
+          org.apache.slider.api.proto.Messages.GetCertificateStoreResponseProto.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.slider.api.proto.Messages.GetCertificateStoreResponseProto.class,
+            org.apache.slider.api.proto.Messages.GetCertificateStoreResponseProto.getDefaultInstance()));
+      }
+    }
+
+    public static BlockingInterface newBlockingStub(
+        com.google.protobuf.BlockingRpcChannel channel) {
+      return new BlockingStub(channel);
+    }
+
+    public interface BlockingInterface {
+      public org.apache.slider.api.proto.Messages.StopClusterResponseProto stopCluster(
+          com.google.protobuf.RpcController controller,
+         

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[31/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClientAPI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClientAPI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClientAPI.java
new file mode 100644
index 0000000..30f6ba9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClientAPI.java
@@ -0,0 +1,368 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.client;
+
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.slider.api.types.NodeInformationList;
+import org.apache.slider.api.types.SliderInstanceDescription;
+import org.apache.slider.common.params.AbstractClusterBuildingActionArgs;
+import org.apache.slider.common.params.ActionAMSuicideArgs;
+import org.apache.slider.common.params.ActionClientArgs;
+import org.apache.slider.common.params.ActionDependencyArgs;
+import org.apache.slider.common.params.ActionDestroyArgs;
+import org.apache.slider.common.params.ActionDiagnosticArgs;
+import org.apache.slider.common.params.ActionEchoArgs;
+import org.apache.slider.common.params.ActionFlexArgs;
+import org.apache.slider.common.params.ActionFreezeArgs;
+import org.apache.slider.common.params.ActionInstallKeytabArgs;
+import org.apache.slider.common.params.ActionInstallPackageArgs;
+import org.apache.slider.common.params.ActionKeytabArgs;
+import org.apache.slider.common.params.ActionNodesArgs;
+import org.apache.slider.common.params.ActionPackageArgs;
+import org.apache.slider.common.params.ActionKillContainerArgs;
+import org.apache.slider.common.params.ActionListArgs;
+import org.apache.slider.common.params.ActionRegistryArgs;
+import org.apache.slider.common.params.ActionResolveArgs;
+import org.apache.slider.common.params.ActionResourceArgs;
+import org.apache.slider.common.params.ActionStatusArgs;
+import org.apache.slider.common.params.ActionThawArgs;
+import org.apache.slider.common.params.ActionUpgradeArgs;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.providers.AbstractClientProvider;
+
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * Interface of those method calls in the slider API that are intended
+ * for direct public invocation.
+ * <p>
+ * Stability: evolving
+ */
+public interface SliderClientAPI extends Service {
+  /**
+   * Destroy a cluster. There's two race conditions here
+   * #1 the cluster is started between verifying that there are no live
+   * clusters of that name.
+   */
+  int actionDestroy(String clustername, ActionDestroyArgs destroyArgs)
+      throws YarnException, IOException;
+
+  int actionDestroy(String clustername) throws YarnException,
+      IOException;
+
+  /**
+   * AM to commit an asynchronous suicide
+   */
+  int actionAmSuicide(String clustername,
+      ActionAMSuicideArgs args) throws YarnException, IOException;
+
+  /**
+   * Get the provider for this cluster
+   * @param provider the name of the provider
+   * @return the provider instance
+   * @throws SliderException problems building the provider
+   */
+  AbstractClientProvider createClientProvider(String provider)
+    throws SliderException;
+
+  /**
+   * Build up the cluster specification/directory
+   *
+   * @param clustername cluster name
+   * @param buildInfo the arguments needed to build the cluster
+   * @throws YarnException Yarn problems
+   * @throws IOException other problems
+   * @throws BadCommandArgumentsException bad arguments.
+   */
+  int actionBuild(String clustername,
+      AbstractClusterBuildingActionArgs buildInfo) throws YarnException, IOException;
+
+  /**
+   * Upload keytab to a designated sub-directory of the user home directory
+   *
+   * @param installKeytabInfo the arguments needed to upload the keytab
+   * @throws YarnException Yarn problems
+   * @throws IOException other problems
+   * @throws BadCommandArgumentsException bad arguments.
+   * @deprecated use #actionKeytab
+   */
+  int actionInstallKeytab(ActionInstallKeytabArgs installKeytabInfo)
+      throws YarnException, IOException;
+
+  /**
+   * Manage keytabs leveraged by slider
+   *
+   * @param keytabInfo the arguments needed to manage the keytab
+   * @throws YarnException Yarn problems
+   * @throws IOException other problems
+   * @throws BadCommandArgumentsException bad arguments.
+   */
+  int actionKeytab(ActionKeytabArgs keytabInfo)
+      throws YarnException, IOException;
+
+  /**
+   * Upload application package to user home directory
+   *
+   * @param installPkgInfo the arguments needed to upload the package
+   * @throws YarnException Yarn problems
+   * @throws IOException other problems
+   * @throws BadCommandArgumentsException bad arguments.
+   */
+  int actionInstallPkg(ActionInstallPackageArgs installPkgInfo)
+      throws YarnException, IOException;
+
+  /**
+   * Manage file resources leveraged by slider
+   *
+   * @param resourceInfo the arguments needed to manage the resource
+   * @throws YarnException Yarn problems
+   * @throws IOException other problems
+   * @throws BadCommandArgumentsException bad arguments.
+   */
+  int actionResource(ActionResourceArgs resourceInfo)
+      throws YarnException, IOException;
+
+  /**
+   * Perform client operations such as install or configure
+   *
+   * @param clientInfo the arguments needed for client operations
+   *
+   * @throws SliderException bad arguments.
+   * @throws IOException problems related to package and destination folders
+   */
+  int actionClient(ActionClientArgs clientInfo)
+      throws IOException, YarnException;
+
+  /**
+   * Managing slider application package
+   *
+   * @param pkgInfo the arguments needed to upload, delete or list the package
+   * @throws YarnException Yarn problems
+   * @throws IOException other problems
+   * @throws BadCommandArgumentsException bad arguments.
+   */
+  int actionPackage(ActionPackageArgs pkgInfo)
+      throws YarnException, IOException;
+
+  /**
+   * Update the cluster specification
+   *
+   * @param clustername cluster name
+   * @param buildInfo the arguments needed to update the cluster
+   * @throws YarnException Yarn problems
+   * @throws IOException other problems
+   */
+  int actionUpdate(String clustername,
+      AbstractClusterBuildingActionArgs buildInfo)
+      throws YarnException, IOException; 
+
+  /**
+   * Upgrade the cluster with a newer version of the application
+   *
+   * @param clustername cluster name
+   * @param buildInfo the arguments needed to upgrade the cluster
+   * @throws YarnException Yarn problems
+   * @throws IOException other problems
+   */
+  int actionUpgrade(String clustername,
+      ActionUpgradeArgs buildInfo)
+      throws YarnException, IOException; 
+
+  /**
+   * Get the report of a this application
+   * @return the app report or null if it could not be found.
+   * @throws IOException
+   * @throws YarnException
+   */
+  ApplicationReport getApplicationReport()
+      throws IOException, YarnException;
+
+  /**
+   * Kill the submitted application via YARN
+   * @throws YarnException
+   * @throws IOException
+   */
+  boolean forceKillApplication(String reason)
+    throws YarnException, IOException;
+
+  /**
+   * Implement the list action: list all nodes
+   * @return exit code of 0 if a list was created
+   */
+  int actionList(String clustername, ActionListArgs args) throws IOException, YarnException;
+
+  /**
+   * Enumerate slider instances for the current user, and the
+   * most recent app report, where available.
+   * @param listOnlyInState boolean to indicate that the instances should
+   * only include those in a YARN state
+   * <code> minAppState &lt;= currentState &lt;= maxAppState </code>
+   *
+   * @param minAppState minimum application state to include in enumeration.
+   * @param maxAppState maximum application state to include
+   * @return a map of application instance name to description
+   * @throws IOException Any IO problem
+   * @throws YarnException YARN problems
+   */
+  Map<String, SliderInstanceDescription> enumSliderInstances(
+      boolean listOnlyInState,
+      YarnApplicationState minAppState,
+      YarnApplicationState maxAppState)
+      throws IOException, YarnException;
+
+  /**
+   * Implement the islive action: probe for a cluster of the given name existing
+   * @return exit code
+   */
+  int actionFlex(String name, ActionFlexArgs args) throws YarnException, IOException;
+
+  /**
+   * Test for a cluster existing probe for a cluster of the given name existing
+   * in the filesystem. If the live param is set, it must be a live cluster
+   * @return exit code
+   */
+  int actionExists(String name, boolean checkLive) throws YarnException, IOException;
+
+  /**
+   * Kill a specific container of the cluster
+   * @param name cluster name
+   * @param args arguments
+   * @return exit code
+   * @throws YarnException
+   * @throws IOException
+   */
+  int actionKillContainer(String name, ActionKillContainerArgs args)
+      throws YarnException, IOException;
+
+  /**
+   * Echo operation (not currently wired up to command line)
+   * @param name cluster name
+   * @param args arguments
+   * @return the echoed text
+   * @throws YarnException
+   * @throws IOException
+   */
+  String actionEcho(String name, ActionEchoArgs args)
+      throws YarnException, IOException;
+
+  /**
+   * Status operation
+   *
+   * @param clustername cluster name
+   * @param statusArgs status arguments
+   * @return 0 -for success, else an exception is thrown
+   * @throws YarnException
+   * @throws IOException
+   */
+  int actionStatus(String clustername, ActionStatusArgs statusArgs)
+      throws YarnException, IOException;
+
+  /**
+   * Version Details
+   * @return exit code
+   */
+  int actionVersion();
+
+  /**
+   * Stop the cluster
+   *
+   * @param clustername cluster name
+   * @param freezeArgs arguments to the stop
+   * @return EXIT_SUCCESS if the cluster was not running by the end of the operation
+   */
+  int actionFreeze(String clustername, ActionFreezeArgs freezeArgs)
+      throws YarnException, IOException;
+
+  /**
+   * Restore a cluster
+   */
+  int actionThaw(String clustername, ActionThawArgs thaw) throws YarnException, IOException;
+
+  /**
+   * Registry operation
+   *
+   * @param args registry Arguments
+   * @return 0 for success, -1 for some issues that aren't errors, just failures
+   * to retrieve information (e.g. no configurations for that entry)
+   * @throws YarnException YARN problems
+   * @throws IOException Network or other problems
+   */
+  int actionResolve(ActionResolveArgs args)
+      throws YarnException, IOException;
+
+  /**
+   * Registry operation
+   *
+   * @param registryArgs registry Arguments
+   * @return 0 for success, -1 for some issues that aren't errors, just failures
+   * to retrieve information (e.g. no configurations for that entry)
+   * @throws YarnException YARN problems
+   * @throws IOException Network or other problems
+   */
+  int actionRegistry(ActionRegistryArgs registryArgs)
+      throws YarnException, IOException;
+
+  /**
+   * diagnostic operation
+   *
+   * @param diagnosticArgs diagnostic Arguments
+   * @return 0 for success, -1 for some issues that aren't errors, just
+   *         failures to retrieve information (e.g. no application name
+   *         specified)
+   * @throws YarnException YARN problems
+   * @throws IOException Network or other problems
+   */
+  int actionDiagnostic(ActionDiagnosticArgs diagnosticArgs);
+
+  /**
+   * Get the registry binding. As this may start the registry, it can take time
+   * and fail
+   * @return the registry 
+   */
+  RegistryOperations getRegistryOperations()
+      throws SliderException, IOException;
+
+  /**
+   * Upload all Slider AM and agent dependency libraries to HDFS, so that they
+   * do not need to be uploaded with every create call. This operation is
+   * Slider version specific. So it needs to be invoked for every single
+   * version of slider/slider-client.
+   * 
+   * @throws SliderException
+   * @throws IOException
+   */
+  int actionDependency(ActionDependencyArgs dependencyArgs) throws IOException,
+      YarnException;
+
+  /**
+   * List the nodes
+   * @param args
+   * @return
+   * @throws YarnException
+   * @throws IOException
+   */
+  NodeInformationList listYarnClusterNodes(ActionNodesArgs args)
+    throws YarnException, IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
new file mode 100644
index 0000000..d471cdb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
@@ -0,0 +1,410 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.client;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hadoop.yarn.util.Records;
+import org.apache.slider.api.types.NodeInformation;
+import org.apache.slider.api.types.NodeInformationList;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.params.ActionNodesArgs;
+import org.apache.slider.common.tools.CoreFileSystem;
+import org.apache.slider.common.tools.Duration;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.BindException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * A class that extends visibility to some of the YarnClientImpl
+ * members and data structures, and factors out pure-YARN operations
+ * from the slider entry point service
+ */
+public class SliderYarnClientImpl extends YarnClientImpl {
+  protected static final Logger log = LoggerFactory.getLogger(SliderYarnClientImpl.class);
+
+  /**
+   * Keyword to use in the {@link #emergencyForceKill(String)}
+   * operation to force kill <i>all</i> application instances belonging
+   * to a specific user
+   */
+  public static final String KILL_ALL = "all";
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    InetSocketAddress clientRpcAddress = SliderUtils.getRmAddress(conf);
+    if (!SliderUtils.isAddressDefined(clientRpcAddress)) {
+      // address isn't known; fail fast
+      throw new BindException("Invalid " + YarnConfiguration.RM_ADDRESS
+          + " value:" + conf.get(YarnConfiguration.RM_ADDRESS)
+          + " - see https://wiki.apache.org/hadoop/UnsetHostnameOrPort");
+    }
+    super.serviceInit(conf);
+  }
+
+  /**
+   * Get the RM Client RPC interface
+   * @return an RPC interface valid after initialization and authentication
+   */
+  public ApplicationClientProtocol getRmClient() {
+    return rmClient;
+  }
+
+  /**
+   * List Slider <i>running</i>instances belonging to a specific user.
+   * @deprecated use {@link #listDeployedInstances(String)}
+   * @param user user: "" means all users
+   * @return a possibly empty list of Slider AMs
+   */
+  public List<ApplicationReport> listInstances(String user)
+    throws YarnException, IOException {
+    return listDeployedInstances(user);
+  }
+
+  /**
+   * List Slider <i>deployed</i>instances belonging to a specific user.
+   * <p>
+   *   Deployed means: known about in the YARN cluster; it will include
+   *   any that are in the failed/finished state, as well as those queued
+   *   for starting.
+   * @param user user: "" means all users
+   * @return a possibly empty list of Slider AMs
+   */
+  public List<ApplicationReport> listDeployedInstances(String user)
+    throws YarnException, IOException {
+    Preconditions.checkArgument(user != null, "Null User");
+    Set<String> types = new HashSet<>(1);
+    types.add(SliderKeys.APP_TYPE);
+    List<ApplicationReport> allApps = getApplications(types);
+    List<ApplicationReport> results = new ArrayList<>();
+    for (ApplicationReport report : allApps) {
+      if (StringUtils.isEmpty(user) || user.equals(report.getUser())) {
+        results.add(report);
+      }
+    }
+    return results;
+  }
+
+  /**
+   * find all instances of a specific app -if there is more than one in the
+   * YARN cluster,
+   * this returns them all
+   * @param user user; use "" for all users
+   * @param appname application name
+   * @return the list of all matching application instances
+   */
+  public List<ApplicationReport> findAllInstances(String user,
+                                                  String appname)
+      throws IOException, YarnException {
+    Preconditions.checkArgument(appname != null, "Null application name");
+
+    List<ApplicationReport> instances = listDeployedInstances(user);
+    List<ApplicationReport> results =
+      new ArrayList<>(instances.size());
+    for (ApplicationReport report : instances) {
+      if (report.getName().equals(appname)) {
+        results.add(report);
+      }
+    }
+    return results;
+  }
+  
+  /**
+   * Helper method to determine if a cluster application is running -or
+   * is earlier in the lifecycle
+   * @param app application report
+   * @return true if the application is considered live
+   */
+  public boolean isApplicationLive(ApplicationReport app) {
+    Preconditions.checkArgument(app != null, "Null app report");
+
+    return app.getYarnApplicationState().ordinal() <= YarnApplicationState.RUNNING.ordinal();
+  }
+
+
+  /**
+   * Kill a running application
+   * @param applicationId app Id
+   * @param reason reason: reason for log
+   * @return the response
+   * @throws YarnException YARN problems
+   * @throws IOException IO problems
+   */
+  public  KillApplicationResponse killRunningApplication(ApplicationId applicationId,
+                                                         String reason)
+      throws YarnException, IOException {
+    Preconditions.checkArgument(applicationId != null, "Null application Id");
+    log.info("Killing application {} - {}", applicationId.getClusterTimestamp(),
+             reason);
+    KillApplicationRequest request =
+      Records.newRecord(KillApplicationRequest.class);
+    request.setApplicationId(applicationId);
+    return getRmClient().forceKillApplication(request);
+  }
+
+  private String getUsername() throws IOException {
+    return UserGroupInformation.getCurrentUser().getShortUserName();
+  }
+  
+  /**
+   * Force kill a yarn application by ID. No niceties here
+   * @param applicationId app Id. "all" means "kill all instances of the current user
+   * 
+   */
+  public void emergencyForceKill(String applicationId)
+      throws YarnException, IOException {
+
+    Preconditions.checkArgument(StringUtils.isNotEmpty(applicationId),
+        "Null/empty application Id");
+
+    if (KILL_ALL.equals(applicationId)) {
+      // user wants all instances killed
+      String user = getUsername();
+      log.info("Killing all applications belonging to {}", user);
+      Collection<ApplicationReport> instances = listDeployedInstances(user);
+      for (ApplicationReport instance : instances) {
+        if (isApplicationLive(instance)) {
+          ApplicationId appId = instance.getApplicationId();
+          log.info("Killing Application {}", appId);
+
+          killRunningApplication(appId, "forced kill");
+        }
+      }
+    } else {
+      ApplicationId appId = ConverterUtils.toApplicationId(applicationId);
+
+      log.info("Killing Application {}", applicationId);
+
+      killRunningApplication(appId, "forced kill");
+    }
+  }
+
+  /**
+   * Monitor the submitted application for reaching the requested state.
+   * Will also report if the app reaches a later state (failed, killed, etc)
+   * Kill application if duration!= null & time expires. 
+   * @param appId Application Id of application to be monitored
+   * @param duration how long to wait -must be more than 0
+   * @param desiredState desired state.
+   * @return the application report -null on a timeout
+   * @throws YarnException
+   * @throws IOException
+   */
+  public ApplicationReport monitorAppToState(
+    ApplicationId appId, YarnApplicationState desiredState, Duration duration)
+    throws YarnException, IOException {
+
+    if (appId == null) {
+      throw new BadCommandArgumentsException("null application ID");
+    }
+    if (duration.limit <= 0) {
+      throw new BadCommandArgumentsException("Invalid monitoring duration");
+    }
+    log.debug("Waiting {} millis for app to reach state {} ",
+              duration.limit,
+              desiredState);
+    duration.start();
+    try {
+      while (true) {
+        // Get application report for the appId we are interested in
+
+        ApplicationReport r = getApplicationReport(appId);
+
+        log.debug("queried status is\n{}",
+          new SliderUtils.OnDemandReportStringifier(r));
+
+        YarnApplicationState state = r.getYarnApplicationState();
+        if (state.ordinal() >= desiredState.ordinal()) {
+          log.debug("App in desired state (or higher) :{}", state);
+          return r;
+        }
+        if (duration.getLimitExceeded()) {
+          log.debug(
+            "Wait limit of {} millis to get to state {}, exceeded; app status\n {}",
+            duration.limit,
+            desiredState,
+            new SliderUtils.OnDemandReportStringifier(r));
+          return null;
+        }
+
+        // sleep 1s.
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException ignored) {
+          log.debug("Thread sleep in monitoring loop interrupted");
+        }
+      }
+    } finally {
+      duration.close();
+    }
+  }
+
+  /**
+   * find all live instances of a specific app -if there is >1 in the cluster,
+   * this returns them all. State should be running or less
+   * @param user user
+   * @param appname application name
+   * @return the list of all matching application instances
+   */
+  public List<ApplicationReport> findAllLiveInstances(String user,
+                                                      String appname) throws
+                                                                      YarnException,
+                                                                      IOException {
+    Preconditions.checkArgument(StringUtils.isNotEmpty(appname),
+        "Null/empty application name");
+    List<ApplicationReport> instances = listDeployedInstances(user);
+    List<ApplicationReport> results =
+      new ArrayList<ApplicationReport>(instances.size());
+    for (ApplicationReport app : instances) {
+      if (app.getName().equals(appname)
+          && isApplicationLive(app)) {
+        results.add(app);
+      }
+    }
+    return results;
+  }
+
+  /**
+   * Find a cluster in the instance list; biased towards live instances
+   * @param instances list of instances
+   * @param appname application name
+   * @return the first found instance, else a failed/finished instance, or null
+   * if there are none of those
+   */
+  public ApplicationReport findClusterInInstanceList(List<ApplicationReport> instances,
+                                                     String appname) {
+    Preconditions.checkArgument(instances != null, "Null instances list");
+    Preconditions.checkArgument(StringUtils.isNotEmpty(appname),
+        "Null/empty application name");
+    // sort by most recent
+    SliderUtils.sortApplicationsByMostRecent(instances);
+    ApplicationReport found = null;
+    for (ApplicationReport app : instances) {
+      if (app.getName().equals(appname)) {
+        if (isApplicationLive(app)) {
+          return app;
+        }
+        // set the found value if not set
+        found = found != null ? found : app;
+      }
+    }
+    return found;
+  }
+
+  /**
+   * Find an app in the instance list in the desired state 
+   * @param instances instance list
+   * @param appname application name
+   * @param desiredState yarn state desired
+   * @return the match or null for none
+   */
+  public ApplicationReport findAppInInstanceList(List<ApplicationReport> instances,
+      String appname,
+      YarnApplicationState desiredState) {
+    Preconditions.checkArgument(instances != null, "Null instances list");
+    Preconditions.checkArgument(StringUtils.isNotEmpty(appname),
+        "Null/empty application name");
+    Preconditions.checkArgument(desiredState != null, "Null desiredState");
+    log.debug("Searching {} records for instance name {} in state '{}'",
+        instances.size(), appname, desiredState);
+    for (ApplicationReport app : instances) {
+      if (app.getName().equals(appname)) {
+
+        YarnApplicationState appstate =
+            app.getYarnApplicationState();
+        log.debug("app ID {} is in state {}", app.getApplicationId(), appstate);
+        if (appstate.equals(desiredState)) {
+          log.debug("match");
+          return app;
+        }
+      }
+    }
+    // nothing found in desired state
+    log.debug("No match");
+    return null;
+  }
+
+  /**
+   * List the nodes in the cluster, possibly filtering by node state or label.
+   *
+   * @param label label to filter by -or "" for any
+   * @param live flag to request running nodes only
+   * @return a possibly empty list of nodes in the cluster
+   * @throws IOException IO problems
+   * @throws YarnException YARN problems
+   */
+  public NodeInformationList listNodes(String label, boolean live)
+    throws IOException, YarnException {
+    Preconditions.checkArgument(label != null, "null label");
+    NodeState[] states;
+    if (live) {
+      states = new NodeState[1];
+      states[0] = NodeState.RUNNING;
+    } else {
+      states = new NodeState[0];
+    }
+    List<NodeReport> reports = getNodeReports(states);
+    NodeInformationList results = new NodeInformationList(reports.size());
+    for (NodeReport report : reports) {
+      if (live && report.getNodeState() != NodeState.RUNNING) {
+        continue;
+      }
+      if (!label.isEmpty() && !report.getNodeLabels().contains(label)) {
+        continue;
+      }
+      // build node info from report
+      NodeInformation info = new NodeInformation();
+      info.hostname = report.getNodeId().getHost();
+      info.healthReport  = report.getHealthReport();
+      info.httpAddress = report.getHttpAddress();
+      info.labels = SliderUtils.extractNodeLabel(report);
+      info.rackName = report.getRackName();
+      info.state = report.getNodeState().toString();
+      results.add(info);
+    }
+    return results;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/TokensOperation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/TokensOperation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/TokensOperation.java
new file mode 100644
index 0000000..9b9c141
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/TokensOperation.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.client;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.slider.common.params.ActionTokensArgs;
+import org.apache.slider.core.exceptions.BadClusterStateException;
+import org.apache.slider.core.exceptions.NotFoundException;
+import static org.apache.slider.core.launch.CredentialUtils.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+
+public class TokensOperation {
+
+  private static final Logger log = LoggerFactory.getLogger(TokensOperation.class);
+  public static final String E_INSECURE
+      = "Cluster is not secure -tokens cannot be acquired";
+  public static final String E_MISSING_SOURCE_FILE = "Missing source file: ";
+  public static final String E_NO_KEYTAB = "No keytab: ";
+
+  public int actionTokens(ActionTokensArgs args, FileSystem fs,
+      Configuration conf,
+      YarnClientImpl yarnClient)
+      throws IOException, YarnException {
+    Credentials credentials;
+    String footnote = "";
+    UserGroupInformation user = UserGroupInformation.getCurrentUser();
+    boolean isSecure = UserGroupInformation.isSecurityEnabled();
+    if (args.keytab != null) {
+      File keytab = args.keytab;
+      if (!keytab.isFile()) {
+        throw new NotFoundException(E_NO_KEYTAB + keytab.getAbsolutePath());
+      }
+      String principal = args.principal;
+      log.info("Logging in as {} from keytab {}", principal, keytab);
+      user = UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+          principal, keytab.getCanonicalPath());
+    }
+    Credentials userCredentials = user.getCredentials();
+    File output = args.output;
+    if (output != null) {
+      if (!isSecure) {
+        throw new BadClusterStateException(E_INSECURE);
+      }
+      credentials = new Credentials(userCredentials);
+      // filesystem
+      addRMRenewableFSDelegationTokens(conf, fs, credentials);
+      addRMDelegationToken(yarnClient, credentials);
+      if (maybeAddTimelineToken(conf, credentials) != null) {
+        log.debug("Added timeline token");
+      }
+      saveTokens(output, credentials);
+      String filename = output.getCanonicalPath();
+      footnote = String.format("%d tokens saved to %s\n" +
+              "To use these in the environment:\n" +
+              "export %s=%s",
+          credentials.numberOfTokens(),
+          filename, UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION, filename);
+    } else if (args.source != null) {
+      File source = args.source;
+      log.info("Reading credentials from file {}", source);
+      if (!source.isFile()) {
+        throw new NotFoundException( E_MISSING_SOURCE_FILE + source.getAbsolutePath());
+      }
+      credentials = Credentials.readTokenStorageFile(args.source, conf);
+    } else {
+      StringBuffer origin = new StringBuffer();
+      File file = locateEnvCredentials(System.getenv(), conf,
+          origin);
+      if (file != null) {
+        log.info("Credential Source {}", origin);
+      } else {
+        log.info("Credential source: logged in user");
+      }
+      credentials = userCredentials;
+    }
+    // list the tokens
+    log.info("\n{}", dumpTokens(credentials, "\n"));
+    if (!footnote.isEmpty()) {
+      log.info(footnote);
+    }
+    return 0;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ipc/SliderApplicationIpcClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ipc/SliderApplicationIpcClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ipc/SliderApplicationIpcClient.java
new file mode 100644
index 0000000..a007326
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ipc/SliderApplicationIpcClient.java
@@ -0,0 +1,245 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.client.ipc;
+
+import com.google.common.base.Preconditions;
+import org.apache.slider.api.SliderClusterProtocol;
+import org.apache.slider.api.types.ApplicationLivenessInformation;
+import org.apache.slider.api.types.ComponentInformation;
+import org.apache.slider.api.types.ContainerInformation;
+import org.apache.slider.api.types.NodeInformation;
+import org.apache.slider.api.types.NodeInformationList;
+import org.apache.slider.api.types.PingInformation;
+import org.apache.slider.api.SliderApplicationApi;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTree;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.core.exceptions.NoSuchNodeException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * Implementation of the Slider RESTy Application API over IPC.
+ * <p>
+ * Operations are executed via the {@link SliderClusterOperations}
+ * instance passed in; raised exceptions may be converted into ones
+ * consistent with the REST API.
+ */
+public class SliderApplicationIpcClient implements SliderApplicationApi {
+
+  private static final Logger log =
+      LoggerFactory.getLogger(SliderApplicationIpcClient.class);
+
+  private final SliderClusterOperations operations;
+
+  public SliderApplicationIpcClient(SliderClusterOperations operations) {
+    Preconditions.checkArgument(operations != null, "null operations");
+    this.operations = operations;
+  }
+
+  /**
+   * Convert received (And potentially unmarshalled) local/remote
+   * exceptions into the equivalents in the REST API.
+   * Best effort. 
+   * <p>
+   * If there is no translation, the original exception is returned.
+   * <p>
+   * If a new exception was created, it will have the message of the 
+   * string value of the original exception, and that original
+   * exception will be the nested cause of this one
+   * @param exception IOException to convert
+   * @return an exception to throw
+   */
+  private IOException convert(IOException exception) {
+    IOException result = exception;
+    if (exception instanceof NoSuchNodeException) {
+      result = new FileNotFoundException(exception.toString());
+      result.initCause(exception);
+    } else {
+      // TODO: remap any other exceptions
+    }
+    return result;
+  }
+  
+  public SliderApplicationIpcClient(SliderClusterProtocol proxy) {
+    this(new SliderClusterOperations(proxy));
+  }
+
+  @Override
+  public AggregateConf getDesiredModel() throws IOException {
+    try {
+      return operations.getModelDesired();
+    } catch (IOException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public ConfTreeOperations getDesiredAppconf() throws IOException {
+    try {
+      return operations.getModelDesiredAppconf();
+    } catch (IOException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public ConfTreeOperations getDesiredResources() throws IOException {
+    try {
+      return operations.getModelDesiredResources();
+    } catch (IOException e) {
+      throw convert(e);
+    }
+  }
+
+
+  @Override
+  public void putDesiredResources(ConfTree updated) throws IOException {
+    try {
+      operations.flex(updated);
+    } catch (IOException e) {
+      throw convert(e);
+    }
+  }
+
+  
+  @Override
+  public AggregateConf getResolvedModel() throws IOException {
+    try {
+      return operations.getModelResolved();
+    } catch (IOException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public ConfTreeOperations getResolvedAppconf() throws IOException {
+    try {
+      return operations.getModelResolvedAppconf();
+    } catch (IOException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public ConfTreeOperations getResolvedResources() throws IOException {
+    try {
+      return operations.getModelResolvedResources();
+    } catch (IOException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public ConfTreeOperations getLiveResources() throws IOException {
+    try {
+      return operations.getLiveResources();
+    } catch (IOException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public Map<String, ContainerInformation> enumContainers() throws IOException {
+    try {
+      return operations.enumContainers();
+    } catch (IOException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public ContainerInformation getContainer(String containerId) throws
+      IOException {
+    try {
+      return operations.getContainer(containerId);
+    } catch (IOException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public Map<String, ComponentInformation> enumComponents() throws IOException {
+    try {
+      return operations.enumComponents();
+    } catch (IOException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public ComponentInformation getComponent(String componentName) throws IOException {
+    try {
+      return operations.getComponent(componentName);
+    } catch (IOException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public NodeInformationList getLiveNodes() throws IOException {
+    try {
+      return operations.getLiveNodes();
+    } catch (IOException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public NodeInformation getLiveNode(String hostname) throws IOException {
+    try {
+      return operations.getLiveNode(hostname);
+    } catch (IOException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public PingInformation ping(String text) throws IOException {
+    return null;
+  }
+
+  @Override
+  public void stop(String text) throws IOException {
+    try {
+      operations.stop(text);
+    } catch (IOException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public ApplicationLivenessInformation getApplicationLiveness() throws
+      IOException {
+    try {
+      return operations.getApplicationLiveness();
+    } catch (IOException e) {
+      throw convert(e);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "IPC implementation of SliderApplicationApi bonded to " + operations;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ipc/SliderClusterOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ipc/SliderClusterOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ipc/SliderClusterOperations.java
new file mode 100644
index 0000000..392f451
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/ipc/SliderClusterOperations.java
@@ -0,0 +1,529 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.client.ipc;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.slider.api.ClusterDescription;
+import org.apache.slider.api.ClusterNode;
+import org.apache.slider.api.SliderClusterProtocol;
+import org.apache.slider.api.StateValues;
+import org.apache.slider.api.proto.Messages;
+
+import static org.apache.slider.api.proto.RestTypeMarshalling.*;
+import org.apache.slider.api.types.ApplicationLivenessInformation;
+import org.apache.slider.api.types.ComponentInformation;
+import org.apache.slider.api.types.ContainerInformation;
+import org.apache.slider.api.types.NodeInformation;
+import org.apache.slider.api.types.NodeInformationList;
+import org.apache.slider.api.types.PingInformation;
+import org.apache.slider.common.tools.Duration;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTree;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.core.exceptions.NoSuchNodeException;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.core.exceptions.WaitTimeoutException;
+import org.apache.slider.core.persist.ConfTreeSerDeser;
+import org.apache.slider.server.services.security.SecurityStore;
+import org.apache.slider.server.services.security.SignCertResponse;
+import org.codehaus.jackson.JsonParseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Cluster operations at a slightly higher level than the RPC code
+ */
+public class SliderClusterOperations {
+  protected static final Logger
+    log = LoggerFactory.getLogger(SliderClusterOperations.class);
+  
+  private final SliderClusterProtocol appMaster;
+  private static final Messages.EmptyPayloadProto EMPTY;
+  static {
+    EMPTY = Messages.EmptyPayloadProto.newBuilder().build(); 
+  }
+
+  public SliderClusterOperations(SliderClusterProtocol appMaster) {
+    this.appMaster = appMaster;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder("SliderClusterOperations{");
+    sb.append("IPC binding=").append(appMaster);
+    sb.append('}');
+    return sb.toString();
+  }
+
+  /**
+   * Get a node from the AM
+   * @param uuid uuid of node
+   * @return deserialized node
+   * @throws IOException IO problems
+   * @throws NoSuchNodeException if the node isn't found
+   */
+  public ClusterNode getNode(String uuid)
+    throws IOException, NoSuchNodeException, YarnException {
+    Messages.GetNodeRequestProto req =
+      Messages.GetNodeRequestProto.newBuilder().setUuid(uuid).build();
+    Messages.GetNodeResponseProto node = appMaster.getNode(req);
+    return ClusterNode.fromProtobuf(node.getClusterNode());
+  }
+
+  /**
+   * Unmarshall a list of nodes from a protobud response
+   * @param nodes node list
+   * @return possibly empty list of cluster nodes
+   * @throws IOException
+   */
+  public List<ClusterNode> convertNodeWireToClusterNodes(List<Messages.RoleInstanceState> nodes)
+    throws IOException {
+    List<ClusterNode> nodeList = new ArrayList<>(nodes.size());
+    for (Messages.RoleInstanceState node : nodes) {
+      nodeList.add(ClusterNode.fromProtobuf(node));
+    }
+    return nodeList;
+  }
+
+  /**
+   * Echo text (debug action)
+   * @param text text
+   * @return the text, echoed back
+   * @throws YarnException
+   * @throws IOException
+   */
+  public String echo(String text) throws YarnException, IOException {
+    Messages.EchoRequestProto.Builder builder =
+      Messages.EchoRequestProto.newBuilder();
+    builder.setText(text);
+    Messages.EchoRequestProto req = builder.build();
+    Messages.EchoResponseProto response = appMaster.echo(req);
+    return response.getText();
+  }
+
+
+  /**
+   * Connect to a live cluster and get its current state
+   * @return its description
+   */
+  public ClusterDescription getClusterDescription()
+    throws YarnException, IOException {
+    
+    Messages.GetJSONClusterStatusRequestProto req =
+      Messages.GetJSONClusterStatusRequestProto.newBuilder().build();
+    Messages.GetJSONClusterStatusResponseProto resp =
+      appMaster.getJSONClusterStatus(req);
+    String statusJson = resp.getClusterSpec();
+    try {
+      return ClusterDescription.fromJson(statusJson);
+    } catch (JsonParseException e) {
+      log.error("Exception " + e + " parsing:\n" + statusJson, e);
+      throw e;
+    }
+  }
+
+  /**
+   * Get the AM instance definition.
+   * <p>
+   *   See {@link SliderClusterProtocol#getInstanceDefinition(Messages.GetInstanceDefinitionRequestProto)}
+   * @return current slider AM aggregate definition
+   * @throws YarnException
+   * @throws IOException
+   */
+  public AggregateConf getInstanceDefinition()
+    throws YarnException, IOException {
+    Messages.GetInstanceDefinitionRequestProto.Builder builder =
+      Messages.GetInstanceDefinitionRequestProto.newBuilder();
+
+    Messages.GetInstanceDefinitionRequestProto request = builder.build();
+    Messages.GetInstanceDefinitionResponseProto response =
+      appMaster.getInstanceDefinition(request);
+
+    ConfTreeSerDeser confTreeSerDeser = new ConfTreeSerDeser();
+
+    ConfTree internal = confTreeSerDeser.fromJson(response.getInternal());
+    ConfTree resources = confTreeSerDeser.fromJson(response.getResources());
+    ConfTree app = confTreeSerDeser.fromJson(response.getApplication());
+    AggregateConf instanceDefinition =
+      new AggregateConf(resources, app, internal);
+    return instanceDefinition;
+  }
+  /**
+   * Kill a container
+   * @param id container ID
+   * @return a success flag
+   * @throws YarnException
+   * @throws IOException
+   */
+  public boolean killContainer(String id) throws
+                                          YarnException,
+                                          IOException {
+    Messages.KillContainerRequestProto.Builder builder =
+      Messages.KillContainerRequestProto.newBuilder();
+    builder.setId(id);
+    Messages.KillContainerRequestProto req = builder.build();
+    Messages.KillContainerResponseProto response = appMaster.killContainer(req);
+    return response.getSuccess();
+  }
+
+  /**
+   * List all node UUIDs in a role
+   * @param role role name or "" for all
+   * @return an array of UUID strings
+   * @throws IOException
+   * @throws YarnException
+   */
+  public String[] listNodeUUIDsByRole(String role) throws IOException, YarnException {
+    Collection<String> uuidList = innerListNodeUUIDSByRole(role);
+    String[] uuids = new String[uuidList.size()];
+    return uuidList.toArray(uuids);
+  }
+
+  public List<String> innerListNodeUUIDSByRole(String role) throws IOException, YarnException {
+    Messages.ListNodeUUIDsByRoleRequestProto req =
+      Messages.ListNodeUUIDsByRoleRequestProto
+              .newBuilder()
+              .setRole(role)
+              .build();
+    Messages.ListNodeUUIDsByRoleResponseProto resp = appMaster.listNodeUUIDsByRole(req);
+    return resp.getUuidList();
+  }
+
+  /**
+   * List all nodes in a role. This is a double round trip: once to list
+   * the nodes in a role, another to get their details
+   * @param role
+   * @return an array of ContainerNode instances
+   * @throws IOException
+   * @throws YarnException
+   */
+  public List<ClusterNode> listClusterNodesInRole(String role)
+      throws IOException, YarnException {
+
+    Collection<String> uuidList = innerListNodeUUIDSByRole(role);
+    Messages.GetClusterNodesRequestProto req =
+      Messages.GetClusterNodesRequestProto
+              .newBuilder()
+              .addAllUuid(uuidList)
+              .build();
+    Messages.GetClusterNodesResponseProto resp = appMaster.getClusterNodes(req);
+    return convertNodeWireToClusterNodes(resp.getClusterNodeList());
+  }
+
+  /**
+   * Get the details on a list of uuids
+   * @param uuids instance IDs
+   * @return a possibly empty list of node details
+   * @throws IOException
+   * @throws YarnException
+   */
+  @VisibleForTesting
+  public List<ClusterNode> listClusterNodes(String[] uuids)
+      throws IOException, YarnException {
+
+    Messages.GetClusterNodesRequestProto req =
+      Messages.GetClusterNodesRequestProto
+              .newBuilder()
+              .addAllUuid(Arrays.asList(uuids))
+              .build();
+    Messages.GetClusterNodesResponseProto resp = appMaster.getClusterNodes(req);
+    return convertNodeWireToClusterNodes(resp.getClusterNodeList());
+  }
+
+  /**
+   * Wait for an instance of a named role to be live (or past it in the lifecycle)
+   * @param role role to look for
+   * @param timeout time to wait
+   * @return the state. If still in CREATED, the cluster didn't come up
+   * in the time period. If LIVE, all is well. If >LIVE, it has shut for a reason
+   * @throws IOException IO
+   * @throws SliderException Slider
+   * @throws WaitTimeoutException if the wait timed out
+   */
+  @VisibleForTesting
+  public int waitForRoleInstanceLive(String role, long timeout)
+    throws WaitTimeoutException, IOException, YarnException {
+    Duration duration = new Duration(timeout);
+    duration.start();
+    boolean live = false;
+    int state = StateValues.STATE_CREATED;
+
+    log.info("Waiting {} millis for a live node in role {}", timeout, role);
+    try {
+      while (!live) {
+        // see if there is a node in that role yet
+        List<String> uuids = innerListNodeUUIDSByRole(role);
+        String[] containers = uuids.toArray(new String[uuids.size()]);
+        int roleCount = containers.length;
+        ClusterNode roleInstance = null;
+        if (roleCount != 0) {
+  
+          // if there is, get the node
+          roleInstance = getNode(containers[0]);
+          if (roleInstance != null) {
+            state = roleInstance.state;
+            live = state >= StateValues.STATE_LIVE;
+          }
+        }
+        if (!live) {
+          if (duration.getLimitExceeded()) {
+            throw new WaitTimeoutException(
+              String.format("Timeout after %d millis" +
+                            " waiting for a live instance of type %s; " +
+                            "instances found %d %s",
+                            timeout, role, roleCount,
+                            (roleInstance != null
+                             ? (" instance -\n" + roleInstance.toString())
+                             : "")
+                           ));
+          } else {
+            try {
+              Thread.sleep(1000);
+            } catch (InterruptedException ignored) {
+              // ignored
+            }
+          }
+        }
+      }
+    } finally {
+      duration.close();
+    }
+    return state;
+  }
+
+  /**
+   * Flex operation
+   * @param resources new resources
+   * @return the response
+   * @throws IOException
+   */
+  public boolean flex(ConfTree resources) throws IOException {
+    Messages.FlexClusterRequestProto request =
+      Messages.FlexClusterRequestProto.newBuilder()
+              .setClusterSpec(resources.toJson())
+              .build();
+    Messages.FlexClusterResponseProto response = appMaster.flexCluster(request);
+    return response.getResponse();
+  }
+
+
+  /**
+   * Commit (possibly delayed) AM suicide
+   *
+   * @param signal exit code
+   * @param text text text to log
+   * @param delay delay in millis
+   * @throws YarnException
+   * @throws IOException
+   */
+  public void amSuicide(String text, int signal, int delay)
+      throws IOException {
+    Messages.AMSuicideRequestProto.Builder builder =
+      Messages.AMSuicideRequestProto.newBuilder();
+    if (text != null) {
+      builder.setText(text);
+    }
+    builder.setSignal(signal);
+    builder.setDelay(delay);
+    Messages.AMSuicideRequestProto req = builder.build();
+    appMaster.amSuicide(req);
+  }
+
+  /**
+   * Get the application liveness
+   * @return current liveness information
+   * @throws IOException
+   */
+  public ApplicationLivenessInformation getLivenessInformation() throws IOException {
+    Messages.GetApplicationLivenessRequestProto.Builder builder =
+        Messages.GetApplicationLivenessRequestProto.newBuilder();
+    Messages.ApplicationLivenessInformationProto wire =
+        appMaster.getLivenessInformation(builder.build());
+    return unmarshall(wire);
+
+  }
+
+  public AggregateConf getModelDesired() throws IOException {
+    return unmarshallToAggregateConf(appMaster.getModelDesired(EMPTY));
+  }
+
+  
+  public ConfTreeOperations getModelDesiredAppconf() throws IOException {
+    return unmarshallToCTO(appMaster.getModelDesiredAppconf(EMPTY));
+  }
+
+  
+  public ConfTreeOperations getModelDesiredResources() throws IOException {
+    return unmarshallToCTO(appMaster.getModelDesiredResources(EMPTY));
+  }
+
+  
+  public AggregateConf getModelResolved() throws IOException {
+    return unmarshallToAggregateConf(appMaster.getModelResolved(EMPTY));
+  }
+
+  
+  public ConfTreeOperations getModelResolvedAppconf() throws IOException {
+    return unmarshallToCTO(appMaster.getModelResolvedAppconf(EMPTY));
+  }
+
+  
+  public ConfTreeOperations getModelResolvedResources() throws IOException {
+    return unmarshallToCTO(appMaster.getModelDesiredResources(EMPTY));
+  }
+
+  
+  public ConfTreeOperations getLiveResources() throws IOException {
+    return unmarshallToCTO(appMaster.getLiveResources(EMPTY));
+  }
+
+  
+  public Map<String, ContainerInformation> enumContainers() throws IOException {
+    Messages.GetLiveContainersResponseProto response =
+        appMaster.getLiveContainers(
+            Messages.GetLiveContainersRequestProto.newBuilder().build());
+
+    int namesCount = response.getNamesCount();
+    int records = response.getContainersCount();
+    if (namesCount != records) {
+      throw new IOException("Number of names returned (" + namesCount
+                      + ") does not match the number of records returned: " 
+                      + records);
+    }
+    Map<String, ContainerInformation> map = new HashMap<>(namesCount);
+    for (int i = 0; i < namesCount; i++) {
+      map.put(response.getNames(i), unmarshall(response.getContainers(i)));
+    }
+    return map;
+  }
+
+  
+  public ContainerInformation getContainer(String containerId) throws
+      IOException {
+    Messages.ContainerInformationProto response =
+        appMaster.getLiveContainer(
+            Messages.GetLiveContainerRequestProto.newBuilder()
+                                                 .setContainerId(containerId)
+                                                 .build());
+    return unmarshall(response);
+  }
+
+  public List<ContainerInformation> getContainers() throws IOException {
+    Messages.GetLiveContainersResponseProto response = appMaster
+        .getLiveContainers(Messages.GetLiveContainersRequestProto.newBuilder()
+                                                                 .build());
+    return unmarshall(response);
+  }
+
+  public Map<String, ComponentInformation> enumComponents() throws IOException {
+    Messages.GetLiveComponentsResponseProto response =
+        appMaster.getLiveComponents(
+            Messages.GetLiveComponentsRequestProto.newBuilder().build());
+
+    int namesCount = response.getNamesCount();
+    int records = response.getComponentsCount();
+    if (namesCount != records) {
+      throw new IOException(
+          "Number of names returned (" + namesCount + ")" +
+          " does not match the number of records returned: " + records);
+    }
+    Map<String, ComponentInformation> map = new HashMap<>(namesCount);
+    for (int i = 0; i < namesCount; i++) {
+      map.put(response.getNames(i), unmarshall(response.getComponents(i)));
+    }
+    return map;
+  }
+
+  public ComponentInformation getComponent(String componentName)
+      throws IOException {
+    Messages.GetLiveComponentRequestProto.Builder builder =
+        Messages.GetLiveComponentRequestProto.newBuilder();
+    builder.setName(componentName);
+    Messages.ComponentInformationProto proto = appMaster.getLiveComponent(builder.build());
+    return unmarshall(proto);
+  }
+
+  public NodeInformationList getLiveNodes() throws IOException {
+    Messages.GetLiveNodesResponseProto response =
+      appMaster.getLiveNodes(Messages.GetLiveNodesRequestProto.newBuilder().build());
+
+    int records = response.getNodesCount();
+    NodeInformationList nil = new NodeInformationList(records);
+    for (int i = 0; i < records; i++) {
+      nil.add(unmarshall(response.getNodes(i)));
+    }
+    return nil;
+  }
+
+  public NodeInformation getLiveNode(String hostname) throws IOException {
+    Messages.GetLiveNodeRequestProto.Builder builder =
+        Messages.GetLiveNodeRequestProto.newBuilder();
+    builder.setName(hostname);
+    return unmarshall(appMaster.getLiveNode(builder.build()));
+  }
+
+  public PingInformation ping(String text) throws IOException {
+    return null;
+  }
+
+  public void stop(String text) throws IOException {
+    amSuicide(text, 3, 0);
+  }
+
+  public ApplicationLivenessInformation getApplicationLiveness() throws
+      IOException {
+    Messages.ApplicationLivenessInformationProto proto =
+        appMaster.getLivenessInformation(
+            Messages.GetApplicationLivenessRequestProto.newBuilder().build()
+        );
+    return unmarshall(proto);
+  }
+
+  public byte[] getClientCertificateStore(String hostname, String clientId,
+      String password, String type) throws IOException {
+    Messages.GetCertificateStoreRequestProto.Builder
+        builder = Messages.GetCertificateStoreRequestProto.newBuilder();
+    if (hostname != null) {
+      builder.setHostname(hostname);
+    }
+    Messages.GetCertificateStoreRequestProto requestProto =
+        builder.setRequesterId(clientId)
+               .setPassword(password)
+               .setType(type)
+               .build();
+    Messages.GetCertificateStoreResponseProto response =
+        appMaster.getClientCertificateStore(requestProto);
+
+    return unmarshall(response);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/rest/BaseRestClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/rest/BaseRestClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/rest/BaseRestClient.java
new file mode 100644
index 0000000..d936a22
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/rest/BaseRestClient.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.client.rest;
+
+import com.google.common.base.Preconditions;
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientHandlerException;
+import com.sun.jersey.api.client.GenericType;
+import com.sun.jersey.api.client.UniformInterfaceException;
+import com.sun.jersey.api.client.WebResource;
+import org.apache.slider.core.exceptions.ExceptionConverter;
+import org.apache.slider.core.restclient.HttpVerb;
+import org.apache.slider.core.restclient.UgiJerseyBinding;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.ws.rs.core.MediaType;
+import java.io.IOException;
+import java.net.URI;
+
+
+/**
+ * This is a base class for Jersey REST clients in Slider.
+ * It supports the execution of operations \u2014with
+ * exceptions uprated to IOExceptions when needed.
+ * <p>
+ * Subclasses can use these operations to provide an API-like view
+ * of the REST model
+ */
+public class BaseRestClient  {
+  private static final Logger log =
+      LoggerFactory.getLogger(BaseRestClient.class);
+  private final Client client;
+
+  public BaseRestClient(
+      Client client) {
+    Preconditions.checkNotNull(client, "null jersey client");
+    this.client = client;
+  }
+
+  /**
+   * Get the jersey client
+   * @return jersey client
+   */
+  public Client getClient() {
+    return client;
+  }
+
+  /**
+   * Execute the operation. Failures are raised as IOException subclasses
+   * @param method method to execute
+   * @param resource resource to work against
+   * @param c class to build
+   * @param <T> type expected
+   * @return an instance of the type T
+   * @throws IOException on any failure
+   */
+  public <T> T exec(HttpVerb method, WebResource resource, Class<T> c)
+      throws IOException {
+    try {
+      Preconditions.checkArgument(c != null);
+      log.debug("{}} {}", method, resource.getURI());
+      return resource.accept(MediaType.APPLICATION_JSON_TYPE)
+              .method(method.getVerb(), c);
+    } catch (ClientHandlerException ex) {
+      throw ExceptionConverter.convertJerseyException(method.getVerb(),
+          resource.getURI().toString(),
+          ex);
+    } catch (UniformInterfaceException ex) {
+      throw UgiJerseyBinding.uprateFaults(method,
+          resource.getURI().toString(),
+          ex);
+    }
+  }
+
+  /**
+   * Execute the operation. Failures are raised as IOException subclasses
+   * @param method method to execute
+   * @param resource resource to work against
+   * @param t type to work with
+   * @param <T> type expected
+   * @return an instance of the type T
+   * @throws IOException on any failure
+   */
+  public <T> T exec(HttpVerb method, WebResource resource, GenericType<T> t)
+      throws IOException {
+    try {
+      Preconditions.checkArgument(t != null);
+      log.debug("{}} {}", method, resource.getURI());
+      resource.accept(MediaType.APPLICATION_JSON_TYPE);
+      return resource.method(method.getVerb(), t);
+    } catch (ClientHandlerException ex) {
+      throw ExceptionConverter.convertJerseyException(method.getVerb(),
+          resource.getURI().toString(),
+          ex);
+    } catch (UniformInterfaceException ex) {
+      throw UgiJerseyBinding.uprateFaults(method, resource.getURI().toString(),
+          ex);
+    }
+  }
+
+
+  /**
+   * Execute the  GET operation. Failures are raised as IOException subclasses
+   * @param resource resource to work against
+   * @param c class to build
+   * @param <T> type expected
+   * @return an instance of the type T
+   * @throws IOException on any failure
+   */
+  public <T> T get(WebResource resource, Class<T> c) throws IOException {
+    return exec(HttpVerb.GET, resource, c);
+  }
+
+  /**
+   * Create a Web resource from the client.
+   *
+   * @param u the URI of the resource.
+   * @return the Web resource.
+   */
+  public WebResource resource(URI u) {
+    return client.resource(u);
+  }
+
+  /**
+   * Create a Web resource from the client.
+   *
+   * @param u the URI of the resource.
+   * @return the Web resource.
+   */
+
+  public WebResource resource(String url) {
+    return client.resource(url);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/rest/RestClientFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/rest/RestClientFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/rest/RestClientFactory.java
new file mode 100644
index 0000000..4286596
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/rest/RestClientFactory.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.client.rest;
+
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.WebResource;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.slider.client.ClientRegistryBinder;
+import org.apache.slider.api.SliderApplicationApi;
+import org.apache.slider.core.registry.info.CustomRegistryConstants;
+
+import java.io.IOException;
+
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.SLIDER_PATH_APPLICATION;
+
+/**
+ * Factory for the Rest client; hides the lookup and instantiation.
+ * <p>
+ * 
+ */
+public class RestClientFactory {
+
+  private final ClientRegistryBinder binder;
+  private final Client jerseyClient;
+  private final String user, serviceclass, instance;
+
+  public RestClientFactory(RegistryOperations operations,
+      Client jerseyClient,
+      String user,
+      String serviceclass,
+      String instance) {
+    this.jerseyClient = jerseyClient;
+    this.user = user;
+    this.serviceclass = serviceclass;
+    this.instance = instance;
+    binder = new ClientRegistryBinder(operations);
+  }
+
+  /**
+   * Locate the AM
+   * @return a resource to the AM
+   * @throws IOException any failure to resolve to the AM
+   */
+  private WebResource locateAppmaster() throws IOException {
+    String restAPI = binder.lookupExternalRestAPI(user, serviceclass, instance,
+        CustomRegistryConstants.AM_REST_BASE);
+    return jerseyClient.resource(restAPI);
+  }
+
+  /**
+   * Locate the slider AM then instantiate a client instance against
+   * its Application API.
+   * @return the instance
+   * @throws IOException on any failure
+   */
+  public SliderApplicationApi createSliderAppApiClient() throws IOException {
+    WebResource appmaster = locateAppmaster();
+    return createSliderAppApiClient(appmaster);
+  }
+
+   /**
+   * Create a Slider application API client instance against
+   * its Application API.
+   * @param appmaster The AM to work against.
+   * @return the instance
+   * @throws IOException on any failure
+   */
+  public SliderApplicationApi createSliderAppApiClient(WebResource appmaster) {
+    WebResource appResource = appmaster.path(SLIDER_PATH_APPLICATION);
+    return new SliderApplicationApiRestClient(jerseyClient, appResource);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/rest/SliderApplicationApiRestClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/rest/SliderApplicationApiRestClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/rest/SliderApplicationApiRestClient.java
new file mode 100644
index 0000000..4283ee8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/rest/SliderApplicationApiRestClient.java
@@ -0,0 +1,326 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.client.rest;
+
+import com.google.common.base.Preconditions;
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientHandlerException;
+import com.sun.jersey.api.client.GenericType;
+import com.sun.jersey.api.client.UniformInterfaceException;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.representation.Form;
+import org.apache.commons.lang.StringUtils;
+import org.apache.slider.api.types.ApplicationLivenessInformation;
+import org.apache.slider.api.types.ComponentInformation;
+import org.apache.slider.api.types.ContainerInformation;
+import org.apache.slider.api.SliderApplicationApi;
+import org.apache.slider.api.types.NodeInformation;
+import org.apache.slider.api.types.NodeInformationList;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTree;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.core.exceptions.ExceptionConverter;
+import org.apache.slider.core.restclient.HttpVerb;
+import org.apache.slider.api.types.PingInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.ws.rs.core.MediaType;
+import java.io.IOException;
+import java.util.Map;
+
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.*;
+
+/**
+ * Implementation of the {@link SliderApplicationApi}
+ */
+public class SliderApplicationApiRestClient extends BaseRestClient
+      implements SliderApplicationApi {
+  private static final Logger log =
+      LoggerFactory.getLogger(SliderApplicationApiRestClient.class);
+  private WebResource appResource;
+
+  /**
+   * Create an instance
+   * @param jerseyClient jersey client for operations
+   * @param appResource resource of application API
+   */
+  public SliderApplicationApiRestClient(Client jerseyClient,
+      WebResource appResource) {
+    super(jerseyClient);
+    this.appResource = appResource;
+  }
+
+  /**
+   * Create an instance
+   * @param jerseyClient jersey client for operations
+   * @param appmaster URL of appmaster/proxy to AM
+   */
+  public SliderApplicationApiRestClient(Client jerseyClient, String appmaster) {
+    super(jerseyClient);
+    WebResource amResource = jerseyClient.resource(appmaster);
+    amResource.type(MediaType.APPLICATION_JSON);
+    this.appResource = amResource.path(SLIDER_PATH_APPLICATION);
+  }
+
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder("SliderApplicationApiRestClient{");
+    sb.append("appResource=").append(appResource);
+    sb.append('}');
+    return sb.toString();
+  }
+
+  /**
+   * Create a resource under the application path
+   * @param subpath path under application
+   * @return a resource under the application path
+   */
+  public WebResource applicationResource(String subpath) {
+    Preconditions.checkArgument(!StringUtils.isEmpty(subpath),
+        "empty path");
+    Preconditions.checkNotNull(appResource, "Null app resource");
+    return appResource.path(subpath);
+  }
+  
+  /**
+   * Get operation against a path under the Application
+   * @param <T> type expected
+   * @param subpath path
+   * @param c class to instantiate
+   * @return instance
+   * @throws IOException on any problem
+   */
+  public <T> T getApplicationResource(String subpath, Class<T> c)
+      throws IOException {
+    return appResourceOperation(HttpVerb.GET, subpath, c);
+  } 
+  
+  /**
+   * Get operation against a path under the Application
+   * @param <T> type expected
+   * @param subpath path
+   * @param t type info
+   * @return instance
+   * @throws IOException on any problem
+   */
+  public <T> T getApplicationResource(String subpath, GenericType<T> t)
+      throws IOException {
+    return appResourceOperation(HttpVerb.GET, subpath, t);
+  }
+
+  /**
+   * 
+   * @param method method to exec
+   * @param <T> type expected
+   * @param subpath path
+   * @param c class to instantiate
+   * @return instance
+   * @throws IOException on any problem
+   */
+  public <T> T appResourceOperation(HttpVerb method, String subpath, Class<T> c)
+      throws IOException {
+    return exec(method, applicationResource(subpath), c);
+  }
+  
+  
+  /**
+   * Get operation against a path under the Application
+   * @param <T> type expected
+   * @param subpath path
+   * @param t type info
+   * @return instance
+   * @throws IOException on any problem
+   */
+  public <T> T appResourceOperation(HttpVerb method, String subpath,
+      GenericType<T> t)
+      throws IOException {
+    return exec(method, applicationResource(subpath), t);
+  }
+
+
+  @Override
+  public AggregateConf getDesiredModel() throws IOException {
+    return getApplicationResource(MODEL_DESIRED, AggregateConf.class);
+  }
+  
+  @Override
+  public ConfTreeOperations getDesiredAppconf() throws IOException {
+    ConfTree resource =
+        getApplicationResource(MODEL_DESIRED_APPCONF, ConfTree.class);
+    return new ConfTreeOperations(resource); 
+  }
+
+  @Override
+  public ConfTreeOperations getDesiredResources() throws IOException {
+    ConfTree resource =
+        getApplicationResource(MODEL_DESIRED_RESOURCES, ConfTree.class);
+    return new ConfTreeOperations(resource); 
+  }
+
+  @Override
+  public void putDesiredResources(ConfTree updated) throws IOException {
+    WebResource resource = applicationResource(MODEL_DESIRED_RESOURCES);
+    try {
+
+      // put operation. The result is discarded; it does help validate
+      // that the operation returned a JSON data structure as well as a 200
+      // response.
+
+      resource.accept(MediaType.APPLICATION_JSON_TYPE)
+              .type(MediaType.APPLICATION_JSON_TYPE)
+              .entity(updated)
+              .put(ConfTree.class);
+    } catch (ClientHandlerException ex) {
+        throw ExceptionConverter.convertJerseyException("PUT",
+            resource.getURI().toString(),
+            ex);
+      } catch (UniformInterfaceException ex) {
+      throw ExceptionConverter.convertJerseyException("PUT",
+          resource.getURI().toString(), ex);
+      }
+  }
+
+  @Override
+  public AggregateConf getResolvedModel() throws IOException {
+    return getApplicationResource(MODEL_RESOLVED, AggregateConf.class);
+  }
+
+
+  @Override
+  public ConfTreeOperations getResolvedAppconf() throws IOException {
+    ConfTree resource =
+        getApplicationResource(MODEL_RESOLVED_APPCONF, ConfTree.class);
+    return new ConfTreeOperations(resource); 
+  }
+
+  @Override
+  public ConfTreeOperations getResolvedResources() throws IOException {
+    ConfTree resource =
+        getApplicationResource(MODEL_RESOLVED_RESOURCES, ConfTree.class);
+    return new ConfTreeOperations(resource); 
+  }
+
+  @Override
+  public ConfTreeOperations getLiveResources() throws IOException {
+    ConfTree resource =
+        getApplicationResource(LIVE_RESOURCES, ConfTree.class);
+    return new ConfTreeOperations(resource); 
+  }
+
+  @Override
+  public Map<String, ContainerInformation> enumContainers() throws
+      IOException {
+    return getApplicationResource(LIVE_CONTAINERS,
+        new GenericType<Map<String, ContainerInformation>>() {
+        });
+  }
+
+  @Override
+  public ContainerInformation getContainer(String containerId) throws
+      IOException {
+    return getApplicationResource(LIVE_CONTAINERS + "/" + containerId,
+        ContainerInformation.class);
+  }
+
+  @Override
+  public Map<String, ComponentInformation> enumComponents() throws
+      IOException {
+    return getApplicationResource(LIVE_COMPONENTS,
+        new GenericType<Map<String, ComponentInformation>>() { });
+  }
+
+  @Override
+  public ComponentInformation getComponent(String componentName) throws
+      IOException {
+    return getApplicationResource(LIVE_COMPONENTS + "/" + componentName,
+        ComponentInformation.class);
+  }
+
+  @Override
+  public NodeInformationList getLiveNodes() throws IOException {
+    return getApplicationResource(LIVE_NODES, NodeInformationList.class);
+  }
+
+  @Override
+  public NodeInformation getLiveNode(String hostname) throws IOException {
+    return getApplicationResource(LIVE_NODES + "/" + hostname,
+        NodeInformation.class);
+  }
+
+  @Override
+  public PingInformation ping(String text) throws IOException {
+    return pingPost(text);
+  }
+  
+  /**
+   * Ping as a GET
+   * @param text text to include
+   * @return the response
+   * @throws IOException on any failure
+   */
+  public PingInformation pingGet(String text) throws IOException {
+    WebResource pingResource = applicationResource(ACTION_PING);
+    pingResource.getUriBuilder().queryParam("body", text);
+    return pingResource.get(PingInformation.class);
+  }
+  
+  /**
+   * Ping as a POST
+   * @param text text to include
+   * @return the response
+   * @throws IOException on any failure
+   */
+  public PingInformation pingPost(String text) throws IOException {
+    WebResource pingResource = applicationResource(ACTION_PING);
+    Form f = new Form();
+    f.add("text", text);
+    return pingResource
+        .type(MediaType.APPLICATION_JSON_TYPE)
+        .post(PingInformation.class, f);
+  }
+  
+  /**
+   * Ping as a POST
+   * @param text text to include
+   * @return the response
+   * @throws IOException on any failure
+   */
+  public PingInformation pingPut(String text) throws IOException {
+    WebResource pingResource = applicationResource(ACTION_PING);
+    Form f = new Form();
+    return pingResource
+        .type(MediaType.TEXT_PLAIN)
+        .put(PingInformation.class, text);
+  }
+
+  @Override
+  public void stop(String text) throws IOException {
+    WebResource resource = applicationResource(ACTION_STOP);
+    resource.post(text);
+  }
+
+  @Override
+  public ApplicationLivenessInformation getApplicationLiveness() throws IOException {
+    return getApplicationResource(LIVE_LIVENESS,
+        ApplicationLivenessInformation.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/Constants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/Constants.java
new file mode 100644
index 0000000..0e3559a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/Constants.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common;
+
+public class Constants {
+  public static final int CONNECT_TIMEOUT = 10000;
+  public static final int RPC_TIMEOUT = 15000;
+
+  public static final String HADOOP_JAAS_DEBUG = "HADOOP_JAAS_DEBUG";
+  public static final String KRB5_CCNAME = "KRB5CCNAME";
+  public static final String JAVA_SECURITY_KRB5_CONF
+    = "java.security.krb5.conf";
+  public static final String JAVA_SECURITY_KRB5_REALM
+    = "java.security.krb5.realm";
+  public static final String SUN_SECURITY_KRB5_DEBUG
+    = "sun.security.krb5.debug";
+  public static final String SUN_SECURITY_SPNEGO_DEBUG
+    = "sun.security.spnego.debug";
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[73/76] [abbrv] hadoop git commit: YARN-5961. Generate native services protobuf classes during build. Contributed by Billie Rinaldi

Posted by ji...@apache.org.
YARN-5961. Generate native services protobuf classes during build. Contributed by Billie Rinaldi


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

Branch: refs/heads/yarn-native-services
Commit: fc259d505ddce1b474ffe9e5ba1c7664d7282282
Parents: 89e8708
Author: Jian He <ji...@apache.org>
Authored: Sat Dec 3 13:27:04 2016 -0800
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 .../hadoop-yarn-slider-core/pom.xml             |    65 +-
 .../org/apache/slider/api/proto/Messages.java   | 34473 -----------------
 .../slider/api/proto/SliderClusterAPI.java      |  2293 --
 3 files changed, 27 insertions(+), 36804 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc259d50/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
index 10cf6b1..7453d12 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
@@ -37,6 +37,33 @@
     
     <plugins>
       <plugin>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-maven-plugins</artifactId>
+        <executions>
+          <execution>
+            <id>compile-protoc</id>
+            <goals>
+              <goal>protoc</goal>
+            </goals>
+            <configuration>
+              <protocVersion>${protobuf.version}</protocVersion>
+              <protocCommand>${protoc.path}</protocCommand>
+              <imports>
+                <param>${basedir}/src/main/proto</param>
+              </imports>
+              <source>
+                <directory>${basedir}/src/main/proto</directory>
+                <includes>
+                  <include>SliderClusterMessages.proto</include>
+                  <include>SliderClusterProtocol.proto</include>
+                </includes>
+              </source>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+
+      <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-jar-plugin</artifactId>
         <executions>
@@ -85,9 +112,6 @@
           <excludes>
             <exclude>**/*.json</exclude>
             <exclude>src/main/resources/webapps/slideram/.keep</exclude>
-            <!-- protobuf generated classes -->
-            <exclude>src/main/java/org/apache/slider/api/proto/Messages.java</exclude>
-            <exclude>src/main/java/org/apache/slider/api/proto/SliderClusterAPI.java</exclude>
           </excludes>
         </configuration>
       </plugin>
@@ -362,41 +386,6 @@
         </plugins>
       </build>
     </profile>
-    <profile>
-      <id>compile-protobuf</id>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-maven-plugins</artifactId>
-            <executions>
-              <execution>
-                <id>compile-protoc</id>
-                <phase>generate-sources</phase>
-                <goals>
-                  <goal>protoc</goal>
-                </goals>
-                <configuration>
-                  <protocVersion>${protobuf.version}</protocVersion>
-                  <protocCommand>protoc</protocCommand>
-                  <imports>
-                    <param>${basedir}/src/main/proto</param>
-                  </imports>
-                  <source>
-                    <directory>${basedir}/src/main/proto</directory>
-                    <includes>
-                      <include>SliderClusterMessages.proto</include>
-                      <include>SliderClusterProtocol.proto</include>
-                    </includes>
-                  </source>
-                  <output>${basedir}/src/main/java</output>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
 
   </profiles>
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[55/76] [abbrv] hadoop git commit: YARN-5958. Fix ASF license warnings for slider core module. Contributed by Billie Rinaldi

Posted by ji...@apache.org.
YARN-5958. Fix ASF license warnings for slider core module. Contributed by Billie Rinaldi


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

Branch: refs/heads/yarn-native-services
Commit: 89e870806c584095e07ac74060611cc23e92eb18
Parents: 0c1ad96
Author: Gour Saha <go...@apache.org>
Authored: Thu Dec 1 17:45:44 2016 -0800
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 .../hadoop-yarn-slider-core/pom.xml             | 61 +++++---------------
 .../src/license/THIRD-PARTY.properties          | 33 -----------
 2 files changed, 14 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/89e87080/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
index 66e9ee9..10cf6b1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
@@ -78,6 +78,20 @@
         </executions>
       </plugin>
 
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <configuration>
+          <excludes>
+            <exclude>**/*.json</exclude>
+            <exclude>src/main/resources/webapps/slideram/.keep</exclude>
+            <!-- protobuf generated classes -->
+            <exclude>src/main/java/org/apache/slider/api/proto/Messages.java</exclude>
+            <exclude>src/main/java/org/apache/slider/api/proto/SliderClusterAPI.java</exclude>
+          </excludes>
+        </configuration>
+      </plugin>
+
     </plugins>
   </build>
   <dependencies>
@@ -384,53 +398,6 @@
       </build>
     </profile>
 
-
-    <profile>
-      <id>rat</id>
-      <build>
-        <plugins>
-
-          <plugin>
-            <groupId>org.apache.rat</groupId>
-            <artifactId>apache-rat-plugin</artifactId>
-            <executions>
-              <execution>
-                <id>check-licenses</id>
-                <goals>
-                  <goal>check</goal>
-                </goals>
-              </execution>
-            </executions>
-            <configuration>
-              <excludes>
-                <exclude>**/*.json</exclude>
-                <exclude>src/test/python/agent.ini</exclude>
-                <exclude>src/test/python/version</exclude>
-                <exclude>**/THIRD-PARTY.properties</exclude>
-                <exclude>src/main/resources/webapps/slideram/.keep</exclude>
-                <exclude>src/main/resources/webapps/slideragent/.keep</exclude>
-                <exclude>src/main/resources/webapps/static/yarn.dt.plugins.js</exclude>
-                <!-- jQuery DataTables files (BSD license) -->
-                <exclude>src/main/resources/webapps/static/dt-1.9.4/**</exclude>
-                <!-- jQuery (MIT license) -->
-                <exclude>src/main/resources/webapps/static/jquery/jquery-1.8.2.min.js</exclude>
-                <!-- jQuery UI (MIT license) -->
-                <exclude>src/main/resources/webapps/static/jquery/jquery-ui-1.9.1.custom.min.js</exclude>
-                <exclude>src/main/resources/webapps/static/jquery/themes-1.9.1/base/jquery-ui.css</exclude>
-                <!-- jQuery jsTree (MIT license) -->
-                <exclude>src/main/resources/webapps/static/jt/jquery.jstree.js</exclude>
-                <!-- protobuf generated classes -->
-                <exclude>src/main/java/org/apache/slider/api/proto/Messages.java</exclude>
-                <exclude>src/main/java/org/apache/slider/api/proto/SliderClusterAPI.java</exclude>
-                <exclude>src/test/app_packages/test_am_config/resources/test.template</exclude>
-                <exclude>src/test/app_packages/test_am_config/test_archive/testfile</exclude>
-              </excludes>
-            </configuration>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-
   </profiles>
 
 </project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89e87080/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/license/THIRD-PARTY.properties
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/license/THIRD-PARTY.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/license/THIRD-PARTY.properties
deleted file mode 100644
index 1abd56e..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/license/THIRD-PARTY.properties
+++ /dev/null
@@ -1,33 +0,0 @@
-# Generated by org.codehaus.mojo.license.AddThirdPartyMojo
-#-------------------------------------------------------------------------------
-# Already used licenses in project :
-# - Apache License
-# - BSD
-# - CDDL + GPLv2 with classpath exception
-# - CDDL 1.1
-# - CDDL License
-# - CDDL+GPL
-# - Common Public License Version 1.0
-# - Eclipse Public License - Version 1.0
-# - GNU Lesser General Public License (LGPL), Version 2.1
-# - GNU Lesser General Public License, Version 2.1
-# - GPL2 w/ CPE
-# - MIT License
-# - MPL 1.1
-# - New BSD License
-# - Public Domain
-# - Revised BSD
-# - The Apache Software License, Version 2.0
-# - The BSD 3-Clause License
-# - The BSD License
-# - The MIT License
-#-------------------------------------------------------------------------------
-# Please fill the missing licenses for dependencies :
-#
-#
-#Thu Oct 15 16:45:02 EDT 2015
-commons-beanutils--commons-beanutils--1.7.0=The Apache Software License, Version 2.0
-javax.servlet--servlet-api--2.5=CDDL License
-javax.servlet.jsp--jsp-api--2.1=CDDL License
-org.apache.zookeeper--zookeeper--3.4.6=The Apache Software License, Version 2.0
-org.codehaus.jettison--jettison--1.1=The Apache Software License, Version 2.0


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[17/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Component.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Component.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Component.java
new file mode 100644
index 0000000..78bb8c1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Component.java
@@ -0,0 +1,217 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.core.exceptions.SliderException;
+import org.codehaus.jackson.annotate.JsonProperty;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Component defined in master package metainfo.json
+ */
+public class Component extends AbstractComponent {
+
+  String category = CATEGORY_MASTER;
+  String publishConfig = Boolean.FALSE.toString();
+  String minInstanceCount = "0";
+  String maxInstanceCount;
+  String autoStartOnFailure = Boolean.FALSE.toString();
+  String appExports;
+  String compExports;
+  String type = TYPE_STANDARD;
+  List<ComponentExport> componentExports = new ArrayList<>();
+  List<DockerContainer> dockerContainers = new ArrayList<>();
+  List<ConfigFile> configFiles = new ArrayList<>();
+
+  public Component() {
+  }
+
+  public String getType() {
+    return type;
+  }
+
+  public void setType(String type) {
+    this.type = type;
+  }
+
+  public String getCategory() {
+    return category;
+  }
+
+  public void setCategory(String category) {
+    this.category = category;
+  }
+
+  public String getPublishConfig() {
+    return publishConfig;
+  }
+
+  public void setPublishConfig(String publishConfig) {
+    this.publishConfig = publishConfig;
+  }
+
+  public String getAutoStartOnFailure() {
+    return autoStartOnFailure;
+  }
+
+  public void setAutoStartOnFailure(String autoStartOnFailure) {
+    this.autoStartOnFailure = autoStartOnFailure;
+  }
+
+  public String getAppExports() {
+    return appExports;
+  }
+
+  public void setAppExports(String appExports) {
+    this.appExports = appExports;
+  }
+
+  public String getCompExports() {
+    return compExports;
+  }
+
+  public void setCompExports(String compExports) {
+    this.compExports = compExports;
+  }
+
+  public String getMinInstanceCount() {
+    return minInstanceCount;
+  }
+  
+  @JsonProperty("dockerContainers")
+  public List<DockerContainer> getDockerContainers() {
+     return this.dockerContainers;
+  }
+  
+  public Boolean getAutoStartOnFailureBoolean() {
+    if (SliderUtils.isUnset(getAutoStartOnFailure())) {
+      return Boolean.FALSE;
+    }
+
+    return Boolean.parseBoolean(getAutoStartOnFailure());
+  }
+
+  public int getMinInstanceCountInt() throws BadConfigException {
+    if (SliderUtils.isUnset(minInstanceCount)) {
+      return 0;
+    }
+
+    try {
+      return Integer.parseInt(minInstanceCount);
+    } catch (NumberFormatException nfe) {
+      throw new BadConfigException(nfe, "Invalid value for minInstanceCount for %s", name);
+    }
+  }
+
+  public int getMaxInstanceCountInt() throws BadConfigException {
+    if (SliderUtils.isUnset(maxInstanceCount)) {
+      return Integer.MAX_VALUE;
+    }
+
+    try {
+      return Integer.parseInt(maxInstanceCount);
+    } catch (NumberFormatException nfe) {
+      throw new BadConfigException(nfe, "Invalid value for maxInstanceCount for %s", name);
+    }
+  }
+
+  public void setMinInstanceCount(String minInstanceCount) {
+    this.minInstanceCount = minInstanceCount;
+  }
+
+  public String getMaxInstanceCount() {
+    return maxInstanceCount;
+  }
+
+  public void setMaxInstanceCount(String maxInstanceCount) {
+    this.maxInstanceCount = maxInstanceCount;
+  }
+
+  public void addComponentExport(ComponentExport export) {
+    componentExports.add(export);
+  }
+
+  public List<ComponentExport> getComponentExports() {
+    return componentExports;
+  }
+
+  public Boolean getRequiresAutoRestart() {
+    return Boolean.parseBoolean(this.autoStartOnFailure);
+  }
+
+  public void addConfigFile(ConfigFile configFile) {
+    this.configFiles.add(configFile);
+  }
+
+  @JsonProperty("configFiles")
+  public List<ConfigFile> getConfigFiles() {
+    return configFiles;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder("{");
+    sb.append("\n\"name\": ").append(name);
+    sb.append(",\n\"category\": ").append(category);
+    sb.append(",\n\"commandScript\" :").append(commandScript);
+    for(DockerContainer dc : dockerContainers){
+      sb.append(",\n\"container\" :").append(dc.toString());
+    }    
+    sb.append('}');
+    return sb.toString();
+  }
+
+  public void validate(String version) throws SliderException {
+    Metainfo.checkNonNull(getName(), "name", "component");
+    Metainfo.checkNonNull(getCategory(), "category", "component");
+    if (!getCategory().equals(CATEGORY_MASTER)
+        && !getCategory().equals(CATEGORY_SLAVE)
+        && !getCategory().equals(CATEGORY_CLIENT)) {
+      throw new SliderException("Invalid category for the component " + getCategory());
+    }
+
+    Metainfo.checkNonNull(getType(), "type", "component");
+    if (!getType().equals(TYPE_DOCKER)
+        && !getType().equals(TYPE_STANDARD)) {
+      throw new SliderException("Invalid type for the component " + getType());
+    }
+
+    if (version.equals(Metainfo.VERSION_TWO_ZERO)) {
+      if (getType().equals(TYPE_DOCKER)) {
+        throw new SliderException(TYPE_DOCKER + " is not supported in version " + Metainfo.VERSION_TWO_ZERO);
+      }
+
+      if (getCommands().size() > 0) {
+        throw new SliderException("commands are not supported in version " + Metainfo.VERSION_TWO_ZERO);
+      }
+    }
+
+    if (commandScript != null) {
+      commandScript.validate(version);
+    }
+
+    if (version.equals(Metainfo.VERSION_TWO_ONE)) {
+      for (ComponentCommand cc : getCommands()) {
+        cc.validate(version);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentCommand.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentCommand.java
new file mode 100644
index 0000000..52117c5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentCommand.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.apache.slider.core.exceptions.SliderException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Represents the metadata associated with the application.
+ */
+public class ComponentCommand implements Validate {
+  protected static final Logger
+      log = LoggerFactory.getLogger(ComponentCommand.class);
+
+
+  private String exec;
+  private String name = "START";
+  private String type = "SHELL";
+
+  /**
+   * Creator.
+   */
+  public ComponentCommand() {
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setExec(String exec) {
+    this.exec = exec;
+  }
+
+  public String getExec() {
+    return exec;
+  }
+
+  public void setType(String type) {
+    this.type = type;
+  }
+
+  public String getType() {
+    return type;
+  }
+
+  public void validate(String version) throws SliderException {
+    Metainfo.checkNonNull(getName(), "name", "componentCommand");
+
+    Metainfo.checkNonNull(getType(), "version", "application");
+  }
+
+  public static ComponentCommand getDefaultComponentCommand() {
+    ComponentCommand cc = new ComponentCommand();
+    cc.setExec("DEFAULT");
+    return cc;
+  }
+
+  public static ComponentCommand getDefaultComponentCommand(String commandName) {
+    ComponentCommand cc = new ComponentCommand();
+    cc.setExec("DEFAULT");
+    cc.setName(commandName);
+    return cc;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentExport.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentExport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentExport.java
new file mode 100644
index 0000000..a18854c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentExport.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+/**
+ *
+ */
+public class ComponentExport {
+  String name;
+  String value;
+
+  public ComponentExport() {
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public String getValue() {
+    return value;
+  }
+
+  public void setValue(String value) {
+    this.value = value;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder("{");
+    sb.append(",\n\"name\": ").append(name);
+    sb.append(",\n\"value\": ").append(value);
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentsInAddonPackage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentsInAddonPackage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentsInAddonPackage.java
new file mode 100644
index 0000000..855e5b6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentsInAddonPackage.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.apache.slider.core.exceptions.SliderException;
+
+public class ComponentsInAddonPackage extends AbstractComponent {
+
+  @Override
+  public void validate(String version) throws SliderException {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ConfigFile.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ConfigFile.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ConfigFile.java
new file mode 100644
index 0000000..cb47512
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ConfigFile.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.apache.slider.core.exceptions.SliderException;
+
+/**
+ *
+ */
+public class ConfigFile implements Validate {
+  String type;
+  String fileName;
+  String dictionaryName;
+
+  public ConfigFile() {
+  }
+
+  public String getType() {
+    return type;
+  }
+
+  public void setType(String type) {
+    this.type = type;
+  }
+
+  public String getFileName() {
+    return fileName;
+  }
+
+  public void setFileName(String fileName) {
+    this.fileName = fileName;
+  }
+
+  public String getDictionaryName() {
+    return dictionaryName;
+  }
+
+  public void setDictionaryName(String dictionaryName) {
+    this.dictionaryName = dictionaryName;
+  }
+
+  public void validate(String version) throws SliderException {
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DefaultConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DefaultConfig.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DefaultConfig.java
new file mode 100644
index 0000000..46c8836
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DefaultConfig.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Application default config
+ */
+public class DefaultConfig {
+  List<PropertyInfo> propertyInfos;
+
+  public DefaultConfig() {
+    propertyInfos = new ArrayList<PropertyInfo>();
+  }
+
+  public void addPropertyInfo(PropertyInfo propertyInfo) {
+    propertyInfos.add(propertyInfo);
+  }
+
+  public List<PropertyInfo> getPropertyInfos() {
+    return propertyInfos;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DefaultConfigParser.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DefaultConfigParser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DefaultConfigParser.java
new file mode 100644
index 0000000..e136775
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DefaultConfigParser.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.apache.commons.digester.Digester;
+import org.xml.sax.SAXException;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ *
+ */
+public class DefaultConfigParser {
+
+  public DefaultConfig parse(InputStream configFileStream) throws IOException {
+    Digester digester = new Digester();
+    digester.setValidating(false);
+
+    digester.addObjectCreate("configuration", DefaultConfig.class);
+
+    digester.addObjectCreate("*/property", PropertyInfo.class);
+    digester.addBeanPropertySetter("*/property/name");
+    digester.addBeanPropertySetter("*/property/value");
+    digester.addBeanPropertySetter("*/property/description");
+    digester.addSetNext("*/property", "addPropertyInfo");
+
+    try {
+      return (DefaultConfig) digester.parse(configFileStream);
+    } catch (IOException e) {
+
+    } catch (SAXException e) {
+
+    } finally {
+      configFileStream.close();
+    }
+
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainer.java
new file mode 100644
index 0000000..4c61e7f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainer.java
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.apache.slider.core.exceptions.SliderException;
+import org.codehaus.jackson.annotate.JsonProperty;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Represents a docker container
+ */
+public class DockerContainer implements Validate {
+  protected static final Logger
+      log = LoggerFactory.getLogger(DockerContainer.class);
+
+  private String name;
+  private String image;
+  private String network;
+  private String useNetworkScript;
+  private String options;
+  private List<DockerContainerMount> mounts = new ArrayList<>();
+  private List<DockerContainerPort> ports = new ArrayList<>();
+  private String statusCommand;
+  private String startCommand;
+  private String commandPath;
+  private String additionalParam;
+  private String runPrivilegedContainer;
+  private List<DockerContainerInputFile> inputFiles = new ArrayList<>();
+  private List<ConfigFile> configFiles = new ArrayList<>();
+
+  public DockerContainer() {
+  }
+
+  @JsonProperty("mounts")
+  public List<DockerContainerMount> getMounts() { return this.mounts; }
+
+  @JsonProperty("ports")
+  public List<DockerContainerPort> getPorts() {
+    return this.ports;
+  }
+
+  @JsonProperty("inputFiles")
+  public List<DockerContainerInputFile> getInputFiles() {
+    return this.inputFiles;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public String getImage() {
+    return image;
+  }
+
+  public void setImage(String image) {
+    this.image = image;
+  }
+
+  public String getNetwork() {
+    return network;
+  }
+
+  public void setNetwork(String network) {
+    this.network = network;
+  }
+
+  public String getUseNetworkScript() {
+    return useNetworkScript;
+  }
+
+  public void setUseNetworkScript(String useNetworkScript) {
+    this.useNetworkScript = useNetworkScript;
+  }
+
+  public String getOptions() {
+    return options;
+  }
+
+  public void setOptions(String options) {
+    this.options = options;
+  }
+
+  @Override
+  public void validate(String version) throws SliderException {
+    Metainfo.checkNonNull(getName(), "name", "dockerContainer");
+    Metainfo.checkNonNull(getImage(), "image", "dockerContainer");
+    for (DockerContainerMount dcm : getMounts()) {
+      dcm.validate(version);
+    }
+    for (DockerContainerPort dcp : getPorts()) {
+      dcp.validate(version);
+    }
+  }
+
+  @JsonProperty("statusCommand")
+  public String getStatusCommand() {
+    return statusCommand;
+  }
+
+  @JsonProperty("statusCommand")
+  public void setStatusCommand(String statusCommand) {
+    this.statusCommand = statusCommand;
+  }
+
+  public String getCommandPath() {
+    return commandPath;
+  }
+
+  public void setCommandPath(String commandPath) {
+    this.commandPath = commandPath;
+  }
+
+  public String getAdditionalParam() {
+    return additionalParam;
+  }
+
+  public void setAdditionalParam(String additionalParam) {
+    this.additionalParam = additionalParam;
+  }
+
+  @JsonProperty("startCommand")
+  public String getStartCommand() {
+    return startCommand;
+  }
+
+  @JsonProperty("startCommand")
+  public void setStartCommand(String startCommand) {
+    this.startCommand = startCommand;
+  }
+
+  @JsonProperty("runPrivilegedContainer")
+  public String getRunPrivilegedContainer() {
+    return runPrivilegedContainer;
+  }
+
+  @JsonProperty("runPrivilegedContainer")
+  public void setRunPrivilegedContainer(String runPrivilegedContainer) {
+    this.runPrivilegedContainer = runPrivilegedContainer;
+  }
+
+  public List<ConfigFile> getConfigFiles() {
+    return configFiles;
+  }
+
+  public void setConfigFiles(List<ConfigFile> configFiles) {
+    this.configFiles = configFiles;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder result = new StringBuilder("DockerContainer [name=")
+        .append(name).append(", image=").append(image).append(", options=")
+        .append(options).append(", mounts=").append(mounts).append(", ports=")
+        .append(ports).append(", statusCommand=").append(statusCommand)
+        .append(", commandPath=").append(commandPath)
+        .append(", additionalParam=").append(additionalParam)
+        .append(", inputFiles=").append(inputFiles).append(", startCommand=")
+        .append(startCommand).append(", runPriviledgedContainer=")
+        .append(runPrivilegedContainer).append(", configFiles=")
+        .append(configFiles).append("]");
+    return result.toString();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerInputFile.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerInputFile.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerInputFile.java
new file mode 100644
index 0000000..0faceb9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerInputFile.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DockerContainerInputFile {
+  protected static final Logger log = LoggerFactory
+      .getLogger(DockerContainerInputFile.class);
+
+  private String containerPath;
+  private String fileLocalPath;
+
+  public DockerContainerInputFile() {
+  }
+
+  public String getContainerMount() {
+    return containerPath;
+  }
+
+  public void setContainerMount(String containerMount) {
+    this.containerPath = containerMount;
+  }
+
+  public String getFileLocalPath() {
+    return fileLocalPath;
+  }
+
+  public void setFileLocalPath(String fileLocalPath) {
+    this.fileLocalPath = fileLocalPath;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerMount.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerMount.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerMount.java
new file mode 100644
index 0000000..61f07f4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerMount.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.apache.slider.core.exceptions.SliderException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Represents a docker container mount
+ */
+public class DockerContainerMount implements Validate {
+  protected static final Logger
+      log = LoggerFactory.getLogger(DockerContainerMount.class);
+
+
+  private String containerMount;
+  private String hostMount;
+
+  public DockerContainerMount() {
+  }
+
+  public String getContainerMount() {
+    return containerMount;
+  }
+
+  public void setContainerMount(String containerMount) {
+    this.containerMount = containerMount;
+  }
+
+  public String getHostMount() {
+    return hostMount;
+  }
+
+  public void setHostMount(String hostMount) {
+    this.hostMount = hostMount;
+  }
+
+  @Override
+  public void validate(String version) throws SliderException {
+    Metainfo.checkNonNull(getContainerMount(), "containerMount", "dockerContainerMount");
+    Metainfo.checkNonNull(getHostMount(), "hostMount", "dockerContainerMount");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerPort.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerPort.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerPort.java
new file mode 100644
index 0000000..0629d9d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerPort.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.apache.slider.core.exceptions.SliderException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Represents a docker container port
+ */
+public class DockerContainerPort implements Validate {
+  protected static final Logger
+      log = LoggerFactory.getLogger(DockerContainerPort.class);
+
+
+  private String containerPort;
+  private String hostPort;
+
+  public DockerContainerPort() {
+  }
+
+  public String getContainerPort() {
+    return containerPort;
+  }
+
+  public void setContainerPort(String containerPort) {
+    this.containerPort = containerPort;
+  }
+
+  public String getHostPort() {
+    return hostPort;
+  }
+
+  public void setHostPort(String hostPort) {
+    this.hostPort = hostPort;
+  }
+
+  @Override
+  public void validate(String version) throws SliderException {
+    Metainfo.checkNonNull(getContainerPort(), "containerPort", "dockerContainerPort");
+    Metainfo.checkNonNull(getHostPort(), "hostPort", "dockerContainerPort");
+  }
+
+  @Override
+  public String toString() {
+    return "DockerContainerPort [containerPort=" + containerPort
+         + ", hostPort=" + hostPort + "]";
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Export.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Export.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Export.java
new file mode 100644
index 0000000..5e0fb24
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Export.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.apache.slider.core.exceptions.SliderException;
+
+/**
+ *
+ */
+public class Export implements Validate {
+  String name;
+  String value;
+
+  public Export() {
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public String getValue() {
+    return value;
+  }
+
+  public void setValue(String value) {
+    this.value = value;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder("{");
+    sb.append(",\n\"name\": ").append(name);
+    sb.append(",\n\"value\": ").append(value);
+    sb.append('}');
+    return sb.toString();
+  }
+
+  public void validate(String version) throws SliderException {
+    Metainfo.checkNonNull(getName(), "name", "export");
+    Metainfo.checkNonNull(getValue(), "value", "export");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ExportGroup.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ExportGroup.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ExportGroup.java
new file mode 100644
index 0000000..3d9f34c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ExportGroup.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.apache.slider.core.exceptions.SliderException;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ *
+ */
+public class ExportGroup implements Validate {
+  String name;
+  List<Export> exports;
+
+  public ExportGroup() {
+    exports = new ArrayList<Export>();
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public void addExport(Export export) {
+    exports.add(export);
+  }
+
+  public List<Export> getExports() {
+    return exports;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder("{");
+    sb.append(",\n\"name\": ").append(name);
+    sb.append(",\n\"exports\" : {");
+    for (Export export : exports) {
+      sb.append("\n").append(export);
+    }
+    sb.append("\n},");
+    sb.append('}');
+    return sb.toString();
+  }
+
+  public void validate(String version) throws SliderException {
+    Metainfo.checkNonNull(getName(), "name", "exportGroup");
+    for(Export exp : getExports()) {
+      exp.validate(version);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Metainfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Metainfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Metainfo.java
new file mode 100644
index 0000000..10c497f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Metainfo.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.exceptions.SliderException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Application metainfo uber class
+ */
+public class Metainfo {
+  protected static final Logger log =
+      LoggerFactory.getLogger(Metainfo.class);
+  public static String VERSION_TWO_ZERO = "2.0";
+  public static String VERSION_TWO_ONE = "2.1";
+
+  String schemaVersion;
+  ApplicationPackage applicationPackage;
+  Application application;
+
+  public String getSchemaVersion() {
+    return schemaVersion;
+  }
+
+  public void setSchemaVersion(String schemaVersion) {
+    this.schemaVersion = schemaVersion;
+  }
+
+  public ApplicationPackage getApplicationPackage() {
+    return applicationPackage;
+  }
+
+  public void setApplicationPackage(ApplicationPackage pkg) {
+    this.applicationPackage = pkg;
+  }
+
+  public Application getApplication() {
+    return application;
+  }
+
+  public void setApplication(Application application) {
+    this.application = application;
+  }
+
+  public Component getApplicationComponent(String roleGroup) {
+    if (application == null) {
+      log.error("Malformed app definition: Expect application as the top level element for metainfo");
+    } else {
+      for (Component component : application.getComponents()) {
+        if (component.getName().equals(roleGroup)) {
+          return component;
+        }
+      }
+    }
+    return null;
+  }
+
+  public List<ConfigFile> getComponentConfigFiles(String roleGroup) {
+    List<ConfigFile> componentConfigFiles = new ArrayList<>();
+    componentConfigFiles.addAll(application.getConfigFiles());
+    Component component = getApplicationComponent(roleGroup);
+    if (component != null) {
+      componentConfigFiles.addAll(component.getConfigFiles());
+    }
+    return componentConfigFiles;
+  }
+
+  public void validate() throws SliderException {
+    if (!VERSION_TWO_ONE.equals(schemaVersion) &&
+        !VERSION_TWO_ZERO.equals(schemaVersion)) {
+      throw new SliderException("Unsupported version " + getSchemaVersion());
+    }
+    if (application != null) {
+      application.validate(schemaVersion);
+    }
+    if (applicationPackage != null) {
+      applicationPackage.validate(schemaVersion);
+    }
+  }
+
+  public static void checkNonNull(String value, String field, String type) throws SliderException {
+    if (SliderUtils.isUnset(value)) {
+      throw new SliderException(type + "." + field + " cannot be null");
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append("Metainfo [schemaVersion=");
+    builder.append(schemaVersion);
+    builder.append(", applicationPackage=");
+    builder.append(applicationPackage);
+    builder.append(", application=");
+    builder.append(application);
+    builder.append("]");
+    return builder.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/MetainfoParser.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/MetainfoParser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/MetainfoParser.java
new file mode 100644
index 0000000..8b520eb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/MetainfoParser.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.apache.commons.digester.Digester;
+
+/**
+ *
+ */
+public class MetainfoParser extends AbstractMetainfoParser{
+  
+  protected void composeSchema(Digester digester){
+    digester.addObjectCreate("metainfo", Metainfo.class);
+    digester.addBeanPropertySetter("metainfo/schemaVersion");
+
+    digester.addObjectCreate("*/application", Application.class);
+    digester.addBeanPropertySetter("*/application/name");
+    digester.addBeanPropertySetter("*/application/comment");
+    digester.addBeanPropertySetter("*/application/version");
+    digester.addBeanPropertySetter("*/application/exportedConfigs");
+
+    digester.addObjectCreate("*/commandOrder", CommandOrder.class);
+    digester.addBeanPropertySetter("*/commandOrder/command");
+    digester.addBeanPropertySetter("*/commandOrder/requires");
+    digester.addSetNext("*/commandOrder", "addCommandOrder");
+
+    digester.addObjectCreate("*/exportGroup", ExportGroup.class);
+    digester.addBeanPropertySetter("*/exportGroup/name");
+    digester.addObjectCreate("*/export", Export.class);
+    digester.addBeanPropertySetter("*/export/name");
+    digester.addBeanPropertySetter("*/export/value");
+    digester.addSetNext("*/export", "addExport");
+    digester.addSetNext("*/exportGroup", "addExportGroup");
+
+    digester.addObjectCreate("*/component", Component.class);
+    digester.addBeanPropertySetter("*/component/name");
+    digester.addBeanPropertySetter("*/component/category");
+    digester.addBeanPropertySetter("*/component/publishConfig");
+    digester.addBeanPropertySetter("*/component/minInstanceCount");
+    digester.addBeanPropertySetter("*/component/maxInstanceCount");
+    digester.addBeanPropertySetter("*/component/autoStartOnFailure");
+    digester.addBeanPropertySetter("*/component/appExports");
+    digester.addBeanPropertySetter("*/component/compExports");
+    digester.addObjectCreate("*/componentExport", ComponentExport.class);
+    digester.addBeanPropertySetter("*/componentExport/name");
+    digester.addBeanPropertySetter("*/componentExport/value");
+    digester.addSetNext("*/componentExport", "addComponentExport");
+    digester.addSetNext("*/component", "addComponent");
+
+    digester.addObjectCreate("*/commandScript", CommandScript.class);
+    digester.addBeanPropertySetter("*/commandScript/script");
+    digester.addBeanPropertySetter("*/commandScript/scriptType");
+    digester.addBeanPropertySetter("*/commandScript/timeout");
+    digester.addSetNext("*/commandScript", "addCommandScript");
+
+    digester.addObjectCreate("*/command", ComponentCommand.class);
+    digester.addBeanPropertySetter("*/command/exec");
+    digester.addBeanPropertySetter("*/command/name");
+    digester.addBeanPropertySetter("*/command/type");
+    digester.addSetNext("*/command", "addCommand");
+
+    digester.addObjectCreate("*/osSpecific", OSSpecific.class);
+    digester.addBeanPropertySetter("*/osSpecific/osType");
+    digester.addObjectCreate("*/osSpecific/packages/package", OSPackage.class);
+    digester.addBeanPropertySetter("*/osSpecific/packages/package/type");
+    digester.addBeanPropertySetter("*/osSpecific/packages/package/name");
+    digester.addSetNext("*/osSpecific/packages/package", "addOSPackage");
+    digester.addSetNext("*/osSpecific", "addOSSpecific");
+
+    digester.addObjectCreate("*/application/packages/package", Package.class);
+    digester.addBeanPropertySetter("*/application/packages/package/type");
+    digester.addBeanPropertySetter("*/application/packages/package/name");
+    digester.addSetNext("*/application/packages/package", "addPackage");
+
+    digester.addObjectCreate("*/configFile", ConfigFile.class);
+    digester.addBeanPropertySetter("*/configFile/type");
+    digester.addBeanPropertySetter("*/configFile/fileName");
+    digester.addBeanPropertySetter("*/configFile/dictionaryName");
+    digester.addSetNext("*/configFile", "addConfigFile");
+
+    digester.addSetRoot("*/application", "setApplication");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSPackage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSPackage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSPackage.java
new file mode 100644
index 0000000..32b4890
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSPackage.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.apache.slider.core.exceptions.SliderException;
+
+/**
+ *
+ */
+public class OSPackage implements Validate {
+  String type;
+  String name;
+
+  public OSPackage() {
+  }
+
+  public String getType() {
+    return type;
+  }
+
+  public void setType(String type) {
+    this.type = type;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public void validate(String version) throws SliderException {
+    Metainfo.checkNonNull(getName(), "name", "osPackage");
+    Metainfo.checkNonNull(getType(), "type", "osPackage");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSSpecific.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSSpecific.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSSpecific.java
new file mode 100644
index 0000000..c06d498
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSSpecific.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.apache.slider.core.exceptions.SliderException;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ *
+ */
+public class OSSpecific implements Validate {
+  String osType;
+  List<OSPackage> packages;
+
+  public OSSpecific() {
+    packages = new ArrayList<OSPackage>();
+  }
+
+  public String getOsType() {
+    return osType;
+  }
+
+  public void setOsType(String osType) {
+    this.osType = osType;
+  }
+
+  public void addOSPackage(OSPackage osPackage) {
+    packages.add(osPackage);
+  }
+
+  public List<OSPackage> getPackages() {
+    return packages;
+  }
+
+  public void validate(String version) throws SliderException {
+    Metainfo.checkNonNull(getOsType(), "osType", "osSpecific");
+    for (OSPackage opkg : getPackages()) {
+      opkg.validate(version);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Package.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Package.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Package.java
new file mode 100644
index 0000000..b88f77d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Package.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.apache.slider.core.exceptions.SliderException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Represents package description.
+ */
+public class Package implements Validate {
+  protected static final Logger
+      log = LoggerFactory.getLogger(Package.class);
+
+
+  private String name;
+  private String type;
+
+  public Package() {
+  }
+
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public String getType() {
+    return type;
+  }
+
+  public void setType(String type) {
+    this.type = type;
+  }
+
+  public void validate(String version) throws SliderException {
+    Metainfo.checkNonNull(getName(), "name", "package");
+    Metainfo.checkNonNull(getType(), "type", "package");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/PropertyInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/PropertyInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/PropertyInfo.java
new file mode 100644
index 0000000..62ee0f5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/PropertyInfo.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+/**
+ * Application config property info
+ */
+public class PropertyInfo {
+  String name;
+  String value;
+  String description;
+
+  public PropertyInfo() {
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public String getValue() {
+    return value;
+  }
+
+  public void setValue(String value) {
+    this.value = value;
+  }
+
+  public String getDescription() {
+    return description;
+  }
+
+  public void setDescription(String description) {
+    this.description = description;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Validate.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Validate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Validate.java
new file mode 100644
index 0000000..ef03dcd
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Validate.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.apache.slider.core.exceptions.SliderException;
+
+/**
+ * Implementer provides a validate method
+ */
+public interface Validate {
+
+  public void validate(String version) throws SliderException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/todo.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/todo.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/todo.md
new file mode 100644
index 0000000..dfd1373
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/todo.md
@@ -0,0 +1,22 @@
+<!---
+   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.
+-->
+  
+# todo list
+
+* Retry on failure
+  * Agent can toleate a configurable number of failures (e.g. 3) before giving up
+* Agent should separate out hostname and label that is received for registration

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMClientProvider.java
new file mode 100644
index 0000000..e5430f2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMClientProvider.java
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers.slideram;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.slider.api.InternalKeys;
+import org.apache.slider.api.ResourceKeys;
+import org.apache.slider.api.RoleKeys;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.SliderXmlConfKeys;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.exceptions.BadClusterStateException;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.core.launch.AbstractLauncher;
+import org.apache.slider.core.launch.JavaCommandLineBuilder;
+import org.apache.slider.providers.AbstractClientProvider;
+import org.apache.slider.providers.PlacementPolicy;
+import org.apache.slider.providers.ProviderRole;
+import org.apache.slider.providers.ProviderUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.slider.api.ResourceKeys.COMPONENT_INSTANCES;
+
+/**
+ * handles the setup of the Slider AM.
+ * This keeps aspects of role, cluster validation and Clusterspec setup
+ * out of the core slider client
+ */
+public class SliderAMClientProvider extends AbstractClientProvider
+    implements SliderKeys {
+
+
+  protected static final Logger log =
+    LoggerFactory.getLogger(SliderAMClientProvider.class);
+  protected static final String NAME = "SliderAM";
+  public static final String INSTANCE_RESOURCE_BASE = PROVIDER_RESOURCE_BASE_ROOT +
+                                                       "slideram/instance/";
+  public static final String INTERNAL_JSON =
+    INSTANCE_RESOURCE_BASE + "internal.json";
+  public static final String APPCONF_JSON =
+    INSTANCE_RESOURCE_BASE + "appconf.json";
+  public static final String RESOURCES_JSON =
+    INSTANCE_RESOURCE_BASE + "resources.json";
+
+  public SliderAMClientProvider(Configuration conf) {
+    super(conf);
+  }
+
+  /**
+   * List of roles
+   */
+  public static final List<ProviderRole> ROLES =
+    new ArrayList<ProviderRole>();
+
+  public static final int KEY_AM = ROLE_AM_PRIORITY_INDEX;
+
+  public static final ProviderRole APPMASTER =
+      new ProviderRole(COMPONENT_AM, KEY_AM,
+          PlacementPolicy.EXCLUDE_FROM_FLEXING,
+          ResourceKeys.DEFAULT_NODE_FAILURE_THRESHOLD, 
+          0, "");
+
+  /**
+   * Initialize role list
+   */
+  static {
+    ROLES.add(APPMASTER);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+
+  @Override
+  public List<ProviderRole> getRoles() {
+    return ROLES;
+  }
+
+
+  @Override //Client
+  public void preflightValidateClusterConfiguration(SliderFileSystem sliderFileSystem,
+                                                    String clustername,
+                                                    Configuration configuration,
+                                                    AggregateConf instanceDefinition,
+                                                    Path clusterDirPath,
+                                                    Path generatedConfDirPath,
+                                                    boolean secure)
+      throws SliderException, IOException {
+
+    super.preflightValidateClusterConfiguration(sliderFileSystem, clustername, configuration, instanceDefinition, clusterDirPath, generatedConfDirPath, secure);
+    //add a check for the directory being writeable by the current user
+    String
+      dataPath = instanceDefinition.getInternalOperations()
+                                   .getGlobalOptions()
+                                   .getMandatoryOption(
+                                     InternalKeys.INTERNAL_DATA_DIR_PATH);
+
+    Path path = new Path(dataPath);
+    sliderFileSystem.verifyDirectoryWriteAccess(path);
+    Path historyPath = new Path(clusterDirPath, SliderKeys.HISTORY_DIR_NAME);
+    sliderFileSystem.verifyDirectoryWriteAccess(historyPath);
+  }
+
+  /**
+   * Verify that an instance definition is considered valid by the provider
+   * @param instanceDefinition instance definition
+   * @throws SliderException if the configuration is not valid
+   */
+  public void validateInstanceDefinition(AggregateConf instanceDefinition, SliderFileSystem fs) throws
+      SliderException {
+
+    super.validateInstanceDefinition(instanceDefinition, fs);
+    
+    // make sure there is no negative entry in the instance count
+    Map<String, Map<String, String>> instanceMap =
+        instanceDefinition.getResources().components;
+    for (Map.Entry<String, Map<String, String>> entry : instanceMap.entrySet()) {
+      MapOperations mapOperations = new MapOperations(entry);
+      int instances = mapOperations.getOptionInt(COMPONENT_INSTANCES, 0);
+      if (instances < 0) {
+        throw new BadClusterStateException(
+            "Component %s has negative instance count: %d",
+            mapOperations.name,
+            instances);
+      }
+    }
+  }
+  
+  /**
+   * The Slider AM sets up all the dependency JARs above slider.jar itself
+   * {@inheritDoc}
+   */
+  public void prepareAMAndConfigForLaunch(SliderFileSystem fileSystem,
+      Configuration serviceConf,
+      AbstractLauncher launcher,
+      AggregateConf instanceDescription,
+      Path snapshotConfDirPath,
+      Path generatedConfDirPath,
+      Configuration clientConfExtras,
+      String libdir,
+      Path tempPath, boolean miniClusterTestRun)
+    throws IOException, SliderException {
+
+    Map<String, LocalResource> providerResources = new HashMap<>();
+
+    ProviderUtils.addProviderJar(providerResources,
+        this,
+        SLIDER_JAR,
+        fileSystem,
+        tempPath,
+        libdir,
+        miniClusterTestRun);
+
+    String libDirProp =
+        System.getProperty(SliderKeys.PROPERTY_LIB_DIR);
+    log.info("Loading all dependencies for AM.");
+    // If slider.tar.gz is available in hdfs use it, else upload all jars
+    Path dependencyLibTarGzip = fileSystem.getDependencyTarGzip();
+    if (fileSystem.isFile(dependencyLibTarGzip)) {
+      SliderUtils.putAmTarGzipAndUpdate(providerResources, fileSystem);
+    } else {
+      ProviderUtils.addAllDependencyJars(providerResources,
+                                         fileSystem,
+                                         tempPath,
+                                         libdir,
+                                         libDirProp);
+    }
+    addKeytabResourceIfNecessary(fileSystem,
+                                 instanceDescription,
+                                 providerResources);
+
+    launcher.addLocalResources(providerResources);
+
+    //also pick up all env variables from a map
+    launcher.copyEnvVars(
+      instanceDescription.getInternalOperations().getOrAddComponent(
+        SliderKeys.COMPONENT_AM));
+  }
+
+  /**
+   * If the cluster is secure, and an HDFS installed keytab is available for AM
+   * authentication, add this keytab as a local resource for the AM launch.
+   *
+   * @param fileSystem
+   * @param instanceDescription
+   * @param providerResources
+   * @throws IOException
+   * @throws BadConfigException if there's no keytab and it is explicitly required.
+   */
+  protected void addKeytabResourceIfNecessary(SliderFileSystem fileSystem,
+                                              AggregateConf instanceDescription,
+                                              Map<String, LocalResource> providerResources)
+    throws IOException, BadConfigException {
+    if (UserGroupInformation.isSecurityEnabled()) {
+      String keytabPathOnHost = instanceDescription.getAppConfOperations()
+          .getComponent(SliderKeys.COMPONENT_AM).get(
+              SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH);
+      if (SliderUtils.isUnset(keytabPathOnHost)) {
+        String amKeytabName = instanceDescription.getAppConfOperations()
+            .getComponent(SliderKeys.COMPONENT_AM).get(
+                SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME);
+        String keytabDir = instanceDescription.getAppConfOperations()
+            .getComponent(SliderKeys.COMPONENT_AM).get(
+                SliderXmlConfKeys.KEY_HDFS_KEYTAB_DIR);
+        Path keytabPath = fileSystem.buildKeytabPath(keytabDir, amKeytabName,
+                                                     instanceDescription.getName());
+        if (fileSystem.getFileSystem().exists(keytabPath)) {
+          LocalResource keytabRes = fileSystem.createAmResource(keytabPath,
+                                                  LocalResourceType.FILE);
+
+          providerResources.put(SliderKeys.KEYTAB_DIR + "/" +
+                                 amKeytabName, keytabRes);
+        } else {
+          log.warn("No keytab file was found at {}.", keytabPath);
+          if (getConf().getBoolean(KEY_AM_LOGIN_KEYTAB_REQUIRED, false)) {
+            throw new BadConfigException("No keytab file was found at %s.", keytabPath);
+
+          } else {
+            log.warn("The AM will be "
+              + "started without a kerberos authenticated identity. "
+              + "The application is therefore not guaranteed to remain "
+              + "operational beyond 24 hours.");
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Update the AM resource with any local needs
+   * @param capability capability to update
+   */
+  public void prepareAMResourceRequirements(MapOperations sliderAM,
+                                            Resource capability) {
+    capability.setMemory(sliderAM.getOptionInt(
+      ResourceKeys.YARN_MEMORY,
+      capability.getMemory()));
+    capability.setVirtualCores(
+        sliderAM.getOptionInt(ResourceKeys.YARN_CORES, capability.getVirtualCores()));
+  }
+  
+  /**
+   * Extract any JVM options from the cluster specification and
+   * add them to the command line
+   */
+  public void addJVMOptions(AggregateConf aggregateConf,
+                            JavaCommandLineBuilder cmdLine)
+      throws BadConfigException {
+
+    MapOperations sliderAM =
+        aggregateConf.getAppConfOperations().getMandatoryComponent(
+        SliderKeys.COMPONENT_AM);
+    cmdLine.forceIPv4().headless();
+    String heap = sliderAM.getOption(RoleKeys.JVM_HEAP,
+                                   DEFAULT_JVM_HEAP);
+    cmdLine.setJVMHeap(heap);
+    String jvmopts = sliderAM.getOption(RoleKeys.JVM_OPTS, "");
+    if (SliderUtils.isSet(jvmopts)) {
+      cmdLine.add(jvmopts);
+    }
+  }
+
+
+  @Override
+  public void prepareInstanceConfiguration(AggregateConf aggregateConf)
+      throws SliderException, IOException {
+    mergeTemplates(aggregateConf,
+        INTERNAL_JSON, RESOURCES_JSON, APPCONF_JSON
+                  );
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[48/76] [abbrv] hadoop git commit: YARN-5943. Write native services container stderr file to log directory. Contributed by Billie Rinaldi

Posted by ji...@apache.org.
YARN-5943. Write native services container stderr file to log directory. Contributed by Billie Rinaldi


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

Branch: refs/heads/yarn-native-services
Commit: 5e0f1b73466f9f2c342b34dd35e03e0533360ff2
Parents: cd198b0
Author: Gour Saha <go...@apache.org>
Authored: Wed Nov 30 10:30:39 2016 -0800
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 .../org/apache/slider/providers/docker/DockerProviderService.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e0f1b73/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerProviderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerProviderService.java
index cc319ee..e4a7cdf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerProviderService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerProviderService.java
@@ -193,7 +193,8 @@ public class DockerProviderService extends AbstractProviderService implements
         "/bin/bash"));
 
     operation.add("> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/"
-        + OUT_FILE + " 2>" + ERR_FILE);
+        + OUT_FILE + " 2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/"
+        + ERR_FILE);
 
     launcher.addCommand(operation.build());
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[49/76] [abbrv] hadoop git commit: YARN-5775. Convert enums in swagger definition to uppercase. Contributed by Gour Saha

Posted by ji...@apache.org.
YARN-5775. Convert enums in swagger definition to uppercase. Contributed by Gour Saha


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

Branch: refs/heads/yarn-native-services
Commit: e7ce6b550b76cb0574036f371f5684dd9079c90d
Parents: 9c4a2d6
Author: Billie Rinaldi <bi...@apache.org>
Authored: Tue Oct 25 11:25:51 2016 -0700
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 ...RN-Simplified-V1-API-Layer-For-Services.yaml | 38 ++++++++++----------
 1 file changed, 19 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7ce6b55/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml
index 6169fcd..7eb3196 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml
@@ -241,10 +241,10 @@ definitions:
         type: string
         description: Artifact type, like docker, tarball, etc. (optional).
         enum:
-          - docker
-          - tarball
-          - application
-        default: docker
+          - DOCKER
+          - TARBALL
+          - APPLICATION
+        default: DOCKER
       uri:
         type: string
         description: Artifact location to support multiple artifact stores (optional).
@@ -303,7 +303,7 @@ definitions:
         type: string
         description: E.g. HTTP (YARN will perform a simple REST call at a regular interval and expect a 204 No content).
         enum:
-          - http
+          - HTTP
       uri:
         type: string
         description: Fully qualified REST uri endpoint.
@@ -335,13 +335,13 @@ definitions:
         type: string
         description: Config file in the standard format like xml, properties, json, yaml, template.
         enum:
-          - xml
-          - properties
-          - json
-          - yaml
-          - template
-          - env
-          - hadoop_xml
+          - XML
+          - PROPERTIES
+          - JSON
+          - YAML
+          - TEMPLATE
+          - ENV
+          - HADOOP_XML
       dest_file:
         type: string
         description: The absolute path that this configuration file should be mounted as, in the application container.
@@ -386,11 +386,11 @@ definitions:
         type: string
         description: enum of the state of the application
         enum:
-          - accepted
-          - started
-          - ready
-          - stopped
-          - failed
+          - ACCEPTED
+          - STARTED
+          - READY
+          - STOPPED
+          - FAILED
   ContainerState:
     description: The current state of the container of an application.
     properties:
@@ -398,8 +398,8 @@ definitions:
         type: string
         description: enum of the state of the container
         enum:
-          - init
-          - ready
+          - INIT
+          - READY
   ApplicationStatus:
     description: The current status of a submitted application, returned as a response to the GET API.
     properties:


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[14/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionFlexCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionFlexCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionFlexCluster.java
new file mode 100644
index 0000000..6b61681
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionFlexCluster.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.actions;
+
+import org.apache.slider.core.conf.ConfTree;
+import org.apache.slider.server.appmaster.SliderAppMaster;
+import org.apache.slider.server.appmaster.state.AppState;
+
+import java.util.concurrent.TimeUnit;
+
+public class ActionFlexCluster extends AsyncAction {
+
+  public final ConfTree resources;
+  
+  public ActionFlexCluster(String name,
+      long delay,
+      TimeUnit timeUnit, ConfTree resources) {
+    super(name, delay, timeUnit, ATTR_CHANGES_APP_SIZE);
+    this.resources = resources;
+  }
+
+  @Override
+  public void execute(SliderAppMaster appMaster,
+      QueueAccess queueService,
+      AppState appState) throws Exception {
+    appMaster.flexCluster(resources);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionHalt.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionHalt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionHalt.java
new file mode 100644
index 0000000..e2ad559
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionHalt.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.actions;
+
+import org.apache.hadoop.util.ExitUtil;
+import org.apache.slider.server.appmaster.SliderAppMaster;
+import org.apache.slider.server.appmaster.state.AppState;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Exit an emergency JVM halt.
+ * @see ExitUtil#halt(int, String) 
+ */
+public class ActionHalt extends AsyncAction {
+
+  private final int status;
+  private final String text;
+
+  public ActionHalt(
+      int status,
+      String text,
+      long delay, TimeUnit timeUnit) {
+    
+    // do not declare that this action halts the cluster ... keep it a surprise
+    super("Halt", delay, timeUnit);
+    this.status = status;
+    this.text = text;
+  }
+
+  @Override
+  public void execute(SliderAppMaster appMaster,
+      QueueAccess queueService,
+      AppState appState) throws Exception {
+    ExitUtil.halt(status, text);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionKillContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionKillContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionKillContainer.java
new file mode 100644
index 0000000..7446e82
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionKillContainer.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.actions;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.slider.server.appmaster.SliderAppMaster;
+import org.apache.slider.server.appmaster.operations.AbstractRMOperation;
+import org.apache.slider.server.appmaster.operations.ContainerReleaseOperation;
+import org.apache.slider.server.appmaster.operations.RMOperationHandler;
+import org.apache.slider.server.appmaster.operations.RMOperationHandlerActions;
+import org.apache.slider.server.appmaster.state.AppState;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Kill a specific container
+ */
+public class ActionKillContainer extends AsyncAction {
+
+  /**
+   *  container to kill
+   */
+  private final ContainerId containerId;
+
+  /**
+   *  handler for the operation
+   */
+  private final RMOperationHandlerActions operationHandler;
+
+  /**
+   * Kill a container
+   * @param containerId container to kill
+   * @param delay
+   * @param timeUnit
+   * @param operationHandler
+   */
+  public ActionKillContainer(
+      ContainerId containerId,
+      long delay,
+      TimeUnit timeUnit,
+      RMOperationHandlerActions operationHandler) {
+    super("kill container", delay, timeUnit, ATTR_CHANGES_APP_SIZE);
+    this.operationHandler = operationHandler;
+    Preconditions.checkArgument(containerId != null);
+    
+    this.containerId = containerId;
+  }
+
+  /**
+   * Get the container ID to kill
+   * @return
+   */
+  public ContainerId getContainerId() {
+    return containerId;
+  }
+
+  @Override
+  public void execute(SliderAppMaster appMaster,
+      QueueAccess queueService,
+      AppState appState) throws Exception {
+      List<AbstractRMOperation> opsList = new LinkedList<>();
+    ContainerReleaseOperation release = new ContainerReleaseOperation(containerId);
+    opsList.add(release);
+    //now apply the operations
+    operationHandler.execute(opsList);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionRegisterServiceInstance.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionRegisterServiceInstance.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionRegisterServiceInstance.java
new file mode 100644
index 0000000..ca330af
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionRegisterServiceInstance.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.actions;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.slider.server.appmaster.SliderAppMaster;
+import org.apache.slider.server.appmaster.state.AppState;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Asynchronous registration operation
+ */
+public class ActionRegisterServiceInstance extends AsyncAction {
+
+  private final String instanceName;
+  private final ApplicationId appId;
+
+  public ActionRegisterServiceInstance(String instanceName,
+      ApplicationId appId) {
+    super("ActionRegisterServiceInstance");
+    this.instanceName = instanceName;
+    this.appId = appId;
+  }
+
+  public ActionRegisterServiceInstance(String instanceName,
+      ApplicationId appId,
+      long delay,
+      TimeUnit timeUnit) {
+    super("ActionRegisterServiceInstance", delay, timeUnit);
+    this.instanceName = instanceName;
+    this.appId = appId;
+  }
+
+  @Override
+  public void execute(SliderAppMaster appMaster,
+      QueueAccess queueService,
+      AppState appState) throws Exception {
+
+    // YARN Registry do the registration
+    appMaster.registerServiceInstance(instanceName, appId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionStartContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionStartContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionStartContainer.java
new file mode 100644
index 0000000..358c844
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionStartContainer.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.actions;
+
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.slider.server.appmaster.SliderAppMaster;
+import org.apache.slider.server.appmaster.state.AppState;
+import org.apache.slider.server.appmaster.state.RoleInstance;
+
+import java.util.Locale;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Start a container
+ * @see SliderAppMaster#startContainer(Container, ContainerLaunchContext, RoleInstance) 
+ */
+public class ActionStartContainer extends AsyncAction {
+
+  private final Container container;
+  private final ContainerLaunchContext ctx;
+  private final RoleInstance instance;
+
+  public ActionStartContainer(String name,
+      Container container,
+      ContainerLaunchContext ctx,
+      RoleInstance instance,
+      long delay, TimeUnit timeUnit) {
+    super(
+        String.format(Locale.ENGLISH,
+            "%s %s: /",
+            name , container.getId().toString()), 
+        delay, 
+        timeUnit);
+    this.container = container;
+    this.ctx = ctx;
+    this.instance = instance;
+  }
+
+  @Override
+  public void execute(SliderAppMaster appMaster,
+      QueueAccess queueService,
+      AppState appState) throws Exception {
+    appMaster.startContainer(container, ctx, instance);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionStopQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionStopQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionStopQueue.java
new file mode 100644
index 0000000..08e8086
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionStopQueue.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.actions;
+
+import org.apache.slider.server.appmaster.SliderAppMaster;
+import org.apache.slider.server.appmaster.state.AppState;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Action to tell a queue executor to stop -after handing this on/executing it
+ */
+public class ActionStopQueue extends AsyncAction {
+  private static final Logger log =
+      LoggerFactory.getLogger(ActionStopQueue.class);
+  
+  public ActionStopQueue(long delay) {
+    super("stop queue", delay);
+  }
+
+  public ActionStopQueue(long delay,
+      TimeUnit timeUnit) {
+    super("stop queue", delay, timeUnit);
+  }
+
+  public ActionStopQueue(String name,
+      long delay,
+      TimeUnit timeUnit) {
+    super(name, delay, timeUnit);
+  }
+
+  @Override
+  public void execute(SliderAppMaster appMaster,
+      QueueAccess queueService,
+      AppState appState) throws Exception {
+    log.warn("STOP");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionStopSlider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionStopSlider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionStopSlider.java
new file mode 100644
index 0000000..055cea5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionStopSlider.java
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.actions;
+
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.slider.core.exceptions.ExceptionConverter;
+import org.apache.slider.core.exceptions.TriggerClusterTeardownException;
+import org.apache.slider.core.main.ExitCodeProvider;
+import org.apache.slider.core.main.LauncherExitCodes;
+import org.apache.slider.server.appmaster.SliderAppMaster;
+import org.apache.slider.server.appmaster.state.AppState;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Trigger an AM exit. This is used to build the exit status message for YARN
+ */
+public class ActionStopSlider extends AsyncAction {
+
+  private int exitCode;
+  private FinalApplicationStatus finalApplicationStatus;
+  private String message;
+  private final Exception ex;
+
+  /**
+   * Simple constructor
+   * @param name action name
+   */
+  public ActionStopSlider(String name) {
+    super(name);
+    this.ex = null;
+  }
+
+  /**
+   * Stop slider
+   * @param name action name
+   * @param delay execution delay
+   * @param timeUnit delay time unit
+   * @param exitCode process exit code
+   * @param finalApplicationStatus yarn status
+   * @param message message for AM
+   */
+  public ActionStopSlider(String name,
+      long delay,
+      TimeUnit timeUnit,
+      int exitCode,
+      FinalApplicationStatus finalApplicationStatus,
+      String message) {
+    super(name, delay, timeUnit, ATTR_HALTS_APP);
+    this.exitCode = exitCode;
+    this.finalApplicationStatus = finalApplicationStatus;
+    this.message = message;
+    this.ex = null;
+  }
+
+  /**
+   * Stop slider
+   * @param name action name
+   * @param exitCode process exit code
+   * @param finalApplicationStatus yarn status
+   * @param message message for AM
+   */
+  public ActionStopSlider(String name,
+      int exitCode,
+      FinalApplicationStatus finalApplicationStatus,
+    String message) {
+    super(name);
+    this.exitCode = exitCode;
+    this.finalApplicationStatus = finalApplicationStatus;
+    this.message = message;
+    this.ex = null;
+  }
+
+  /**
+   * Simple constructor
+   * @param ex teardown exception
+   */
+  public ActionStopSlider(TriggerClusterTeardownException ex) {
+    this("stop",
+        ex.getExitCode(),
+        ex.getFinalApplicationStatus(),
+        ex.getMessage());
+  }
+  
+  /**
+   * Build from an exception.
+   * <p>
+   * If the exception implements
+   * {@link ExitCodeProvider} then the exit code is extracted from that
+   * @param ex exception.
+   */
+  public ActionStopSlider(Exception ex) {
+    super("stop");
+    if (ex instanceof ExitCodeProvider) {
+      setExitCode(((ExitCodeProvider)ex).getExitCode());
+    } else {
+      setExitCode(LauncherExitCodes.EXIT_EXCEPTION_THROWN);
+    }
+    setFinalApplicationStatus(FinalApplicationStatus.FAILED);
+    setMessage(ex.getMessage());
+    this.ex = ex;
+  }
+  
+  @Override
+  public void execute(SliderAppMaster appMaster,
+      QueueAccess queueService,
+      AppState appState) throws Exception {
+    SliderAppMaster.getLog().info("SliderAppMasterApi.stopCluster: {}",
+        message);
+    appMaster.onAMStop(this);
+  }
+
+  @Override
+  public String toString() {
+    return String.format("%s:  exit code = %d, %s: %s;",
+        name, exitCode, finalApplicationStatus, message) ;
+  }
+
+  public int getExitCode() {
+    return exitCode;
+  }
+
+  public void setExitCode(int exitCode) {
+    this.exitCode = exitCode;
+  }
+
+  public FinalApplicationStatus getFinalApplicationStatus() {
+    return finalApplicationStatus;
+  }
+
+  public void setFinalApplicationStatus(FinalApplicationStatus finalApplicationStatus) {
+    this.finalApplicationStatus = finalApplicationStatus;
+  }
+
+  public String getMessage() {
+    return message;
+  }
+
+  public void setMessage(String message) {
+    this.message = message;
+  }
+
+  public Exception getEx() {
+    return ex;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionUpgradeContainers.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionUpgradeContainers.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionUpgradeContainers.java
new file mode 100644
index 0000000..05fcbcc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ActionUpgradeContainers.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.actions;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.slider.server.appmaster.SliderAppMaster;
+import org.apache.slider.server.appmaster.state.AppState;
+
+public class ActionUpgradeContainers extends AsyncAction {
+  private int exitCode;
+  private FinalApplicationStatus finalApplicationStatus;
+  private String message;
+  private Set<String> containers = new HashSet<>();
+  private Set<String> components = new HashSet<>();
+
+  public ActionUpgradeContainers(String name,
+      long delay,
+      TimeUnit timeUnit,
+      int exitCode,
+      FinalApplicationStatus finalApplicationStatus,
+      List<String> containers,
+      List<String> components,
+      String message) {
+    super(name, delay, timeUnit);
+    this.exitCode = exitCode;
+    this.finalApplicationStatus = finalApplicationStatus;
+    this.containers.addAll(containers);
+    this.components.addAll(components);
+    this.message = message;
+  }
+
+  @Override
+  public void execute(SliderAppMaster appMaster, QueueAccess queueService,
+      AppState appState) throws Exception {
+    if (CollectionUtils.isNotEmpty(this.containers)
+        || CollectionUtils.isNotEmpty(this.components)) {
+      SliderAppMaster.getLog().info("SliderAppMaster.upgradeContainers: {}",
+          message);
+      appMaster.onUpgradeContainers(this);
+    }
+  }
+
+  public int getExitCode() {
+    return exitCode;
+  }
+
+  public void setExitCode(int exitCode) {
+    this.exitCode = exitCode;
+  }
+
+  public FinalApplicationStatus getFinalApplicationStatus() {
+    return finalApplicationStatus;
+  }
+
+  public void setFinalApplicationStatus(
+      FinalApplicationStatus finalApplicationStatus) {
+    this.finalApplicationStatus = finalApplicationStatus;
+  }
+
+  public String getMessage() {
+    return message;
+  }
+
+  public void setMessage(String message) {
+    this.message = message;
+  }
+
+  public Set<String> getContainers() {
+    return containers;
+  }
+
+  public void setContainers(Set<String> containers) {
+    this.containers = containers;
+  }
+
+  public Set<String> getComponents() {
+    return components;
+  }
+
+  public void setComponents(Set<String> components) {
+    this.components = components;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/AsyncAction.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/AsyncAction.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/AsyncAction.java
new file mode 100644
index 0000000..f9a1fd5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/AsyncAction.java
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.actions;
+
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.server.appmaster.SliderAppMaster;
+import org.apache.slider.server.appmaster.state.AppState;
+
+import java.io.IOException;
+import java.util.concurrent.Delayed;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+public abstract class AsyncAction implements Delayed {
+
+  private static final AtomicLong sequencer = new AtomicLong(0);
+
+  public final String name;
+  private long nanos;
+  public final int attrs;
+  private final long sequenceNumber = sequencer.incrementAndGet();
+
+
+  protected AsyncAction(String name) {
+    this(name, 0);
+  }
+
+  protected AsyncAction(String name,
+      long delayMillis) {
+    this(name, delayMillis, TimeUnit.MILLISECONDS);
+  }
+
+  protected AsyncAction(String name,
+      long delay,
+      TimeUnit timeUnit) {
+    this(name, delay, timeUnit, 0);
+  }
+
+  protected AsyncAction(String name,
+      long delay,
+      TimeUnit timeUnit,
+      int attrs) {
+    this.name = name;
+    this.setNanos(convertAndOffset(delay, timeUnit));
+    this.attrs = attrs;
+  }
+
+  protected long convertAndOffset(long delay, TimeUnit timeUnit) {
+    return now() + TimeUnit.NANOSECONDS.convert(delay, timeUnit);
+  }
+
+  /**
+   * The current time in nanos
+   * @return now
+   */
+  protected long now() {
+    return System.nanoTime();
+  }
+
+  @Override
+  public long getDelay(TimeUnit unit) {
+    return unit.convert(getNanos() - now(), TimeUnit.NANOSECONDS);
+  }
+
+  @Override
+  public int compareTo(Delayed that) {
+    if (this == that) {
+      return 0;
+    }
+    return SliderUtils.compareTo(
+        getDelay(TimeUnit.NANOSECONDS),
+        that.getDelay(TimeUnit.NANOSECONDS));
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder(super.toString());
+    sb.append(" name='").append(name).append('\'');
+    sb.append(", delay=").append(getDelay(TimeUnit.SECONDS));
+    sb.append(", attrs=").append(attrs);
+    sb.append(", sequenceNumber=").append(sequenceNumber);
+    sb.append('}');
+    return sb.toString();
+  }
+
+  protected int getAttrs() {
+    return attrs;
+  }
+
+  /**
+   * Ask if an action has an of the specified bits set. 
+   * This is not an equality test.
+   * @param attr attribute
+   * @return true iff the action has any of the bits in the attr arg set
+   */
+  public boolean hasAttr(int attr) {
+    return (attrs & attr) != 0;
+  }
+
+  /**
+   * Actual application
+   * @param appMaster
+   * @param queueService
+   * @param appState
+   * @throws IOException
+   */
+  public abstract void execute(SliderAppMaster appMaster,
+      QueueAccess queueService, AppState appState) throws Exception;
+
+  public long getNanos() {
+    return nanos;
+  }
+
+  public void setNanos(long nanos) {
+    this.nanos = nanos;
+  }
+  
+  public static final int ATTR_CHANGES_APP_SIZE = 1;
+  public static final int ATTR_HALTS_APP = 2;
+  public static final int ATTR_REVIEWS_APP_SIZE = 4;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/EscalateOutstandingRequests.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/EscalateOutstandingRequests.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/EscalateOutstandingRequests.java
new file mode 100644
index 0000000..2c545ea
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/EscalateOutstandingRequests.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.actions;
+
+import org.apache.slider.server.appmaster.SliderAppMaster;
+import org.apache.slider.server.appmaster.state.AppState;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Escalate outstanding requests by asking AM
+ */
+public class EscalateOutstandingRequests extends AsyncAction {
+
+  public EscalateOutstandingRequests() {
+    super("EscalateOutstandingRequests");
+  }
+
+  public EscalateOutstandingRequests(long delay,
+      TimeUnit timeUnit) {
+    super("EscalateOutstandingRequests", delay, timeUnit, ATTR_REVIEWS_APP_SIZE);
+  }
+
+  @Override
+  public void execute(SliderAppMaster appMaster, QueueAccess queueService, AppState appState) throws
+      Exception {
+    appMaster.escalateOutstandingRequests();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ProviderReportedContainerLoss.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ProviderReportedContainerLoss.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ProviderReportedContainerLoss.java
new file mode 100644
index 0000000..41fe494
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ProviderReportedContainerLoss.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.actions;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.slider.server.appmaster.SliderAppMaster;
+import org.apache.slider.server.appmaster.state.AppState;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Report container loss to the AM
+ * {@link SliderAppMaster#providerLostContainer(ContainerId)}
+ */
+public class ProviderReportedContainerLoss extends AsyncAction {
+
+  private final ContainerId containerId;
+  
+  public ProviderReportedContainerLoss(ContainerId containerId) {
+    this("lost container", 0, TimeUnit.MILLISECONDS, containerId);
+  }
+
+  public ProviderReportedContainerLoss(String name,
+      long delay,
+      TimeUnit timeUnit,
+      ContainerId containerId) {
+    super(name, delay, timeUnit);
+    this.containerId = containerId;
+  }
+
+  @Override
+  public void execute(SliderAppMaster appMaster,
+      QueueAccess queueService,
+      AppState appState) throws Exception {
+    appMaster.providerLostContainer(containerId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ProviderStartupCompleted.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ProviderStartupCompleted.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ProviderStartupCompleted.java
new file mode 100644
index 0000000..957a35f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ProviderStartupCompleted.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.actions;
+
+import org.apache.slider.server.appmaster.SliderAppMaster;
+import org.apache.slider.server.appmaster.state.AppState;
+
+public class ProviderStartupCompleted extends AsyncAction {
+
+  public ProviderStartupCompleted() {
+    super("ProviderStartupCompleted");
+  }
+
+  @Override
+  public void execute(SliderAppMaster appMaster,
+      QueueAccess queueService,
+      AppState appState) throws Exception {
+    appMaster.eventCallbackEvent(null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueAccess.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueAccess.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueAccess.java
new file mode 100644
index 0000000..0396891
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueAccess.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.actions;
+
+/**
+ * Access for queue operations
+ */
+public interface QueueAccess {
+  /**
+   * Put an action on the immediate queue -to be executed when the queue
+   * reaches it.
+   * @param action action to queue
+   */
+  void put(AsyncAction action);
+
+  /**
+   * Put a delayed action: this will only be added to the main queue
+   * after its action time has been reached
+   * @param action action to queue
+   */
+  void schedule(AsyncAction action);
+
+  /**
+   * Remove an action from the queues.
+   * @param action action to remove
+   * @return true if the action was removed
+   */
+  boolean remove(AsyncAction action);
+
+  /**
+   * Add a named renewing action
+   * @param name name
+   * @param renewingAction wrapped action
+   */
+  void renewing(String name,
+      RenewingAction<? extends AsyncAction> renewingAction);
+
+  /**
+   * Look up a renewing action
+   * @param name name of the action
+   * @return the action or null if none was found
+   */
+  RenewingAction<? extends AsyncAction> lookupRenewingAction(String name);
+
+  /**
+   * Remove a renewing action
+   * @param name action name name of the action
+   * @return true if the action was found and removed.
+   */
+  boolean removeRenewingAction(String name);
+
+  /**
+   * Look in the immediate queue for any actions of a specific attribute
+   */
+  boolean hasQueuedActionWithAttribute(int attr);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueExecutor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueExecutor.java
new file mode 100644
index 0000000..d0fc2cf
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueExecutor.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.actions;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.slider.server.appmaster.SliderAppMaster;
+import org.apache.slider.server.appmaster.state.AppState;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * Executor for async actions - hands them off to the AM as 
+ * appropriate
+ */
+public class QueueExecutor implements Runnable {
+  private static final Logger log =
+      LoggerFactory.getLogger(QueueExecutor.class);
+
+  private final SliderAppMaster appMaster;
+  private final QueueService actionQueues;
+  private final AppState appState;
+
+
+  public QueueExecutor(SliderAppMaster appMaster,
+      QueueService actionQueues) {
+    Preconditions.checkNotNull(appMaster);
+    Preconditions.checkNotNull(actionQueues);
+
+    this.appMaster = appMaster;
+    this.actionQueues = actionQueues;
+    this.appState = appMaster.getAppState();
+  }
+
+  @VisibleForTesting
+  public QueueExecutor(QueueService actionQueues) {
+    Preconditions.checkNotNull(actionQueues);
+    this.appMaster = null;
+    this.appState = null;
+    this.actionQueues = actionQueues;
+  }
+
+  /**
+   * Run until the queue has been told to stop
+   */
+  @Override
+  public void run() {
+    AsyncAction take = null;
+    try {
+      log.info("Queue Executor run() started");
+      do {
+        take = actionQueues.actionQueue.take();
+        log.debug("Executing {}", take);
+        
+        take.execute(appMaster, actionQueues, appState);
+        log.debug("Completed {}", take);
+
+      } while (!(take instanceof ActionStopQueue));
+      log.info("Queue Executor run() stopped");
+    } catch (InterruptedException e) {
+      // interrupted: exit
+    } catch (Throwable e) {
+      log.error("Exception processing {}: {}", take, e, e);
+      if (appMaster != null) {
+        appMaster.onExceptionInThread(Thread.currentThread(), e);
+      }
+    }
+    // tag completed
+    actionQueues.complete();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueService.java
new file mode 100644
index 0000000..34acade
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueService.java
@@ -0,0 +1,202 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.actions;
+
+
+import org.apache.slider.server.services.workflow.ServiceThreadFactory;
+import org.apache.slider.server.services.workflow.WorkflowExecutorService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.BlockingDeque;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.DelayQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * The Queue service provides immediate and scheduled queues, as well
+ * as an executor thread that moves queued actions from the scheduled
+ * queue to the immediate one.
+ * 
+ * <p>
+ * This code to be revisited to see if all that was needed is the single scheduled
+ * queue, implicitly making actions immediate by giving them an execution
+ * time of "now". It would force having a sequence number to all actions, one 
+ * which the queue would have to set from its (monotonic, thread-safe) counter
+ * on every submission, with a modified comparison operator. This would guarantee
+ * that earlier submissions were picked before later ones.
+ */
+public class QueueService extends WorkflowExecutorService<ExecutorService>
+implements Runnable, QueueAccess {
+  private static final Logger log =
+      LoggerFactory.getLogger(QueueService.class);
+  public static final String NAME = "Action Queue";
+  private final AtomicBoolean completed = new AtomicBoolean(false);
+
+  /**
+   * Immediate actions.
+   */
+  public final BlockingDeque<AsyncAction> actionQueue =
+      new LinkedBlockingDeque<>();
+
+  /**
+   * Actions to be scheduled in the future
+   */
+  public final DelayQueue<AsyncAction> scheduledActions = new DelayQueue<>();
+
+  /**
+   * Map of renewing actions by name ... this is to allow them to 
+   * be cancelled by name
+   */
+  private final Map<String, RenewingAction<? extends AsyncAction>> renewingActions
+      = new ConcurrentHashMap<>();
+
+  /**
+   * Create a queue instance with a single thread executor
+   */
+  public QueueService() {
+    super(NAME,
+        ServiceThreadFactory.singleThreadExecutor(NAME, true));
+  }
+
+  @Override
+  public void put(AsyncAction action) {
+    log.debug("Queueing {}", action);
+    actionQueue.add(action);
+  }
+
+  @Override
+  public void schedule(AsyncAction action) {
+    log.debug("Scheduling {}", action);
+    scheduledActions.add(action);
+  }
+
+  @Override
+  public boolean remove(AsyncAction action) {
+    boolean removedFromDelayQueue = scheduledActions.remove(action);
+    boolean removedFromActions = actionQueue.remove(action);
+    return removedFromActions || removedFromDelayQueue;
+  }
+  
+  @Override
+  public void renewing(String name,
+      RenewingAction<? extends AsyncAction> renewingAction) {
+    log.debug("Adding renewing Action \"{}\": {}", name,
+        renewingAction.getAction());
+    if (removeRenewingAction(name)) {
+      log.debug("Removed predecessor action");
+    }
+    renewingActions.put(name, renewingAction);
+    schedule(renewingAction);
+  } 
+
+  @Override
+  public RenewingAction<? extends AsyncAction> lookupRenewingAction(String name) {
+    return renewingActions.get(name);
+  }
+
+  @Override
+  public boolean removeRenewingAction(String name) {
+    RenewingAction<? extends AsyncAction> action = renewingActions.remove(name);
+     return action != null && remove(action);
+  }
+  
+  /**
+   * Stop the service by scheduling an {@link ActionStopQueue} action
+   * ..if the processor thread is working this will propagate through
+   * and stop the queue handling after all other actions complete.
+   * @throws Exception
+   */
+  @Override
+  protected void serviceStop() throws Exception {
+    ActionStopQueue stopQueue = new ActionStopQueue("serviceStop: "+ this,
+        0, TimeUnit.MILLISECONDS);
+    schedule(stopQueue);
+    super.serviceStop();
+  }
+
+  /**
+   * Flush an action queue of all types of a specific action
+   * @param clazz 
+   */
+  protected void flushActionQueue(Class<? extends AsyncAction> clazz) {
+    Iterator<AsyncAction> iterator =
+        actionQueue.descendingIterator();
+    while (iterator.hasNext()) {
+      AsyncAction next = iterator.next();
+      if (next.getClass().equals(clazz)) {
+        iterator.remove();
+      }
+    }
+  }
+
+  @Override
+  public boolean hasQueuedActionWithAttribute(int attr) {
+    for (AsyncAction action : actionQueue) {
+      if (action.hasAttr(attr)) {
+        return true;
+      }
+    }
+    return false;
+  }
+  
+  /**
+   * Run until the queue has been told to stop
+   */
+  @Override
+  public void run() {
+    try {
+
+      log.info("QueueService processor started");
+
+      AsyncAction take;
+      do {
+        take = scheduledActions.take();
+        log.debug("Propagating {}", take);
+        actionQueue.put(take);
+      } while (!(take instanceof ActionStopQueue));
+      log.info("QueueService processor terminated");
+    } catch (InterruptedException e) {
+      // interrupted during actions
+    }
+    // the thread exits, but does not tag the service as complete. That's expected
+    // to be done by the stop queue
+  }
+
+
+  /**
+   * Check to see if the queue executor has completed
+   * @return the status
+   */
+  public boolean isCompleted() {
+    return completed.get();
+  }
+
+  /**
+   * Package scoped method to mark the queue service as finished
+   */
+  void complete() {
+    completed.set(true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/RegisterComponentInstance.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/RegisterComponentInstance.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/RegisterComponentInstance.java
new file mode 100644
index 0000000..4cf4981
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/RegisterComponentInstance.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.actions;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.slider.server.appmaster.SliderAppMaster;
+import org.apache.slider.server.appmaster.state.AppState;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Notify the app master that it should register a component instance
+ * in the registry
+ * {@link SliderAppMaster#registerComponent(ContainerId)}
+ */
+public class RegisterComponentInstance extends AsyncAction {
+
+  public final ContainerId containerId;
+  public final String description;
+  public final String type;
+
+  public RegisterComponentInstance(ContainerId containerId,
+      String description,
+      String type,
+      long delay,
+      TimeUnit timeUnit) {
+    super("RegisterComponentInstance :" + containerId,
+        delay, timeUnit);
+    this.description = description;
+    this.type = type;
+    Preconditions.checkArgument(containerId != null);
+    this.containerId = containerId;
+  }
+
+  @Override
+  public void execute(SliderAppMaster appMaster,
+      QueueAccess queueService,
+      AppState appState) throws Exception {
+
+    appMaster.registerComponent(containerId, description, type);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/RenewingAction.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/RenewingAction.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/RenewingAction.java
new file mode 100644
index 0000000..f3143ea
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/RenewingAction.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.actions;
+
+import com.google.common.base.Preconditions;
+import org.apache.slider.server.appmaster.SliderAppMaster;
+import org.apache.slider.server.appmaster.state.AppState;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+/**
+ * This action executes then reschedules an inner action; a limit
+ * can specify the number of times to run
+ */
+
+public class RenewingAction<A extends AsyncAction> extends AsyncAction {
+  private static final Logger log =
+      LoggerFactory.getLogger(RenewingAction.class);
+  private final A action;
+  private long interval;
+  private TimeUnit timeUnit;
+  public final AtomicInteger executionCount = new AtomicInteger();
+  private final ReentrantReadWriteLock intervalLock = new ReentrantReadWriteLock();
+  private final Lock intervalReadLock = intervalLock.readLock();
+  private final Lock intervalWriteLock = intervalLock.writeLock();
+  public final int limit;
+
+
+  /**
+   * Rescheduling action
+   * @param action action to execute
+   * @param initialDelay initial delay
+   * @param interval interval for later delays
+   * @param timeUnit time unit for all times
+   * @param limit limit on the no. of executions. If 0 or less: no limit
+   */
+  public RenewingAction(A action,
+      long initialDelay,
+      long interval,
+      TimeUnit timeUnit,
+      int limit) {
+    super("renewing " + action.name, initialDelay, timeUnit, action.getAttrs());
+    Preconditions.checkArgument(interval > 0, "invalid interval: " + interval);
+    this.action = action;
+    this.interval = interval;
+    this.timeUnit = timeUnit;
+    this.limit = limit;
+  }
+
+  /**
+   * Execute the inner action then reschedule ourselves
+   * @param appMaster
+   * @param queueService
+   * @param appState
+   * @throws Exception
+   */
+  @Override
+  public void execute(SliderAppMaster appMaster,
+      QueueAccess queueService,
+      AppState appState)
+      throws Exception {
+    long exCount = executionCount.incrementAndGet();
+    log.debug("{}: Executing inner action count # {}", this, exCount);
+    action.execute(appMaster, queueService, appState);
+    boolean reschedule = true;
+    if (limit > 0) {
+      reschedule = limit > exCount;
+    }
+    if (reschedule) {
+      this.setNanos(convertAndOffset(getInterval(), getTimeUnit()));
+      log.debug("{}: rescheduling, new offset {} mS ", this,
+          getDelay(TimeUnit.MILLISECONDS));
+      queueService.schedule(this);
+    }
+  }
+
+  /**
+   * Get the action
+   * @return
+   */
+  public A getAction() {
+    return action;
+  }
+
+  public long getInterval() {
+    intervalReadLock.lock();
+    try {
+      return interval;
+    } finally {
+      intervalReadLock.unlock();
+    }
+  }
+
+  public void updateInterval(long delay, TimeUnit timeUnit) {
+    intervalWriteLock.lock();
+    try {
+      interval = delay;
+      this.timeUnit = timeUnit;
+    } finally {
+      intervalWriteLock.unlock();
+    }
+  }
+
+  public TimeUnit getTimeUnit() {
+    intervalReadLock.lock();
+    try {
+      return timeUnit;
+    } finally {
+      intervalReadLock.unlock();
+    }
+  }
+
+  public int getExecutionCount() {
+    return executionCount.get();
+  }
+
+  public int getLimit() {
+    return limit;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ResetFailureWindow.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ResetFailureWindow.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ResetFailureWindow.java
new file mode 100644
index 0000000..28bcf55
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ResetFailureWindow.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.actions;
+
+import org.apache.slider.server.appmaster.SliderAppMaster;
+import org.apache.slider.server.appmaster.state.AppState;
+
+/**
+ * Requests the AM to reset the failure window
+ */
+public class ResetFailureWindow extends AsyncAction {
+
+  public ResetFailureWindow() {
+    super("ResetFailureWindow");
+  }
+
+  @Override
+  public void execute(SliderAppMaster appMaster,
+      QueueAccess queueService,
+      AppState appState) throws Exception {
+    appState.resetFailureCounts();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ReviewAndFlexApplicationSize.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ReviewAndFlexApplicationSize.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ReviewAndFlexApplicationSize.java
new file mode 100644
index 0000000..bf7edf9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/ReviewAndFlexApplicationSize.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.actions;
+
+import org.apache.slider.server.appmaster.SliderAppMaster;
+import org.apache.slider.server.appmaster.state.AppState;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Tell the AM to execute the full flex review operation
+ */
+public class ReviewAndFlexApplicationSize extends AsyncAction {
+
+  public ReviewAndFlexApplicationSize(String name,
+      long delay,
+      TimeUnit timeUnit) {
+    super(name, delay, timeUnit, ATTR_REVIEWS_APP_SIZE);
+  }
+
+  @Override
+  public void execute(SliderAppMaster appMaster,
+      QueueAccess queueService,
+      AppState appState) throws Exception {
+    appMaster.handleReviewAndFlexApplicationSize(this);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/UnregisterComponentInstance.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/UnregisterComponentInstance.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/UnregisterComponentInstance.java
new file mode 100644
index 0000000..575fe8f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/actions/UnregisterComponentInstance.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.actions;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.slider.server.appmaster.SliderAppMaster;
+import org.apache.slider.server.appmaster.state.AppState;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Tell AM to unregister this component instance
+ * {@link SliderAppMaster#unregisterComponent(ContainerId)}
+ */
+public class UnregisterComponentInstance extends AsyncAction {
+  
+
+  public final ContainerId containerId;
+
+  public UnregisterComponentInstance(ContainerId containerId,
+      long delay,
+      TimeUnit timeUnit) {
+    super("UnregisterComponentInstance :" + containerId.toString(),
+        delay, timeUnit);
+    this.containerId = containerId;
+  }
+
+  @Override
+  public void execute(SliderAppMaster appMaster,
+      QueueAccess queueService,
+      AppState appState) throws Exception {
+    appMaster.unregisterComponent(containerId);
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/BoolMetric.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/BoolMetric.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/BoolMetric.java
new file mode 100644
index 0000000..33f8d85
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/BoolMetric.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.management;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Metric;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * A bool metric, mapped to an integer. true maps to 1,  false to zero,
+ */
+public class BoolMetric implements Metric, Gauge<Integer> {
+
+  private final AtomicBoolean value;
+
+  public BoolMetric(boolean b) {
+    value = new AtomicBoolean(b);
+  }
+
+  public void set(boolean b) {
+    value.set(b);
+  }
+
+  public boolean get() {
+    return value.get();
+  }
+
+  @Override
+  public Integer getValue() {
+    return value.get() ? 1 : 0;
+  }
+
+  /**
+   * Evaluate from a string. Returns true if the string is considered to match 'true',
+   * false otherwise.
+   * @param s source
+   * @return true if the input parses to an integer other than 0. False if it doesn't parse
+   * or parses to 0.
+   */
+  public static boolean fromString(String s) {
+    try {
+      return Integer.valueOf(s) != 0;
+    } catch (NumberFormatException e) {
+      return false;
+    }
+  }
+
+  @Override
+  public String toString() {
+    return value.toString();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    BoolMetric that = (BoolMetric) o;
+    return get() == that.get();
+  }
+
+  @Override
+  public int hashCode() {
+    return value.hashCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/BoolMetricPredicate.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/BoolMetricPredicate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/BoolMetricPredicate.java
new file mode 100644
index 0000000..82bcd3a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/BoolMetricPredicate.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.management;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Metric;
+
+/**
+ * A metric which takes a predicate and returns 1 if the predicate evaluates
+ * to true. The predicate is evaluated whenever the metric is read.
+ */
+public class BoolMetricPredicate implements Metric, Gauge<Integer> {
+
+  private final Eval predicate;
+
+  public BoolMetricPredicate(Eval predicate) {
+    this.predicate = predicate;
+  }
+
+  @Override
+  public Integer getValue() {
+    return predicate.eval() ? 1: 0;
+  }
+
+  public interface Eval {
+    boolean eval();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/LongGauge.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/LongGauge.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/LongGauge.java
new file mode 100644
index 0000000..c93467b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/LongGauge.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.management;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Metric;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This is a {@link AtomicLong} which acts as a metrics gauge: its state can be exposed as
+ * a metrics.
+ * It also exposes some of the same method names as the Codahale Counter class, so that
+ * it's easy to swap in.
+ *
+ */
+public class LongGauge extends AtomicLong implements Metric, Gauge<Long> {
+
+  /**
+   * Instantiate
+   * @param val current value
+   */
+  public LongGauge(long val) {
+    super(val);
+  }
+
+  /**
+   * Instantiate with value 0
+   */
+  public LongGauge() {
+    this(0);
+  }
+
+  /**
+   * Get the value as a metric
+   * @return current value
+   */
+  @Override
+  public Long getValue() {
+    return get();
+  }
+
+  /**
+   * Method from {@Code counter}; used here for drop-in replacement
+   * without any recompile
+   * @return current value
+   */
+  public Long getCount() {
+    return get();
+  }
+
+  /**
+   * {@code ++}
+   */
+  public void inc() {
+    incrementAndGet();
+  }
+
+  /**
+   * {@code --}
+   */
+  public void dec() {
+    decrementAndGet();
+  }
+
+  /**
+   * Decrement to the floor of 0. Operations in parallel may cause confusion here,
+   * but it will still never go below zero
+   * @param delta delta
+   * @return the current value
+   */
+  public long decToFloor(long delta) {
+    long l = get();
+    long r = l - delta;
+    if (r < 0) {
+      r = 0;
+    }
+    // if this fails, the decrement has been lost
+    compareAndSet(l, r);
+    return get();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/LongMetricFunction.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/LongMetricFunction.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/LongMetricFunction.java
new file mode 100644
index 0000000..1de7345
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/LongMetricFunction.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.management;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Metric;
+
+/**
+ * A metric which takes a function to generate a long value.
+ * The function is evaluated whenever the metric is read.
+ */
+public class LongMetricFunction implements Metric, Gauge<Long> {
+
+  private final Eval function;
+
+  public LongMetricFunction(Eval function) {
+    this.function = function;
+  }
+
+  @Override
+  public Long getValue() {
+    return function.eval();
+  }
+
+  public interface Eval {
+    long eval();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MeterAndCounter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MeterAndCounter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MeterAndCounter.java
new file mode 100644
index 0000000..02ab7bc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MeterAndCounter.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.management;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.MetricRegistry;
+
+/**
+ * A combined meter and counter that can be used to measure load.
+ * Hash and equality are derived from the name
+ */
+public class MeterAndCounter {
+
+  /**
+   * suffix for counters: {@value}
+   */
+  public static final String COUNTER = ".counter";
+
+  /**
+   * suffix for meters: {@value}
+   */
+  public static final String METER = ".meter";
+
+  final Meter meter;
+  final Counter counter;
+  final String name;
+
+  /**
+   * Construct an instance
+   * @param metrics metrics to bond to
+   * @param name name before suffixes are appended
+   */
+  public MeterAndCounter(MetricRegistry metrics, String name) {
+    this.name = name;
+    counter = metrics.counter(name + COUNTER);
+    meter = metrics.meter(name + METER);
+  }
+
+  /**
+   * Construct an instance
+   * @param metrics metrics to bond to
+   * @param clazz class to use to derive name
+   * @param name name before suffixes are appended
+   */
+
+  public MeterAndCounter(MetricRegistry metrics, Class clazz, String name) {
+    this.name = name;
+    counter = metrics.counter(MetricRegistry.name(clazz, name + COUNTER));
+    meter = metrics.meter(MetricRegistry.name(clazz, name + METER));
+  }
+
+  /**
+   * Increment the counter, mark the meter
+   */
+  public void mark() {
+    counter.inc();
+    meter.mark();
+  }
+
+  public void inc() {
+    mark();
+  }
+
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    MeterAndCounter that = (MeterAndCounter) o;
+
+    return name.equals(that.name);
+
+  }
+
+  @Override
+  public int hashCode() {
+    return name.hashCode();
+  }
+
+  /**
+   * Get the count.
+   * @return the current count
+   */
+  public long getCount() {
+    return counter.getCount();
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[52/76] [abbrv] hadoop git commit: YARN-5941. Slider handles "per.component" for multiple components incorrectly. Contributed by Billie Rinaldi

Posted by ji...@apache.org.
YARN-5941. Slider handles "per.component" for multiple components incorrectly. Contributed by Billie Rinaldi


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

Branch: refs/heads/yarn-native-services
Commit: 427fe372b89e63e5f3533b8ad17bf7f0f06cdc78
Parents: 5e0f1b7
Author: Gour Saha <go...@apache.org>
Authored: Wed Nov 30 14:00:22 2016 -0800
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/slider/api/OptionKeys.java  | 14 ++++++++++++++
 .../org/apache/slider/common/SliderKeys.java    |  2 --
 .../apache/slider/providers/ProviderUtils.java  | 20 ++++++++++++++++----
 3 files changed, 30 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/427fe372/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/OptionKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/OptionKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/OptionKeys.java
index 434b1d9..988627d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/OptionKeys.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/OptionKeys.java
@@ -53,8 +53,22 @@ public interface OptionKeys extends InternalKeys {
    * Prefix for export options: {@value}
    */
   String EXPORT_PREFIX = "export.";
+  /**
+   * Type suffix for config file and package options: {@value}
+   */
   String TYPE_SUFFIX = ".type";
+  /**
+   * Name suffix for config file and package options: {@value}
+   */
   String NAME_SUFFIX = ".name";
+  /**
+   * Per component suffix for config file options: {@value}
+   */
+  String PER_COMPONENT = ".per.component";
+  /**
+   * Per group suffix for config file options: {@value}
+   */
+  String PER_GROUP = ".per.group";
 
   /**
    * Zookeeper quorum host list: {@value}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/427fe372/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
index 3d25d33..adf40ce 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
@@ -304,8 +304,6 @@ public interface SliderKeys extends SliderXmlConfKeys {
 
   String APP_RESOURCES = "application.resources";
   String APP_RESOURCES_DIR = "app/resources";
-  String PER_COMPONENT = "per.component";
-  String PER_GROUP = "per.group";
 
   String APP_PACKAGES_DIR = "app/packages";
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/427fe372/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java
index 39986c1..bc237f5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java
@@ -614,8 +614,16 @@ public class ProviderUtils implements RoleKeys, SliderKeys {
         throw new BadConfigException("Config format " + configFileType +
             " doesn't exist");
       }
+      boolean perComponent = appConf.getComponentOptBool(roleGroup,
+          OptionKeys.CONF_FILE_PREFIX + configEntry.getKey() + OptionKeys
+              .PER_COMPONENT, false);
+      boolean perGroup = appConf.getComponentOptBool(roleGroup,
+          OptionKeys.CONF_FILE_PREFIX + configEntry.getKey() + OptionKeys
+              .PER_GROUP, false);
+
       localizeConfigFile(launcher, roleName, roleGroup, configEntry.getKey(),
-          configFormat, configFileName, configs, env, fileSystem, clusterName);
+          configFormat, configFileName, configs, env, fileSystem,
+          clusterName, perComponent, perGroup);
     }
   }
 
@@ -631,6 +639,8 @@ public class ProviderUtils implements RoleKeys, SliderKeys {
    * @param env environment variables
    * @param fileSystem file system
    * @param clusterName app name
+   * @param perComponent true if file should be created per unique component
+   * @param perGroup true if file should be created per component group
    * @throws IOException file cannot be uploaded
    */
   public void localizeConfigFile(ContainerLauncher launcher,
@@ -639,7 +649,9 @@ public class ProviderUtils implements RoleKeys, SliderKeys {
       Map<String, Map<String, String>> configs,
       MapOperations env,
       SliderFileSystem fileSystem,
-      String clusterName)
+      String clusterName,
+      boolean perComponent,
+      boolean perGroup)
       throws IOException {
     if (launcher == null) {
       return;
@@ -655,9 +667,9 @@ public class ProviderUtils implements RoleKeys, SliderKeys {
     }
 
     String folder = null;
-    if ("true".equals(config.get(PER_COMPONENT))) {
+    if (perComponent) {
       folder = roleName;
-    } else if ("true".equals(config.get(PER_GROUP))) {
+    } else if (perGroup) {
       folder = roleGroup;
     }
     if (folder != null) {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[57/76] [abbrv] hadoop git commit: YARN-5909. Remove agent related code in slider AM. Contributed by Jian He

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/Register.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/Register.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/Register.java
deleted file mode 100644
index 0150079..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/Register.java
+++ /dev/null
@@ -1,193 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.server.appmaster.web.rest.agent;
-
-import org.apache.slider.providers.agent.State;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.annotate.JsonProperty;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-import java.util.Map;
-
-/** Data model for agent to send heartbeat to ambari and/or app master. */
-@JsonIgnoreProperties(ignoreUnknown = true)
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-public class Register {
-  private int responseId = -1;
-  private long timestamp;
-  private String label;
-  private int currentPingPort;
-  private HostInfo hardwareProfile;
-  private String publicHostname;
-  private String tags;
-  private AgentEnv agentEnv;
-  private String agentVersion;
-  private State actualState;
-  private State expectedState;
-  private Map<String, String> allocatedPorts;
-  private Map<String, String> logFolders;
-  private String pkg;
-  private String appVersion;
-
-  @JsonProperty("responseId")
-  public int getResponseId() {
-    return responseId;
-  }
-
-  @JsonProperty("responseId")
-  public void setResponseId(int responseId) {
-    this.responseId = responseId;
-  }
-
-  public long getTimestamp() {
-    return timestamp;
-  }
-
-  public void setTimestamp(long timestamp) {
-    this.timestamp = timestamp;
-  }
-
-  public String getLabel() {
-    return label;
-  }
-
-  public void setLabel(String label) {
-    this.label = label;
-  }
-
-  public String getTags() {
-    return tags;
-  }
-
-  public void setTags(String tags) {
-    this.tags = tags;
-  }
-
-  public HostInfo getHardwareProfile() {
-    return hardwareProfile;
-  }
-
-  public void setHardwareProfile(HostInfo hardwareProfile) {
-    this.hardwareProfile = hardwareProfile;
-  }
-
-  public String getPublicHostname() {
-    return publicHostname;
-  }
-
-  public void setPublicHostname(String name) {
-    this.publicHostname = name;
-  }
-
-  public AgentEnv getAgentEnv() {
-    return agentEnv;
-  }
-
-  public void setAgentEnv(AgentEnv env) {
-    this.agentEnv = env;
-  }
-
-  public String getAgentVersion() {
-    return agentVersion;
-  }
-
-  public void setAgentVersion(String agentVersion) {
-    this.agentVersion = agentVersion;
-  }
-
-  public int getCurrentPingPort() {
-    return currentPingPort;
-  }
-
-  public void setCurrentPingPort(int currentPingPort) {
-    this.currentPingPort = currentPingPort;
-  }
-
-  public State getActualState() {
-    return actualState;
-  }
-
-  public void setActualState(State actualState) {
-    this.actualState = actualState;
-  }
-
-  public State getExpectedState() {
-    return expectedState;
-  }
-
-  public void setExpectedState(State expectedState) {
-    this.expectedState = expectedState;
-  }
-
-  /** @return the allocated ports, or <code>null</code> if none are present */
-  @JsonProperty("allocatedPorts")
-  public Map<String, String> getAllocatedPorts() {
-    return allocatedPorts;
-  }
-
-  /** @param ports allocated ports */
-  @JsonProperty("allocatedPorts")
-  public void setAllocatedPorts(Map<String, String> ports) {
-    this.allocatedPorts = ports;
-  }
-
-  /** @return the log folders, or <code>null</code> if none are present */
-  @JsonProperty("logFolders")
-  public Map<String, String> getLogFolders() {
-    return logFolders;
-  }
-
-  /** @param logFolders assigned log folders */
-  @JsonProperty("logFolders")
-  public void setLogFolders(Map<String, String> logFolders) {
-    this.logFolders = logFolders;
-  }
-
-  public String getPkg() {
-    return pkg;
-  }
-
-  public void setPkg(String pkg) {
-    this.pkg = pkg;
-  }
-
-  @JsonProperty("appVersion")
-  public String getAppVersion() {
-    return appVersion;
-  }
-
-  @JsonProperty("appVersion")
-  public void setAppVersion(String appVersion) {
-    this.appVersion = appVersion;
-  }
-
-  @Override
-  public String toString() {
-    String ret = "responseId=" + responseId + "\n" +
-                 "timestamp=" + timestamp + "\n" +
-                 "label=" + label + "\n" +
-                 "hostname=" + publicHostname + "\n" +
-                 "expectedState=" + expectedState + "\n" +
-                 "actualState=" + actualState + "\n" +
-                 "appVersion=" + appVersion + "\n";
-
-    if (hardwareProfile != null) {
-      ret = ret + "hardwareprofile=" + this.hardwareProfile.toString();
-    }
-    return ret;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationCommand.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationCommand.java
deleted file mode 100644
index 4b87dd2..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationCommand.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.server.appmaster.web.rest.agent;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-/**
- *
- */
-@JsonIgnoreProperties(ignoreUnknown = true)
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-public class RegistrationCommand {
-
-  private String command;
-
-  public String getCommand() {
-    return command;
-  }
-
-  public void setCommand(String command) {
-    this.command = command;
-  }
-
-  public RegistrationCommand(String command) {
-
-    this.command = command;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationResponse.java
deleted file mode 100644
index 80b7a5e..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationResponse.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.server.appmaster.web.rest.agent;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.annotate.JsonProperty;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-import java.util.List;
-
-@JsonIgnoreProperties(ignoreUnknown = true)
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-public class RegistrationResponse {
-
-  @JsonProperty("response")
-  private RegistrationStatus response;
-
-  /**
-   * exitstatus is a code of error which was rised on server side. exitstatus
-   * = 0 (OK - Default) exitstatus = 1 (Registration failed because different
-   * version of agent and server)
-   */
-  @JsonProperty("exitstatus")
-  private int exitstatus;
-
-  /** log - message, which will be printed to agents log */
-  @JsonProperty("log")
-  private String log;
-
-  /** tags - tags associated with the container */
-  @JsonProperty("tags")
-  private String tags;
-  
-  @JsonProperty("package")
-  private String pkg;
-
-  //Response id to start with, usually zero.
-  @JsonProperty("responseId")
-  private long responseId;
-
-  @JsonProperty("statusCommands")
-  private List<StatusCommand> statusCommands = null;
-
-  public RegistrationResponse() {
-  }
-
-  public RegistrationStatus getResponse() {
-    return response;
-  }
-
-  public void setResponse(RegistrationStatus response) {
-    this.response = response;
-  }
-
-  public int getExitstatus() {
-    return exitstatus;
-  }
-
-  public void setExitstatus(int exitstatus) {
-    this.exitstatus = exitstatus;
-  }
-
-  public RegistrationStatus getResponseStatus() {
-    return response;
-  }
-
-  public void setResponseStatus(RegistrationStatus response) {
-    this.response = response;
-  }
-
-  public List<StatusCommand> getStatusCommands() {
-    return statusCommands;
-  }
-
-  public void setStatusCommands(List<StatusCommand> statusCommands) {
-    this.statusCommands = statusCommands;
-  }
-
-  public long getResponseId() {
-    return responseId;
-  }
-
-  public void setResponseId(long responseId) {
-    this.responseId = responseId;
-  }
-
-  public String getTags() {
-    return tags;
-  }
-
-  public void setTags(String tags) {
-    this.tags = tags;
-  }
-
-  public String getLog() {
-    return log;
-  }
-
-  public void setLog(String log) {
-    this.log = log;
-  }
-
-  public String getPkg() {
-    return pkg;
-  }
-
-  public void setPkg(String pkg) {
-    this.pkg = pkg;
-  }
-
-  @Override
-  public String toString() {
-    return "RegistrationResponse{" +
-           "response=" + response +
-           ", responseId=" + responseId +
-           ", statusCommands=" + statusCommands +
-           '}';
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationStatus.java
deleted file mode 100644
index 8374710..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationStatus.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.server.appmaster.web.rest.agent;
-
-public enum RegistrationStatus {
-    OK,
-    FAILED
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/StatusCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/StatusCommand.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/StatusCommand.java
deleted file mode 100644
index 5b205b5..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/StatusCommand.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.server.appmaster.web.rest.agent;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.annotate.JsonProperty;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Command to report the status of a list of services in roles.
- */
-@JsonIgnoreProperties(ignoreUnknown = true)
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-public class StatusCommand {
-  public static String STATUS_COMMAND = "STATUS";
-  public static String GET_CONFIG_COMMAND = "GET_CONFIG";
-
-  AgentCommandType agentCommandType;
-
-  private String clusterName;
-  private String serviceName;
-  private String componentName;
-  private Map<String, Map<String, String>> configurations;
-  private Map<String, String> commandParams = new HashMap<String, String>();
-  private Map<String, String> hostLevelParams = new HashMap<String, String>();
-  private String roleCommand;
-  private boolean yarnDockerMode;
-
-  public StatusCommand() {
-    this.agentCommandType = AgentCommandType.STATUS_COMMAND;
-  }
-
-  @JsonProperty("clusterName")
-  public String getClusterName() {
-    return clusterName;
-  }
-
-  @JsonProperty("clusterName")
-  public void setClusterName(String clusterName) {
-    this.clusterName = clusterName;
-  }
-
-  @JsonProperty("serviceName")
-  public String getServiceName() {
-    return serviceName;
-  }
-
-  @JsonProperty("serviceName")
-  public void setServiceName(String serviceName) {
-    this.serviceName = serviceName;
-  }
-
-  @JsonProperty("componentName")
-  public String getComponentName() {
-    return componentName;
-  }
-
-  @JsonProperty("componentName")
-  public void setComponentName(String componentName) {
-    this.componentName = componentName;
-  }
-
-  @JsonProperty("configurations")
-  public Map<String, Map<String, String>> getConfigurations() {
-    return configurations;
-  }
-
-  @JsonProperty("configurations")
-  public void setConfigurations(Map<String, Map<String, String>> configurations) {
-    this.configurations = configurations;
-  }
-
-  @JsonProperty("hostLevelParams")
-  public Map<String, String> getHostLevelParams() {
-    return hostLevelParams;
-  }
-
-  @JsonProperty("hostLevelParams")
-  public void setHostLevelParams(Map<String, String> params) {
-    this.hostLevelParams = params;
-  }
-
-  @JsonProperty("commandParams")
-  public Map<String, String> getCommandParams() {
-    return commandParams;
-  }
-
-  @JsonProperty("commandParams")
-  public void setCommandParams(Map<String, String> commandParams) {
-    this.commandParams = commandParams;
-  }
-
-  @JsonProperty("commandType")
-  public AgentCommandType getCommandType() {
-    return agentCommandType;
-  }
-
-  @JsonProperty("commandType")
-  public void setCommandType(AgentCommandType commandType) {
-    this.agentCommandType = commandType;
-  }
-
-  @JsonProperty("roleCommand")
-  public String getRoleCommand() {
-    return roleCommand;
-  }
-
-  @JsonProperty("roleCommand")
-  public void setRoleCommand(String roleCommand) {
-    this.roleCommand = roleCommand;
-  }
-  
-  @JsonProperty("yarnDockerMode")
-  public boolean isYarnDockerMode() {
-    return yarnDockerMode;
-  }
-
-  @JsonProperty("yarnDockerMode")
-  public void setYarnDockerMode(boolean yarnDockerMode) {
-    this.yarnDockerMode = yarnDockerMode;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder builder = new StringBuilder();
-    builder.append("StatusCommand [agentCommandType=").append(agentCommandType)
-        .append(", clusterName=").append(clusterName).append(", serviceName=")
-        .append(serviceName).append(", componentName=").append(componentName)
-        .append(", configurations=").append(configurations)
-        .append(", commandParams=").append(commandParams)
-        .append(", hostLevelParams=").append(hostLevelParams)
-        .append(", roleCommand=").append(roleCommand).append("]");
-    return builder.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider.java
deleted file mode 100644
index 0bea8fa..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.slider.common.tools.SliderFileSystem;
-import org.apache.slider.core.conf.AggregateConf;
-import org.apache.slider.core.exceptions.BadConfigException;
-import org.apache.slider.tools.TestUtility;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Set;
-
-/**
- *
- */
-public class TestAgentClientProvider {
-  protected static final Logger log =
-      LoggerFactory.getLogger(TestAgentClientProvider.class);
-  @Rule
-  public TemporaryFolder folder = new TemporaryFolder();
-
-  @Test
-  public void testGetApplicationTags() throws Exception {
-    Configuration configuration = new Configuration();
-    FileSystem fs = FileSystem.getLocal(configuration);
-    SliderFileSystem sliderFileSystem = new SliderFileSystem(fs, configuration);
-
-    AgentClientProvider provider = new AgentClientProvider(null);
-    String zipFileName = TestUtility.createAppPackage(
-        folder,
-        "testpkg",
-        "test.zip",
-        "target/test-classes/org/apache/slider/common/tools/test");
-    Set<String> tags = provider.getApplicationTags(sliderFileSystem, zipFileName);
-    assert tags != null;
-    assert !tags.isEmpty();
-    assert tags.contains("Name: STORM");
-    assert tags.contains("Description: Apache Hadoop Stream processing framework");
-    assert tags.contains("Version: 0.9.1.2.1");
-
-  }
-
-  @Test
-  public void testValidateInstanceDefinition() throws Exception {
-    AgentClientProvider provider = new AgentClientProvider(null);
-    AggregateConf instanceDefinition = new AggregateConf();
-
-    try {
-      provider.validateInstanceDefinition(instanceDefinition, null);
-      Assert.assertFalse("Should fail with BadConfigException", true);
-    } catch (BadConfigException e) {
-      log.info(e.toString());
-      Assert.assertTrue(e.getMessage().contains("Application definition must be provided"));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentLaunchParameter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentLaunchParameter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentLaunchParameter.java
deleted file mode 100644
index ec62b54..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentLaunchParameter.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent;
-
-import org.junit.Assert;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-/**
- *
- */
-public class TestAgentLaunchParameter {
-  protected static final Logger log =
-      LoggerFactory.getLogger(TestAgentLaunchParameter.class);
-
-  @Test
-  public void testTestAgentLaunchParameter() throws Exception {
-    AgentLaunchParameter alp = new AgentLaunchParameter("");
-    Assert.assertEquals("", alp.getNextLaunchParameter("abc"));
-    Assert.assertEquals("", alp.getNextLaunchParameter("HBASE_MASTER"));
-
-    alp = new AgentLaunchParameter("a:1:2:3|b:5:6:NONE");
-    Assert.assertEquals("1", alp.getNextLaunchParameter("a"));
-    Assert.assertEquals("2", alp.getNextLaunchParameter("a"));
-    Assert.assertEquals("3", alp.getNextLaunchParameter("a"));
-    Assert.assertEquals("3", alp.getNextLaunchParameter("a"));
-
-    Assert.assertEquals("5", alp.getNextLaunchParameter("b"));
-    Assert.assertEquals("6", alp.getNextLaunchParameter("b"));
-    Assert.assertEquals("", alp.getNextLaunchParameter("b"));
-    Assert.assertEquals("", alp.getNextLaunchParameter("b"));
-    Assert.assertEquals("", alp.getNextLaunchParameter("c"));
-
-    alp = new AgentLaunchParameter("|a:1:3|b::5:NONE:");
-    Assert.assertEquals("1", alp.getNextLaunchParameter("a"));
-    Assert.assertEquals("3", alp.getNextLaunchParameter("a"));
-    Assert.assertEquals("3", alp.getNextLaunchParameter("a"));
-
-    Assert.assertEquals("", alp.getNextLaunchParameter("b"));
-    Assert.assertEquals("5", alp.getNextLaunchParameter("b"));
-    Assert.assertEquals("", alp.getNextLaunchParameter("b"));
-    Assert.assertEquals("", alp.getNextLaunchParameter("b"));
-
-    alp = new AgentLaunchParameter("|:");
-    Assert.assertEquals("", alp.getNextLaunchParameter("b"));
-    Assert.assertEquals("", alp.getNextLaunchParameter("a"));
-
-    alp = new AgentLaunchParameter("HBASE_MASTER:a,b:DO_NOT_REGISTER:");
-    Assert.assertEquals("a,b", alp.getNextLaunchParameter("HBASE_MASTER"));
-    Assert.assertEquals("DO_NOT_REGISTER", alp.getNextLaunchParameter("HBASE_MASTER"));
-    Assert.assertEquals("DO_NOT_REGISTER", alp.getNextLaunchParameter("HBASE_MASTER"));
-
-    alp = new AgentLaunchParameter("HBASE_MASTER:a,b:DO_NOT_REGISTER::c:::");
-    Assert.assertEquals("a,b", alp.getNextLaunchParameter("HBASE_MASTER"));
-    Assert.assertEquals("DO_NOT_REGISTER", alp.getNextLaunchParameter("HBASE_MASTER"));
-    Assert.assertEquals("", alp.getNextLaunchParameter("HBASE_MASTER"));
-    Assert.assertEquals("c", alp.getNextLaunchParameter("HBASE_MASTER"));
-    Assert.assertEquals("c", alp.getNextLaunchParameter("HBASE_MASTER"));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentUtils.java
deleted file mode 100644
index 5e1dc7f..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentUtils.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.slider.providers.agent;
-
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileWriter;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.slider.common.tools.SliderFileSystem;
-import org.apache.slider.providers.agent.application.metadata.Metainfo;
-import org.apache.slider.tools.TestUtility;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class TestAgentUtils {
-  protected static final Logger log =
-      LoggerFactory.getLogger(TestAgentUtils.class);
-  @Rule
-  public TemporaryFolder folder = new TemporaryFolder();
-  private static final String metainfo_str = "<metainfo>\n"
-      + "  <schemaVersion>2.0</schemaVersion>\n"
-      + "  <application>\n"
-      + "      <name>MYTESTAPPLICATION</name>\n"
-      + "      <comment>\n"
-      + "        My Test Application\n"
-      + "      </comment>\n"
-      + "      <version>1.0</version>\n"
-      + "      <type>YARN-APP</type>\n"
-      + "      <components>\n"
-      + "        <component>\n"
-      + "          <name>REST</name>\n"
-      + "          <category>MASTER</category>\n"
-      + "          <commandScript>\n"
-      + "            <script>scripts/rest.py</script>\n"
-      + "            <scriptType>PYTHON</scriptType>\n"
-      + "            <timeout>600</timeout>\n"
-      + "          </commandScript>\n"
-      + "        </component>\n"
-      + "      </components>\n"
-      + "  </application>\n"
-      + "</metainfo>";
-
-  @Test
-  public void testGetApplicationMetainfo() throws Exception {
-    String zipFileName = TestUtility.createAppPackage(
-        folder,
-        "testpkg",
-        "test.zip",
-        "target/test-classes/org/apache/slider/common/tools/test");
-    Configuration configuration = new Configuration();
-    FileSystem fs = FileSystem.getLocal(configuration);
-    log.info("fs working dir is {}", fs.getWorkingDirectory().toString());
-    SliderFileSystem sliderFileSystem = new SliderFileSystem(fs, configuration);
-
-    // Without accompany metainfo file, read metainfo from the zip file
-    Metainfo metainfo = AgentUtils.getApplicationMetainfo(
-        sliderFileSystem, zipFileName, false);
-    Assert.assertNotNull(metainfo.getApplication());
-    Assert.assertEquals("STORM", metainfo.getApplication().getName());
-
-    // With accompany metainfo file, read metainfo from the accompany file
-    String acompanyFileName = zipFileName + ".metainfo.xml";
-    File f = new File(acompanyFileName);
-    try (BufferedWriter writer = new BufferedWriter(new FileWriter(f))) {
-      writer.write(metainfo_str);
-    }
-    metainfo = AgentUtils.getApplicationMetainfo(
-        sliderFileSystem, zipFileName, false);
-    Assert.assertNotNull(metainfo.getApplication());
-    Assert.assertEquals("MYTESTAPPLICATION", metainfo.getApplication().getName());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAppDefinitionPersister.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAppDefinitionPersister.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAppDefinitionPersister.java
deleted file mode 100644
index dedf4f6..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAppDefinitionPersister.java
+++ /dev/null
@@ -1,264 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent;
-
-import com.google.common.io.Files;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.slider.common.params.ActionCreateArgs;
-import org.apache.slider.common.params.AddonArgsDelegate;
-import org.apache.slider.common.tools.SliderFileSystem;
-import org.apache.slider.core.conf.ConfTree;
-import org.apache.slider.core.conf.ConfTreeOperations;
-import org.apache.slider.core.exceptions.BadConfigException;
-import org.apache.slider.core.persist.AppDefinitionPersister;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.PrintWriter;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- *
- */
-public class TestAppDefinitionPersister {
-  protected static final Logger log =
-      LoggerFactory.getLogger(TestAppDefinitionPersister.class);
-  @Rule
-  public TemporaryFolder folder = new TemporaryFolder();
-
-  /**
-   * @BeforeClass public static void initialize() { BasicConfigurator.resetConfiguration();
-   * BasicConfigurator.configure(); }*
-   */
-
-
-  @Test
-  public void testAppDefinitionPersister() throws Exception {
-    Configuration configuration = new Configuration();
-    FileSystem fs = FileSystem.getLocal(configuration);
-    log.info("fs working dir is {}", fs.getWorkingDirectory().toString());
-    SliderFileSystem sliderFileSystem = new SliderFileSystem(fs, configuration);
-
-    AppDefinitionPersister adp = new AppDefinitionPersister(sliderFileSystem);
-    String clustername = "c1";
-    ActionCreateArgs buildInfo = new ActionCreateArgs();
-    buildInfo.appMetaInfo = null;
-    buildInfo.appDef = null;
-    buildInfo.addonDelegate = new AddonArgsDelegate();
-
-    // nothing to do
-    adp.processSuppliedDefinitions(clustername, buildInfo, null);
-    adp.persistPackages();
-    List<AppDefinitionPersister.AppDefinition> appDefinitions = adp.getAppDefinitions();
-    Assert.assertTrue(appDefinitions.size() == 0);
-
-    ConfTree ct = new ConfTree();
-    ConfTreeOperations appConf = new ConfTreeOperations(ct);
-    final File tempDir = Files.createTempDir();
-    final File metainfo = new File(tempDir, "metainfo.json");
-
-    // unreadable metainfo
-    buildInfo.appMetaInfo = metainfo;
-
-    try {
-      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
-    } catch (BadConfigException bce) {
-      log.info(bce.getMessage());
-      Assert.assertTrue(bce.getMessage().contains(
-          "Path specified with "
-              + "--metainfo either cannot be read or is not a file"));
-    }
-
-    try (PrintWriter writer = new PrintWriter(metainfo.getAbsolutePath(), "UTF-8")) {
-      writer.println("{");
-      writer.println("}");
-    }
-    buildInfo.appDef = metainfo;
-
-    try {
-      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
-    } catch (BadConfigException bce) {
-      log.info(bce.getMessage());
-      Assert.assertTrue(bce.getMessage().contains(
-          "Both --metainfo and --appdef cannot be specified"));
-    }
-
-    // both --metainfojson and --appdef cannot be specified
-    buildInfo.appMetaInfo = null;
-    buildInfo.appMetaInfoJson = "{}";
-    try {
-      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
-    } catch (BadConfigException bce) {
-      log.info(bce.getMessage());
-      Assert.assertTrue(bce.getMessage().contains(
-          "Both --metainfojson and --appdef cannot be specified"));
-    }
-
-    buildInfo.appDef = null;
-
-    buildInfo.appMetaInfoJson = "";
-    try {
-      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
-    } catch (BadConfigException bce) {
-      log.info(bce.getMessage());
-      Assert.assertTrue(bce.getMessage().contains(
-          "Empty string specified with --metainfojson"));
-    }
-    buildInfo.appMetaInfo = metainfo;
-
-    // both --metainfo and --metainfojson cannot be specified
-    buildInfo.appMetaInfoJson = "{}";
-    try {
-      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
-    } catch (BadConfigException bce) {
-      log.info(bce.getMessage());
-      Assert.assertTrue(bce.getMessage().contains(
-          "Both --metainfo and --metainfojson cannot be specified"));
-    }
-    buildInfo.appMetaInfoJson = null;
-
-    appConf.getGlobalOptions().set(AgentKeys.APP_DEF, metainfo.getAbsolutePath());
-
-    try {
-      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
-    } catch (BadConfigException bce) {
-      log.info(bce.getMessage());
-      Assert.assertTrue(bce.getMessage().contains(
-          "application.def cannot "
-              + "not be set if --metainfo is specified in the cmd line"));
-    }
-
-    appConf.getGlobalOptions().remove(AgentKeys.APP_DEF);
-
-    adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
-    appDefinitions = adp.getAppDefinitions();
-    Assert.assertTrue(appDefinitions.size() == 1);
-    Assert.assertTrue(appConf.getGlobalOptions().get(AgentKeys.APP_DEF).contains("appdef/appPkg.zip"));
-    log.info(appDefinitions.get(0).toString());
-    Assert.assertTrue(appDefinitions.get(0).appDefPkgOrFolder.toString().endsWith("default"));
-    Assert.assertTrue(appDefinitions.get(0).targetFolderInFs.toString().contains("cluster/c1/appdef"));
-    Assert.assertEquals("appPkg.zip", appDefinitions.get(0).pkgName);
-
-    buildInfo.appDef = tempDir;
-    buildInfo.appMetaInfo = null;
-
-    appConf.getGlobalOptions().set(AgentKeys.APP_DEF, metainfo.getAbsolutePath());
-
-    try {
-      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
-    } catch (BadConfigException bce) {
-      log.info(bce.getMessage());
-      Assert.assertTrue(bce.getMessage().contains("application.def must not be set if --appdef is provided"));
-    }
-
-    adp.getAppDefinitions().clear();
-    appConf.getGlobalOptions().remove(AgentKeys.APP_DEF);
-    adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
-    appDefinitions = adp.getAppDefinitions();
-    Assert.assertTrue(appDefinitions.size() == 1);
-    Assert.assertTrue(appConf.getGlobalOptions().get(AgentKeys.APP_DEF).contains("appdef/appPkg.zip"));
-    log.info(appDefinitions.get(0).toString());
-    Assert.assertTrue(appDefinitions.get(0).appDefPkgOrFolder.toString().endsWith(tempDir.toString()));
-    Assert.assertTrue(appDefinitions.get(0).targetFolderInFs.toString().contains("cluster/c1/appdef"));
-    Assert.assertEquals("appPkg.zip", appDefinitions.get(0).pkgName);
-
-    adp.getAppDefinitions().clear();
-    buildInfo.appDef = null;
-    buildInfo.appMetaInfo = null;
-    appConf.getGlobalOptions().remove(AgentKeys.APP_DEF);
-
-    ArrayList<String> list = new ArrayList<String>() {{
-      add("addon1");
-      add("");
-      add("addon2");
-      add(metainfo.getAbsolutePath());
-    }};
-
-    buildInfo.addonDelegate.addonTuples = list;
-    try {
-      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
-    } catch (BadConfigException bce) {
-      log.info(bce.getMessage());
-      Assert.assertTrue(bce.getMessage().contains("addon package can only be specified if main app package is specified"));
-    }
-
-    buildInfo.appMetaInfo = metainfo;
-
-    try {
-      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
-    } catch (BadConfigException bce) {
-      log.info(bce.getMessage());
-      Assert.assertTrue(bce.getMessage().contains("Invalid path for addon package addon1"));
-    }
-
-    appConf.getGlobalOptions().remove(AgentKeys.APP_DEF);
-
-    list = new ArrayList<String>() {{
-      add("addon1");
-      add(tempDir.getAbsolutePath());
-      add("addon2");
-      add(metainfo.getAbsolutePath());
-    }};
-
-    buildInfo.addonDelegate.addonTuples = list;
-    adp.getAppDefinitions().clear();
-
-    adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
-    appDefinitions = adp.getAppDefinitions();
-
-    Assert.assertTrue(appDefinitions.size() == 3);
-    Assert.assertTrue(appConf.getGlobalOptions().get(AgentKeys.APP_DEF).contains("appdef/appPkg.zip"));
-    Assert.assertTrue(appConf.getGlobalOptions().get("application.addon.addon1").contains(
-        "addons/addon1/addon_addon1.zip"));
-    Assert.assertTrue(appConf.getGlobalOptions().get("application.addon.addon2").contains(
-        "addons/addon2/addon_addon2.zip"));
-    log.info(appConf.getGlobalOptions().get("application.addons"));
-    Assert.assertTrue(appConf.getGlobalOptions().get("application.addons").contains(
-        "application.addon.addon2,application.addon.addon1")
-                      || appConf.getGlobalOptions().get("application.addons").contains(
-        "application.addon.addon1,application.addon.addon2"));
-    int seen = 0;
-    for (AppDefinitionPersister.AppDefinition adp_ad : appDefinitions) {
-      if (adp_ad.pkgName.equals("appPkg.zip")) {
-        log.info(adp_ad.toString());
-        Assert.assertTrue(adp_ad.appDefPkgOrFolder.toString().endsWith("default"));
-        Assert.assertTrue(adp_ad.targetFolderInFs.toString().contains("cluster/c1/appdef"));
-        seen++;
-      }
-      if (adp_ad.pkgName.equals("addon_addon1.zip")) {
-        log.info(adp_ad.toString());
-        Assert.assertTrue(adp_ad.appDefPkgOrFolder.toString().endsWith(tempDir.toString()));
-        Assert.assertTrue(adp_ad.targetFolderInFs.toString().contains("addons/addon1"));
-        seen++;
-      }
-      if (adp_ad.pkgName.equals("addon_addon2.zip")) {
-        log.info(adp_ad.toString());
-        Assert.assertTrue(adp_ad.appDefPkgOrFolder.toString().endsWith("metainfo.json"));
-        Assert.assertTrue(adp_ad.targetFolderInFs.toString().contains("addons/addon2"));
-        seen++;
-      }
-    }
-    Assert.assertEquals(3, seen);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestComponentTagProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestComponentTagProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestComponentTagProvider.java
deleted file mode 100644
index 7b38ee3..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestComponentTagProvider.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.slider.providers.agent;
-
-import org.junit.Assert;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-public class TestComponentTagProvider {
-  protected static final Logger log =
-      LoggerFactory.getLogger(TestComponentTagProvider.class);
-
-  @Test
-  public void testTagProvider() throws Exception {
-    ComponentTagProvider ctp = new ComponentTagProvider();
-    Assert.assertEquals("", ctp.getTag(null, null));
-    Assert.assertEquals("", ctp.getTag(null, "cid"));
-    Assert.assertEquals("", ctp.getTag("comp1", null));
-
-    Assert.assertEquals("1", ctp.getTag("comp1", "cid1"));
-    Assert.assertEquals("2", ctp.getTag("comp1", "cid2"));
-    Assert.assertEquals("3", ctp.getTag("comp1", "cid3"));
-    ctp.releaseTag("comp1", "cid2");
-    Assert.assertEquals("2", ctp.getTag("comp1", "cid22"));
-
-    ctp.releaseTag("comp1", "cid4");
-    ctp.recordAssignedTag("comp1", "cid5", "5");
-    Assert.assertEquals("4", ctp.getTag("comp1", "cid4"));
-    Assert.assertEquals("4", ctp.getTag("comp1", "cid4"));
-    Assert.assertEquals("6", ctp.getTag("comp1", "cid6"));
-
-    ctp.recordAssignedTag("comp1", "cid55", "5");
-    Assert.assertEquals("5", ctp.getTag("comp1", "cid55"));
-
-    ctp.recordAssignedTag("comp2", "cidb3", "3");
-    Assert.assertEquals("1", ctp.getTag("comp2", "cidb1"));
-    Assert.assertEquals("2", ctp.getTag("comp2", "cidb2"));
-    Assert.assertEquals("4", ctp.getTag("comp2", "cidb4"));
-
-    ctp.recordAssignedTag("comp2", "cidb5", "six");
-    ctp.recordAssignedTag("comp2", "cidb5", "-55");
-    ctp.recordAssignedTag("comp2", "cidb5", "tags");
-    ctp.recordAssignedTag("comp2", "cidb5", null);
-    ctp.recordAssignedTag("comp2", "cidb5", "");
-    ctp.recordAssignedTag("comp2", "cidb5", "5");
-    Assert.assertEquals("6", ctp.getTag("comp2", "cidb6"));
-
-    ctp.recordAssignedTag("comp2", null, "5");
-    ctp.recordAssignedTag(null, null, "5");
-    ctp.releaseTag("comp1", null);
-    ctp.releaseTag(null, "cid4");
-    ctp.releaseTag(null, null);
-  }
-
-  @Test
-  public void testTagProviderWithThread() throws Exception {
-    ComponentTagProvider ctp = new ComponentTagProvider();
-    Thread thread = new Thread(new Taggged(ctp));
-    Thread thread2 = new Thread(new Taggged(ctp));
-    Thread thread3 = new Thread(new Taggged(ctp));
-    thread.start();
-    thread2.start();
-    thread3.start();
-    ctp.getTag("comp1", "cid50");
-    thread.join();
-    thread2.join();
-    thread3.join();
-    Assert.assertEquals("101", ctp.getTag("comp1", "cid101"));
-  }
-
-  public class Taggged implements Runnable {
-    private final ComponentTagProvider ctp;
-
-    public Taggged(ComponentTagProvider ctp) {
-      this.ctp = ctp;
-    }
-
-    public void run() {
-      for (int i = 0; i < 100; i++) {
-        String containerId = "cid" + (i + 1);
-        this.ctp.getTag("comp1", containerId);
-      }
-      for (int i = 0; i < 100; i++) {
-        String containerId = "cid" + (i + 1);
-        this.ctp.getTag("comp1", containerId);
-      }
-      for (int i = 0; i < 100; i += 2) {
-        String containerId = "cid" + (i + 1);
-        this.ctp.releaseTag("comp1", containerId);
-      }
-      for (int i = 0; i < 100; i += 2) {
-        String containerId = "cid" + (i + 1);
-        this.ctp.getTag("comp1", containerId);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestState.java
deleted file mode 100644
index 6a2e5ab5..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestState.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent;
-
-import org.junit.Assert;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class TestState {
-  protected static final Logger log = LoggerFactory.getLogger(TestState.class);
-
-  @Test
-  public void testState() throws Exception {
-    State state = State.STARTED;
-    Assert.assertEquals(Command.STOP, state.getSupportedCommand(false, true));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestConfigParser.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestConfigParser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestConfigParser.java
deleted file mode 100644
index 3aa44a1..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestConfigParser.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.junit.Assert;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-
-/**
- *
- */
-public class TestConfigParser {
-  protected static final Logger log =
-      LoggerFactory.getLogger(TestConfigParser.class);
-  private static final String config_1_str = "<configuration>\n"
-                                             + "  <property>\n"
-                                             + "    <name>security.client.protocol.acl</name>\n"
-                                             + "    <value>*</value>\n"
-                                             + "    <description>ACL for HRegionInterface protocol implementations (ie. \n"
-                                             + "    clients talking to HRegionServers)\n"
-                                             + "    The ACL is a comma-separated list of user and group names. The user and \n"
-                                             + "    group list is separated by a blank. For e.g. \"alice,bob users,wheel\". \n"
-                                             + "    A special value of \"*\" means all users are allowed.</description>\n"
-                                             + "  </property>\n"
-                                             + "\n"
-                                             + "  <property>\n"
-                                             + "    <name>security.admin.protocol.acl</name>\n"
-                                             + "    <value>*</value>\n"
-                                             + "    <description>ACL for HMasterInterface protocol implementation (ie. \n"
-                                             + "    clients talking to HMaster for admin operations).\n"
-                                             + "    The ACL is a comma-separated list of user and group names. The user and \n"
-                                             + "    group list is separated by a blank. For e.g. \"alice,bob users,wheel\". \n"
-                                             + "    A special value of \"*\" means all users are allowed.</description>\n"
-                                             + "  </property>\n"
-                                             + "\n"
-                                             + "  <property>\n"
-                                             + "    <name>security.masterregion.protocol.acl</name>\n"
-                                             + "    <value>*</value>\n"
-                                             + "    <description>ACL for HMasterRegionInterface protocol implementations\n"
-                                             + "    (for HRegionServers communicating with HMaster)\n"
-                                             + "    The ACL is a comma-separated list of user and group names. The user and \n"
-                                             + "    group list is separated by a blank. For e.g. \"alice,bob users,wheel\". \n"
-                                             + "    A special value of \"*\" means all users are allowed.</description>\n"
-                                             + "  </property>\n"
-                                             + "  <property>\n"
-                                             + "    <name>emptyVal</name>\n"
-                                             + "    <value></value>\n"
-                                             + "    <description>non-empty-desc</description>\n"
-                                             + "  </property>\n"
-                                             + "  <property>\n"
-                                             + "    <name>emptyDesc</name>\n"
-                                             + "    <value></value>\n"
-                                             + "    <description></description>\n"
-                                             + "  </property>\n"
-                                             + "  <property>\n"
-                                             + "    <name>noDesc</name>\n"
-                                             + "    <value></value>\n"
-                                             + "  </property>\n"
-                                             + "</configuration>";
-
-  @Test
-  public void testParse() throws IOException {
-
-    InputStream config_1 = new ByteArrayInputStream(config_1_str.getBytes());
-    DefaultConfig config = new DefaultConfigParser().parse(config_1);
-    Assert.assertNotNull(config);
-    Assert.assertNotNull(config.getPropertyInfos());
-    Assert.assertEquals(6, config.getPropertyInfos().size());
-    for (PropertyInfo pInfo : config.getPropertyInfos()) {
-      if (pInfo.getName().equals("security.client.protocol.acl")) {
-        Assert.assertEquals("*", pInfo.getValue());
-        Assert.assertTrue(pInfo.getDescription().startsWith("ACL for HRegionInterface "));
-      }
-      if (pInfo.getName().equals("emptyVal")) {
-        Assert.assertEquals("", pInfo.getValue());
-        Assert.assertEquals("non-empty-desc", pInfo.getDescription());
-      }
-      if (pInfo.getName().equals("emptyDesc")) {
-        Assert.assertEquals("", pInfo.getValue());
-        Assert.assertEquals("", pInfo.getDescription());
-      }
-      if (pInfo.getName().equals("noDesc")) {
-        Assert.assertEquals("", pInfo.getValue());
-        Assert.assertNull(pInfo.getDescription());
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestMetainfoParser.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestMetainfoParser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestMetainfoParser.java
deleted file mode 100644
index ba1912a..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestMetainfoParser.java
+++ /dev/null
@@ -1,177 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.apache.slider.providers.agent.AgentProviderService;
-import org.junit.Assert;
-import org.junit.Test;
-import org.mockito.Mockito;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.List;
-
-import static org.mockito.Mockito.doReturn;
-
-/**
- *
- */
-public class TestMetainfoParser {
-  protected static final Logger log =
-      LoggerFactory.getLogger(TestMetainfoParser.class);
-  public static final String METAINFO_XML =
-      "/org/apache/slider/providers/agent/application/metadata/metainfo.xml";
-
-  @Test
-  public void testParse() throws IOException {
-
-    InputStream resStream = this.getClass().getResourceAsStream(
-        METAINFO_XML);
-    MetainfoParser parser = new MetainfoParser();
-    Metainfo metainfo = parser.fromXmlStream(resStream);
-    Assert.assertNotNull(metainfo);
-    Assert.assertNotNull(metainfo.getApplication());
-    Application application = metainfo.getApplication();
-    assert "STORM".equals(application.getName());
-    assert 6 == application.getComponents().size();
-    OSPackage pkg = application.getOSSpecifics().get(0).getPackages().get(0);
-    assert "tarball".equals(pkg.getType());
-    assert "files/apache-storm-0.9.1.2.1.1.0-237.tar.gz".equals(pkg.getName());
-    boolean found = false;
-    for (Component comp : application.getComponents()) {
-      if (comp != null && comp.getName().equals("NIMBUS")) {
-        found = true;
-        Assert.assertEquals(0, comp.getComponentExports().size());
-      }
-      if (comp != null && comp.getName().equals("SUPERVISOR")) {
-        Assert.assertEquals(1, comp.getComponentExports().size());
-      }
-      if (comp != null && comp.getName().equals("ANOTHER_COMPONENT")) {
-        assert 2 == comp.getCommands().size();
-        assert "start command".equals(comp.getCommands().get(0).getExec());
-        assert "START".equals(comp.getCommands().get(0).getName());
-        assert "stop command".equals(comp.getCommands().get(1).getExec());
-        assert "STOP".equals(comp.getCommands().get(1).getName());
-      }
-    }
-    assert found;
-    Assert.assertEquals(0, application.getConfigFiles().size());
-    assert 1 == application.getPackages().size();
-    Package p = application.getPackages().get(0);
-    assert "tarball".equals(p.getType());
-    assert "test-tarball-name.tgz".equals(p.getName());
-  }
-
-  @Test
-  public void testJsonParse() throws IOException {
-    String metaInfo1_json = "{\n"
-                            + "\"schemaVersion\":\"2.2\",\n"
-                            + "\"application\":{\n"
-                            +     "\"name\": \"MEMCACHED\","
-                            +     "\"exportGroups\": ["
-                            +        "{"
-                            +          "\"name\": \"Servers\","
-                            +          "\"exports\": ["
-                            +            "{"
-                            +               "\"name\": \"host_port\","
-                            +               "\"value\": \"${MEMCACHED_HOST}:${site.global.port}\""
-                            +            "}"
-                            +          "]"
-                            +        "}"
-                            +      "],"
-                            +     "\"components\": ["
-                            +        "{"
-                            +          "\"name\": \"MEMCACHED\","
-                            +          "\"compExports\": \"Servers-host_port\","
-                            +          "\"commands\": ["
-                            +            "{"
-                            +               "\"exec\": \"java -classpath /usr/myapps/memcached/*:/usr/lib/hadoop/lib/* com.thimbleware.jmemcached.Main\""
-                            +            "}"
-                            +          "]"
-                            +        "},"
-                            +        "{"
-                            +          "\"name\": \"MEMCACHED2\","
-                            +          "\"commands\": ["
-                            +            "{"
-                            +               "\"exec\": \"scripts/config.py\","
-                            +               "\"type\": \"PYTHON\","
-                            +               "\"name\": \"CONFIGURE\""
-                            +            "}"
-                            +          "],"
-                            +          "\"dockerContainers\": ["
-                            +            "{"
-                            +               "\"name\": \"redis\","
-                            +               "\"image\": \"dockerhub/redis\","
-                            +               "\"options\": \"-net=bridge\","
-                            +               "\"mounts\": ["
-                            +                 "{"
-                            +                   "\"containerMount\": \"/tmp/conf\","
-                            +                   "\"hostMount\": \"{$conf:@//site/global/app_root}/conf\""
-                            +                 "}"
-                            +               "]"
-                            +            "}"
-                            +          "]"
-                            +        "}"
-                            +      "]"
-                            +   "}"
-                            + "}";
-
-    MetainfoParser parser = new MetainfoParser();
-    Metainfo mInfo = parser.fromJsonString(metaInfo1_json);
-    Assert.assertEquals("2.2", mInfo.getSchemaVersion());
-
-    Application app = mInfo.getApplication();
-    Assert.assertNotNull(app);
-
-    Assert.assertEquals("MEMCACHED", app.getName());
-    List<ExportGroup> egs = app.getExportGroups();
-    Assert.assertEquals(1, egs.size());
-    ExportGroup eg = egs.get(0);
-    Assert.assertEquals("Servers", eg.getName());
-    List<Export> exports = eg.getExports();
-    Assert.assertEquals(1, exports.size());
-    Export export = exports.get(0);
-    Assert.assertEquals("host_port", export.getName());
-    Assert.assertEquals("${MEMCACHED_HOST}:${site.global.port}", export.getValue());
-
-    List<Component> components = app.getComponents();
-    Assert.assertEquals(2, components.size());
-
-    Component c1 = mInfo.getApplicationComponent("MEMCACHED");
-    Assert.assertNotNull(c1);
-    Assert.assertEquals("MEMCACHED", c1.getName());
-    Assert.assertEquals("Servers-host_port", c1.getCompExports());
-    Assert.assertEquals(1, c1.getCommands().size());
-    ComponentCommand cmd = c1.getCommands().get(0);
-    Assert.assertEquals("START", cmd.getName());
-    Assert.assertEquals("SHELL", cmd.getType());
-    Assert.assertEquals("java -classpath /usr/myapps/memcached/*:/usr/lib/hadoop/lib/* com.thimbleware.jmemcached.Main",
-                        cmd.getExec());
-
-    Component c2 = mInfo.getApplicationComponent("MEMCACHED2");
-    Assert.assertNotNull(c2);
-    Assert.assertEquals("MEMCACHED2", c2.getName());
-    Assert.assertEquals(1, c2.getCommands().size());
-    cmd = c2.getCommands().get(0);
-    Assert.assertEquals("CONFIGURE", cmd.getName());
-    Assert.assertEquals("PYTHON", cmd.getType());
-    Assert.assertEquals("scripts/config.py", cmd.getExec());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestAgentProviderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestAgentProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestAgentProviderService.java
deleted file mode 100644
index 7fceac7..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestAgentProviderService.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.server.appmaster.web.rest.publisher;
-
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.slider.common.tools.SliderFileSystem;
-import org.apache.slider.providers.agent.AgentProviderService;
-import org.apache.slider.server.appmaster.actions.QueueAccess;
-import org.apache.slider.server.appmaster.state.StateAccessForProviders;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- *
- */
-public class TestAgentProviderService extends AgentProviderService {
-  protected static final Logger log =
-      LoggerFactory.getLogger(TestAgentProviderService.class);
-
-  public TestAgentProviderService() {
-    super();
-    log.info("TestAgentProviderService created");
-  }
-
-  @Override
-  public void bind(StateAccessForProviders stateAccessor,
-      QueueAccess queueAccess,
-      List<Container> liveContainers) {
-    super.bind(stateAccessor, queueAccess, liveContainers);
-    Map<String,String> dummyProps = new HashMap<String, String>();
-    dummyProps.put("prop1", "val1");
-    dummyProps.put("prop2", "val2");
-    log.info("publishing dummy-site.xml with values {}", dummyProps);
-    publishApplicationInstanceData("dummy-site", "dummy configuration",
-                                   dummyProps.entrySet());
-    // publishing global config for testing purposes
-    publishApplicationInstanceData("global", "global configuration",
-                                   stateAccessor.getAppConfSnapshot()
-                                       .getGlobalOptions().entrySet());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestSliderProviderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestSliderProviderFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestSliderProviderFactory.java
deleted file mode 100644
index f49e15a..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestSliderProviderFactory.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.server.appmaster.web.rest.publisher;
-
-import org.apache.slider.providers.ProviderService;
-import org.apache.slider.providers.agent.AgentProviderFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- *
- */
-public class TestSliderProviderFactory extends AgentProviderFactory{
-  protected static final Logger log =
-      LoggerFactory.getLogger(TestSliderProviderFactory.class);
-
-  public TestSliderProviderFactory() {
-    log.info("Created TestSliderProviderFactory");
-  }
-
-  @Override
-  public ProviderService createServerProvider() {
-    log.info("Creating TestAgentProviderService");
-    return new TestAgentProviderService();
-  }
-}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[08/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java
new file mode 100644
index 0000000..ad91183
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/StateAccessForProviders.java
@@ -0,0 +1,313 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.slider.api.ClusterDescription;
+import org.apache.slider.api.ClusterNode;
+import org.apache.slider.api.StatusKeys;
+import org.apache.slider.api.types.ApplicationLivenessInformation;
+import org.apache.slider.api.types.ComponentInformation;
+import org.apache.slider.api.types.NodeInformation;
+import org.apache.slider.api.types.RoleStatistics;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.core.exceptions.NoSuchNodeException;
+import org.apache.slider.core.registry.docstore.PublishedConfigSet;
+import org.apache.slider.core.registry.docstore.PublishedExportsSet;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * The methods to offer state access to the providers and other parts of
+ * the system which want read-only access to the state.
+ */
+public interface StateAccessForProviders {
+
+  /**
+   * Get a map of role status entries by role Id
+   * @return the map of currently defined roles.
+   */
+  Map<Integer, RoleStatus> getRoleStatusMap();
+
+  /**
+   * Get the name of the application
+   * @return the name
+   */
+  String getApplicationName();
+
+  /**
+   * Get the published configurations
+   * @return the configuration set
+   */
+  PublishedConfigSet getPublishedSliderConfigurations();
+
+  /**
+   * Get the published exports set
+   * @return
+   */
+  PublishedExportsSet getPublishedExportsSet();
+
+  /**
+   * Get a named published config set
+   * @param name name to look up
+   * @return the instance or null
+   */
+  PublishedConfigSet getPublishedConfigSet(String name);
+
+  /**
+   * Get a named published config set, creating it if need be.
+   * @param name name to look up
+   * @return the instance -possibly a new one
+   */
+  PublishedConfigSet getOrCreatePublishedConfigSet(String name);
+
+  /**
+   * List the config sets -this takes a clone of the current set
+   * @return a list of config sets
+   */
+  List<String> listConfigSets();
+
+  /**
+   * Get a map of all the failed containers
+   * @return map of recorded failed containers
+   */
+  Map<ContainerId, RoleInstance> getFailedContainers();
+
+  /**
+   * Get the live containers.
+   * 
+   * @return the live nodes
+   */
+  Map<ContainerId, RoleInstance> getLiveContainers();
+
+  /**
+   * Get the current cluster description 
+   * @return the actual state of the cluster
+   */
+  ClusterDescription getClusterStatus();
+
+  /**
+   * Get at the snapshot of the resource config
+   * Changes here do not affect the application state.
+   * @return the most recent settings
+   */
+  ConfTreeOperations getResourcesSnapshot();
+
+  /**
+   * Get at the snapshot of the appconf config
+   * Changes here do not affect the application state.
+   * @return the most recent settings
+   */
+  ConfTreeOperations getAppConfSnapshot();
+
+  /**
+   * Get at the snapshot of the internals config.
+   * Changes here do not affect the application state.
+   * @return the internals settings
+   */
+
+  ConfTreeOperations getInternalsSnapshot();
+
+  /**
+   * Flag set to indicate the application is live -this only happens
+   * after the buildInstance operation
+   */
+  boolean isApplicationLive();
+
+  long getSnapshotTime();
+
+  /**
+   * Get a snapshot of the entire aggregate configuration
+   * @return the aggregate configuration
+   */
+  AggregateConf getInstanceDefinitionSnapshot();
+
+  /**
+   * Get the desired/unresolved value
+   * @return unresolved
+   */
+  AggregateConf getUnresolvedInstanceDefinition();
+
+  /**
+   * Look up a role from its key -or fail 
+   *
+   * @param key key to resolve
+   * @return the status
+   * @throws YarnRuntimeException on no match
+   */
+  RoleStatus lookupRoleStatus(int key);
+
+  /**
+   * Look up a role from its key -or fail 
+   *
+   * @param c container in a role
+   * @return the status
+   * @throws YarnRuntimeException on no match
+   */
+  RoleStatus lookupRoleStatus(Container c) throws YarnRuntimeException;
+
+  /**
+   * Look up a role from its key -or fail 
+   *
+   * @param name container in a role
+   * @return the status
+   * @throws YarnRuntimeException on no match
+   */
+  RoleStatus lookupRoleStatus(String name) throws YarnRuntimeException;
+
+  /**
+   * Clone a list of active containers
+   * @return the active containers at the time
+   * the call was made
+   */
+  List<RoleInstance> cloneOwnedContainerList();
+
+  /**
+   * Get the number of active containers
+   * @return the number of active containers the time the call was made
+   */
+  int getNumOwnedContainers();
+
+  /**
+   * Get any active container with the given ID
+   * @param id container Id
+   * @return the active container or null if it is not found
+   */
+  RoleInstance getOwnedContainer(ContainerId id);
+
+  /**
+   * Get any active container with the given ID
+   * @param id container Id
+   * @return the active container or null if it is not found
+   */
+  RoleInstance getOwnedContainer(String id) throws NoSuchNodeException;
+
+  /**
+   * Create a clone of the list of live cluster nodes.
+   * @return the list of nodes, may be empty
+   */
+  List<RoleInstance> cloneLiveContainerInfoList();
+
+  /**
+   * Get the {@link RoleInstance} details on a container.
+   * This is an O(n) operation
+   * @param containerId the container ID
+   * @return null if there is no such node
+   * @throws NoSuchNodeException if the node cannot be found
+   */
+  RoleInstance getLiveInstanceByContainerID(String containerId)
+    throws NoSuchNodeException;
+
+  /**
+   * Get the details on a list of instaces referred to by ID.
+   * Unknown nodes are not returned
+   * <i>Important: the order of the results are undefined</i>
+   * @param containerIDs the containers
+   * @return list of instances
+   */
+  List<RoleInstance> getLiveInstancesByContainerIDs(
+    Collection<String> containerIDs);
+
+  /**
+   * Update the cluster description with anything interesting
+   */
+  ClusterDescription refreshClusterStatus();
+
+  /**
+   * Get a deep clone of the role status list. Concurrent events may mean this
+   * list (or indeed, some of the role status entries) may be inconsistent
+   * @return a snapshot of the role status entries
+   */
+  List<RoleStatus> cloneRoleStatusList();
+
+  /**
+   * get application liveness information
+   * @return a snapshot of the current liveness information
+   */
+  ApplicationLivenessInformation getApplicationLivenessInformation();
+
+  /**
+   * Get the live statistics map
+   * @return a map of statistics values, defined in the {@link StatusKeys}
+   * keylist.
+   */
+  Map<String, Integer> getLiveStatistics();
+
+  /**
+   * Get a snapshot of component information.
+   * <p>
+   *   This does <i>not</i> include any container list, which 
+   *   is more expensive to create.
+   * @return a map of current role status values.
+   */
+  Map<String, ComponentInformation> getComponentInfoSnapshot();
+
+  /**
+   * Find out about the nodes for specific roles
+   * @return 
+   */
+  Map<String, Map<String, ClusterNode>> getRoleClusterNodeMapping();
+
+  /**
+   * Enum all role instances by role.
+   * @param role role, or "" for all roles
+   * @return a list of instances, may be empty
+   */
+  List<RoleInstance> enumLiveInstancesInRole(String role);
+
+  /**
+   * Look up all containers of a specific component name 
+   * @param component component/role name
+   * @return list of instances. This is a snapshot
+   */
+  List<RoleInstance> lookupRoleContainers(String component);
+
+  /**
+   * Get the JSON serializable information about a component
+   * @param component component to look up
+   * @return a structure describing the component.
+   */
+  ComponentInformation getComponentInformation(String component);
+
+
+  /**
+   * Get a clone of the nodemap.
+   * The instances inside are not cloned
+   * @return a possibly empty map of hostname top info
+   */
+  Map<String, NodeInformation> getNodeInformationSnapshot();
+
+  /**
+   * get information on a node
+   * @param hostname hostname to look up
+   * @return the information, or null if there is no information held.
+   */
+  NodeInformation getNodeInformation(String hostname);
+
+  /**
+   * Get the aggregate statistics across all roles
+   * @return role statistics
+   */
+  RoleStatistics getRoleStatistics();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/AgentService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/AgentService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/AgentService.java
new file mode 100644
index 0000000..f840035
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/AgentService.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web;
+
+import org.apache.slider.server.appmaster.web.rest.agent.AgentWebApp;
+import org.apache.slider.server.services.workflow.ClosingService;
+import org.apache.slider.server.services.workflow.WorkflowCompositeService;
+
+/**
+ * agent service gives the agent webapp lifecycle integration
+ */
+public class AgentService extends ClosingService<AgentWebApp> {
+
+
+  public AgentService(String name) {
+    super(name);
+  }
+
+  public AgentService(String name, AgentWebApp app) {
+    super(name, app);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/HttpCacheHeaders.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/HttpCacheHeaders.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/HttpCacheHeaders.java
new file mode 100644
index 0000000..be8960d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/HttpCacheHeaders.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web;
+
+/*
+
+
+  ,  );
+  long now = System.currentTimeMillis();
+  httpRes.addDateHeader ( "Expires", now );
+  httpRes.addDateHeader ( "Date", now );
+  httpRes.addHeader ( "Pragma", "no-cache" );
+ */
+public interface HttpCacheHeaders {
+  String HTTP_HEADER_CACHE_CONTROL = "Cache-Control";
+  String HTTP_HEADER_CACHE_CONTROL_NONE = "no-cache";
+  String HTTP_HEADER_PRAGMA = "Pragma";
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMController.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMController.java
new file mode 100644
index 0000000..c3c6e60
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMController.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web;
+
+import com.google.inject.Inject;
+import org.apache.hadoop.yarn.webapp.Controller;
+import org.apache.slider.server.appmaster.web.layout.AppLayout;
+import org.apache.slider.server.appmaster.web.layout.ClusterSpecificationView;
+import org.apache.slider.server.appmaster.web.layout.ContainerStatsView;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 
+ */
+public class SliderAMController extends Controller {
+  private static final Logger log = LoggerFactory.getLogger(SliderAMController.class);
+
+  private final WebAppApi slider;
+  
+  @Inject
+  public SliderAMController(WebAppApi slider, RequestContext ctx) {
+    super(ctx);
+    this.slider = slider;
+  }
+  
+  @Override
+  public void index() {
+    setTitle("Slider App Master");
+    
+    updateAppState();
+    
+    render(AppLayout.class);
+  }
+  
+  public void containerStats() {
+    setTitle("Container Statistics");
+    
+    updateAppState();
+    
+    render(ContainerStatsView.class);
+  }
+  
+  public void specification() {
+    setTitle("Cluster Specification");
+    
+    render(ClusterSpecificationView.class);
+  }
+
+  private void updateAppState() {
+    //TODO don't do this on every request?
+    slider.getAppState().refreshClusterStatus();
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java
new file mode 100644
index 0000000..7ecc00c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web;
+
+import com.codahale.metrics.servlets.HealthCheckServlet;
+import com.codahale.metrics.servlets.MetricsServlet;
+import com.codahale.metrics.servlets.PingServlet;
+import com.codahale.metrics.servlets.ThreadDumpServlet;
+import com.google.common.base.Preconditions;
+import com.sun.jersey.api.container.filter.GZIPContentEncodingFilter;
+import com.sun.jersey.api.core.ResourceConfig;
+import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
+import com.sun.jersey.spi.container.servlet.ServletContainer;
+import org.apache.hadoop.yarn.webapp.Dispatcher;
+import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.WebApp;
+import org.apache.slider.server.appmaster.management.MetricsAndMonitoring;
+import org.apache.slider.server.appmaster.web.rest.AMWadlGeneratorConfig;
+import org.apache.slider.server.appmaster.web.rest.AMWebServices;
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.*;
+import org.apache.slider.server.appmaster.web.rest.SliderJacksonJaxbJsonProvider;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * 
+ */
+public class SliderAMWebApp extends WebApp {
+  public static final String BASE_PATH = "slideram";
+  public static final String CONTAINER_STATS = "/stats";
+  public static final String CLUSTER_SPEC = "/spec";
+
+  private final WebAppApi webAppApi;
+
+  public SliderAMWebApp(WebAppApi webAppApi) {
+    Preconditions.checkArgument(webAppApi != null, "webAppApi null");
+    this.webAppApi = webAppApi;
+  }
+
+  @Override
+  public void setup() {
+    Logger.getLogger("com.sun.jersey").setLevel(Level.FINEST);
+    // Make one of these to ensure that the jax-b annotations
+    // are properly picked up.
+    bind(SliderJacksonJaxbJsonProvider.class);
+    
+    // Get exceptions printed to the screen
+    bind(GenericExceptionHandler.class);
+    // bind the REST interface
+    bind(AMWebServices.class);
+    //bind(AMAgentWebServices.class);
+    route("/", SliderAMController.class);
+    route(CONTAINER_STATS, SliderAMController.class, "containerStats");
+    route(CLUSTER_SPEC, SliderAMController.class, "specification");
+  }
+
+  @Override
+  public void configureServlets() {
+    setup();
+
+    serve("/", "/__stop").with(Dispatcher.class);
+
+    for (String path : this.getServePathSpecs()) {
+      serve(path).with(Dispatcher.class);
+    }
+
+    // metrics
+    MetricsAndMonitoring monitoring =
+        webAppApi.getMetricsAndMonitoring();
+    serve(SYSTEM_HEALTHCHECK).with(new HealthCheckServlet(monitoring.getHealth()));
+    serve(SYSTEM_METRICS).with(new MetricsServlet(monitoring.getMetrics()));
+    serve(SYSTEM_PING).with(new PingServlet());
+    serve(SYSTEM_THREADS).with(new ThreadDumpServlet());
+
+    String regex = "(?!/ws)";
+    serveRegex(regex).with(SliderDefaultWrapperServlet.class); 
+
+    Map<String, String> params = new HashMap<>();
+    params.put(ResourceConfig.FEATURE_IMPLICIT_VIEWABLES, "true");
+    params.put(ServletContainer.FEATURE_FILTER_FORWARD_ON_404, "true");
+    params.put(ResourceConfig.FEATURE_XMLROOTELEMENT_PROCESSING, "true");
+    params.put(ResourceConfig.PROPERTY_CONTAINER_REQUEST_FILTERS, GZIPContentEncodingFilter.class.getName());
+    params.put(ResourceConfig.PROPERTY_CONTAINER_RESPONSE_FILTERS, GZIPContentEncodingFilter.class.getName());
+    //params.put("com.sun.jersey.spi.container.ContainerRequestFilters", "com.sun.jersey.api.container.filter.LoggingFilter");
+    //params.put("com.sun.jersey.spi.container.ContainerResponseFilters", "com.sun.jersey.api.container.filter.LoggingFilter");
+    //params.put("com.sun.jersey.config.feature.Trace", "true");
+    params.put("com.sun.jersey.config.property.WadlGeneratorConfig",
+        AMWadlGeneratorConfig.CLASSNAME);
+    filter("/*").through(GuiceContainer.class, params);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderDefaultWrapperServlet.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderDefaultWrapperServlet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderDefaultWrapperServlet.java
new file mode 100644
index 0000000..12c41ac
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderDefaultWrapperServlet.java
@@ -0,0 +1,48 @@
+/*
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  *     http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+package org.apache.slider.server.appmaster.web;
+
+import com.google.inject.Singleton;
+import org.apache.hadoop.yarn.webapp.DefaultWrapperServlet;
+
+import javax.servlet.RequestDispatcher;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletRequestWrapper;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+
+/**
+  *
+  */
+@Singleton
+public class SliderDefaultWrapperServlet extends DefaultWrapperServlet {
+  @Override
+  protected void doPost(HttpServletRequest req, HttpServletResponse resp)
+      throws ServletException, IOException {
+    RequestDispatcher rd = getServletContext().getNamedDispatcher("default");
+
+    HttpServletRequest wrapped = new HttpServletRequestWrapper(req) {
+      public String getServletPath() {
+        return "";
+      }
+    };
+
+    rd.forward(wrapped, resp);
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
new file mode 100644
index 0000000..0f99d6d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web;
+
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.slider.api.SliderClusterProtocol;
+import org.apache.slider.providers.ProviderService;
+import org.apache.slider.server.appmaster.AppMasterActionOperations;
+import org.apache.slider.server.appmaster.actions.QueueAccess;
+import org.apache.slider.server.appmaster.management.MetricsAndMonitoring;
+import org.apache.slider.server.appmaster.state.AppState;
+import org.apache.slider.server.appmaster.state.RoleStatus;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+import org.apache.slider.server.appmaster.web.rest.agent.AgentRestOperations;
+import org.apache.slider.server.appmaster.web.rest.application.resources.ContentCache;
+import org.apache.slider.server.services.security.CertificateManager;
+
+import java.util.Map;
+
+/**
+ * Interface to pass information from the Slider AppMaster to the WebApp
+ */
+public interface WebAppApi {
+
+  /**
+   * The {@link AppState} for the current cluster
+   */
+  StateAccessForProviders getAppState();
+  
+  /**
+   * The {@link ProviderService} for the current cluster
+   */
+  ProviderService getProviderService();
+
+
+  /**
+   * The {@link CertificateManager} for the current cluster
+   */
+  CertificateManager getCertificateManager();
+
+  /**
+   * Generate a mapping from role name to its {@link RoleStatus}. Be aware that this
+   * is a computed value and not just a getter
+   */
+  Map<String, RoleStatus> getRoleStatusByName();
+
+  /**
+   * Returns an interface that can support the agent-based REST operations.
+   */
+  AgentRestOperations getAgentRestOperations();
+  
+  /**
+   * Registry operations accessor
+   * @return registry access
+   */
+  RegistryOperations getRegistryOperations();
+
+  /**
+   * Metrics and monitoring service
+   * @return the (singleton) instance
+   */
+  MetricsAndMonitoring getMetricsAndMonitoring();
+
+  /**
+   * Get the queue accessor
+   * @return the immediate and scheduled queues
+   */
+  QueueAccess getQueues();
+
+  /**
+   * API for AM operations
+   * @return current operations implementation
+   */
+  AppMasterActionOperations getAMOperations();
+
+  /**
+   * Local cache of content
+   * @return the cache
+   */
+  ContentCache getContentCache();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
new file mode 100644
index 0000000..a0fe310
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web;
+
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.slider.providers.ProviderService;
+import org.apache.slider.server.appmaster.AppMasterActionOperations;
+import org.apache.slider.server.appmaster.actions.QueueAccess;
+import org.apache.slider.server.appmaster.management.MetricsAndMonitoring;
+import org.apache.slider.server.appmaster.state.RoleStatus;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+import org.apache.slider.server.appmaster.web.rest.agent.AgentRestOperations;
+import org.apache.slider.server.appmaster.web.rest.application.resources.ContentCache;
+import org.apache.slider.server.services.security.CertificateManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * 
+ */
+public class WebAppApiImpl implements WebAppApi {
+  private static final Logger log = LoggerFactory.getLogger(WebAppApiImpl.class);
+
+  protected final StateAccessForProviders appState;
+  protected final ProviderService provider;
+  protected final CertificateManager certificateManager;
+  private final RegistryOperations registryOperations;
+  private final MetricsAndMonitoring metricsAndMonitoring;
+  private final QueueAccess queues;
+  private final AppMasterActionOperations appMasterOperations;
+  private final ContentCache contentCache;
+
+  public WebAppApiImpl(StateAccessForProviders appState,
+      ProviderService provider,
+      CertificateManager certificateManager,
+      RegistryOperations registryOperations,
+      MetricsAndMonitoring metricsAndMonitoring,
+      QueueAccess queues,
+      AppMasterActionOperations appMasterOperations,
+      ContentCache contentCache) {
+    this.appMasterOperations = appMasterOperations;
+    this.contentCache = contentCache;
+    checkNotNull(appState);
+    checkNotNull(provider);
+    this.queues = queues;
+
+    this.registryOperations = registryOperations;
+    this.appState = appState;
+    this.provider = provider;
+    this.certificateManager = certificateManager;
+    this.metricsAndMonitoring = metricsAndMonitoring;
+  }
+
+  @Override
+  public StateAccessForProviders getAppState() {
+    return appState;
+  }
+
+  @Override
+  public ProviderService getProviderService() {
+    return provider;
+  }
+
+  @Override
+  public CertificateManager getCertificateManager() {
+    return certificateManager;
+  }
+
+  @Override
+  public Map<String,RoleStatus> getRoleStatusByName() {
+    List<RoleStatus> roleStatuses = appState.cloneRoleStatusList();
+    Map<String, RoleStatus> map = new TreeMap<>();
+    for (RoleStatus status : roleStatuses) {
+      map.put(status.getName(), status);
+    }
+    return map;
+  }
+
+  @Override
+  public AgentRestOperations getAgentRestOperations() {
+    return provider.getAgentRestOperations();
+  }
+
+  @Override
+  public RegistryOperations getRegistryOperations() {
+    return registryOperations;
+  }
+
+  @Override
+  public MetricsAndMonitoring getMetricsAndMonitoring() {
+    return metricsAndMonitoring;
+  }
+
+  @Override
+  public QueueAccess getQueues() {
+    return queues;
+  }
+
+  @Override
+  public AppMasterActionOperations getAMOperations() {
+    return appMasterOperations;
+  }
+
+  @Override
+  public ContentCache getContentCache() {
+    return contentCache;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/AppLayout.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/AppLayout.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/AppLayout.java
new file mode 100644
index 0000000..d9a2cda
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/AppLayout.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.layout;
+
+import org.apache.hadoop.yarn.webapp.SubView;
+import org.apache.slider.server.appmaster.web.view.IndexBlock;
+
+/**
+ * 
+ */
+public class AppLayout extends WebUILayout {
+
+  @Override
+  protected Class<? extends SubView> content() {
+    return IndexBlock.class;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/ClusterSpecificationView.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/ClusterSpecificationView.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/ClusterSpecificationView.java
new file mode 100644
index 0000000..b54ca71
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/ClusterSpecificationView.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.layout;
+
+import org.apache.hadoop.yarn.webapp.SubView;
+import org.apache.slider.server.appmaster.web.view.ClusterSpecificationBlock;
+
+/**
+ * 
+ */
+public class ClusterSpecificationView extends WebUILayout {
+
+  @Override
+  protected Class<? extends SubView> content() {
+    return ClusterSpecificationBlock.class;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/ContainerStatsView.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/ContainerStatsView.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/ContainerStatsView.java
new file mode 100644
index 0000000..39ba0ad
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/ContainerStatsView.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.layout;
+
+import org.apache.hadoop.yarn.webapp.SubView;
+import org.apache.slider.server.appmaster.web.view.ContainerStatsBlock;
+
+
+
+/**
+ * 
+ */
+public class ContainerStatsView extends WebUILayout {
+
+  @Override
+  protected Class<? extends SubView> content() {
+    return ContainerStatsBlock.class;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/WebUILayout.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/WebUILayout.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/WebUILayout.java
new file mode 100644
index 0000000..1681f59
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/layout/WebUILayout.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.layout;
+
+import org.apache.hadoop.yarn.webapp.SubView;
+import org.apache.hadoop.yarn.webapp.view.TwoColumnLayout;
+import org.apache.slider.server.appmaster.web.view.NavBlock;
+
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION_ID;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID;
+
+/**
+ * 
+ */
+public class WebUILayout extends TwoColumnLayout {
+  
+  @Override 
+  protected void preHead(Page.HTML<_> html) {
+    set(ACCORDION_ID, "nav");
+    set(initID(ACCORDION, "nav"), "{autoHeight:false, active:0}");
+  }
+  
+  @Override
+  protected Class<? extends SubView> nav() {
+    return NavBlock.class;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWadlGenerator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWadlGenerator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWadlGenerator.java
new file mode 100644
index 0000000..05aaa5b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWadlGenerator.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest;
+
+import com.sun.jersey.server.wadl.ApplicationDescription;
+import com.sun.jersey.server.wadl.WadlGenerator;
+import com.sun.jersey.server.wadl.WadlGeneratorImpl;
+import com.sun.research.ws.wadl.Application;
+import com.sun.research.ws.wadl.Resource;
+import com.sun.research.ws.wadl.Resources;
+
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ *
+ */
+public class AMWadlGenerator extends WadlGeneratorImpl {
+  @Override
+  /**
+   * This method is called once the WADL application has been assembled, so it
+   * affords an opportunity to edit the resources presented by the WADL.  In
+   * this case, we're removing the internal "/agents" resources.
+   */
+  public void attachTypes(ApplicationDescription egd) {
+    super.attachTypes(egd);
+
+    Application application = egd.getApplication();
+    List<Resources> resources = application.getResources();
+
+    for (Resources appResources : resources) {
+      List<Resource> resourceList = appResources.getResource();
+      for (Resource appResource : resourceList) {
+        String path = appResource.getPath();
+        if (RestPaths.SLIDER_CONTEXT_ROOT.equals(path)) {
+          List<Object> sliderResources = appResource.getMethodOrResource();
+          Iterator<Object> itor = sliderResources.iterator();
+          while (itor.hasNext()) {
+            Object sliderRes = itor.next();
+            if (sliderRes instanceof Resource) {
+              Resource res = (Resource) sliderRes;
+              if (RestPaths.SLIDER_SUBPATH_AGENTS.equals(res.getPath())) {
+                // assuming I'll get a list modification issue if I remove at this
+                // point
+                itor.remove();
+              }
+            }
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public void setWadlGeneratorDelegate(WadlGenerator delegate) {
+    // do nothing
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWadlGeneratorConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWadlGeneratorConfig.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWadlGeneratorConfig.java
new file mode 100644
index 0000000..ea9f22b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWadlGeneratorConfig.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest;
+
+import com.sun.jersey.api.wadl.config.WadlGeneratorConfig;
+import com.sun.jersey.api.wadl.config.WadlGeneratorDescription;
+
+import java.util.List;
+
+/**
+ * App master's WADL generation support
+ */
+public class AMWadlGeneratorConfig extends WadlGeneratorConfig {
+
+  public static final String CLASSNAME = "org.apache.slider.server.appmaster.web.rest.AMWadlGeneratorConfig";
+  @Override
+  public List<WadlGeneratorDescription> configure() {
+    return generator(AMWadlGenerator.class).descriptions();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java
new file mode 100644
index 0000000..03bf703
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest;
+
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.apache.slider.server.appmaster.web.rest.agent.AgentResource;
+import org.apache.slider.server.appmaster.web.rest.application.ApplicationResource;
+import org.apache.slider.server.appmaster.web.rest.management.ManagementResource;
+import org.apache.slider.server.appmaster.web.rest.publisher.PublisherResource;
+import org.apache.slider.server.appmaster.web.rest.registry.RegistryResource;
+
+import javax.ws.rs.Path;
+
+/**
+ *  The available REST services exposed by a slider AM. 
+ */
+@Singleton
+@Path(RestPaths.SLIDER_CONTEXT_ROOT)
+public class AMWebServices {
+  
+  /** AM/WebApp info object */
+  private WebAppApi slider;
+  private final ManagementResource managementResource;
+  private final PublisherResource publisherResource;
+  private final RegistryResource registryResource;
+  private final ApplicationResource applicationResource;
+
+  @Inject
+  public AMWebServices(WebAppApi slider) {
+    this.slider = slider;
+    managementResource = new ManagementResource(slider);
+    publisherResource = new PublisherResource(slider);
+    registryResource = new RegistryResource(slider);
+    applicationResource = new ApplicationResource(slider);
+  }
+
+  @Path(RestPaths.SLIDER_SUBPATH_MANAGEMENT)
+  public ManagementResource getManagementResource() {
+    return managementResource;
+  }
+
+  @Path(RestPaths.SLIDER_SUBPATH_PUBLISHER)
+  public PublisherResource getPublisherResource() {
+    return publisherResource;
+  }
+ 
+  @Path(RestPaths.SLIDER_SUBPATH_REGISTRY)
+  public RegistryResource getRegistryResource() {
+    return registryResource;
+  }
+  
+  @Path(RestPaths.SLIDER_SUBPATH_APPLICATION)
+  public ApplicationResource getApplicationResource() {
+    return applicationResource;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AbstractSliderResource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AbstractSliderResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AbstractSliderResource.java
new file mode 100644
index 0000000..7ff83b6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AbstractSliderResource.java
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest;
+
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.registry.client.exceptions.AuthenticationFailedException;
+import org.apache.hadoop.registry.client.exceptions.NoPathPermissionsException;
+import org.apache.hadoop.yarn.webapp.ForbiddenException;
+import org.apache.hadoop.yarn.webapp.NotFoundException;
+import org.apache.slider.server.appmaster.management.MetricsAndMonitoring;
+import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Response;
+import java.io.FileNotFoundException;
+import java.net.URI;
+import java.net.URL;
+
+/**
+ * Abstract resource base class for REST resources
+ * that use the slider WebAppApi
+ */
+public abstract class AbstractSliderResource {
+  private static final Logger log =
+      LoggerFactory.getLogger(AbstractSliderResource.class);
+  protected final WebAppApi slider;
+  protected final MetricsAndMonitoring metricsAndMonitoring;
+
+  protected AbstractSliderResource(WebAppApi slider) {
+    this.slider = slider;
+    metricsAndMonitoring = slider.getMetricsAndMonitoring();
+  }
+
+  /**
+   * Generate a redirect to the WASL
+   * @param request to base the URL on
+   * @return a 302 response
+   */
+  protected Response redirectToAppWadl(HttpServletRequest request) {
+    try {
+      URI location = new URL(request.getScheme(),
+          request.getServerName(),
+          request.getServerPort(),
+          RestPaths.APPLICATION_WADL).toURI();
+      return Response.temporaryRedirect(location).build();
+    } catch (Exception e) {
+      log.error("Error during redirect to WADL", e);
+      throw new WebApplicationException(Response.serverError().build());
+    }
+  }
+
+  /**
+   * Convert any exception caught into a web application
+   * exception for rethrowing
+   * @param path path of request
+   * @param ex exception
+   * @return an exception to throw
+   */
+  public WebApplicationException buildException(String path,
+      Exception ex) {
+    try {
+      throw ex;
+    } catch (WebApplicationException e) {
+      // rethrow direct
+      throw e;
+    } catch (FileNotFoundException e) {
+      return new NotFoundException("Not found: " + path);
+    } catch (PathNotFoundException e) {
+      return new NotFoundException("Not found: " + path);
+    } catch (AuthenticationFailedException e) {
+      return new ForbiddenException(path);
+    } catch (NoPathPermissionsException e) {
+      return new ForbiddenException(path);
+    } catch (Exception e) {
+      log.error("Error during generation of response: {}", e, e);
+      return new WebApplicationException(e);
+    }
+  }
+
+  /**
+   * Mark an GET operation on a path
+   * @param verb HTTP Verb
+   * @param path path relative to slider API
+   */
+  protected void mark(String verb, String path) {
+    metricsAndMonitoring.markMeterAndCounter(verb + "-" + path);
+  }
+
+  /**
+   * Mark an GET operation on a path
+   * @param verb HTTP Verb
+   * @param path path relative to slider API
+   */
+  protected void mark(String verb, String path, String subpath) {
+    metricsAndMonitoring.markMeterAndCounter(verb + "-" + path + subpath);
+  }
+
+  /**
+   * Mark a GET operation on a path
+   * @param path path relative to slider API
+   */
+  protected void markGet(String path) {
+    mark("GET", path);
+  }
+
+  /**
+   * Mark a GET operation on a path
+   * @param path path relative to slider API
+   */
+  protected void markGet(String path, String subpath) {
+    mark("GET", path, subpath);
+  }
+
+  /**
+   * Mark a GET operation on a path
+   * @param path path relative to slider API
+   */
+  protected void markPost(String path, String subpath) {
+    mark("POST", path, subpath);
+  }
+
+  /**
+   * Mark a GET operation on a path
+   * @param path path relative to slider API
+   */
+  protected void markPut(String path, String subpath) {
+    mark("PUT", path, subpath);
+  }
+
+  /**
+   * Mark a GET operation on a path
+   * @param path path relative to slider API
+   */
+  protected void markDelete(String path, String subpath) {
+    mark("DELETE", path, subpath);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilter.java
new file mode 100644
index 0000000..d5b6b36
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilter.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest;
+
+import org.apache.hadoop.yarn.server.webproxy.WebAppProxyServlet;
+import org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter;
+import org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpPrincipal;
+import org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpServletRequestWrapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.Cookie;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+
+/**
+ * This is a filter which is used to forward insecure operations
+ * There's some metrics to track all operations too
+ */
+public class InsecureAmFilter extends AmIpFilter {
+  public static final String WS_CONTEXT_ROOT = "slider.rest.context.root";
+  protected static final Logger log =
+      LoggerFactory.getLogger(InsecureAmFilter.class);
+
+  private String wsContextRoot;
+
+
+  @Override
+  public void init(FilterConfig conf) throws ServletException {
+    super.init(conf);
+    wsContextRoot = conf.getInitParameter(WS_CONTEXT_ROOT);
+    if (wsContextRoot == null) {
+      throw new ServletException("No value set for " + WS_CONTEXT_ROOT);
+    }
+  }
+
+  private void rejectNonHttpRequests(ServletRequest req) throws
+      ServletException {
+    if (!(req instanceof HttpServletRequest)) {
+      throw new ServletException("This filter only works for HTTP/HTTPS");
+    }
+  }  
+
+  @Override
+  public void doFilter(ServletRequest req,
+      ServletResponse resp,
+      FilterChain chain) throws IOException, ServletException {
+    rejectNonHttpRequests(req);
+    HttpServletRequest httpReq = (HttpServletRequest) req;
+    HttpServletResponse httpResp = (HttpServletResponse) resp;
+
+
+    String requestURI = httpReq.getRequestURI();
+    if (requestURI == null || !requestURI.startsWith(wsContextRoot)) {
+      // hand off to the AM filter if it is not the context root
+      super.doFilter(req, resp, chain);
+      return;
+    }
+
+    String user = null;
+
+    if (httpReq.getCookies() != null) {
+      for (Cookie c : httpReq.getCookies()) {
+        if (WebAppProxyServlet.PROXY_USER_COOKIE_NAME.equals(c.getName())) {
+          user = c.getValue();
+          break;
+        }
+      }
+    }
+    
+    if (user == null) {
+      log.debug("Could not find " + WebAppProxyServlet.PROXY_USER_COOKIE_NAME
+               + " cookie, so user will not be set");
+      chain.doFilter(req, resp);
+    } else {
+      final AmIpPrincipal principal = new AmIpPrincipal(user);
+      ServletRequest requestWrapper = new AmIpServletRequestWrapper(httpReq,
+          principal);
+      chain.doFilter(requestWrapper, resp);
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilterInitializer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilterInitializer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilterInitializer.java
new file mode 100644
index 0000000..42a5bdd
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/InsecureAmFilterInitializer.java
@@ -0,0 +1,102 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.FilterContainer;
+import org.apache.hadoop.http.FilterInitializer;
+import org.apache.hadoop.http.HttpConfig;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class InsecureAmFilterInitializer extends FilterInitializer {
+  private static final String FILTER_NAME = "AM_PROXY_FILTER";
+  private static final String FILTER_CLASS =
+      InsecureAmFilter.class.getCanonicalName();
+  private static final String HTTPS_PREFIX = "https://";
+  private static final String HTTP_PREFIX = "http://";
+
+  static final String PROXY_HOSTS = "PROXY_HOSTS";
+  static final String PROXY_HOSTS_DELIMITER = ",";
+  static final String PROXY_URI_BASES = "PROXY_URI_BASES";
+  static final String PROXY_URI_BASES_DELIMITER = ",";
+
+  private Configuration configuration;
+
+  public static final String NAME =
+      "org.apache.slider.server.appmaster.web.rest.InsecureAmFilterInitializer";
+
+  @Override
+  public void initFilter(FilterContainer container, Configuration conf) {
+    configuration = conf;
+    Map<String, String> params = new HashMap<String, String>();
+    String proxy = WebAppUtils.getProxyHostAndPort(conf);
+    String[] parts = proxy.split(":");
+    params.put(InsecureAmFilter.PROXY_HOST, parts[0]);
+    // todo:  eventually call WebAppUtils.getHttpSchemePrefix
+    params.put(InsecureAmFilter.PROXY_URI_BASE, getHttpSchemePrefix()
+                                                + proxy +
+                                                getApplicationWebProxyBase());
+    params.put(InsecureAmFilter.WS_CONTEXT_ROOT, RestPaths.WS_CONTEXT_ROOT);
+    container.addFilter(FILTER_NAME, FILTER_CLASS, params);
+  }
+
+  private void classicAmFilterInitializerInit(FilterContainer container,
+      Configuration conf) {
+    Map<String, String> params = new HashMap<String, String>();
+    List<String> proxies = WebAppUtils.getProxyHostsAndPortsForAmFilter(conf);
+    StringBuilder sb = new StringBuilder();
+    for (String proxy : proxies) {
+      sb.append(proxy.split(":")[0]).append(PROXY_HOSTS_DELIMITER);
+    }
+    sb.setLength(sb.length() - 1);
+    params.put(PROXY_HOSTS, sb.toString());
+
+    String prefix = WebAppUtils.getHttpSchemePrefix(conf);
+    String proxyBase = getApplicationWebProxyBase();
+    sb = new StringBuilder();
+    for (String proxy : proxies) {
+      sb.append(prefix).append(proxy).append(proxyBase)
+        .append(PROXY_HOSTS_DELIMITER);
+    }
+    sb.setLength(sb.length() - 1);
+    params.put(PROXY_URI_BASES, sb.toString());
+
+  }
+
+  @VisibleForTesting
+  protected String getApplicationWebProxyBase() {
+    return System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV);
+  }
+
+  private String getHttpSchemePrefix() {
+    return HttpConfig.Policy.HTTPS_ONLY ==
+           HttpConfig.Policy.fromString(configuration
+               .get(
+                   YarnConfiguration.YARN_HTTP_POLICY_KEY,
+                   YarnConfiguration.YARN_HTTP_POLICY_DEFAULT))
+           ? HTTPS_PREFIX : HTTP_PREFIX;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
new file mode 100644
index 0000000..ae9eb0f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest;
+
+/**
+ * Paths in the REST App
+ */
+public class RestPaths {
+
+  public static final String WS_CONTEXT = "ws";
+  public static final String AGENT_WS_CONTEXT = "ws";
+
+  /**
+   * Root path for the web services context: {@value}
+   */
+  public static final String WS_CONTEXT_ROOT = "/" + WS_CONTEXT;
+
+  /**
+   * agent content root: {@value}
+   */
+  public static final String WS_AGENT_CONTEXT_ROOT = "/" + AGENT_WS_CONTEXT;
+  public static final String V1_SLIDER = "/v1/slider";
+  public static final String SLIDER_CONTEXT_ROOT = WS_CONTEXT_ROOT + V1_SLIDER;
+  public static final String RELATIVE_API = WS_CONTEXT + V1_SLIDER;
+  public static final String SLIDER_AGENT_CONTEXT_ROOT = WS_AGENT_CONTEXT_ROOT + V1_SLIDER;
+  public static final String MANAGEMENT = "mgmt";
+  public static final String SLIDER_SUBPATH_MANAGEMENT = "/" + MANAGEMENT;
+  public static final String SLIDER_SUBPATH_AGENTS = "/agents";
+  public static final String SLIDER_SUBPATH_PUBLISHER = "/publisher";
+
+
+  /**
+   * management path: {@value}
+   */
+  public static final String SLIDER_PATH_MANAGEMENT = SLIDER_CONTEXT_ROOT
+                                      + SLIDER_SUBPATH_MANAGEMENT;
+
+  public static final String RELATIVE_PATH_MANAGEMENT = RELATIVE_API
+                                      + SLIDER_SUBPATH_MANAGEMENT;
+
+  /**
+   * Agents: {@value}
+   */
+  public static final String SLIDER_PATH_AGENTS = SLIDER_AGENT_CONTEXT_ROOT
+                                      + SLIDER_SUBPATH_AGENTS;
+  
+  /**
+   * Publisher: {@value}
+   */
+  public static final String SLIDER_PATH_PUBLISHER = SLIDER_CONTEXT_ROOT
+                                      + SLIDER_SUBPATH_PUBLISHER;
+
+  public static final String RELATIVE_PATH_PUBLISHER = RELATIVE_API
+                                      + SLIDER_SUBPATH_PUBLISHER;
+
+  /**
+   * Registry subpath: {@value} 
+   */
+  public static final String SLIDER_SUBPATH_REGISTRY = "/registry";
+
+  /**
+   * Registry: {@value}
+   */
+  public static final String SLIDER_PATH_REGISTRY = SLIDER_CONTEXT_ROOT
+                                                    + SLIDER_SUBPATH_REGISTRY;
+  public static final String RELATIVE_PATH_REGISTRY = RELATIVE_API
+                                                    + SLIDER_SUBPATH_REGISTRY;
+
+
+  /**
+   * The regular expressions used to define valid configuration names/url path
+   * fragments: {@value}
+   */
+  public static final String PUBLISHED_CONFIGURATION_REGEXP
+      = "[a-z0-9][a-z0-9_\\+-]*";
+
+  public static final String PUBLISHED_CONFIGURATION_SET_REGEXP
+      = "[a-z0-9][a-z0-9_.\\+-]*";
+
+  public static final String SLIDER_CONFIGSET = "slider";
+  public static final String SLIDER_EXPORTS = "exports";
+
+  public static final String SLIDER_CLASSPATH = "classpath";
+
+  /**
+   * Codahale Metrics - base path: {@value}
+   */
+
+  public static final String SYSTEM = "/system";
+
+
+  /**
+   * Codahale Metrics - health: {@value}
+   */
+  public static final String SYSTEM_HEALTHCHECK = SYSTEM + "/health";
+  /**
+   * Codahale Metrics - metrics: {@value}
+   */
+  public static final String SYSTEM_METRICS = SYSTEM + "/metrics";
+  /**
+   * Codahale Metrics - metrics as JSON: {@value}
+   */
+  public static final String SYSTEM_METRICS_JSON = SYSTEM_METRICS + "?format=json";
+  /**
+   * Codahale Metrics - ping: {@value}
+   */
+  public static final String SYSTEM_PING = SYSTEM + "/ping";
+  /**
+   * Codahale Metrics - thread dump: {@value}
+   */
+  public static final String SYSTEM_THREADS = SYSTEM + "/threads";
+
+  /**
+   * application subpath
+   */
+  public static final String SLIDER_SUBPATH_APPLICATION = "/application";
+  
+  /**
+   * management path: {@value}
+   */
+  public static final String SLIDER_PATH_APPLICATION =
+      SLIDER_CONTEXT_ROOT + SLIDER_SUBPATH_APPLICATION;
+
+
+  public static final String APPLICATION_WADL = "/application.wadl";
+  public static final String LIVE = "/live";
+  public static final String LIVE_RESOURCES = "/live/resources";
+  public static final String LIVE_CONTAINERS = "/live/containers";
+  public static final String LIVE_COMPONENTS = "/live/components";
+  public static final String LIVE_NODES = "/live/nodes";
+  public static final String LIVE_LIVENESS = "/live/liveness";
+  public static final String LIVE_STATISTICS = "/live/statistics";
+  public static final String MODEL = "/model";
+  public static final String MODEL_DESIRED = MODEL +"/desired";
+  public static final String MODEL_DESIRED_APPCONF = MODEL_DESIRED +"/appconf";
+  public static final String MODEL_DESIRED_RESOURCES = MODEL_DESIRED +"/resources";
+  public static final String MODEL_RESOLVED = "/model/resolved";
+  public static final String MODEL_RESOLVED_APPCONF = MODEL_RESOLVED +"/appconf";
+  public static final String MODEL_RESOLVED_RESOURCES = MODEL_RESOLVED +"/resources";
+  public static final String MODEL_INTERNAL = "/model/internal";
+
+  public static final String ACTION = "/action";
+  public static final String ACTION_PING = ACTION + "/ping";
+  public static final String ACTION_STOP = ACTION + "/stop";
+
+  /**
+   * Path to a role
+   * @param name role name
+   * @return a path to it
+   */
+  public String pathToRole(String name) {
+
+    // ws/v1/slider/application/live/components/$name
+    return SLIDER_PATH_APPLICATION + LIVE_COMPONENTS + "/" + name;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/SliderJacksonJaxbJsonProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/SliderJacksonJaxbJsonProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/SliderJacksonJaxbJsonProvider.java
new file mode 100644
index 0000000..86d68a8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/SliderJacksonJaxbJsonProvider.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest;
+
+import com.google.inject.Singleton;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.codehaus.jackson.jaxrs.JacksonJaxbJsonProvider;
+import org.codehaus.jackson.map.AnnotationIntrospector;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.introspect.JacksonAnnotationIntrospector;
+import org.codehaus.jackson.xc.JaxbAnnotationIntrospector;
+
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.ext.Provider;
+
+/**
+ * Implementation of JAX-RS abstractions based on {@link
+ * JacksonJaxbJsonProvider} needed to deserialize JSON content to, or serialize
+ * it from, POJO objects.
+ */
+@Singleton
+@Provider
+@Unstable
+@Private
+public class SliderJacksonJaxbJsonProvider extends JacksonJaxbJsonProvider {
+
+  public SliderJacksonJaxbJsonProvider() {
+  }
+
+  @Override
+  public ObjectMapper locateMapper(Class<?> type, MediaType mediaType) {
+    ObjectMapper mapper = super.locateMapper(type, mediaType);
+    AnnotationIntrospector introspector = new AnnotationIntrospector.Pair(
+        new JaxbAnnotationIntrospector(),
+        new JacksonAnnotationIntrospector()
+    );
+    mapper.setAnnotationIntrospector(introspector);
+    //mapper.setSerializationInclusion(Inclusion.NON_NULL);
+    return mapper;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentCommandType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentCommandType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentCommandType.java
new file mode 100644
index 0000000..17cd8f2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentCommandType.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.agent;
+
+public enum AgentCommandType {
+  EXECUTION_COMMAND,
+  STATUS_COMMAND,
+  REGISTRATION_COMMAND
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[71/76] [abbrv] hadoop git commit: YARN-5961. Generate native services protobuf classes during build. Contributed by Billie Rinaldi

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc259d50/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/proto/SliderClusterAPI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/proto/SliderClusterAPI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/proto/SliderClusterAPI.java
deleted file mode 100644
index 081b7fa..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/proto/SliderClusterAPI.java
+++ /dev/null
@@ -1,2293 +0,0 @@
-// Generated by the protocol buffer compiler.  DO NOT EDIT!
-// source: SliderClusterProtocol.proto
-
-package org.apache.slider.api.proto;
-
-public final class SliderClusterAPI {
-  private SliderClusterAPI() {}
-  public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
-  }
-  /**
-   * Protobuf service {@code org.apache.slider.api.SliderClusterProtocolPB}
-   *
-   * <pre>
-   **
-   * Protocol used from between Slider Client and AM
-   * </pre>
-   */
-  public static abstract class SliderClusterProtocolPB
-      implements com.google.protobuf.Service {
-    protected SliderClusterProtocolPB() {}
-
-    public interface Interface {
-      /**
-       * <code>rpc stopCluster(.org.apache.slider.api.StopClusterRequestProto) returns (.org.apache.slider.api.StopClusterResponseProto);</code>
-       */
-      public abstract void stopCluster(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.StopClusterRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.StopClusterResponseProto> done);
-
-      /**
-       * <code>rpc upgradeContainers(.org.apache.slider.api.UpgradeContainersRequestProto) returns (.org.apache.slider.api.UpgradeContainersResponseProto);</code>
-       *
-       * <pre>
-       **
-       * Upgrade containers 
-       * </pre>
-       */
-      public abstract void upgradeContainers(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.UpgradeContainersRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.UpgradeContainersResponseProto> done);
-
-      /**
-       * <code>rpc flexCluster(.org.apache.slider.api.FlexClusterRequestProto) returns (.org.apache.slider.api.FlexClusterResponseProto);</code>
-       *
-       * <pre>
-       **
-       * Flex the cluster. 
-       * </pre>
-       */
-      public abstract void flexCluster(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.FlexClusterRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.FlexClusterResponseProto> done);
-
-      /**
-       * <code>rpc getJSONClusterStatus(.org.apache.slider.api.GetJSONClusterStatusRequestProto) returns (.org.apache.slider.api.GetJSONClusterStatusResponseProto);</code>
-       *
-       * <pre>
-       **
-       * Get the current cluster status
-       * </pre>
-       */
-      public abstract void getJSONClusterStatus(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetJSONClusterStatusRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetJSONClusterStatusResponseProto> done);
-
-      /**
-       * <code>rpc getInstanceDefinition(.org.apache.slider.api.GetInstanceDefinitionRequestProto) returns (.org.apache.slider.api.GetInstanceDefinitionResponseProto);</code>
-       *
-       * <pre>
-       **
-       * Get the instance definition
-       * </pre>
-       */
-      public abstract void getInstanceDefinition(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetInstanceDefinitionRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetInstanceDefinitionResponseProto> done);
-
-      /**
-       * <code>rpc listNodeUUIDsByRole(.org.apache.slider.api.ListNodeUUIDsByRoleRequestProto) returns (.org.apache.slider.api.ListNodeUUIDsByRoleResponseProto);</code>
-       *
-       * <pre>
-       **
-       * List all running nodes in a role
-       * </pre>
-       */
-      public abstract void listNodeUUIDsByRole(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleResponseProto> done);
-
-      /**
-       * <code>rpc getNode(.org.apache.slider.api.GetNodeRequestProto) returns (.org.apache.slider.api.GetNodeResponseProto);</code>
-       *
-       * <pre>
-       **
-       * Get the details on a node
-       * </pre>
-       */
-      public abstract void getNode(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetNodeRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetNodeResponseProto> done);
-
-      /**
-       * <code>rpc getClusterNodes(.org.apache.slider.api.GetClusterNodesRequestProto) returns (.org.apache.slider.api.GetClusterNodesResponseProto);</code>
-       *
-       * <pre>
-       **
-       * Get the 
-       * details on a list of nodes.
-       * Unknown nodes are not returned
-       * &lt;i&gt;Important: the order of the results are undefined&lt;/i&gt;
-       * </pre>
-       */
-      public abstract void getClusterNodes(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetClusterNodesRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetClusterNodesResponseProto> done);
-
-      /**
-       * <code>rpc echo(.org.apache.slider.api.EchoRequestProto) returns (.org.apache.slider.api.EchoResponseProto);</code>
-       *
-       * <pre>
-       **
-       * echo some text
-       * </pre>
-       */
-      public abstract void echo(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.EchoRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.EchoResponseProto> done);
-
-      /**
-       * <code>rpc killContainer(.org.apache.slider.api.KillContainerRequestProto) returns (.org.apache.slider.api.KillContainerResponseProto);</code>
-       *
-       * <pre>
-       **
-       * kill a container
-       * </pre>
-       */
-      public abstract void killContainer(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.KillContainerRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.KillContainerResponseProto> done);
-
-      /**
-       * <code>rpc amSuicide(.org.apache.slider.api.AMSuicideRequestProto) returns (.org.apache.slider.api.AMSuicideResponseProto);</code>
-       *
-       * <pre>
-       **
-       * kill the AM
-       * </pre>
-       */
-      public abstract void amSuicide(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.AMSuicideRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.AMSuicideResponseProto> done);
-
-      /**
-       * <code>rpc getLivenessInformation(.org.apache.slider.api.GetApplicationLivenessRequestProto) returns (.org.apache.slider.api.ApplicationLivenessInformationProto);</code>
-       */
-      public abstract void getLivenessInformation(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetApplicationLivenessRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ApplicationLivenessInformationProto> done);
-
-      /**
-       * <code>rpc getLiveContainers(.org.apache.slider.api.GetLiveContainersRequestProto) returns (.org.apache.slider.api.GetLiveContainersResponseProto);</code>
-       */
-      public abstract void getLiveContainers(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetLiveContainersRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveContainersResponseProto> done);
-
-      /**
-       * <code>rpc getLiveContainer(.org.apache.slider.api.GetLiveContainerRequestProto) returns (.org.apache.slider.api.ContainerInformationProto);</code>
-       */
-      public abstract void getLiveContainer(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetLiveContainerRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ContainerInformationProto> done);
-
-      /**
-       * <code>rpc getLiveComponents(.org.apache.slider.api.GetLiveComponentsRequestProto) returns (.org.apache.slider.api.GetLiveComponentsResponseProto);</code>
-       */
-      public abstract void getLiveComponents(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetLiveComponentsRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveComponentsResponseProto> done);
-
-      /**
-       * <code>rpc getLiveComponent(.org.apache.slider.api.GetLiveComponentRequestProto) returns (.org.apache.slider.api.ComponentInformationProto);</code>
-       */
-      public abstract void getLiveComponent(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetLiveComponentRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ComponentInformationProto> done);
-
-      /**
-       * <code>rpc getLiveNodes(.org.apache.slider.api.GetLiveNodesRequestProto) returns (.org.apache.slider.api.GetLiveNodesResponseProto);</code>
-       */
-      public abstract void getLiveNodes(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetLiveNodesRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveNodesResponseProto> done);
-
-      /**
-       * <code>rpc getLiveNode(.org.apache.slider.api.GetLiveNodeRequestProto) returns (.org.apache.slider.api.NodeInformationProto);</code>
-       */
-      public abstract void getLiveNode(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetLiveNodeRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.NodeInformationProto> done);
-
-      /**
-       * <code>rpc getModelDesired(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
-       *
-       * <pre>
-       * AggregateConf getModelDesired()
-       * </pre>
-       */
-      public abstract void getModelDesired(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
-
-      /**
-       * <code>rpc getModelDesiredAppconf(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
-       *
-       * <pre>
-       * ConfTree getModelDesiredAppconf
-       * </pre>
-       */
-      public abstract void getModelDesiredAppconf(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
-
-      /**
-       * <code>rpc getModelDesiredResources(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
-       *
-       * <pre>
-       * ConfTree getModelDesiredResources
-       * </pre>
-       */
-      public abstract void getModelDesiredResources(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
-
-      /**
-       * <code>rpc getModelResolved(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
-       *
-       * <pre>
-       * AggregateConf getModelResolved()
-       * </pre>
-       */
-      public abstract void getModelResolved(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
-
-      /**
-       * <code>rpc getModelResolvedAppconf(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
-       *
-       * <pre>
-       * ConfTree getModelResolvedAppconf
-       * </pre>
-       */
-      public abstract void getModelResolvedAppconf(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
-
-      /**
-       * <code>rpc getModelResolvedResources(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
-       *
-       * <pre>
-       * ConfTree getModelResolvedResources
-       * </pre>
-       */
-      public abstract void getModelResolvedResources(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
-
-      /**
-       * <code>rpc getLiveResources(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
-       *
-       * <pre>
-       * ConfTree getLiveResources
-       * </pre>
-       */
-      public abstract void getLiveResources(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
-
-      /**
-       * <code>rpc getClientCertificateStore(.org.apache.slider.api.GetCertificateStoreRequestProto) returns (.org.apache.slider.api.GetCertificateStoreResponseProto);</code>
-       */
-      public abstract void getClientCertificateStore(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetCertificateStoreRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetCertificateStoreResponseProto> done);
-
-    }
-
-    public static com.google.protobuf.Service newReflectiveService(
-        final Interface impl) {
-      return new SliderClusterProtocolPB() {
-        @java.lang.Override
-        public  void stopCluster(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.StopClusterRequestProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.StopClusterResponseProto> done) {
-          impl.stopCluster(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void upgradeContainers(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.UpgradeContainersRequestProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.UpgradeContainersResponseProto> done) {
-          impl.upgradeContainers(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void flexCluster(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.FlexClusterRequestProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.FlexClusterResponseProto> done) {
-          impl.flexCluster(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void getJSONClusterStatus(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.GetJSONClusterStatusRequestProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetJSONClusterStatusResponseProto> done) {
-          impl.getJSONClusterStatus(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void getInstanceDefinition(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.GetInstanceDefinitionRequestProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetInstanceDefinitionResponseProto> done) {
-          impl.getInstanceDefinition(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void listNodeUUIDsByRole(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleRequestProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleResponseProto> done) {
-          impl.listNodeUUIDsByRole(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void getNode(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.GetNodeRequestProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetNodeResponseProto> done) {
-          impl.getNode(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void getClusterNodes(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.GetClusterNodesRequestProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetClusterNodesResponseProto> done) {
-          impl.getClusterNodes(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void echo(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.EchoRequestProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.EchoResponseProto> done) {
-          impl.echo(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void killContainer(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.KillContainerRequestProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.KillContainerResponseProto> done) {
-          impl.killContainer(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void amSuicide(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.AMSuicideRequestProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.AMSuicideResponseProto> done) {
-          impl.amSuicide(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void getLivenessInformation(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.GetApplicationLivenessRequestProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ApplicationLivenessInformationProto> done) {
-          impl.getLivenessInformation(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void getLiveContainers(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.GetLiveContainersRequestProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveContainersResponseProto> done) {
-          impl.getLiveContainers(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void getLiveContainer(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.GetLiveContainerRequestProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ContainerInformationProto> done) {
-          impl.getLiveContainer(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void getLiveComponents(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.GetLiveComponentsRequestProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveComponentsResponseProto> done) {
-          impl.getLiveComponents(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void getLiveComponent(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.GetLiveComponentRequestProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ComponentInformationProto> done) {
-          impl.getLiveComponent(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void getLiveNodes(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.GetLiveNodesRequestProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveNodesResponseProto> done) {
-          impl.getLiveNodes(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void getLiveNode(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.GetLiveNodeRequestProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.NodeInformationProto> done) {
-          impl.getLiveNode(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void getModelDesired(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
-          impl.getModelDesired(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void getModelDesiredAppconf(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
-          impl.getModelDesiredAppconf(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void getModelDesiredResources(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
-          impl.getModelDesiredResources(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void getModelResolved(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
-          impl.getModelResolved(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void getModelResolvedAppconf(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
-          impl.getModelResolvedAppconf(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void getModelResolvedResources(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
-          impl.getModelResolvedResources(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void getLiveResources(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
-          impl.getLiveResources(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void getClientCertificateStore(
-            com.google.protobuf.RpcController controller,
-            org.apache.slider.api.proto.Messages.GetCertificateStoreRequestProto request,
-            com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetCertificateStoreResponseProto> done) {
-          impl.getClientCertificateStore(controller, request, done);
-        }
-
-      };
-    }
-
-    public static com.google.protobuf.BlockingService
-        newReflectiveBlockingService(final BlockingInterface impl) {
-      return new com.google.protobuf.BlockingService() {
-        public final com.google.protobuf.Descriptors.ServiceDescriptor
-            getDescriptorForType() {
-          return getDescriptor();
-        }
-
-        public final com.google.protobuf.Message callBlockingMethod(
-            com.google.protobuf.Descriptors.MethodDescriptor method,
-            com.google.protobuf.RpcController controller,
-            com.google.protobuf.Message request)
-            throws com.google.protobuf.ServiceException {
-          if (method.getService() != getDescriptor()) {
-            throw new java.lang.IllegalArgumentException(
-              "Service.callBlockingMethod() given method descriptor for " +
-              "wrong service type.");
-          }
-          switch(method.getIndex()) {
-            case 0:
-              return impl.stopCluster(controller, (org.apache.slider.api.proto.Messages.StopClusterRequestProto)request);
-            case 1:
-              return impl.upgradeContainers(controller, (org.apache.slider.api.proto.Messages.UpgradeContainersRequestProto)request);
-            case 2:
-              return impl.flexCluster(controller, (org.apache.slider.api.proto.Messages.FlexClusterRequestProto)request);
-            case 3:
-              return impl.getJSONClusterStatus(controller, (org.apache.slider.api.proto.Messages.GetJSONClusterStatusRequestProto)request);
-            case 4:
-              return impl.getInstanceDefinition(controller, (org.apache.slider.api.proto.Messages.GetInstanceDefinitionRequestProto)request);
-            case 5:
-              return impl.listNodeUUIDsByRole(controller, (org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleRequestProto)request);
-            case 6:
-              return impl.getNode(controller, (org.apache.slider.api.proto.Messages.GetNodeRequestProto)request);
-            case 7:
-              return impl.getClusterNodes(controller, (org.apache.slider.api.proto.Messages.GetClusterNodesRequestProto)request);
-            case 8:
-              return impl.echo(controller, (org.apache.slider.api.proto.Messages.EchoRequestProto)request);
-            case 9:
-              return impl.killContainer(controller, (org.apache.slider.api.proto.Messages.KillContainerRequestProto)request);
-            case 10:
-              return impl.amSuicide(controller, (org.apache.slider.api.proto.Messages.AMSuicideRequestProto)request);
-            case 11:
-              return impl.getLivenessInformation(controller, (org.apache.slider.api.proto.Messages.GetApplicationLivenessRequestProto)request);
-            case 12:
-              return impl.getLiveContainers(controller, (org.apache.slider.api.proto.Messages.GetLiveContainersRequestProto)request);
-            case 13:
-              return impl.getLiveContainer(controller, (org.apache.slider.api.proto.Messages.GetLiveContainerRequestProto)request);
-            case 14:
-              return impl.getLiveComponents(controller, (org.apache.slider.api.proto.Messages.GetLiveComponentsRequestProto)request);
-            case 15:
-              return impl.getLiveComponent(controller, (org.apache.slider.api.proto.Messages.GetLiveComponentRequestProto)request);
-            case 16:
-              return impl.getLiveNodes(controller, (org.apache.slider.api.proto.Messages.GetLiveNodesRequestProto)request);
-            case 17:
-              return impl.getLiveNode(controller, (org.apache.slider.api.proto.Messages.GetLiveNodeRequestProto)request);
-            case 18:
-              return impl.getModelDesired(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request);
-            case 19:
-              return impl.getModelDesiredAppconf(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request);
-            case 20:
-              return impl.getModelDesiredResources(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request);
-            case 21:
-              return impl.getModelResolved(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request);
-            case 22:
-              return impl.getModelResolvedAppconf(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request);
-            case 23:
-              return impl.getModelResolvedResources(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request);
-            case 24:
-              return impl.getLiveResources(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request);
-            case 25:
-              return impl.getClientCertificateStore(controller, (org.apache.slider.api.proto.Messages.GetCertificateStoreRequestProto)request);
-            default:
-              throw new java.lang.AssertionError("Can't get here.");
-          }
-        }
-
-        public final com.google.protobuf.Message
-            getRequestPrototype(
-            com.google.protobuf.Descriptors.MethodDescriptor method) {
-          if (method.getService() != getDescriptor()) {
-            throw new java.lang.IllegalArgumentException(
-              "Service.getRequestPrototype() given method " +
-              "descriptor for wrong service type.");
-          }
-          switch(method.getIndex()) {
-            case 0:
-              return org.apache.slider.api.proto.Messages.StopClusterRequestProto.getDefaultInstance();
-            case 1:
-              return org.apache.slider.api.proto.Messages.UpgradeContainersRequestProto.getDefaultInstance();
-            case 2:
-              return org.apache.slider.api.proto.Messages.FlexClusterRequestProto.getDefaultInstance();
-            case 3:
-              return org.apache.slider.api.proto.Messages.GetJSONClusterStatusRequestProto.getDefaultInstance();
-            case 4:
-              return org.apache.slider.api.proto.Messages.GetInstanceDefinitionRequestProto.getDefaultInstance();
-            case 5:
-              return org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleRequestProto.getDefaultInstance();
-            case 6:
-              return org.apache.slider.api.proto.Messages.GetNodeRequestProto.getDefaultInstance();
-            case 7:
-              return org.apache.slider.api.proto.Messages.GetClusterNodesRequestProto.getDefaultInstance();
-            case 8:
-              return org.apache.slider.api.proto.Messages.EchoRequestProto.getDefaultInstance();
-            case 9:
-              return org.apache.slider.api.proto.Messages.KillContainerRequestProto.getDefaultInstance();
-            case 10:
-              return org.apache.slider.api.proto.Messages.AMSuicideRequestProto.getDefaultInstance();
-            case 11:
-              return org.apache.slider.api.proto.Messages.GetApplicationLivenessRequestProto.getDefaultInstance();
-            case 12:
-              return org.apache.slider.api.proto.Messages.GetLiveContainersRequestProto.getDefaultInstance();
-            case 13:
-              return org.apache.slider.api.proto.Messages.GetLiveContainerRequestProto.getDefaultInstance();
-            case 14:
-              return org.apache.slider.api.proto.Messages.GetLiveComponentsRequestProto.getDefaultInstance();
-            case 15:
-              return org.apache.slider.api.proto.Messages.GetLiveComponentRequestProto.getDefaultInstance();
-            case 16:
-              return org.apache.slider.api.proto.Messages.GetLiveNodesRequestProto.getDefaultInstance();
-            case 17:
-              return org.apache.slider.api.proto.Messages.GetLiveNodeRequestProto.getDefaultInstance();
-            case 18:
-              return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
-            case 19:
-              return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
-            case 20:
-              return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
-            case 21:
-              return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
-            case 22:
-              return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
-            case 23:
-              return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
-            case 24:
-              return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
-            case 25:
-              return org.apache.slider.api.proto.Messages.GetCertificateStoreRequestProto.getDefaultInstance();
-            default:
-              throw new java.lang.AssertionError("Can't get here.");
-          }
-        }
-
-        public final com.google.protobuf.Message
-            getResponsePrototype(
-            com.google.protobuf.Descriptors.MethodDescriptor method) {
-          if (method.getService() != getDescriptor()) {
-            throw new java.lang.IllegalArgumentException(
-              "Service.getResponsePrototype() given method " +
-              "descriptor for wrong service type.");
-          }
-          switch(method.getIndex()) {
-            case 0:
-              return org.apache.slider.api.proto.Messages.StopClusterResponseProto.getDefaultInstance();
-            case 1:
-              return org.apache.slider.api.proto.Messages.UpgradeContainersResponseProto.getDefaultInstance();
-            case 2:
-              return org.apache.slider.api.proto.Messages.FlexClusterResponseProto.getDefaultInstance();
-            case 3:
-              return org.apache.slider.api.proto.Messages.GetJSONClusterStatusResponseProto.getDefaultInstance();
-            case 4:
-              return org.apache.slider.api.proto.Messages.GetInstanceDefinitionResponseProto.getDefaultInstance();
-            case 5:
-              return org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleResponseProto.getDefaultInstance();
-            case 6:
-              return org.apache.slider.api.proto.Messages.GetNodeResponseProto.getDefaultInstance();
-            case 7:
-              return org.apache.slider.api.proto.Messages.GetClusterNodesResponseProto.getDefaultInstance();
-            case 8:
-              return org.apache.slider.api.proto.Messages.EchoResponseProto.getDefaultInstance();
-            case 9:
-              return org.apache.slider.api.proto.Messages.KillContainerResponseProto.getDefaultInstance();
-            case 10:
-              return org.apache.slider.api.proto.Messages.AMSuicideResponseProto.getDefaultInstance();
-            case 11:
-              return org.apache.slider.api.proto.Messages.ApplicationLivenessInformationProto.getDefaultInstance();
-            case 12:
-              return org.apache.slider.api.proto.Messages.GetLiveContainersResponseProto.getDefaultInstance();
-            case 13:
-              return org.apache.slider.api.proto.Messages.ContainerInformationProto.getDefaultInstance();
-            case 14:
-              return org.apache.slider.api.proto.Messages.GetLiveComponentsResponseProto.getDefaultInstance();
-            case 15:
-              return org.apache.slider.api.proto.Messages.ComponentInformationProto.getDefaultInstance();
-            case 16:
-              return org.apache.slider.api.proto.Messages.GetLiveNodesResponseProto.getDefaultInstance();
-            case 17:
-              return org.apache.slider.api.proto.Messages.NodeInformationProto.getDefaultInstance();
-            case 18:
-              return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
-            case 19:
-              return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
-            case 20:
-              return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
-            case 21:
-              return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
-            case 22:
-              return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
-            case 23:
-              return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
-            case 24:
-              return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
-            case 25:
-              return org.apache.slider.api.proto.Messages.GetCertificateStoreResponseProto.getDefaultInstance();
-            default:
-              throw new java.lang.AssertionError("Can't get here.");
-          }
-        }
-
-      };
-    }
-
-    /**
-     * <code>rpc stopCluster(.org.apache.slider.api.StopClusterRequestProto) returns (.org.apache.slider.api.StopClusterResponseProto);</code>
-     */
-    public abstract void stopCluster(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.StopClusterRequestProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.StopClusterResponseProto> done);
-
-    /**
-     * <code>rpc upgradeContainers(.org.apache.slider.api.UpgradeContainersRequestProto) returns (.org.apache.slider.api.UpgradeContainersResponseProto);</code>
-     *
-     * <pre>
-     **
-     * Upgrade containers 
-     * </pre>
-     */
-    public abstract void upgradeContainers(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.UpgradeContainersRequestProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.UpgradeContainersResponseProto> done);
-
-    /**
-     * <code>rpc flexCluster(.org.apache.slider.api.FlexClusterRequestProto) returns (.org.apache.slider.api.FlexClusterResponseProto);</code>
-     *
-     * <pre>
-     **
-     * Flex the cluster. 
-     * </pre>
-     */
-    public abstract void flexCluster(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.FlexClusterRequestProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.FlexClusterResponseProto> done);
-
-    /**
-     * <code>rpc getJSONClusterStatus(.org.apache.slider.api.GetJSONClusterStatusRequestProto) returns (.org.apache.slider.api.GetJSONClusterStatusResponseProto);</code>
-     *
-     * <pre>
-     **
-     * Get the current cluster status
-     * </pre>
-     */
-    public abstract void getJSONClusterStatus(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.GetJSONClusterStatusRequestProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetJSONClusterStatusResponseProto> done);
-
-    /**
-     * <code>rpc getInstanceDefinition(.org.apache.slider.api.GetInstanceDefinitionRequestProto) returns (.org.apache.slider.api.GetInstanceDefinitionResponseProto);</code>
-     *
-     * <pre>
-     **
-     * Get the instance definition
-     * </pre>
-     */
-    public abstract void getInstanceDefinition(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.GetInstanceDefinitionRequestProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetInstanceDefinitionResponseProto> done);
-
-    /**
-     * <code>rpc listNodeUUIDsByRole(.org.apache.slider.api.ListNodeUUIDsByRoleRequestProto) returns (.org.apache.slider.api.ListNodeUUIDsByRoleResponseProto);</code>
-     *
-     * <pre>
-     **
-     * List all running nodes in a role
-     * </pre>
-     */
-    public abstract void listNodeUUIDsByRole(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleRequestProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleResponseProto> done);
-
-    /**
-     * <code>rpc getNode(.org.apache.slider.api.GetNodeRequestProto) returns (.org.apache.slider.api.GetNodeResponseProto);</code>
-     *
-     * <pre>
-     **
-     * Get the details on a node
-     * </pre>
-     */
-    public abstract void getNode(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.GetNodeRequestProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetNodeResponseProto> done);
-
-    /**
-     * <code>rpc getClusterNodes(.org.apache.slider.api.GetClusterNodesRequestProto) returns (.org.apache.slider.api.GetClusterNodesResponseProto);</code>
-     *
-     * <pre>
-     **
-     * Get the 
-     * details on a list of nodes.
-     * Unknown nodes are not returned
-     * &lt;i&gt;Important: the order of the results are undefined&lt;/i&gt;
-     * </pre>
-     */
-    public abstract void getClusterNodes(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.GetClusterNodesRequestProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetClusterNodesResponseProto> done);
-
-    /**
-     * <code>rpc echo(.org.apache.slider.api.EchoRequestProto) returns (.org.apache.slider.api.EchoResponseProto);</code>
-     *
-     * <pre>
-     **
-     * echo some text
-     * </pre>
-     */
-    public abstract void echo(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.EchoRequestProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.EchoResponseProto> done);
-
-    /**
-     * <code>rpc killContainer(.org.apache.slider.api.KillContainerRequestProto) returns (.org.apache.slider.api.KillContainerResponseProto);</code>
-     *
-     * <pre>
-     **
-     * kill a container
-     * </pre>
-     */
-    public abstract void killContainer(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.KillContainerRequestProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.KillContainerResponseProto> done);
-
-    /**
-     * <code>rpc amSuicide(.org.apache.slider.api.AMSuicideRequestProto) returns (.org.apache.slider.api.AMSuicideResponseProto);</code>
-     *
-     * <pre>
-     **
-     * kill the AM
-     * </pre>
-     */
-    public abstract void amSuicide(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.AMSuicideRequestProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.AMSuicideResponseProto> done);
-
-    /**
-     * <code>rpc getLivenessInformation(.org.apache.slider.api.GetApplicationLivenessRequestProto) returns (.org.apache.slider.api.ApplicationLivenessInformationProto);</code>
-     */
-    public abstract void getLivenessInformation(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.GetApplicationLivenessRequestProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ApplicationLivenessInformationProto> done);
-
-    /**
-     * <code>rpc getLiveContainers(.org.apache.slider.api.GetLiveContainersRequestProto) returns (.org.apache.slider.api.GetLiveContainersResponseProto);</code>
-     */
-    public abstract void getLiveContainers(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.GetLiveContainersRequestProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveContainersResponseProto> done);
-
-    /**
-     * <code>rpc getLiveContainer(.org.apache.slider.api.GetLiveContainerRequestProto) returns (.org.apache.slider.api.ContainerInformationProto);</code>
-     */
-    public abstract void getLiveContainer(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.GetLiveContainerRequestProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ContainerInformationProto> done);
-
-    /**
-     * <code>rpc getLiveComponents(.org.apache.slider.api.GetLiveComponentsRequestProto) returns (.org.apache.slider.api.GetLiveComponentsResponseProto);</code>
-     */
-    public abstract void getLiveComponents(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.GetLiveComponentsRequestProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveComponentsResponseProto> done);
-
-    /**
-     * <code>rpc getLiveComponent(.org.apache.slider.api.GetLiveComponentRequestProto) returns (.org.apache.slider.api.ComponentInformationProto);</code>
-     */
-    public abstract void getLiveComponent(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.GetLiveComponentRequestProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ComponentInformationProto> done);
-
-    /**
-     * <code>rpc getLiveNodes(.org.apache.slider.api.GetLiveNodesRequestProto) returns (.org.apache.slider.api.GetLiveNodesResponseProto);</code>
-     */
-    public abstract void getLiveNodes(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.GetLiveNodesRequestProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveNodesResponseProto> done);
-
-    /**
-     * <code>rpc getLiveNode(.org.apache.slider.api.GetLiveNodeRequestProto) returns (.org.apache.slider.api.NodeInformationProto);</code>
-     */
-    public abstract void getLiveNode(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.GetLiveNodeRequestProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.NodeInformationProto> done);
-
-    /**
-     * <code>rpc getModelDesired(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
-     *
-     * <pre>
-     * AggregateConf getModelDesired()
-     * </pre>
-     */
-    public abstract void getModelDesired(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
-
-    /**
-     * <code>rpc getModelDesiredAppconf(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
-     *
-     * <pre>
-     * ConfTree getModelDesiredAppconf
-     * </pre>
-     */
-    public abstract void getModelDesiredAppconf(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
-
-    /**
-     * <code>rpc getModelDesiredResources(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
-     *
-     * <pre>
-     * ConfTree getModelDesiredResources
-     * </pre>
-     */
-    public abstract void getModelDesiredResources(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
-
-    /**
-     * <code>rpc getModelResolved(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
-     *
-     * <pre>
-     * AggregateConf getModelResolved()
-     * </pre>
-     */
-    public abstract void getModelResolved(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
-
-    /**
-     * <code>rpc getModelResolvedAppconf(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
-     *
-     * <pre>
-     * ConfTree getModelResolvedAppconf
-     * </pre>
-     */
-    public abstract void getModelResolvedAppconf(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
-
-    /**
-     * <code>rpc getModelResolvedResources(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
-     *
-     * <pre>
-     * ConfTree getModelResolvedResources
-     * </pre>
-     */
-    public abstract void getModelResolvedResources(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
-
-    /**
-     * <code>rpc getLiveResources(.org.apache.slider.api.EmptyPayloadProto) returns (.org.apache.slider.api.WrappedJsonProto);</code>
-     *
-     * <pre>
-     * ConfTree getLiveResources
-     * </pre>
-     */
-    public abstract void getLiveResources(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done);
-
-    /**
-     * <code>rpc getClientCertificateStore(.org.apache.slider.api.GetCertificateStoreRequestProto) returns (.org.apache.slider.api.GetCertificateStoreResponseProto);</code>
-     */
-    public abstract void getClientCertificateStore(
-        com.google.protobuf.RpcController controller,
-        org.apache.slider.api.proto.Messages.GetCertificateStoreRequestProto request,
-        com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetCertificateStoreResponseProto> done);
-
-    public static final
-        com.google.protobuf.Descriptors.ServiceDescriptor
-        getDescriptor() {
-      return org.apache.slider.api.proto.SliderClusterAPI.getDescriptor().getServices().get(0);
-    }
-    public final com.google.protobuf.Descriptors.ServiceDescriptor
-        getDescriptorForType() {
-      return getDescriptor();
-    }
-
-    public final void callMethod(
-        com.google.protobuf.Descriptors.MethodDescriptor method,
-        com.google.protobuf.RpcController controller,
-        com.google.protobuf.Message request,
-        com.google.protobuf.RpcCallback<
-          com.google.protobuf.Message> done) {
-      if (method.getService() != getDescriptor()) {
-        throw new java.lang.IllegalArgumentException(
-          "Service.callMethod() given method descriptor for wrong " +
-          "service type.");
-      }
-      switch(method.getIndex()) {
-        case 0:
-          this.stopCluster(controller, (org.apache.slider.api.proto.Messages.StopClusterRequestProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.StopClusterResponseProto>specializeCallback(
-              done));
-          return;
-        case 1:
-          this.upgradeContainers(controller, (org.apache.slider.api.proto.Messages.UpgradeContainersRequestProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.UpgradeContainersResponseProto>specializeCallback(
-              done));
-          return;
-        case 2:
-          this.flexCluster(controller, (org.apache.slider.api.proto.Messages.FlexClusterRequestProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.FlexClusterResponseProto>specializeCallback(
-              done));
-          return;
-        case 3:
-          this.getJSONClusterStatus(controller, (org.apache.slider.api.proto.Messages.GetJSONClusterStatusRequestProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.GetJSONClusterStatusResponseProto>specializeCallback(
-              done));
-          return;
-        case 4:
-          this.getInstanceDefinition(controller, (org.apache.slider.api.proto.Messages.GetInstanceDefinitionRequestProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.GetInstanceDefinitionResponseProto>specializeCallback(
-              done));
-          return;
-        case 5:
-          this.listNodeUUIDsByRole(controller, (org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleRequestProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleResponseProto>specializeCallback(
-              done));
-          return;
-        case 6:
-          this.getNode(controller, (org.apache.slider.api.proto.Messages.GetNodeRequestProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.GetNodeResponseProto>specializeCallback(
-              done));
-          return;
-        case 7:
-          this.getClusterNodes(controller, (org.apache.slider.api.proto.Messages.GetClusterNodesRequestProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.GetClusterNodesResponseProto>specializeCallback(
-              done));
-          return;
-        case 8:
-          this.echo(controller, (org.apache.slider.api.proto.Messages.EchoRequestProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.EchoResponseProto>specializeCallback(
-              done));
-          return;
-        case 9:
-          this.killContainer(controller, (org.apache.slider.api.proto.Messages.KillContainerRequestProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.KillContainerResponseProto>specializeCallback(
-              done));
-          return;
-        case 10:
-          this.amSuicide(controller, (org.apache.slider.api.proto.Messages.AMSuicideRequestProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.AMSuicideResponseProto>specializeCallback(
-              done));
-          return;
-        case 11:
-          this.getLivenessInformation(controller, (org.apache.slider.api.proto.Messages.GetApplicationLivenessRequestProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.ApplicationLivenessInformationProto>specializeCallback(
-              done));
-          return;
-        case 12:
-          this.getLiveContainers(controller, (org.apache.slider.api.proto.Messages.GetLiveContainersRequestProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.GetLiveContainersResponseProto>specializeCallback(
-              done));
-          return;
-        case 13:
-          this.getLiveContainer(controller, (org.apache.slider.api.proto.Messages.GetLiveContainerRequestProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.ContainerInformationProto>specializeCallback(
-              done));
-          return;
-        case 14:
-          this.getLiveComponents(controller, (org.apache.slider.api.proto.Messages.GetLiveComponentsRequestProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.GetLiveComponentsResponseProto>specializeCallback(
-              done));
-          return;
-        case 15:
-          this.getLiveComponent(controller, (org.apache.slider.api.proto.Messages.GetLiveComponentRequestProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.ComponentInformationProto>specializeCallback(
-              done));
-          return;
-        case 16:
-          this.getLiveNodes(controller, (org.apache.slider.api.proto.Messages.GetLiveNodesRequestProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.GetLiveNodesResponseProto>specializeCallback(
-              done));
-          return;
-        case 17:
-          this.getLiveNode(controller, (org.apache.slider.api.proto.Messages.GetLiveNodeRequestProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.NodeInformationProto>specializeCallback(
-              done));
-          return;
-        case 18:
-          this.getModelDesired(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.WrappedJsonProto>specializeCallback(
-              done));
-          return;
-        case 19:
-          this.getModelDesiredAppconf(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.WrappedJsonProto>specializeCallback(
-              done));
-          return;
-        case 20:
-          this.getModelDesiredResources(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.WrappedJsonProto>specializeCallback(
-              done));
-          return;
-        case 21:
-          this.getModelResolved(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.WrappedJsonProto>specializeCallback(
-              done));
-          return;
-        case 22:
-          this.getModelResolvedAppconf(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.WrappedJsonProto>specializeCallback(
-              done));
-          return;
-        case 23:
-          this.getModelResolvedResources(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.WrappedJsonProto>specializeCallback(
-              done));
-          return;
-        case 24:
-          this.getLiveResources(controller, (org.apache.slider.api.proto.Messages.EmptyPayloadProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.WrappedJsonProto>specializeCallback(
-              done));
-          return;
-        case 25:
-          this.getClientCertificateStore(controller, (org.apache.slider.api.proto.Messages.GetCertificateStoreRequestProto)request,
-            com.google.protobuf.RpcUtil.<org.apache.slider.api.proto.Messages.GetCertificateStoreResponseProto>specializeCallback(
-              done));
-          return;
-        default:
-          throw new java.lang.AssertionError("Can't get here.");
-      }
-    }
-
-    public final com.google.protobuf.Message
-        getRequestPrototype(
-        com.google.protobuf.Descriptors.MethodDescriptor method) {
-      if (method.getService() != getDescriptor()) {
-        throw new java.lang.IllegalArgumentException(
-          "Service.getRequestPrototype() given method " +
-          "descriptor for wrong service type.");
-      }
-      switch(method.getIndex()) {
-        case 0:
-          return org.apache.slider.api.proto.Messages.StopClusterRequestProto.getDefaultInstance();
-        case 1:
-          return org.apache.slider.api.proto.Messages.UpgradeContainersRequestProto.getDefaultInstance();
-        case 2:
-          return org.apache.slider.api.proto.Messages.FlexClusterRequestProto.getDefaultInstance();
-        case 3:
-          return org.apache.slider.api.proto.Messages.GetJSONClusterStatusRequestProto.getDefaultInstance();
-        case 4:
-          return org.apache.slider.api.proto.Messages.GetInstanceDefinitionRequestProto.getDefaultInstance();
-        case 5:
-          return org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleRequestProto.getDefaultInstance();
-        case 6:
-          return org.apache.slider.api.proto.Messages.GetNodeRequestProto.getDefaultInstance();
-        case 7:
-          return org.apache.slider.api.proto.Messages.GetClusterNodesRequestProto.getDefaultInstance();
-        case 8:
-          return org.apache.slider.api.proto.Messages.EchoRequestProto.getDefaultInstance();
-        case 9:
-          return org.apache.slider.api.proto.Messages.KillContainerRequestProto.getDefaultInstance();
-        case 10:
-          return org.apache.slider.api.proto.Messages.AMSuicideRequestProto.getDefaultInstance();
-        case 11:
-          return org.apache.slider.api.proto.Messages.GetApplicationLivenessRequestProto.getDefaultInstance();
-        case 12:
-          return org.apache.slider.api.proto.Messages.GetLiveContainersRequestProto.getDefaultInstance();
-        case 13:
-          return org.apache.slider.api.proto.Messages.GetLiveContainerRequestProto.getDefaultInstance();
-        case 14:
-          return org.apache.slider.api.proto.Messages.GetLiveComponentsRequestProto.getDefaultInstance();
-        case 15:
-          return org.apache.slider.api.proto.Messages.GetLiveComponentRequestProto.getDefaultInstance();
-        case 16:
-          return org.apache.slider.api.proto.Messages.GetLiveNodesRequestProto.getDefaultInstance();
-        case 17:
-          return org.apache.slider.api.proto.Messages.GetLiveNodeRequestProto.getDefaultInstance();
-        case 18:
-          return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
-        case 19:
-          return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
-        case 20:
-          return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
-        case 21:
-          return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
-        case 22:
-          return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
-        case 23:
-          return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
-        case 24:
-          return org.apache.slider.api.proto.Messages.EmptyPayloadProto.getDefaultInstance();
-        case 25:
-          return org.apache.slider.api.proto.Messages.GetCertificateStoreRequestProto.getDefaultInstance();
-        default:
-          throw new java.lang.AssertionError("Can't get here.");
-      }
-    }
-
-    public final com.google.protobuf.Message
-        getResponsePrototype(
-        com.google.protobuf.Descriptors.MethodDescriptor method) {
-      if (method.getService() != getDescriptor()) {
-        throw new java.lang.IllegalArgumentException(
-          "Service.getResponsePrototype() given method " +
-          "descriptor for wrong service type.");
-      }
-      switch(method.getIndex()) {
-        case 0:
-          return org.apache.slider.api.proto.Messages.StopClusterResponseProto.getDefaultInstance();
-        case 1:
-          return org.apache.slider.api.proto.Messages.UpgradeContainersResponseProto.getDefaultInstance();
-        case 2:
-          return org.apache.slider.api.proto.Messages.FlexClusterResponseProto.getDefaultInstance();
-        case 3:
-          return org.apache.slider.api.proto.Messages.GetJSONClusterStatusResponseProto.getDefaultInstance();
-        case 4:
-          return org.apache.slider.api.proto.Messages.GetInstanceDefinitionResponseProto.getDefaultInstance();
-        case 5:
-          return org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleResponseProto.getDefaultInstance();
-        case 6:
-          return org.apache.slider.api.proto.Messages.GetNodeResponseProto.getDefaultInstance();
-        case 7:
-          return org.apache.slider.api.proto.Messages.GetClusterNodesResponseProto.getDefaultInstance();
-        case 8:
-          return org.apache.slider.api.proto.Messages.EchoResponseProto.getDefaultInstance();
-        case 9:
-          return org.apache.slider.api.proto.Messages.KillContainerResponseProto.getDefaultInstance();
-        case 10:
-          return org.apache.slider.api.proto.Messages.AMSuicideResponseProto.getDefaultInstance();
-        case 11:
-          return org.apache.slider.api.proto.Messages.ApplicationLivenessInformationProto.getDefaultInstance();
-        case 12:
-          return org.apache.slider.api.proto.Messages.GetLiveContainersResponseProto.getDefaultInstance();
-        case 13:
-          return org.apache.slider.api.proto.Messages.ContainerInformationProto.getDefaultInstance();
-        case 14:
-          return org.apache.slider.api.proto.Messages.GetLiveComponentsResponseProto.getDefaultInstance();
-        case 15:
-          return org.apache.slider.api.proto.Messages.ComponentInformationProto.getDefaultInstance();
-        case 16:
-          return org.apache.slider.api.proto.Messages.GetLiveNodesResponseProto.getDefaultInstance();
-        case 17:
-          return org.apache.slider.api.proto.Messages.NodeInformationProto.getDefaultInstance();
-        case 18:
-          return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
-        case 19:
-          return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
-        case 20:
-          return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
-        case 21:
-          return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
-        case 22:
-          return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
-        case 23:
-          return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
-        case 24:
-          return org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance();
-        case 25:
-          return org.apache.slider.api.proto.Messages.GetCertificateStoreResponseProto.getDefaultInstance();
-        default:
-          throw new java.lang.AssertionError("Can't get here.");
-      }
-    }
-
-    public static Stub newStub(
-        com.google.protobuf.RpcChannel channel) {
-      return new Stub(channel);
-    }
-
-    public static final class Stub extends org.apache.slider.api.proto.SliderClusterAPI.SliderClusterProtocolPB implements Interface {
-      private Stub(com.google.protobuf.RpcChannel channel) {
-        this.channel = channel;
-      }
-
-      private final com.google.protobuf.RpcChannel channel;
-
-      public com.google.protobuf.RpcChannel getChannel() {
-        return channel;
-      }
-
-      public  void stopCluster(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.StopClusterRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.StopClusterResponseProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(0),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.StopClusterResponseProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.StopClusterResponseProto.class,
-            org.apache.slider.api.proto.Messages.StopClusterResponseProto.getDefaultInstance()));
-      }
-
-      public  void upgradeContainers(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.UpgradeContainersRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.UpgradeContainersResponseProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(1),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.UpgradeContainersResponseProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.UpgradeContainersResponseProto.class,
-            org.apache.slider.api.proto.Messages.UpgradeContainersResponseProto.getDefaultInstance()));
-      }
-
-      public  void flexCluster(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.FlexClusterRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.FlexClusterResponseProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(2),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.FlexClusterResponseProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.FlexClusterResponseProto.class,
-            org.apache.slider.api.proto.Messages.FlexClusterResponseProto.getDefaultInstance()));
-      }
-
-      public  void getJSONClusterStatus(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetJSONClusterStatusRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetJSONClusterStatusResponseProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(3),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.GetJSONClusterStatusResponseProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.GetJSONClusterStatusResponseProto.class,
-            org.apache.slider.api.proto.Messages.GetJSONClusterStatusResponseProto.getDefaultInstance()));
-      }
-
-      public  void getInstanceDefinition(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetInstanceDefinitionRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetInstanceDefinitionResponseProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(4),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.GetInstanceDefinitionResponseProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.GetInstanceDefinitionResponseProto.class,
-            org.apache.slider.api.proto.Messages.GetInstanceDefinitionResponseProto.getDefaultInstance()));
-      }
-
-      public  void listNodeUUIDsByRole(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleResponseProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(5),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleResponseProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleResponseProto.class,
-            org.apache.slider.api.proto.Messages.ListNodeUUIDsByRoleResponseProto.getDefaultInstance()));
-      }
-
-      public  void getNode(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetNodeRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetNodeResponseProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(6),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.GetNodeResponseProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.GetNodeResponseProto.class,
-            org.apache.slider.api.proto.Messages.GetNodeResponseProto.getDefaultInstance()));
-      }
-
-      public  void getClusterNodes(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetClusterNodesRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetClusterNodesResponseProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(7),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.GetClusterNodesResponseProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.GetClusterNodesResponseProto.class,
-            org.apache.slider.api.proto.Messages.GetClusterNodesResponseProto.getDefaultInstance()));
-      }
-
-      public  void echo(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.EchoRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.EchoResponseProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(8),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.EchoResponseProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.EchoResponseProto.class,
-            org.apache.slider.api.proto.Messages.EchoResponseProto.getDefaultInstance()));
-      }
-
-      public  void killContainer(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.KillContainerRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.KillContainerResponseProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(9),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.KillContainerResponseProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.KillContainerResponseProto.class,
-            org.apache.slider.api.proto.Messages.KillContainerResponseProto.getDefaultInstance()));
-      }
-
-      public  void amSuicide(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.AMSuicideRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.AMSuicideResponseProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(10),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.AMSuicideResponseProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.AMSuicideResponseProto.class,
-            org.apache.slider.api.proto.Messages.AMSuicideResponseProto.getDefaultInstance()));
-      }
-
-      public  void getLivenessInformation(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetApplicationLivenessRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ApplicationLivenessInformationProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(11),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.ApplicationLivenessInformationProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.ApplicationLivenessInformationProto.class,
-            org.apache.slider.api.proto.Messages.ApplicationLivenessInformationProto.getDefaultInstance()));
-      }
-
-      public  void getLiveContainers(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetLiveContainersRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveContainersResponseProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(12),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.GetLiveContainersResponseProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.GetLiveContainersResponseProto.class,
-            org.apache.slider.api.proto.Messages.GetLiveContainersResponseProto.getDefaultInstance()));
-      }
-
-      public  void getLiveContainer(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetLiveContainerRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ContainerInformationProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(13),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.ContainerInformationProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.ContainerInformationProto.class,
-            org.apache.slider.api.proto.Messages.ContainerInformationProto.getDefaultInstance()));
-      }
-
-      public  void getLiveComponents(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetLiveComponentsRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveComponentsResponseProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(14),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.GetLiveComponentsResponseProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.GetLiveComponentsResponseProto.class,
-            org.apache.slider.api.proto.Messages.GetLiveComponentsResponseProto.getDefaultInstance()));
-      }
-
-      public  void getLiveComponent(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetLiveComponentRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.ComponentInformationProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(15),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.ComponentInformationProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.ComponentInformationProto.class,
-            org.apache.slider.api.proto.Messages.ComponentInformationProto.getDefaultInstance()));
-      }
-
-      public  void getLiveNodes(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetLiveNodesRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetLiveNodesResponseProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(16),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.GetLiveNodesResponseProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.GetLiveNodesResponseProto.class,
-            org.apache.slider.api.proto.Messages.GetLiveNodesResponseProto.getDefaultInstance()));
-      }
-
-      public  void getLiveNode(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetLiveNodeRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.NodeInformationProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(17),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.NodeInformationProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.NodeInformationProto.class,
-            org.apache.slider.api.proto.Messages.NodeInformationProto.getDefaultInstance()));
-      }
-
-      public  void getModelDesired(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(18),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.WrappedJsonProto.class,
-            org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance()));
-      }
-
-      public  void getModelDesiredAppconf(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(19),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.WrappedJsonProto.class,
-            org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance()));
-      }
-
-      public  void getModelDesiredResources(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(20),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.WrappedJsonProto.class,
-            org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance()));
-      }
-
-      public  void getModelResolved(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(21),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.WrappedJsonProto.class,
-            org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance()));
-      }
-
-      public  void getModelResolvedAppconf(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(22),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.WrappedJsonProto.class,
-            org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance()));
-      }
-
-      public  void getModelResolvedResources(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(23),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.WrappedJsonProto.class,
-            org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance()));
-      }
-
-      public  void getLiveResources(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.EmptyPayloadProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.WrappedJsonProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(24),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.WrappedJsonProto.class,
-            org.apache.slider.api.proto.Messages.WrappedJsonProto.getDefaultInstance()));
-      }
-
-      public  void getClientCertificateStore(
-          com.google.protobuf.RpcController controller,
-          org.apache.slider.api.proto.Messages.GetCertificateStoreRequestProto request,
-          com.google.protobuf.RpcCallback<org.apache.slider.api.proto.Messages.GetCertificateStoreResponseProto> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(25),
-          controller,
-          request,
-          org.apache.slider.api.proto.Messages.GetCertificateStoreResponseProto.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.slider.api.proto.Messages.GetCertificateStoreResponseProto.class,
-            org.apache.slider.api.proto.Messages.GetCertificateStoreResponseProto.getDefaultInstance()));
-      }
-    }
-
-    public static BlockingInterface newBlockingStub(
-        com.google.protobuf.BlockingRpcChannel channel) {
-      return new BlockingStub(channel);
-    }
-
-    public interface BlockingInterface {
-      public org.apache.slider.api.proto.Messages.StopClusterResponseProto stopCluster(
-          com.google.protobuf.RpcController controller,
-     

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[25/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderVersionInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderVersionInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderVersionInfo.java
new file mode 100644
index 0000000..86025ee
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderVersionInfo.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.tools;
+
+import org.apache.hadoop.util.VersionInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.util.Locale;
+import java.util.Properties;
+
+/**
+ * Extract the version properties, which will look something like
+ * <pre>
+ * application.name=${pom.name}
+ * application.version=${pom.version}
+ * application.build=${buildNumber}
+ * application.build.java.version=${java.version}
+ * application.build.info=${pom.name}-${pom.version} Built against ${buildNumber} on ${java.version} by ${user.name}
+ * </pre>
+ * 
+ * the <code>mvn process-resources</code> target will expand the properties
+ * and add the resources to target/classes, which will then look something like
+ * <pre>
+ *   application.name=Slider Core
+ *   application.version=0.7.1-SNAPSHOT
+ *   application.build=1dd69
+ *   application.build.java.version=1.7.0_45
+ *   application.build.user=stevel
+ *   application.build.info=Slider Core-0.7.1-SNAPSHOT Built against 1dd69 on 1.7.0_45 by stevel
+ * </pre>
+ * 
+ * Note: the values will change and more properties added.
+ */
+public class SliderVersionInfo {
+  private static final Logger log = LoggerFactory.getLogger(SliderVersionInfo.class);
+
+  /**
+   * Name of the resource containing the filled-in-at-runtime props
+   */
+  public static final String VERSION_RESOURCE =
+      "org/apache/slider/providers/dynamic/application.properties";
+
+  public static final String APP_NAME = "application.name";
+  public static final String APP_VERSION = "application.version";
+  public static final String APP_BUILD = "application.build";
+  public static final String APP_BUILD_JAVA_VERSION = "application.build.java.version";
+  public static final String APP_BUILD_USER = "application.build.user";
+  public static final String APP_BUILD_INFO = "application.build.info";
+  public static final String HADOOP_BUILD_INFO = "hadoop.build.info";
+  public static final String HADOOP_DEPLOYED_INFO = "hadoop.deployed.info";
+
+
+  public static Properties loadVersionProperties()  {
+    Properties props = new Properties();
+    URL resURL = SliderVersionInfo.class.getClassLoader()
+                                   .getResource(VERSION_RESOURCE);
+    assert resURL != null : "Null resource " + VERSION_RESOURCE;
+
+    try {
+      InputStream inStream = resURL.openStream();
+      assert inStream != null : "Null input stream from " + VERSION_RESOURCE;
+      props.load(inStream);
+    } catch (IOException e) {
+      log.warn("IOE loading " + VERSION_RESOURCE, e);
+    }
+    return props;
+  }
+
+  /**
+   * Load the version info and print it
+   * @param logger logger
+   */
+  public static void loadAndPrintVersionInfo(Logger logger) {
+    Properties props = loadVersionProperties();
+    logger.info(props.getProperty(APP_BUILD_INFO));
+    logger.info("Compiled against Hadoop {}",
+                props.getProperty(HADOOP_BUILD_INFO));
+    logger.info(getHadoopVersionString());
+  }
+  
+  public static String getHadoopVersionString() {
+    return String.format(Locale.ENGLISH,
+        "Hadoop runtime version %s with source checksum %s and build date %s",
+        VersionInfo.getBranch(),
+        VersionInfo.getSrcChecksum(),
+        VersionInfo.getDate());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/AbstractInputPropertiesValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/AbstractInputPropertiesValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/AbstractInputPropertiesValidator.java
new file mode 100644
index 0000000..336b4dc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/AbstractInputPropertiesValidator.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.core.conf;
+
+import org.apache.slider.core.exceptions.BadConfigException;
+
+/**
+ *
+ */
+public abstract class AbstractInputPropertiesValidator implements InputPropertiesValidator{
+
+  abstract void validatePropertyNamePrefix(String key) throws BadConfigException;
+
+  public void validate(ConfTreeOperations props)
+      throws BadConfigException {
+    validateGlobalProperties(props);
+    validateComponentProperties(props);
+
+  }
+
+  protected void validateComponentProperties(ConfTreeOperations props)
+      throws BadConfigException {
+    for (String compName : props.getComponentNames()) {
+      MapOperations mo = props.getComponent(compName);
+      if (mo == null) continue;
+      for (String key : mo.keySet()) {
+        validatePropertyNamePrefix(key);
+      }
+    }
+  }
+
+  abstract void validateGlobalProperties(ConfTreeOperations props)
+      throws BadConfigException;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/AggregateConf.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/AggregateConf.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/AggregateConf.java
new file mode 100644
index 0000000..18c3156
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/AggregateConf.java
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.conf;
+
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.codehaus.jackson.annotate.JsonIgnore;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.io.IOException;
+
+/**
+ * Aggregate Configuration.
+ *
+ * It is serializable to JSON
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public final class AggregateConf {
+
+  private String name;
+  private ConfTree resources;
+  private ConfTree internal;
+  private ConfTree appConf;
+
+  private ConfTreeOperations resourceOperations;
+  private ConfTreeOperations appConfOperations;
+  private ConfTreeOperations internalOperations;
+
+  private String passphrase;
+
+  public AggregateConf() {
+    this(new ConfTree(), new ConfTree(), new ConfTree());
+  }
+
+  public AggregateConf(String name) {
+    this(new ConfTree(), new ConfTree(), new ConfTree());
+    this.name = name;
+  }
+
+  public AggregateConf(ConfTree resources,
+                       ConfTree appConf,
+                       ConfTree internal) {
+    setResources(resources);
+    setAppConf(appConf);
+    setInternal(internal);
+  }
+
+  /**
+   * Take a snapshot of the configuration
+   * @param instanceDefinition source
+   * @throws IOException marshalling/copying problems
+   */
+  public AggregateConf(AggregateConf instanceDefinition) throws IOException {
+    ConfTreeOperations resourcesSnapshot =
+        ConfTreeOperations.fromInstance(instanceDefinition.getResources());
+    ConfTreeOperations appConfSnapshot =
+        ConfTreeOperations.fromInstance(instanceDefinition.getAppConf());
+    ConfTreeOperations internalsSnapshot =
+        ConfTreeOperations.fromInstance(instanceDefinition.getInternal());
+    //build a new aggregate from the snapshots
+    setResources(resourcesSnapshot.confTree);
+    setAppConf(appConfSnapshot.confTree);
+    setInternal(internalsSnapshot.confTree);
+  }
+  
+  public void setResources(ConfTree resources) {
+    this.resources = resources;
+    resourceOperations = new ConfTreeOperations(resources);
+  }
+
+  public void setAppConf(ConfTree appConf) {
+    this.appConf = appConf;
+    appConfOperations = new ConfTreeOperations(appConf);
+  }
+
+  public ConfTree getInternal() {
+    return internal;
+  }
+
+  public void setInternal(ConfTree internal) {
+    this.internal = internal;
+    internalOperations = new ConfTreeOperations(internal);
+  }
+
+  public ConfTree getResources() {
+    return resources;
+  }
+
+  public ConfTree getAppConf() {
+    return appConf;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  @JsonIgnore
+  public ConfTreeOperations getResourceOperations() {
+    return resourceOperations;
+  }
+
+
+  @JsonIgnore
+  public ConfTreeOperations getAppConfOperations() {
+    return appConfOperations;
+  }
+
+  @JsonIgnore
+  public ConfTreeOperations getInternalOperations() {
+    return internalOperations;
+  }
+
+  /**
+   * predicate to query if all sections have data structures
+   * @return true if every section is non-null
+   */
+  @JsonIgnore
+  public boolean isComplete() {
+    return resources != null && appConf != null && internal != null;
+  }
+
+  public void validate() throws BadConfigException {
+    if (!isComplete()) {
+      throw new BadConfigException("Incomplete instance %s", this);
+    }
+    resourceOperations.validate();
+    internalOperations.validate();
+    appConfOperations.validate();
+  }
+
+  public void resolve() throws BadConfigException {
+    validate();
+    resourceOperations.resolve();
+    internalOperations.resolve();
+    appConfOperations.resolve();
+  }
+
+  @JsonIgnore
+  public String getPassphrase() {
+    if (passphrase == null) {
+      passphrase = RandomStringUtils.randomAlphanumeric(
+          Integer.valueOf(SliderKeys.PASS_LEN));
+    }
+
+    return passphrase;
+  }
+
+  /**
+   * Is this app package versioned?
+   * 
+   * @return true if {@link SliderKeys#APP_VERSION} was set in the app config
+   *         provided during creation of this app
+   * @since 0.80.0-incubating
+   */
+  public boolean isVersioned() {
+    return StringUtils.isNotEmpty(getAppConfOperations().getGlobalOptions()
+        .get(SliderKeys.APP_VERSION));
+  }
+
+  /**
+   * string operation includes all the inner conftrees
+   * @return a string description
+   */
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+      new StringBuilder("{");
+    sb.append(",\n\"internal\": ").append(internal);
+    sb.append(",\n\"resources\": ").append(resources);
+    sb.append(",\n\"appConf\" :").append(appConf);
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/ConfTree.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/ConfTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/ConfTree.java
new file mode 100644
index 0000000..be7c56f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/ConfTree.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.conf;
+
+import org.apache.slider.core.persist.ConfTreeSerDeser;
+import org.apache.slider.core.persist.PersistKeys;
+import org.codehaus.jackson.JsonGenerationException;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.JsonMappingException;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A conf tree represents one of the configuration trees
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public final class ConfTree {
+
+  /**
+   * Size of an initial map. This is kept low so the cost of having
+   * many conf trees in a process is low.
+   */
+  public static final int INITAL_MAP_CAPACITY = 3;
+
+  protected static final Logger
+    log = LoggerFactory.getLogger(ConfTree.class);
+
+  /**
+   * version counter
+   */
+  public String schema = PersistKeys.SCHEMA;
+
+  /**
+   * Metadata
+   */
+  public Map<String, Object> metadata = new HashMap<>(INITAL_MAP_CAPACITY);
+
+
+  /**
+   * Global options
+   */
+  public Map<String, String> global =
+    new HashMap<>(INITAL_MAP_CAPACITY);
+
+
+  /**
+   * Credentials
+   */
+  public Map<String, List<String>> credentials =
+      new HashMap<>(INITAL_MAP_CAPACITY);
+
+  /**
+   * Role options, 
+   * role -> option -> value
+   */
+  public Map<String, Map<String, String>> components =
+    new HashMap<>(INITAL_MAP_CAPACITY);
+
+
+  /**
+   * Shallow clone
+   * @return a shallow clone
+   * @throws CloneNotSupportedException
+   */
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    return super.clone();
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return toJson();
+    } catch (Exception e) {
+      log.warn("Failed to convert to JSON ", e);
+      return super.toString();
+    }
+  }
+
+  /**
+   * Convert to a JSON string
+   * @return a JSON string description
+   * @throws IOException Problems mapping/writing the object
+   */
+  public String toJson() throws IOException,
+                                JsonGenerationException,
+                                JsonMappingException {
+    return ConfTreeSerDeser.toString(this);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java
new file mode 100644
index 0000000..d24a158
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java
@@ -0,0 +1,477 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.conf;
+
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.core.persist.ConfTreeSerDeser;
+import org.apache.slider.core.persist.PersistKeys;
+import org.codehaus.jackson.JsonGenerationException;
+import org.codehaus.jackson.annotate.JsonIgnore;
+import org.codehaus.jackson.map.JsonMappingException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class ConfTreeOperations {
+
+  public final ConfTree confTree;
+  private final MapOperations globalOptions;
+
+  protected static final Logger
+    log = LoggerFactory.getLogger(ConfTreeOperations.class);
+
+
+  public ConfTreeOperations(ConfTree confTree) {
+    assert confTree != null : "null tree";
+    assert confTree.components != null : "null tree components";
+    this.confTree = confTree;
+    globalOptions = new MapOperations("global", confTree.global);
+  }
+
+  /**
+   * Get the underlying conf tree
+   * @return the tree
+   */
+  public ConfTree getConfTree() {
+    return confTree;
+  }
+
+  /**
+   * Validate the configuration
+   * @throws BadConfigException
+   */
+  public void validate() throws BadConfigException {
+    validate(null);
+  }
+
+  /**
+   * Validate the configuration
+   * @param validator a provided properties validator
+   * @throws BadConfigException
+   */
+  public void validate(InputPropertiesValidator validator) throws BadConfigException {
+    String version = confTree.schema;
+    if (version == null) {
+      throw new BadConfigException("'version' undefined");
+    }
+    if (!PersistKeys.SCHEMA.equals(version)) {
+      throw new BadConfigException(
+          "version %s incompatible with supported version %s",
+          version,
+          PersistKeys.SCHEMA);
+    }
+    if (validator != null) {
+      validator.validate(this);
+    }
+  }
+
+  /**
+   * Resolve a ConfTree by mapping all global options into each component
+   * -if there is none there already
+   */
+  public void resolve() {
+    for (Map.Entry<String, Map<String, String>> comp : confTree.components.entrySet()) {
+      mergeInGlobal(comp.getValue());
+    }
+  }
+
+  /**
+   * Merge any options
+   * @param component dest values
+   */
+  public void mergeInGlobal(Map<String, String> component) {
+    SliderUtils.mergeMapsIgnoreDuplicateKeys(component, confTree.global);
+  }
+
+  /**
+   * Get operations on the global set
+   * @return a wrapped map
+   */
+  public MapOperations getGlobalOptions() {
+    return globalOptions;
+  }
+
+
+  /**
+   * look up a component and return its options
+   * @param component component name
+   * @return component mapping or null
+   */
+  public MapOperations getComponent(String component) {
+    Map<String, String> instance = confTree.components.get(component);
+    if (instance != null) {
+      return new MapOperations(component, instance);
+    }
+    return null;
+  }
+
+  /**
+   * look up a component and return its options with the specified replacements
+   * @param component component name
+   * @param replacementOptions replacement options
+   * @return component mapping or null
+   */
+  public MapOperations getComponent(String component, Map<String,String>
+      replacementOptions) {
+    Map<String, String> instance = confTree.components.get(component);
+    if (instance != null) {
+      Map<String, String> newInstance = new HashMap<>();
+      newInstance.putAll(instance);
+      newInstance.putAll(replacementOptions);
+      return new MapOperations(component, newInstance);
+    }
+    return null;
+  }
+
+  /**
+   * Get at the underlying component map
+   * @return a map of components. This is the raw ConfTree data structure
+   */
+  public Map<String, Map<String, String>> getComponents() {
+    return confTree.components;
+  }
+
+  /**
+   * Get a component -adding it to the components map if
+   * none with that name exists
+   * @param name role
+   * @return role mapping
+   */
+  public MapOperations getOrAddComponent(String name) {
+    MapOperations operations = getComponent(name);
+    if (operations != null) {
+      return operations;
+    }
+    //create a new instances
+    Map<String, String> map = new HashMap<>();
+    confTree.components.put(name, map);
+    return new MapOperations(name, map);
+  }
+
+
+  /*
+   * return the Set of names names
+   */
+  @JsonIgnore
+  public Set<String> getComponentNames() {
+    return new HashSet<String>(confTree.components.keySet());
+  }
+  
+  
+
+  /**
+   * Get a component whose presence is mandatory
+   * @param name component name
+   * @return the mapping
+   * @throws BadConfigException if the name is not there
+   */
+  public MapOperations getMandatoryComponent(String name) throws
+                                                          BadConfigException {
+    MapOperations ops = getComponent(name);
+    if (ops == null) {
+      throw new BadConfigException("Missing component " + name);
+    }
+    return ops;
+  }
+
+  /**
+   * Set a global option, converting it to a string as needed
+   * @param key key
+   * @param value non null value
+   */
+  public void set(String key, Object value) {
+    globalOptions.put(key, value.toString());
+  }
+  /**
+   * get a global option
+   * @param key key
+   * @return value or null
+   * 
+   */
+  public String get(String key) {
+    return globalOptions.get(key);
+  }
+
+  /**
+   * Propagate all global keys matching a prefix
+   * @param src source
+   * @param prefix prefix
+   */
+  public void propagateGlobalKeys(ConfTree src, String prefix) {
+    Map<String, String> global = src.global;
+    for (Map.Entry<String, String> entry : global.entrySet()) {
+      String key = entry.getKey();
+      if (key.startsWith(prefix)) {
+        set(key, entry.getValue());
+      }
+    }
+  }
+
+  /**
+   * Propagate all global keys matching a prefix
+   * @param src source
+   * @param prefix prefix
+   */
+  public void propagateGlobalKeys(ConfTreeOperations src, String prefix) {
+    propagateGlobalKeys(src.confTree, prefix);
+  }
+
+  /**
+   * Merge the map of a single component
+   * @param component component name
+   * @param map map to merge
+   */
+  public void mergeSingleComponentMap(String component, Map<String, String> map) {
+    MapOperations comp = getOrAddComponent(component);
+    comp.putAll(map);
+  }
+  /**
+   * Merge the map of a single component
+   * @param component component name
+   * @param map map to merge
+   */
+  public void mergeSingleComponentMapPrefix(String component,
+                                            Map<String, String> map,
+                                            String prefix,
+                                            boolean overwrite) {
+    MapOperations comp = getOrAddComponent(component);
+    comp.mergeMapPrefixedKeys(map,prefix, overwrite);
+  }
+
+  /**
+   * Merge in components
+   * @param commandOptions component options on the CLI
+   */
+  public void mergeComponents(Map<String, Map<String, String>> commandOptions) {
+    for (Map.Entry<String, Map<String, String>> entry : commandOptions.entrySet()) {
+      mergeSingleComponentMap(entry.getKey(), entry.getValue());
+    }
+  }
+
+  /**
+   * Merge in components
+   * @param commandOptions component options on the CLI
+   */
+  public void mergeComponentsPrefix(Map<String,
+    Map<String, String>> commandOptions,
+                                    String prefix,
+                                    boolean overwrite) {
+    for (Map.Entry<String, Map<String, String>> entry : commandOptions.entrySet()) {
+      mergeSingleComponentMapPrefix(entry.getKey(), entry.getValue(), prefix, overwrite);
+    }
+  }
+
+  /**
+   * Merge in another tree -no overwrites of global or conf data
+   * (note that metadata does a naive putAll merge/overwrite)
+   * @param that the other tree
+   */
+  public void mergeWithoutOverwrite(ConfTree that) {
+
+    getGlobalOptions().mergeWithoutOverwrite(that.global);
+    confTree.metadata.putAll(that.metadata);
+    confTree.credentials.putAll(that.credentials);
+
+    for (Map.Entry<String, Map<String, String>> entry : that.components.entrySet()) {
+      MapOperations comp = getOrAddComponent(entry.getKey());
+      comp.mergeWithoutOverwrite(entry.getValue());
+    }
+  }
+  
+  /**
+   * Merge in another tree with overwrites
+   * @param that the other tree
+   */
+  public void merge(ConfTree that) {
+
+    getGlobalOptions().putAll(that.global);
+    confTree.metadata.putAll(that.metadata);
+    confTree.credentials.putAll(that.credentials);
+
+    for (Map.Entry<String, Map<String, String>> entry : that.components.entrySet()) {
+      MapOperations comp = getOrAddComponent(entry.getKey());
+      comp.putAll(entry.getValue());
+    }
+  }
+
+  
+  /**
+   * Load from a resource. The inner conf tree is the loaded data -unresolved
+   * @param resource resource
+   * @return loaded value
+   * @throws IOException load failure
+   */
+  public static ConfTreeOperations fromResource(String resource) throws
+                                                                 IOException {
+    ConfTreeSerDeser confTreeSerDeser = new ConfTreeSerDeser();
+    ConfTreeOperations ops = new ConfTreeOperations(
+       confTreeSerDeser.fromResource(resource) );
+    return ops;      
+  }
+  
+  /**
+   * Load from a resource. The inner conf tree is the loaded data -unresolved
+   * @param resource resource
+   * @return loaded value
+   * @throws IOException load failure
+   */
+  public static ConfTreeOperations fromFile(File resource) throws
+                                                                 IOException {
+    ConfTreeSerDeser confTreeSerDeser = new ConfTreeSerDeser();
+    ConfTreeOperations ops = new ConfTreeOperations(
+       confTreeSerDeser.fromFile(resource) );
+    return ops;
+  }
+
+  /**
+   * Build from an existing instance -which is cloned via JSON ser/deser
+   * @param instance the source instance
+   * @return loaded value
+   * @throws IOException load failure
+   */
+  public static ConfTreeOperations fromInstance(ConfTree instance) throws
+                                                                 IOException {
+    ConfTreeSerDeser confTreeSerDeser = new ConfTreeSerDeser();
+    ConfTreeOperations ops = new ConfTreeOperations(
+       confTreeSerDeser.fromJson(confTreeSerDeser.toJson(instance)) );
+    return ops;
+  }
+
+  /**
+   * Load from a file and merge it in
+   * @param file file
+   * @throws IOException any IO problem
+   * @throws BadConfigException if the file is invalid
+   */
+  public void mergeFile(File file) throws IOException, BadConfigException {
+    mergeFile(file, null);
+  }
+
+  /**
+   * Load from a file and merge it in
+   * @param file file
+   * @param validator properties validator
+   * @throws IOException any IO problem
+   * @throws BadConfigException if the file is invalid
+   */
+  public void mergeFile(File file, InputPropertiesValidator validator) throws IOException, BadConfigException {
+    ConfTreeSerDeser confTreeSerDeser = new ConfTreeSerDeser();
+    ConfTree tree = confTreeSerDeser.fromFile(file);
+    ConfTreeOperations ops = new ConfTreeOperations(tree);
+    ops.validate(validator);
+    merge(ops.confTree);
+  }
+
+  @Override
+  public String toString() {
+    return confTree.toString();
+  }
+
+  /**
+   * Convert to a JSON string
+   * @return a JSON string description
+   */
+  public String toJson() throws IOException,
+                                JsonGenerationException,
+                                JsonMappingException {
+    return confTree.toJson();
+  }
+
+  /**
+   * Get a component option
+   * @param name component name
+   * @param option option name
+   * @param defVal default value
+   * @return resolved value
+   */
+  public String getComponentOpt(String name, String option, String defVal) {
+    MapOperations roleopts = getComponent(name);
+    if (roleopts == null) {
+      return defVal;
+    }
+    return roleopts.getOption(option, defVal);
+  }
+
+  /**
+   * Get a component opt; use {@link Integer#decode(String)} so as to take hex
+   * oct and bin values too.
+   *
+   * @param name component name
+   * @param option option name
+   * @param defVal default value
+   * @return parsed value
+   * @throws NumberFormatException if the role could not be parsed.
+   */
+  public int getComponentOptInt(String name, String option, int defVal) {
+    String val = getComponentOpt(name, option, Integer.toString(defVal));
+    return Integer.decode(val);
+  }
+
+  /**
+   * Get a component opt as a boolean using {@link Boolean#valueOf(String)}.
+   *
+   * @param name component name
+   * @param option option name
+   * @param defVal default value
+   * @return parsed value
+   * @throws NumberFormatException if the role could not be parsed.
+   */
+  public boolean getComponentOptBool(String name, String option, boolean defVal) {
+    String val = getComponentOpt(name, option, Boolean.toString(defVal));
+    return Boolean.valueOf(val);
+  }
+
+  /**
+   * Set a component option, creating the component if necessary
+   * @param component component name
+   * @param option option name
+   * @param val value
+   */
+  public void setComponentOpt(String component, String option, String val) {
+    Map<String, String> roleopts = getOrAddComponent(component);
+    roleopts.put(option, val);
+  }
+
+  /**
+   * Set an integer role option, creating the role if necessary
+   * @param role role name
+   * @param option option name
+   * @param val integer value
+   */
+  public void setComponentOpt(String role, String option, int val) {
+    setComponentOpt(role, option, Integer.toString(val));
+  }
+  /**
+   * Set a long role option, creating the role if necessary
+   * @param role role name
+   * @param option option name
+   * @param val long value
+   */
+  public void setComponentOpt(String role, String option, long val) {
+    setComponentOpt(role, option, Long.toString(val));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/InputPropertiesValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/InputPropertiesValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/InputPropertiesValidator.java
new file mode 100644
index 0000000..237c240
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/InputPropertiesValidator.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.core.conf;
+
+import org.apache.slider.core.exceptions.BadConfigException;
+
+/**
+ *
+ */
+public interface InputPropertiesValidator {
+  void validate(ConfTreeOperations props) throws BadConfigException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/MapOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/MapOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/MapOperations.java
new file mode 100644
index 0000000..9714a0f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/MapOperations.java
@@ -0,0 +1,344 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.conf;
+
+import com.google.common.base.Preconditions;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Standard map operations.
+ *
+ * This delegates the standard map interface to the map passed in,
+ * so it can be used to add more actions to the map.
+ */
+public class MapOperations implements Map<String, String> {
+  private static final Logger log =
+    LoggerFactory.getLogger(MapOperations.class);
+  public static final String DAYS = ".days";
+  public static final String HOURS = ".hours";
+  public static final String MINUTES = ".minutes";
+  public static final String SECONDS = ".seconds";
+
+  /**
+   * Global options
+   */
+  public final Map<String, String> options;
+
+  public final String name;
+
+  public MapOperations() {
+    options = new HashMap<String, String>();
+    name = "";
+  }
+
+  /**
+   * Create an instance
+   * @param name name
+   * @param options source of options
+   */
+  public MapOperations(String name, Map<String, String> options) {
+    Preconditions.checkArgument(options != null, "null map");
+    this.options = options;
+    this.name = name;
+  }
+
+  /**
+   * Create an instance from an iterative map entry
+   * @param entry entry to work with
+   */
+  public MapOperations(Map.Entry<String, Map<String, String>> entry) {
+    Preconditions.checkArgument(entry != null, "null entry");
+    this.name = entry.getKey();
+    this.options = entry.getValue();
+  }
+
+  /**
+   * Get an option value
+   *
+   * @param key key
+   * @param defVal default value
+   * @return option in map or the default
+   */
+  public String getOption(String key, String defVal) {
+    String val = options.get(key);
+    return val != null ? val : defVal;
+  }
+
+  /**
+   * Get a boolean option
+   *
+   * @param key option key
+   * @param defVal default value
+   * @return option true if the option equals "true", or the default value
+   * if the option was not defined at all.
+   */
+  public Boolean getOptionBool(String key, boolean defVal) {
+    String val = getOption(key, Boolean.toString(defVal));
+    return Boolean.valueOf(val);
+  }
+
+  /**
+   * Get a cluster option or value
+   *
+   * @param key option key
+   * @return the value
+   * @throws BadConfigException if the option is missing
+   */
+
+  public String getMandatoryOption(String key) throws BadConfigException {
+    String val = options.get(key);
+    if (val == null) {
+      if (log.isDebugEnabled()) {
+        log.debug("Missing key {} from config containing {}",
+                  key, this);
+      }
+      String text = "Missing option " + key;
+      if (SliderUtils.isSet(name)) {
+        text += " from set " + name;
+      }
+      throw new BadConfigException(text);
+    }
+    return val;
+  }
+
+  /**
+   * Get an integer option; use {@link Integer#decode(String)} so as to take hex
+   * oct and bin values too.
+   *
+   * @param option option name
+   * @param defVal default value
+   * @return parsed value
+   * @throws NumberFormatException if the role could not be parsed.
+   */
+  public int getOptionInt(String option, int defVal) {
+    String val = getOption(option, Integer.toString(defVal));
+    return Integer.decode(val);
+  }
+
+  /**
+   * Get a long option; use {@link Long#decode(String)} so as to take hex
+   * oct and bin values too.
+   *
+   * @param option option name
+   * @param defVal default value
+   * @return parsed value
+   * @throws NumberFormatException
+   */
+  public long getOptionLong(String option, long defVal) {
+    String val = getOption(option, Long.toString(defVal));
+    return Long.decode(val);
+  }
+
+  /**
+   * Get a mandatory integer option; use {@link Integer#decode(String)} so as to take hex
+   * oct and bin values too.
+   *
+   * @param option option name
+   * @return parsed value
+   * @throws NumberFormatException if the option could not be parsed.
+   * @throws BadConfigException if the option could not be found
+   */
+  public int getMandatoryOptionInt(String option) throws BadConfigException {
+    getMandatoryOption(option);
+    return getOptionInt(option, 0);
+  }
+
+  /**
+   * Verify that an option is set: that is defined AND non-empty
+   * @param key
+   * @throws BadConfigException
+   */
+  public void verifyOptionSet(String key) throws BadConfigException {
+    if (SliderUtils.isUnset(getOption(key, null))) {
+      throw new BadConfigException("Unset option %s", key);
+    }
+  }
+  
+  public void mergeWithoutOverwrite(Map<String, String> that) {
+    SliderUtils.mergeMapsIgnoreDuplicateKeys(options, that);
+  }
+
+  /**
+   * Merge a map by prefixed keys
+   * @param that the map to merge in
+   * @param prefix prefix to match on
+   * @param overwrite flag to enable overwrite
+   */
+  public void mergeMapPrefixedKeys(Map<String, String> that,
+                                    String prefix,
+                                    boolean overwrite) {
+    for (Map.Entry<String, String> entry : that.entrySet()) {
+      String key = entry.getKey();
+      if (key.startsWith(prefix)) {
+        if (overwrite || get(key) == null) {
+          put(key, entry.getValue());
+        }
+      }
+    }
+  }
+
+  /**
+   * Set a property if it is not already set
+   * @param key key
+   * @param value value
+   */
+  public void putIfUnset(String key, String value) {
+    if (get(key) == null) {
+      put(key, value);
+    }
+  }
+  
+  public void set(String key, Object value) {
+    assert value != null;
+    put(key, value.toString());
+  }
+
+  public int size() {
+    return options.size();
+  }
+
+  public boolean isEmpty() {
+    return options.isEmpty();
+  }
+
+  public boolean containsValue(Object value) {
+    return options.containsValue(value);
+  }
+
+  public boolean containsKey(Object key) {
+    return options.containsKey(key);
+  }
+
+  public String get(Object key) {
+    return options.get(key);
+  }
+
+  public String put(String key, String value) {
+    return options.put(key, value);
+  }
+
+  public String remove(Object key) {
+    return options.remove(key);
+  }
+
+  public void putAll(Map<? extends String, ? extends String> m) {
+    options.putAll(m);
+  }
+
+  public void clear() {
+    options.clear();
+  }
+
+  public Set<String> keySet() {
+    return options.keySet();
+  }
+
+  public Collection<String> values() {
+    return options.values();
+  }
+
+  public Set<Map.Entry<String, String>> entrySet() {
+    return options.entrySet();
+  }
+
+  @SuppressWarnings("EqualsWhichDoesntCheckParameterClass")
+  public boolean equals(Object o) {
+    return options.equals(o);
+  }
+
+  @Override
+  public int hashCode() {
+    return options.hashCode();
+  }
+
+  public boolean isSet(String key) {
+    return SliderUtils.isSet(get(key));
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append(name).append("=\n");
+
+    for (Entry<String, String> entry : options.entrySet()) {
+      builder.append("  ")
+             .append(entry.getKey())
+             .append('=')
+             .append(entry.getValue())
+             .append('\n');
+    }
+    return builder.toString();
+  }
+
+  /**
+   * Get the time range of a set of keys
+   * @param basekey base key to which suffix gets applied
+   * @param defDays
+   * @param defHours
+   * @param defMins
+   * @param defSecs
+   * @return the aggregate time range in seconds
+   */
+  public long getTimeRange(String basekey,
+      int defDays,
+      int defHours,
+      int defMins,
+      int defSecs) {
+    Preconditions.checkArgument(basekey != null);
+    int days = getOptionInt(basekey + DAYS, defDays);
+    int hours = getOptionInt(basekey + HOURS, defHours);
+
+    int minutes = getOptionInt(basekey + MINUTES, defMins);
+    int seconds = getOptionInt(basekey + SECONDS, defSecs);
+    // range check
+    Preconditions.checkState(days >= 0 && hours >= 0 && minutes >= 0
+                             && seconds >= 0,
+        "Time range for %s has negative time component %s:%s:%s:%s",
+        basekey, days, hours, minutes, seconds);
+
+    // calculate total time, schedule the reset if expected
+    long totalMinutes = (long) days * 24 * 60 + (long) hours * 24 + minutes;
+    return totalMinutes * 60 + seconds;
+  }
+
+  /**
+   * Get all entries with a specific prefix
+   * @param prefix prefix
+   * @return a prefixed map, possibly empty
+   */
+  public Map<String, String> prefixedWith(String prefix) {
+
+    Map<String, String> prefixed = new HashMap<>(size());
+    for (Entry<String, String> entry: entrySet()) {
+      if (entry.getKey().startsWith(prefix)) {
+        prefixed.put(entry.getKey(), entry.getValue());
+      }
+    }
+    return prefixed;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/ResourcesInputPropertiesValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/ResourcesInputPropertiesValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/ResourcesInputPropertiesValidator.java
new file mode 100644
index 0000000..19f6f8d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/ResourcesInputPropertiesValidator.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.core.conf;
+
+import org.apache.slider.api.ResourceKeys;
+import org.apache.slider.core.exceptions.BadConfigException;
+
+/**
+ *
+ */
+public class ResourcesInputPropertiesValidator
+    extends AbstractInputPropertiesValidator {
+
+  void validatePropertyNamePrefix(String key) throws BadConfigException {
+    if (!key.startsWith("yarn.") && !key.equals(ResourceKeys.UNIQUE_NAMES)) {
+      throw new BadConfigException(
+          "argument %s does not have 'yarn.' prefix", key);
+    }
+  }
+
+  protected void validateGlobalProperties(ConfTreeOperations props)
+      throws BadConfigException {
+    for (String key : props.getGlobalOptions().keySet()) {
+      validatePropertyNamePrefix(key);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/TemplateInputPropertiesValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/TemplateInputPropertiesValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/TemplateInputPropertiesValidator.java
new file mode 100644
index 0000000..aad2757
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/conf/TemplateInputPropertiesValidator.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.core.conf;
+
+import org.apache.slider.core.exceptions.BadConfigException;
+
+/**
+ *
+ */
+public class TemplateInputPropertiesValidator
+    extends AbstractInputPropertiesValidator {
+
+  void validatePropertyNamePrefix(String key) throws BadConfigException {
+    if (key.startsWith("yarn.")) {
+      throw new BadConfigException(
+          "argument %s has 'yarn.' prefix - this is not allowed in templates", key);
+    }
+  }
+
+  @Override
+  void validateGlobalProperties(ConfTreeOperations props) {
+    // do nothing
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadClusterStateException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadClusterStateException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadClusterStateException.java
new file mode 100644
index 0000000..e73ce57
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadClusterStateException.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.exceptions;
+
+
+/**
+ * The system is in a bad state
+ */
+public class BadClusterStateException extends SliderException {
+  public BadClusterStateException(String message,
+                                  Object... args) {
+    super(EXIT_BAD_STATE, message, args);
+  }
+
+  public BadClusterStateException(Throwable throwable,
+                                  String message, Object... args) {
+    super(EXIT_BAD_STATE, throwable, message, args);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadCommandArgumentsException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadCommandArgumentsException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadCommandArgumentsException.java
new file mode 100644
index 0000000..0d5d686
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadCommandArgumentsException.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.core.exceptions;
+
+public class BadCommandArgumentsException extends SliderException {
+  public BadCommandArgumentsException(String s, Object... args) {
+    super(EXIT_COMMAND_ARGUMENT_ERROR, s, args);
+  }
+
+  public BadCommandArgumentsException(Throwable throwable, String message,
+                                      Object... args) {
+    super(EXIT_COMMAND_ARGUMENT_ERROR, throwable, message, args);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadConfigException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadConfigException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadConfigException.java
new file mode 100644
index 0000000..65a8ea8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/BadConfigException.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.core.exceptions;
+
+/**
+ * An exception to raise on a bad configuration
+ */
+public class BadConfigException extends SliderException {
+
+  public BadConfigException(String s) {
+    super(EXIT_BAD_CONFIGURATION, s);
+  }
+
+  public BadConfigException(String message, Object... args) {
+    super(EXIT_BAD_CONFIGURATION, message, args);
+  }
+
+  public BadConfigException(
+                            Throwable throwable,
+                            String message, Object... args) {
+    super(EXIT_BAD_CONFIGURATION, throwable, message, args);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ErrorStrings.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ErrorStrings.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ErrorStrings.java
new file mode 100644
index 0000000..8b04969
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ErrorStrings.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.exceptions;
+
+public interface ErrorStrings {
+  String E_UNSTABLE_CLUSTER = "Unstable Application Instance :";
+  String E_CLUSTER_RUNNING = "Application Instance running";
+  String E_ALREADY_EXISTS = "already exists";
+  String PRINTF_E_INSTANCE_ALREADY_EXISTS = "Application Instance \"%s\" already exists and is defined in %s";
+  String PRINTF_E_INSTANCE_DIR_ALREADY_EXISTS = "Application Instance dir already exists: %s";
+  String E_MISSING_PATH = "Missing path ";
+  String E_INCOMPLETE_CLUSTER_SPEC =
+    "Cluster specification is marked as incomplete: ";
+  String E_UNKNOWN_INSTANCE = "Unknown application instance ";
+  String E_DESTROY_CREATE_RACE_CONDITION =
+      "created while it was being destroyed";
+  String E_UNKNOWN_ROLE = "Unknown role ";
+  /**
+   * ERROR Strings
+   */
+  String ERROR_NO_ACTION = "No action specified";
+  String ERROR_UNKNOWN_ACTION = "Unknown command: ";
+  String ERROR_NOT_ENOUGH_ARGUMENTS =
+    "Not enough arguments for action: ";
+  String ERROR_PARSE_FAILURE =
+      "Failed to parse ";
+  /**
+   * All the remaining values after argument processing
+   */
+  String ERROR_TOO_MANY_ARGUMENTS =
+    "Too many arguments";
+  String ERROR_DUPLICATE_ENTRY = "Duplicate entry for ";
+  String E_APPLICATION_NOT_RUNNING = "Application not running";
+  String E_FINISHED_APPLICATION = E_APPLICATION_NOT_RUNNING + ": %s state=%s ";
+  String E_NO_IMAGE_OR_HOME_DIR_SPECIFIED =
+    "Neither an image path nor binary home directory were specified";
+  String E_BOTH_IMAGE_AND_HOME_DIR_SPECIFIED =
+    "Both application image path and home dir have been provided";
+  String E_CONFIGURATION_DIRECTORY_NOT_FOUND =
+    "Configuration directory \"%s\" not found";
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ExceptionConverter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ExceptionConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ExceptionConverter.java
new file mode 100644
index 0000000..efec676
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ExceptionConverter.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.exceptions;
+
+import com.sun.jersey.api.client.ClientHandlerException;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.UniformInterfaceException;
+import org.apache.hadoop.fs.InvalidRequestException;
+import org.apache.hadoop.fs.PathAccessDeniedException;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.yarn.webapp.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.http.HttpServletResponse;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+/**
+ * static methods to convert exceptions into different types, including
+ * extraction of details and finer-grained conversions.
+ */
+public class ExceptionConverter {
+  private static final Logger
+      log = LoggerFactory.getLogger(ExceptionConverter.class);
+
+  /**
+   * Uprate error codes 400 and up into faults; 
+   * 404 is converted to a {@link FileNotFoundException},
+   * 401 to {@link ForbiddenException}
+   * FileNotFoundException for an unknown resource
+   * PathAccessDeniedException for access denied
+   * PathIOException for anything else
+   * @param verb HTTP Verb used
+   * @param targetURL URL being targeted 
+   * @param exception original exception
+   * @return a new exception, the original one nested as a cause
+   */
+  public static IOException convertJerseyException(String verb,
+      String targetURL,
+      UniformInterfaceException exception) {
+
+    IOException ioe = null;
+    ClientResponse response = exception.getResponse();
+    if (response != null) {
+      int status = response.getStatus();
+      String body = "";
+      try {
+        if (response.hasEntity()) {
+          body = response.getEntity(String.class);
+          log.error("{} {} returned status {} and body\n{}",
+              verb, targetURL, status, body);
+        } else {
+          log.error("{} {} returned status {} and empty body",
+              verb, targetURL, status);
+        }
+      } catch (Exception e) {
+        log.warn("Failed to extract body from client response", e);
+      }
+      
+      if (status == HttpServletResponse.SC_UNAUTHORIZED
+          || status == HttpServletResponse.SC_FORBIDDEN) {
+        ioe = new PathAccessDeniedException(targetURL);
+      } else if (status == HttpServletResponse.SC_BAD_REQUEST
+          || status == HttpServletResponse.SC_NOT_ACCEPTABLE
+          || status == HttpServletResponse.SC_UNSUPPORTED_MEDIA_TYPE) {
+        // bad request
+        ioe = new InvalidRequestException(
+            String.format("Bad %s request: status code %d against %s",
+                verb, status, targetURL));
+      } else if (status > 400 && status < 500) {
+        ioe =  new FileNotFoundException(targetURL);
+      }
+      if (ioe == null) {
+        ioe = new PathIOException(targetURL,
+            verb + " " + targetURL
+            + " failed with status code : " + status
+            + ":" + exception);
+      }
+    } else {
+      ioe = new PathIOException(targetURL, 
+          verb + " " + targetURL + " failed: " + exception);
+    }
+    ioe.initCause(exception);
+    return ioe; 
+  }
+
+  /**
+   * Handle a client-side Jersey exception.
+   * <p>
+   * If there's an inner IOException, return that.
+   * <p>
+   * Otherwise: create a new wrapper IOE including verb and target details
+   * @param verb HTTP Verb used
+   * @param targetURL URL being targeted 
+   * @param exception original exception
+   * @return an exception to throw
+   */
+  public static IOException convertJerseyException(String verb,
+      String targetURL,
+      ClientHandlerException exception) {
+    if (exception.getCause() instanceof IOException) {
+      return (IOException)exception.getCause();
+    } else {
+      IOException ioe = new IOException(
+          verb + " " + targetURL + " failed: " + exception);
+      ioe.initCause(exception);
+      return ioe;
+    } 
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/NoSuchNodeException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/NoSuchNodeException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/NoSuchNodeException.java
new file mode 100644
index 0000000..ad2f1a4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/NoSuchNodeException.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.exceptions;
+
+import java.io.IOException;
+
+/**
+ * Exception raised when a node cannot be found in the structure
+ * that is being examined.
+ */
+public class NoSuchNodeException extends IOException {
+
+  public NoSuchNodeException(String uuid) {
+    super(uuid);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/NotFoundException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/NotFoundException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/NotFoundException.java
new file mode 100644
index 0000000..40cb94d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/NotFoundException.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.exceptions;
+
+
+/**
+ * Whatever was being resolved: it was not found
+ */
+public class NotFoundException extends SliderException {
+  public NotFoundException(String message,
+      Object... args) {
+    super(EXIT_NOT_FOUND, message, args);
+  }
+
+  public NotFoundException(Throwable throwable,
+      String message, Object... args) {
+    super(EXIT_NOT_FOUND, throwable, message, args);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ServiceNotReadyException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ServiceNotReadyException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ServiceNotReadyException.java
new file mode 100644
index 0000000..435bc1a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/ServiceNotReadyException.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.exceptions;
+
+import java.io.IOException;
+
+/**
+ * This is an exception raised when the service does not consider itself
+ * live (yet)
+ */
+public class ServiceNotReadyException extends IOException {
+
+  public static final String E_NOT_READY =
+      "Service not ready for access: please retry";
+
+  public ServiceNotReadyException(String message) {
+    super(message);
+  }
+
+  public ServiceNotReadyException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  public ServiceNotReadyException(Throwable cause) {
+    super(cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/SliderException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/SliderException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/SliderException.java
new file mode 100644
index 0000000..7f3134a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/SliderException.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.core.exceptions;
+
+import org.apache.slider.common.SliderExitCodes;
+import org.apache.slider.core.main.ServiceLaunchException;
+
+public class SliderException extends ServiceLaunchException implements
+    SliderExitCodes {
+  public SliderException() {
+    super(EXIT_EXCEPTION_THROWN, "SliderException");
+  }
+
+  public SliderException(int code, String message) {
+    super(code, message);
+  }
+
+  public SliderException(String s) {
+    super(EXIT_EXCEPTION_THROWN, s);
+  }
+
+  public SliderException(String s, Throwable throwable) {
+    super(EXIT_EXCEPTION_THROWN, s, throwable);
+  }
+
+  /**
+   * Format the exception as you create it
+   * @param code exit code
+   * @param message exception message -sprintf formatted
+   * @param args arguments for the formatting
+   */
+  public SliderException(int code, String message, Object... args) {
+    super(code, String.format(message, args));
+  }
+
+  /**
+   * Format the exception, include a throwable. 
+   * The throwable comes before the message so that it is out of the varargs
+   * @param code exit code
+   * @param throwable thrown
+   * @param message message
+   * @param args arguments
+   */
+  public SliderException(int code,
+      Throwable throwable,
+      String message,
+      Object... args) {
+    super(code, String.format(message, args), throwable);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/SliderInternalStateException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/SliderInternalStateException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/SliderInternalStateException.java
new file mode 100644
index 0000000..deddbbc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/SliderInternalStateException.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.core.exceptions;
+
+public class SliderInternalStateException extends SliderException {
+  public SliderInternalStateException(String s) {
+    super(EXIT_INTERNAL_ERROR, s);
+  }
+
+  public SliderInternalStateException(String s, Throwable throwable) {
+    super(EXIT_INTERNAL_ERROR, throwable, s);
+  }
+
+  public SliderInternalStateException(String message,
+      Object... args) {
+    super(EXIT_INTERNAL_ERROR, message, args);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/TriggerClusterTeardownException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/TriggerClusterTeardownException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/TriggerClusterTeardownException.java
new file mode 100644
index 0000000..bb9f430
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/TriggerClusterTeardownException.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.exceptions;
+
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+
+/**
+ * An Exception to be thrown for an explicit "shut down the cluster" operation
+ * raised by the application state or other parts of the AM
+ */
+public class TriggerClusterTeardownException extends SliderException {
+
+  private final FinalApplicationStatus finalApplicationStatus;
+  
+  public TriggerClusterTeardownException(int code,
+      FinalApplicationStatus finalApplicationStatus, String message,
+      Object... args) {
+    super(code, message, args);
+    this.finalApplicationStatus = finalApplicationStatus;
+  }
+
+  public FinalApplicationStatus getFinalApplicationStatus() {
+    return finalApplicationStatus;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/UnknownApplicationInstanceException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/UnknownApplicationInstanceException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/UnknownApplicationInstanceException.java
new file mode 100644
index 0000000..a1f8ae9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/UnknownApplicationInstanceException.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.core.exceptions;
+
+public class UnknownApplicationInstanceException extends SliderException {
+  public UnknownApplicationInstanceException(String s) {
+    super(EXIT_UNKNOWN_INSTANCE, s);
+  }
+
+  public UnknownApplicationInstanceException(String s, Throwable throwable) {
+    super(EXIT_UNKNOWN_INSTANCE, throwable, s);
+  }
+
+  public UnknownApplicationInstanceException(String message,
+      Object... args) {
+    super(EXIT_UNKNOWN_INSTANCE, message, args);
+  }
+
+  /**
+   * Create an instance with the standard exception name
+   * @param name name
+   * @return an instance to throw
+   */
+  public static UnknownApplicationInstanceException unknownInstance(String name) {
+    return new UnknownApplicationInstanceException(ErrorStrings.E_UNKNOWN_INSTANCE
+                                   + ": " + name);
+  }
+  public static UnknownApplicationInstanceException unknownInstance(String name,
+      Throwable throwable) {
+    UnknownApplicationInstanceException exception =
+      unknownInstance(name);
+    exception.initCause(throwable);
+    return exception;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/UsageException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/UsageException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/UsageException.java
new file mode 100644
index 0000000..8684294
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/UsageException.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.core.exceptions;
+
+/**
+ * Used to raise a usage exception ... this has the exit code
+ * {@link #EXIT_USAGE}
+ */
+public class UsageException extends SliderException {
+  public UsageException(String s, Object... args) {
+    super(EXIT_USAGE, s, args);
+  }
+
+  public UsageException(Throwable throwable, String message,
+      Object... args) {
+    super(EXIT_USAGE, throwable, message, args);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/WaitTimeoutException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/WaitTimeoutException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/WaitTimeoutException.java
new file mode 100644
index 0000000..5ad3fdc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/exceptions/WaitTimeoutException.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.exceptions;
+
+import java.io.IOException;
+
+/**
+ * Called when some spinning operation timed out
+ */
+public class WaitTimeoutException extends IOException {
+  public WaitTimeoutException(String message) {
+    super(message);
+  }
+
+  public WaitTimeoutException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[43/76] [abbrv] hadoop git commit: YARN-5813. Slider should not try to set a negative lifetime timeout value. Contributed by Jian He

Posted by ji...@apache.org.
YARN-5813. Slider should not try to set a negative lifetime timeout value. Contributed by Jian He


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

Branch: refs/heads/yarn-native-services
Commit: 92f6fe53fd412a44db87e06707c091f35b1c541f
Parents: c3f32f4
Author: Gour Saha <go...@apache.org>
Authored: Tue Nov 1 17:39:54 2016 -0700
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/slider/client/SliderClient.java     | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/92f6fe53/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
index d1f88c5..ea10ed0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
@@ -2120,7 +2120,9 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     amLauncher.setKeepContainersOverRestarts(true);
     // set lifetime in submission context;
     Map<ApplicationTimeoutType, Long> appTimeout = new HashMap<>();
-    appTimeout.put(ApplicationTimeoutType.LIFETIME, lifetime);
+    if (lifetime >= 0) {
+      appTimeout.put(ApplicationTimeoutType.LIFETIME, lifetime);
+    }
     amLauncher.submissionContext.setApplicationTimeouts(appTimeout);
     int maxAppAttempts = config.getInt(KEY_AM_RESTART_LIMIT, 0);
     amLauncher.setMaxAppAttempts(maxAppAttempts);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[18/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentRoles.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentRoles.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentRoles.java
new file mode 100644
index 0000000..281895a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentRoles.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers.agent;
+
+import org.apache.slider.providers.ProviderRole;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class AgentRoles {
+
+  /**
+   * List of roles Agent provider does not have any roles by default. All roles are read from the application
+   * specification.
+   */
+  protected static final List<ProviderRole> ROLES =
+      new ArrayList<ProviderRole>();
+
+  public static List<ProviderRole> getRoles() {
+    return ROLES;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentUtils.java
new file mode 100644
index 0000000..cfcfc5d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentUtils.java
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.providers.agent.application.metadata.AbstractMetainfoParser;
+import org.apache.slider.providers.agent.application.metadata.AddonPackageMetainfoParser;
+import org.apache.slider.providers.agent.application.metadata.DefaultConfig;
+import org.apache.slider.providers.agent.application.metadata.DefaultConfigParser;
+import org.apache.slider.providers.agent.application.metadata.Metainfo;
+import org.apache.slider.providers.agent.application.metadata.MetainfoParser;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ *
+ */
+public class AgentUtils {
+  private static final Logger log = LoggerFactory.getLogger(AgentUtils.class);
+
+  public static Metainfo getApplicationMetainfoFromSummaryFile(
+      SliderFileSystem fileSystem, String metainfoPath, boolean metainfoForAddon) {
+    FileSystem fs = fileSystem.getFileSystem();
+    Path appPathXML = new Path(metainfoPath + ".metainfo.xml");
+    Path appPathJson = new Path(metainfoPath + ".metainfo.json");
+    Path appPathUsed = null;
+    try {
+      FSDataInputStream appStream = null;
+      if (fs.exists(appPathXML)) {
+        appPathUsed = appPathXML;
+        appStream = fs.open(appPathXML);
+        return parseMetainfo(appStream, metainfoForAddon, "xml");
+      } else if (fs.exists(appPathJson)) {
+        appPathUsed = appPathJson;
+        appStream = fs.open(appPathJson);
+        return parseMetainfo(appStream, metainfoForAddon, "json");
+      }
+    } catch (IOException e) {
+      log.info("Failed to get metainfo from summary file {} - {}", appPathUsed,
+          e.getMessage());
+      log.debug("Failed to get metainfo", e);
+    }
+    return null;
+  }
+
+  public static Metainfo getApplicationMetainfo(SliderFileSystem fileSystem,
+      String metainfoPath, boolean metainfoForAddon) throws IOException,
+      BadConfigException {
+    log.info("Reading metainfo at {}", metainfoPath);
+    Metainfo metainfo = getApplicationMetainfoFromSummaryFile(fileSystem,
+        metainfoPath, metainfoForAddon);
+    if (metainfo != null) {
+      log.info("Got metainfo from summary file");
+      return metainfo;
+    }
+
+    FileSystem fs = fileSystem.getFileSystem();
+    Path appPath = new Path(metainfoPath);
+
+    InputStream metainfoJsonStream = SliderUtils.getApplicationResourceInputStream(
+        fs, appPath, "metainfo.json");
+    if (metainfoJsonStream == null) {
+      InputStream metainfoXMLStream = SliderUtils.getApplicationResourceInputStream(
+          fs, appPath, "metainfo.xml");
+      if (metainfoXMLStream != null) {
+        metainfo = parseMetainfo(metainfoXMLStream, metainfoForAddon, "xml");
+      }
+    } else {
+      metainfo = parseMetainfo(metainfoJsonStream, metainfoForAddon, "json");
+    }
+
+    if (metainfo == null) {
+      log.error("metainfo is unavailable at {}.", metainfoPath);
+      throw new FileNotFoundException("metainfo.xml/json is required in app package. " +
+                                      appPath);
+    }
+    return metainfo;
+  }
+
+  private static Metainfo parseMetainfo(InputStream stream,
+      boolean metainfoForAddon, String type) throws IOException {
+    AbstractMetainfoParser metainfoParser = null;
+    if (metainfoForAddon) {
+      metainfoParser = new AddonPackageMetainfoParser();
+    } else {
+      metainfoParser = new MetainfoParser();
+    }
+    if (type.equals("xml")) {
+      return metainfoParser.fromXmlStream(stream);
+    } else if (type.equals("json")) {
+      return metainfoParser.fromJsonStream(stream);
+    }
+    return null;
+  }
+
+  static DefaultConfig getDefaultConfig(SliderFileSystem fileSystem,
+                                        String appDef, String configFileName)
+      throws IOException {
+    // this is the path inside the zip file
+    String fileToRead = "configuration/" + configFileName;
+    log.info("Reading default config file {} at {}", fileToRead, appDef);
+    InputStream configStream = SliderUtils.getApplicationResourceInputStream(
+        fileSystem.getFileSystem(), new Path(appDef), fileToRead);
+    if (configStream == null) {
+      log.error("{} is unavailable at {}.", fileToRead, appDef);
+      throw new IOException("Expected config file " + fileToRead + " is not available.");
+    }
+
+    return new DefaultConfigParser().parse(configStream);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/Command.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/Command.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/Command.java
new file mode 100644
index 0000000..647cb86
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/Command.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers.agent;
+
+/** The states a component instance can be. */
+public enum Command {
+  NOP,           // do nothing
+  INSTALL,       // Install the component
+  INSTALL_ADDON, // Install add on packages if any
+  START,         // Start the component
+  STOP,          // Stop the component
+  UPGRADE,       // The component will undergo upgrade
+  TERMINATE;     // Send terminate signal to agent
+
+  public static Command getCommand(String commandVal) {
+    if (commandVal.equals(Command.START.toString())) {
+      return Command.START;
+    }
+    if (commandVal.equals(Command.INSTALL.toString())) {
+      return Command.INSTALL;
+    }
+    if (commandVal.equals(Command.STOP.toString())) {
+      return Command.STOP;
+    }
+    if (commandVal.equals(Command.UPGRADE.toString())) {
+      return Command.UPGRADE;
+    }
+    if (commandVal.equals(Command.TERMINATE.toString())) {
+      return Command.TERMINATE;
+    }
+
+    return Command.NOP;
+  }
+
+  public static String transform(Command command, boolean isUpgrade) {
+    switch (command) {
+    case STOP:
+      return isUpgrade ? "UPGRADE_STOP" : command.name();
+    default:
+      return command.name();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/CommandResult.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/CommandResult.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/CommandResult.java
new file mode 100644
index 0000000..35d9116
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/CommandResult.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers.agent;
+
+/** Command results. */
+public enum CommandResult {
+  IN_PROGRESS,  // Command is in progress
+  COMPLETED,    // Command has successfully completed
+  FAILED;        // Command has failed
+
+  public static CommandResult getCommandResult(String commandResVal) {
+    if (commandResVal.equals(CommandResult.COMPLETED.toString())) {
+      return CommandResult.COMPLETED;
+    }
+    if (commandResVal.equals(CommandResult.FAILED.toString())) {
+      return CommandResult.FAILED;
+    }
+    if (commandResVal.equals(CommandResult.IN_PROGRESS.toString())) {
+      return CommandResult.IN_PROGRESS;
+    }
+
+    throw new IllegalArgumentException("Unrecognized value " + commandResVal);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentCommandOrder.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentCommandOrder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentCommandOrder.java
new file mode 100644
index 0000000..91f1259
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentCommandOrder.java
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers.agent;
+
+import org.apache.slider.providers.agent.application.metadata.CommandOrder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Stores the command dependency order for all components in a service. <commandOrder>
+ * <command>SUPERVISOR-START</command> <requires>NIMBUS-STARTED</requires> </commandOrder> Means, SUPERVISOR START
+ * requires NIMBUS to be STARTED
+ */
+public class ComponentCommandOrder {
+  public static final Logger log =
+      LoggerFactory.getLogger(ComponentCommandOrder.class);
+  private static char SPLIT_CHAR = '-';
+  Map<Command, Map<String, List<ComponentState>>> dependencies =
+      new HashMap<Command, Map<String, List<ComponentState>>>();
+
+  public ComponentCommandOrder(List<CommandOrder> commandOrders) {
+    if (commandOrders != null && commandOrders.size() > 0) {
+      for (CommandOrder commandOrder : commandOrders) {
+        ComponentCommand componentCmd = getComponentCommand(commandOrder.getCommand());
+        String requires = commandOrder.getRequires();
+        List<ComponentState> requiredStates = parseRequiredStates(requires);
+        if (requiredStates.size() > 0) {
+          Map<String, List<ComponentState>> compDep = dependencies.get(componentCmd.command);
+          if (compDep == null) {
+            compDep = new HashMap<>();
+            dependencies.put(componentCmd.command, compDep);
+          }
+
+          List<ComponentState> requirements = compDep.get(componentCmd.componentName);
+          if (requirements == null) {
+            requirements = new ArrayList<>();
+            compDep.put(componentCmd.componentName, requirements);
+          }
+
+          requirements.addAll(requiredStates);
+        }
+      }
+    }
+  }
+
+  private List<ComponentState> parseRequiredStates(String requires) {
+    if (requires == null || requires.length() < 2) {
+      throw new IllegalArgumentException("Input cannot be null and must contain component and state.");
+    }
+
+    String[] componentStates = requires.split(",");
+    List<ComponentState> retList = new ArrayList<ComponentState>();
+    for (String componentStateStr : componentStates) {
+      retList.add(getComponentState(componentStateStr));
+    }
+
+    return retList;
+  }
+
+  private ComponentCommand getComponentCommand(String compCmdStr) {
+    if (compCmdStr == null || compCmdStr.trim().length() < 2) {
+      throw new IllegalArgumentException("Input cannot be null and must contain component and command.");
+    }
+
+    compCmdStr = compCmdStr.trim();
+    int splitIndex = compCmdStr.lastIndexOf(SPLIT_CHAR);
+    if (splitIndex == -1 || splitIndex == 0 || splitIndex == compCmdStr.length() - 1) {
+      throw new IllegalArgumentException("Input does not appear to be well-formed.");
+    }
+    String compStr = compCmdStr.substring(0, splitIndex);
+    String cmdStr = compCmdStr.substring(splitIndex + 1);
+
+    Command cmd = Command.valueOf(cmdStr);
+
+    if (cmd != Command.START) {
+      throw new IllegalArgumentException("Dependency order can only be specified for START.");
+    }
+    return new ComponentCommand(compStr, cmd);
+  }
+
+  private ComponentState getComponentState(String compStStr) {
+    if (compStStr == null || compStStr.trim().length() < 2) {
+      throw new IllegalArgumentException("Input cannot be null.");
+    }
+
+    compStStr = compStStr.trim();
+    int splitIndex = compStStr.lastIndexOf(SPLIT_CHAR);
+    if (splitIndex == -1 || splitIndex == 0 || splitIndex == compStStr.length() - 1) {
+      throw new IllegalArgumentException("Input does not appear to be well-formed.");
+    }
+    String compStr = compStStr.substring(0, splitIndex);
+    String stateStr = compStStr.substring(splitIndex + 1);
+
+    State state = State.valueOf(stateStr);
+    if (state != State.STARTED && state != State.INSTALLED) {
+      throw new IllegalArgumentException("Dependency order can only be specified against STARTED/INSTALLED.");
+    }
+    return new ComponentState(compStr, state);
+  }
+
+  // dependency is still on component level, but not package level
+  // so use component name to check dependency, not component-package
+  public boolean canExecute(String component, Command command, Collection<ComponentInstanceState> currentStates) {
+    boolean canExecute = true;
+    if (dependencies.containsKey(command) && dependencies.get(command).containsKey(component)) {
+      List<ComponentState> required = dependencies.get(command).get(component);
+      for (ComponentState stateToMatch : required) {
+        for (ComponentInstanceState currState : currentStates) {
+          log.debug("Checking schedule {} {} against dependency {} is {}",
+                    component, command, currState.getComponentName(), currState.getState());
+          if (currState.getComponentName().equals(stateToMatch.componentName)) {
+            if (currState.getState() != stateToMatch.state) {
+              if (stateToMatch.state == State.STARTED) {
+                log.info("Cannot schedule {} {} as dependency {} is {}",
+                         component, command, currState.getComponentName(), currState.getState());
+                canExecute = false;
+              } else {
+                //state is INSTALLED
+                if (currState.getState() != State.STARTING && currState.getState() != State.STARTED) {
+                  log.info("Cannot schedule {} {} as dependency {} is {}",
+                           component, command, currState.getComponentName(), currState.getState());
+                  canExecute = false;
+                }
+              }
+            }
+          }
+          if (!canExecute) {
+            break;
+          }
+        }
+        if (!canExecute) {
+          break;
+        }
+      }
+    }
+
+    return canExecute;
+  }
+
+  static class ComponentState {
+    public String componentName;
+    public State state;
+
+    public ComponentState(String componentName, State state) {
+      this.componentName = componentName;
+      this.state = state;
+    }
+  }
+
+  static class ComponentCommand {
+    public String componentName;
+    public Command command;
+
+    public ComponentCommand(String componentName, Command command) {
+      this.componentName = componentName;
+      this.command = command;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentInstanceState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentInstanceState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentInstanceState.java
new file mode 100644
index 0000000..6ee0ebb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentInstanceState.java
@@ -0,0 +1,340 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers.agent;
+
+import java.util.Map;
+import java.util.TreeMap;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.slider.providers.agent.application.metadata.Component;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** This class implements a simple state machine for component instances. */
+public class ComponentInstanceState {
+  public static final Logger log =
+      LoggerFactory.getLogger(ComponentInstanceState.class);
+  private static int MAX_FAILURE_TOLERATED = 3;
+  private static String INVALID_TRANSITION_ERROR =
+      "Result %s for command %s is not expected for component %s in state %s.";
+
+  private final String componentName;
+  private final ContainerId containerId;
+  private final String containerIdAsString;
+  private final String applicationId;
+  private State state = State.INIT;
+  private State targetState = State.STARTED;
+  private int failuresSeen = 0;
+  private Boolean configReported = false;
+  private long lastHeartbeat = 0;
+  private String ip;
+  private String hostname;
+  private ContainerState containerState;
+
+  private Map<String, State> pkgStatuses;
+  private String nextPkgToInstall;
+
+  private boolean stopInitiated;
+
+  public ComponentInstanceState(String componentName,
+      ContainerId containerId,
+      String applicationId) {
+    this(componentName, containerId, applicationId,
+        new TreeMap<String, State>());
+  }
+
+  public ComponentInstanceState(String componentName,
+      ContainerId containerId,
+      String applicationId, Map<String, State> pkgStatuses) {
+    this.componentName = componentName;
+    this.containerId = containerId;
+    this.containerIdAsString = containerId.toString();
+    this.applicationId = applicationId;
+    this.containerState = ContainerState.INIT;
+    this.lastHeartbeat = System.currentTimeMillis();
+    this.pkgStatuses = pkgStatuses;
+  }
+  
+  public String getComponentName() {
+    return componentName;
+  }
+
+  public Boolean getConfigReported() {
+    return configReported;
+  }
+
+  public void setConfigReported(Boolean configReported) {
+    this.configReported = configReported;
+  }
+
+  public ContainerState getContainerState() {
+    return containerState;
+  }
+
+  public void setContainerState(ContainerState containerState) {
+    this.containerState = containerState;
+  }
+
+  public long getLastHeartbeat() {
+    return lastHeartbeat;
+  }
+
+  /**
+   * Update the heartbeat, and change container state
+   * to mark as healthy if appropriate
+   * @param heartbeatTime last time the heartbeat was seen
+   * @return the current container state
+   */
+  public ContainerState heartbeat(long heartbeatTime) {
+    this.lastHeartbeat = heartbeatTime;
+    if(containerState == ContainerState.UNHEALTHY ||
+       containerState == ContainerState.INIT) {
+      containerState = ContainerState.HEALTHY;
+    }
+    return containerState;
+  }
+  
+
+  public ContainerId getContainerId() {
+    return containerId;
+  }
+
+  public void commandIssued(Command command) {
+    commandIssued(command, false);
+  }
+
+  public void commandIssued(Command command, boolean isInUpgradeMode) {
+    Command expected = getNextCommand(isInUpgradeMode);
+    if (expected != command) {
+      throw new IllegalArgumentException("Command " + command + " is not allowed in state " + state);
+    }
+    if (expected == Command.INSTALL_ADDON) {
+      // for add on packages, the pkg must be nextPkgToInstall
+      State currentState = pkgStatuses.get(nextPkgToInstall);
+      log.debug("Command issued: component: {} is in {}", componentName,
+          currentState);
+      State nextState = currentState.getNextState(command);
+      pkgStatuses.put(nextPkgToInstall, nextState);
+      log.debug("Command issued: component: {} is now in {}", componentName,
+          nextState);
+    } else {
+      // for master package
+      state = state.getNextState(command);
+    }
+  }
+
+  public void applyCommandResult(CommandResult result, Command command,
+      String pkg) {
+    // if the heartbeat is for a package
+    // update that package's state in the component status
+    // and don't bother with the master pkg
+    if (StringUtils.isNotEmpty(pkg)
+        && !Component.MASTER_PACKAGE_NAME.equals(pkg)) {
+      log.debug("This result is for component: {} pkg: {}", componentName, pkg);
+      State previousPkgState = pkgStatuses.get(pkg);
+      log.debug("Currently component: {} pkg: {} is in state: {}",
+          componentName, pkg, previousPkgState.toString());
+      State nextPkgState = previousPkgState.getNextState(result);
+      pkgStatuses.put(pkg, nextPkgState);
+      log.debug("Component: {} pkg: {} next state: {}", componentName, pkg,
+          nextPkgState);
+    } else {
+      log.debug("This result is for component: {} master package",
+          componentName);
+      applyCommandResult(result, command);
+    }
+  }
+
+  public void applyCommandResult(CommandResult result, Command command) {
+    if (!this.state.couldHaveIssued(command)) {
+      throw new IllegalStateException("Invalid command " + command + " for state " + this.state);
+    }
+
+    try {
+      if (result == CommandResult.FAILED) {
+        failuresSeen++;
+      } else if (result == CommandResult.COMPLETED) {
+        failuresSeen = 0;
+      }
+      state = state.getNextState(result);
+    } catch (IllegalArgumentException e) {
+      String message = String.format(INVALID_TRANSITION_ERROR,
+                                     result.toString(),
+                                     command.toString(),
+                                     componentName,
+                                     state.toString());
+      log.warn(message);
+      throw new IllegalStateException(message);
+    }
+  }
+
+  public boolean hasPendingCommand() {
+    if (state.canIssueCommands() &&
+        state != targetState &&
+        failuresSeen < MAX_FAILURE_TOLERATED) {
+      return true;
+    }
+
+    return false;
+  }
+
+  public Command getNextCommand() {
+    return getNextCommand(false);
+  }
+
+  public Command getNextCommand(boolean isInUpgradeMode) {
+    if (!hasPendingCommand()) {
+      nextPkgToInstall = null;
+      return Command.NOP;
+    }
+
+    log.debug("In getNextCommand, checking for component: {} ", componentName);
+    // if the master pkg is just installed, check if any add on pkg need to be
+    // installed
+    nextPkgToInstall = null;
+    if (state == State.INSTALLED) {
+      for (Map.Entry<String, State> pkgStatus : pkgStatuses.entrySet()) {
+        String pkg = pkgStatus.getKey();
+        State pkgState = pkgStatus.getValue();
+        log.debug("In getNextCommand, pkg: {} is in {}", pkg, pkgState);
+        if (pkgState == State.INSTALLING) {
+          // first check if any pkg is install in progress, if so, wait
+          // so we don't need to do anything, just return NOP
+          log.debug("In getNextCommand, pkg: {} we are issuing NOP", pkg);
+          nextPkgToInstall = pkg;
+          return Command.NOP;
+        } else if (pkgState == State.INIT) {
+          // temporarily storing pkg here
+          // in case no pkg in 'installing' state
+          // will return the package to install
+          nextPkgToInstall = pkg;
+        }
+      }
+      // when we reach here, no pkg is in 'installing' state
+      if (nextPkgToInstall != null) {
+        // nextPkgToInstall != null means some pkg is in INIT state 
+        // issue 'install' to the pkg we have stored in nextPkgToInstall
+        log.debug("In getNextCommand, pkg: {} we are issuing install addon",
+            nextPkgToInstall);
+        return Command.INSTALL_ADDON;
+      }
+    }
+    return this.state.getSupportedCommand(isInUpgradeMode, stopInitiated);
+  }
+
+  public State getState() {
+    return state;
+  }
+
+  @VisibleForTesting
+  protected void setState(State state) {
+    this.state = state;
+  }
+
+  public State getTargetState() {
+    return targetState;
+  }
+
+  public void setTargetState(State targetState) {
+    this.targetState = targetState;
+  }
+
+  public String getNextPkgToInstall() {
+    return nextPkgToInstall;
+  }
+
+  public boolean isStopInitiated() {
+    return stopInitiated;
+  }
+
+  public void setStopInitiated(boolean stopInitiated) {
+    this.stopInitiated = stopInitiated;
+  }
+
+  @Override
+  public int hashCode() {
+    int hashCode = 1;
+
+    hashCode = hashCode ^ (componentName != null ? componentName.hashCode() : 0);
+    hashCode = hashCode ^ (containerIdAsString != null ? containerIdAsString.hashCode() : 0);
+    hashCode = hashCode ^ (applicationId != null ? applicationId.hashCode() : 0);
+    return hashCode;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+
+    if (o == null || getClass() != o.getClass()) return false;
+
+    ComponentInstanceState that = (ComponentInstanceState) o;
+
+    if (this.componentName != null ?
+        !this.componentName.equals(that.componentName) : this.componentName != null) {
+      return false;
+    }
+
+    if (this.containerIdAsString != null ?
+        !this.containerIdAsString.equals(that.containerIdAsString) : this.containerIdAsString != null) {
+      return false;
+    }
+
+    if (this.applicationId != null ?
+        !this.applicationId.equals(that.applicationId) : this.applicationId != null) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder("ComponentInstanceState{");
+    sb.append("containerIdAsString='").append(containerIdAsString).append('\'');
+    sb.append(", state=").append(state);
+    sb.append(", failuresSeen=").append(failuresSeen);
+    sb.append(", lastHeartbeat=").append(lastHeartbeat);
+    sb.append(", containerState=").append(containerState);
+    sb.append(", componentName='").append(componentName).append('\'');
+    sb.append(", ip=").append(ip);
+    sb.append(", hostname='").append(hostname).append('\'');
+    sb.append('}');
+    return sb.toString();
+  }
+
+  public String getIp() {
+    return ip;
+  }
+
+  public void setIp(String ip) {
+    this.ip = ip;
+  }
+
+  public String getHostname() {
+    return hostname;
+  }
+
+  public void setHostname(String hostname) {
+    this.hostname = hostname;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentTagProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentTagProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentTagProvider.java
new file mode 100644
index 0000000..68f63fa
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentTagProvider.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent;
+
+import org.apache.slider.common.tools.SliderUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.ConcurrentHashMap;
+
+/** A simple tag provider that attempts to associate tags from 1-N to all container of a component */
+public class ComponentTagProvider {
+  private static final Logger log = LoggerFactory.getLogger(ComponentTagProvider.class);
+  private static String FREE = "free";
+  private final ConcurrentHashMap<String, ConcurrentHashMap<String, String>> allTags;
+
+  public ComponentTagProvider() {
+    allTags = new ConcurrentHashMap<String, ConcurrentHashMap<String, String>>();
+  }
+
+  /**
+   * Record an assigned tag to a container
+   *
+   * @param component
+   * @param containerId
+   * @param tag
+   */
+  public void recordAssignedTag(String component, String containerId, String tag) {
+    if (SliderUtils.isSet(component) && SliderUtils.isSet(containerId)) {
+      Integer key = null;
+      try {
+        key = Integer.valueOf(tag);
+      } catch (NumberFormatException nfe) {
+        //ignore
+      }
+      if (key != null && key > 0) {
+        ConcurrentHashMap<String, String> compTags = getComponentSpecificTags(component);
+        synchronized (compTags) {
+          for (int index = 1; index <= key.intValue(); index++) {
+            String tempKey = new Integer(index).toString();
+            if (!compTags.containsKey(tempKey)) {
+              compTags.put(tempKey, FREE);
+            }
+          }
+          compTags.put(key.toString(), containerId);
+        }
+      }
+    }
+  }
+
+  /**
+   * Get a tag for container
+   *
+   * @param component
+   * @param containerId
+   *
+   * @return
+   */
+  public String getTag(String component, String containerId) {
+    if (SliderUtils.isSet(component) && SliderUtils.isSet(containerId)) {
+      ConcurrentHashMap<String, String> compTags = getComponentSpecificTags(component);
+      synchronized (compTags) {
+        for (String key : compTags.keySet()) {
+          if (compTags.get(key).equals(containerId)) {
+            return key;
+          }
+        }
+        for (String key : compTags.keySet()) {
+          if (compTags.get(key).equals(FREE)) {
+            compTags.put(key, containerId);
+            return key;
+          }
+        }
+        String newKey = new Integer(compTags.size() + 1).toString();
+        compTags.put(newKey, containerId);
+        return newKey;
+      }
+    }
+    return "";
+  }
+
+  /**
+   * Release a tag associated with a container
+   *
+   * @param component
+   * @param containerId
+   */
+  public void releaseTag(String component, String containerId) {
+    if (SliderUtils.isSet(component) && SliderUtils.isSet(containerId)) {
+      ConcurrentHashMap<String, String> compTags = allTags.get(component);
+      if (compTags != null) {
+        synchronized (compTags) {
+          for (String key : compTags.keySet()) {
+            if (compTags.get(key).equals(containerId)) {
+              compTags.put(key, FREE);
+            }
+          }
+        }
+      }
+    }
+  }
+
+  private ConcurrentHashMap<String, String> getComponentSpecificTags(String component) {
+    if (!allTags.containsKey(component)) {
+      synchronized (allTags) {
+        if (!allTags.containsKey(component)) {
+          allTags.put(component, new ConcurrentHashMap<String, String>());
+        }
+      }
+    }
+    return allTags.get(component);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ContainerState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ContainerState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ContainerState.java
new file mode 100644
index 0000000..0394ba2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ContainerState.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers.agent;
+
+/** The states a component instance can be. */
+public enum ContainerState {
+  INIT,           // Container is not net activated
+  HEALTHY,     // Agent is heartbeating
+  UNHEALTHY,      // Container is unhealthy - no heartbeat for some interval
+  HEARTBEAT_LOST;  // Container is lost - request a new instance
+
+  /**
+   * Indicates whether or not it is a valid state to produce a command.
+   *
+   * @return true if command can be issued for this state.
+   */
+  public boolean canIssueCommands() {
+    switch (this) {
+      case HEALTHY:
+        return true;
+      default:
+        return false;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/HeartbeatMonitor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/HeartbeatMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/HeartbeatMonitor.java
new file mode 100644
index 0000000..4293916
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/HeartbeatMonitor.java
@@ -0,0 +1,130 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+/** Monitors the container state and heartbeats. */
+public class HeartbeatMonitor implements Runnable {
+  protected static final Logger log =
+      LoggerFactory.getLogger(HeartbeatMonitor.class);
+  private final int threadWakeupInterval; //1 minute
+  private final AgentProviderService provider;
+  private volatile boolean shouldRun = true;
+  private Thread monitorThread = null;
+
+  public HeartbeatMonitor(AgentProviderService provider, int threadWakeupInterval) {
+    this.provider = provider;
+    this.threadWakeupInterval = threadWakeupInterval;
+  }
+
+  public void shutdown() {
+    shouldRun = false;
+  }
+
+  public void start() {
+    log.info("Starting heartbeat monitor with interval {}", threadWakeupInterval);
+    monitorThread = new Thread(this);
+    monitorThread.start();
+  }
+
+  void join(long millis) throws InterruptedException {
+    if (isAlive()) {
+      monitorThread.join(millis);
+    }
+  }
+
+  public boolean isAlive() {
+    return monitorThread != null && monitorThread.isAlive();
+  }
+
+  @Override
+  public void run() {
+    while (shouldRun) {
+      try {
+        log.debug("Putting monitor to sleep for " + threadWakeupInterval + " " +
+                  "milliseconds");
+        Thread.sleep(threadWakeupInterval);
+        doWork(System.currentTimeMillis());
+      } catch (InterruptedException ex) {
+        log.warn("Scheduler thread is interrupted going to stop", ex);
+        shouldRun = false;
+      } catch (Exception ex) {
+        log.warn("Exception received", ex);
+      } catch (Throwable t) {
+        log.warn("ERROR", t);
+      }
+    }
+  }
+
+  /**
+   * Every interval the current state of the container are checked. If the state is INIT or HEALTHY and no HB are
+   * received in last check interval they are marked as UNHEALTHY. INIT is when the agent is started but it did not
+   * communicate at all. HEALTHY being the AM has received heartbeats. After an interval as UNHEALTHY the container is
+   * declared unavailable
+   * @param now current time in milliseconds ... tests can set this explicitly
+   */
+  @VisibleForTesting
+  public void doWork(long now) {
+    Map<String, ComponentInstanceState> componentStatuses = provider.getComponentStatuses();
+    if (componentStatuses != null) {
+      for (String containerLabel : componentStatuses.keySet()) {
+        ComponentInstanceState componentInstanceState = componentStatuses.get(containerLabel);
+        long timeSinceLastHeartbeat = now - componentInstanceState.getLastHeartbeat();
+
+        if (timeSinceLastHeartbeat > threadWakeupInterval) {
+          switch (componentInstanceState.getContainerState()) {
+            case INIT:
+            case HEALTHY:
+              componentInstanceState.setContainerState(ContainerState.UNHEALTHY);
+              log.warn(
+                  "Component {} marked UNHEALTHY. Last heartbeat received at {} approx. {} ms. back.",
+                  componentInstanceState,
+                  componentInstanceState.getLastHeartbeat(),
+                  timeSinceLastHeartbeat);
+              break;
+            case UNHEALTHY:
+              if (timeSinceLastHeartbeat > threadWakeupInterval * 2) {
+                componentInstanceState.setContainerState(
+                    ContainerState.HEARTBEAT_LOST);
+                log.warn(
+                    "Component {} marked HEARTBEAT_LOST. Last heartbeat received at {} approx. {} ms. back.",
+                    componentInstanceState, componentInstanceState.getLastHeartbeat(),
+                    timeSinceLastHeartbeat);
+                ContainerId containerId =
+                    componentInstanceState.getContainerId();
+                provider.lostContainer(containerLabel, containerId);
+              }
+              break;
+            case HEARTBEAT_LOST:
+              // unexpected case
+              log.warn("Heartbeat from lost component: {}", componentInstanceState);
+              break;
+          }
+            
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/State.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/State.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/State.java
new file mode 100644
index 0000000..5603f8d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/State.java
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers.agent;
+
+/** The states a component instance can be. */
+public enum State {
+  INIT,           // Not installed
+  INSTALLING,     // Being installed
+  INSTALLED,      // Installed (or stopped)
+  STARTING,       // Starting
+  STARTED,        // Started
+  INSTALL_FAILED, // Install failed, start failure in INSTALLED
+  UPGRADING,      // Undergoing upgrade, perform necessary pre-upgrade steps
+  UPGRADED,       // Pre-upgrade steps completed
+  STOPPING,       // Stop has been issued
+  STOPPED,        // Agent has stopped
+  TERMINATING;    // Terminate signal to ask the agent to kill itself
+                  // No need for state TERMINATED (as the agent is dead by then)
+
+  /**
+   * Indicates whether or not it is a valid state to produce a command.
+   *
+   * @return true if command can be issued for this state.
+   */
+  public boolean canIssueCommands() {
+    switch (this) {
+      case INSTALLING:
+      case STARTING:
+      case UPGRADING:
+      case STOPPING:
+      case TERMINATING:
+        return false;
+      default:
+        return true;
+    }
+  }
+
+  /**
+   * Returns valid command in this state.
+   *
+   * @return command allowed in this state.
+   */
+  public Command getSupportedCommand() {
+    return getSupportedCommand(false);
+  }
+
+  public Command getSupportedCommand(boolean isInUpgradeMode) {
+    return getSupportedCommand(isInUpgradeMode, false);
+  }
+
+  public Command getSupportedCommand(boolean isInUpgradeMode,
+      boolean stopInitiated) {
+    switch (this) {
+      case INIT:
+      case INSTALL_FAILED:
+        return Command.INSTALL;
+      case INSTALLED:
+        return Command.START;
+      case STARTED:
+      return isInUpgradeMode ? Command.UPGRADE : (stopInitiated) ? Command.STOP
+          : Command.NOP;
+      case UPGRADED:
+        return Command.STOP;
+      case STOPPED:
+        return Command.TERMINATE;
+      default:
+        return Command.NOP;
+    }
+  }
+
+  /**
+   * Returns next state based on the command result.
+   *
+   * @return next state.
+   */
+  public State getNextState(CommandResult result) throws IllegalArgumentException {
+    switch (result) {
+      case IN_PROGRESS:
+        if (this == State.INSTALLING || this == State.STARTING
+            || this == State.UPGRADING || this == State.STOPPING
+            || this == State.TERMINATING) {
+          return this;
+        } else {
+          throw new IllegalArgumentException(result + " is not valid for " + this);
+        }
+      case COMPLETED:
+        if (this == State.INSTALLING) {
+          return State.INSTALLED;
+        } else if (this == State.STARTING) {
+          return State.STARTED;
+        } else if (this == State.UPGRADING) {
+          return State.UPGRADED;
+        } else if (this == State.STOPPING) {
+          return State.STOPPED;
+        } else {
+          throw new IllegalArgumentException(result + " is not valid for " + this);
+        }
+      case FAILED:
+        if (this == State.INSTALLING) {
+          return State.INSTALL_FAILED;
+        } else if (this == State.STARTING) {
+          return State.INSTALLED;
+        } else if (this == State.UPGRADING) {
+          // if pre-upgrade failed, force stop now, so mark it upgraded
+          // what other options can be exposed to app owner?
+          return State.UPGRADED;
+        } else if (this == State.STOPPING) {
+          // if stop fails, force mark it stopped (and let container terminate)
+          return State.STOPPED;
+        } else if (this == State.STOPPED) {
+          // if in stopped state, force mark it as terminating
+          return State.TERMINATING;
+        } else {
+          throw new IllegalArgumentException(result + " is not valid for " + this);
+        }
+      default:
+        throw new IllegalArgumentException("Bad command result " + result);
+    }
+  }
+
+  /**
+   * Returns next state based on the command.
+   *
+   * @return next state.
+   */
+  public State getNextState(Command command) throws IllegalArgumentException {
+    switch (command) {
+      case INSTALL:
+        if (this == State.INIT || this == State.INSTALL_FAILED) {
+          return State.INSTALLING;
+        } else {
+          throw new IllegalArgumentException(command + " is not valid for " + this);
+        }
+      case INSTALL_ADDON:
+          if (this == State.INIT || this == State.INSTALL_FAILED) {
+            return State.INSTALLING;
+          } else {
+            throw new IllegalArgumentException(command + " is not valid for " + this);
+          }
+      case START:
+        if (this == State.INSTALLED) {
+          return State.STARTING;
+        } else {
+          throw new IllegalArgumentException(command + " is not valid for " + this);
+        }
+      case UPGRADE:
+        if (this == State.STARTED) {
+          return State.UPGRADING;
+        } else {
+          throw new IllegalArgumentException(command + " is not valid for " + this);
+        }
+      case STOP:
+        if (this == State.STARTED || this == State.UPGRADED) {
+          return State.STOPPING;
+        } else {
+          throw new IllegalArgumentException(command + " is not valid for " + this);
+        }
+      case TERMINATE:
+        if (this == State.STOPPED) {
+          return State.TERMINATING;
+        } else {
+          throw new IllegalArgumentException(command + " is not valid for " + this);
+        }
+      case NOP:
+        return this;
+      default:
+        throw new IllegalArgumentException("Bad command " + command);
+    }
+  }
+
+  public boolean couldHaveIssued(Command command) {
+    if ((this == State.INSTALLING && command == Command.INSTALL)
+        || (this == State.STARTING && command == Command.START)
+        || (this == State.UPGRADING && command == Command.UPGRADE)
+        || (this == State.STOPPING 
+           && (command == Command.STOP || command == Command.NOP))
+        || (this == State.TERMINATING && command == Command.TERMINATE)
+       ) {
+      return true;
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractComponent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractComponent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractComponent.java
new file mode 100644
index 0000000..b6ae4de
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractComponent.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/**
+ *  Component defined in master package metainfo.json
+ */
+public abstract class AbstractComponent implements Validate {
+  public static final String TYPE_STANDARD = "STANDARD";
+  public static final String TYPE_DOCKER = "DOCKER";
+  public static final String TYPE_PYTHON = "PYTHON";
+  public static final String CATEGORY_MASTER = "MASTER";
+  public static final String CATEGORY_SLAVE = "SLAVE";
+  public static final String CATEGORY_CLIENT = "CLIENT";
+  public static final String MASTER_PACKAGE_NAME = "MASTER";
+
+  protected String name;
+  protected CommandScript commandScript;
+  protected List<ComponentCommand> commands = new ArrayList<>();
+
+  public AbstractComponent() {
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public CommandScript getCommandScript() {
+    return commandScript;
+  }
+
+  public void addCommandScript(CommandScript commandScript) {
+    this.commandScript = commandScript;
+  }
+
+  @JsonProperty("commands")
+  public List<ComponentCommand> getCommands() {
+    return commands;
+  }
+
+  public void setCommands(List<ComponentCommand> commands) {
+    this.commands = commands;
+  }
+
+  public void addCommand(ComponentCommand command) {
+    commands.add(command);
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder("{");
+    sb.append("\n\"name\": ").append(name);
+    sb.append(",\n\"commandScript\" :").append(commandScript);
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractMetainfoParser.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractMetainfoParser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractMetainfoParser.java
new file mode 100644
index 0000000..67d1f15
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractMetainfoParser.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+
+import org.apache.commons.digester.Digester;
+import org.apache.commons.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.xml.sax.SAXException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringWriter;
+
+/**
+ * This abstract class provide common functionality to parse metainfo.json for
+ * either master package or add on packages.
+ */
+public abstract class AbstractMetainfoParser {
+  protected final GsonBuilder gsonBuilder = new GsonBuilder();
+  protected final Gson gson;
+  private static final Logger log = LoggerFactory
+      .getLogger(AbstractMetainfoParser.class);
+
+  public AbstractMetainfoParser() {
+    gson = gsonBuilder.create();
+  }
+
+  /**
+   * Convert to a JSON string
+   *
+   * @return a JSON string description
+   *
+   * @throws IOException Problems mapping/writing the object
+   */
+  public String toJsonString(Metainfo metaInfo) throws IOException {
+    return gson.toJson(metaInfo);
+  }
+
+  /**
+   * Convert from JSON
+   *
+   * @param json input
+   *
+   * @return the parsed JSON
+   *
+   * @throws IOException IO
+   */
+  public Metainfo fromJsonString(String json)
+      throws IOException {
+    return gson.fromJson(json, Metainfo.class);
+  }
+
+  /**
+   * Parse metainfo from an IOStream
+   *
+   * @param is
+   *
+   * @return
+   *
+   * @throws IOException
+   */
+  public Metainfo fromJsonStream(InputStream is) throws IOException {
+    log.debug("loading from xml stream");
+    StringWriter writer = new StringWriter();
+    IOUtils.copy(is, writer);
+    return fromJsonString(writer.toString());
+  }
+
+  /**
+   * Parse metainfo from an XML formatted IOStream
+   *
+   * @param metainfoStream
+   *
+   * @return
+   *
+   * @throws IOException
+   */
+  public Metainfo fromXmlStream(InputStream metainfoStream) throws IOException {
+    log.debug("loading from xml stream");
+    Digester digester = new Digester();
+    digester.setValidating(false);
+
+    composeSchema(digester);
+
+    try {
+      return (Metainfo) digester.parse(metainfoStream);
+    } catch (IOException e) {
+      log.debug("IOException in metainfoparser during fromXmlStream: "
+          + e.getMessage());
+    } catch (SAXException e) {
+      log.debug("SAXException in metainfoparser during fromXmlStream: "
+          + e.getMessage());
+    } finally {
+      if (metainfoStream != null) {
+        metainfoStream.close();
+      }
+    }
+
+    return null;
+  }
+
+  /**
+   * Compose the schema for the metainfo
+   *
+   * @param Digester - The Digester object we passed in to compose the schema
+   *
+   * @return
+   *
+   * @throws IOException
+   */
+  abstract protected void composeSchema(Digester digester);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractMetainfoSchema.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractMetainfoSchema.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractMetainfoSchema.java
new file mode 100644
index 0000000..cfa2895
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractMetainfoSchema.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.codehaus.jackson.annotate.JsonProperty;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Application type defined in the metainfo
+ */
+public abstract class AbstractMetainfoSchema implements Validate {
+  protected String name;
+  protected String comment;
+  protected String version;
+  protected List<ConfigFile> configFiles = new ArrayList<>();
+
+  public AbstractMetainfoSchema() {
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public String getComment() {
+    return comment;
+  }
+
+  public void setComment(String comment) {
+    this.comment = comment;
+  }
+
+  public String getVersion() {
+    return version;
+  }
+
+  public void setVersion(String version) {
+    this.version = version;
+  }
+
+  public void addConfigFile(ConfigFile configFile) {
+    this.configFiles.add(configFile);
+  }
+
+  @JsonProperty("configFiles")
+  public List<ConfigFile> getConfigFiles() {
+    return configFiles;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AddonPackageMetainfoParser.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AddonPackageMetainfoParser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AddonPackageMetainfoParser.java
new file mode 100644
index 0000000..c75837f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AddonPackageMetainfoParser.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.apache.commons.digester.Digester;
+
+/**
+ *
+ */
+public class AddonPackageMetainfoParser extends AbstractMetainfoParser {
+
+  protected void composeSchema(Digester digester) {
+    digester.addObjectCreate("metainfo", Metainfo.class);
+    digester.addBeanPropertySetter("metainfo/schemaVersion");
+
+    digester.addObjectCreate("*/applicationPackage", ApplicationPackage.class);
+    digester.addBeanPropertySetter("*/applicationPackage/name");
+    digester.addBeanPropertySetter("*/applicationPackage/comment");
+    digester.addBeanPropertySetter("*/applicationPackage/version");
+
+    digester.addObjectCreate("*/component", ComponentsInAddonPackage.class);
+    digester.addBeanPropertySetter("*/component/name");
+    digester.addSetNext("*/component", "addComponent");
+
+    digester.addObjectCreate("*/commandScript", CommandScript.class);
+    digester.addBeanPropertySetter("*/commandScript/script");
+    digester.addBeanPropertySetter("*/commandScript/scriptType");
+    digester.addBeanPropertySetter("*/commandScript/timeout");
+    digester.addSetNext("*/commandScript", "addCommandScript");
+
+    digester.addObjectCreate("*/configFile", ConfigFile.class);
+    digester.addBeanPropertySetter("*/configFile/type");
+    digester.addBeanPropertySetter("*/configFile/fileName");
+    digester.addBeanPropertySetter("*/configFile/dictionaryName");
+    digester.addSetNext("*/configFile", "addConfigFile");
+
+    digester.addSetRoot("*/applicationPackage", "setApplicationPackage");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Application.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Application.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Application.java
new file mode 100644
index 0000000..5556c7f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Application.java
@@ -0,0 +1,193 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.SliderException;
+import org.codehaus.jackson.annotate.JsonIgnore;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Application type defined in the metainfo
+ */
+public class Application extends AbstractMetainfoSchema {
+  String exportedConfigs;
+  List<ExportGroup> exportGroups = new ArrayList<>();
+  List<OSSpecific> osSpecifics = new ArrayList<>();
+  List<CommandOrder> commandOrders = new ArrayList<>();
+  List<Package> packages = new ArrayList<>();
+  private List<Component> components = new ArrayList<>();
+
+  public Application() {
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public String getComment() {
+    return comment;
+  }
+
+  public void setComment(String comment) {
+    this.comment = comment;
+  }
+
+  public String getVersion() {
+    return version;
+  }
+
+  public void setVersion(String version) {
+    this.version = version;
+  }
+
+  public String getExportedConfigs() {
+    return exportedConfigs;
+  }
+
+  public void setExportedConfigs(String exportedConfigs) {
+    this.exportedConfigs = exportedConfigs;
+  }
+
+  public void addConfigFile(ConfigFile configFile) {
+    this.configFiles.add(configFile);
+  }
+
+  @JsonProperty("configFiles")
+  public List<ConfigFile> getConfigFiles() {
+    return configFiles;
+  }
+
+  public void addComponent(Component component) {
+    components.add(component);
+  }
+
+  @JsonProperty("components")
+  public List<Component> getComponents() {
+    return components;
+  }
+
+  public void addExportGroup(ExportGroup exportGroup) {
+    exportGroups.add(exportGroup);
+  }
+
+  @JsonProperty("exportGroups")
+  public List<ExportGroup> getExportGroups() {
+    return exportGroups;
+  }
+
+  public void addOSSpecific(OSSpecific osSpecific) {
+    osSpecifics.add(osSpecific);
+  }
+
+  @JsonIgnore
+  public List<OSSpecific> getOSSpecifics() {
+    return osSpecifics;
+  }
+
+  public void addCommandOrder(CommandOrder commandOrder) {
+    commandOrders.add(commandOrder);
+  }
+
+  @JsonProperty("commandOrders")
+  public List<CommandOrder> getCommandOrders() {
+    return commandOrders;
+  }
+
+  public void addPackage(Package pkg) {
+    packages.add(pkg);
+  }
+
+  @JsonProperty("packages")
+  public List<Package> getPackages() {
+    return packages;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder("{");
+    sb.append(",\n\"name\": ").append(name);
+    sb.append(",\n\"comment\": ").append(comment);
+    sb.append(",\n\"version\" :").append(version);
+    sb.append(",\n\"components\" : {");
+    for (Component component : components) {
+      sb.append("\n").append(component.toString());
+    }
+    sb.append("\n},");
+    sb.append('}');
+    return sb.toString();
+  }
+
+  public void validate(String version) throws SliderException {
+    if(SliderUtils.isUnset(version)) {
+      throw new BadCommandArgumentsException("schema version cannot be null");
+    }
+
+    Metainfo.checkNonNull(getName(), "name", "application");
+
+    Metainfo.checkNonNull(getVersion(), "version", "application");
+
+    if(getComponents().size() == 0) {
+      throw new SliderException("application must contain at least one component");
+    }
+
+    if(version.equals(Metainfo.VERSION_TWO_ZERO)) {
+      if(getPackages().size() > 0) {
+        throw new SliderException("packages is not supported in version " + version);
+      }
+    }
+
+    if(version.equals(Metainfo.VERSION_TWO_ONE)) {
+      if(getOSSpecifics().size() > 0) {
+        throw new SliderException("osSpecifics is not supported in version " + version);
+      }
+    }
+
+    for(CommandOrder co : getCommandOrders()) {
+      co.validate(version);
+    }
+
+    for(Component comp : getComponents()) {
+      comp.validate(version);
+    }
+
+    for(ConfigFile cf : getConfigFiles()) {
+      cf.validate(version);
+    }
+
+    for(ExportGroup eg : getExportGroups()) {
+      eg.validate(version);
+    }
+
+    for(Package pkg : getPackages()) {
+      pkg.validate(version);
+    }
+
+    for(OSSpecific os : getOSSpecifics()) {
+      os.validate(version);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ApplicationPackage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ApplicationPackage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ApplicationPackage.java
new file mode 100644
index 0000000..a94a213
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ApplicationPackage.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.slider.core.exceptions.SliderException;
+
+public class ApplicationPackage extends AbstractMetainfoSchema{
+  private List<ComponentsInAddonPackage> components = new ArrayList<ComponentsInAddonPackage>();
+
+  public void addComponent(ComponentsInAddonPackage component) {
+    components.add(component);
+  }
+
+  // we must override getcomponent() as well. otherwise it is pointing to the
+  // overriden components of type List<Component>
+  public List<ComponentsInAddonPackage> getComponents(){
+    return this.components;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder("{");
+    sb.append("\n\"name\": ").append(name);
+    sb.append(",\n\"comment\": ").append(comment);
+    sb.append(",\n\"version\" :").append(version);
+    sb.append(",\n\"components\" : {");
+    for (ComponentsInAddonPackage component : components) {
+      sb.append("\n").append(component);
+    }
+    sb.append("\n},");
+    sb.append('}');
+    return sb.toString();
+  }
+
+  @Override
+  public void validate(String version) throws SliderException {
+    if (name == null || name.isEmpty()) {
+      throw new SliderException(
+          "Missing name in metainfo.json for add on packages");
+    }
+    if (components.isEmpty()) {
+      throw new SliderException(
+          "Missing components in metainfo.json for add on packages");
+    }
+    for (ComponentsInAddonPackage component : components) {
+      if (component.name == null || component.name.isEmpty()) {
+        throw new SliderException(
+            "Missing name of components in metainfo.json for add on packages");
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/CommandOrder.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/CommandOrder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/CommandOrder.java
new file mode 100644
index 0000000..40d8cc6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/CommandOrder.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.apache.slider.core.exceptions.SliderException;
+
+/**
+ *
+ */
+public class CommandOrder implements Validate {
+  String command;
+  String requires;
+
+  public CommandOrder() {
+  }
+
+  public String getCommand() {
+    return command;
+  }
+
+  public void setCommand(String command) {
+    this.command = command;
+  }
+
+  public String getRequires() {
+    return requires;
+  }
+
+  public void setRequires(String requires) {
+    this.requires = requires;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder("{");
+    sb.append(",\n\"command\": ").append(command);
+    sb.append(",\n\"requires\": ").append(requires);
+    sb.append('}');
+    return sb.toString();
+  }
+
+  public void validate(String version) throws SliderException {
+    Metainfo.checkNonNull(getCommand(), "command", "package");
+    Metainfo.checkNonNull(getRequires(), "requires", "package");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/CommandScript.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/CommandScript.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/CommandScript.java
new file mode 100644
index 0000000..9915ba1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/CommandScript.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.apache.slider.core.exceptions.SliderException;
+
+/**
+ * CommandScript that implements all component commands
+ */
+public class CommandScript implements Validate {
+  String script;
+  String scriptType;
+  long timeout;
+
+  public CommandScript() {
+
+  }
+
+  public String getScript() {
+    return script;
+  }
+
+  public void setScript(String script) {
+    this.script = script;
+  }
+
+  public String getScriptType() {
+    return scriptType;
+  }
+
+  public void setScriptType(String scriptType) {
+    this.scriptType = scriptType;
+  }
+
+  public long getTimeout() {
+    return timeout;
+  }
+
+  public void setTimeout(long timeout) {
+    this.timeout = timeout;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder("{");
+    sb.append(",\n\"script\": ").append(script);
+    sb.append(",\n\"scriptType\": ").append(scriptType);
+    sb.append(",\n\"timeout\" :").append(timeout);
+    sb.append('}');
+    return sb.toString();
+  }
+
+  public void validate(String version) throws SliderException {
+    Metainfo.checkNonNull(getScript(), "script", "commandScript");
+    Metainfo.checkNonNull(getScriptType(), "scriptType", "commandScript");
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[59/76] [abbrv] hadoop git commit: YARN-5909. Remove agent related code in slider AM. Contributed by Jian He

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Component.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Component.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Component.java
deleted file mode 100644
index 78bb8c1..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Component.java
+++ /dev/null
@@ -1,217 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.apache.slider.common.tools.SliderUtils;
-import org.apache.slider.core.exceptions.BadConfigException;
-import org.apache.slider.core.exceptions.SliderException;
-import org.codehaus.jackson.annotate.JsonProperty;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Component defined in master package metainfo.json
- */
-public class Component extends AbstractComponent {
-
-  String category = CATEGORY_MASTER;
-  String publishConfig = Boolean.FALSE.toString();
-  String minInstanceCount = "0";
-  String maxInstanceCount;
-  String autoStartOnFailure = Boolean.FALSE.toString();
-  String appExports;
-  String compExports;
-  String type = TYPE_STANDARD;
-  List<ComponentExport> componentExports = new ArrayList<>();
-  List<DockerContainer> dockerContainers = new ArrayList<>();
-  List<ConfigFile> configFiles = new ArrayList<>();
-
-  public Component() {
-  }
-
-  public String getType() {
-    return type;
-  }
-
-  public void setType(String type) {
-    this.type = type;
-  }
-
-  public String getCategory() {
-    return category;
-  }
-
-  public void setCategory(String category) {
-    this.category = category;
-  }
-
-  public String getPublishConfig() {
-    return publishConfig;
-  }
-
-  public void setPublishConfig(String publishConfig) {
-    this.publishConfig = publishConfig;
-  }
-
-  public String getAutoStartOnFailure() {
-    return autoStartOnFailure;
-  }
-
-  public void setAutoStartOnFailure(String autoStartOnFailure) {
-    this.autoStartOnFailure = autoStartOnFailure;
-  }
-
-  public String getAppExports() {
-    return appExports;
-  }
-
-  public void setAppExports(String appExports) {
-    this.appExports = appExports;
-  }
-
-  public String getCompExports() {
-    return compExports;
-  }
-
-  public void setCompExports(String compExports) {
-    this.compExports = compExports;
-  }
-
-  public String getMinInstanceCount() {
-    return minInstanceCount;
-  }
-  
-  @JsonProperty("dockerContainers")
-  public List<DockerContainer> getDockerContainers() {
-     return this.dockerContainers;
-  }
-  
-  public Boolean getAutoStartOnFailureBoolean() {
-    if (SliderUtils.isUnset(getAutoStartOnFailure())) {
-      return Boolean.FALSE;
-    }
-
-    return Boolean.parseBoolean(getAutoStartOnFailure());
-  }
-
-  public int getMinInstanceCountInt() throws BadConfigException {
-    if (SliderUtils.isUnset(minInstanceCount)) {
-      return 0;
-    }
-
-    try {
-      return Integer.parseInt(minInstanceCount);
-    } catch (NumberFormatException nfe) {
-      throw new BadConfigException(nfe, "Invalid value for minInstanceCount for %s", name);
-    }
-  }
-
-  public int getMaxInstanceCountInt() throws BadConfigException {
-    if (SliderUtils.isUnset(maxInstanceCount)) {
-      return Integer.MAX_VALUE;
-    }
-
-    try {
-      return Integer.parseInt(maxInstanceCount);
-    } catch (NumberFormatException nfe) {
-      throw new BadConfigException(nfe, "Invalid value for maxInstanceCount for %s", name);
-    }
-  }
-
-  public void setMinInstanceCount(String minInstanceCount) {
-    this.minInstanceCount = minInstanceCount;
-  }
-
-  public String getMaxInstanceCount() {
-    return maxInstanceCount;
-  }
-
-  public void setMaxInstanceCount(String maxInstanceCount) {
-    this.maxInstanceCount = maxInstanceCount;
-  }
-
-  public void addComponentExport(ComponentExport export) {
-    componentExports.add(export);
-  }
-
-  public List<ComponentExport> getComponentExports() {
-    return componentExports;
-  }
-
-  public Boolean getRequiresAutoRestart() {
-    return Boolean.parseBoolean(this.autoStartOnFailure);
-  }
-
-  public void addConfigFile(ConfigFile configFile) {
-    this.configFiles.add(configFile);
-  }
-
-  @JsonProperty("configFiles")
-  public List<ConfigFile> getConfigFiles() {
-    return configFiles;
-  }
-
-  @Override
-  public String toString() {
-    final StringBuilder sb =
-        new StringBuilder("{");
-    sb.append("\n\"name\": ").append(name);
-    sb.append(",\n\"category\": ").append(category);
-    sb.append(",\n\"commandScript\" :").append(commandScript);
-    for(DockerContainer dc : dockerContainers){
-      sb.append(",\n\"container\" :").append(dc.toString());
-    }    
-    sb.append('}');
-    return sb.toString();
-  }
-
-  public void validate(String version) throws SliderException {
-    Metainfo.checkNonNull(getName(), "name", "component");
-    Metainfo.checkNonNull(getCategory(), "category", "component");
-    if (!getCategory().equals(CATEGORY_MASTER)
-        && !getCategory().equals(CATEGORY_SLAVE)
-        && !getCategory().equals(CATEGORY_CLIENT)) {
-      throw new SliderException("Invalid category for the component " + getCategory());
-    }
-
-    Metainfo.checkNonNull(getType(), "type", "component");
-    if (!getType().equals(TYPE_DOCKER)
-        && !getType().equals(TYPE_STANDARD)) {
-      throw new SliderException("Invalid type for the component " + getType());
-    }
-
-    if (version.equals(Metainfo.VERSION_TWO_ZERO)) {
-      if (getType().equals(TYPE_DOCKER)) {
-        throw new SliderException(TYPE_DOCKER + " is not supported in version " + Metainfo.VERSION_TWO_ZERO);
-      }
-
-      if (getCommands().size() > 0) {
-        throw new SliderException("commands are not supported in version " + Metainfo.VERSION_TWO_ZERO);
-      }
-    }
-
-    if (commandScript != null) {
-      commandScript.validate(version);
-    }
-
-    if (version.equals(Metainfo.VERSION_TWO_ONE)) {
-      for (ComponentCommand cc : getCommands()) {
-        cc.validate(version);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentCommand.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentCommand.java
deleted file mode 100644
index 52117c5..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentCommand.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *  or more contributor license agreements.  See the NOTICE file
- *  distributed with this work for additional information
- *  regarding copyright ownership.  The ASF licenses this file
- *  to you under the Apache License, Version 2.0 (the
- *  "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *       http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.apache.slider.core.exceptions.SliderException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Represents the metadata associated with the application.
- */
-public class ComponentCommand implements Validate {
-  protected static final Logger
-      log = LoggerFactory.getLogger(ComponentCommand.class);
-
-
-  private String exec;
-  private String name = "START";
-  private String type = "SHELL";
-
-  /**
-   * Creator.
-   */
-  public ComponentCommand() {
-  }
-
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public void setExec(String exec) {
-    this.exec = exec;
-  }
-
-  public String getExec() {
-    return exec;
-  }
-
-  public void setType(String type) {
-    this.type = type;
-  }
-
-  public String getType() {
-    return type;
-  }
-
-  public void validate(String version) throws SliderException {
-    Metainfo.checkNonNull(getName(), "name", "componentCommand");
-
-    Metainfo.checkNonNull(getType(), "version", "application");
-  }
-
-  public static ComponentCommand getDefaultComponentCommand() {
-    ComponentCommand cc = new ComponentCommand();
-    cc.setExec("DEFAULT");
-    return cc;
-  }
-
-  public static ComponentCommand getDefaultComponentCommand(String commandName) {
-    ComponentCommand cc = new ComponentCommand();
-    cc.setExec("DEFAULT");
-    cc.setName(commandName);
-    return cc;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentExport.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentExport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentExport.java
deleted file mode 100644
index a18854c..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentExport.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-/**
- *
- */
-public class ComponentExport {
-  String name;
-  String value;
-
-  public ComponentExport() {
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  public String getValue() {
-    return value;
-  }
-
-  public void setValue(String value) {
-    this.value = value;
-  }
-
-  @Override
-  public String toString() {
-    final StringBuilder sb =
-        new StringBuilder("{");
-    sb.append(",\n\"name\": ").append(name);
-    sb.append(",\n\"value\": ").append(value);
-    sb.append('}');
-    return sb.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentsInAddonPackage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentsInAddonPackage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentsInAddonPackage.java
deleted file mode 100644
index 855e5b6..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ComponentsInAddonPackage.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.apache.slider.core.exceptions.SliderException;
-
-public class ComponentsInAddonPackage extends AbstractComponent {
-
-  @Override
-  public void validate(String version) throws SliderException {
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ConfigFile.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ConfigFile.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ConfigFile.java
deleted file mode 100644
index cb47512..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ConfigFile.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.apache.slider.core.exceptions.SliderException;
-
-/**
- *
- */
-public class ConfigFile implements Validate {
-  String type;
-  String fileName;
-  String dictionaryName;
-
-  public ConfigFile() {
-  }
-
-  public String getType() {
-    return type;
-  }
-
-  public void setType(String type) {
-    this.type = type;
-  }
-
-  public String getFileName() {
-    return fileName;
-  }
-
-  public void setFileName(String fileName) {
-    this.fileName = fileName;
-  }
-
-  public String getDictionaryName() {
-    return dictionaryName;
-  }
-
-  public void setDictionaryName(String dictionaryName) {
-    this.dictionaryName = dictionaryName;
-  }
-
-  public void validate(String version) throws SliderException {
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DefaultConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DefaultConfig.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DefaultConfig.java
deleted file mode 100644
index 46c8836..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DefaultConfig.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Application default config
- */
-public class DefaultConfig {
-  List<PropertyInfo> propertyInfos;
-
-  public DefaultConfig() {
-    propertyInfos = new ArrayList<PropertyInfo>();
-  }
-
-  public void addPropertyInfo(PropertyInfo propertyInfo) {
-    propertyInfos.add(propertyInfo);
-  }
-
-  public List<PropertyInfo> getPropertyInfos() {
-    return propertyInfos;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DefaultConfigParser.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DefaultConfigParser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DefaultConfigParser.java
deleted file mode 100644
index e136775..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DefaultConfigParser.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.apache.commons.digester.Digester;
-import org.xml.sax.SAXException;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-/**
- *
- */
-public class DefaultConfigParser {
-
-  public DefaultConfig parse(InputStream configFileStream) throws IOException {
-    Digester digester = new Digester();
-    digester.setValidating(false);
-
-    digester.addObjectCreate("configuration", DefaultConfig.class);
-
-    digester.addObjectCreate("*/property", PropertyInfo.class);
-    digester.addBeanPropertySetter("*/property/name");
-    digester.addBeanPropertySetter("*/property/value");
-    digester.addBeanPropertySetter("*/property/description");
-    digester.addSetNext("*/property", "addPropertyInfo");
-
-    try {
-      return (DefaultConfig) digester.parse(configFileStream);
-    } catch (IOException e) {
-
-    } catch (SAXException e) {
-
-    } finally {
-      configFileStream.close();
-    }
-
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainer.java
deleted file mode 100644
index 4c61e7f..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainer.java
+++ /dev/null
@@ -1,187 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *  or more contributor license agreements.  See the NOTICE file
- *  distributed with this work for additional information
- *  regarding copyright ownership.  The ASF licenses this file
- *  to you under the Apache License, Version 2.0 (the
- *  "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *       http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.apache.slider.core.exceptions.SliderException;
-import org.codehaus.jackson.annotate.JsonProperty;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Represents a docker container
- */
-public class DockerContainer implements Validate {
-  protected static final Logger
-      log = LoggerFactory.getLogger(DockerContainer.class);
-
-  private String name;
-  private String image;
-  private String network;
-  private String useNetworkScript;
-  private String options;
-  private List<DockerContainerMount> mounts = new ArrayList<>();
-  private List<DockerContainerPort> ports = new ArrayList<>();
-  private String statusCommand;
-  private String startCommand;
-  private String commandPath;
-  private String additionalParam;
-  private String runPrivilegedContainer;
-  private List<DockerContainerInputFile> inputFiles = new ArrayList<>();
-  private List<ConfigFile> configFiles = new ArrayList<>();
-
-  public DockerContainer() {
-  }
-
-  @JsonProperty("mounts")
-  public List<DockerContainerMount> getMounts() { return this.mounts; }
-
-  @JsonProperty("ports")
-  public List<DockerContainerPort> getPorts() {
-    return this.ports;
-  }
-
-  @JsonProperty("inputFiles")
-  public List<DockerContainerInputFile> getInputFiles() {
-    return this.inputFiles;
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  public String getImage() {
-    return image;
-  }
-
-  public void setImage(String image) {
-    this.image = image;
-  }
-
-  public String getNetwork() {
-    return network;
-  }
-
-  public void setNetwork(String network) {
-    this.network = network;
-  }
-
-  public String getUseNetworkScript() {
-    return useNetworkScript;
-  }
-
-  public void setUseNetworkScript(String useNetworkScript) {
-    this.useNetworkScript = useNetworkScript;
-  }
-
-  public String getOptions() {
-    return options;
-  }
-
-  public void setOptions(String options) {
-    this.options = options;
-  }
-
-  @Override
-  public void validate(String version) throws SliderException {
-    Metainfo.checkNonNull(getName(), "name", "dockerContainer");
-    Metainfo.checkNonNull(getImage(), "image", "dockerContainer");
-    for (DockerContainerMount dcm : getMounts()) {
-      dcm.validate(version);
-    }
-    for (DockerContainerPort dcp : getPorts()) {
-      dcp.validate(version);
-    }
-  }
-
-  @JsonProperty("statusCommand")
-  public String getStatusCommand() {
-    return statusCommand;
-  }
-
-  @JsonProperty("statusCommand")
-  public void setStatusCommand(String statusCommand) {
-    this.statusCommand = statusCommand;
-  }
-
-  public String getCommandPath() {
-    return commandPath;
-  }
-
-  public void setCommandPath(String commandPath) {
-    this.commandPath = commandPath;
-  }
-
-  public String getAdditionalParam() {
-    return additionalParam;
-  }
-
-  public void setAdditionalParam(String additionalParam) {
-    this.additionalParam = additionalParam;
-  }
-
-  @JsonProperty("startCommand")
-  public String getStartCommand() {
-    return startCommand;
-  }
-
-  @JsonProperty("startCommand")
-  public void setStartCommand(String startCommand) {
-    this.startCommand = startCommand;
-  }
-
-  @JsonProperty("runPrivilegedContainer")
-  public String getRunPrivilegedContainer() {
-    return runPrivilegedContainer;
-  }
-
-  @JsonProperty("runPrivilegedContainer")
-  public void setRunPrivilegedContainer(String runPrivilegedContainer) {
-    this.runPrivilegedContainer = runPrivilegedContainer;
-  }
-
-  public List<ConfigFile> getConfigFiles() {
-    return configFiles;
-  }
-
-  public void setConfigFiles(List<ConfigFile> configFiles) {
-    this.configFiles = configFiles;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder result = new StringBuilder("DockerContainer [name=")
-        .append(name).append(", image=").append(image).append(", options=")
-        .append(options).append(", mounts=").append(mounts).append(", ports=")
-        .append(ports).append(", statusCommand=").append(statusCommand)
-        .append(", commandPath=").append(commandPath)
-        .append(", additionalParam=").append(additionalParam)
-        .append(", inputFiles=").append(inputFiles).append(", startCommand=")
-        .append(startCommand).append(", runPriviledgedContainer=")
-        .append(runPrivilegedContainer).append(", configFiles=")
-        .append(configFiles).append("]");
-    return result.toString();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerInputFile.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerInputFile.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerInputFile.java
deleted file mode 100644
index 0faceb9..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerInputFile.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *  or more contributor license agreements.  See the NOTICE file
- *  distributed with this work for additional information
- *  regarding copyright ownership.  The ASF licenses this file
- *  to you under the Apache License, Version 2.0 (the
- *  "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *       http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class DockerContainerInputFile {
-  protected static final Logger log = LoggerFactory
-      .getLogger(DockerContainerInputFile.class);
-
-  private String containerPath;
-  private String fileLocalPath;
-
-  public DockerContainerInputFile() {
-  }
-
-  public String getContainerMount() {
-    return containerPath;
-  }
-
-  public void setContainerMount(String containerMount) {
-    this.containerPath = containerMount;
-  }
-
-  public String getFileLocalPath() {
-    return fileLocalPath;
-  }
-
-  public void setFileLocalPath(String fileLocalPath) {
-    this.fileLocalPath = fileLocalPath;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerMount.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerMount.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerMount.java
deleted file mode 100644
index 61f07f4..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerMount.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *  or more contributor license agreements.  See the NOTICE file
- *  distributed with this work for additional information
- *  regarding copyright ownership.  The ASF licenses this file
- *  to you under the Apache License, Version 2.0 (the
- *  "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *       http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.apache.slider.core.exceptions.SliderException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Represents a docker container mount
- */
-public class DockerContainerMount implements Validate {
-  protected static final Logger
-      log = LoggerFactory.getLogger(DockerContainerMount.class);
-
-
-  private String containerMount;
-  private String hostMount;
-
-  public DockerContainerMount() {
-  }
-
-  public String getContainerMount() {
-    return containerMount;
-  }
-
-  public void setContainerMount(String containerMount) {
-    this.containerMount = containerMount;
-  }
-
-  public String getHostMount() {
-    return hostMount;
-  }
-
-  public void setHostMount(String hostMount) {
-    this.hostMount = hostMount;
-  }
-
-  @Override
-  public void validate(String version) throws SliderException {
-    Metainfo.checkNonNull(getContainerMount(), "containerMount", "dockerContainerMount");
-    Metainfo.checkNonNull(getHostMount(), "hostMount", "dockerContainerMount");
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerPort.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerPort.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerPort.java
deleted file mode 100644
index 0629d9d..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/DockerContainerPort.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *  or more contributor license agreements.  See the NOTICE file
- *  distributed with this work for additional information
- *  regarding copyright ownership.  The ASF licenses this file
- *  to you under the Apache License, Version 2.0 (the
- *  "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *       http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.apache.slider.core.exceptions.SliderException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Represents a docker container port
- */
-public class DockerContainerPort implements Validate {
-  protected static final Logger
-      log = LoggerFactory.getLogger(DockerContainerPort.class);
-
-
-  private String containerPort;
-  private String hostPort;
-
-  public DockerContainerPort() {
-  }
-
-  public String getContainerPort() {
-    return containerPort;
-  }
-
-  public void setContainerPort(String containerPort) {
-    this.containerPort = containerPort;
-  }
-
-  public String getHostPort() {
-    return hostPort;
-  }
-
-  public void setHostPort(String hostPort) {
-    this.hostPort = hostPort;
-  }
-
-  @Override
-  public void validate(String version) throws SliderException {
-    Metainfo.checkNonNull(getContainerPort(), "containerPort", "dockerContainerPort");
-    Metainfo.checkNonNull(getHostPort(), "hostPort", "dockerContainerPort");
-  }
-
-  @Override
-  public String toString() {
-    return "DockerContainerPort [containerPort=" + containerPort
-         + ", hostPort=" + hostPort + "]";
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Export.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Export.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Export.java
deleted file mode 100644
index 5e0fb24..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Export.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.apache.slider.core.exceptions.SliderException;
-
-/**
- *
- */
-public class Export implements Validate {
-  String name;
-  String value;
-
-  public Export() {
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  public String getValue() {
-    return value;
-  }
-
-  public void setValue(String value) {
-    this.value = value;
-  }
-
-  @Override
-  public String toString() {
-    final StringBuilder sb =
-        new StringBuilder("{");
-    sb.append(",\n\"name\": ").append(name);
-    sb.append(",\n\"value\": ").append(value);
-    sb.append('}');
-    return sb.toString();
-  }
-
-  public void validate(String version) throws SliderException {
-    Metainfo.checkNonNull(getName(), "name", "export");
-    Metainfo.checkNonNull(getValue(), "value", "export");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ExportGroup.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ExportGroup.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ExportGroup.java
deleted file mode 100644
index 3d9f34c..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ExportGroup.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.apache.slider.core.exceptions.SliderException;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- *
- */
-public class ExportGroup implements Validate {
-  String name;
-  List<Export> exports;
-
-  public ExportGroup() {
-    exports = new ArrayList<Export>();
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  public void addExport(Export export) {
-    exports.add(export);
-  }
-
-  public List<Export> getExports() {
-    return exports;
-  }
-
-  @Override
-  public String toString() {
-    final StringBuilder sb =
-        new StringBuilder("{");
-    sb.append(",\n\"name\": ").append(name);
-    sb.append(",\n\"exports\" : {");
-    for (Export export : exports) {
-      sb.append("\n").append(export);
-    }
-    sb.append("\n},");
-    sb.append('}');
-    return sb.toString();
-  }
-
-  public void validate(String version) throws SliderException {
-    Metainfo.checkNonNull(getName(), "name", "exportGroup");
-    for(Export exp : getExports()) {
-      exp.validate(version);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Metainfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Metainfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Metainfo.java
deleted file mode 100644
index 10c497f..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Metainfo.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.apache.slider.common.tools.SliderUtils;
-import org.apache.slider.core.exceptions.SliderException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Application metainfo uber class
- */
-public class Metainfo {
-  protected static final Logger log =
-      LoggerFactory.getLogger(Metainfo.class);
-  public static String VERSION_TWO_ZERO = "2.0";
-  public static String VERSION_TWO_ONE = "2.1";
-
-  String schemaVersion;
-  ApplicationPackage applicationPackage;
-  Application application;
-
-  public String getSchemaVersion() {
-    return schemaVersion;
-  }
-
-  public void setSchemaVersion(String schemaVersion) {
-    this.schemaVersion = schemaVersion;
-  }
-
-  public ApplicationPackage getApplicationPackage() {
-    return applicationPackage;
-  }
-
-  public void setApplicationPackage(ApplicationPackage pkg) {
-    this.applicationPackage = pkg;
-  }
-
-  public Application getApplication() {
-    return application;
-  }
-
-  public void setApplication(Application application) {
-    this.application = application;
-  }
-
-  public Component getApplicationComponent(String roleGroup) {
-    if (application == null) {
-      log.error("Malformed app definition: Expect application as the top level element for metainfo");
-    } else {
-      for (Component component : application.getComponents()) {
-        if (component.getName().equals(roleGroup)) {
-          return component;
-        }
-      }
-    }
-    return null;
-  }
-
-  public List<ConfigFile> getComponentConfigFiles(String roleGroup) {
-    List<ConfigFile> componentConfigFiles = new ArrayList<>();
-    componentConfigFiles.addAll(application.getConfigFiles());
-    Component component = getApplicationComponent(roleGroup);
-    if (component != null) {
-      componentConfigFiles.addAll(component.getConfigFiles());
-    }
-    return componentConfigFiles;
-  }
-
-  public void validate() throws SliderException {
-    if (!VERSION_TWO_ONE.equals(schemaVersion) &&
-        !VERSION_TWO_ZERO.equals(schemaVersion)) {
-      throw new SliderException("Unsupported version " + getSchemaVersion());
-    }
-    if (application != null) {
-      application.validate(schemaVersion);
-    }
-    if (applicationPackage != null) {
-      applicationPackage.validate(schemaVersion);
-    }
-  }
-
-  public static void checkNonNull(String value, String field, String type) throws SliderException {
-    if (SliderUtils.isUnset(value)) {
-      throw new SliderException(type + "." + field + " cannot be null");
-    }
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder builder = new StringBuilder();
-    builder.append("Metainfo [schemaVersion=");
-    builder.append(schemaVersion);
-    builder.append(", applicationPackage=");
-    builder.append(applicationPackage);
-    builder.append(", application=");
-    builder.append(application);
-    builder.append("]");
-    return builder.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/MetainfoParser.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/MetainfoParser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/MetainfoParser.java
deleted file mode 100644
index 8b520eb..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/MetainfoParser.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.apache.commons.digester.Digester;
-
-/**
- *
- */
-public class MetainfoParser extends AbstractMetainfoParser{
-  
-  protected void composeSchema(Digester digester){
-    digester.addObjectCreate("metainfo", Metainfo.class);
-    digester.addBeanPropertySetter("metainfo/schemaVersion");
-
-    digester.addObjectCreate("*/application", Application.class);
-    digester.addBeanPropertySetter("*/application/name");
-    digester.addBeanPropertySetter("*/application/comment");
-    digester.addBeanPropertySetter("*/application/version");
-    digester.addBeanPropertySetter("*/application/exportedConfigs");
-
-    digester.addObjectCreate("*/commandOrder", CommandOrder.class);
-    digester.addBeanPropertySetter("*/commandOrder/command");
-    digester.addBeanPropertySetter("*/commandOrder/requires");
-    digester.addSetNext("*/commandOrder", "addCommandOrder");
-
-    digester.addObjectCreate("*/exportGroup", ExportGroup.class);
-    digester.addBeanPropertySetter("*/exportGroup/name");
-    digester.addObjectCreate("*/export", Export.class);
-    digester.addBeanPropertySetter("*/export/name");
-    digester.addBeanPropertySetter("*/export/value");
-    digester.addSetNext("*/export", "addExport");
-    digester.addSetNext("*/exportGroup", "addExportGroup");
-
-    digester.addObjectCreate("*/component", Component.class);
-    digester.addBeanPropertySetter("*/component/name");
-    digester.addBeanPropertySetter("*/component/category");
-    digester.addBeanPropertySetter("*/component/publishConfig");
-    digester.addBeanPropertySetter("*/component/minInstanceCount");
-    digester.addBeanPropertySetter("*/component/maxInstanceCount");
-    digester.addBeanPropertySetter("*/component/autoStartOnFailure");
-    digester.addBeanPropertySetter("*/component/appExports");
-    digester.addBeanPropertySetter("*/component/compExports");
-    digester.addObjectCreate("*/componentExport", ComponentExport.class);
-    digester.addBeanPropertySetter("*/componentExport/name");
-    digester.addBeanPropertySetter("*/componentExport/value");
-    digester.addSetNext("*/componentExport", "addComponentExport");
-    digester.addSetNext("*/component", "addComponent");
-
-    digester.addObjectCreate("*/commandScript", CommandScript.class);
-    digester.addBeanPropertySetter("*/commandScript/script");
-    digester.addBeanPropertySetter("*/commandScript/scriptType");
-    digester.addBeanPropertySetter("*/commandScript/timeout");
-    digester.addSetNext("*/commandScript", "addCommandScript");
-
-    digester.addObjectCreate("*/command", ComponentCommand.class);
-    digester.addBeanPropertySetter("*/command/exec");
-    digester.addBeanPropertySetter("*/command/name");
-    digester.addBeanPropertySetter("*/command/type");
-    digester.addSetNext("*/command", "addCommand");
-
-    digester.addObjectCreate("*/osSpecific", OSSpecific.class);
-    digester.addBeanPropertySetter("*/osSpecific/osType");
-    digester.addObjectCreate("*/osSpecific/packages/package", OSPackage.class);
-    digester.addBeanPropertySetter("*/osSpecific/packages/package/type");
-    digester.addBeanPropertySetter("*/osSpecific/packages/package/name");
-    digester.addSetNext("*/osSpecific/packages/package", "addOSPackage");
-    digester.addSetNext("*/osSpecific", "addOSSpecific");
-
-    digester.addObjectCreate("*/application/packages/package", Package.class);
-    digester.addBeanPropertySetter("*/application/packages/package/type");
-    digester.addBeanPropertySetter("*/application/packages/package/name");
-    digester.addSetNext("*/application/packages/package", "addPackage");
-
-    digester.addObjectCreate("*/configFile", ConfigFile.class);
-    digester.addBeanPropertySetter("*/configFile/type");
-    digester.addBeanPropertySetter("*/configFile/fileName");
-    digester.addBeanPropertySetter("*/configFile/dictionaryName");
-    digester.addSetNext("*/configFile", "addConfigFile");
-
-    digester.addSetRoot("*/application", "setApplication");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSPackage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSPackage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSPackage.java
deleted file mode 100644
index 32b4890..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSPackage.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.apache.slider.core.exceptions.SliderException;
-
-/**
- *
- */
-public class OSPackage implements Validate {
-  String type;
-  String name;
-
-  public OSPackage() {
-  }
-
-  public String getType() {
-    return type;
-  }
-
-  public void setType(String type) {
-    this.type = type;
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  public void validate(String version) throws SliderException {
-    Metainfo.checkNonNull(getName(), "name", "osPackage");
-    Metainfo.checkNonNull(getType(), "type", "osPackage");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSSpecific.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSSpecific.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSSpecific.java
deleted file mode 100644
index c06d498..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/OSSpecific.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.apache.slider.core.exceptions.SliderException;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- *
- */
-public class OSSpecific implements Validate {
-  String osType;
-  List<OSPackage> packages;
-
-  public OSSpecific() {
-    packages = new ArrayList<OSPackage>();
-  }
-
-  public String getOsType() {
-    return osType;
-  }
-
-  public void setOsType(String osType) {
-    this.osType = osType;
-  }
-
-  public void addOSPackage(OSPackage osPackage) {
-    packages.add(osPackage);
-  }
-
-  public List<OSPackage> getPackages() {
-    return packages;
-  }
-
-  public void validate(String version) throws SliderException {
-    Metainfo.checkNonNull(getOsType(), "osType", "osSpecific");
-    for (OSPackage opkg : getPackages()) {
-      opkg.validate(version);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Package.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Package.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Package.java
deleted file mode 100644
index b88f77d..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Package.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *  or more contributor license agreements.  See the NOTICE file
- *  distributed with this work for additional information
- *  regarding copyright ownership.  The ASF licenses this file
- *  to you under the Apache License, Version 2.0 (the
- *  "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *       http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.apache.slider.core.exceptions.SliderException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Represents package description.
- */
-public class Package implements Validate {
-  protected static final Logger
-      log = LoggerFactory.getLogger(Package.class);
-
-
-  private String name;
-  private String type;
-
-  public Package() {
-  }
-
-
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public String getType() {
-    return type;
-  }
-
-  public void setType(String type) {
-    this.type = type;
-  }
-
-  public void validate(String version) throws SliderException {
-    Metainfo.checkNonNull(getName(), "name", "package");
-    Metainfo.checkNonNull(getType(), "type", "package");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/PropertyInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/PropertyInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/PropertyInfo.java
deleted file mode 100644
index 62ee0f5..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/PropertyInfo.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-/**
- * Application config property info
- */
-public class PropertyInfo {
-  String name;
-  String value;
-  String description;
-
-  public PropertyInfo() {
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  public String getValue() {
-    return value;
-  }
-
-  public void setValue(String value) {
-    this.value = value;
-  }
-
-  public String getDescription() {
-    return description;
-  }
-
-  public void setDescription(String description) {
-    this.description = description;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Validate.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Validate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Validate.java
deleted file mode 100644
index ef03dcd..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Validate.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.apache.slider.core.exceptions.SliderException;
-
-/**
- * Implementer provides a validate method
- */
-public interface Validate {
-
-  public void validate(String version) throws SliderException;
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/todo.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/todo.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/todo.md
deleted file mode 100644
index dfd1373..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/todo.md
+++ /dev/null
@@ -1,22 +0,0 @@
-<!---
-   Licensed to the Apache Software Foundation (ASF) under one or more
-   contributor license agreements.  See the NOTICE file distributed with
-   this work for additional information regarding copyright ownership.
-   The ASF licenses this file to You under the Apache License, Version 2.0
-   (the "License"); you may not use this file except in compliance with
-   the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-   Unless required by applicable law or agreed to in writing, software
-   distributed under the License is distributed on an "AS IS" BASIS,
-   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-   See the License for the specific language governing permissions and
-   limitations under the License.
--->
-  
-# todo list
-
-* Retry on failure
-  * Agent can toleate a configurable number of failures (e.g. 3) before giving up
-* Agent should separate out hostname and label that is received for registration

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
index 57ec218..34b6a7d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
@@ -20,12 +20,8 @@ package org.apache.slider.server.appmaster;
 
 import com.codahale.metrics.MetricRegistry;
 import com.codahale.metrics.health.HealthCheckRegistry;
-import com.codahale.metrics.jvm.GarbageCollectorMetricSet;
-import com.codahale.metrics.jvm.MemoryUsageGaugeSet;
-import com.codahale.metrics.jvm.ThreadStatesGaugeSet;
 import com.google.common.base.Preconditions;
 import com.google.protobuf.BlockingService;
-
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -36,8 +32,14 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
 import org.apache.hadoop.registry.client.binding.RegistryTypeUtils;
 import org.apache.hadoop.registry.client.binding.RegistryUtils;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.hadoop.registry.client.types.yarn.PersistencePolicies;
+import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes;
+import org.apache.hadoop.registry.server.integration.RMRegistryOperationsService;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
@@ -66,18 +68,9 @@ import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
 import org.apache.hadoop.yarn.client.api.async.NMClientAsync;
 import org.apache.hadoop.yarn.client.api.async.impl.NMClientAsyncImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import static org.apache.hadoop.yarn.conf.YarnConfiguration.*;
-import static org.apache.slider.common.Constants.HADOOP_JAAS_DEBUG;
-
 import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
-import org.apache.hadoop.registry.client.api.RegistryOperations;
-import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
-import org.apache.hadoop.registry.client.types.yarn.PersistencePolicies;
-import org.apache.hadoop.registry.client.types.ServiceRecord;
-import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes;
-import org.apache.hadoop.registry.server.integration.RMRegistryOperationsService;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
 import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
@@ -122,18 +115,16 @@ import org.apache.slider.providers.ProviderCompleted;
 import org.apache.slider.providers.ProviderRole;
 import org.apache.slider.providers.ProviderService;
 import org.apache.slider.providers.SliderProviderFactory;
-import org.apache.slider.providers.agent.AgentKeys;
-import org.apache.slider.providers.agent.AgentProviderService;
 import org.apache.slider.providers.slideram.SliderAMClientProvider;
 import org.apache.slider.providers.slideram.SliderAMProviderService;
 import org.apache.slider.server.appmaster.actions.ActionRegisterServiceInstance;
-import org.apache.slider.server.appmaster.actions.EscalateOutstandingRequests;
-import org.apache.slider.server.appmaster.actions.RegisterComponentInstance;
-import org.apache.slider.server.appmaster.actions.QueueExecutor;
-import org.apache.slider.server.appmaster.actions.QueueService;
 import org.apache.slider.server.appmaster.actions.ActionStopSlider;
 import org.apache.slider.server.appmaster.actions.ActionUpgradeContainers;
 import org.apache.slider.server.appmaster.actions.AsyncAction;
+import org.apache.slider.server.appmaster.actions.EscalateOutstandingRequests;
+import org.apache.slider.server.appmaster.actions.QueueExecutor;
+import org.apache.slider.server.appmaster.actions.QueueService;
+import org.apache.slider.server.appmaster.actions.RegisterComponentInstance;
 import org.apache.slider.server.appmaster.actions.RenewingAction;
 import org.apache.slider.server.appmaster.actions.ResetFailureWindow;
 import org.apache.slider.server.appmaster.actions.ReviewAndFlexApplicationSize;
@@ -143,26 +134,24 @@ import org.apache.slider.server.appmaster.management.YarnServiceHealthCheck;
 import org.apache.slider.server.appmaster.monkey.ChaosKillAM;
 import org.apache.slider.server.appmaster.monkey.ChaosKillContainer;
 import org.apache.slider.server.appmaster.monkey.ChaosMonkeyService;
+import org.apache.slider.server.appmaster.operations.AbstractRMOperation;
 import org.apache.slider.server.appmaster.operations.AsyncRMOperationHandler;
 import org.apache.slider.server.appmaster.operations.ProviderNotifyingOperationHandler;
+import org.apache.slider.server.appmaster.operations.RMOperationHandler;
 import org.apache.slider.server.appmaster.rpc.RpcBinder;
 import org.apache.slider.server.appmaster.rpc.SliderAMPolicyProvider;
 import org.apache.slider.server.appmaster.rpc.SliderClusterProtocolPBImpl;
-import org.apache.slider.server.appmaster.operations.AbstractRMOperation;
 import org.apache.slider.server.appmaster.rpc.SliderIPCService;
 import org.apache.slider.server.appmaster.security.SecurityConfiguration;
 import org.apache.slider.server.appmaster.state.AppState;
 import org.apache.slider.server.appmaster.state.AppStateBindingInfo;
 import org.apache.slider.server.appmaster.state.ContainerAssignment;
 import org.apache.slider.server.appmaster.state.ProviderAppState;
-import org.apache.slider.server.appmaster.operations.RMOperationHandler;
 import org.apache.slider.server.appmaster.state.RoleInstance;
-import org.apache.slider.server.appmaster.web.AgentService;
-import org.apache.slider.server.appmaster.web.rest.InsecureAmFilterInitializer;
-import org.apache.slider.server.appmaster.web.rest.agent.AgentWebApp;
 import org.apache.slider.server.appmaster.web.SliderAMWebApp;
 import org.apache.slider.server.appmaster.web.WebAppApi;
 import org.apache.slider.server.appmaster.web.WebAppApiImpl;
+import org.apache.slider.server.appmaster.web.rest.InsecureAmFilterInitializer;
 import org.apache.slider.server.appmaster.web.rest.RestPaths;
 import org.apache.slider.server.appmaster.web.rest.application.ApplicationResouceContentCacheFactory;
 import org.apache.slider.server.appmaster.web.rest.application.resources.ContentCache;
@@ -181,7 +170,6 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URL;
-import java.net.URLClassLoader;
 import java.nio.ByteBuffer;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
@@ -199,6 +187,9 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.ReentrantLock;
 
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.*;
+import static org.apache.slider.common.Constants.HADOOP_JAAS_DEBUG;
+
 /**
  * This is the AM, which directly implements the callbacks from the AM and NM
  */
@@ -765,11 +756,6 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
                                     appMasterContainerID.toString(),
                                     clustername);
       certificateManager.setPassphrase(instanceDefinition.getPassphrase());
- 
-      if (component.getOptionBool(
-          AgentKeys.KEY_AGENT_TWO_WAY_SSL_ENABLED, false)) {
-        uploadServerCertForLocalization(clustername, fs);
-      }
 
       // Web service endpoints: initialize
       WebAppApiImpl webAppApi =
@@ -784,10 +770,6 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
               contentCache);
       initAMFilterOptions(serviceConf);
 
-      if (providerService instanceof AgentProviderService) {
-        // start the agent web app
-        startAgentWebApp(appInformation, serviceConf, webAppApi);
-      }
       int webAppPort = deployWebApplication(webAppApi);
 
       String scheme = WebAppUtils.HTTP_PREFIX;
@@ -1165,26 +1147,6 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     return portScanner.getAvailablePort();
   }
 
-  private void uploadServerCertForLocalization(String clustername,
-                                               SliderFileSystem fs)
-      throws IOException {
-    Path certsDir = fs.buildClusterSecurityDirPath(clustername);
-    if (!fs.getFileSystem().exists(certsDir)) {
-      fs.getFileSystem().mkdirs(certsDir,
-        new FsPermission(FsAction.ALL, FsAction.NONE, FsAction.NONE));
-    }
-    Path destPath = new Path(certsDir, SliderKeys.CRT_FILE_NAME);
-    if (!fs.getFileSystem().exists(destPath)) {
-      fs.getFileSystem().copyFromLocalFile(
-          new Path(CertificateManager.getServerCertficateFilePath().getAbsolutePath()),
-          destPath);
-      log.info("Uploaded server cert to localization path {}", destPath);
-    }
-
-    fs.getFileSystem().setPermission(destPath,
-        new FsPermission(FsAction.READ, FsAction.NONE, FsAction.NONE));
-  }
-
   protected void login(String principal, File localKeytabFile)
       throws IOException, SliderException {
     log.info("Logging in as {} with keytab {}", principal, localKeytabFile);
@@ -1221,53 +1183,6 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
   }
 
   /**
-   * Set up and start the agent web application 
-   * @param appInformation application information
-   * @param serviceConf service configuration
-   * @param webAppApi web app API instance to bind to
-   * @throws IOException
-   */
-  private void startAgentWebApp(MapOperations appInformation,
-      Configuration serviceConf, WebAppApiImpl webAppApi) throws IOException, SliderException {
-    URL[] urls = ((URLClassLoader) AgentWebApp.class.getClassLoader() ).getURLs();
-    StringBuilder sb = new StringBuilder("AM classpath:");
-    for (URL url : urls) {
-      sb.append("\n").append(url.toString());
-    }
-    LOG_YARN.debug(sb.append("\n").toString());
-    initAMFilterOptions(serviceConf);
-
-
-    // Start up the agent web app and track the URL for it
-    MapOperations appMasterConfig = getInstanceDefinition()
-        .getAppConfOperations().getComponent(SliderKeys.COMPONENT_AM);
-    AgentWebApp agentWebApp = AgentWebApp.$for(AgentWebApp.BASE_PATH,
-        webAppApi,
-        RestPaths.AGENT_WS_CONTEXT)
-        .withComponentConfig(appMasterConfig)
-        .withPort(getPortToRequest())
-        .withSecuredPort(getPortToRequest())
-            .start();
-    agentOpsUrl =
-        "https://" + appMasterHostname + ":" + agentWebApp.getSecuredPort();
-    agentStatusUrl =
-        "https://" + appMasterHostname + ":" + agentWebApp.getPort();
-    AgentService agentService =
-      new AgentService("slider-agent", agentWebApp);
-
-    agentService.init(serviceConf);
-    agentService.start();
-    addService(agentService);
-
-    appInformation.put(StatusKeys.INFO_AM_AGENT_OPS_URL, agentOpsUrl + "/");
-    appInformation.put(StatusKeys.INFO_AM_AGENT_STATUS_URL, agentStatusUrl + "/");
-    appInformation.set(StatusKeys.INFO_AM_AGENT_STATUS_PORT,
-                       agentWebApp.getPort());
-    appInformation.set(StatusKeys.INFO_AM_AGENT_OPS_PORT,
-                       agentWebApp.getSecuredPort());
-  }
-
-  /**
    * Set up the AM filter 
    * @param serviceConf configuration to patch
    */
@@ -1327,17 +1242,8 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
         serviceRecord);
 
     // provider service dynamic definitions.
-    if (providerService instanceof AgentProviderService) {
-      URL agentOpsURI = new URL(agentOpsUrl);
-      URL agentStatusURI = new URL(agentStatusUrl);
-      ((AgentProviderService)providerService).applyInitialRegistryDefinitions(
-          amWebURI,
-          agentOpsURI,
-          agentStatusURI,
-          serviceRecord);
-    } else {
-      providerService.applyInitialRegistryDefinitions(amWebURI, serviceRecord);
-    }
+    providerService.applyInitialRegistryDefinitions(amWebURI, serviceRecord);
+
 
     // set any provided attributes
     setProvidedServiceRecordAttributes(
@@ -1800,11 +1706,6 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     }
     LOG_YARN.info("Final list of containers to be upgraded (total {}) : {}",
         containers.size(), containers);
-    if (providerService instanceof AgentProviderService) {
-      AgentProviderService agentProviderService = (AgentProviderService) providerService;
-      agentProviderService.setInUpgradeMode(true);
-      agentProviderService.addUpgradeContainers(containers);
-    }
   }
 
   // create a reverse map of roles -> set of all live containers
@@ -1968,11 +1869,6 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
    * Shutdown operation: release all containers
    */
   private void releaseAllContainers() {
-    if (providerService instanceof AgentProviderService) {
-      log.info("Setting stopInitiated flag to true");
-      AgentProviderService agentProviderService = (AgentProviderService) providerService;
-      agentProviderService.setAppStopInitiated(true);
-    }
     // Add the sleep here (before releasing containers) so that applications get
     // time to perform graceful shutdown
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/AgentService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/AgentService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/AgentService.java
deleted file mode 100644
index f840035..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/AgentService.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.server.appmaster.web;
-
-import org.apache.slider.server.appmaster.web.rest.agent.AgentWebApp;
-import org.apache.slider.server.services.workflow.ClosingService;
-import org.apache.slider.server.services.workflow.WorkflowCompositeService;
-
-/**
- * agent service gives the agent webapp lifecycle integration
- */
-public class AgentService extends ClosingService<AgentWebApp> {
-
-
-  public AgentService(String name) {
-    super(name);
-  }
-
-  public AgentService(String name, AgentWebApp app) {
-    super(name, app);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
index 0f99d6d..65a3591 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
@@ -17,7 +17,6 @@
 package org.apache.slider.server.appmaster.web;
 
 import org.apache.hadoop.registry.client.api.RegistryOperations;
-import org.apache.slider.api.SliderClusterProtocol;
 import org.apache.slider.providers.ProviderService;
 import org.apache.slider.server.appmaster.AppMasterActionOperations;
 import org.apache.slider.server.appmaster.actions.QueueAccess;
@@ -25,7 +24,6 @@ import org.apache.slider.server.appmaster.management.MetricsAndMonitoring;
 import org.apache.slider.server.appmaster.state.AppState;
 import org.apache.slider.server.appmaster.state.RoleStatus;
 import org.apache.slider.server.appmaster.state.StateAccessForProviders;
-import org.apache.slider.server.appmaster.web.rest.agent.AgentRestOperations;
 import org.apache.slider.server.appmaster.web.rest.application.resources.ContentCache;
 import org.apache.slider.server.services.security.CertificateManager;
 
@@ -57,11 +55,6 @@ public interface WebAppApi {
    * is a computed value and not just a getter
    */
   Map<String, RoleStatus> getRoleStatusByName();
-
-  /**
-   * Returns an interface that can support the agent-based REST operations.
-   */
-  AgentRestOperations getAgentRestOperations();
   
   /**
    * Registry operations accessor


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[35/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/proto/RestTypeMarshalling.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/proto/RestTypeMarshalling.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/proto/RestTypeMarshalling.java
new file mode 100644
index 0000000..17fd965
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/proto/RestTypeMarshalling.java
@@ -0,0 +1,319 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.api.proto;
+
+import com.google.protobuf.ByteString;
+import org.apache.commons.io.IOUtils;
+import org.apache.slider.api.types.ApplicationLivenessInformation;
+import org.apache.slider.api.types.ComponentInformation;
+import org.apache.slider.api.types.ContainerInformation;
+import org.apache.slider.api.types.NodeEntryInformation;
+import org.apache.slider.api.types.NodeInformation;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTree;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.core.persist.AggregateConfSerDeser;
+import org.apache.slider.core.persist.ConfTreeSerDeser;
+import org.apache.slider.server.services.security.SecurityStore;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Class to handle marshalling of REST
+ * types to/from Protobuf records.
+ */
+public class RestTypeMarshalling {
+
+  public static Messages.ApplicationLivenessInformationProto
+  marshall(ApplicationLivenessInformation info) {
+
+    Messages.ApplicationLivenessInformationProto.Builder builder =
+        Messages.ApplicationLivenessInformationProto.newBuilder();
+    builder.setAllRequestsSatisfied(info.allRequestsSatisfied);
+    builder.setRequestsOutstanding(info.requestsOutstanding);
+    return builder.build();
+  }
+
+  public static ApplicationLivenessInformation
+  unmarshall(Messages.ApplicationLivenessInformationProto wire) {
+    ApplicationLivenessInformation info = new ApplicationLivenessInformation();
+    info.allRequestsSatisfied = wire.getAllRequestsSatisfied();
+    info.requestsOutstanding = wire.getRequestsOutstanding();
+    return info;
+  }
+
+  public static ComponentInformation
+  unmarshall(Messages.ComponentInformationProto wire) {
+    ComponentInformation info = new ComponentInformation();
+    info.name = wire.getName();
+    info.priority = wire.getPriority();
+    info.placementPolicy = wire.getPlacementPolicy();
+    
+    info.actual = wire.getActual();
+    info.completed = wire.getCompleted();
+    info.desired = wire.getDesired();
+    info.failed = wire.getFailed();
+    info.releasing = wire.getReleasing();
+    info.requested = wire.getRequested();
+    info.started = wire.getStarted();
+    info.startFailed = wire.getStartFailed();
+    info.totalRequested = wire.getTotalRequested();
+    info.containers = new ArrayList<>(wire.getContainersList());
+    if (wire.hasFailureMessage()) {
+      info.failureMessage = wire.getFailureMessage();
+    }
+    if (wire.hasPendingAntiAffineRequestCount()) {
+      info.pendingAntiAffineRequestCount = wire.getPendingAntiAffineRequestCount();
+    }
+    if (wire.hasIsAARequestOutstanding()) {
+      info.isAARequestOutstanding = wire.getIsAARequestOutstanding();
+    }
+    return info;
+  }
+
+  public static Messages.GetCertificateStoreResponseProto marshall(
+      SecurityStore securityStore) throws IOException {
+    Messages.GetCertificateStoreResponseProto.Builder builder =
+        Messages.GetCertificateStoreResponseProto.newBuilder();
+    builder.setStore(ByteString.copyFrom(getStoreBytes(securityStore)));
+
+    return builder.build();
+  }
+
+  private static byte[] getStoreBytes(SecurityStore securityStore)
+      throws IOException {
+    InputStream is = null;
+    byte[] storeBytes;
+    try {
+      is = new FileInputStream(securityStore.getFile());
+      storeBytes = IOUtils.toByteArray(is);
+    } finally {
+      if (is != null) {
+        is.close();
+      }
+    }
+    return storeBytes;
+  }
+
+  public static byte[] unmarshall(Messages.GetCertificateStoreResponseProto response) {
+    return response.getStore().toByteArray();
+  }
+
+  public static Messages.ComponentInformationProto marshall(ComponentInformation info) {
+
+    Messages.ComponentInformationProto.Builder builder =
+        Messages.ComponentInformationProto.newBuilder();
+    builder.setName(info.name);
+    builder.setPriority(info.priority);
+    builder.setPlacementPolicy(info.placementPolicy);
+    
+    builder.setActual(info.actual);
+    builder.setCompleted(info.completed);
+    builder.setDesired(info.desired);
+    builder.setFailed(info.failed);
+    builder.setReleasing(info.releasing);
+    builder.setRequested(info.requested);
+    builder.setStarted(info.started);
+    builder.setStartFailed(info.startFailed);
+    builder.setTotalRequested(info.totalRequested);
+    builder.setNodeFailed(info.nodeFailed);
+    builder.setPreempted(info.preempted);
+    builder.setFailedRecently(info.failedRecently);
+    if (info.failureMessage != null) {
+      builder.setFailureMessage(info.failureMessage);
+    }
+    if (info.containers != null) {
+      builder.addAllContainers(info.containers);
+    }
+    builder.setPendingAntiAffineRequestCount(info.pendingAntiAffineRequestCount);
+    builder.setIsAARequestOutstanding(info.isAARequestOutstanding);
+    return builder.build();
+  }
+
+  public static Messages.NodeInformationProto marshall(NodeInformation info) {
+
+    Messages.NodeInformationProto.Builder builder =
+        Messages.NodeInformationProto.newBuilder();
+    builder.setHostname(info.hostname);
+    builder.setLastUpdated(info.lastUpdated);
+    builder.setState(info.state != null? info.state : "unknown");
+    builder.setRackName(info.rackName != null ? info.rackName : "");
+    builder.setHealthReport(info.healthReport != null ? info.healthReport : "");
+    builder.setHttpAddress(info.httpAddress != null ? info.httpAddress : "");
+    builder.setLabels(info.labels != null ? info.labels: "");
+
+
+    if (info.entries != null) {
+      for (Map.Entry<String, NodeEntryInformation> elt : info.entries.entrySet()) {
+        NodeEntryInformation entry = elt.getValue();
+        Messages.NodeEntryInformationProto.Builder node =
+            Messages.NodeEntryInformationProto.newBuilder();
+        node.setPriority(entry.priority);
+        node.setName(elt.getKey());
+        node.setFailed(entry.failed);
+        node.setFailedRecently(entry.failedRecently);
+        node.setLive(entry.live);
+        node.setLastUsed(entry.lastUsed);
+        node.setPreempted(entry.preempted);
+        node.setRequested(entry.requested);
+        node.setReleasing(entry.releasing);
+        node.setStartFailed(entry.startFailed);
+        node.setStarting(entry.starting);
+        builder.addEntries(node.build());
+      }
+    }
+    return builder.build();
+  }
+
+  public static NodeInformation unmarshall(Messages.NodeInformationProto wire) {
+    NodeInformation info = new NodeInformation();
+    info.healthReport = wire.getHealthReport();
+    info.hostname = wire.getHostname();
+    info.httpAddress = wire.getHttpAddress();
+    info.labels = wire.getLabels();
+    info.lastUpdated = wire.getLastUpdated();
+    info.rackName = wire.getRackName();
+    info.state = wire.getState();
+    List<Messages.NodeEntryInformationProto> entriesList = wire.getEntriesList();
+    if (entriesList != null) {
+      info.entries = new HashMap<>(entriesList.size());
+      for (Messages.NodeEntryInformationProto entry : entriesList) {
+        NodeEntryInformation nei = new NodeEntryInformation();
+        nei.failed = entry.getFailed();
+        nei.failedRecently = entry.getFailedRecently();
+        nei.lastUsed = entry.getLastUsed();
+        nei.live = entry.getLive();
+        nei.preempted = entry.getPreempted();
+        nei.priority = entry.getPriority();
+        nei.requested = entry.getRequested();
+        nei.releasing = entry.getReleasing();
+        nei.startFailed = entry.getStartFailed();
+        nei.starting = entry.getStarting();
+        info.entries.put(entry.getName(), nei);
+      }
+    }
+    return info;
+  }
+
+  public static ContainerInformation unmarshall(Messages.ContainerInformationProto wire) {
+    ContainerInformation info = new ContainerInformation();
+    info.containerId = wire.getContainerId();
+    info.component = wire.getComponent();
+    info.appVersion = wire.getAppVersion();
+    info.state = wire.getState();
+    if (wire.hasReleased()) {
+      info.released = wire.getReleased();
+    }
+    if (wire.hasExitCode()) {
+      info.exitCode = wire.getExitCode();
+    }
+    if (wire.hasDiagnostics()) {
+      info.diagnostics = wire.getDiagnostics();
+    }
+    if (wire.hasHost()) {
+      info.host = wire.getHost();
+    }
+    if (wire.hasHostURL()) {
+      info.host = wire.getHostURL();
+    }
+    info.createTime = wire.getCreateTime();
+    info.startTime = wire.getStartTime();
+    info.output = wire.getOutputList().toArray(
+        new String[wire.getOutputCount()]
+        );
+    if (wire.hasPlacement()) {
+      info.placement = wire.getPlacement();
+    }
+    return info;
+  }
+
+  public static List<ContainerInformation> unmarshall(Messages.GetLiveContainersResponseProto wire) {
+    List<ContainerInformation> infoList = new ArrayList<>(wire.getContainersList().size());
+    for (Messages.ContainerInformationProto container : wire.getContainersList()) {
+      infoList.add(unmarshall(container));
+    }
+    return infoList;
+  }
+
+  public static Messages.ContainerInformationProto marshall(ContainerInformation info) {
+
+    Messages.ContainerInformationProto.Builder builder =
+        Messages.ContainerInformationProto.newBuilder();
+    if (info.containerId != null) {
+      builder.setContainerId(info.containerId);
+    }
+    if (info.component != null) {
+      builder.setComponent(info.component);
+    }
+    if (info.appVersion != null) {
+      builder.setAppVersion(info.appVersion);
+    }
+    builder.setCreateTime(info.createTime);
+    if (info.diagnostics != null) {
+      builder.setDiagnostics(info.diagnostics);
+    }
+    if (info.host != null) {
+      builder.setHost(info.host);
+    }
+    if (info.hostURL != null) {
+      builder.setHostURL(info.hostURL);
+    }
+    if (info.output != null) {
+      builder.addAllOutput(Arrays.asList(info.output));
+    }
+    if (info.released != null) {
+      builder.setReleased(info.released);
+    }
+    if (info.placement != null) {
+      builder.setPlacement(info.placement);
+    }
+    builder.setStartTime(info.startTime);
+    builder.setState(info.state);
+    return builder.build();
+  }
+
+  public static String unmarshall(Messages.WrappedJsonProto wire) {
+    return wire.getJson();
+  }
+
+  public static ConfTree unmarshallToConfTree(Messages.WrappedJsonProto wire) throws
+      IOException {
+    return new ConfTreeSerDeser().fromJson(wire.getJson());
+  }
+  
+  public static ConfTreeOperations unmarshallToCTO(Messages.WrappedJsonProto wire) throws
+      IOException {
+    return new ConfTreeOperations(new ConfTreeSerDeser().fromJson(wire.getJson()));
+  }
+
+  public static AggregateConf unmarshallToAggregateConf(Messages.WrappedJsonProto wire) throws
+      IOException {
+    return new AggregateConfSerDeser().fromJson(wire.getJson());
+  }
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[27/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/CoreFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/CoreFileSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/CoreFileSystem.java
new file mode 100644
index 0000000..aa5edf1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/CoreFileSystem.java
@@ -0,0 +1,915 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.tools;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hadoop.yarn.util.Records;
+import org.apache.slider.common.SliderExitCodes;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.SliderXmlConfKeys;
+import org.apache.slider.core.exceptions.BadClusterStateException;
+import org.apache.slider.core.exceptions.ErrorStrings;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.core.exceptions.UnknownApplicationInstanceException;
+import org.apache.slider.core.persist.Filenames;
+import org.apache.slider.core.persist.InstancePaths;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Enumeration;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipFile;
+
+import static org.apache.slider.common.SliderXmlConfKeys.CLUSTER_DIRECTORY_PERMISSIONS;
+import static org.apache.slider.common.SliderXmlConfKeys.DEFAULT_CLUSTER_DIRECTORY_PERMISSIONS;
+
+public class CoreFileSystem {
+  private static final Logger
+    log = LoggerFactory.getLogger(CoreFileSystem.class);
+
+  private static final String UTF_8 = "UTF-8";
+
+  protected final FileSystem fileSystem;
+  protected final Configuration configuration;
+
+  public CoreFileSystem(FileSystem fileSystem, Configuration configuration) {
+    Preconditions.checkNotNull(fileSystem,
+                               "Cannot create a CoreFileSystem with a null FileSystem");
+    Preconditions.checkNotNull(configuration,
+                               "Cannot create a CoreFileSystem with a null Configuration");
+    this.fileSystem = fileSystem;
+    this.configuration = configuration;
+  }
+
+  public CoreFileSystem(Configuration configuration) throws IOException {
+    Preconditions.checkNotNull(configuration,
+                               "Cannot create a CoreFileSystem with a null Configuration");
+    this.fileSystem = FileSystem.get(configuration);
+    this.configuration = fileSystem.getConf();
+  }
+  
+  /**
+   * Get the temp path for this cluster
+   * @param clustername name of the cluster
+   * @return path for temp files (is not purged)
+   */
+  public Path getTempPathForCluster(String clustername) {
+    Path clusterDir = buildClusterDirPath(clustername);
+    return new Path(clusterDir, SliderKeys.TMP_DIR_PREFIX);
+  }
+
+  /**
+   * Returns the underlying FileSystem for this object.
+   *
+   * @return filesystem
+   */
+  public FileSystem getFileSystem() {
+    return fileSystem;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+      new StringBuilder("CoreFileSystem{");
+    sb.append("fileSystem=").append(fileSystem.getUri());
+    sb.append('}');
+    return sb.toString();
+  }
+
+  /**
+   * Build up the path string for a cluster instance -no attempt to
+   * create the directory is made
+   *
+   * @param clustername name of the cluster
+   * @return the path for persistent data
+   */
+  public Path buildClusterDirPath(String clustername) {
+    Preconditions.checkNotNull(clustername);
+    Path path = getBaseApplicationPath();
+    return new Path(path, SliderKeys.CLUSTER_DIRECTORY + "/" + clustername);
+  }
+
+  /**
+   * Build up the path string for app def folder -no attempt to
+   * create the directory is made
+   *
+   * @param clustername name of the cluster
+   * @return the path for persistent data
+   */
+  public Path buildAppDefDirPath(String clustername) {
+    Path path = buildClusterDirPath(clustername);
+    return new Path(path, SliderKeys.APP_DEF_DIR);
+  }
+
+  /**
+   * Build up the path string for addon folder -no attempt to
+   * create the directory is made
+   *
+   * @param clustername name of the cluster
+   * @return the path for persistent data
+   */
+  public Path buildAddonDirPath(String clustername, String addonId) {
+    Preconditions.checkNotNull(addonId);
+    Path path = buildClusterDirPath(clustername);
+    return new Path(path, SliderKeys.ADDONS_DIR + "/" + addonId);
+  }
+
+  /**
+   * Build up the path string for package install location -no attempt to
+   * create the directory is made
+   *
+   * @return the path for persistent app package
+   */
+  public Path buildPackageDirPath(String packageName, String packageVersion) {
+    Preconditions.checkNotNull(packageName);
+    Path path = getBaseApplicationPath();
+    path = new Path(path, SliderKeys.PACKAGE_DIRECTORY + "/" + packageName);
+    if (SliderUtils.isSet(packageVersion)) {
+      path = new Path(path, packageVersion);
+    }
+    return path;
+  }
+
+  /**
+   * Build up the path string for package install location -no attempt to
+   * create the directory is made
+   *
+   * @return the path for persistent app package
+   */
+  public Path buildClusterSecurityDirPath(String clusterName) {
+    Preconditions.checkNotNull(clusterName);
+    Path path = buildClusterDirPath(clusterName);
+    return new Path(path, SliderKeys.SECURITY_DIR);
+  }
+
+  /**
+   * Build up the path string for keytab install location -no attempt to
+   * create the directory is made
+   *
+   * @return the path for keytab
+   */
+  public Path buildKeytabInstallationDirPath(String keytabFolder) {
+    Preconditions.checkNotNull(keytabFolder);
+    Path path = getBaseApplicationPath();
+    return new Path(path, SliderKeys.KEYTAB_DIR + "/" + keytabFolder);
+  }
+
+  /**
+   * Build up the path string for keytab install location -no attempt to
+   * create the directory is made
+   *
+   * @return the path for keytab installation location
+   */
+  public Path buildKeytabPath(String keytabDir, String keytabName, String clusterName) {
+    Path homePath = getHomeDirectory();
+    Path baseKeytabDir;
+    if (keytabDir != null) {
+      baseKeytabDir = new Path(homePath, keytabDir);
+    } else {
+      baseKeytabDir = new Path(buildClusterDirPath(clusterName),
+                               SliderKeys.KEYTAB_DIR);
+    }
+    return keytabName == null ? baseKeytabDir :
+        new Path(baseKeytabDir, keytabName);
+  }
+
+  /**
+   * Build up the path string for resource install location -no attempt to
+   * create the directory is made
+   *
+   * @return the path for resource
+   */
+  public Path buildResourcePath(String resourceFolder) {
+    Preconditions.checkNotNull(resourceFolder);
+    Path path = getBaseApplicationPath();
+    return new Path(path, SliderKeys.RESOURCE_DIR + "/" + resourceFolder);
+  }
+
+  /**
+   * Build up the path string for resource install location -no attempt to
+   * create the directory is made
+   *
+   * @return the path for resource
+   */
+  public Path buildResourcePath(String dirName, String fileName) {
+    Preconditions.checkNotNull(dirName);
+    Preconditions.checkNotNull(fileName);
+    Path path = getBaseApplicationPath();
+    return new Path(path, SliderKeys.RESOURCE_DIR + "/" + dirName + "/" + fileName);
+  }
+
+  /**
+   * Build up the path string for cluster resource install location -no
+   * attempt to create the directory is made
+   *
+   * @return the path for resource
+   */
+  public Path buildClusterResourcePath(String clusterName, String component) {
+    Preconditions.checkNotNull(clusterName);
+    Path path = buildClusterDirPath(clusterName);
+    return new Path(path, SliderKeys.RESOURCE_DIR + "/" + component);
+  }
+
+  /**
+   * Build up the path string for cluster resource install location -no
+   * attempt to create the directory is made
+   *
+   * @return the path for resource
+   */
+  public Path buildClusterResourcePath(String clusterName) {
+    Preconditions.checkNotNull(clusterName);
+    Path path = buildClusterDirPath(clusterName);
+    return new Path(path, SliderKeys.RESOURCE_DIR);
+  }
+
+  /**
+   * Create the Slider cluster path for a named cluster and all its subdirs
+   * This is a directory; a mkdirs() operation is executed
+   * to ensure that it is there.
+   *
+   * @param clustername name of the cluster
+   * @return the path to the cluster directory
+   * @throws java.io.IOException                      trouble
+   * @throws SliderException slider-specific exceptions
+   */
+  public Path createClusterDirectories(String clustername, Configuration conf)
+      throws IOException, SliderException {
+
+
+    Path clusterDirectory = buildClusterDirPath(clustername);
+    InstancePaths instancePaths = new InstancePaths(clusterDirectory);
+    createClusterDirectories(instancePaths);
+    return clusterDirectory;
+  }
+  
+  /**
+   * Create the Slider cluster path for a named cluster and all its subdirs
+   * This is a directory; a mkdirs() operation is executed
+   * to ensure that it is there.
+   *
+   * @param instancePaths instance paths
+   * @throws IOException trouble
+   * @throws SliderException slider-specific exceptions
+   */
+  public void createClusterDirectories(InstancePaths instancePaths) throws
+      IOException, SliderException {
+    Path instanceDir = instancePaths.instanceDir;
+
+    verifyDirectoryNonexistent(instanceDir);
+    FsPermission clusterPerms = getInstanceDirectoryPermissions();
+    createWithPermissions(instanceDir, clusterPerms);
+    createWithPermissions(instancePaths.snapshotConfPath, clusterPerms);
+    createWithPermissions(instancePaths.generatedConfPath, clusterPerms);
+    createWithPermissions(instancePaths.historyPath, clusterPerms);
+    createWithPermissions(instancePaths.tmpPathAM, clusterPerms);
+
+    // Data Directory
+    String dataOpts =
+      configuration.get(SliderXmlConfKeys.DATA_DIRECTORY_PERMISSIONS,
+               SliderXmlConfKeys.DEFAULT_DATA_DIRECTORY_PERMISSIONS);
+    log.debug("Setting data directory permissions to {}", dataOpts);
+    createWithPermissions(instancePaths.dataPath, new FsPermission(dataOpts));
+
+  }
+
+  /**
+   * Create a directory with the given permissions.
+   *
+   * @param dir          directory
+   * @param clusterPerms cluster permissions
+   * @throws IOException  IO problem
+   * @throws BadClusterStateException any cluster state problem
+   */
+  public void createWithPermissions(Path dir, FsPermission clusterPerms) throws
+          IOException,
+          BadClusterStateException {
+    if (fileSystem.isFile(dir)) {
+      // HADOOP-9361 shows some filesystems don't correctly fail here
+      throw new BadClusterStateException(
+              "Cannot create a directory over a file %s", dir);
+    }
+    log.debug("mkdir {} with perms {}", dir, clusterPerms);
+    //no mask whatoever
+    fileSystem.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "000");
+    fileSystem.mkdirs(dir, clusterPerms);
+    //and force set it anyway just to make sure
+    fileSystem.setPermission(dir, clusterPerms);
+  }
+
+  /**
+   * Get the permissions of a path
+   *
+   * @param path path to check
+   * @return the permissions
+   * @throws IOException any IO problem (including file not found)
+   */
+  public FsPermission getPathPermissions(Path path) throws IOException {
+    FileStatus status = fileSystem.getFileStatus(path);
+    return status.getPermission();
+  }
+
+  public FsPermission getInstanceDirectoryPermissions() {
+    String clusterDirPermsOct =
+      configuration.get(CLUSTER_DIRECTORY_PERMISSIONS,
+                        DEFAULT_CLUSTER_DIRECTORY_PERMISSIONS);
+    return new FsPermission(clusterDirPermsOct);
+  }
+
+  /**
+   * Verify that the cluster directory is not present
+   *
+   * @param clustername      name of the cluster
+   * @param clusterDirectory actual directory to look for
+   * @throws IOException trouble with FS
+   * @throws SliderException If the directory exists
+   */
+  public void verifyClusterDirectoryNonexistent(String clustername,
+                                                Path clusterDirectory)
+      throws IOException, SliderException {
+    if (fileSystem.exists(clusterDirectory)) {
+      throw new SliderException(SliderExitCodes.EXIT_INSTANCE_EXISTS,
+              ErrorStrings.PRINTF_E_INSTANCE_ALREADY_EXISTS, clustername,
+              clusterDirectory);
+    }
+  }
+  /**
+   * Verify that the given directory is not present
+   *
+   * @param clusterDirectory actual directory to look for
+   * @throws IOException    trouble with FS
+   * @throws SliderException If the directory exists
+   */
+  public void verifyDirectoryNonexistent(Path clusterDirectory) throws
+          IOException,
+      SliderException {
+    if (fileSystem.exists(clusterDirectory)) {
+      
+      log.error("Dir {} exists: {}",
+                clusterDirectory,
+                listFSDir(clusterDirectory));
+      throw new SliderException(SliderExitCodes.EXIT_INSTANCE_EXISTS,
+              ErrorStrings.PRINTF_E_INSTANCE_DIR_ALREADY_EXISTS,
+              clusterDirectory);
+    }
+  }
+
+  /**
+   * Verify that a user has write access to a directory.
+   * It does this by creating then deleting a temp file
+   *
+   * @param dirPath actual directory to look for
+   * @throws FileNotFoundException file not found
+   * @throws IOException  trouble with FS
+   * @throws BadClusterStateException if the directory is not writeable
+   */
+  public void verifyDirectoryWriteAccess(Path dirPath) throws IOException,
+      SliderException {
+    verifyPathExists(dirPath);
+    Path tempFile = new Path(dirPath, "tmp-file-for-checks");
+    try {
+      FSDataOutputStream out ;
+      out = fileSystem.create(tempFile, true);
+      IOUtils.closeStream(out);
+      fileSystem.delete(tempFile, false);
+    } catch (IOException e) {
+      log.warn("Failed to create file {}: {}", tempFile, e);
+      throw new BadClusterStateException(e,
+              "Unable to write to directory %s : %s", dirPath, e.toString());
+    }
+  }
+
+  /**
+   * Verify that a path exists
+   * @param path path to check
+   * @throws FileNotFoundException file not found
+   * @throws IOException  trouble with FS
+   */
+  public void verifyPathExists(Path path) throws IOException {
+    if (!fileSystem.exists(path)) {
+      throw new FileNotFoundException(path.toString());
+    }
+  }
+
+  /**
+   * Verify that a path exists
+   * @param path path to check
+   * @throws FileNotFoundException file not found or is not a file
+   * @throws IOException  trouble with FS
+   */
+  public void verifyFileExists(Path path) throws IOException {
+    FileStatus status = fileSystem.getFileStatus(path);
+
+    if (!status.isFile()) {
+      throw new FileNotFoundException("Not a file: " + path.toString());
+    }
+  }
+
+  /**
+   * Given a path, check if it exists and is a file
+   * 
+   * @param path
+   *          absolute path to the file to check
+   * @returns true if and only if path exists and is a file, false for all other
+   *          reasons including if file check throws IOException
+   */
+  public boolean isFile(Path path) {
+    boolean isFile = false;
+    try {
+      FileStatus status = fileSystem.getFileStatus(path);
+      if (status.isFile()) {
+        isFile = true;
+      }
+    } catch (IOException e) {
+      // ignore, isFile is already set to false
+    }
+    return isFile;
+  }
+
+  /**
+   * Verify that a file exists in the zip file given by path
+   * @param path path to zip file
+   * @param file file expected to be in zip
+   * @throws FileNotFoundException file not found or is not a zip file
+   * @throws IOException  trouble with FS
+   */
+  public void verifyFileExistsInZip(Path path, String file) throws IOException {
+    fileSystem.copyToLocalFile(path, new Path("/tmp"));
+    File dst = new File((new Path("/tmp", path.getName())).toString());
+    Enumeration<? extends ZipEntry> entries;
+    ZipFile zipFile = new ZipFile(dst);
+    boolean found = false;
+
+    try {
+      entries = zipFile.entries();
+      while (entries.hasMoreElements()) {
+        ZipEntry entry = entries.nextElement();
+        String nm = entry.getName();
+        if (nm.endsWith(file)) {
+          found = true;
+          break;
+        }
+      }
+    } finally {
+      zipFile.close();
+    }
+    dst.delete();
+    if (!found) throw new FileNotFoundException("file: " + file + " not found in " + path);
+    log.info("Verification of " + path + " passed");
+  }
+  /**
+   * Create the application-instance specific temporary directory
+   * in the DFS
+   *
+   * @param clustername name of the cluster
+   * @param subdir       application ID
+   * @return the path; this directory will already have been created
+   */
+  public Path createAppInstanceTempPath(String clustername, String subdir)
+      throws IOException {
+    Path tmp = getTempPathForCluster(clustername);
+    Path instancePath = new Path(tmp, subdir);
+    fileSystem.mkdirs(instancePath);
+    return instancePath;
+  }
+
+  /**
+   * Create the application-instance specific temporary directory
+   * in the DFS
+   *
+   * @param clustername name of the cluster
+   * @return the path; this directory will already have been deleted
+   */
+  public Path purgeAppInstanceTempFiles(String clustername) throws
+          IOException {
+    Path tmp = getTempPathForCluster(clustername);
+    fileSystem.delete(tmp, true);
+    return tmp;
+  }
+
+  /**
+   * Get the base path
+   *
+   * @return the base path optionally configured by 
+   * {@link SliderXmlConfKeys#KEY_SLIDER_BASE_PATH}
+   */
+  public Path getBaseApplicationPath() {
+    String configuredBasePath = configuration.get(SliderXmlConfKeys.KEY_SLIDER_BASE_PATH);
+    return configuredBasePath != null ? new Path(configuredBasePath) :
+           new Path(getHomeDirectory(), SliderKeys.SLIDER_BASE_DIRECTORY);
+  }
+
+  /**
+   * Get slider dependency parent dir in HDFS
+   * 
+   * @return the parent dir path of slider.tar.gz in HDFS
+   */
+  public Path getDependencyPath() {
+    String parentDir = (SliderUtils.isHdp()) ? SliderKeys.SLIDER_DEPENDENCY_HDP_PARENT_DIR
+        + SliderKeys.SLIDER_DEPENDENCY_DIR
+        : SliderKeys.SLIDER_DEPENDENCY_DIR;
+    Path dependencyPath = new Path(String.format(parentDir,
+        SliderUtils.getSliderVersion()));
+    return dependencyPath;
+  }
+
+  /**
+   * Get slider.tar.gz absolute filepath in HDFS
+   * 
+   * @return the absolute path to slider.tar.gz in HDFS
+   */
+  public Path getDependencyTarGzip() {
+    Path dependencyLibAmPath = getDependencyPath();
+    Path dependencyLibTarGzip = new Path(
+        dependencyLibAmPath.toUri().toString(),
+        SliderKeys.SLIDER_DEPENDENCY_TAR_GZ_FILE_NAME
+            + SliderKeys.SLIDER_DEPENDENCY_TAR_GZ_FILE_EXT);
+    return dependencyLibTarGzip;
+  }
+
+  public Path getHomeDirectory() {
+    return fileSystem.getHomeDirectory();
+  }
+
+  public boolean maybeAddImagePath(Map<String, LocalResource> localResources,
+                                   Path imagePath) throws IOException {
+    if (imagePath != null) {
+      LocalResource resource = createAmResource(imagePath,
+          LocalResourceType.ARCHIVE);
+      localResources.put(SliderKeys.LOCAL_TARBALL_INSTALL_SUBDIR, resource);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  public boolean maybeAddImagePath(Map<String, LocalResource> localResources,
+                                   String imagePath) throws IOException {
+    
+    return imagePath != null &&
+           maybeAddImagePath(localResources, new Path(imagePath));
+  }
+  
+  
+  
+
+  /**
+   * Create an AM resource from the
+   *
+   * @param destPath     dest path in filesystem
+   * @param resourceType resource type
+   * @return the resource set up wih application-level visibility and the
+   * timestamp & size set from the file stats.
+   */
+  public LocalResource createAmResource(Path destPath, LocalResourceType resourceType) throws IOException {
+    FileStatus destStatus = fileSystem.getFileStatus(destPath);
+    LocalResource amResource = Records.newRecord(LocalResource.class);
+    amResource.setType(resourceType);
+    // Set visibility of the resource
+    // Setting to most private option
+    amResource.setVisibility(LocalResourceVisibility.APPLICATION);
+    // Set the resource to be copied over
+    amResource.setResource(ConverterUtils.getYarnUrlFromPath(fileSystem
+        .resolvePath(destStatus.getPath())));
+    // Set timestamp and length of file so that the framework
+    // can do basic sanity checks for the local resource
+    // after it has been copied over to ensure it is the same
+    // resource the client intended to use with the application
+    amResource.setTimestamp(destStatus.getModificationTime());
+    amResource.setSize(destStatus.getLen());
+    return amResource;
+  }
+
+  /**
+   * Register all files under a fs path as a directory to push out
+   *
+   * @param srcDir          src dir
+   * @param destRelativeDir dest dir (no trailing /)
+   * @return the map of entries
+   */
+  public Map<String, LocalResource> submitDirectory(Path srcDir, String destRelativeDir) throws IOException {
+    //now register each of the files in the directory to be
+    //copied to the destination
+    FileStatus[] fileset = fileSystem.listStatus(srcDir);
+    Map<String, LocalResource> localResources =
+            new HashMap<String, LocalResource>(fileset.length);
+    for (FileStatus entry : fileset) {
+
+      LocalResource resource = createAmResource(entry.getPath(),
+              LocalResourceType.FILE);
+      String relativePath = destRelativeDir + "/" + entry.getPath().getName();
+      localResources.put(relativePath, resource);
+    }
+    return localResources;
+  }
+
+  /**
+   * Submit a JAR containing a specific class, returning
+   * the resource to be mapped in
+   *
+   * @param clazz   class to look for
+   * @param subdir  subdirectory (expected to end in a "/")
+   * @param jarName <i>At the destination</i>
+   * @return the local resource ref
+   * @throws IOException trouble copying to HDFS
+   */
+  public LocalResource submitJarWithClass(Class clazz, Path tempPath, String subdir, String jarName)
+          throws IOException, SliderException {
+    File localFile = SliderUtils.findContainingJarOrFail(clazz);
+    return submitFile(localFile, tempPath, subdir, jarName);
+  }
+
+  /**
+   * Submit a local file to the filesystem references by the instance's cluster
+   * filesystem
+   *
+   * @param localFile    filename
+   * @param subdir       subdirectory (expected to end in a "/")
+   * @param destFileName destination filename
+   * @return the local resource ref
+   * @throws IOException trouble copying to HDFS
+   */
+  public LocalResource submitFile(File localFile, Path tempPath, String subdir, String destFileName)
+      throws IOException {
+    Path src = new Path(localFile.toString());
+    Path subdirPath = new Path(tempPath, subdir);
+    fileSystem.mkdirs(subdirPath);
+    Path destPath = new Path(subdirPath, destFileName);
+    log.debug("Copying {} (size={} bytes) to {}", localFile, localFile.length(), destPath);
+
+    fileSystem.copyFromLocalFile(false, true, src, destPath);
+
+    // Set the type of resource - file or archive
+    // archives are untarred at destination
+    // we don't need the jar file to be untarred for now
+    return createAmResource(destPath, LocalResourceType.FILE);
+  }
+
+  /**
+   * Submit the AM tar.gz resource referenced by the instance's cluster
+   * filesystem. Also, update the providerResources object with the new
+   * resource.
+   * 
+   * @param providerResources
+   *          the provider resource map to be updated
+   * @throws IOException
+   *           trouble copying to HDFS
+   */
+  public void submitTarGzipAndUpdate(
+      Map<String, LocalResource> providerResources) throws IOException,
+      BadClusterStateException {
+    Path dependencyLibTarGzip = getDependencyTarGzip();
+    LocalResource lc = createAmResource(dependencyLibTarGzip,
+        LocalResourceType.ARCHIVE);
+    providerResources.put(SliderKeys.SLIDER_DEPENDENCY_LOCALIZED_DIR_LINK, lc);
+  }
+
+  /**
+   * Copy local file(s) to destination HDFS directory. If {@code localPath} is a
+   * local directory then all files matching the {@code filenameFilter}
+   * (optional) are copied, otherwise {@code filenameFilter} is ignored.
+   * 
+   * @param localPath
+   *          a local file or directory path
+   * @param filenameFilter
+   *          if {@code localPath} is a directory then filenameFilter is used as
+   *          a filter (if specified)
+   * @param destDir
+   *          the destination HDFS directory where the file(s) should be copied
+   * @param fp
+   *          file permissions of all the directories and files that will be
+   *          created in this api
+   * @throws IOException
+   */
+  public void copyLocalFilesToHdfs(File localPath,
+      FilenameFilter filenameFilter, Path destDir, FsPermission fp)
+      throws IOException {
+    if (localPath == null || destDir == null) {
+      throw new IOException("Either localPath or destDir is null");
+    }
+    fileSystem.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY,
+        "000");
+    fileSystem.mkdirs(destDir, fp);
+    if (localPath.isDirectory()) {
+      // copy all local files under localPath to destDir (honoring filename
+      // filter if provided
+      File[] localFiles = localPath.listFiles(filenameFilter);
+      Path[] localFilePaths = new Path[localFiles.length];
+      int i = 0;
+      for (File localFile : localFiles) {
+        localFilePaths[i++] = new Path(localFile.getPath());
+      }
+      log.info("Copying {} files from {} to {}", i, localPath.toURI(),
+          destDir.toUri());
+      fileSystem.copyFromLocalFile(false, true, localFilePaths, destDir);
+    } else {
+      log.info("Copying file {} to {}", localPath.toURI(), destDir.toUri());
+      fileSystem.copyFromLocalFile(false, true, new Path(localPath.getPath()),
+          destDir);
+    }
+    // set permissions for all the files created in the destDir
+    fileSystem.setPermission(destDir, fp);
+  }
+
+  public void copyLocalFileToHdfs(File localPath,
+      Path destPath, FsPermission fp)
+      throws IOException {
+    if (localPath == null || destPath == null) {
+      throw new IOException("Either localPath or destPath is null");
+    }
+    fileSystem.getConf().set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY,
+        "000");
+    fileSystem.mkdirs(destPath.getParent(), fp);
+    log.info("Copying file {} to {}", localPath.toURI(), destPath.toUri());
+    
+    fileSystem.copyFromLocalFile(false, true, new Path(localPath.getPath()),
+        destPath);
+    // set file permissions of the destPath
+    fileSystem.setPermission(destPath, fp);
+  }
+
+  public void copyHdfsFileToLocal(Path hdfsPath, File destFile)
+      throws IOException {
+    if (hdfsPath == null || destFile == null) {
+      throw new IOException("Either hdfsPath or destPath is null");
+    }
+    log.info("Copying file {} to {}", hdfsPath.toUri(), destFile.toURI());
+
+    Path destPath = new Path(destFile.getPath());
+    fileSystem.copyToLocalFile(hdfsPath, destPath);
+  }
+
+  /**
+   * list entries in a filesystem directory
+   *
+   * @param path directory
+   * @return a listing, one to a line
+   * @throws IOException
+   */
+  public String listFSDir(Path path) throws IOException {
+    FileStatus[] stats = fileSystem.listStatus(path);
+    StringBuilder builder = new StringBuilder();
+    for (FileStatus stat : stats) {
+      builder.append(stat.getPath().toString())
+              .append("\t")
+              .append(stat.getLen())
+              .append("\n");
+    }
+    return builder.toString();
+  }
+
+  /**
+   * List all application instances persisted for this user, giving the 
+   * path. The instance name is the last element in the path
+   * @return a possibly empty map of application instance names to paths
+   */
+  public Map<String, Path> listPersistentInstances() throws IOException {
+    FileSystem fs = getFileSystem();
+    Path path = new Path(getBaseApplicationPath(), SliderKeys.CLUSTER_DIRECTORY);
+    log.debug("Looking for all persisted application at {}", path.toString());
+    if (!fs.exists(path)) {
+      // special case: no instances have ever been created
+      return new HashMap<String, Path>(0);
+    }
+    FileStatus[] statuses = fs.listStatus(path);
+    Map<String, Path> instances = new HashMap<String, Path>(statuses.length);
+
+    // enum the child entries
+    for (FileStatus status : statuses) {
+      if (status.isDirectory()) {
+        // for directories, look for an internal.json underneath
+        Path child = status.getPath();
+        Path internalJson = new Path(child, Filenames.INTERNAL);
+        if (fs.exists(internalJson)) {
+          // success => this is an instance
+          instances.put(child.getName(), child);
+        } else {
+          log.info("Malformed cluster found at {}. It does not appear to be a valid persisted instance.",
+                   child.toString());
+        }
+      }
+    }
+    return instances;
+  }
+
+  public void touch(Path path, boolean overwrite) throws IOException {
+    FSDataOutputStream out = null;
+    try {
+      out = fileSystem.create(path, overwrite);
+    } finally {
+      IOUtils.closeStream(out);
+    }
+  }
+
+  public void cat(Path path, boolean overwrite, String data) throws IOException {
+    FSDataOutputStream out = null;
+    try {
+      out = fileSystem.create(path, overwrite);
+      byte[] bytes = data.getBytes(Charset.forName("UTF-8"));
+      out.write(bytes);
+    } finally {
+      IOUtils.closeStream(out);
+    }
+  }
+
+  public String cat(Path path) throws IOException {
+    FileStatus status = fileSystem.getFileStatus(path);
+    byte[] b = new byte[(int) status.getLen()];
+    FSDataInputStream in = null;
+    try {
+      in = fileSystem.open(path);
+      int count = in.read(b);
+      return new String(b, 0, count, UTF_8);
+    } finally {
+      IOUtils.closeStream(in);
+    }
+  }
+
+  /**
+   * Create a path that must exist in the cluster fs
+   * @param uri uri to create
+   * @return the path
+   * @throws SliderException if the path does not exist
+   */
+  public Path createPathThatMustExist(String uri) throws
+      SliderException, IOException {
+    Preconditions.checkNotNull(uri);
+    Path path = new Path(uri);
+    verifyPathExists(path);
+    return path;
+  }
+
+  /**
+   * Locate an application conf json in the FS. This includes a check to verify
+   * that the file is there.
+   *
+   * @param clustername name of the cluster
+   * @return the path to the spec.
+   * @throws IOException IO problems
+   * @throws SliderException if the path isn't there
+   */
+  public Path locateInstanceDefinition(String clustername) throws IOException,
+      SliderException {
+    Path clusterDirectory = buildClusterDirPath(clustername);
+    Path appConfPath =
+            new Path(clusterDirectory, Filenames.APPCONF);
+    verifyClusterSpecExists(clustername, appConfPath);
+    return appConfPath;
+  }
+
+  /**
+   * Verify that a cluster specification exists
+   * @param clustername name of the cluster (For errors only)
+   * @param clusterSpecPath cluster specification path
+   * @throws IOException IO problems
+   * @throws SliderException if the cluster specification is not present
+   */
+  public void verifyClusterSpecExists(String clustername, Path clusterSpecPath)
+      throws IOException,
+      SliderException {
+    if (!fileSystem.isFile(clusterSpecPath)) {
+      log.debug("Missing specification file {}", clusterSpecPath);
+      throw UnknownApplicationInstanceException.unknownInstance(
+          clustername + "\n (definition not found at " + clusterSpecPath);
+    }
+  }
+
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/Duration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/Duration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/Duration.java
new file mode 100644
index 0000000..e5fa424
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/Duration.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.common.tools;
+
+import java.io.Closeable;
+
+/**
+ * A duration in milliseconds. This class can be used
+ * to count time, and to be polled to see if a time limit has
+ * passed.
+ */
+public class Duration implements Closeable {
+  public long start, finish;
+  public final long limit;
+
+  /**
+   * Create a duration instance with a limit of 0
+   */
+  public Duration() {
+    this(0);
+  }
+
+  /**
+   * Create a duration with a limit specified in millis
+   * @param limit duration in milliseconds
+   */
+  public Duration(long limit) {
+    this.limit = limit;
+  }
+
+  /**
+   * Start
+   * @return self
+   */
+  public Duration start() {
+    start = now();
+    return this;
+  }
+
+  /**
+   * The close operation relays to {@link #finish()}.
+   * Implementing it allows Duration instances to be automatically
+   * finish()'d in Java7 try blocks for when used in measuring durations.
+   */
+  @Override
+  public final void close() {
+    finish();
+  }
+
+  public void finish() {
+    finish = now();
+  }
+
+  protected long now() {
+    return System.nanoTime()/1000000;
+  }
+
+  public long getInterval() {
+    return finish - start;
+  }
+
+  /**
+   * return true if the limit has been exceeded
+   * @return true if a limit was set and the current time
+   * exceeds it.
+   */
+  public boolean getLimitExceeded() {
+    return limit >= 0 && ((now() - start) > limit);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append("Duration");
+     if (finish >= start) {
+       builder.append(" finished at ").append(getInterval()).append(" millis;");
+     } else {
+       if (start > 0) {
+         builder.append(" started but not yet finished;");
+       } else {
+         builder.append(" unstarted;");
+       }
+     }
+    if (limit > 0) {
+      builder.append(" limit: ").append(limit).append(" millis");
+      if (getLimitExceeded()) {
+        builder.append(" -  exceeded");
+      }
+    }
+    return  builder.toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/PortScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/PortScanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/PortScanner.java
new file mode 100644
index 0000000..64783b6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/PortScanner.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.common.tools;
+
+import org.apache.slider.common.SliderExitCodes;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.core.exceptions.SliderException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * a scanner which can take an input string for a range or scan the lot.
+ */
+public class PortScanner {
+  private static Pattern NUMBER_RANGE = Pattern.compile("^(\\d+)\\s*-\\s*(\\d+)$");
+  private static Pattern SINGLE_NUMBER = Pattern.compile("^\\d+$");
+
+  private List<Integer> remainingPortsToCheck;
+
+  public PortScanner() {
+  }
+
+  public void setPortRange(String input) throws BadConfigException {
+    // first split based on commas
+    Set<Integer> inputPorts= new TreeSet<Integer>();
+    String[] ranges = input.split(",");
+    for ( String range : ranges ) {
+      if (range.trim().isEmpty()) {
+        continue;
+      }
+      Matcher m = SINGLE_NUMBER.matcher(range.trim());
+      if (m.find()) {
+        inputPorts.add(Integer.parseInt(m.group()));
+        continue;
+      }
+      m = NUMBER_RANGE.matcher(range.trim());
+      if (m.find()) {
+        String[] boundaryValues = m.group(0).split("-");
+        int start = Integer.parseInt(boundaryValues[0].trim());
+        int end = Integer.parseInt(boundaryValues[1].trim());
+        if (end < start) {
+          throw new BadConfigException("End of port range is before start: "
+              + range + " in input: " + input);
+        }
+        for (int i = start; i < end + 1; i++) {
+          inputPorts.add(i);
+        }
+        continue;
+      }
+      throw new BadConfigException("Bad port range: " + range + " in input: "
+          + input);
+    }
+    if (inputPorts.size() == 0) {
+      throw new BadConfigException("No ports found in range: " + input);
+    }
+    this.remainingPortsToCheck = new ArrayList<Integer>(inputPorts);
+  }
+
+  public List<Integer> getRemainingPortsToCheck() {
+    return remainingPortsToCheck;
+  }
+
+  public int getAvailablePort() throws SliderException, IOException {
+    if (remainingPortsToCheck != null) {
+      return getAvailablePortViaPortArray();
+    } else {
+      return SliderUtils.getOpenPort();
+    }
+  }
+
+  private int getAvailablePortViaPortArray() throws SliderException {
+    boolean found = false;
+    int availablePort = -1;
+    Iterator<Integer> portsToCheck = this.remainingPortsToCheck.iterator();
+    while (portsToCheck.hasNext() && !found) {
+      int portToCheck = portsToCheck.next();
+      found = SliderUtils.isPortAvailable(portToCheck);
+      if (found) {
+        availablePort = portToCheck;
+        portsToCheck.remove();
+      }
+    }
+
+    if (availablePort < 0) {
+      throw new SliderException(SliderExitCodes.EXIT_BAD_CONFIGURATION,
+        "No available ports found in configured range {}",
+        remainingPortsToCheck);
+    }
+
+    return availablePort;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderFileSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderFileSystem.java
new file mode 100644
index 0000000..294f37e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderFileSystem.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.tools;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+
+import java.io.IOException;
+
+/**
+ * Extends Core Filesystem with operations to manipulate ClusterDescription
+ * persistent state
+ */
+public class SliderFileSystem extends CoreFileSystem {
+
+  public SliderFileSystem(FileSystem fileSystem,
+      Configuration configuration) {
+    super(fileSystem, configuration);
+  }
+
+  public SliderFileSystem(Configuration configuration) throws IOException {
+    super(configuration);
+  }
+
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[20/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderCore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderCore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderCore.java
new file mode 100644
index 0000000..9767430
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderCore.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTree;
+import org.apache.slider.core.exceptions.SliderException;
+
+import java.util.List;
+public interface ProviderCore {
+
+  String getName();
+
+  List<ProviderRole> getRoles();
+
+  Configuration getConf();
+
+  /**
+   * Verify that an instance definition is considered valid by the provider
+   * @param instanceDefinition instance definition
+   * @throws SliderException if the configuration is not valid
+   */
+  void validateInstanceDefinition(AggregateConf instanceDefinition) throws
+      SliderException;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderRole.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderRole.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderRole.java
new file mode 100644
index 0000000..761ac0f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderRole.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers;
+
+import org.apache.slider.api.ResourceKeys;
+
+/**
+ * Provider role and key for use in app requests.
+ * 
+ * This class uses the role name as the key for hashes and in equality tests,
+ * and ignores the other values.
+ */
+public final class ProviderRole {
+  public final String name;
+  public final String group;
+  public final int id;
+  public int placementPolicy;
+  public int nodeFailureThreshold;
+  public final long placementTimeoutSeconds;
+  public final String labelExpression;
+
+  public ProviderRole(String name, int id) {
+    this(name,
+        name,
+        id,
+        PlacementPolicy.DEFAULT,
+        ResourceKeys.DEFAULT_NODE_FAILURE_THRESHOLD,
+        ResourceKeys.DEFAULT_PLACEMENT_ESCALATE_DELAY_SECONDS,
+        ResourceKeys.DEF_YARN_LABEL_EXPRESSION);
+  }
+
+  /**
+   * Create a provider role
+   * @param name role/component name
+   * @param id ID. This becomes the YARN priority
+   * @param policy placement policy
+   * @param nodeFailureThreshold threshold for node failures (within a reset interval)
+   * after which a node failure is considered an app failure
+   * @param placementTimeoutSeconds for lax placement, timeout in seconds before
+   * @param labelExpression label expression for requests; may be null
+   */
+  public ProviderRole(String name,
+      int id,
+      int policy,
+      int nodeFailureThreshold,
+      long placementTimeoutSeconds,
+      String labelExpression) {
+    this(name,
+        name,
+        id,
+        policy,
+        nodeFailureThreshold,
+        placementTimeoutSeconds,
+        labelExpression);
+  }
+
+  /**
+   * Create a provider role with a role group
+   * @param name role/component name
+   * @param group role/component group
+   * @param id ID. This becomes the YARN priority
+   * @param policy placement policy
+   * @param nodeFailureThreshold threshold for node failures (within a reset interval)
+   * after which a node failure is considered an app failure
+   * @param placementTimeoutSeconds for lax placement, timeout in seconds before
+   * @param labelExpression label expression for requests; may be null
+   */
+  public ProviderRole(String name,
+      String group,
+      int id,
+      int policy,
+      int nodeFailureThreshold,
+      long placementTimeoutSeconds,
+      String labelExpression) {
+    this.name = name;
+    if (group == null) {
+      this.group = name;
+    } else {
+      this.group = group;
+    }
+    this.id = id;
+    this.placementPolicy = policy;
+    this.nodeFailureThreshold = nodeFailureThreshold;
+    this.placementTimeoutSeconds = placementTimeoutSeconds;
+    this.labelExpression = labelExpression;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    ProviderRole that = (ProviderRole) o;
+    return name.equals(that.name);
+  }
+
+  @Override
+  public int hashCode() {
+    return name.hashCode();
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder("ProviderRole{");
+    sb.append("name='").append(name).append('\'');
+    sb.append(", group=").append(group);
+    sb.append(", id=").append(id);
+    sb.append(", placementPolicy=").append(placementPolicy);
+    sb.append(", nodeFailureThreshold=").append(nodeFailureThreshold);
+    sb.append(", placementTimeoutSeconds=").append(placementTimeoutSeconds);
+    sb.append(", labelExpression='").append(labelExpression).append('\'');
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
new file mode 100644
index 0000000..f754eee
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
@@ -0,0 +1,217 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.slider.api.ClusterDescription;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.core.launch.ContainerLauncher;
+import org.apache.slider.core.main.ExitCodeProvider;
+import org.apache.slider.server.appmaster.actions.QueueAccess;
+import org.apache.slider.server.appmaster.operations.RMOperationHandlerActions;
+import org.apache.slider.server.appmaster.state.ContainerReleaseSelector;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+import org.apache.slider.server.appmaster.web.rest.agent.AgentRestOperations;
+import org.apache.slider.server.services.yarnregistry.YarnRegistryViewForProviders;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.util.List;
+import java.util.Map;
+
+public interface ProviderService extends ProviderCore,
+    Service,
+    RMOperationHandlerActions,
+    ExitCodeProvider {
+
+  /**
+   * Set up the entire container launch context
+   * @param containerLauncher
+   * @param instanceDefinition
+   * @param container
+   * @param providerRole
+   * @param sliderFileSystem
+   * @param generatedConfPath
+   * @param appComponent
+   * @param containerTmpDirPath
+   */
+  void buildContainerLaunchContext(ContainerLauncher containerLauncher,
+      AggregateConf instanceDefinition,
+      Container container,
+      ProviderRole providerRole,
+      SliderFileSystem sliderFileSystem,
+      Path generatedConfPath,
+      MapOperations resourceComponent,
+      MapOperations appComponent,
+      Path containerTmpDirPath) throws
+      IOException,
+      SliderException;
+
+  /**
+   * Notify the providers of container completion
+   * @param containerId container that has completed
+   */
+  void notifyContainerCompleted(ContainerId containerId);
+
+  /**
+   * Execute a process in the AM
+   * @param instanceDefinition cluster description
+   * @param confDir configuration directory
+   * @param env environment
+   * @param execInProgress the callback for the exec events
+   * @return true if a process was actually started
+   * @throws IOException
+   * @throws SliderException
+   */
+  boolean exec(AggregateConf instanceDefinition,
+               File confDir,
+               Map<String, String> env,
+               ProviderCompleted execInProgress) throws IOException,
+      SliderException;
+
+  /**
+   * Scan through the roles and see if it is supported.
+   * @param role role to look for
+   * @return true if the role is known about -and therefore
+   * that a launcher thread can be deployed to launch it
+   */
+  boolean isSupportedRole(String role);
+
+  /**
+   * Load a specific XML configuration file for the provider config
+   * @param confDir configuration directory
+   * @return a configuration to be included in status
+   * @throws BadCommandArgumentsException
+   * @throws IOException
+   */
+  Configuration loadProviderConfigurationInformation(File confDir)
+    throws BadCommandArgumentsException, IOException;
+
+  /**
+   * The application configuration should be initialized here
+   * 
+   * @param instanceDefinition
+   * @param fileSystem
+   * @throws IOException
+   * @throws SliderException
+   */
+  void initializeApplicationConfiguration(AggregateConf instanceDefinition,
+      SliderFileSystem fileSystem) throws IOException, SliderException;
+
+  /**
+   * This is a validation of the application configuration on the AM.
+   * Here is where things like the existence of keytabs and other
+   * not-seen-client-side properties can be tested, before
+   * the actual process is spawned. 
+   * @param instanceDefinition clusterSpecification
+   * @param confDir configuration directory
+   * @param secure flag to indicate that secure mode checks must exist
+   * @throws IOException IO problemsn
+   * @throws SliderException any failure
+   */
+  void validateApplicationConfiguration(AggregateConf instanceDefinition,
+                                        File confDir,
+                                        boolean secure
+                                       ) throws IOException, SliderException;
+
+  /*
+     * Build the provider status, can be empty
+     * @return the provider status - map of entries to add to the info section
+     */
+  Map<String, String> buildProviderStatus();
+  
+  /**
+   * Build a map of data intended for the AM webapp that is specific
+   * about this provider. The key is some text to be displayed, and the
+   * value can be a URL that will create an anchor over the key text.
+   * 
+   * If no anchor is needed/desired, insert the key with a null value.
+   * @return the details
+   */
+  Map<String, MonitorDetail> buildMonitorDetails(ClusterDescription clusterSpec);
+
+  /**
+   * Get a human friendly name for web UIs and messages
+   * @return a name string. Default is simply the service instance name.
+   */
+  String getHumanName();
+
+  public void bind(StateAccessForProviders stateAccessor,
+      QueueAccess queueAccess,
+      List<Container> liveContainers);
+
+  /**
+   * Bind to the YARN registry
+   * @param yarnRegistry YARN registry
+   */
+  void bindToYarnRegistry(YarnRegistryViewForProviders yarnRegistry);
+
+  /**
+   * Returns the agent rest operations interface.
+   * @return  the interface if available, null otherwise.
+   */
+  AgentRestOperations getAgentRestOperations();
+
+  /**
+   * Build up the endpoint details for this service
+   * @param details
+   */
+  void buildEndpointDetails(Map<String, MonitorDetail> details);
+
+  /**
+   * Prior to going live -register the initial service registry data
+   * @param amWebURI URL to the AM. This may be proxied, so use relative paths
+   * @param agentOpsURI URI for agent operations. This will not be proxied
+   * @param agentStatusURI URI For agent status. Again: no proxy
+   * @param serviceRecord service record to build up
+   */
+  void applyInitialRegistryDefinitions(URL amWebURI,
+      URL agentOpsURI,
+      URL agentStatusURI,
+      ServiceRecord serviceRecord)
+      throws IOException;
+
+  /**
+   * Create the container release selector for this provider...any policy
+   * can be implemented
+   * @return the selector to use for choosing containers.
+   */
+  ContainerReleaseSelector createContainerReleaseSelector();
+
+  /**
+   * On AM restart (for whatever reason) this API is required to rebuild the AM
+   * internal state with the containers which were already assigned and running
+   * 
+   * @param liveContainers
+   * @param applicationId
+   * @param providerRoles
+   */
+  void rebuildContainerDetails(List<Container> liveContainers,
+      String applicationId, Map<Integer, ProviderRole> providerRoles);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java
new file mode 100644
index 0000000..07d106b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java
@@ -0,0 +1,530 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.slider.api.ClusterDescription;
+import org.apache.slider.api.InternalKeys;
+import org.apache.slider.api.OptionKeys;
+import org.apache.slider.api.ResourceKeys;
+import org.apache.slider.api.RoleKeys;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.core.exceptions.SliderInternalStateException;
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+/**
+ * this is a factoring out of methods handy for providers. It's bonded to a log at
+ * construction time
+ */
+public class ProviderUtils implements RoleKeys {
+
+  protected final Logger log;
+
+  /**
+   * Create an instace
+   * @param log log directory to use -usually the provider
+   */
+  
+  public ProviderUtils(Logger log) {
+    this.log = log;
+  }
+
+  /**
+   * Add oneself to the classpath. This does not work
+   * on minicluster test runs where the JAR is not built up
+   * @param providerResources map of provider resources to add these entries to
+   * @param provider provider to add
+   * @param jarName name of the jar to use
+   * @param sliderFileSystem target filesystem
+   * @param tempPath path in the cluster FS for temp files
+   * @param libdir relative directory to place resources
+   * @param miniClusterTestRun
+   * @return true if the class was found in a JAR
+   * 
+   * @throws FileNotFoundException if the JAR was not found and this is NOT
+   * a mini cluster test run
+   * @throws IOException IO problems
+   * @throws SliderException any Slider problem
+   */
+  public static boolean addProviderJar(Map<String, LocalResource> providerResources,
+      Object provider,
+      String jarName,
+      SliderFileSystem sliderFileSystem,
+      Path tempPath,
+      String libdir,
+      boolean miniClusterTestRun) throws
+      IOException,
+      SliderException {
+    try {
+      SliderUtils.putJar(providerResources,
+          sliderFileSystem,
+          provider.getClass(),
+          tempPath,
+          libdir,
+          jarName);
+      return true;
+    } catch (FileNotFoundException e) {
+      if (miniClusterTestRun) {
+        return false;
+      } else {
+        throw e;
+      }
+    }
+  }
+
+  /**
+   * Add/overwrite the agent tarball (overwritten every time application is restarted)
+   * @param provider
+   * @param tarName
+   * @param sliderFileSystem
+   * @param agentDir
+   * @return true the location could be determined and the file added
+   * @throws IOException
+   */
+  public static boolean addAgentTar(Object provider,
+                                    String tarName,
+                                    SliderFileSystem sliderFileSystem,
+                                    Path agentDir) throws
+  IOException {
+    File localFile = SliderUtils.findContainingJar(provider.getClass());
+    if(localFile != null) {
+      String parentDir = localFile.getParent();
+      Path agentTarPath = new Path(parentDir, tarName);
+      sliderFileSystem.getFileSystem().copyFromLocalFile(false, true, agentTarPath, agentDir);
+      return true;
+    }
+    return false;
+  }
+  
+  /**
+   * Add a set of dependencies to the provider resources being built up,
+   * by copying them from the local classpath to the remote one, then
+   * registering them
+   * @param providerResources map of provider resources to add these entries to
+   * @param sliderFileSystem target filesystem
+   * @param tempPath path in the cluster FS for temp files
+   * @param libdir relative directory to place resources
+   * @param resources list of resource names (e.g. "hbase.jar"
+   * @param classes list of classes where classes[i] refers to a class in
+   * resources[i]
+   * @throws IOException IO problems
+   * @throws SliderException any Slider problem
+   */
+  public static void addDependencyJars(Map<String, LocalResource> providerResources,
+                                       SliderFileSystem sliderFileSystem,
+                                       Path tempPath,
+                                       String libdir,
+                                       String[] resources,
+                                       Class[] classes
+                                      ) throws
+                                        IOException,
+      SliderException {
+    if (resources.length != classes.length) {
+      throw new SliderInternalStateException(
+        "mismatch in Jar names [%d] and classes [%d]",
+        resources.length,
+        classes.length);
+    }
+    int size = resources.length;
+    for (int i = 0; i < size; i++) {
+      String jarName = resources[i];
+      Class clazz = classes[i];
+      SliderUtils.putJar(providerResources,
+          sliderFileSystem,
+          clazz,
+          tempPath,
+          libdir,
+          jarName);
+    }
+    
+  }
+
+  /**
+   * Loads all dependency jars from the default path
+   * @param providerResources map of provider resources to add these entries to
+   * @param sliderFileSystem target filesystem
+   * @param tempPath path in the cluster FS for temp files
+   * @param libDir relative directory to place resources
+   * @param libLocalSrcDir explicitly supplied local libs dir
+   * @throws IOException
+   * @throws SliderException
+   */
+  public static void addAllDependencyJars(Map<String, LocalResource> providerResources,
+                                          SliderFileSystem sliderFileSystem,
+                                          Path tempPath,
+                                          String libDir,
+                                          String libLocalSrcDir)
+      throws IOException, SliderException {
+    String libSrcToUse = libLocalSrcDir;
+    if (SliderUtils.isSet(libLocalSrcDir)) {
+      File file = new File(libLocalSrcDir);
+      if (!file.exists() || !file.isDirectory()) {
+        throw new BadCommandArgumentsException("Supplied lib src dir %s is not valid", libLocalSrcDir);
+      }
+    }
+    SliderUtils.putAllJars(providerResources, sliderFileSystem, tempPath, libDir, libSrcToUse);
+  }
+
+  /**
+   * build the log directory
+   * @return the log dir
+   */
+  public String getLogdir() throws IOException {
+    String logdir = System.getenv("LOGDIR");
+    if (logdir == null) {
+      logdir =
+        SliderKeys.TMP_LOGDIR_PREFIX + UserGroupInformation.getCurrentUser().getShortUserName();
+    }
+    return logdir;
+  }
+
+
+  public void validateNodeCount(AggregateConf instanceDescription,
+                                String name, int min, int max) throws
+                                                               BadCommandArgumentsException {
+    MapOperations component =
+      instanceDescription.getResourceOperations().getComponent(name);
+    int count;
+    if (component == null) {
+      count = 0;
+    } else {
+      count = component.getOptionInt(ResourceKeys.COMPONENT_INSTANCES, 0);
+    }
+    validateNodeCount(name, count, min, max);
+  }
+  
+  /**
+   * Validate the node count and heap size values of a node class 
+   * <p>
+   * If max &lt;= 0:  min &lt;= count
+   * If max &gt; 0:  min &lt;= count &lt;= max
+   * @param name node class name
+   * @param count requested node count
+   * @param min requested heap size
+   * @param max maximum value. 
+   * @throws BadCommandArgumentsException if the values are out of range
+   */
+  public void validateNodeCount(String name,
+                                int count,
+                                int min,
+                                int max) throws BadCommandArgumentsException {
+    if (count < min) {
+      throw new BadCommandArgumentsException(
+        "requested no of %s nodes: %d is below the minimum of %d", name, count,
+        min);
+    }
+    if (max > 0 && count > max) {
+      throw new BadCommandArgumentsException(
+        "requested no of %s nodes: %d is above the maximum of %d", name, count,
+        max);
+    }
+  }
+
+  /**
+   * copy all options beginning site. into the site.xml
+   * @param clusterSpec cluster specification
+   * @param sitexml map for XML file to build up
+   */
+  public void propagateSiteOptions(ClusterDescription clusterSpec,
+                                    Map<String, String> sitexml) {
+    Map<String, String> options = clusterSpec.options;
+    propagateSiteOptions(options, sitexml);
+  }
+
+  public void propagateSiteOptions(Map<String, String> options,
+                                   Map<String, String> sitexml) {
+    propagateSiteOptions(options, sitexml, "");
+  }
+
+  public void propagateSiteOptions(Map<String, String> options,
+                                   Map<String, String> sitexml,
+                                   String configName) {
+    propagateSiteOptions(options, sitexml, configName, null);
+  }
+
+  public void propagateSiteOptions(Map<String, String> options,
+                                   Map<String, String> sitexml,
+                                   String configName,
+                                   Map<String,String> tokenMap) {
+    String prefix = OptionKeys.SITE_XML_PREFIX +
+                    (!configName.isEmpty() ? configName + "." : "");
+    for (Map.Entry<String, String> entry : options.entrySet()) {
+      String key = entry.getKey();
+      if (key.startsWith(prefix)) {
+        String envName = key.substring(prefix.length());
+        if (!envName.isEmpty()) {
+          String value = entry.getValue();
+          if (tokenMap != null) {
+            for (Map.Entry<String,String> token : tokenMap.entrySet()) {
+              value = value.replaceAll(Pattern.quote(token.getKey()),
+                                       token.getValue());
+            }
+          }
+          sitexml.put(envName, value);
+        }
+      }
+    }
+  }
+
+  /**
+   * Propagate an option from the cluster specification option map
+   * to the site XML map, using the site key for the name
+   * @param global global config spec
+   * @param optionKey key in the option map
+   * @param sitexml  map for XML file to build up
+   * @param siteKey key to assign the value to in the site XML
+   * @throws BadConfigException if the option is missing from the cluster spec
+   */
+  public void propagateOption(MapOperations global,
+                              String optionKey,
+                              Map<String, String> sitexml,
+                              String siteKey) throws BadConfigException {
+    sitexml.put(siteKey, global.getMandatoryOption(optionKey));
+  }
+
+
+  /**
+   * Build the image dir. This path is relative and only valid at the far end
+   * @param instanceDefinition instance definition
+   * @param bindir bin subdir
+   * @param script script in bin subdir
+   * @return the path to the script
+   * @throws FileNotFoundException if a file is not found, or it is not a directory* 
+   */
+  public String buildPathToHomeDir(AggregateConf instanceDefinition,
+                                  String bindir,
+                                  String script) throws
+                                                 FileNotFoundException,
+                                                 BadConfigException {
+    MapOperations globalOptions =
+      instanceDefinition.getInternalOperations().getGlobalOptions();
+    String applicationHome =
+      globalOptions.get(InternalKeys.INTERNAL_APPLICATION_HOME);
+    String imagePath =
+      globalOptions.get(InternalKeys.INTERNAL_APPLICATION_IMAGE_PATH);
+    return buildPathToHomeDir(imagePath, applicationHome, bindir, script);
+  }
+
+  public String buildPathToHomeDir(String imagePath,
+                                   String applicationHome,
+                                   String bindir, String script) throws
+                                                                 FileNotFoundException {
+    String path;
+    File scriptFile;
+    if (imagePath != null) {
+      File tarball = new File(SliderKeys.LOCAL_TARBALL_INSTALL_SUBDIR);
+      scriptFile = findBinScriptInExpandedArchive(tarball, bindir, script);
+      // now work back from the script to build the relative path
+      // to the binary which will be valid remote or local
+      StringBuilder builder = new StringBuilder();
+      builder.append(SliderKeys.LOCAL_TARBALL_INSTALL_SUBDIR);
+      builder.append("/");
+      //for the script, we want the name of ../..
+      File archive = scriptFile.getParentFile().getParentFile();
+      builder.append(archive.getName());
+      path = builder.toString();
+
+    } else {
+      // using a home directory which is required to be present on 
+      // the local system -so will be absolute and resolvable
+      File homedir = new File(applicationHome);
+      path = homedir.getAbsolutePath();
+
+      //this is absolute, resolve its entire path
+      SliderUtils.verifyIsDir(homedir, log);
+      File bin = new File(homedir, bindir);
+      SliderUtils.verifyIsDir(bin, log);
+      scriptFile = new File(bin, script);
+      SliderUtils.verifyFileExists(scriptFile, log);
+    }
+    return path;
+  }
+
+  
+  /**
+   * Build the image dir. This path is relative and only valid at the far end
+   * @param instance instance options
+   * @param bindir bin subdir
+   * @param script script in bin subdir
+   * @return the path to the script
+   * @throws FileNotFoundException if a file is not found, or it is not a directory* 
+   */
+  public String buildPathToScript(AggregateConf instance,
+                                String bindir,
+                                String script) throws FileNotFoundException {
+    return buildPathToScript(instance.getInternalOperations(), bindir, script);
+  }
+  /**
+   * Build the image dir. This path is relative and only valid at the far end
+   * @param internal internal options
+   * @param bindir bin subdir
+   * @param script script in bin subdir
+   * @return the path to the script
+   * @throws FileNotFoundException if a file is not found, or it is not a directory* 
+   */
+  public String buildPathToScript(ConfTreeOperations internal,
+                                String bindir,
+                                String script) throws FileNotFoundException {
+    
+    String homedir = buildPathToHomeDir(
+      internal.get(InternalKeys.INTERNAL_APPLICATION_IMAGE_PATH),
+      internal.get(InternalKeys.INTERNAL_APPLICATION_HOME),
+      bindir,
+      script);
+    return buildScriptPath(bindir, script, homedir);
+  }
+  
+  
+
+  public String buildScriptPath(String bindir, String script, String homedir) {
+    StringBuilder builder = new StringBuilder(homedir);
+    builder.append("/");
+    builder.append(bindir);
+    builder.append("/");
+    builder.append(script);
+    return builder.toString();
+  }
+
+
+  public static String convertToAppRelativePath(File file) {
+    return convertToAppRelativePath(file.getPath());
+  }
+
+  public static String convertToAppRelativePath(String path) {
+    return ApplicationConstants.Environment.PWD.$() + "/" + path;
+  }
+
+
+  public static void validatePathReferencesLocalDir(String meaning, String path)
+      throws BadConfigException {
+    File file = new File(path);
+    if (!file.exists()) {
+      throw new BadConfigException("%s directory %s not found", meaning, file);
+    }
+    if (!file.isDirectory()) {
+      throw new BadConfigException("%s is not a directory: %s", meaning, file);
+    }
+  }
+
+  /**
+   * get the user name
+   * @return the user name
+   */
+  public String getUserName() throws IOException {
+    return UserGroupInformation.getCurrentUser().getShortUserName();
+  }
+
+  /**
+   * Find a script in an expanded archive
+   * @param base base directory
+   * @param bindir bin subdir
+   * @param script script in bin subdir
+   * @return the path to the script
+   * @throws FileNotFoundException if a file is not found, or it is not a directory
+   */
+  public File findBinScriptInExpandedArchive(File base,
+                                             String bindir,
+                                             String script)
+      throws FileNotFoundException {
+    
+    SliderUtils.verifyIsDir(base, log);
+    File[] ls = base.listFiles();
+    if (ls == null) {
+      //here for the IDE to be happy, as the previous check will pick this case
+      throw new FileNotFoundException("Failed to list directory " + base);
+    }
+
+    log.debug("Found {} entries in {}", ls.length, base);
+    List<File> directories = new LinkedList<File>();
+    StringBuilder dirs = new StringBuilder();
+    for (File file : ls) {
+      log.debug("{}", false);
+      if (file.isDirectory()) {
+        directories.add(file);
+        dirs.append(file.getPath()).append(" ");
+      }
+    }
+    if (directories.size() > 1) {
+      throw new FileNotFoundException(
+        "Too many directories in archive to identify binary: " + dirs);
+    }
+    if (directories.isEmpty()) {
+      throw new FileNotFoundException(
+        "No directory found in archive " + base);
+    }
+    File archive = directories.get(0);
+    File bin = new File(archive, bindir);
+    SliderUtils.verifyIsDir(bin, log);
+    File scriptFile = new File(bin, script);
+    SliderUtils.verifyFileExists(scriptFile, log);
+    return scriptFile;
+  }
+
+  /**
+   * Return any additional arguments (argv) to provide when starting this role
+   * 
+   * @param roleOptions
+   *          The options for this role
+   * @return A non-null String which contains command line arguments for this role, or the empty string.
+   */
+  public static String getAdditionalArgs(Map<String,String> roleOptions) {
+    if (roleOptions.containsKey(RoleKeys.ROLE_ADDITIONAL_ARGS)) {
+      String additionalArgs = roleOptions.get(RoleKeys.ROLE_ADDITIONAL_ARGS);
+      if (null != additionalArgs) {
+        return additionalArgs;
+      }
+    }
+
+    return "";
+  }
+  
+  public int getRoleResourceRequirement(String val,
+                                        int defVal,
+                                        int maxVal) {
+    if (val==null) {
+      val = Integer.toString(defVal);
+    }
+    Integer intVal;
+    if (ResourceKeys.YARN_RESOURCE_MAX.equals(val)) {
+      intVal = maxVal;
+    } else {
+      intVal = Integer.decode(val);
+    }
+    return intVal;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/SliderProviderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/SliderProviderFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/SliderProviderFactory.java
new file mode 100644
index 0000000..5dd4a32
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/SliderProviderFactory.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.SliderXmlConfKeys;
+import org.apache.slider.core.exceptions.BadClusterStateException;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.providers.agent.AgentKeys;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Base class for factories
+ */
+public abstract class SliderProviderFactory extends Configured {
+
+  public static final String DEFAULT_CLUSTER_TYPE = AgentKeys.PROVIDER_AGENT;
+  
+  protected static final Logger log =
+    LoggerFactory.getLogger(SliderProviderFactory.class);
+  public static final String PROVIDER_NOT_FOUND =
+    "Unable to find provider of application type %s";
+
+  public SliderProviderFactory(Configuration conf) {
+    super(conf);
+  }
+
+  protected SliderProviderFactory() {
+  }
+
+  public abstract AbstractClientProvider createClientProvider();
+
+  public abstract ProviderService createServerProvider();
+
+  /**
+   * Create a provider for a specific application
+   * @param application app
+   * @return app instance
+   * @throws SliderException on any instantiation problem
+   */
+  public static SliderProviderFactory createSliderProviderFactory(String application) throws
+      SliderException {
+    Configuration conf = loadSliderConfiguration();
+    if (application == null) {
+      application = DEFAULT_CLUSTER_TYPE;
+    }
+    String providerKey =
+      String.format(SliderXmlConfKeys.KEY_PROVIDER, application);
+    if (application.contains(".")) {
+      log.debug("Treating {} as a classname", application);
+      String name = "classname.key";
+      conf.set(name, application);
+      providerKey = name;
+    }
+    
+    Class<? extends SliderProviderFactory> providerClass;
+    try {
+      providerClass = conf.getClass(providerKey, null, SliderProviderFactory.class);
+    } catch (RuntimeException e) {
+      throw new BadClusterStateException(e, "Failed to load provider %s: %s", application, e);
+    }
+    if (providerClass == null) {
+      throw new BadClusterStateException(PROVIDER_NOT_FOUND, application);
+    }
+
+    Exception ex;
+    try {
+      SliderProviderFactory providerFactory = providerClass.newInstance();
+      providerFactory.setConf(conf);
+      return providerFactory;
+    } catch (Exception e) {
+      ex = e;
+    }
+    //by here the operation failed and ex is set to the value 
+    throw new BadClusterStateException(ex,
+                              "Failed to create an instance of %s : %s",
+                              providerClass,
+                              ex);
+  }
+
+  /**
+   * Load a configuration with the {@link SliderKeys#SLIDER_XML} resource
+   * included
+   * @return a configuration instance
+   */
+  public static Configuration loadSliderConfiguration() {
+    Configuration conf = new Configuration();
+    conf.addResource(SliderKeys.SLIDER_XML);
+    return conf;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
new file mode 100644
index 0000000..4c6a50b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
@@ -0,0 +1,701 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers.agent;
+
+import com.google.common.io.Files;
+import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
+import org.apache.commons.compress.archivers.tar.TarArchiveInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.registry.client.binding.RegistryUtils;
+import org.apache.slider.api.InternalKeys;
+import org.apache.slider.api.ResourceKeys;
+import org.apache.slider.client.ClientUtils;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.core.launch.AbstractLauncher;
+import org.apache.slider.core.registry.docstore.PublishedConfiguration;
+import org.apache.slider.providers.AbstractClientProvider;
+import org.apache.slider.providers.ProviderRole;
+import org.apache.slider.providers.ProviderUtils;
+import org.apache.slider.providers.agent.application.metadata.Application;
+import org.apache.slider.providers.agent.application.metadata.Component;
+import org.apache.slider.providers.agent.application.metadata.ConfigFile;
+import org.apache.slider.providers.agent.application.metadata.Metainfo;
+import org.apache.slider.providers.agent.application.metadata.MetainfoParser;
+import org.apache.slider.providers.agent.application.metadata.OSPackage;
+import org.apache.slider.providers.agent.application.metadata.OSSpecific;
+import org.apache.slider.providers.agent.application.metadata.Package;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
+
+/** This class implements  the client-side aspects of the agent deployer */
+public class AgentClientProvider extends AbstractClientProvider
+    implements AgentKeys, SliderKeys {
+
+
+  protected static final Logger log =
+      LoggerFactory.getLogger(AgentClientProvider.class);
+  protected static final String NAME = "agent";
+  private static final ProviderUtils providerUtils = new ProviderUtils(log);
+  public static final String E_COULD_NOT_READ_METAINFO
+      = "Not a valid app package. Could not read metainfo.";
+
+  protected Map<String, Metainfo> metaInfoMap = new ConcurrentHashMap<String, Metainfo>();
+
+  protected AgentClientProvider(Configuration conf) {
+    super(conf);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+
+  @Override
+  public List<ProviderRole> getRoles() {
+    return AgentRoles.getRoles();
+  }
+
+  @Override //Client
+  public void preflightValidateClusterConfiguration(SliderFileSystem sliderFileSystem,
+                                                    String clustername,
+                                                    Configuration configuration,
+                                                    AggregateConf instanceDefinition,
+                                                    Path clusterDirPath,
+                                                    Path generatedConfDirPath,
+                                                    boolean secure) throws
+      SliderException,
+      IOException {
+    super.preflightValidateClusterConfiguration(sliderFileSystem, clustername,
+                                                configuration,
+                                                instanceDefinition,
+                                                clusterDirPath,
+                                                generatedConfDirPath, secure);
+
+    String appDef = SliderUtils.getApplicationDefinitionPath(instanceDefinition
+        .getAppConfOperations());
+    Path appDefPath = new Path(appDef);
+    sliderFileSystem.verifyFileExists(appDefPath);
+
+    String agentConf = instanceDefinition.getAppConfOperations().
+        getGlobalOptions().getOption(AgentKeys.AGENT_CONF, "");
+    if (StringUtils.isNotEmpty(agentConf)) {
+      sliderFileSystem.verifyFileExists(new Path(agentConf));
+    }
+
+    String appHome = instanceDefinition.getAppConfOperations().
+        getGlobalOptions().get(AgentKeys.PACKAGE_PATH);
+    if (SliderUtils.isUnset(appHome)) {
+      String agentImage = instanceDefinition.getInternalOperations().
+          get(InternalKeys.INTERNAL_APPLICATION_IMAGE_PATH);
+      sliderFileSystem.verifyFileExists(new Path(agentImage));
+    }
+  }
+
+  @Override
+  public void validateInstanceDefinition(AggregateConf instanceDefinition, SliderFileSystem fs) throws
+      SliderException {
+    super.validateInstanceDefinition(instanceDefinition, fs);
+    log.debug("Validating conf {}", instanceDefinition);
+    ConfTreeOperations resources =
+        instanceDefinition.getResourceOperations();
+
+    providerUtils.validateNodeCount(instanceDefinition, ROLE_NODE,
+                                    0, -1);
+
+    String appDef = null;
+    try {
+      // Validate the app definition
+      appDef = SliderUtils.getApplicationDefinitionPath(instanceDefinition
+          .getAppConfOperations());
+    } catch (BadConfigException bce) {
+      throw new BadConfigException("Application definition must be provided. " + bce.getMessage());
+    }
+
+    log.info("Validating app definition {}", appDef);
+    String extension = appDef.substring(appDef.lastIndexOf(".") + 1, appDef.length());
+    if (!"zip".equals(extension.toLowerCase(Locale.ENGLISH))) {
+      throw new BadConfigException("App definition must be packaged as a .zip file. File provided is " + appDef);
+    }
+
+    Set<String> names = resources.getComponentNames();
+    names.remove(SliderKeys.COMPONENT_AM);
+    Map<Integer, String> priorityMap = new HashMap<Integer, String>();
+
+    Metainfo metaInfo = getMetainfo(fs, appDef);
+
+    for (String name : names) {
+      MapOperations component = resources.getMandatoryComponent(name);
+
+      if (metaInfo != null) {
+        Component componentDef = metaInfo.getApplicationComponent(name);
+        if (componentDef == null) {
+          throw new BadConfigException(
+              "Component %s is not a member of application.", name);
+        }
+      }
+
+      int priority =
+          component.getMandatoryOptionInt(ResourceKeys.COMPONENT_PRIORITY);
+      if (priority <= 0) {
+        throw new BadConfigException("Component %s %s value out of range %d",
+                                     name,
+                                     ResourceKeys.COMPONENT_PRIORITY,
+                                     priority);
+      }
+
+      String existing = priorityMap.get(priority);
+      if (existing != null) {
+        throw new BadConfigException(
+            "Component %s has a %s value %d which duplicates that of %s",
+            name,
+            ResourceKeys.COMPONENT_PRIORITY,
+            priority,
+            existing);
+      }
+      priorityMap.put(priority, name);
+    }
+
+    // fileSystem may be null for tests
+    if (metaInfo != null) {
+      for (String name : names) {
+        Component componentDef = metaInfo.getApplicationComponent(name);
+        if (componentDef == null) {
+          throw new BadConfigException(
+              "Component %s is not a member of application.", name);
+        }
+
+        // ensure that intance count is 0 for client components
+        if ("CLIENT".equals(componentDef.getCategory())) {
+          MapOperations componentConfig = resources.getMandatoryComponent(name);
+          int count =
+              componentConfig.getMandatoryOptionInt(ResourceKeys.COMPONENT_INSTANCES);
+          if (count > 0) {
+            throw new BadConfigException("Component %s is of type CLIENT and cannot be instantiated."
+                                         + " Use \"slider client install ...\" command instead.",
+                                         name);
+          }
+        } else {
+          MapOperations componentConfig = resources.getMandatoryComponent(name);
+          int count =
+              componentConfig.getMandatoryOptionInt(ResourceKeys.COMPONENT_INSTANCES);
+          int definedMinCount = componentDef.getMinInstanceCountInt();
+          int definedMaxCount = componentDef.getMaxInstanceCountInt();
+          if (count < definedMinCount || count > definedMaxCount) {
+            throw new BadConfigException("Component %s, %s value %d out of range. "
+                                         + "Expected minimum is %d and maximum is %d",
+                                         name,
+                                         ResourceKeys.COMPONENT_INSTANCES,
+                                         count,
+                                         definedMinCount,
+                                         definedMaxCount);
+          }
+        }
+      }
+    }
+  }
+
+
+  @Override
+  public void prepareAMAndConfigForLaunch(SliderFileSystem fileSystem,
+                                          Configuration serviceConf,
+                                          AbstractLauncher launcher,
+                                          AggregateConf instanceDefinition,
+                                          Path snapshotConfDirPath,
+                                          Path generatedConfDirPath,
+                                          Configuration clientConfExtras,
+                                          String libdir,
+                                          Path tempPath,
+                                          boolean miniClusterTestRun) throws
+      IOException,
+      SliderException {
+    String agentImage = instanceDefinition.getInternalOperations().
+        get(InternalKeys.INTERNAL_APPLICATION_IMAGE_PATH);
+    if (SliderUtils.isUnset(agentImage)) {
+      Path agentPath = new Path(tempPath.getParent(), AgentKeys.PROVIDER_AGENT);
+      log.info("Automatically uploading the agent tarball at {}", agentPath);
+      fileSystem.getFileSystem().mkdirs(agentPath);
+      if (ProviderUtils.addAgentTar(this, AGENT_TAR, fileSystem, agentPath)) {
+        instanceDefinition.getInternalOperations().set(
+            InternalKeys.INTERNAL_APPLICATION_IMAGE_PATH,
+            new Path(agentPath, AGENT_TAR).toUri());
+      }
+    }
+  }
+
+  @Override
+  public Set<String> getApplicationTags(SliderFileSystem fileSystem,
+                                        String appDef) throws SliderException {
+    Set<String> tags;
+    Metainfo metaInfo = getMetainfo(fileSystem, appDef);
+
+    if (metaInfo == null) {
+      log.error("Error retrieving metainfo from {}", appDef);
+      throw new SliderException("Error parsing metainfo file, possibly bad structure.");
+    }
+
+    Application application = metaInfo.getApplication();
+    tags = new HashSet<String>();
+    tags.add("Name: " + application.getName());
+    tags.add("Version: " + application.getVersion());
+    tags.add("Description: " + SliderUtils.truncate(application.getComment(), 80));
+
+    return tags;
+  }
+
+  @Override
+  public void processClientOperation(SliderFileSystem fileSystem,
+                                     RegistryOperations rops,
+                                     Configuration configuration,
+                                     String operation,
+                                     File clientInstallPath,
+                                     File appPackage,
+                                     JSONObject config,
+                                     String name) throws SliderException {
+    // create temp folder
+    // create sub-folders app_pkg, agent_pkg, command
+    File tmpDir = Files.createTempDir();
+    log.info("Command is being executed at {}", tmpDir.getAbsolutePath());
+    File appPkgDir = new File(tmpDir, "app_pkg");
+    appPkgDir.mkdir();
+
+    File agentPkgDir = new File(tmpDir, "agent_pkg");
+    agentPkgDir.mkdir();
+
+    File cmdDir = new File(tmpDir, "command");
+    cmdDir.mkdir();
+
+    Metainfo metaInfo = null;
+    JSONObject defaultConfig = null;
+    try {
+      // expand app package into /app_pkg
+      ZipInputStream zipInputStream = null;
+      try {
+        zipInputStream = new ZipInputStream(new FileInputStream(appPackage));
+        {
+          ZipEntry zipEntry = zipInputStream.getNextEntry();
+          while (zipEntry != null) {
+            log.info("Processing {}", zipEntry.getName());
+            String filePath = appPkgDir + File.separator + zipEntry.getName();
+            if (!zipEntry.isDirectory()) {
+              log.info("Extracting file {}", filePath);
+              extractFile(zipInputStream, filePath);
+
+              if ("metainfo.xml".equals(zipEntry.getName())) {
+                FileInputStream input = null;
+                try {
+                  input = new FileInputStream(filePath);
+                  metaInfo = new MetainfoParser().fromXmlStream(input);
+                } finally {
+                  IOUtils.closeStream(input);
+                }
+              } else if ("metainfo.json".equals(zipEntry.getName())) {
+                FileInputStream input = null;
+                try {
+                  input = new FileInputStream(filePath);
+                  metaInfo = new MetainfoParser().fromJsonStream(input);
+                } finally {
+                  IOUtils.closeStream(input);
+                }
+              } else if ("clientInstallConfig-default.json".equals(zipEntry.getName())) {
+                try {
+                  defaultConfig = new JSONObject(FileUtils.readFileToString(new File(filePath), Charset.defaultCharset()));
+                } catch (JSONException jex) {
+                  throw new SliderException("Unable to read default client config.", jex);
+                }
+              }
+            } else {
+              log.info("Creating dir {}", filePath);
+              File dir = new File(filePath);
+              dir.mkdir();
+            }
+            zipInputStream.closeEntry();
+            zipEntry = zipInputStream.getNextEntry();
+          }
+        }
+      } finally {
+        zipInputStream.close();
+      }
+
+      if (metaInfo == null) {
+        throw new BadConfigException(E_COULD_NOT_READ_METAINFO);
+      }
+
+      String clientScript = null;
+      String clientComponent = null;
+      for (Component component : metaInfo.getApplication().getComponents()) {
+        if (component.getCategory().equals("CLIENT")) {
+          clientComponent = component.getName();
+          if (component.getCommandScript() != null) {
+            clientScript = component.getCommandScript().getScript();
+          }
+          break;
+        }
+      }
+
+      if (SliderUtils.isUnset(clientScript)) {
+        log.info("Installing CLIENT without script");
+        List<Package> packages = metaInfo.getApplication().getPackages();
+        if (packages.size() > 0) {
+          // retrieve package resources from HDFS and extract
+          for (Package pkg : packages) {
+            Path pkgPath = fileSystem.buildResourcePath(pkg.getName());
+            if (!fileSystem.isFile(pkgPath) && name != null) {
+              pkgPath = fileSystem.buildResourcePath(name, pkg.getName());
+            }
+            if (!fileSystem.isFile(pkgPath)) {
+              throw new IOException("Package doesn't exist as a resource: " +
+                  pkg.getName());
+            }
+            if ("archive".equals(pkg.getType())) {
+              File pkgFile = new File(tmpDir, pkg.getName());
+              fileSystem.copyHdfsFileToLocal(pkgPath, pkgFile);
+              expandTar(pkgFile, clientInstallPath);
+            } else {
+              File pkgFile = new File(clientInstallPath, pkg.getName());
+              fileSystem.copyHdfsFileToLocal(pkgPath, pkgFile);
+            }
+          }
+        } else {
+          // extract tarball from app def
+          for (OSSpecific osSpecific : metaInfo.getApplication()
+              .getOSSpecifics()) {
+            for (OSPackage pkg : osSpecific.getPackages()) {
+              if ("tarball".equals(pkg.getType())) {
+                File pkgFile = new File(appPkgDir, pkg.getName());
+                expandTar(pkgFile, clientInstallPath);
+              }
+            }
+          }
+        }
+        if (name == null) {
+          log.warn("Conf files not being generated because no app name was " +
+              "provided");
+          return;
+        }
+        File confInstallDir;
+        String clientRoot = null;
+        if (config != null) {
+          try {
+            clientRoot = config.getJSONObject("global")
+                .getString(AgentKeys.APP_CLIENT_ROOT);
+          } catch (JSONException e) {
+            log.info("Couldn't read {} from provided client config, falling " +
+                "back on default", AgentKeys.APP_CLIENT_ROOT);
+          }
+        }
+        if (clientRoot == null && defaultConfig != null) {
+          try {
+            clientRoot = defaultConfig.getJSONObject("global")
+                .getString(AgentKeys.APP_CLIENT_ROOT);
+          } catch (JSONException e) {
+            log.info("Couldn't read {} from default client config, using {}",
+                AgentKeys.APP_CLIENT_ROOT, clientInstallPath);
+          }
+        }
+        if (clientRoot == null) {
+          confInstallDir = clientInstallPath;
+        } else {
+          confInstallDir = new File(new File(clientInstallPath, clientRoot), "conf");
+          if (!confInstallDir.exists()) {
+            confInstallDir.mkdirs();
+          }
+        }
+        String user = RegistryUtils.currentUser();
+        for (ConfigFile configFile : metaInfo.getComponentConfigFiles(clientComponent)) {
+          retrieveConfigFile(rops, configuration, configFile, name, user,
+              confInstallDir);
+        }
+      } else {
+        log.info("Installing CLIENT using script {}", clientScript);
+        expandAgentTar(agentPkgDir);
+
+        JSONObject commandJson = getCommandJson(defaultConfig, config, metaInfo, clientInstallPath, name);
+        FileWriter file = new FileWriter(new File(cmdDir, "command.json"));
+        try {
+          file.write(commandJson.toString());
+
+        } catch (IOException e) {
+          log.error("Couldn't write command.json to file");
+        } finally {
+          file.flush();
+          file.close();
+        }
+
+        runCommand(appPkgDir, agentPkgDir, cmdDir, clientScript);
+      }
+
+    } catch (IOException ioex) {
+      log.warn("Error while executing INSTALL command {}", ioex.getMessage());
+      throw new SliderException("INSTALL client failed.");
+    }
+  }
+
+  protected void runCommand(
+      File appPkgDir,
+      File agentPkgDir,
+      File cmdDir,
+      String clientScript) throws SliderException {
+    int exitCode = 0;
+    Exception exp = null;
+    try {
+      String clientScriptPath = appPkgDir.getAbsolutePath() + File.separator + "package" +
+                                File.separator + clientScript;
+      List<String> command = Arrays.asList(AgentKeys.PYTHON_EXE,
+               "-S",
+               clientScriptPath,
+               "INSTALL",
+               cmdDir.getAbsolutePath() + File.separator + "command.json",
+               appPkgDir.getAbsolutePath() + File.separator + "package",
+               cmdDir.getAbsolutePath() + File.separator + "command-out.json",
+               "DEBUG");
+      ProcessBuilder pb = new ProcessBuilder(command);
+      log.info("Command: " + StringUtils.join(pb.command(), " "));
+      pb.environment().put(SliderKeys.PYTHONPATH,
+                           agentPkgDir.getAbsolutePath()
+                           + File.separator + "slider-agent" + File.pathSeparator
+                           + agentPkgDir.getAbsolutePath()
+                           + File.separator + "slider-agent/jinja2");
+      log.info("{}={}", SliderKeys.PYTHONPATH, pb.environment().get(SliderKeys.PYTHONPATH));
+
+      Process proc = pb.start();
+      InputStream stderr = proc.getErrorStream();
+      InputStream stdout = proc.getInputStream();
+      BufferedReader stdOutReader = new BufferedReader(new InputStreamReader(stdout));
+      BufferedReader stdErrReader = new BufferedReader(new InputStreamReader(stderr));
+
+      proc.waitFor();
+
+      String line;
+      while ((line = stdOutReader.readLine()) != null) {
+        log.info("Stdout: " + line);
+      }
+      while ((line = stdErrReader.readLine()) != null) {
+        log.info("Stderr: " + line);
+      }
+
+      exitCode = proc.exitValue();
+      log.info("Exit value is {}", exitCode);
+    } catch (IOException e) {
+      exp = e;
+    } catch (InterruptedException e) {
+      exp = e;
+    }
+
+    if (exitCode != 0) {
+      throw new SliderException("INSTALL client failed with exit code " + exitCode);
+    }
+
+    if (exp != null) {
+      log.error("Error while executing INSTALL command {}. Stack trace {}",
+                exp.getMessage(),
+                ExceptionUtils.getStackTrace(exp));
+      throw new SliderException("INSTALL client failed.", exp);
+    }
+  }
+
+  private void expandAgentTar(File agentPkgDir) throws IOException {
+    String libDirProp =
+        System.getProperty(SliderKeys.PROPERTY_LIB_DIR);
+    File tarFile = new File(libDirProp, SliderKeys.AGENT_TAR);
+    expandTar(tarFile, agentPkgDir);
+  }
+
+  private void expandTar(File tarFile, File destDir) throws IOException {
+    log.info("Expanding tar {} to {}", tarFile, destDir);
+    TarArchiveInputStream tarIn = new TarArchiveInputStream(
+        new GzipCompressorInputStream(
+            new BufferedInputStream(
+                new FileInputStream(tarFile)
+            )
+        )
+    );
+    try {
+      TarArchiveEntry tarEntry = tarIn.getNextTarEntry();
+      while (tarEntry != null) {
+        File destPath = new File(destDir, tarEntry.getName());
+        File parent = destPath.getParentFile();
+        if (!parent.exists()) {
+          parent.mkdirs();
+        }
+        if (tarEntry.isDirectory()) {
+          destPath.mkdirs();
+        } else {
+          byte[] byteToRead = new byte[1024];
+          BufferedOutputStream buffOut =
+              new BufferedOutputStream(new FileOutputStream(destPath));
+          try {
+            int len;
+            while ((len = tarIn.read(byteToRead)) != -1) {
+              buffOut.write(byteToRead, 0, len);
+            }
+          } finally {
+            buffOut.close();
+          }
+        }
+        if ((tarEntry.getMode() & 0100) != 0) {
+          destPath.setExecutable(true);
+        }
+        tarEntry = tarIn.getNextTarEntry();
+      }
+    } finally {
+      tarIn.close();
+    }
+  }
+
+  private void retrieveConfigFile(RegistryOperations rops,
+      Configuration configuration, ConfigFile configFile, String name,
+      String user, File destDir) throws IOException, SliderException {
+    log.info("Retrieving config {} to {}", configFile.getDictionaryName(),
+        destDir);
+    PublishedConfiguration published = ClientUtils.getConfigFromRegistry(rops,
+        configuration, configFile.getDictionaryName(), name, user, true);
+    ClientUtils.saveOrReturnConfig(published, configFile.getType(),
+        destDir, configFile.getFileName());
+  }
+
+  protected JSONObject getCommandJson(JSONObject defaultConfig,
+                                      JSONObject inputConfig,
+                                      Metainfo metainfo,
+                                      File clientInstallPath,
+                                      String name) throws SliderException {
+    try {
+      JSONObject pkgList = new JSONObject();
+      pkgList.put(AgentKeys.PACKAGE_LIST,
+                  AgentProviderService.getPackageListFromApplication(metainfo.getApplication()));
+      JSONObject obj = new JSONObject();
+      obj.put("hostLevelParams", pkgList);
+
+      String user = RegistryUtils.currentUser();
+      JSONObject configuration = new JSONObject();
+      JSONObject global = new JSONObject();
+      global.put("app_install_dir", clientInstallPath.getAbsolutePath());
+      global.put("app_user", user);
+      if (name != null) {
+        global.put("app_name", name);
+      }
+
+      if (defaultConfig != null) {
+        readConfigEntries(defaultConfig, clientInstallPath, global, name, user);
+      }
+      if (inputConfig != null) {
+        readConfigEntries(inputConfig, clientInstallPath, global, name, user);
+      }
+
+      configuration.put("global", global);
+      obj.put("configurations", configuration);
+      return obj;
+    } catch (JSONException jex) {
+      log.warn("Error while executing INSTALL command {}", jex.getMessage());
+      throw new SliderException("INSTALL client failed.");
+    }
+  }
+
+  private void readConfigEntries(JSONObject inpConfig,
+                                 File clientInstallPath,
+                                 JSONObject globalConfig,
+                                 String name, String user)
+      throws JSONException {
+    JSONObject globalSection = inpConfig.getJSONObject("global");
+    Iterator it = globalSection.keys();
+    while (it.hasNext()) {
+      String key = (String) it.next();
+      String value = globalSection.getString(key);
+      if (SliderUtils.isSet(value)) {
+        value = value.replace("{app_install_dir}", clientInstallPath.getAbsolutePath());
+        value = value.replace("{app_user}", user);
+        if (name != null) {
+          value = value.replace("{app_name}", name);
+        }
+      }
+      if (globalConfig.has(key)) {
+        // last one wins
+        globalConfig.remove(key);
+      }
+      globalConfig.put(key, value);
+    }
+  }
+
+  private void extractFile(ZipInputStream zipInputStream, String filePath) throws IOException {
+    BufferedOutputStream output = new BufferedOutputStream(new FileOutputStream(filePath));
+    try {
+      byte[] bytesRead = new byte[4096];
+      int read = 0;
+      while ((read = zipInputStream.read(bytesRead)) != -1) {
+        output.write(bytesRead, 0, read);
+      }
+    } finally {
+      output.close();
+    }
+  }
+
+  private Metainfo getMetainfo(SliderFileSystem fs, String appDef) {
+    Metainfo metaInfo = metaInfoMap.get(appDef);
+    if (fs != null && metaInfo == null) {
+      try {
+        metaInfo = AgentUtils.getApplicationMetainfo(fs, appDef, false);
+        metaInfoMap.put(appDef, metaInfo);
+      } catch (IOException ioe) {
+        // Ignore missing metainfo file for now
+        log.info("Missing metainfo {}", ioe.getMessage());
+      } catch (BadConfigException bce) {
+        log.info("Bad Configuration {}", bce.getMessage());
+      }
+    }
+    return metaInfo;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java
new file mode 100644
index 0000000..01a3f1a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.providers.agent;
+
+/*
+
+ */
+public interface AgentKeys {
+
+  String PROVIDER_AGENT = "agent";
+  String ROLE_NODE = "echo";
+
+  /**
+   * Template stored in the slider classpath -to use if there is
+   * no site-specific template
+   * {@value}
+   */
+  String CONF_RESOURCE = "org/apache/slider/providers/agent/conf/";
+  /*  URL to talk back to Agent Controller*/
+  String CONTROLLER_URL = "agent.controller.url";
+  /**
+   * The location of pre-installed agent path.
+   * This can be also be dynamically computed based on Yarn installation of agent.
+   */
+  String PACKAGE_PATH = "agent.package.root";
+  /**
+   * The location of the script implementing the command.
+   */
+  String SCRIPT_PATH = "agent.script";
+  /**
+   * Execution home for the agent.
+   */
+  String APP_HOME = "app.home";
+  String APP_ROOT = "site.global.app_root";
+  String APP_CLIENT_ROOT = "client_root";
+  /**
+   * Runas user of the application
+   */
+  String RUNAS_USER = "site.global.app_user";
+  /**
+   * Name of the service.
+   */
+  String SERVICE_NAME = "app.name";
+  String ARG_LABEL = "--label";
+  String ARG_HOST = "--host";
+  String ARG_PORT = "--port";
+  String ARG_SECURED_PORT = "--secured_port";
+  String ARG_ZOOKEEPER_QUORUM = "--zk-quorum";
+  String ARG_ZOOKEEPER_REGISTRY_PATH = "--zk-reg-path";
+  String ARG_DEBUG = "--debug";
+  String AGENT_MAIN_SCRIPT_ROOT = "./infra/agent/slider-agent/";
+  String AGENT_JINJA2_ROOT = "./infra/agent/slider-agent/jinja2";
+  String AGENT_MAIN_SCRIPT = "agent/main.py";
+
+  String APP_DEF = "application.def";
+  String ADDON_PREFIX = "application.addon.";
+  String ADDONS = "application.addons";
+  String AGENT_VERSION = "agent.version";
+  String AGENT_CONF = "agent.conf";
+  String ADDON_FOR_ALL_COMPONENTS = "ALL";
+
+  String APP_RESOURCES = "application.resources";
+  String APP_RESOURCES_DIR = "app/resources";
+
+  String APP_CONF_DIR = "app/conf";
+
+  String AGENT_INSTALL_DIR = "infra/agent";
+  String APP_DEFINITION_DIR = "app/definition";
+  String ADDON_DEFINITION_DIR = "addon/definition";
+  String AGENT_CONFIG_FILE = "infra/conf/agent.ini";
+  String AGENT_VERSION_FILE = "infra/version";
+  String APP_PACKAGES_DIR = "app/packages";
+  String PER_COMPONENT = "per.component";
+  String PER_GROUP = "per.group";
+
+  String JAVA_HOME = "java_home";
+  String PACKAGE_LIST = "package_list";
+  String SYSTEM_CONFIGS = "system_configs";
+  String WAIT_HEARTBEAT = "wait.heartbeat";
+  String PYTHON_EXE = "python";
+  String CREATE_DEF_ZK_NODE = "create.default.zookeeper.node";
+  String HEARTBEAT_MONITOR_INTERVAL = "heartbeat.monitor.interval";
+  String AGENT_INSTANCE_DEBUG_DATA = "agent.instance.debug.data";
+  String AGENT_OUT_FILE = "slider-agent.out";
+  String KEY_AGENT_TWO_WAY_SSL_ENABLED = "ssl.server.client.auth";
+  String INFRA_RUN_SECURITY_DIR = "infra/run/security/";
+  String CERT_FILE_LOCALIZATION_PATH = INFRA_RUN_SECURITY_DIR + "ca.crt";
+  String KEY_CONTAINER_LAUNCH_DELAY = "container.launch.delay.sec";
+  String TEST_RELAX_VERIFICATION = "test.relax.validation";
+  String AM_CONFIG_GENERATION = "am.config.generation";
+}
+
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentLaunchParameter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentLaunchParameter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentLaunchParameter.java
new file mode 100644
index 0000000..18c6374
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentLaunchParameter.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers.agent;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+class AgentLaunchParameter {
+  public static final Logger log =
+      LoggerFactory.getLogger(AgentLaunchParameter.class);
+  private static final String DEFAULT_PARAMETER = "";
+  private static final String ANY_COMPONENT = "ANY";
+  private static final String NONE_VALUE = "NONE";
+  private final Map<String, CommandTracker> launchParameterTracker;
+
+  public AgentLaunchParameter(String parameters) {
+    launchParameterTracker = parseExpectedLaunchParameters(parameters);
+  }
+
+  /**
+   * Get command for the component type
+   *
+   * @param componentGroup
+   *
+   * @return
+   */
+  public String getNextLaunchParameter(String componentGroup) {
+    if (launchParameterTracker != null) {
+      if (launchParameterTracker.containsKey(componentGroup)
+          || launchParameterTracker.containsKey(ANY_COMPONENT)) {
+        synchronized (this) {
+          CommandTracker indexTracker = null;
+          if (launchParameterTracker.containsKey(componentGroup)) {
+            indexTracker = launchParameterTracker.get(componentGroup);
+          } else {
+            indexTracker = launchParameterTracker.get(ANY_COMPONENT);
+          }
+
+          return indexTracker.getNextCommand();
+        }
+      }
+    }
+
+    return DEFAULT_PARAMETER;
+  }
+
+  /**
+   * Parse launch parameters of the form ANY:PARAM_FOR_FIRST:PARAM_FOR_SECOND:...:PARAM_FOR_REST|HBASE_MASTER:...
+   *
+   * E.g. ANY:DO_NOT_REGISTER:DO_NOT_HEARTBEAT:NONE For any container, first one gets DO_NOT_REGISTER second one gets
+   * DO_NOT_HEARTBEAT, then all of the rest get nothing
+   *
+   * E.g. HBASE_MASTER:FAIL_AFTER_START:NONE For HBASE_MASTER, first one gets FAIL_AFTER_START then "" for all
+   *
+   * @param launchParameters
+   *
+   * @return
+   */
+  Map<String, CommandTracker> parseExpectedLaunchParameters(String launchParameters) {
+    Map<String, CommandTracker> trackers = null;
+    if (launchParameters != null && launchParameters.length() > 0) {
+      String[] componentSpecificParameters = launchParameters.split(Pattern.quote("|"));
+      for (String componentSpecificParameter : componentSpecificParameters) {
+        if (componentSpecificParameter.length() != 0) {
+          String[] parameters = componentSpecificParameter.split(Pattern.quote(":"));
+
+          if (parameters.length > 1 && parameters[0].length() > 0) {
+
+            for (int index = 1; index < parameters.length; index++) {
+              if (parameters[index].equals(NONE_VALUE)) {
+                parameters[index] = DEFAULT_PARAMETER;
+              }
+            }
+
+            if (trackers == null) {
+              trackers = new HashMap<String, CommandTracker>(10);
+            }
+            String componentName = parameters[0];
+            CommandTracker tracker = new CommandTracker(Arrays.copyOfRange(parameters, 1, parameters.length));
+            trackers.put(componentName, tracker);
+          }
+        }
+      }
+    }
+
+    return trackers;
+  }
+
+  class CommandTracker {
+    private final int maxIndex;
+    private final String[] launchCommands;
+    private int currentIndex;
+
+    CommandTracker(String[] launchCommands) {
+      this.currentIndex = 0;
+      this.maxIndex = launchCommands.length - 1;
+      this.launchCommands = launchCommands;
+    }
+
+    String getNextCommand() {
+      String retVal = launchCommands[currentIndex];
+      if (currentIndex != maxIndex) {
+        currentIndex++;
+      }
+
+      return retVal;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderFactory.java
new file mode 100644
index 0000000..d5ca749
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderFactory.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers.agent;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.slider.providers.AbstractClientProvider;
+import org.apache.slider.providers.ProviderService;
+import org.apache.slider.providers.SliderProviderFactory;
+
+public class AgentProviderFactory extends SliderProviderFactory {
+
+  public static final String CLASSNAME =
+      "org.apache.slider.providers.agent.AgentProviderFactory";
+
+  public AgentProviderFactory() {
+  }
+
+  public AgentProviderFactory(Configuration conf) {
+    super(conf);
+  }
+
+  @Override
+  public AbstractClientProvider createClientProvider() {
+    return new AgentClientProvider(getConf());
+  }
+
+  @Override
+  public ProviderService createServerProvider() {
+    return new AgentProviderService();
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[29/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionListArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionListArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionListArgs.java
new file mode 100644
index 0000000..739b5fc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionListArgs.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+
+@Parameters(commandNames = {SliderActions.ACTION_LIST},
+            commandDescription = SliderActions.DESCRIBE_ACTION_LIST)
+
+public class ActionListArgs extends AbstractActionArgs {
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_LIST;
+  }
+
+  @Parameter(names = {ARG_LIVE},
+          description = "List only live application instances")
+  public boolean live;
+
+  @Parameter(names = {ARG_STATE},
+      description = "list only applications in the specific YARN state")
+  public String state = "";
+  
+  @Parameter(names = {ARG_VERBOSE},
+      description = "print out information in details")
+  public boolean verbose = false;
+
+  @Parameter(names = {ARG_CONTAINERS},
+      description = "List containers of an application instance")
+  public boolean containers;
+
+  @Parameter(names = {ARG_VERSION},
+      description = "Filter containers by app version (used with " +
+                    ARG_CONTAINERS + ")")
+  public String version;
+
+  @Parameter(names = {ARG_COMPONENTS}, variableArity = true,
+      description = "Filter containers by component names (used with " +
+                    ARG_CONTAINERS + ")")
+  public Set<String> components = new HashSet<>(0);
+
+  /**
+   * Get the min #of params expected
+   * @return the min number of params in the {@link #parameters} field
+   */
+  public int getMinParams() {
+    return 0;
+  }
+
+  @Override
+  public int getMaxParams() {
+    return 1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionLookupArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionLookupArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionLookupArgs.java
new file mode 100644
index 0000000..1b73522
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionLookupArgs.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import org.apache.commons.lang.StringUtils;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.UsageException;
+
+import java.io.File;
+
+@Parameters(commandNames = {SliderActions.ACTION_LOOKUP},
+            commandDescription = SliderActions.DESCRIBE_ACTION_LOOKUP)
+
+public class ActionLookupArgs extends AbstractActionArgs {
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_LOOKUP;
+  }
+
+  public int getMinParams() {
+    return 0;
+  }
+  public int getMaxParams() {
+    return 0;
+  }
+  
+  @Parameter(names = {ARG_ID},
+             description = "ID of the application")
+  public String id;
+
+  @Parameter(names = {ARG_OUTPUT, ARG_OUTPUT_SHORT},
+      description = "output file for any application report")
+  public File outputFile;
+
+  @Override
+  public void validate() throws BadCommandArgumentsException, UsageException {
+    super.validate();
+    if (StringUtils.isEmpty(id)) {
+      throw new BadCommandArgumentsException("Missing mandatory argument "
+                                             + ARG_ID);
+    }
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder(SliderActions.ACTION_LOOKUP);
+    if (id!=null) {
+      sb.append(" ");
+      sb.append(ARG_ID).append(" ").append(id);
+    }
+    if (outputFile != null) {
+      sb.append(" ");
+      sb.append(ARG_OUTPUT).append(" ").append(outputFile.getAbsolutePath());
+    }
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionNodesArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionNodesArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionNodesArgs.java
new file mode 100644
index 0000000..ec38c80
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionNodesArgs.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+
+import java.io.File;
+
+@Parameters(commandNames = {SliderActions.ACTION_NODES},
+            commandDescription = SliderActions.DESCRIBE_ACTION_NODES)
+public class ActionNodesArgs extends AbstractActionArgs {
+
+  /**
+   * Instance for API use; on CLI the name is derived from {@link #getClusterName()}.
+   */
+  public String instance;
+
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_NODES;
+  }
+
+  @Parameter(names = {ARG_OUTPUT, ARG_OUTPUT_SHORT},
+             description = "Output file for the information")
+  public File outputFile;
+
+  @Parameter(names = {ARG_LABEL})
+  public String label = "";
+
+  @Parameter(names = {ARG_HEALTHY} )
+  public boolean healthy;
+
+  @Override
+  public int getMinParams() {
+    return 0;
+  }
+
+  @Override
+  public int getMaxParams() {
+    return 1;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+      "ActionNodesArgs{");
+    sb.append("instance='").append(instance).append('\'');
+    sb.append(", outputFile=").append(outputFile);
+    sb.append(", label='").append(label).append('\'');
+    sb.append(", healthy=").append(healthy);
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionPackageArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionPackageArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionPackageArgs.java
new file mode 100644
index 0000000..4833934
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionPackageArgs.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+
+@Parameters(commandNames = {SliderActions.ACTION_PACKAGE},
+            commandDescription = SliderActions.DESCRIBE_ACTION_PACKAGE)
+
+public class ActionPackageArgs extends AbstractActionArgs {
+
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_PACKAGE;
+  }
+
+  @Parameter(names = {ARG_INSTALL},
+      description = "Install package in the sub-folder 'package' of the user's Slider base directory")
+  public boolean install;
+
+  @Parameter(names = {ARG_PKGDELETE},
+      description = "Delete package operation")
+  public boolean delete;
+
+  @Parameter(names = {ARG_PKGLIST},
+      description = "List of package(s) installed")
+  public boolean list;
+
+  @Parameter(names = {ARG_PKGINSTANCES},
+      description = "Lists all application instances referring to package")
+  public boolean instances;
+
+  @Parameter(names = {ARG_PACKAGE},
+             description = "Path to app package on local disk")
+  public String packageURI;
+
+  @Parameter(names = {ARG_NAME},
+             description = "Package name")
+  public String name;
+
+  @Parameter(names = {ARG_VERSION}, description = "Package version")
+  public String version;
+
+  @Parameter(names = {ARG_REPLACE_PKG}, 
+      description = "Overwrite existing package")
+  public boolean replacePkg = false;
+
+  @Parameter(names = {ARG_OUTPUT, ARG_OUTPUT_SHORT},
+      description = "Output file for package data")
+  public String out;
+
+  /**
+   * Get the min #of params expected
+   * @return the min number of params in the {@link #parameters} field
+   */
+  public int getMinParams() {
+    return 0;
+  }
+
+  @Override
+  public int getMaxParams() {
+    return 1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionRegistryArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionRegistryArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionRegistryArgs.java
new file mode 100644
index 0000000..da1b0e5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionRegistryArgs.java
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.UsageException;
+import org.apache.slider.core.registry.docstore.ConfigFormat;
+
+import static org.apache.slider.common.params.SliderActions.ACTION_REGISTRY;
+import static org.apache.slider.common.params.SliderActions.DESCRIBE_ACTION_REGISTRY;
+import java.io.File;
+
+/**
+ * Registry actions
+ * 
+ * --instance {app name}, if  a / is in it, refers underneath?
+ * --dest {destfile}
+ * --list : list instances of slider service
+ * --listfiles 
+ */
+@Parameters(commandNames = {ACTION_REGISTRY},
+            commandDescription = DESCRIBE_ACTION_REGISTRY)
+
+public class ActionRegistryArgs extends AbstractActionArgs {
+
+  public static final String USAGE =
+      "Usage: " + SliderActions.ACTION_REGISTRY
+      + " ("
+      + Arguments.ARG_LIST + "|"
+      + Arguments.ARG_LISTCONF + "|"
+      + Arguments.ARG_LISTEXP + "|"
+      + Arguments.ARG_LISTFILES + "|"
+      + Arguments.ARG_GETCONF + "|"
+      + Arguments.ARG_GETEXP + "> "
+      + Arguments.ARG_NAME + " <name> "
+      + " )"
+      + "[" + Arguments.ARG_VERBOSE + "] "
+      + "[" + Arguments.ARG_USER + "] "
+      + "[" + Arguments.ARG_OUTPUT + " <filename> ] "
+      + "[" + Arguments.ARG_SERVICETYPE + " <servicetype> ] "
+      + "[" + Arguments.ARG_FORMAT + " <xml|json|properties>] "
+      + System.getProperty("line.separator")
+      + "Arguments.ARG_GETEXP only supports " + Arguments.ARG_FORMAT + " json"
+      ;
+  public ActionRegistryArgs() {
+  }
+
+  public ActionRegistryArgs(String name) {
+    this.name = name;
+  }
+
+  @Override
+  public String getActionName() {
+    return ACTION_REGISTRY;
+  }
+
+  /**
+   * Get the min #of params expected
+   * @return the min number of params in the {@link #parameters} field
+   */
+  @Override
+  public int getMinParams() {
+    return 0;
+  }
+  
+  @Parameter(names = {ARG_LIST}, 
+      description = "list services")
+  public boolean list;
+
+  @Parameter(names = {ARG_LISTCONF}, 
+      description = "list configurations")
+  public boolean listConf;
+
+  @Parameter(names = {ARG_GETCONF},
+      description = "get configuration")
+  public String getConf;
+
+  @Parameter(names = {ARG_LISTEXP},
+             description = "list exports")
+  public boolean listExports;
+
+  @Parameter(names = {ARG_GETEXP},
+             description = "get export")
+  public String getExport;
+
+  @Parameter(names = {ARG_LISTFILES},
+      description = "list files")
+  public String listFiles;
+
+  @Parameter(names = {ARG_GETFILES},
+      description = "get files")
+  public String getFiles;
+
+  //--format 
+  @Parameter(names = ARG_FORMAT,
+      description = "Format for a response: <xml|json|properties>")
+  public String format = ConfigFormat.XML.toString() ;
+
+  @Parameter(names = {ARG_OUTPUT, ARG_OUTPUT_SHORT, ARG_DEST},
+      description = "Output destination")
+  public File out;
+
+  @Parameter(names = {ARG_NAME},
+      description = "name of an instance")
+  public String name;
+
+  @Parameter(names = {ARG_SERVICETYPE},
+      description = "optional service type")
+  public String serviceType = SliderKeys.APP_TYPE;
+
+  @Parameter(names = {ARG_VERBOSE},
+      description = "verbose output")
+  public boolean verbose;
+
+  @Parameter(names = {ARG_INTERNAL},
+      description = "fetch internal registry entries")
+  public boolean internal;
+
+  @Parameter(names = {ARG_USER},
+      description = "the name of the user whose application is being resolved")
+  public String user;
+
+  /**
+   * validate health of all the different operations
+   * @throws BadCommandArgumentsException
+   */
+  @Override
+  public void validate() throws BadCommandArgumentsException, UsageException {
+    super.validate();
+
+    //verify that at most one of the operations is set
+    int gets = s(getConf) + s(getFiles) + s(getExport);
+    int lists = s(list) + s(listConf) + s(listFiles) + s(listExports);
+    int set = lists + gets;
+    if (set > 1) {
+      throw new UsageException(USAGE);
+    }
+
+    if (out != null && ( set == 0)) {
+      throw new UsageException("output path"
+           + " is only supported on 'get' operations: ");
+    }
+    if (!list && !is(name)) {
+      throw new UsageException("Argument " + ARG_NAME
+           +" missing: ");
+
+    }
+  }
+  
+  private int s(String arg) {
+    return is(arg) ? 1 : 0;
+  }
+
+  private boolean is(String arg) {
+    return arg != null;
+  }
+
+  private int s(boolean arg) {
+    return arg ? 1 : 0;
+  }
+
+  private String ifdef(String arg, boolean val) {
+    return val ? (arg + " "): "";
+  }
+
+  private String ifdef(String arg, String val) {
+    if (is(val)) {
+      return arg + " " + val + " ";
+    } else {
+      return "";
+    }
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder(ACTION_REGISTRY);
+    sb.append(' ');
+    sb.append(ifdef(ARG_LIST, list));
+    sb.append(ifdef(ARG_LISTCONF, listConf));
+    sb.append(ifdef(ARG_LISTFILES, listFiles));
+    sb.append(ifdef(ARG_GETCONF, getConf));
+    sb.append(ifdef(ARG_GETFILES, getFiles));
+
+    sb.append(ifdef(ARG_NAME, name));
+    sb.append(ifdef(ARG_SERVICETYPE, serviceType));
+
+
+    sb.append(ifdef(ARG_VERBOSE, verbose));
+    sb.append(ifdef(ARG_INTERNAL, internal));
+
+    if (out != null) {
+      sb.append(ifdef(ARG_OUTPUT, out.toString()));
+    }
+    sb.append(ifdef(ARG_FORMAT, format));
+
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionResolveArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionResolveArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionResolveArgs.java
new file mode 100644
index 0000000..2ee075a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionResolveArgs.java
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import org.apache.commons.lang.StringUtils;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.UsageException;
+
+import java.io.File;
+
+import static org.apache.slider.common.params.SliderActions.ACTION_RESOLVE;
+import static org.apache.slider.common.params.SliderActions.DESCRIBE_ACTION_REGISTRY;
+
+/**
+ * Resolve registry entries
+ * 
+ * --path {path}
+ * --out {destfile}
+ * --verbose
+ * --list
+ */
+@Parameters(commandNames = {ACTION_RESOLVE},
+            commandDescription = DESCRIBE_ACTION_REGISTRY)
+public class ActionResolveArgs extends AbstractActionArgs {
+
+  public static final String USAGE =
+      "Usage: " + SliderActions.ACTION_RESOLVE
+      + " "
+      + ARG_PATH + " <path> "
+      + "[" + ARG_LIST + "] "
+      + "[" + ARG_OUTPUT + " <filename> ] "
+      + "[" + ARG_DESTDIR + " <directory> ] "
+      ;
+  public ActionResolveArgs() {
+  }
+
+  @Override
+  public String getActionName() {
+    return ACTION_RESOLVE;
+  }
+
+  /**
+   * Get the min #of params expected
+   * @return the min number of params in the {@link #parameters} field
+   */
+  @Override
+  public int getMinParams() {
+    return 0;
+  }
+  
+  @Parameter(names = {ARG_LIST}, 
+      description = "list services")
+  public boolean list;
+
+  @Parameter(names = {ARG_PATH},
+      description = "resolve a path")
+  public String path;
+
+  @Parameter(names = {ARG_DESTDIR},
+      description = "destination directory for operations")
+  public File destdir;
+
+  @Parameter(names = {ARG_OUTPUT, ARG_OUTPUT_SHORT},
+      description = "dest file")
+  public File out;
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder(ACTION_RESOLVE).append(" ");
+    sb.append(ARG_PATH).append(" ").append(path).append(" ");
+    if (list) {
+      sb.append(ARG_LIST).append(" ");
+    }
+    if (destdir != null) {
+      sb.append(ARG_DESTDIR).append(" ").append(destdir).append(" ");
+    }
+    if (out != null) {
+      sb.append(ARG_OUTPUT).append(" ").append(out).append(" ");
+    }
+    return sb.toString();
+  }
+
+  @Override
+  public void validate() throws BadCommandArgumentsException, UsageException {
+    super.validate();
+    if (StringUtils.isEmpty(path)) {
+      throw new BadCommandArgumentsException("Missing mandatory argument "
+                                             + ARG_PATH);
+    }
+    if (list && out != null) {
+      throw new BadCommandArgumentsException("Argument "
+                                             + ARG_OUTPUT +
+                                             " not supported for " + ARG_LIST);
+    }
+    if (out != null && destdir != null) {
+      throw new BadCommandArgumentsException(
+          ARG_OUTPUT + " and " + ARG_DESTDIR + " cannot be used together"
+      );
+    }
+  }
+
+  public String getPath() {
+    return path;
+  }
+
+  public void setPath(String path) {
+    this.path = path;
+  }
+
+  public boolean isList() {
+    return list;
+  }
+
+  public void setList(boolean list) {
+    this.list = list;
+  }
+
+  public File getDestdir() {
+    return destdir;
+  }
+
+  public void setDestdir(File destdir) {
+    this.destdir = destdir;
+  }
+
+  public File getOut() {
+    return out;
+  }
+
+  public void setOut(File out) {
+    this.out = out;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionResourceArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionResourceArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionResourceArgs.java
new file mode 100644
index 0000000..60fcc87
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionResourceArgs.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+
+@Parameters(commandNames = {SliderActions.ACTION_RESOURCE},
+    commandDescription = SliderActions.DESCRIBE_ACTION_RESOURCE)
+
+public class ActionResourceArgs  extends AbstractActionArgs {
+
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_RESOURCE;
+  }
+
+  @Parameter(names = {ARG_INSTALL},
+      description = "Install the resource(s)")
+  public boolean install;
+
+  @Parameter(names = {ARG_DELETE},
+      description = "Delete the file")
+  public boolean delete;
+
+  @Parameter(names = {ARG_LIST},
+      description = "List of installed files")
+  public boolean list;
+
+  @Parameter(names = {ARG_RESOURCE},
+      description = "Name of the file or directory")
+  public String resource;
+
+  @Parameter(names = {ARG_DESTDIR},
+      description = "The name of the folder in which to store the resources")
+  public String folder;
+
+  @Parameter(names = {ARG_OVERWRITE}, description = "Overwrite existing resource(s)")
+  public boolean overwrite = false;
+
+  /**
+   * Get the min #of params expected
+   * @return the min number of params in the {@link #parameters} field
+   */
+  public int getMinParams() {
+    return 0;
+  }
+
+  @Override
+  public int getMaxParams() {
+    return 3;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionStatusArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionStatusArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionStatusArgs.java
new file mode 100644
index 0000000..00178df
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionStatusArgs.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+
+@Parameters(commandNames = {SliderActions.ACTION_STATUS},
+            commandDescription = SliderActions.DESCRIBE_ACTION_STATUS)
+
+public class ActionStatusArgs extends AbstractActionArgs {
+
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_STATUS;
+  }
+
+  @Parameter(names = {ARG_OUTPUT, ARG_OUTPUT_SHORT},
+             description = "Output file for the status information")
+  public String output;
+
+  public String getOutput() {
+    return output;
+  }
+
+  public void setOutput(String output) {
+    this.output = output;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionThawArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionThawArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionThawArgs.java
new file mode 100644
index 0000000..b43a14e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionThawArgs.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameters;
+import com.beust.jcommander.ParametersDelegate;
+
+import java.io.File;
+
+@Parameters(commandNames = {SliderActions.ACTION_THAW},
+            commandDescription = SliderActions.DESCRIBE_ACTION_THAW)
+public class ActionThawArgs extends AbstractActionArgs implements
+                                                       WaitTimeAccessor,
+                                                       LaunchArgsAccessor {
+
+
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_THAW;
+  }
+
+  @Override
+  public int getWaittime() {
+    return launchArgs.getWaittime();
+  }
+
+  @ParametersDelegate
+  LaunchArgsDelegate launchArgs = new LaunchArgsDelegate();
+
+  @Override
+  public String getRmAddress() {
+    return launchArgs.getRmAddress();
+  }
+
+  @Override
+  public void setWaittime(int waittime) {
+    launchArgs.setWaittime(waittime);
+  }
+
+
+  @Override
+  public File getOutputFile() {
+    return launchArgs.getOutputFile();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionTokensArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionTokensArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionTokensArgs.java
new file mode 100644
index 0000000..9f93c4e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionTokensArgs.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.UsageException;
+
+import java.io.File;
+
+@Parameters(commandNames = {SliderActions.ACTION_TOKENS},
+            commandDescription = "save tokens to a file or list tokens in a file")
+public class ActionTokensArgs extends AbstractActionArgs {
+
+  public static final String DUPLICATE_ARGS = "Only one of " +
+      ARG_SOURCE + " and " + ARG_OUTPUT + " allowed";
+
+  public static final String MISSING_KT_PROVIDER =
+      "Both " + ARG_KEYTAB + " and " + ARG_PRINCIPAL
+      + " must be provided";
+
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_TOKENS;
+  }
+
+  @Parameter(names = {ARG_OUTPUT},
+             description = "File to write")
+  public File output;
+
+  @Parameter(names = {ARG_SOURCE},
+             description = "source file")
+  public File source;
+
+  @Parameter(names = {ARG_KEYTAB}, description = "keytab to use")
+  public File keytab;
+
+  @Parameter(names = {ARG_PRINCIPAL}, description = "principal to log in from a keytab")
+  public String principal="";
+
+  /**
+   * Get the min #of params expected
+   * @return the min number of params in the {@link #parameters} field
+   */
+  public int getMinParams() {
+    return 0;
+  }
+
+  @Override
+  public void validate() throws BadCommandArgumentsException, UsageException {
+    super.validate();
+    if (output != null && source != null) {
+      throw new BadCommandArgumentsException(DUPLICATE_ARGS);
+    }
+
+    // this is actually a !xor
+    if (keytab != null ^ !principal.isEmpty()) {
+      throw new BadCommandArgumentsException(MISSING_KT_PROVIDER);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionUpdateArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionUpdateArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionUpdateArgs.java
new file mode 100644
index 0000000..9d76bd8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionUpdateArgs.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameters;
+
+@Parameters(commandNames = {SliderActions.ACTION_UPDATE},
+            commandDescription = SliderActions.DESCRIBE_ACTION_UPDATE)
+
+public class ActionUpdateArgs extends AbstractClusterBuildingActionArgs {
+
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_UPDATE;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionUpgradeArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionUpgradeArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionUpgradeArgs.java
new file mode 100644
index 0000000..6ef51b2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionUpgradeArgs.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.beust.jcommander.ParametersDelegate;
+
+@Parameters(commandNames = { SliderActions.ACTION_UPGRADE },
+            commandDescription = SliderActions.DESCRIBE_ACTION_UPGRADE)
+public class ActionUpgradeArgs extends AbstractClusterBuildingActionArgs
+    implements WaitTimeAccessor, LaunchArgsAccessor {
+
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_UPGRADE;
+  }
+
+  @ParametersDelegate
+  LaunchArgsDelegate launchArgs = new LaunchArgsDelegate();
+
+  @Override
+  public File getOutputFile() {
+    return launchArgs.getOutputFile();
+  }
+
+  @Override
+  public String getRmAddress() {
+    return launchArgs.getRmAddress();
+  }
+
+  @Override
+  public int getWaittime() {
+    return launchArgs.getWaittime();
+  }
+
+  @Override
+  public void setWaittime(int waittime) {
+    launchArgs.setWaittime(waittime);
+  }
+
+  @Parameter(names={ARG_CONTAINERS}, variableArity = true,
+             description = "stop specific containers")
+  public List<String> containers = new ArrayList<>(0);
+
+  @Parameter(names={ARG_COMPONENTS}, variableArity = true,
+      description = "stop all containers of specific components")
+  public List<String> components = new ArrayList<>(0);
+
+  @Parameter(names = {ARG_FORCE},
+      description = "force spec upgrade operation")
+  public boolean force;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionVersionArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionVersionArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionVersionArgs.java
new file mode 100644
index 0000000..b9d212b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionVersionArgs.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameters;
+
+/**
+ * The version command
+ */
+@Parameters(commandNames = {SliderActions.ACTION_VERSION},
+            commandDescription = SliderActions.DESCRIBE_ACTION_VERSION)
+public class ActionVersionArgs extends AbstractActionArgs {
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_VERSION;
+  }
+
+  public int getMinParams() {
+    return 0;
+  }
+
+  /**
+   * This action does not need hadoop services
+   * @return false
+   */
+  @Override
+  public boolean getHadoopServicesRequired() {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AddonArgsDelegate.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AddonArgsDelegate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AddonArgsDelegate.java
new file mode 100644
index 0000000..3ef8e19
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AddonArgsDelegate.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+public class AddonArgsDelegate extends AbstractArgsDelegate {
+
+  /**
+   * This is a listing of addon packages
+   */
+  @Parameter(names = {ARG_ADDON},
+      arity = 2,
+      description = "--addon <name> <folder or package>",
+      splitter = DontSplitArguments.class)
+  public List<String> addonTuples = new ArrayList<>(0);
+
+
+  /**
+   * Get the list of addons (may be empty, but never null)
+   *
+   * @return map of named addons
+   *
+   * @throws BadCommandArgumentsException parse problem
+   */
+  public Map<String, String> getAddonMap() throws BadCommandArgumentsException {
+    return convertTupleListToMap("addon", addonTuples);
+  }
+
+  public List<String> getAddonTuples() {
+    return addonTuples;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AppAndResouceOptionArgsDelegate.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AppAndResouceOptionArgsDelegate.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AppAndResouceOptionArgsDelegate.java
new file mode 100644
index 0000000..f171708
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AppAndResouceOptionArgsDelegate.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Delegate for application and resource options
+ */
+public class AppAndResouceOptionArgsDelegate extends AbstractArgsDelegate {
+
+
+  /**
+   * Options key value
+   */
+  @Parameter(names = {ARG_OPTION, ARG_OPTION_SHORT}, arity = 2,
+             description = ARG_OPTION + "<name> <value>",
+             splitter = DontSplitArguments.class)
+  public List<String> optionTuples = new ArrayList<>(0);
+
+
+  /**
+   * All the app component option triples
+   */
+  @Parameter(names = {ARG_COMP_OPT,  ARG_COMP_OPT_SHORT,  ARG_ROLEOPT}, arity = 3,
+             description = "Component option " + ARG_COMP_OPT +
+                           " <component> <name> <option>",
+             splitter = DontSplitArguments.class)
+  public List<String> compOptTriples = new ArrayList<>(0);
+
+  /**
+   * Resource Options
+   */
+  @Parameter(names = {ARG_RESOURCE_OPT, ARG_RESOURCE_OPT_SHORT}, arity = 2,
+             description = "Resource option "+ ARG_RESOURCE_OPT + "<name> <value>",
+             splitter = DontSplitArguments.class)
+  public List<String> resOptionTuples = new ArrayList<>(0);
+
+
+  /**
+   * All the resource component option triples
+   */
+  @Parameter(names = {ARG_RES_COMP_OPT, ARG_RES_COMP_OPT_SHORT,}, arity = 3,
+             description = "Component resource option " + ARG_RES_COMP_OPT +
+                           " <component> <name> <option>",
+             splitter = DontSplitArguments.class)
+  public List<String> resCompOptTriples = new ArrayList<>(0);
+
+
+  public Map<String, String> getOptionsMap() throws
+                                             BadCommandArgumentsException {
+    return convertTupleListToMap(ARG_OPTION, optionTuples);
+  }
+
+  /**
+   * Get the role heap mapping (may be empty, but never null)
+   * @return role heap mapping
+   * @throws BadCommandArgumentsException parse problem
+   */
+  public Map<String, Map<String, String>> getCompOptionMap() throws
+                                                             BadCommandArgumentsException {
+    return convertTripleListToMaps(ARG_COMP_OPT, compOptTriples);
+  }
+
+  public Map<String, String> getResourceOptionsMap() throws
+                                             BadCommandArgumentsException {
+    return convertTupleListToMap(ARG_RESOURCE_OPT, resOptionTuples);
+  }
+
+  /**
+   * Get the role heap mapping (may be empty, but never null)
+   * @return role heap mapping
+   * @throws BadCommandArgumentsException parse problem
+   */
+  public Map<String, Map<String, String>> getResourceCompOptionMap() throws
+                                                             BadCommandArgumentsException {
+    return convertTripleListToMaps(ARG_RES_COMP_OPT, resCompOptTriples);
+  }
+
+  public void setOption(String key, String value) {
+    optionTuples.add(key);
+    optionTuples.add(value);
+  }
+
+  public void setResourceOption(String key, String value) {
+    resOptionTuples.add(key);
+    resOptionTuples.add(value);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ArgOps.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ArgOps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ArgOps.java
new file mode 100644
index 0000000..12a2032
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ArgOps.java
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.ErrorStrings;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Static argument manipulation operations
+ */
+public class ArgOps {
+
+  private static final Logger
+    log = LoggerFactory.getLogger(ArgOps.class);
+
+  /**
+   * create a 3-tuple
+   */
+  public static List<Object> triple(String msg, int min, int max) {
+    List<Object> l = new ArrayList<>(3);
+    l.add(msg);
+    l.add(min);
+    l.add(max);
+    return l;
+  }
+
+  public static void applyFileSystemBinding(String filesystemBinding,
+      Configuration conf) {
+    if (filesystemBinding != null) {
+      //filesystem argument was set -this overwrites any defaults in the
+      //configuration
+      FileSystem.setDefaultUri(conf, filesystemBinding);
+    }
+  }
+
+  public static void splitPairs(Collection<String> pairs,
+                                Map<String, String> dest) {
+    for (String prop : pairs) {
+      String[] keyval = prop.split("=", 2);
+      if (keyval.length == 2) {
+        dest.put(keyval[0], keyval[1]);
+      }
+    }
+  }
+
+
+  public static void applyDefinitions(Map<String, String> definitionMap,
+                                      Configuration conf) {
+    for (Map.Entry<String, String> entry : definitionMap.entrySet()) {
+      String key = entry.getKey();
+      String val = entry.getValue();
+      log.debug("configuration[{}]<=\"{}\"", key, val);
+      conf.set(key, val, "command line");
+    }
+  }
+
+  /**
+   * Create a map from a tuple list like ['worker','2','master','1] into a map
+   * ['worker':'2',"master":'1'];
+   * Duplicate entries also trigger errors
+   * @param description description for errors
+   * @param list list to conver to tuples
+   * @return the map of key value pairs -unordered.
+   * @throws BadCommandArgumentsException odd #of arguments received
+   */
+  public static Map<String, String> convertTupleListToMap(String description,
+                                                          List<String> list) throws
+                                                                             BadCommandArgumentsException {
+    Map<String, String> results = new HashMap<>();
+    if (list != null && !list.isEmpty()) {
+      int size = list.size();
+      if (size % 2 != 0) {
+        //odd number of elements, not permitted
+        throw new BadCommandArgumentsException(
+          ErrorStrings.ERROR_PARSE_FAILURE + description);
+      }
+      for (int count = 0; count < size; count += 2) {
+        String key = list.get(count);
+        String val = list.get(count + 1);
+        if (results.get(key) != null) {
+          throw new BadCommandArgumentsException(
+            ErrorStrings.ERROR_DUPLICATE_ENTRY + description
+            + ": " + key);
+        }
+        results.put(key, val);
+      }
+    }
+    return results;
+  }
+
+  /**
+   * Create a map from a tuple list like
+   * ['worker','heapsize','5G','master','heapsize','2M'] into a map
+   * ['worker':'2',"master":'1'];
+   * Duplicate entries also trigger errors
+
+   * @throws BadCommandArgumentsException odd #of arguments received
+   */
+  public static Map<String, Map<String, String>> convertTripleListToMaps(String description,
+         List<String> list) throws BadCommandArgumentsException {
+
+    Map<String, Map<String, String>> results = new HashMap<>();
+    if (list != null && !list.isEmpty()) {
+      int size = list.size();
+      if (size % 3 != 0) {
+        //wrong number of elements, not permitted
+        throw new BadCommandArgumentsException(
+          ErrorStrings.ERROR_PARSE_FAILURE + description);
+      }
+      for (int count = 0; count < size; count += 3) {
+        String role = list.get(count);
+        String key = list.get(count + 1);
+        String val = list.get(count + 2);
+        Map<String, String> roleMap = results.get(role);
+        if (roleMap == null) {
+          //demand create new role map
+          roleMap = new HashMap<>();
+          results.put(role, roleMap);
+        }
+        if (roleMap.get(key) != null) {
+          throw new BadCommandArgumentsException(
+            ErrorStrings.ERROR_DUPLICATE_ENTRY + description
+            + ": for key " + key + " under " + role);
+        }
+        roleMap.put(key, val);
+      }
+    }
+    return results;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/Arguments.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
new file mode 100644
index 0000000..aec4e26
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+/**
+ * Here are all the arguments that may be parsed by the client or server
+ * command lines. 
+ * 
+ * Important: Please keep the main list in alphabetical order
+ * so it is easier to see what arguments are there
+ */
+public interface Arguments {
+  String ARG_ADDON = "--addon";
+  String ARG_ALL = "--all";
+  String ARG_ALIAS = "--alias";
+  String ARG_APPLICATION = "--application";
+  String ARG_APPDEF = "--appdef";
+  String ARG_APP_HOME = "--apphome";
+  String ARG_BASE_PATH = "--basepath";
+  String ARG_CLIENT = "--client";
+  String ARG_CONFDIR = "--appconf";
+  String ARG_COMPONENT = "--component";
+  String ARG_COMPONENT_SHORT = "--comp";
+  String ARG_COMPONENTS = "--components";
+  String ARG_COMP_OPT= "--compopt";
+  String ARG_COMP_OPT_SHORT = "--co";
+  String ARG_CONFIG = "--config";
+  String ARG_CONTAINERS = "--containers";
+  String ARG_CREDENTIALS = "--credentials";
+  String ARG_DEBUG = "--debug";
+  String ARG_DEFINE = "-D";
+  String ARG_DELETE = "--delete";
+  String ARG_DEST = "--dest";
+  String ARG_DESTDIR = "--destdir";
+  String ARG_DESTFILE = "--destfile";
+  String ARG_EXITCODE = "--exitcode";
+  String ARG_FAIL = "--fail";
+  /**
+   filesystem-uri: {@value}
+   */
+  String ARG_FILESYSTEM = "--fs";
+  String ARG_FILESYSTEM_LONG = "--filesystem";
+  String ARG_FOLDER = "--folder";
+  String ARG_FORCE = "--force";
+  String ARG_FORMAT = "--format";
+  String ARG_GETCERTSTORE = "--getcertstore";
+  String ARG_GETCONF = "--getconf";
+  String ARG_GETEXP = "--getexp";
+  String ARG_GETFILES = "--getfiles";
+  String ARG_HEALTHY= "--healthy";
+  String ARG_HELP = "--help";
+  String ARG_HOSTNAME = "--hostname";
+  String ARG_ID = "--id";
+  String ARG_IMAGE = "--image";
+  String ARG_INSTALL = "--install";
+  String ARG_INTERNAL = "--internal";
+  String ARG_KEYLEN = "--keylen";
+  String ARG_KEYTAB = "--keytab";
+  String ARG_KEYSTORE = "--keystore";
+  String ARG_KEYTABINSTALL = ARG_INSTALL;
+  String ARG_KEYTABDELETE = ARG_DELETE;
+  String ARG_KEYTABLIST = "--list";
+  String ARG_LABEL = "--label";
+  String ARG_LEVEL = "--level";
+  String ARG_LIST = "--list";
+  String ARG_LISTCONF = "--listconf";
+  String ARG_LISTEXP = "--listexp";
+  String ARG_LISTFILES = "--listfiles";
+  String ARG_LIVE = "--live";
+  String ARG_MANAGER = "--manager";
+  String ARG_MANAGER_SHORT = "--m";
+  String ARG_MESSAGE = "--message";
+  String ARG_METAINFO = "--metainfo";
+  String ARG_METAINFO_JSON = "--metainfojson";
+  String ARG_NAME = "--name";
+  String ARG_OPTION = "--option";
+  String ARG_OPTION_SHORT = "-O";
+  String ARG_OUTPUT = "--out";
+  String ARG_OUTPUT_SHORT = "-o";
+  String ARG_OVERWRITE = "--overwrite";
+  String ARG_PACKAGE = "--package";
+  String ARG_PASSWORD = "--password";
+  String ARG_PATH = "--path";
+  String ARG_PKGDELETE = ARG_DELETE;
+  String ARG_PKGINSTANCES = "--instances";
+  String ARG_PKGLIST = ARG_LIST;
+  String ARG_PRINCIPAL = "--principal";
+  String ARG_PROVIDER = "--provider";
+  String ARG_QUEUE = "--queue";
+  String ARG_REPLACE_PKG = "--replacepkg";
+  String ARG_RESOURCE = "--resource";
+  String ARG_RESOURCES = "--resources";
+  String ARG_RES_COMP_OPT = "--rescompopt";
+  String ARG_RES_COMP_OPT_SHORT = "--rco";
+  String ARG_RESOURCE_MANAGER = "--rm";
+  String ARG_RESOURCE_OPT = "--resopt";
+  String ARG_RESOURCE_OPT_SHORT = "-ro";
+  String ARG_SECURE = "--secure";
+  String ARG_SERVICETYPE = "--servicetype";
+  String ARG_SERVICES = "--services";
+  String ARG_SLIDER = "--slider";
+  String ARG_SOURCE = "--source";
+  String ARG_STATE = "--state";
+  String ARG_SYSPROP = "-S";
+  String ARG_TEMPLATE = "--template";
+  String ARG_TRUSTSTORE = "--truststore";
+  String ARG_USER = "--user";
+  String ARG_UPLOAD = "--upload";
+  String ARG_VERBOSE = "--verbose";
+  String ARG_VERSION = "--version";
+  String ARG_WAIT = "--wait";
+  String ARG_YARN = "--yarn";
+  String ARG_ZKHOSTS = "--zkhosts";
+  String ARG_ZKPATH = "--zkpath";
+  String ARG_ZKPORT = "--zkport";
+/*
+ STOP: DO NOT ADD YOUR ARGUMENTS HERE. GO BACK AND INSERT THEM IN THE
+ RIGHT PLACE IN THE LIST
+ */
+
+
+  /**
+   * Deprecated: use ARG_COMPONENT
+   */
+  @Deprecated
+  String ARG_ROLE = "--role";
+
+  /**
+   * Deprecated: use ARG_COMP_OPT
+   */
+  @Deprecated
+  String ARG_ROLEOPT = "--roleopt";
+
+  /**
+   * server: URI for the cluster
+   */
+  String ARG_CLUSTER_URI = "-cluster-uri";
+
+
+  /**
+   * server: Path for the resource manager instance (required)
+   */
+  String ARG_RM_ADDR = "--rm";
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java
new file mode 100644
index 0000000..4016cc9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.slider.common.SliderXmlConfKeys;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.ErrorStrings;
+import org.apache.slider.core.exceptions.SliderException;
+
+import java.util.Collection;
+
+/**
+ * Slider Client CLI Args
+ */
+
+public class ClientArgs extends CommonArgs {
+
+  /*
+   
+   All the arguments for specific actions
+  
+   */
+  /**
+   * This is not bonded to jcommander, it is set up
+   * after the construction to point to the relevant
+   * entry
+   * 
+   * KEEP IN ALPHABETICAL ORDER
+   */
+  private AbstractClusterBuildingActionArgs buildingActionArgs;
+
+  // =========================================================
+  // Keep all of these in alphabetical order. Thanks.
+  // =========================================================
+
+  private final ActionAMSuicideArgs actionAMSuicideArgs = new ActionAMSuicideArgs();
+  private final ActionBuildArgs actionBuildArgs = new ActionBuildArgs();
+  private final ActionClientArgs actionClientArgs = new ActionClientArgs();
+  private final ActionCreateArgs actionCreateArgs = new ActionCreateArgs();
+  private final ActionDependencyArgs actionDependencyArgs = new ActionDependencyArgs();
+  private final ActionDestroyArgs actionDestroyArgs = new ActionDestroyArgs();
+  private final ActionDiagnosticArgs actionDiagnosticArgs = new ActionDiagnosticArgs();
+  private final ActionExistsArgs actionExistsArgs = new ActionExistsArgs();
+  private final ActionFlexArgs actionFlexArgs = new ActionFlexArgs();
+  private final ActionFreezeArgs actionFreezeArgs = new ActionFreezeArgs();
+  private final ActionHelpArgs actionHelpArgs = new ActionHelpArgs();
+  private final ActionInstallPackageArgs actionInstallPackageArgs = new ActionInstallPackageArgs();
+  private final ActionInstallKeytabArgs actionInstallKeytabArgs = new ActionInstallKeytabArgs();
+  private final ActionKDiagArgs actionKDiagArgs = new ActionKDiagArgs();
+  private final ActionKeytabArgs actionKeytabArgs = new ActionKeytabArgs();
+  private final ActionKillContainerArgs actionKillContainerArgs =
+    new ActionKillContainerArgs();
+  private final ActionListArgs actionListArgs = new ActionListArgs();
+  private final ActionLookupArgs actionLookupArgs = new ActionLookupArgs();
+  private final ActionNodesArgs actionNodesArgs = new ActionNodesArgs();
+  private final ActionPackageArgs actionPackageArgs = new ActionPackageArgs();
+  private final ActionRegistryArgs actionRegistryArgs = new ActionRegistryArgs();
+  private final ActionResolveArgs actionResolveArgs = new ActionResolveArgs();
+  private final ActionResourceArgs actionResourceArgs = new ActionResourceArgs();
+  private final ActionStatusArgs actionStatusArgs = new ActionStatusArgs();
+  private final ActionThawArgs actionThawArgs = new ActionThawArgs();
+  private final ActionTokensArgs actionTokenArgs = new ActionTokensArgs();
+  private final ActionUpdateArgs actionUpdateArgs = new ActionUpdateArgs();
+  private final ActionUpgradeArgs actionUpgradeArgs = new ActionUpgradeArgs();
+  private final ActionVersionArgs actionVersionArgs = new ActionVersionArgs();
+
+  public ClientArgs(String[] args) {
+    super(args);
+  }
+
+  public ClientArgs(Collection args) {
+    super(args);
+  }
+
+  @Override
+  protected void addActionArguments() {
+
+    addActions(
+        actionAMSuicideArgs,
+        actionBuildArgs,
+        actionClientArgs,
+        actionCreateArgs,
+        actionDependencyArgs,
+        actionDestroyArgs,
+        actionDiagnosticArgs,
+        actionExistsArgs,
+        actionFlexArgs,
+        actionFreezeArgs,
+        actionHelpArgs,
+        actionInstallKeytabArgs,
+        actionInstallPackageArgs,
+        actionKDiagArgs,
+        actionKeytabArgs,
+        actionKillContainerArgs,
+        actionListArgs,
+        actionLookupArgs,
+        actionNodesArgs,
+        actionPackageArgs,
+        actionRegistryArgs,
+        actionResolveArgs,
+        actionResourceArgs,
+        actionStatusArgs,
+        actionThawArgs,
+        actionTokenArgs,
+        actionUpdateArgs,
+        actionUpgradeArgs,
+        actionVersionArgs
+    );
+  }
+
+  @Override
+  public void applyDefinitions(Configuration conf) throws
+                                                   BadCommandArgumentsException {
+    super.applyDefinitions(conf);
+    //RM
+    if (getManager() != null) {
+      log.debug("Setting RM to {}", getManager());
+      conf.set(YarnConfiguration.RM_ADDRESS, getManager());
+    }
+    if (getBasePath() != null) {
+      log.debug("Setting basePath to {}", getBasePath());
+      conf.set(SliderXmlConfKeys.KEY_SLIDER_BASE_PATH,
+          getBasePath().toString());
+    }
+  }
+
+  public ActionDiagnosticArgs getActionDiagnosticArgs() {
+	  return actionDiagnosticArgs;
+  }
+
+  public AbstractClusterBuildingActionArgs getBuildingActionArgs() {
+    return buildingActionArgs;
+  }
+
+  public ActionAMSuicideArgs getActionAMSuicideArgs() {
+    return actionAMSuicideArgs;
+  }
+
+  public ActionBuildArgs getActionBuildArgs() {
+    return actionBuildArgs;
+  }
+
+  public ActionInstallPackageArgs getActionInstallPackageArgs() { return actionInstallPackageArgs; }
+
+  public ActionClientArgs getActionClientArgs() { return actionClientArgs; }
+
+  public ActionPackageArgs getActionPackageArgs() { return actionPackageArgs; }
+
+  public ActionInstallKeytabArgs getActionInstallKeytabArgs() { return actionInstallKeytabArgs; }
+
+  public ActionKDiagArgs getActionKDiagArgs() {
+    return actionKDiagArgs;
+  }
+
+  public ActionKeytabArgs getActionKeytabArgs() { return actionKeytabArgs; }
+
+  public ActionUpdateArgs getActionUpdateArgs() {
+    return actionUpdateArgs;
+  }
+
+  public ActionUpgradeArgs getActionUpgradeArgs() {
+    return actionUpgradeArgs;
+  }
+
+  public ActionCreateArgs getActionCreateArgs() {
+    return actionCreateArgs;
+  }
+
+  public ActionDependencyArgs getActionDependencyArgs() {
+    return actionDependencyArgs;
+  }
+
+  public ActionDestroyArgs getActionDestroyArgs() {
+    return actionDestroyArgs;
+  }
+
+  public ActionExistsArgs getActionExistsArgs() {
+    return actionExistsArgs;
+  }
+
+  public ActionFlexArgs getActionFlexArgs() {
+    return actionFlexArgs;
+  }
+
+  public ActionFreezeArgs getActionFreezeArgs() {
+    return actionFreezeArgs;
+  }
+
+  public ActionKillContainerArgs getActionKillContainerArgs() {
+    return actionKillContainerArgs;
+  }
+
+  public ActionListArgs getActionListArgs() {
+    return actionListArgs;
+  }
+
+  public ActionNodesArgs getActionNodesArgs() {
+    return actionNodesArgs;
+  }
+
+  public ActionLookupArgs getActionLookupArgs() {
+    return actionLookupArgs;
+  }
+
+  public ActionRegistryArgs getActionRegistryArgs() {
+    return actionRegistryArgs;
+  }
+
+  public ActionResolveArgs getActionResolveArgs() {
+    return actionResolveArgs;
+  }
+
+  public ActionResourceArgs getActionResourceArgs() {
+    return actionResourceArgs;
+  }
+
+  public ActionStatusArgs getActionStatusArgs() {
+    return actionStatusArgs;
+  }
+
+  public ActionThawArgs getActionThawArgs() {
+    return actionThawArgs;
+  }
+
+  public ActionTokensArgs getActionTokenArgs() {
+    return actionTokenArgs;
+  }
+
+  /**
+   * Look at the chosen action and bind it as the core action for the operation.
+   * @throws SliderException bad argument or similar
+   */
+  @Override
+  public void applyAction() throws SliderException {
+    String action = getAction();
+    if (SliderUtils.isUnset(action)) {
+      action = ACTION_HELP;
+    }
+    switch (action) {
+      case ACTION_BUILD:
+        bindCoreAction(actionBuildArgs);
+        //its a builder, so set those actions too
+        buildingActionArgs = actionBuildArgs;
+        break;
+
+      case ACTION_CREATE:
+        bindCoreAction(actionCreateArgs);
+        //its a builder, so set those actions too
+        buildingActionArgs = actionCreateArgs;
+        break;
+
+      case ACTION_FREEZE:
+        bindCoreAction(actionFreezeArgs);
+        break;
+
+      case ACTION_THAW:
+        bindCoreAction(actionThawArgs);
+        break;
+
+      case ACTION_AM_SUICIDE:
+        bindCoreAction(actionAMSuicideArgs);
+        break;
+
+      case ACTION_CLIENT:
+        bindCoreAction(actionClientArgs);
+        break;
+
+      case ACTION_DEPENDENCY:
+        bindCoreAction(actionDependencyArgs);
+        break;
+
+      case ACTION_DESTROY:
+        bindCoreAction(actionDestroyArgs);
+        break;
+
+      case ACTION_DIAGNOSTICS:
+        bindCoreAction(actionDiagnosticArgs);
+        break;
+
+      case ACTION_EXISTS:
+        bindCoreAction(actionExistsArgs);
+        break;
+
+      case ACTION_FLEX:
+        bindCoreAction(actionFlexArgs);
+        break;
+
+      case ACTION_HELP:
+        bindCoreAction(actionHelpArgs);
+        break;
+
+      case ACTION_INSTALL_KEYTAB:
+        bindCoreAction(actionInstallKeytabArgs);
+        break;
+
+      case ACTION_INSTALL_PACKAGE:
+        bindCoreAction(actionInstallPackageArgs);
+        break;
+
+      case ACTION_KDIAG:
+        bindCoreAction(actionKDiagArgs);
+        break;
+
+      case ACTION_KEYTAB:
+        bindCoreAction(actionKeytabArgs);
+        break;
+
+      case ACTION_KILL_CONTAINER:
+        bindCoreAction(actionKillContainerArgs);
+        break;
+
+      case ACTION_LIST:
+        bindCoreAction(actionListArgs);
+        break;
+
+      case ACTION_LOOKUP:
+        bindCoreAction(actionLookupArgs);
+        break;
+
+      case ACTION_NODES:
+        bindCoreAction(actionNodesArgs);
+        break;
+
+      case ACTION_PACKAGE:
+        bindCoreAction(actionPackageArgs);
+        break;
+
+      case ACTION_REGISTRY:
+        bindCoreAction(actionRegistryArgs);
+        break;
+
+      case ACTION_RESOLVE:
+        bindCoreAction(actionResolveArgs);
+        break;
+
+      case ACTION_RESOURCE:
+        bindCoreAction(actionResourceArgs);
+        break;
+
+      case ACTION_STATUS:
+        bindCoreAction(actionStatusArgs);
+        break;
+
+      case ACTION_TOKENS:
+        bindCoreAction(actionTokenArgs);
+        break;
+
+      case ACTION_UPDATE:
+        bindCoreAction(actionUpdateArgs);
+        break;
+
+      case ACTION_UPGRADE:
+        bindCoreAction(actionUpgradeArgs);
+        break;
+
+      case ACTION_VERSION:
+        bindCoreAction(actionVersionArgs);
+        break;
+
+      default:
+        throw new BadCommandArgumentsException(ErrorStrings.ERROR_UNKNOWN_ACTION
+        + " " + action);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/CommonArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/CommonArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/CommonArgs.java
new file mode 100644
index 0000000..162a87d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/CommonArgs.java
@@ -0,0 +1,303 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterDescription;
+import com.beust.jcommander.ParameterException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.ErrorStrings;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.core.exceptions.UsageException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * This class contains the common argument set for all tne entry points,
+ * and the core parsing logic to verify that the action is on the list
+ * of allowed actions -and that the remaining number of arguments is
+ * in the range allowed
+ */
+
+public abstract class CommonArgs extends ArgOps implements SliderActions,
+                                                           Arguments {
+
+  protected static final Logger log = LoggerFactory.getLogger(CommonArgs.class);
+
+
+  private static final int DIFF_BETWEEN_DESCIPTION_AND_COMMAND_NAME = 30;
+
+
+  @Parameter(names = ARG_HELP, help = true)
+  public boolean help;
+
+
+  /**
+   -D name=value
+
+   Define an HBase configuration option which overrides any options in
+   the configuration XML files of the image or in the image configuration
+   directory. The values will be persisted.
+   Configuration options are only passed to the cluster when creating or reconfiguring a cluster.
+
+   */
+
+  public Map<String, String> definitionMap = new HashMap<String, String>();
+  /**
+   * System properties
+   */
+  public Map<String, String> syspropsMap = new HashMap<String, String>();
+
+
+  /**
+   * fields
+   */
+  public final JCommander commander;
+  private final String[] args;
+
+  private AbstractActionArgs coreAction;
+
+  /**
+   * get the name: relies on arg 1 being the cluster name in all operations 
+   * @return the name argument, null if there is none
+   */
+  public String getClusterName() {
+    return coreAction.getClusterName();
+  }
+
+  protected CommonArgs(String[] args) {
+    this.args = args;
+    commander = new JCommander(this);
+  }
+
+  protected CommonArgs(Collection args) {
+    List<String> argsAsStrings = SliderUtils.collectionToStringList(args);
+    this.args = argsAsStrings.toArray(new String[argsAsStrings.size()]);
+    commander = new JCommander(this);
+  }
+
+  public String usage() {
+    return usage(this, null);
+  }
+
+  public static String usage(CommonArgs serviceArgs, String commandOfInterest) {
+    String result = null;
+    StringBuilder helperMessage = new StringBuilder();
+    if (commandOfInterest == null) {
+      // JCommander.usage is too verbose for a command with many options like
+      // slider no short version of that is found Instead, we compose our msg by
+      helperMessage.append("\nUsage: slider COMMAND [options]\n");
+      helperMessage.append("where COMMAND is one of\n");
+      for (String jcommand : serviceArgs.commander.getCommands().keySet()) {
+        helperMessage.append(String.format("\t%-"
+            + DIFF_BETWEEN_DESCIPTION_AND_COMMAND_NAME + "s%s", jcommand,
+            serviceArgs.commander.getCommandDescription(jcommand) + "\n"));
+      }
+      helperMessage
+          .append("Most commands print help when invoked without parameters or with --help");
+      result = helperMessage.toString();
+    } else {
+      helperMessage.append("\nUsage: slider ").append(commandOfInterest);
+      helperMessage.append(serviceArgs.coreAction.getMinParams() > 0 ? " <application>" : "");
+      helperMessage.append("\n");
+      for (ParameterDescription paramDesc : serviceArgs.commander.getCommands()
+          .get(commandOfInterest).getParameters()) {
+        String optional = paramDesc.getParameter().required() ? "  (required)"
+            : "  (optional)";
+        String paramName = paramDesc.getParameterized().getType() == Boolean.TYPE ? paramDesc
+            .getLongestName() : paramDesc.getLongestName() + " <"
+            + paramDesc.getParameterized().getName() + ">";
+        helperMessage.append(String.format("\t%-"
+            + DIFF_BETWEEN_DESCIPTION_AND_COMMAND_NAME + "s%s", paramName,
+            paramDesc.getDescription() + optional + "\n"));
+        result = helperMessage.toString();
+      }
+    }
+    return result;
+  }
+
+  public static String usage(CommonArgs serviceArgs) {
+    return usage(serviceArgs, null);
+  }
+
+  /**
+   * Parse routine -includes registering the action-specific argument classes
+   * and postprocess it
+   * @throws SliderException on any problem
+   */
+  public void parse() throws SliderException {
+    addActionArguments();
+    try {
+      commander.parse(getArgs());
+    } catch (ParameterException e) {
+      throw new BadCommandArgumentsException(e, "%s in %s",
+                                             e.toString(),
+                                             (getArgs() != null
+                                              ? (SliderUtils.join(getArgs(),
+                                                 " ", false))
+                                              : "[]"));
+    }
+    //now copy back to this class some of the attributes that are common to all
+    //actions
+    postProcess();
+  }
+
+  /**
+   * Add a command
+   * @param name action
+   * @param arg value
+   */
+  protected void addAction(String name, Object arg) {
+    commander.addCommand(name, arg);
+  }
+
+  protected void addActions(Object... actions) {
+    for (Object action : actions) {
+      commander.addCommand(action);
+    }
+  }
+
+  /**
+   * Override point to add a set of actions
+   */
+  protected void addActionArguments() {
+
+  }
+
+  /**
+   * validate args via {@link #validate()}
+   * then postprocess the arguments
+   */
+  public void postProcess() throws SliderException {
+    applyAction();
+    validate();
+
+    //apply entry set
+    for (Map.Entry<String, String> entry : syspropsMap.entrySet()) {
+      System.setProperty(entry.getKey(), entry.getValue());
+    }
+  }
+
+
+  /**
+   * Implementors must implement their action apply routine here
+   */
+  public abstract void applyAction() throws SliderException;
+
+
+  /**
+   * Bind the core action; this extracts any attributes that are used
+   * across routines
+   * @param action action to bind
+   */
+  protected void bindCoreAction(AbstractActionArgs action) {
+    coreAction = action;
+
+    splitPairs(coreAction.definitions, definitionMap);
+    splitPairs(coreAction.sysprops, syspropsMap);
+  }
+
+  /**
+   * Get the core action -type depends on the action
+   * @return the action class
+   */
+  public AbstractActionArgs getCoreAction() {
+    return coreAction;
+  }
+
+  /**
+   * Validate the arguments against the action requested
+   */
+  public void validate() throws BadCommandArgumentsException, UsageException {
+    if (coreAction == null) {
+      throw new UsageException(ErrorStrings.ERROR_NO_ACTION + usage());
+    }
+    log.debug("action={}", getAction());
+    // let the action validate itself
+    try {
+      coreAction.validate();
+    } catch (BadCommandArgumentsException e) {
+      StringBuilder badArgMsgBuilder = new StringBuilder();
+      badArgMsgBuilder.append(e.toString()).append("\n");
+      badArgMsgBuilder.append(usage(this, coreAction.getActionName()));
+      throw new BadCommandArgumentsException(badArgMsgBuilder.toString());
+    } catch (UsageException e) {
+      StringBuilder badArgMsgBuilder = new StringBuilder();
+      badArgMsgBuilder.append(e.toString()).append("\n");
+      badArgMsgBuilder.append(usage(this, coreAction.getActionName()));
+      throw new UsageException(badArgMsgBuilder.toString());
+    }
+  }
+
+  /**
+   * Apply all the definitions on the command line to the configuration
+   * @param conf config
+   */
+  public void applyDefinitions(Configuration conf) throws
+                                                   BadCommandArgumentsException {
+    applyDefinitions(definitionMap, conf);
+  }
+
+
+  /**
+   * If the Filesystem binding was provided, it overrides anything in
+   * the configuration
+   * @param conf configuration
+   */
+  public void applyFileSystemBinding(Configuration conf) {
+    ArgOps.applyFileSystemBinding(getFilesystemBinding(), conf);
+  }
+
+  public boolean isDebug() {
+    return coreAction.debug;
+  }
+
+
+  public String getFilesystemBinding() {
+    return coreAction.filesystemBinding;
+  }
+
+  public Path getBasePath() { return coreAction.basePath; }
+
+  public String getManager() {
+    return coreAction.manager;
+  }
+
+  public String getAction() {
+    return commander.getParsedCommand();
+  }
+
+  public List<String> getActionArgs() {
+    return coreAction.parameters;
+  }
+
+  public String[] getArgs() {
+    return args;
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[61/76] [abbrv] hadoop git commit: YARN-5909. Remove agent related code in slider AM. Contributed by Jian He

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
deleted file mode 100644
index 499812e..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
+++ /dev/null
@@ -1,2850 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.slider.providers.agent;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.registry.client.types.Endpoint;
-import org.apache.hadoop.registry.client.types.ProtocolTypes;
-import org.apache.hadoop.registry.client.types.ServiceRecord;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.api.ApplicationConstants;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.api.records.LocalResourceType;
-import org.apache.slider.api.ClusterDescription;
-import org.apache.slider.api.ClusterNode;
-import org.apache.slider.api.InternalKeys;
-import org.apache.slider.api.OptionKeys;
-import org.apache.slider.api.ResourceKeys;
-import org.apache.slider.api.StatusKeys;
-import org.apache.slider.common.SliderExitCodes;
-import org.apache.slider.common.SliderKeys;
-import org.apache.slider.common.SliderXmlConfKeys;
-import org.apache.slider.common.tools.SliderFileSystem;
-import org.apache.slider.common.tools.SliderUtils;
-import org.apache.slider.core.conf.AggregateConf;
-import org.apache.slider.core.conf.ConfTreeOperations;
-import org.apache.slider.core.conf.MapOperations;
-import org.apache.slider.core.exceptions.BadConfigException;
-import org.apache.slider.core.exceptions.NoSuchNodeException;
-import org.apache.slider.core.exceptions.SliderException;
-import org.apache.slider.core.launch.CommandLineBuilder;
-import org.apache.slider.core.launch.ContainerLauncher;
-import org.apache.slider.core.registry.docstore.ConfigFormat;
-import org.apache.slider.core.registry.docstore.ConfigUtils;
-import org.apache.slider.core.registry.docstore.ExportEntry;
-import org.apache.slider.core.registry.docstore.PublishedConfiguration;
-import org.apache.slider.core.registry.docstore.PublishedExports;
-import org.apache.slider.core.registry.info.CustomRegistryConstants;
-import org.apache.slider.providers.AbstractProviderService;
-import org.apache.slider.providers.MonitorDetail;
-import org.apache.slider.providers.ProviderCore;
-import org.apache.slider.providers.ProviderRole;
-import org.apache.slider.providers.ProviderUtils;
-import org.apache.slider.providers.agent.application.metadata.AbstractComponent;
-import org.apache.slider.providers.agent.application.metadata.Application;
-import org.apache.slider.providers.agent.application.metadata.CommandOrder;
-import org.apache.slider.providers.agent.application.metadata.CommandScript;
-import org.apache.slider.providers.agent.application.metadata.Component;
-import org.apache.slider.providers.agent.application.metadata.ComponentCommand;
-import org.apache.slider.providers.agent.application.metadata.ComponentExport;
-import org.apache.slider.providers.agent.application.metadata.ComponentsInAddonPackage;
-import org.apache.slider.providers.agent.application.metadata.ConfigFile;
-import org.apache.slider.providers.agent.application.metadata.DefaultConfig;
-import org.apache.slider.providers.agent.application.metadata.DockerContainer;
-import org.apache.slider.providers.agent.application.metadata.Export;
-import org.apache.slider.providers.agent.application.metadata.ExportGroup;
-import org.apache.slider.providers.agent.application.metadata.Metainfo;
-import org.apache.slider.providers.agent.application.metadata.OSPackage;
-import org.apache.slider.providers.agent.application.metadata.OSSpecific;
-import org.apache.slider.providers.agent.application.metadata.Package;
-import org.apache.slider.providers.agent.application.metadata.PropertyInfo;
-import org.apache.slider.server.appmaster.actions.ProviderReportedContainerLoss;
-import org.apache.slider.server.appmaster.actions.RegisterComponentInstance;
-import org.apache.slider.server.appmaster.state.ContainerPriority;
-import org.apache.slider.server.appmaster.state.RoleInstance;
-import org.apache.slider.server.appmaster.state.StateAccessForProviders;
-import org.apache.slider.server.appmaster.web.rest.agent.AgentCommandType;
-import org.apache.slider.server.appmaster.web.rest.agent.AgentRestOperations;
-import org.apache.slider.server.appmaster.web.rest.agent.CommandReport;
-import org.apache.slider.server.appmaster.web.rest.agent.ComponentStatus;
-import org.apache.slider.server.appmaster.web.rest.agent.ExecutionCommand;
-import org.apache.slider.server.appmaster.web.rest.agent.HeartBeat;
-import org.apache.slider.server.appmaster.web.rest.agent.HeartBeatResponse;
-import org.apache.slider.server.appmaster.web.rest.agent.Register;
-import org.apache.slider.server.appmaster.web.rest.agent.RegistrationResponse;
-import org.apache.slider.server.appmaster.web.rest.agent.RegistrationStatus;
-import org.apache.slider.server.appmaster.web.rest.agent.StatusCommand;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Scanner;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.regex.Pattern;
-
-import static org.apache.slider.api.RoleKeys.ROLE_PREFIX;
-import static org.apache.slider.server.appmaster.web.rest.RestPaths.SLIDER_PATH_AGENTS;
-
-/**
- * This class implements the server-side logic for application deployment through Slider application package
- */
-public class AgentProviderService extends AbstractProviderService implements
-    ProviderCore,
-    AgentKeys,
-    SliderKeys, AgentRestOperations {
-
-
-  protected static final Logger log =
-      LoggerFactory.getLogger(AgentProviderService.class);
-  private static final ProviderUtils providerUtils = new ProviderUtils(log);
-  private static final String LABEL_MAKER = "___";
-  private static final String CONTAINER_ID = "container_id";
-  private static final String GLOBAL_CONFIG_TAG = "global";
-  private static final String COMPONENT_TAG = "component";
-  private static final String APPLICATION_TAG = "application";
-  private static final String COMPONENT_DATA_TAG = "ComponentInstanceData";
-  private static final String SHARED_PORT_TAG = "SHARED";
-  private static final String PER_CONTAINER_TAG = "{PER_CONTAINER}";
-  private static final int MAX_LOG_ENTRIES = 40;
-  private static final int DEFAULT_HEARTBEAT_MONITOR_INTERVAL = 60 * 1000;
-
-  private final Object syncLock = new Object();
-  private final ComponentTagProvider tags = new ComponentTagProvider();
-  private int heartbeatMonitorInterval = 0;
-  private AgentClientProvider clientProvider;
-  private AtomicInteger taskId = new AtomicInteger(0);
-  private volatile Map<String, MetainfoHolder> metaInfoMap = new HashMap<>();
-  private SliderFileSystem fileSystem = null;
-  private Map<String, DefaultConfig> defaultConfigs = null;
-  private ComponentCommandOrder commandOrder = new ComponentCommandOrder();
-  private HeartbeatMonitor monitor;
-  private Boolean canAnyMasterPublish = null;
-  private AgentLaunchParameter agentLaunchParameter = null;
-  private String clusterName = null;
-  private boolean isInUpgradeMode;
-  private Set<String> upgradeContainers = new HashSet<String>();
-  private boolean appStopInitiated;
-
-  private final Map<String, ComponentInstanceState> componentStatuses =
-      new ConcurrentHashMap<String, ComponentInstanceState>();
-  private final Map<String, Map<String, String>> componentInstanceData =
-      new ConcurrentHashMap<String, Map<String, String>>();
-  private final Map<String, Map<String, List<ExportEntry>>> exportGroups =
-      new ConcurrentHashMap<String, Map<String, List<ExportEntry>>>();
-  private final Map<String, Map<String, String>> allocatedPorts =
-      new ConcurrentHashMap<String, Map<String, String>>();
-  private final Map<String, Metainfo> packageMetainfo = 
-      new ConcurrentHashMap<String, Metainfo>();
-
-  private final Map<String, ExportEntry> logFolderExports =
-      Collections.synchronizedMap(new LinkedHashMap<String, ExportEntry>(MAX_LOG_ENTRIES, 0.75f, false) {
-        protected boolean removeEldestEntry(Map.Entry eldest) {
-          return size() > MAX_LOG_ENTRIES;
-        }
-      });
-  private final Map<String, ExportEntry> workFolderExports =
-      Collections.synchronizedMap(new LinkedHashMap<String, ExportEntry>(MAX_LOG_ENTRIES, 0.75f, false) {
-        protected boolean removeEldestEntry(Map.Entry eldest) {
-          return size() > MAX_LOG_ENTRIES;
-        }
-      });
-  private final Map<String, Set<String>> containerExportsMap =
-      new HashMap<String, Set<String>>();
-
-  private static class MetainfoHolder {
-    Metainfo metaInfo;
-    private Map<String, DefaultConfig> defaultConfigs = null;
-
-    public MetainfoHolder(Metainfo metaInfo,
-        Map<String, DefaultConfig> defaultConfigs) {
-      this.metaInfo = metaInfo;
-      this.defaultConfigs = defaultConfigs;
-    }
-  }
-
-  /**
-   * Create an instance of AgentProviderService
-   */
-  public AgentProviderService() {
-    super("AgentProviderService");
-    setAgentRestOperations(this);
-    setHeartbeatMonitorInterval(DEFAULT_HEARTBEAT_MONITOR_INTERVAL);
-  }
-
-  @Override
-  public String getHumanName() {
-    return "Slider Agent";
-  }
-
-  @Override
-  public List<ProviderRole> getRoles() {
-    return AgentRoles.getRoles();
-  }
-
-  @Override
-  protected void serviceInit(Configuration conf) throws Exception {
-    super.serviceInit(conf);
-    clientProvider = new AgentClientProvider(conf);
-  }
-
-  @Override
-  public void validateInstanceDefinition(AggregateConf instanceDefinition)
-      throws
-      SliderException {
-    clientProvider.validateInstanceDefinition(instanceDefinition, null);
-
-    ConfTreeOperations resources =
-        instanceDefinition.getResourceOperations();
-
-    Set<String> names = resources.getComponentNames();
-    names.remove(COMPONENT_AM);
-    for (String name : names) {
-      Component componentDef = getApplicationComponent(name);
-      if (componentDef == null) {
-        // component member is validated elsewhere, so we don't need to throw
-        // an exception here
-        continue;
-      }
-
-      MapOperations componentConfig = resources.getMandatoryComponent(name);
-      int count =
-          componentConfig.getMandatoryOptionInt(ResourceKeys.COMPONENT_INSTANCES);
-      int definedMinCount = componentDef.getMinInstanceCountInt();
-      int definedMaxCount = componentDef.getMaxInstanceCountInt();
-      if (count < definedMinCount || count > definedMaxCount) {
-        throw new BadConfigException("Component %s, %s value %d out of range. "
-                                     + "Expected minimum is %d and maximum is %d",
-                                     name,
-                                     ResourceKeys.COMPONENT_INSTANCES,
-                                     count,
-                                     definedMinCount,
-                                     definedMaxCount);
-      }
-    }
-  }
-
-  // Reads the metainfo.xml in the application package and loads it
-  private void buildMetainfo(AggregateConf instanceDefinition,
-                             SliderFileSystem fileSystem,
-                             String roleGroup)
-      throws IOException, SliderException {
-    String mapKey = instanceDefinition.getAppConfOperations()
-        .getComponentOpt(roleGroup, ROLE_PREFIX, DEFAULT_METAINFO_MAP_KEY);
-    String appDef = SliderUtils.getApplicationDefinitionPath(
-        instanceDefinition.getAppConfOperations(), roleGroup);
-    MapOperations component = null;
-    if (roleGroup != null) {
-      component = instanceDefinition.getAppConfOperations().getComponent(roleGroup);
-    }
-
-    MetainfoHolder metaInfoHolder = metaInfoMap.get(mapKey);
-    if (metaInfoHolder == null) {
-      synchronized (syncLock) {
-        if (this.fileSystem == null) {
-          this.fileSystem = fileSystem;
-        }
-        metaInfoHolder = metaInfoMap.get(mapKey);
-        if (metaInfoHolder == null) {
-          readAndSetHeartbeatMonitoringInterval(instanceDefinition);
-          initializeAgentDebugCommands(instanceDefinition);
-
-          Metainfo metaInfo = getApplicationMetainfo(fileSystem, appDef, false);
-          log.info("Master package metainfo: {}", metaInfo.toString());
-          if (metaInfo == null || metaInfo.getApplication() == null) {
-            log.error("metainfo.xml is unavailable or malformed at {}.", appDef);
-            throw new SliderException(
-                "metainfo.xml is required in app package.");
-          }
-          List<CommandOrder> commandOrders = metaInfo.getApplication()
-              .getCommandOrders();
-          if (!DEFAULT_METAINFO_MAP_KEY.equals(mapKey)) {
-            for (Component comp : metaInfo.getApplication().getComponents()) {
-              comp.setName(mapKey + comp.getName());
-              log.info("Modifying external metainfo component name to {}",
-                  comp.getName());
-            }
-            for (CommandOrder co : commandOrders) {
-              log.info("Adding prefix {} to command order {}",
-                  mapKey, co);
-              co.setCommand(mapKey + co.getCommand());
-              co.setRequires(mapKey + co.getRequires());
-            }
-          }
-          log.debug("Merging command orders {} for {}", commandOrders,
-              roleGroup);
-          commandOrder.mergeCommandOrders(commandOrders,
-              instanceDefinition.getResourceOperations());
-          Map<String, DefaultConfig> defaultConfigs =
-              initializeDefaultConfigs(fileSystem, appDef, metaInfo);
-          metaInfoMap.put(mapKey, new MetainfoHolder(metaInfo, defaultConfigs));
-          monitor = new HeartbeatMonitor(this, getHeartbeatMonitorInterval());
-          monitor.start();
-
-          // build a map from component to metainfo
-          String addonAppDefString = instanceDefinition.getAppConfOperations()
-              .getGlobalOptions().getOption(ADDONS, null);
-          if (component != null) {
-            addonAppDefString = component.getOption(ADDONS, addonAppDefString);
-          }
-          log.debug("All addon appdefs: {}", addonAppDefString);
-          if (addonAppDefString != null) {
-            Scanner scanner = new Scanner(addonAppDefString).useDelimiter(",");
-            while (scanner.hasNext()) {
-              String addonAppDef = scanner.next();
-              String addonAppDefPath = instanceDefinition
-                  .getAppConfOperations().getGlobalOptions().get(addonAppDef);
-              if (component != null) {
-                addonAppDefPath = component.getOption(addonAppDef, addonAppDefPath);
-              }
-              log.debug("Addon package {} is stored at: {}", addonAppDef
-                  + addonAppDefPath);
-              Metainfo addonMetaInfo = getApplicationMetainfo(fileSystem,
-                  addonAppDefPath, true);
-              addonMetaInfo.validate();
-              packageMetainfo.put(addonMetaInfo.getApplicationPackage()
-                  .getName(), addonMetaInfo);
-            }
-            log.info("Metainfo map for master and addon: {}",
-                packageMetainfo.toString());
-          }
-        }
-      }
-    }
-  }
-
-  @Override
-  public void initializeApplicationConfiguration(
-      AggregateConf instanceDefinition, SliderFileSystem fileSystem,
-      String roleGroup)
-      throws IOException, SliderException {
-    buildMetainfo(instanceDefinition, fileSystem, roleGroup);
-  }
-
-  @Override
-  public void buildContainerLaunchContext(ContainerLauncher launcher,
-                                          AggregateConf instanceDefinition,
-                                          Container container,
-                                          ProviderRole providerRole,
-                                          SliderFileSystem fileSystem,
-                                          Path generatedConfPath,
-                                          MapOperations resourceComponent,
-                                          MapOperations appComponent,
-                                          Path containerTmpDirPath) throws
-      IOException,
-      SliderException {
-    
-    String roleName = providerRole.name;
-    String roleGroup = providerRole.group;
-    String appDef = SliderUtils.getApplicationDefinitionPath(instanceDefinition
-        .getAppConfOperations(), roleGroup);
-
-    initializeApplicationConfiguration(instanceDefinition, fileSystem, roleGroup);
-
-    log.info("Build launch context for Agent");
-    log.debug(instanceDefinition.toString());
-    
-    //if we are launching docker based app on yarn, then we need to pass docker image
-    if (isYarnDockerContainer(roleGroup)) {
-      launcher.setYarnDockerMode(true);
-      launcher.setDockerImage(getConfigFromMetaInfo(roleGroup, "image"));
-      launcher.setDockerNetwork(getConfigFromMetaInfo(roleGroup, "network"));
-      launcher.setRunPrivilegedContainer(getConfigFromMetaInfo(roleGroup, "runPriviledgedContainer"));
-      launcher
-          .setYarnContainerMountPoints(getConfigFromMetaInfoWithAppConfigOverriding(
-              roleGroup, "yarn.container.mount.points"));
-    }
-
-    // Set the environment
-    launcher.putEnv(SliderUtils.buildEnvMap(appComponent,
-        providerUtils.getStandardTokenMap(getAmState().getAppConfSnapshot(),
-            getAmState().getInternalsSnapshot(), roleName, roleGroup,
-            getClusterName())));
-
-    String workDir = ApplicationConstants.Environment.PWD.$();
-    launcher.setEnv("AGENT_WORK_ROOT", workDir);
-    log.info("AGENT_WORK_ROOT set to {}", workDir);
-    String logDir = ApplicationConstants.LOG_DIR_EXPANSION_VAR;
-    launcher.setEnv("AGENT_LOG_ROOT", logDir);
-    log.info("AGENT_LOG_ROOT set to {}", logDir);
-    if (System.getenv(HADOOP_USER_NAME) != null) {
-      launcher.setEnv(HADOOP_USER_NAME, System.getenv(HADOOP_USER_NAME));
-    }
-    // for 2-Way SSL
-    launcher.setEnv(SLIDER_PASSPHRASE, instanceDefinition.getPassphrase());
-    //add english env
-    launcher.setEnv("LANG", "en_US.UTF-8");
-    launcher.setEnv("LC_ALL", "en_US.UTF-8");
-    launcher.setEnv("LANGUAGE", "en_US.UTF-8");
-
-    //local resources
-
-    // TODO: Should agent need to support App Home
-    String scriptPath = new File(AGENT_MAIN_SCRIPT_ROOT, AGENT_MAIN_SCRIPT).getPath();
-    String appHome = instanceDefinition.getAppConfOperations().
-        getGlobalOptions().get(PACKAGE_PATH);
-    if (SliderUtils.isSet(appHome)) {
-      scriptPath = new File(appHome, AGENT_MAIN_SCRIPT).getPath();
-    }
-
-    // set PYTHONPATH
-    List<String> pythonPaths = new ArrayList<String>();
-    pythonPaths.add(AGENT_MAIN_SCRIPT_ROOT);
-    pythonPaths.add(AGENT_JINJA2_ROOT);
-    String pythonPath = StringUtils.join(File.pathSeparator, pythonPaths);
-    launcher.setEnv(PYTHONPATH, pythonPath);
-    log.info("PYTHONPATH set to {}", pythonPath);
-
-    Path agentImagePath = null;
-    String agentImage = instanceDefinition.getInternalOperations().
-        get(InternalKeys.INTERNAL_APPLICATION_IMAGE_PATH);
-    if (SliderUtils.isUnset(agentImage)) {
-      agentImagePath =
-          new Path(new Path(new Path(instanceDefinition.getInternalOperations().get(InternalKeys.INTERNAL_TMP_DIR),
-                                     container.getId().getApplicationAttemptId().getApplicationId().toString()),
-                            PROVIDER_AGENT),
-                   AGENT_TAR);
-    } else {
-       agentImagePath = new Path(agentImage);
-    }
-
-    if (fileSystem.getFileSystem().exists(agentImagePath)) {
-      LocalResource agentImageRes = fileSystem.createAmResource(agentImagePath, LocalResourceType.ARCHIVE);
-      launcher.addLocalResource(AGENT_INSTALL_DIR, agentImageRes);
-    } else {
-      String msg =
-          String.format("Required agent image slider-agent.tar.gz is unavailable at %s", agentImagePath.toString());
-      MapOperations compOps = appComponent;
-      boolean relaxVerificationForTest = compOps != null ? Boolean.valueOf(compOps.
-          getOptionBool(TEST_RELAX_VERIFICATION, false)) : false;
-      log.error(msg);
-
-      if (!relaxVerificationForTest) {
-        throw new SliderException(SliderExitCodes.EXIT_DEPLOYMENT_FAILED, msg);
-      }
-    }
-
-    log.info("Using {} for agent.", scriptPath);
-    LocalResource appDefRes = fileSystem.createAmResource(
-        fileSystem.getFileSystem().resolvePath(new Path(appDef)),
-        LocalResourceType.ARCHIVE);
-    launcher.addLocalResource(APP_DEFINITION_DIR, appDefRes);
-
-    for (Package pkg : getMetaInfo(roleGroup).getApplication().getPackages()) {
-      Path pkgPath = fileSystem.buildResourcePath(pkg.getName());
-      if (!fileSystem.isFile(pkgPath)) {
-        pkgPath = fileSystem.buildResourcePath(getClusterName(),
-            pkg.getName());
-      }
-      if (!fileSystem.isFile(pkgPath)) {
-        throw new IOException("Package doesn't exist as a resource: " +
-            pkg.getName());
-      }
-      log.info("Adding resource {}", pkg.getName());
-      LocalResourceType type = LocalResourceType.FILE;
-      if ("archive".equals(pkg.getType())) {
-        type = LocalResourceType.ARCHIVE;
-      }
-      LocalResource packageResource = fileSystem.createAmResource(
-          pkgPath, type);
-      launcher.addLocalResource(APP_PACKAGES_DIR, packageResource);
-    }
-
-    String agentConf = instanceDefinition.getAppConfOperations().
-        getGlobalOptions().getOption(AGENT_CONF, "");
-    if (SliderUtils.isSet(agentConf)) {
-      LocalResource agentConfRes = fileSystem.createAmResource(fileSystem
-                                                                   .getFileSystem().resolvePath(new Path(agentConf)),
-                                                               LocalResourceType.FILE);
-      launcher.addLocalResource(AGENT_CONFIG_FILE, agentConfRes);
-    }
-
-    String agentVer = instanceDefinition.getAppConfOperations().
-        getGlobalOptions().getOption(AGENT_VERSION, null);
-    if (agentVer != null) {
-      LocalResource agentVerRes = fileSystem.createAmResource(
-          fileSystem.getFileSystem().resolvePath(new Path(agentVer)),
-          LocalResourceType.FILE);
-      launcher.addLocalResource(AGENT_VERSION_FILE, agentVerRes);
-    }
-
-    if (SliderUtils.isHadoopClusterSecure(getConfig())) {
-      providerUtils.localizeServiceKeytabs(launcher, instanceDefinition,
-          fileSystem, getClusterName());
-    }
-
-    MapOperations amComponent = instanceDefinition.
-        getAppConfOperations().getComponent(COMPONENT_AM);
-    if (providerUtils.hasTwoWaySSLEnabled(amComponent)) {
-      providerUtils.localizeContainerSSLResources(launcher, container,
-          fileSystem, getClusterName());
-    }
-
-    if (providerUtils.areStoresRequested(appComponent)) {
-      providerUtils.localizeContainerSecurityStores(launcher, container,
-          roleName, fileSystem, instanceDefinition, appComponent,
-          getClusterName());
-    }
-
-    //add the configuration resources
-    launcher.addLocalResources(fileSystem.submitDirectory(
-        generatedConfPath,
-        PROPAGATED_CONF_DIR_NAME));
-
-    if (appComponent.getOptionBool(AM_CONFIG_GENERATION, false)) {
-      // build and localize configuration files
-      Map<String, Map<String, String>> configurations =
-          buildCommandConfigurations(instanceDefinition.getAppConfOperations(),
-              instanceDefinition.getInternalOperations(),
-              container.getId().toString(), roleName, roleGroup);
-      for (ConfigFile configFile : getMetaInfo(roleGroup)
-          .getComponentConfigFiles(roleGroup)) {
-        localizeConfigFile(launcher, roleName, roleGroup, configFile,
-            configurations, launcher.getEnv(), fileSystem);
-      }
-    }
-
-    String label = getContainerLabel(container, roleName, roleGroup);
-    CommandLineBuilder operation = new CommandLineBuilder();
-
-    String pythonExec = instanceDefinition.getAppConfOperations()
-        .getGlobalOptions().getOption(SliderXmlConfKeys.PYTHON_EXECUTABLE_PATH,
-                                      PYTHON_EXE);
-
-    operation.add(pythonExec);
-
-    operation.add(scriptPath);
-    operation.add(ARG_LABEL, label);
-    operation.add(ARG_ZOOKEEPER_QUORUM);
-    operation.add(getClusterOptionPropertyValue(OptionKeys.ZOOKEEPER_QUORUM));
-    operation.add(ARG_ZOOKEEPER_REGISTRY_PATH);
-    operation.add(getZkRegistryPath());
-
-    String debugCmd = agentLaunchParameter.getNextLaunchParameter(roleGroup);
-    if (SliderUtils.isSet(debugCmd)) {
-      operation.add(ARG_DEBUG);
-      operation.add(debugCmd);
-    }
-
-    operation.add("> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/"
-        + AGENT_OUT_FILE + " 2>&1");
-
-    launcher.addCommand(operation.build());
-
-    // localize addon package
-    String addonAppDefString = instanceDefinition.getAppConfOperations()
-        .getGlobalOptions().getOption(ADDONS, null);
-    log.debug("All addon appdefs: {}", addonAppDefString);
-    if (addonAppDefString != null) {
-      Scanner scanner = new Scanner(addonAppDefString).useDelimiter(",");
-      while (scanner.hasNext()) {
-        String addonAppDef = scanner.next();
-        String addonAppDefPath = instanceDefinition
-            .getAppConfOperations().getGlobalOptions().get(addonAppDef);
-        log.debug("Addon package {} is stored at: {}", addonAppDef, addonAppDefPath);
-        LocalResource addonPkgRes = fileSystem.createAmResource(
-            fileSystem.getFileSystem().resolvePath(new Path(addonAppDefPath)),
-            LocalResourceType.ARCHIVE);
-        launcher.addLocalResource(ADDON_DEFINITION_DIR + "/" + addonAppDef, addonPkgRes);
-      }
-      log.debug("Metainfo map for master and addon: {}",
-          packageMetainfo.toString());
-    }    
-
-    // Additional files to localize in addition to the application def
-    String appResourcesString = instanceDefinition.getAppConfOperations()
-        .getGlobalOptions().getOption(APP_RESOURCES, null);
-    log.info("Configuration value for extra resources to localize: {}", appResourcesString);
-    if (null != appResourcesString) {
-      try (Scanner scanner = new Scanner(appResourcesString).useDelimiter(",")) {
-        while (scanner.hasNext()) {
-          String resource = scanner.next();
-          Path resourcePath = new Path(resource);
-          LocalResource extraResource = fileSystem.createAmResource(
-              fileSystem.getFileSystem().resolvePath(resourcePath),
-              LocalResourceType.FILE);
-          String destination = APP_RESOURCES_DIR + "/" + resourcePath.getName();
-          log.info("Localizing {} to {}", resourcePath, destination);
-          // TODO Can we try harder to avoid collisions?
-          launcher.addLocalResource(destination, extraResource);
-        }
-      }
-    }
-
-    // initialize addon pkg states for all componentInstanceStatus
-    Map<String, State> pkgStatuses = new TreeMap<>();
-    for (Metainfo appPkg : packageMetainfo.values()) {
-      // check each component of that addon to see if they apply to this
-      // component 'role'
-      for (ComponentsInAddonPackage comp : appPkg.getApplicationPackage()
-          .getComponents()) {
-        log.debug("Current component: {} component in metainfo: {}", roleName,
-            comp.getName());
-        if (comp.getName().equals(roleGroup)
-            || comp.getName().equals(ADDON_FOR_ALL_COMPONENTS)) {
-          pkgStatuses.put(appPkg.getApplicationPackage().getName(), State.INIT);
-        }
-      }
-    }
-    log.debug("For component: {} pkg status map: {}", roleName,
-        pkgStatuses.toString());
-    
-    // initialize the component instance state
-    getComponentStatuses().put(label,
-                               new ComponentInstanceState(
-                                   roleGroup,
-                                   container.getId(),
-                                   getClusterInfoPropertyValue(OptionKeys.APPLICATION_NAME),
-                                   pkgStatuses));
-  }
-
-  @VisibleForTesting
-  protected void localizeConfigFile(ContainerLauncher launcher,
-                                     String roleName, String roleGroup,
-                                     ConfigFile configFile,
-                                     Map<String, Map<String, String>> configs,
-                                     MapOperations env,
-                                     SliderFileSystem fileSystem)
-      throws IOException {
-    ConfigFormat configFormat = ConfigFormat.resolve(configFile.getType());
-    providerUtils.localizeConfigFile(launcher, roleName, roleGroup,
-        configFile.getDictionaryName(), configFormat, configFile.getFileName(),
-        configs, env, fileSystem, getClusterName());
-  }
-
-  /**
-   * build the zookeeper registry path.
-   * 
-   * @return the path the service registered at
-   * @throws NullPointerException if the service has not yet registered
-   */
-  private String getZkRegistryPath() {
-    Preconditions.checkNotNull(yarnRegistry, "Yarn registry not bound");
-    String path = yarnRegistry.getAbsoluteSelfRegistrationPath();
-    Preconditions.checkNotNull(path, "Service record path not defined");
-    return path;
-  }
-
-  @Override
-  public void rebuildContainerDetails(List<Container> liveContainers,
-                                      String applicationId, Map<Integer, ProviderRole> providerRoleMap) {
-    for (Container container : liveContainers) {
-      // get the role name and label
-      ProviderRole role = providerRoleMap.get(ContainerPriority
-                                                  .extractRole(container));
-      if (role != null) {
-        String roleName = role.name;
-        String roleGroup = role.group;
-        String label = getContainerLabel(container, roleName, roleGroup);
-        log.info("Rebuilding in-memory: container {} in role {} in cluster {}",
-                 container.getId(), roleName, applicationId);
-        getComponentStatuses().put(label,
-            new ComponentInstanceState(roleGroup, container.getId(),
-                                       applicationId));
-      } else {
-        log.warn("Role not found for container {} in cluster {}",
-                 container.getId(), applicationId);
-      }
-    }
-  }
-
-  @Override
-  public boolean isSupportedRole(String role) {
-    return true;
-  }
-
-  /**
-   * Handle registration calls from the agents
-   *
-   * @param registration registration entry
-   *
-   * @return response
-   */
-  @Override
-  public RegistrationResponse handleRegistration(Register registration) {
-    log.info("Handling registration: {}", registration);
-    RegistrationResponse response = new RegistrationResponse();
-    String label = registration.getLabel();
-    String pkg = registration.getPkg();
-    State agentState = registration.getActualState();
-    String appVersion = registration.getAppVersion();
-
-    log.info("label: {} pkg: {}", label, pkg);
-
-    if (getComponentStatuses().containsKey(label)) {
-      response.setResponseStatus(RegistrationStatus.OK);
-      ComponentInstanceState componentStatus = getComponentStatuses().get(label);
-      componentStatus.heartbeat(System.currentTimeMillis());
-      updateComponentStatusWithAgentState(componentStatus, agentState);
-
-      String roleName = getRoleName(label);
-      String roleGroup = getRoleGroup(label);
-      String containerId = getContainerId(label);
-
-      if (SliderUtils.isSet(registration.getTags())) {
-        tags.recordAssignedTag(roleName, containerId, registration.getTags());
-      } else {
-        response.setTags(tags.getTag(roleName, containerId));
-      }
-
-      String hostFqdn = registration.getPublicHostname();
-      Map<String, String> ports = registration.getAllocatedPorts();
-      if (ports != null && !ports.isEmpty()) {
-        processAllocatedPorts(hostFqdn, roleName, roleGroup, containerId, ports);
-      }
-
-      Map<String, String> folders = registration.getLogFolders();
-      if (folders != null && !folders.isEmpty()) {
-        publishFolderPaths(folders, containerId, roleName, hostFqdn);
-      }
-
-      // Set app version if empty. It gets unset during upgrade - why?
-      checkAndSetContainerAppVersion(containerId, appVersion);
-    } else {
-      response.setResponseStatus(RegistrationStatus.FAILED);
-      response.setLog("Label not recognized.");
-      log.warn("Received registration request from unknown label {}", label);
-    }
-    log.info("Registration response: {}", response);
-    return response;
-  }
-
-  // Checks if app version is empty. Sets it to the version as reported by the
-  // container during registration phase.
-  private void checkAndSetContainerAppVersion(String containerId,
-      String appVersion) {
-    StateAccessForProviders amState = getAmState();
-    try {
-      RoleInstance role = amState.getOwnedContainer(containerId);
-      if (role != null) {
-        String currentAppVersion = role.appVersion;
-        log.debug("Container = {}, app version current = {} new = {}",
-            containerId, currentAppVersion, appVersion);
-        if (currentAppVersion == null
-            || currentAppVersion.equals(APP_VERSION_UNKNOWN)) {
-          amState.getOwnedContainer(containerId).appVersion = appVersion;
-        }
-      }
-    } catch (NoSuchNodeException e) {
-      // ignore - there is nothing to do if we don't find a container
-      log.warn("Owned container {} not found - {}", containerId, e);
-    }
-  }
-
-  /**
-   * Handle heartbeat response from agents
-   *
-   * @param heartBeat incoming heartbeat from Agent
-   *
-   * @return response to send back
-   */
-  @Override
-  public HeartBeatResponse handleHeartBeat(HeartBeat heartBeat) {
-    log.debug("Handling heartbeat: {}", heartBeat);
-    HeartBeatResponse response = new HeartBeatResponse();
-    long id = heartBeat.getResponseId();
-    response.setResponseId(id + 1L);
-
-    String label = heartBeat.getHostname();
-    String pkg = heartBeat.getPackage();
-
-    log.debug("package received: " + pkg);
-    
-    String roleName = getRoleName(label);
-    String roleGroup = getRoleGroup(label);
-    String containerId = getContainerId(label);
-    boolean doUpgrade = false;
-    if (isInUpgradeMode && upgradeContainers.contains(containerId)) {
-      doUpgrade = true;
-    }
-
-    CommandScript cmdScript = getScriptPathForMasterPackage(roleGroup);
-    List<ComponentCommand> commands = getApplicationComponent(roleGroup).getCommands();
-
-    if (!isDockerContainer(roleGroup) && !isYarnDockerContainer(roleGroup)
-        && (cmdScript == null || cmdScript.getScript() == null)
-        && commands.size() == 0) {
-      log.error(
-          "role.script is unavailable for {}. Commands will not be sent.",
-          roleName);
-      return response;
-    }
-
-    String scriptPath = null;
-    long timeout = 600L;
-    if (cmdScript != null) {
-      scriptPath = cmdScript.getScript();
-      timeout = cmdScript.getTimeout();
-    }
-
-    if (timeout == 0L) {
-      timeout = 600L;
-    }
-
-    if (!getComponentStatuses().containsKey(label)) {
-      // container is completed but still heart-beating, send terminate signal
-      log.info(
-          "Sending terminate signal to completed container (still heartbeating): {}",
-          label);
-      response.setTerminateAgent(true);
-      return response;
-    }
-
-    List<ComponentStatus> statuses = heartBeat.getComponentStatus();
-    if (statuses != null && !statuses.isEmpty()) {
-      log.info("status from agent: " + statuses.toString());
-      for(ComponentStatus status : statuses){
-        providerUtils.updateServiceRecord(getAmState(), yarnRegistry,
-            containerId, roleName, Collections.singletonList(status.getIp()),
-            status.getHostname());
-      }
-    }
-
-    Boolean isMaster = isMaster(roleGroup);
-    ComponentInstanceState componentStatus = getComponentStatuses().get(label);
-    componentStatus.heartbeat(System.currentTimeMillis());
-    if (doUpgrade) {
-      switch (componentStatus.getState()) {
-      case STARTED:
-        componentStatus.setTargetState(State.UPGRADED);
-        break;
-      case UPGRADED:
-        componentStatus.setTargetState(State.STOPPED);
-        break;
-      case STOPPED:
-        componentStatus.setTargetState(State.TERMINATING);
-        break;
-      default:
-        break;
-      }
-      log.info("Current state = {} target state {}",
-          componentStatus.getState(), componentStatus.getTargetState());
-    }
-
-    if (appStopInitiated && !componentStatus.isStopInitiated()) {
-      log.info("Stop initiated for label {}", label);
-      componentStatus.setTargetState(State.STOPPED);
-      componentStatus.setStopInitiated(true);
-    }
-
-    publishConfigAndExportGroups(heartBeat, componentStatus, roleGroup);
-    CommandResult result = null;
-    List<CommandReport> reports = heartBeat.getReports();
-    if (SliderUtils.isNotEmpty(reports)) {
-      CommandReport report = reports.get(0);
-      Map<String, String> ports = report.getAllocatedPorts();
-      if (SliderUtils.isNotEmpty(ports)) {
-        processAllocatedPorts(heartBeat.getFqdn(), roleName, roleGroup, containerId, ports);
-      }
-      result = CommandResult.getCommandResult(report.getStatus());
-      Command command = Command.getCommand(report.getRoleCommand());
-      componentStatus.applyCommandResult(result, command, pkg);
-      log.info("Component operation. Status: {}; new container state: {};"
-          + " new component state: {}", result,
-          componentStatus.getContainerState(), componentStatus.getState());
-
-      if (command == Command.INSTALL && SliderUtils.isNotEmpty(report.getFolders())) {
-        publishFolderPaths(report.getFolders(), containerId, roleName, heartBeat.getFqdn());
-      }
-    }
-
-    int waitForCount = getAmState().getInstanceDefinitionSnapshot().
-        getAppConfOperations().getComponentOptInt(roleGroup, WAIT_HEARTBEAT, 0);
-
-    if (id < waitForCount) {
-      log.info("Waiting until heartbeat count {}. Current val: {}", waitForCount, id);
-      getComponentStatuses().put(label, componentStatus);
-      return response;
-    }
-
-    Command command = componentStatus.getNextCommand(doUpgrade);
-    try {
-      if (Command.NOP != command) {
-        log.debug("For comp {} pkg {} issuing {}", roleName,
-            componentStatus.getNextPkgToInstall(), command.toString());
-        if (command == Command.INSTALL) {
-          log.info("Installing {} on {}.", roleName, containerId);
-          if (isDockerContainer(roleGroup) || isYarnDockerContainer(roleGroup)){
-            addInstallDockerCommand(roleName, roleGroup, containerId,
-                response, null, timeout);
-          } else if (scriptPath != null) {
-            addInstallCommand(roleName, roleGroup, containerId, response,
-                scriptPath, null, timeout, null);
-          } else {
-            // commands
-            ComponentCommand installCmd = null;
-            for (ComponentCommand compCmd : commands) {
-              if (compCmd.getName().equals("INSTALL")) {
-                installCmd = compCmd;
-              }
-            }
-            addInstallCommand(roleName, roleGroup, containerId, response, null,
-                installCmd, timeout, null);
-          }
-          componentStatus.commandIssued(command);
-        } else if (command == Command.INSTALL_ADDON) {
-          String nextPkgToInstall = componentStatus.getNextPkgToInstall();
-          // retrieve scriptPath or command of that package for the component
-          for (ComponentsInAddonPackage comp : packageMetainfo
-              .get(nextPkgToInstall).getApplicationPackage().getComponents()) {
-            // given nextPkgToInstall and roleName is determined, the if below
-            // should only execute once per heartbeat
-            log.debug("Addon component: {} pkg: {} script: {}", comp.getName(),
-                nextPkgToInstall, comp.getCommandScript().getScript());
-            if (comp.getName().equals(roleGroup)
-                || comp.getName().equals(ADDON_FOR_ALL_COMPONENTS)) {
-              scriptPath = comp.getCommandScript().getScript();
-              if (scriptPath != null) {
-                addInstallCommand(roleName, roleGroup, containerId, response,
-                    scriptPath, null, timeout, nextPkgToInstall);
-              } else {
-                ComponentCommand installCmd = null;
-                for (ComponentCommand compCmd : comp.getCommands()) {
-                  if (compCmd.getName().equals("INSTALL")) {
-                    installCmd = compCmd;
-                  }
-                }
-                addInstallCommand(roleName, roleGroup, containerId, response,
-                    null, installCmd, timeout, nextPkgToInstall);
-              }
-            }
-          }
-          componentStatus.commandIssued(command);
-        } else if (command == Command.START) {
-          // check against dependencies
-          boolean canExecute = commandOrder.canExecute(roleGroup, command, getComponentStatuses().values());
-          if (canExecute) {
-            log.info("Starting {} on {}.", roleName, containerId);
-            if (isDockerContainer(roleGroup) || isYarnDockerContainer(roleGroup)){
-              addStartDockerCommand(roleName, roleGroup, containerId,
-                  response, null, timeout, false);
-            } else if (scriptPath != null) {
-              addStartCommand(roleName,
-                              roleGroup,
-                              containerId,
-                              response,
-                              scriptPath,
-                              null,
-                              null,
-                              timeout,
-                              isMarkedAutoRestart(roleGroup));
-            } else {
-              ComponentCommand startCmd = null;
-              for (ComponentCommand compCmd : commands) {
-                if (compCmd.getName().equals("START")) {
-                  startCmd = compCmd;
-                }
-              }
-              ComponentCommand stopCmd = null;
-              for (ComponentCommand compCmd : commands) {
-                if (compCmd.getName().equals("STOP")) {
-                  stopCmd = compCmd;
-                }
-              }
-              addStartCommand(roleName, roleGroup, containerId, response, null,
-                  startCmd, stopCmd, timeout, false);
-            }
-            componentStatus.commandIssued(command);
-          } else {
-            log.info("Start of {} on {} delayed as dependencies have not started.", roleName, containerId);
-          }
-        } else if (command == Command.UPGRADE) {
-          addUpgradeCommand(roleName, roleGroup, containerId, response,
-              scriptPath, timeout);
-          componentStatus.commandIssued(command, true);
-        } else if (command == Command.STOP) {
-          log.info("Stop command being sent to container with id {}",
-              containerId);
-          addStopCommand(roleName, roleGroup, containerId, response, scriptPath,
-              timeout, doUpgrade);
-          componentStatus.commandIssued(command);
-        } else if (command == Command.TERMINATE) {
-          log.info("A formal terminate command is being sent to container {}"
-              + " in state {}", label, componentStatus.getState());
-          response.setTerminateAgent(true);
-        }
-      }
-
-      // if there is no outstanding command then retrieve config
-      if (isMaster && componentStatus.getState() == State.STARTED
-          && command == Command.NOP) {
-        if (!componentStatus.getConfigReported()) {
-          log.info("Requesting applied config for {} on {}.", roleName, containerId);
-          if (isDockerContainer(roleGroup) || isYarnDockerContainer(roleGroup)){
-            addGetConfigDockerCommand(roleName, roleGroup, containerId, response);
-          } else {
-            addGetConfigCommand(roleName, roleGroup, containerId, response);
-          }
-        }
-      }
-      
-      // if restart is required then signal
-      response.setRestartEnabled(false);
-      if (componentStatus.getState() == State.STARTED
-          && command == Command.NOP && isMarkedAutoRestart(roleGroup)) {
-        response.setRestartEnabled(true);
-      }
-
-      //If INSTALL_FAILED and no INSTALL is scheduled let the agent fail
-      if (componentStatus.getState() == State.INSTALL_FAILED
-         && command == Command.NOP) {
-        log.warn("Sending terminate signal to container that failed installation: {}", label);
-        response.setTerminateAgent(true);
-      }
-
-    } catch (SliderException e) {
-      log.warn("Component instance failed operation.", e);
-      componentStatus.applyCommandResult(CommandResult.FAILED, command, null);
-    }
-
-    log.debug("Heartbeat response: " + response);
-    return response;
-  }
-
-  private boolean isDockerContainer(String roleGroup) {
-    String type = getApplicationComponent(roleGroup).getType();
-    if (SliderUtils.isSet(type)) {
-      return type.toLowerCase().equals(SliderUtils.DOCKER) || type.toLowerCase().equals(SliderUtils.DOCKER_YARN);
-    }
-    return false;
-  }
-
-  private boolean isYarnDockerContainer(String roleGroup) {
-    String type = getApplicationComponent(roleGroup).getType();
-    if (SliderUtils.isSet(type)) {
-      return type.toLowerCase().equals(SliderUtils.DOCKER_YARN);
-    }
-    return false;
-  }
-
-  protected void processAllocatedPorts(String fqdn,
-                                       String roleName,
-                                       String roleGroup,
-                                       String containerId,
-                                       Map<String, String> ports) {
-    RoleInstance instance;
-    try {
-      instance = getAmState().getOwnedContainer(containerId);
-    } catch (NoSuchNodeException e) {
-      log.warn("Failed to locate instance of container {}", containerId, e);
-      instance = null;
-    }
-    for (Map.Entry<String, String> port : ports.entrySet()) {
-      String portname = port.getKey();
-      String portNo = port.getValue();
-      log.info("Recording allocated port for {} as {}", portname, portNo);
-
-      // add the allocated ports to the global list as well as per container list
-      // per container allocation will over-write each other in the global
-      this.getAllocatedPorts().put(portname, portNo);
-      this.getAllocatedPorts(containerId).put(portname, portNo);
-      if (instance != null) {
-        try {
-          // if the returned value is not a single port number then there are no
-          // meaningful way for Slider to use it during export
-          // No need to error out as it may not be the responsibility of the component
-          // to allocate port or the component may need an array of ports
-          instance.registerPortEndpoint(Integer.valueOf(portNo), portname);
-        } catch (NumberFormatException e) {
-          log.warn("Failed to parse {}", portNo, e);
-        }
-      }
-    }
-
-    processAndPublishComponentSpecificData(ports, containerId, fqdn, roleGroup);
-    processAndPublishComponentSpecificExports(ports, containerId, fqdn, roleName, roleGroup);
-
-    // and update registration entries
-    if (instance != null) {
-      queueAccess.put(new RegisterComponentInstance(instance.getId(),
-          roleName, roleGroup, 0, TimeUnit.MILLISECONDS));
-    }
-  }
-
-  private void updateComponentStatusWithAgentState(
-      ComponentInstanceState componentStatus, State agentState) {
-    if (agentState != null) {
-      componentStatus.setState(agentState);
-    }
-  }
-
-  @Override
-  public Map<String, MonitorDetail> buildMonitorDetails(ClusterDescription clusterDesc) {
-    Map<String, MonitorDetail> details = super.buildMonitorDetails(clusterDesc);
-    buildRoleHostDetails(details);
-    return details;
-  }
-
-  public void applyInitialRegistryDefinitions(URL amWebURI,
-      URL agentOpsURI,
-      URL agentStatusURI,
-      ServiceRecord serviceRecord)
-    throws IOException {
-    super.applyInitialRegistryDefinitions(amWebURI,
-                                          serviceRecord);
-
-    try {
-      URL restURL = new URL(agentOpsURI, SLIDER_PATH_AGENTS);
-      URL agentStatusURL = new URL(agentStatusURI, SLIDER_PATH_AGENTS);
-
-      serviceRecord.addInternalEndpoint(
-          new Endpoint(CustomRegistryConstants.AGENT_SECURE_REST_API,
-                       ProtocolTypes.PROTOCOL_REST,
-                       restURL.toURI()));
-      serviceRecord.addInternalEndpoint(
-          new Endpoint(CustomRegistryConstants.AGENT_ONEWAY_REST_API,
-                       ProtocolTypes.PROTOCOL_REST,
-                       agentStatusURL.toURI()));
-    } catch (URISyntaxException e) {
-      throw new IOException(e);
-    }
-
-    // identify client component
-    Component client = null;
-    for (Component component : getMetaInfo().getApplication().getComponents()) {
-      if (component.getCategory().equals("CLIENT")) {
-        client = component;
-        break;
-      }
-    }
-    if (client == null) {
-      log.info("No client component specified, not publishing client configs");
-      return;
-    }
-
-    // register AM-generated client configs
-    ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
-    MapOperations clientOperations = appConf.getOrAddComponent(client.getName());
-    appConf.resolve();
-    if (!clientOperations.getOptionBool(AM_CONFIG_GENERATION,
-        false)) {
-      log.info("AM config generation is false, not publishing client configs");
-      return;
-    }
-
-    // build and localize configuration files
-    Map<String, Map<String, String>> configurations = new TreeMap<>();
-    Map<String, String> tokens = providerUtils.getStandardTokenMap(appConf,
-        getAmState().getInternalsSnapshot(), client.getName(),
-        client.getName(), getClusterName());
-
-    for (ConfigFile configFile : getMetaInfo().getComponentConfigFiles(client.getName())) {
-      addNamedConfiguration(configFile.getDictionaryName(),
-          appConf.getGlobalOptions().options, configurations, tokens, null,
-          client.getName(), client.getName());
-      if (appConf.getComponent(client.getName()) != null) {
-        addNamedConfiguration(configFile.getDictionaryName(),
-            appConf.getComponent(client.getName()).options, configurations,
-            tokens, null, client.getName(), client.getName());
-      }
-    }
-
-    //do a final replacement of re-used configs
-    dereferenceAllConfigs(configurations);
-
-    for (ConfigFile configFile : getMetaInfo().getComponentConfigFiles(client.getName())) {
-      ConfigFormat configFormat = ConfigFormat.resolve(configFile.getType());
-
-      Map<String, String> config = configurations.get(configFile.getDictionaryName());
-      ConfigUtils.prepConfigForTemplateOutputter(configFormat, config,
-          fileSystem, getClusterName(),
-          new File(configFile.getFileName()).getName());
-      PublishedConfiguration publishedConfiguration =
-          new PublishedConfiguration(configFile.getDictionaryName(),
-              config.entrySet());
-      getAmState().getPublishedSliderConfigurations().put(
-          configFile.getDictionaryName(), publishedConfiguration);
-      log.info("Publishing AM configuration {}", configFile.getDictionaryName());
-    }
-  }
-
-  @Override
-  public void notifyContainerCompleted(ContainerId containerId) {
-    // containers get allocated and free'ed without being assigned to any
-    // component - so many of the data structures may not be initialized
-    if (containerId != null) {
-      String containerIdStr = containerId.toString();
-      if (getComponentInstanceData().containsKey(containerIdStr)) {
-        getComponentInstanceData().remove(containerIdStr);
-        log.info("Removing container specific data for {}", containerIdStr);
-        publishComponentInstanceData();
-      }
-
-      if (this.allocatedPorts.containsKey(containerIdStr)) {
-        Map<String, String> portsByContainerId = getAllocatedPorts(containerIdStr);
-        this.allocatedPorts.remove(containerIdStr);
-        // free up the allocations from global as well
-        // if multiple containers allocate global ports then last one
-        // wins and similarly first one removes it - its not supported anyway
-        for(String portName : portsByContainerId.keySet()) {
-          getAllocatedPorts().remove(portName);
-        }
-
-      }
-
-      String componentName = null;
-      synchronized (this.componentStatuses) {
-        for (String label : getComponentStatuses().keySet()) {
-          if (label.startsWith(containerIdStr)) {
-            componentName = getRoleName(label);
-            log.info("Removing component status for label {}", label);
-            getComponentStatuses().remove(label);
-          }
-        }
-      }
-
-      tags.releaseTag(componentName, containerIdStr);
-
-      synchronized (this.containerExportsMap) {
-        Set<String> containerExportSets = containerExportsMap.get(containerIdStr);
-        if (containerExportSets != null) {
-          for (String containerExportStr : containerExportSets) {
-            String[] parts = containerExportStr.split(":");
-            Map<String, List<ExportEntry>> exportGroup = getCurrentExports(parts[0]);
-            List<ExportEntry> exports = exportGroup.get(parts[1]);
-            List<ExportEntry> exportToRemove = new ArrayList<ExportEntry>();
-            for (ExportEntry export : exports) {
-              if (containerIdStr.equals(export.getContainerId())) {
-                exportToRemove.add(export);
-              }
-            }
-            exports.removeAll(exportToRemove);
-          }
-          log.info("Removing container exports for {}", containerIdStr);
-          containerExportsMap.remove(containerIdStr);
-        }
-      }
-    }
-  }
-
-  /**
-   * Reads and sets the heartbeat monitoring interval. If bad value is provided then log it and set to default.
-   *
-   * @param instanceDefinition
-   */
-  private void readAndSetHeartbeatMonitoringInterval(AggregateConf instanceDefinition) {
-    String hbMonitorInterval = instanceDefinition.getAppConfOperations().
-        getGlobalOptions().getOption(HEARTBEAT_MONITOR_INTERVAL,
-                                     Integer.toString(DEFAULT_HEARTBEAT_MONITOR_INTERVAL));
-    try {
-      setHeartbeatMonitorInterval(Integer.parseInt(hbMonitorInterval));
-    } catch (NumberFormatException e) {
-      log.warn(
-          "Bad value {} for {}. Defaulting to ",
-          hbMonitorInterval,
-          HEARTBEAT_MONITOR_INTERVAL,
-          DEFAULT_HEARTBEAT_MONITOR_INTERVAL);
-    }
-  }
-
-  /**
-   * Reads and sets the heartbeat monitoring interval. If bad value is provided then log it and set to default.
-   *
-   * @param instanceDefinition
-   */
-  private void initializeAgentDebugCommands(AggregateConf instanceDefinition) {
-    String launchParameterStr = instanceDefinition.getAppConfOperations().
-        getGlobalOptions().getOption(AGENT_INSTANCE_DEBUG_DATA, "");
-    agentLaunchParameter = new AgentLaunchParameter(launchParameterStr);
-  }
-
-  @VisibleForTesting
-  protected Map<String, ExportEntry> getLogFolderExports() {
-    return logFolderExports;
-  }
-
-  @VisibleForTesting
-  protected Map<String, ExportEntry> getWorkFolderExports() {
-    return workFolderExports;
-  }
-
-  protected Metainfo getMetaInfo() {
-    return getMetaInfo(null);
-  }
-
-  @VisibleForTesting
-  protected Metainfo getMetaInfo(String roleGroup) {
-    String mapKey = DEFAULT_METAINFO_MAP_KEY;
-    if (roleGroup != null) {
-      ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
-      mapKey = appConf.getComponentOpt(roleGroup, ROLE_PREFIX,
-          DEFAULT_METAINFO_MAP_KEY);
-    }
-    MetainfoHolder mh = this.metaInfoMap.get(mapKey);
-    if (mh == null) {
-      return null;
-    }
-    return mh.metaInfo;
-  }
-
-  @VisibleForTesting
-  protected Map<String, ComponentInstanceState> getComponentStatuses() {
-    return componentStatuses;
-  }
-
-  @VisibleForTesting
-  protected Metainfo getApplicationMetainfo(SliderFileSystem fileSystem,
-      String appDef, boolean addonPackage) throws IOException,
-      BadConfigException {
-    return AgentUtils.getApplicationMetainfo(fileSystem, appDef, addonPackage);
-  }
-
-  @VisibleForTesting
-  protected Metainfo getApplicationMetainfo(SliderFileSystem fileSystem,
-      String appDef) throws IOException, BadConfigException {
-    return getApplicationMetainfo(fileSystem, appDef, false);
-  }
-
-  @VisibleForTesting
-  protected void setHeartbeatMonitorInterval(int heartbeatMonitorInterval) {
-    this.heartbeatMonitorInterval = heartbeatMonitorInterval;
-  }
-
-  public void setInUpgradeMode(boolean inUpgradeMode) {
-    this.isInUpgradeMode = inUpgradeMode;
-  }
-
-  public void addUpgradeContainers(Set<String> upgradeContainers) {
-    this.upgradeContainers.addAll(upgradeContainers);
-  }
-
-  public void setAppStopInitiated(boolean appStopInitiated) {
-    this.appStopInitiated = appStopInitiated;
-  }
-
-  /**
-   * Read all default configs
-   *
-   * @param fileSystem fs
-   * @param appDef app default path
-   * @param metainfo metadata
-   *
-   * @return configuration maps
-   * 
-   * @throws IOException
-   */
-  protected Map<String, DefaultConfig> initializeDefaultConfigs(SliderFileSystem fileSystem,
-                                                                String appDef, Metainfo metainfo) throws IOException {
-    Map<String, DefaultConfig> defaultConfigMap = new HashMap<>();
-    if (SliderUtils.isNotEmpty(metainfo.getApplication().getConfigFiles())) {
-      for (ConfigFile configFile : metainfo.getApplication().getConfigFiles()) {
-        DefaultConfig config = null;
-        try {
-          config = AgentUtils.getDefaultConfig(fileSystem, appDef, configFile.getDictionaryName() + ".xml");
-        } catch (IOException e) {
-          log.warn("Default config file not found. Only the config as input during create will be applied for {}",
-                   configFile.getDictionaryName());
-        }
-        if (config != null) {
-          defaultConfigMap.put(configFile.getDictionaryName(), config);
-        }
-      }
-    }
-
-    return defaultConfigMap;
-  }
-
-  protected Map<String, DefaultConfig> getDefaultConfigs(String roleGroup) {
-    ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
-    String mapKey = appConf.getComponentOpt(roleGroup, ROLE_PREFIX,
-        DEFAULT_METAINFO_MAP_KEY);
-    return metaInfoMap.get(mapKey).defaultConfigs;
-  }
-
-  private int getHeartbeatMonitorInterval() {
-    return this.heartbeatMonitorInterval;
-  }
-
-  private String getClusterName() {
-    if (SliderUtils.isUnset(clusterName)) {
-      clusterName = getAmState().getInternalsSnapshot().get(OptionKeys.APPLICATION_NAME);
-    }
-    return clusterName;
-  }
-
-  @VisibleForTesting
-  protected void publishApplicationInstanceData(String name, String description,
-                                                Iterable<Map.Entry<String, String>> entries) {
-    providerUtils.publishApplicationInstanceData(name, description, entries,
-        getAmState());
-  }
-
-  /**
-   * Get a list of all hosts for all role/container per role
-   *
-   * @return the map of role->node
-   */
-  protected Map<String, Map<String, ClusterNode>> getRoleClusterNodeMapping() {
-    return amState.getRoleClusterNodeMapping();
-  }
-
-  private String getContainerLabel(Container container, String role, String group) {
-    if (role.equals(group)) {
-      return container.getId().toString() + LABEL_MAKER + role;
-    } else {
-      return container.getId().toString() + LABEL_MAKER + role + LABEL_MAKER +
-          group;
-    }
-  }
-
-  protected String getClusterInfoPropertyValue(String name) {
-    StateAccessForProviders accessor = getAmState();
-    assert accessor.isApplicationLive();
-    ClusterDescription description = accessor.getClusterStatus();
-    return description.getInfo(name);
-  }
-
-  protected String getClusterOptionPropertyValue(String name)
-      throws BadConfigException {
-    StateAccessForProviders accessor = getAmState();
-    assert accessor.isApplicationLive();
-    ClusterDescription description = accessor.getClusterStatus();
-    return description.getMandatoryOption(name);
-  }
-
-  /**
-   * Lost heartbeat from the container - release it and ask for a replacement (async operation)
-   *
-   * @param label
-   * @param containerId
-   */
-  protected void lostContainer(
-      String label,
-      ContainerId containerId) {
-    getComponentStatuses().remove(label);
-    getQueueAccess().put(new ProviderReportedContainerLoss(containerId));
-  }
-
-  /**
-   * Build the provider status, can be empty
-   *
-   * @return the provider status - map of entries to add to the info section
-   */
-  public Map<String, String> buildProviderStatus() {
-    Map<String, String> stats = new HashMap<String, String>();
-    return stats;
-  }
-
-  @VisibleForTesting
-  protected void publishFolderPaths(
-      Map<String, String> folders, String containerId, String componentName, String hostFqdn) {
-    providerUtils.publishFolderPaths(folders, containerId, componentName, hostFqdn,
-        getAmState(), getLogFolderExports(), getWorkFolderExports());
-  }
-
-  /**
-   * Process return status for component instances
-   *
-   * @param heartBeat
-   * @param componentStatus
-   */
-  protected void publishConfigAndExportGroups(HeartBeat heartBeat,
-      ComponentInstanceState componentStatus, String componentGroup) {
-    List<ComponentStatus> statuses = heartBeat.getComponentStatus();
-    if (statuses != null && !statuses.isEmpty()) {
-      log.info("Processing {} status reports.", statuses.size());
-      for (ComponentStatus status : statuses) {
-        log.info("Status report: {}", status.toString());
-
-        if (status.getConfigs() != null) {
-          Application application = getMetaInfo(componentGroup).getApplication();
-
-          if ((!canAnyMasterPublishConfig(componentGroup) || canPublishConfig(componentGroup)) &&
-              !getAmState().getAppConfSnapshot().getComponentOptBool(
-                  componentGroup, AM_CONFIG_GENERATION, false)) {
-            // If no Master can explicitly publish then publish if its a master
-            // Otherwise, wait till the master that can publish is ready
-
-            Set<String> exportedConfigs = new HashSet();
-            String exportedConfigsStr = application.getExportedConfigs();
-            boolean exportedAllConfigs = exportedConfigsStr == null || exportedConfigsStr.isEmpty();
-            if (!exportedAllConfigs) {
-              for (String exportedConfig : exportedConfigsStr.split(",")) {
-                if (exportedConfig.trim().length() > 0) {
-                  exportedConfigs.add(exportedConfig.trim());
-                }
-              }
-            }
-
-            for (String key : status.getConfigs().keySet()) {
-              if ((!exportedAllConfigs && exportedConfigs.contains(key)) ||
-                  exportedAllConfigs) {
-                Map<String, String> configs = status.getConfigs().get(key);
-                publishApplicationInstanceData(key, key, configs.entrySet());
-              }
-            }
-          }
-
-          List<ExportGroup> appExportGroups = application.getExportGroups();
-          boolean hasExportGroups = SliderUtils.isNotEmpty(appExportGroups);
-
-          Set<String> appExports = new HashSet();
-          String appExportsStr = getApplicationComponent(componentGroup).getAppExports();
-          if (SliderUtils.isSet(appExportsStr)) {
-            for (String appExport : appExportsStr.split(",")) {
-              if (!appExport.trim().isEmpty()) {
-                appExports.add(appExport.trim());
-              }
-            }
-          }
-
-          if (hasExportGroups && !appExports.isEmpty()) {
-            String configKeyFormat = "${site.%s.%s}";
-            String hostKeyFormat = "${%s_HOST}";
-
-            // publish export groups if any
-            Map<String, String> replaceTokens = new HashMap<String, String>();
-            for (Map.Entry<String, Map<String, ClusterNode>> entry : getRoleClusterNodeMapping().entrySet()) {
-              String hostName = providerUtils.getHostsList(
-                  entry.getValue().values(), true).iterator().next();
-              replaceTokens.put(String.format(hostKeyFormat, entry.getKey().toUpperCase(Locale.ENGLISH)), hostName);
-            }
-
-            for (String key : status.getConfigs().keySet()) {
-              Map<String, String> configs = status.getConfigs().get(key);
-              for (String configKey : configs.keySet()) {
-                String lookupKey = String.format(configKeyFormat, key, configKey);
-                replaceTokens.put(lookupKey, configs.get(configKey));
-              }
-            }
-
-            Set<String> modifiedGroups = new HashSet<String>();
-            for (ExportGroup exportGroup : appExportGroups) {
-              List<Export> exports = exportGroup.getExports();
-              if (SliderUtils.isNotEmpty(exports)) {
-                String exportGroupName = exportGroup.getName();
-                ConcurrentHashMap<String, List<ExportEntry>> map =
-                    (ConcurrentHashMap<String, List<ExportEntry>>)getCurrentExports(exportGroupName);
-                for (Export export : exports) {
-                  if (canBeExported(exportGroupName, export.getName(), appExports)) {
-                    String value = export.getValue();
-                    // replace host names
-                    for (String token : replaceTokens.keySet()) {
-                      if (value.contains(token)) {
-                        value = value.replace(token, replaceTokens.get(token));
-                      }
-                    }
-                    ExportEntry entry = new ExportEntry();
-                    entry.setLevel(APPLICATION_TAG);
-                    entry.setValue(value);
-                    entry.setUpdatedTime(new Date().toString());
-                    // over-write, app exports are singletons
-                    map.put(export.getName(), new ArrayList(Arrays.asList(entry)));
-                    log.info("Preparing to publish. Key {} and Value {}", export.getName(), value);
-                  }
-                }
-                modifiedGroups.add(exportGroupName);
-              }
-            }
-            publishModifiedExportGroups(modifiedGroups);
-          }
-
-          log.info("Received and processed config for {}", heartBeat.getHostname());
-          componentStatus.setConfigReported(true);
-
-        }
-      }
-    }
-  }
-
-  private boolean canBeExported(String exportGroupName, String name, Set<String> appExports) {
-    return appExports.contains(String.format("%s-%s", exportGroupName, name));
-  }
-
-  protected Map<String, List<ExportEntry>> getCurrentExports(String groupName) {
-    if (!this.exportGroups.containsKey(groupName)) {
-      synchronized (this.exportGroups) {
-        if (!this.exportGroups.containsKey(groupName)) {
-          this.exportGroups.put(groupName, new ConcurrentHashMap<String, List<ExportEntry>>());
-        }
-      }
-    }
-
-    return this.exportGroups.get(groupName);
-  }
-
-  private void publishModifiedExportGroups(Set<String> modifiedGroups) {
-    for (String roleGroup : modifiedGroups) {
-      Map<String, List<ExportEntry>> entries = this.exportGroups.get(roleGroup);
-      // Publish in old format for the time being
-      Map<String, String> simpleEntries = new HashMap<String, String>();
-      for (Entry<String, List<ExportEntry>> entry : entries.entrySet()) {
-        List<ExportEntry> exports = entry.getValue();
-        if (SliderUtils.isNotEmpty(exports)) {
-          // there is no support for multiple exports per name - so extract only the first one
-          simpleEntries.put(entry.getKey(), entry.getValue().get(0).getValue());
-        }
-      }
-      publishApplicationInstanceData(roleGroup, roleGroup,
-          simpleEntries.entrySet());
-
-      PublishedExports exports = new PublishedExports(roleGroup);
-      exports.setUpdated(new Date().getTime());
-      exports.putValues(entries.entrySet());
-      getAmState().getPublishedExportsSet().put(roleGroup, exports);
-    }
-  }
-
-  /** Publish component instance specific data if the component demands it */
-  protected void processAndPublishComponentSpecificData(Map<String, String> ports,
-                                                        String containerId,
-                                                        String hostFqdn,
-                                                        String componentGroup) {
-    String portVarFormat = "${site.%s}";
-    String hostNamePattern = "${THIS_HOST}";
-    Map<String, String> toPublish = new HashMap<String, String>();
-
-    Application application = getMetaInfo(componentGroup).getApplication();
-    for (Component component : application.getComponents()) {
-      if (component.getName().equals(componentGroup)) {
-        if (component.getComponentExports().size() > 0) {
-
-          for (ComponentExport export : component.getComponentExports()) {
-            String templateToExport = export.getValue();
-            for (String portName : ports.keySet()) {
-              boolean publishData = false;
-              String portValPattern = String.format(portVarFormat, portName);
-              if (templateToExport.contains(portValPattern)) {
-                templateToExport = templateToExport.replace(portValPattern, ports.get(portName));
-                publishData = true;
-              }
-              if (templateToExport.contains(hostNamePattern)) {
-                templateToExport = templateToExport.replace(hostNamePattern, hostFqdn);
-                publishData = true;
-              }
-              if (publishData) {
-                toPublish.put(export.getName(), templateToExport);
-                log.info("Publishing {} for name {} and container {}",
-                         templateToExport, export.getName(), containerId);
-              }
-            }
-          }
-        }
-      }
-    }
-
-    if (toPublish.size() > 0) {
-      Map<String, String> perContainerData = null;
-      if (!getComponentInstanceData().containsKey(containerId)) {
-        perContainerData = new ConcurrentHashMap<String, String>();
-      } else {
-        perContainerData = getComponentInstanceData().get(containerId);
-      }
-      perContainerData.putAll(toPublish);
-      getComponentInstanceData().put(containerId, perContainerData);
-      publishComponentInstanceData();
-    }
-  }
-
-  /** Publish component instance specific data if the component demands it */
-  protected void processAndPublishComponentSpecificExports(Map<String, String> ports,
-                                                           String containerId,
-                                                           String hostFqdn,
-                                                           String compName,
-                                                           String compGroup) {
-    String portVarFormat = "${site.%s}";
-    String hostNamePattern = "${" + compGroup + "_HOST}";
-
-    List<ExportGroup> appExportGroups = getMetaInfo(compGroup).getApplication().getExportGroups();
-    Component component = getApplicationComponent(compGroup);
-    if (component != null && SliderUtils.isSet(component.getCompExports())
-        && SliderUtils.isNotEmpty(appExportGroups)) {
-
-      Set<String> compExports = new HashSet();
-      String compExportsStr = component.getCompExports();
-      for (String compExport : compExportsStr.split(",")) {
-        if (!compExport.trim().isEmpty()) {
-          compExports.add(compExport.trim());
-        }
-      }
-
-      Date now = new Date();
-      Set<String> modifiedGroups = new HashSet<String>();
-      for (ExportGroup exportGroup : appExportGroups) {
-        List<Export> exports = exportGroup.getExports();
-        if (SliderUtils.isNotEmpty(exports)) {
-          String exportGroupName = exportGroup.getName();
-          ConcurrentHashMap<String, List<ExportEntry>> map =
-              (ConcurrentHashMap<String, List<ExportEntry>>) getCurrentExports(exportGroupName);
-          for (Export export : exports) {
-            if (canBeExported(exportGroupName, export.getName(), compExports)) {
-              log.info("Attempting to publish {} of group {} for component type {}",
-                       export.getName(), exportGroupName, compName);
-              String templateToExport = export.getValue();
-              for (String portName : ports.keySet()) {
-                boolean publishData = false;
-                String portValPattern = String.format(portVarFormat, portName);
-                if (templateToExport.contains(portValPattern)) {
-                  templateToExport = templateToExport.replace(portValPattern, ports.get(portName));
-                  publishData = true;
-                }
-                if (templateToExport.contains(hostNamePattern)) {
-                  templateToExport = templateToExport.replace(hostNamePattern, hostFqdn);
-                  publishData = true;
-                }
-                if (publishData) {
-                  ExportEntry entryToAdd = new ExportEntry();
-                  entryToAdd.setLevel(COMPONENT_TAG);
-                  entryToAdd.setValue(templateToExport);
-                  entryToAdd.setUpdatedTime(now.toString());
-                  entryToAdd.setContainerId(containerId);
-                  entryToAdd.setTag(tags.getTag(compName, containerId));
-
-                  List<ExportEntry> existingList =
-                      map.putIfAbsent(export.getName(), new CopyOnWriteArrayList(Arrays.asList(entryToAdd)));
-
-                  // in-place edit, no lock needed
-                  if (existingList != null) {
-                    boolean updatedInPlace = false;
-                    for (ExportEntry entry : existingList) {
-                      if (containerId.toLowerCase(Locale.ENGLISH)
-                                     .equals(entry.getContainerId())) {
-                        entryToAdd.setValue(templateToExport);
-                        entryToAdd.setUpdatedTime(now.toString());
-                        updatedInPlace = true;
-                      }
-                    }
-                    if (!updatedInPlace) {
-                      existingList.add(entryToAdd);
-                    }
-                  }
-
-                  log.info("Publishing {} for name {} and container {}",
-                           templateToExport, export.getName(), containerId);
-                  modifiedGroups.add(exportGroupName);
-                  synchronized (containerExportsMap) {
-                    if (!containerExportsMap.containsKey(containerId)) {
-                      containerExportsMap.put(containerId, new HashSet<String>());
-                    }
-                    Set<String> containerExportMaps = containerExportsMap.get(containerId);
-                    containerExportMaps.add(String.format("%s:%s", exportGroupName, export.getName()));
-                  }
-                }
-              }
-            }
-          }
-        }
-      }
-      publishModifiedExportGroups(modifiedGroups);
-    }
-  }
-
-  private void publishComponentInstanceData() {
-    Map<String, String> dataToPublish = new HashMap<String, String>();
-    for (String container : getComponentInstanceData().keySet()) {
-      for (String prop : getComponentInstanceData().get(container).keySet()) {
-        dataToPublish.put(
-            container + "." + prop, getComponentInstanceData().get(container).get(prop));
-      }
-    }
-    publishApplicationInstanceData(COMPONENT_DATA_TAG, COMPONENT_DATA_TAG, dataToPublish.entrySet());
-  }
-
-  /**
-   * Return Component based on group
-   *
-   * @param roleGroup component group
-   *
-   * @return the component entry or null for no match
-   */
-  protected Component getApplicationComponent(String roleGroup) {
-    Metainfo metainfo = getMetaInfo(roleGroup);
-    if (metainfo == null) {
-      return null;
-    }
-    return metainfo.getApplicationComponent(roleGroup);
-  }
-
-  /**
-   * Extract script path from the application metainfo
-   *
-   * @param roleGroup component group
-   * @return the script path or null for no match
-   */
-  protected CommandScript getScriptPathForMasterPackage(String roleGroup) {
-    Component component = getApplicationComponent(roleGroup);
-    if (component != null) {
-      return component.getCommandScript();
-    }
-    return null;
-  }
-
-  /**
-   * Is the role of type MASTER
-   *
-   * @param roleGroup component group
-   *
-   * @return true if the role category is MASTER
-   */
-  protected boolean isMaster(String roleGroup) {
-    Component component = getApplicationComponent(roleGroup);
-    if (component != null) {
-      if (component.getCategory().equals("MASTER")) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  /**
-   * Can the role publish configuration
-   *
-   * @param roleGroup component group
-   *
-   * @return true if it can be pubished
-   */
-  protected boolean canPublishConfig(String roleGroup) {
-    Component component = getApplicationComponent(roleGroup);
-    if (component != null) {
-      return Boolean.TRUE.toString().equals(component.getPublishConfig());
-    }
-    return false;
-  }
-
-  /**
-   * Checks if the role is marked auto-restart
-   *
-   * @param roleGroup component group
-   *
-   * @return true if it is auto-restart
-   */
-  protected boolean isMarkedAutoRestart(String roleGroup) {
-    Component component = getApplicationComponent(roleGroup);
-    if (component != null) {
-      return component.getAutoStartOnFailureBoolean();
-    }
-    return false;
-  }
-
-  /**
-   * Can any master publish config explicitly, if not a random master is used
-   *
-   * @return true if the condition holds
-   */
-  protected boolean canAnyMasterPublishConfig(String roleGroup) {
-    if (canAnyMasterPublish == null) {
-      Application application = getMetaInfo(roleGroup).getApplication();
-      if (application == null) {
-        log.error("Malformed app definition: Expect application as root element in the metainfo.xml");
-      } else {
-        for (Component component : application.getComponents()) {
-          if (Boolean.TRUE.toString().equals(component.getPublishConfig()) &&
-              component.getCategory().equals("MASTER")) {
-            canAnyMasterPublish = true;
-          }
-        }
-      }
-    }
-
-    if (canAnyMasterPublish == null) {
-      canAnyMasterPublish = false;
-    }
-    return canAnyMasterPublish;
-  }
-
-  private String getRoleName(String label) {
-    int index1 = label.indexOf(LABEL_MAKER);
-    int index2 = label.lastIndexOf(LABEL_MAKER);
-    if (index1 == index2) {
-      return label.substring(index1 + LABEL_MAKER.length());
-    } else {
-      return label.substring(index1 + LABEL_MAKER.length(), index2);
-    }
-  }
-
-  private String getRoleGroup(String label) {
-    return label.substring(label.lastIndexOf(LABEL_MAKER) + LABEL_MAKER.length());
-  }
-
-  private String getContainerId(String label) {
-    return label.substring(0, label.indexOf(LABEL_MAKER));
-  }
-
-  /**
-   * Add install command to the heartbeat response
-   *
-   * @param roleName
-   * @param roleGroup
-   * @param containerId
-   * @param response
-   * @param scriptPath
-   * @param pkg
-   *          when this field is null, it indicates the command is for the
-   *          master package; while not null, for the package named by this
-   *          field
-   * @throws SliderException
-   */
-  @VisibleForTesting
-  protected void addInstallCommand(String roleName,
-                                   String roleGroup,
-                                   String containerId,
-                                   HeartBeatResponse response,
-                                   String scriptPath,
-                                   ComponentCommand compCmd,
-                                   long timeout,
-                                   String pkg)
-      throws SliderException {
-    assert getAmState().isApplicationLive();
-    ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
-
-    ExecutionCommand cmd = new ExecutionCommand(AgentCommandType.EXECUTION_COMMAND);
-    prepareExecutionCommand(cmd);
-    String clusterName = getClusterName();
-    cmd.setClusterName(clusterName);
-    cmd.setRoleCommand(Command.INSTALL.toString());
-    cmd.setServiceName(clusterName);
-    cmd.setComponentName(roleName);
-    cmd.setRole(roleName);
-    cmd.setPkg(pkg);
-    Map<String, String> hostLevelParams = new TreeMap<String, String>();
-    hostLevelParams.put(JAVA_HOME, appConf.getGlobalOptions().getOption(JAVA_HOME, getJDKDir()));
-    hostLevelParams.put(PACKAGE_LIST, getPackageList(roleGroup));
-    hostLevelParams.put(CONTAINER_ID, containerId);
-    cmd.setHostLevelParams(hostLevelParams);
-
-    Map<String, Map<String, String>> configurations =
-        buildCommandConfigurations(appConf, getAmState().getInternalsSnapshot(),
-            containerId, roleName, roleGroup);
-    cmd.setConfigurations(configurations);
-    Map<String, Map<String, String>> componentConfigurations = buildComponentConfigurations(appConf);
-    cmd.setComponentConfigurations(componentConfigurations);
-
-    if (SliderUtils.isSet(scriptPath)) {
-      cmd.setCommandParams(commandParametersSet(scriptPath, timeout, false));
-    } else {
-      // assume it to be default shell command
-      ComponentCommand effectiveCommand = compCmd;
-      if (effectiveCommand == null) {
-        effectiveCommand = ComponentCommand.getDefaultComponentCommand("INSTALL");
-      }
-      cmd.setCommandParams(commandParametersSet(effectiveCommand, timeout, false));
-      configurations.get("global").put("exec_cmd", effectiveCommand.getExec());
-    }
-
-    cmd.setHostname(getClusterInfoPropertyValue(StatusKeys.INFO_AM_HOSTNAME));
-
-    response.addExecutionCommand(cmd);
-
-    log.debug("command looks like: {} ",  cmd);
-  }
-
-  @VisibleForTesting
-  protected void addInstallDockerCommand(String roleName,
-                                   String roleGroup,
-                                   String containerId,
-                                   HeartBeatResponse response,
-                                   ComponentCommand compCmd,
-                                   long timeout)
-      throws SliderException {
-    assert getAmState().isApplicationLive();
-    ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
-
-    ExecutionCommand cmd = new ExecutionCommand(AgentCommandType.EXECUTION_COMMAND);
-    prepareExecutionCommand(cmd);
-    String clusterName = getClusterName();
-    cmd.setClusterName(clusterName);
-    cmd.setRoleCommand(Command.INSTALL.toString());
-    cmd.setServiceName(clusterName);
-    cmd.setComponentName(roleName);
-    cmd.setRole(roleName);
-    Map<String, String> hostLevelParams = new TreeMap<String, String>();
-    hostLevelParams.put(PACKAGE_LIST, getPackageList(roleGroup));
-    hostLevelParams.put(CONTAINER_ID, containerId);
-    cmd.setHostLevelParams(hostLevelParams);
-
-    Map<String, Map<String, String>> configurations = buildCommandConfigurations(
-        appConf, getAmState().getInternalsSnapshot(), containerId, roleName,
-        roleGroup);
-    cmd.setConfigurations(configurations);
-    Map<String, Map<String, String>> componentConfigurations = buildComponentConfigurations(appConf);
-    cmd.setComponentConfigurations(componentConfigurations);
-    
-    ComponentCommand effectiveCommand = compCmd;
-    if (compCmd == null) {
-      effectiveCommand = new ComponentCommand();
-      effectiveCommand.setName("INSTALL");
-      effectiveCommand.setExec("DEFAULT");
-    }
-    cmd.setCommandParams(setCommandParameters(effectiveCommand, timeout, false));
-    configurations.get("global").put("exec_cmd", effectiveCommand.getExec());
-
-    cmd.setHostname(getClusterInfoPropertyValue(StatusKeys.INFO_AM_HOSTNAME));
-    cmd.addContainerDetails(roleGroup, getMetaInfo(roleGroup));
-
-    Map<String, String> dockerConfig = new HashMap<String, String>();
-    if(isYarnDockerContainer(roleGroup)){
-      //put nothing
-      cmd.setYarnDockerMode(true);
-    } else {
-      dockerConfig.put(
-          "docker.command_path",
-          getConfigFromMetaInfoWithAppConfigOverriding(roleGroup,
-              "commandPath"));
-      dockerConfig.put("docker.image_name",
-          getConfigFromMetaInfo(roleGroup, "image"));
-    }
-    configurations.put("docker", dockerConfig);
-
-    log.debug("Docker- command: {}", cmd.toString());
-
-    response.addExecutionCommand(cmd);
-  }
-
-  private Map<String, String> setCommandParameters(String scriptPath,
-      long timeout, boolean recordConfig) {
-    Map<String, String> cmdParams = new TreeMap<String, String>();
-    cmdParams.put("service_package_folder",
-        "${AGENT_WORK_ROOT}/work/app/definition/package");
-    cmdParams.put("script", scriptPath);
-    cmdParams.put("schema_version", "2.0");
-    cmdParams.put("command_timeout", Long.toString(timeout));
-    cmdParams.put("script_type", AbstractComponent.TYPE_PYTHON);
-    cmdParams.put("record_config", Boolean.toString(recordConfig));
-    return cmdParams;
-  }
-
-  private Map<String, String> setCommandParameters(ComponentCommand compCmd,
-      long timeout, boolean recordConfig) {
-    Map<String, String> cmdParams = new TreeMap<String, String>();
-    cmdParams.put("service_package_folder",
-        "${AGENT_WORK_ROOT}/work/app/definition/package");
-    cmdParams.put("command", compCmd.getExec());
-    cmdParams.put("schema_version", "2.0");
-    cmdParams.put("command_timeout", Long.toString(timeout));
-    cmdParams.put("script_type", compCmd.getType());
-    cmdParams.put("record_config", Boolean.toString(recordConfig));
-    return cmdParams;
-  }
-
-  private Map<String, Map<String, String>> buildComponentConfigurations(
-      ConfTreeOperations appConf) {
-    return appConf.getComponents();
-  }
-
-  protected static String getPackageListFromApplication(Application application) {
-    String pkgFormatString = "{\"type\":\"%s\",\"name\":\"%s\"}";
-    String pkgListFormatString = "[%s]";
-    List<

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[45/76] [abbrv] hadoop git commit: YARN-5610. Initial code for native services REST API. Contributed by Gour Saha

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Resource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Resource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Resource.java
new file mode 100644
index 0000000..a3780cc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Resource.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.services.resource;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Resource determines the amount of resources (vcores, memory, network, etc.)
+ * usable by a container. This field determines the resource to be applied for
+ * all the containers of a component or application. The resource specified at
+ * the app (or global) level can be overriden at the component level. Only one
+ * of profile OR cpu &amp; memory are exepected. It raises a validation
+ * exception otherwise.
+ **/
+
+@ApiModel(description = "Resource determines the amount of resources (vcores, memory, network, etc.) usable by a container. This field determines the resource to be applied for all the containers of a component or application. The resource specified at the app (or global) level can be overriden at the component level. Only one of profile OR cpu & memory are exepected. It raises a validation exception otherwise.")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+public class Resource extends BaseResource {
+  private static final long serialVersionUID = -6431667797380250037L;
+
+  private String profile = null;
+  private Integer cpus = null;
+  private String memory = null;
+
+  /**
+   * Each resource profile has a unique id which is associated with a
+   * cluster-level predefined memory, cpus, etc.
+   **/
+  public Resource profile(String profile) {
+    this.profile = profile;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Each resource profile has a unique id which is associated with a cluster-level predefined memory, cpus, etc.")
+  @JsonProperty("profile")
+  public String getProfile() {
+    return profile;
+  }
+
+  public void setProfile(String profile) {
+    this.profile = profile;
+  }
+
+  /**
+   * Amount of vcores allocated to each container (optional but overrides cpus
+   * in profile if specified).
+   **/
+  public Resource cpus(Integer cpus) {
+    this.cpus = cpus;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Amount of vcores allocated to each container (optional but overrides cpus in profile if specified).")
+  @JsonProperty("cpus")
+  public Integer getCpus() {
+    return cpus;
+  }
+
+  public void setCpus(Integer cpus) {
+    this.cpus = cpus;
+  }
+
+  /**
+   * Amount of memory allocated to each container (optional but overrides memory
+   * in profile if specified). Currently accepts only an integer value and
+   * default unit is in MB.
+   **/
+  public Resource memory(String memory) {
+    this.memory = memory;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Amount of memory allocated to each container (optional but overrides memory in profile if specified). Currently accepts only an integer value and default unit is in MB.")
+  @JsonProperty("memory")
+  public String getMemory() {
+    return memory;
+  }
+
+  public void setMemory(String memory) {
+    this.memory = memory;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    Resource resource = (Resource) o;
+    return Objects.equals(this.profile, resource.profile)
+        && Objects.equals(this.cpus, resource.cpus)
+        && Objects.equals(this.memory, resource.memory);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(profile, cpus, memory);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class Resource {\n");
+
+    sb.append("    profile: ").append(toIndentedString(profile)).append("\n");
+    sb.append("    cpus: ").append(toIndentedString(cpus)).append("\n");
+    sb.append("    memory: ").append(toIndentedString(memory)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    return super.clone();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiConstants.java
new file mode 100644
index 0000000..4c16546
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiConstants.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.services.utils;
+
+public interface RestApiConstants {
+  String CONTEXT_ROOT = "/services/v1";
+  String APPLICATIONS_API_RESOURCE_PATH = "/applications";
+  String CONTAINERS_API_RESOURCE_PATH = "/containers";
+  String SLIDER_APPMASTER_COMPONENT_NAME = "slider-appmaster";
+  String SLIDER_CONFIG_SCHEMA = "http://example.org/specification/v2.0.0";
+  String METAINFO_SCHEMA_VERSION = "2.1";
+  String COMPONENT_TYPE_YARN_DOCKER = "yarn_docker";
+
+  String DEFAULT_START_CMD = "/bootstrap/privileged-centos6-sshd";
+  String DEFAULT_COMPONENT_NAME = "DEFAULT";
+  String DEFAULT_IMAGE = "centos:centos6";
+  String DEFAULT_NETWORK = "bridge";
+  String DEFAULT_COMMAND_PATH = "/usr/bin/docker";
+  String DEFAULT_USE_NETWORK_SCRIPT = "yes";
+
+  String PLACEHOLDER_APP_NAME = "${APP_NAME}";
+  String PLACEHOLDER_APP_COMPONENT_NAME = "${APP_COMPONENT_NAME}";
+  String PLACEHOLDER_COMPONENT_ID = "${COMPONENT_ID}";
+
+  String PROPERTY_REST_SERVICE_HOST = "REST_SERVICE_HOST";
+  String PROPERTY_REST_SERVICE_PORT = "REST_SERVICE_PORT";
+  String PROPERTY_APP_LIFETIME = "docker.lifetime";
+  String PROPERTY_APP_RUNAS_USER = "APP_RUNAS_USER";
+  Long DEFAULT_UNLIMITED_LIFETIME = -1l;
+
+  Integer HTTP_STATUS_CODE_ACCEPTED = 202;
+  String ARTIFACT_TYPE_SLIDER_ZIP = "slider-zip";
+
+  Integer GET_APPLICATIONS_THREAD_POOL_SIZE = 200;
+
+  String PROPERTY_PYTHON_PATH = "python.path";
+  String PROPERTY_COMPONENT_TYPE = "site.global.component_type";
+  String PROPERTY_DNS_DEPENDENCY = "site.global.dns.dependency";
+
+  String COMPONENT_TYPE_EXTERNAL = "external";
+
+  String COMMAND_ORDER_SUFFIX_START = "-START";
+  String COMMAND_ORDER_SUFFIX_STARTED = "-STARTED";
+  String EXPORT_GROUP_NAME = "QuickLinks";
+
+  Integer ERROR_CODE_APP_DOES_NOT_EXIST = 404001;
+  Integer ERROR_CODE_APP_IS_NOT_RUNNING = 404002;
+  Integer ERROR_CODE_APP_SUBMITTED_BUT_NOT_RUNNING_YET = 404003;
+  Integer ERROR_CODE_APP_NAME_INVALID = 404004;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiErrorMessages.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiErrorMessages.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiErrorMessages.java
new file mode 100644
index 0000000..685f85a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiErrorMessages.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.services.utils;
+
+public interface RestApiErrorMessages {
+  String ERROR_APPLICATION_NAME_INVALID =
+      "Application name is either empty or not provided";
+  String ERROR_APPLICATION_NAME_INVALID_FORMAT =
+      "Application name is not valid - only lower case letters, digits,"
+          + " underscore and hyphen are allowed";
+
+  String ERROR_APPLICATION_NOT_RUNNING = "Application not running";
+  String ERROR_APPLICATION_DOES_NOT_EXIST = "Application not found";
+
+  String ERROR_SUFFIX_FOR_COMPONENT =
+      " for component %s (nor at the global level)";
+  String ERROR_ARTIFACT_INVALID = "Artifact is not provided";
+  String ERROR_ARTIFACT_FOR_COMP_INVALID =
+      ERROR_ARTIFACT_INVALID + ERROR_SUFFIX_FOR_COMPONENT;
+  String ERROR_ARTIFACT_ID_INVALID =
+      "Artifact id (like docker image name) is either empty or not provided";
+  String ERROR_ARTIFACT_ID_FOR_COMP_INVALID =
+      ERROR_ARTIFACT_ID_INVALID + ERROR_SUFFIX_FOR_COMPONENT;
+
+  String ERROR_RESOURCE_INVALID = "Resource is not provided";
+  String ERROR_RESOURCE_FOR_COMP_INVALID =
+      ERROR_RESOURCE_INVALID + ERROR_SUFFIX_FOR_COMPONENT;
+  String ERROR_RESOURCE_MEMORY_INVALID =
+      "Application resource or memory not provided";
+  String ERROR_RESOURCE_CPUS_INVALID =
+      "Application resource or cpus not provided";
+  String ERROR_RESOURCE_CPUS_INVALID_RANGE =
+      "Unacceptable no of cpus specified, either zero or negative";
+  String ERROR_RESOURCE_MEMORY_FOR_COMP_INVALID =
+      ERROR_RESOURCE_MEMORY_INVALID + ERROR_SUFFIX_FOR_COMPONENT;
+  String ERROR_RESOURCE_CPUS_FOR_COMP_INVALID =
+      ERROR_RESOURCE_CPUS_INVALID + ERROR_SUFFIX_FOR_COMPONENT;
+  String ERROR_RESOURCE_CPUS_FOR_COMP_INVALID_RANGE =
+      ERROR_RESOURCE_CPUS_INVALID_RANGE
+          + " for component %s (or at the global level)";
+  String ERROR_CONTAINERS_COUNT_INVALID =
+      "Required no of containers not specified";
+  String ERROR_CONTAINERS_COUNT_FOR_COMP_INVALID =
+      ERROR_CONTAINERS_COUNT_INVALID + ERROR_SUFFIX_FOR_COMPONENT;
+
+  String ERROR_RESOURCE_PROFILE_MULTIPLE_VALUES_NOT_SUPPORTED =
+      "Cannot specify" + " cpus/memory along with profile";
+  String ERROR_RESOURCE_PROFILE_MULTIPLE_VALUES_FOR_COMP_NOT_SUPPORTED =
+      ERROR_RESOURCE_PROFILE_MULTIPLE_VALUES_NOT_SUPPORTED
+          + " for component %s";
+  String ERROR_RESOURCE_PROFILE_NOT_SUPPORTED_YET =
+      "Resource profile is not " + "supported yet. Please specify cpus/memory.";
+
+  String ERROR_APPLICATION_IN_USE = "Application name is already in use";
+  String ERROR_NULL_ARTIFACT_ID =
+      "Artifact Id can not be null if artifact type is none";
+  String ERROR_ABSENT_NUM_OF_INSTANCE =
+      "Num of instances should appear either globally or per component";
+  String ERROR_ABSENT_LAUNCH_COMMAND =
+      "launch command should appear if type is slider-zip or none";
+
+  String ERROR_QUICKLINKS_FOR_COMP_INVALID =
+      "Quicklinks specified at component level, needs corresponding values set at application level";
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/webapp/ApplicationApiWebApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/webapp/ApplicationApiWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/webapp/ApplicationApiWebApp.java
new file mode 100644
index 0000000..b1b6d7c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/webapp/ApplicationApiWebApp.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.services.webapp;
+
+import static org.apache.hadoop.yarn.services.utils.RestApiConstants.*;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.URI;
+import java.util.Arrays;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.http.HttpServer2;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
+import org.mortbay.jetty.webapp.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class launches the web application using Hadoop HttpServer2 (which uses
+ * an embedded Jetty container). This is the entry point to your application.
+ * The Java command used to launch this app should call the main method.
+ */
+public class ApplicationApiWebApp extends AbstractService {
+  private static final Logger logger = LoggerFactory
+      .getLogger(ApplicationApiWebApp.class);
+  private static final String SEP = ";";
+
+  // REST API server for YARN native services
+  private HttpServer2 applicationApiServer;
+
+  public static void main(String[] args) throws IOException {
+    ApplicationApiWebApp apiWebApp = new ApplicationApiWebApp();
+    try {
+      apiWebApp.startWebApp();
+    } catch (Exception e) {
+      if (apiWebApp != null) {
+        apiWebApp.close();
+      }
+    }
+  }
+
+  public ApplicationApiWebApp() {
+    super(ApplicationApiWebApp.class.getName());
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    startWebApp();
+    super.serviceStart();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    if (applicationApiServer != null) {
+      applicationApiServer.stop();
+    }
+    super.serviceStop();
+  }
+
+  protected void startWebApp() throws IOException {
+    // The port that we should run on can be set into an environment variable
+    // Look for that variable and default to 9191 if it isn't there.
+    String webPort = System.getenv(PROPERTY_REST_SERVICE_PORT);
+    if (StringUtils.isEmpty(webPort)) {
+      webPort = "9191";
+    }
+
+    String webHost = System.getenv(PROPERTY_REST_SERVICE_HOST);
+    if (StringUtils.isEmpty(webHost)) {
+      webHost = InetAddress.getLocalHost().getHostName();
+    }
+    logger.info("YARN native services REST API running on host {} and port {}",
+        webHost, webPort);
+    logger.info("Configuration = {}", getConfig());
+
+    applicationApiServer = new HttpServer2.Builder()
+        .setName("services-rest-api")
+        .addEndpoint(URI.create("http://" + webHost + ":" + webPort)).build();
+
+    String apiPackages = "org.apache.hadoop.yarn.services.api" + SEP
+        + "org.apache.hadoop.yarn.services.api.impl" + SEP
+        + "org.apache.hadoop.yarn.services.resource" + SEP
+        + "org.apache.hadoop.yarn.services.utils" + SEP
+        + "org.apache.hadoop.yarn.services.webapp" + SEP
+        + GenericExceptionHandler.class.getPackage().getName() + SEP
+        + YarnJacksonJaxbJsonProvider.class.getPackage().getName();
+    applicationApiServer.addJerseyResourcePackage(apiPackages, CONTEXT_ROOT
+        + "/*");
+
+    try {
+      logger.info("Application starting up. Logging start...");
+      applicationApiServer.start();
+      logger.info("Server status = {}", applicationApiServer.toString());
+      for (Configuration conf : applicationApiServer.getWebAppContext()
+          .getConfigurations()) {
+        logger.info("Configurations = {}", conf);
+      }
+      logger.info("Context Path = {}", Arrays.asList(applicationApiServer
+          .getWebAppContext().getContextPath()));
+      logger.info("ResourceBase = {}", Arrays.asList(applicationApiServer
+          .getWebAppContext().getResourceBase()));
+      logger.info("War = {}",
+          Arrays.asList(applicationApiServer.getWebAppContext().getWar()));
+    } catch (Exception ex) {
+      logger.error("Hadoop HttpServer2 App **failed**", ex);
+      throw ex;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/log4j-server.properties
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/log4j-server.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/log4j-server.properties
new file mode 100644
index 0000000..8c679b9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/log4j-server.properties
@@ -0,0 +1,76 @@
+# 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.
+#
+
+# This is the log4j configuration for YARN Services REST API Server
+
+# Log rotation based on size (100KB) with a max of 10 backup files
+log4j.rootLogger=INFO, restservicelog
+log4j.threshhold=ALL
+
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2} (%F:%M(%L)) - %m%n
+
+log4j.appender.restservicelog=org.apache.log4j.RollingFileAppender
+log4j.appender.restservicelog.layout=org.apache.log4j.PatternLayout
+log4j.appender.restservicelog.File=${REST_SERVICE_LOG_DIR}/restservice.log
+log4j.appender.restservicelog.MaxFileSize=1GB
+log4j.appender.restservicelog.MaxBackupIndex=10
+
+# log layout skips stack-trace creation operations by avoiding line numbers and method
+log4j.appender.restservicelog.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} - %m%n
+
+# debug edition is much more expensive
+#log4j.appender.restservicelog.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n
+
+# configure stderr
+# set the conversion pattern of stderr
+# Print the date in ISO 8601 format
+log4j.appender.stderr=org.apache.log4j.ConsoleAppender
+log4j.appender.stderr.Target=System.err
+log4j.appender.stderr.layout=org.apache.log4j.PatternLayout
+log4j.appender.stderr.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} - %m%n
+
+log4j.appender.subprocess=org.apache.log4j.ConsoleAppender
+log4j.appender.subprocess.layout=org.apache.log4j.PatternLayout
+log4j.appender.subprocess.layout.ConversionPattern=[%c{1}]: %m%n
+
+# for debugging REST API Service
+#log4j.logger.org.apache.hadoop.yarn.services=DEBUG
+
+# uncomment to debug service lifecycle issues
+#log4j.logger.org.apache.hadoop.yarn.service.launcher=DEBUG
+#log4j.logger.org.apache.hadoop.yarn.service=DEBUG
+
+# uncomment for YARN operations
+#log4j.logger.org.apache.hadoop.yarn.client=DEBUG
+
+# uncomment this to debug security problems
+#log4j.logger.org.apache.hadoop.security=DEBUG
+
+#crank back on some noise
+log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR
+log4j.logger.org.apache.hadoop.hdfs=WARN
+log4j.logger.org.apache.hadoop.hdfs.shortcircuit=ERROR
+
+log4j.logger.org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor=WARN
+log4j.logger.org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdaterImpl=WARN
+log4j.logger.org.apache.zookeeper=WARN
+log4j.logger.org.apache.curator.framework.state=ERROR
+log4j.logger.org.apache.curator.framework.imps=WARN
+
+log4j.logger.org.mortbay.log=DEBUG

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/webapps/services-rest-api/app
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/webapps/services-rest-api/app b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/webapps/services-rest-api/app
new file mode 100644
index 0000000..6a077b1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/webapps/services-rest-api/app
@@ -0,0 +1,16 @@
+# 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.
+
+DON'T DELETE. REST WEBAPP RUN SCRIPT WILL STOP WORKING.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/scripts/run_rest_service.sh
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/scripts/run_rest_service.sh b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/scripts/run_rest_service.sh
new file mode 100644
index 0000000..9f15b7e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/scripts/run_rest_service.sh
@@ -0,0 +1,28 @@
+#!/usr/bin/env bash
+
+# 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.
+
+export SLIDER_VERSION=${project.version}
+export HDP_VERSION=${HDP_VERSION:-$SLIDER_VERSION}
+export SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )"
+export LIB_PARENT_DIR=`dirname $SCRIPT_DIR`
+export JAVA_HOME=${JAVA_HOME:-/usr/jdk64/jdk1.8.0_40}
+export HADOOP_CONF_DIR=${HADOOP_CONF_DIR:-/etc/hadoop/conf}
+export REST_SERVICE_PORT=${REST_SERVICE_PORT:-9191}
+export APP_RUNAS_USER=${APP_RUNAS_USER:-root}
+export REST_SERVICE_LOG_DIR=${REST_SERVICE_LOG_DIR:-/tmp/}
+export JAVA_OPTS="-Xms256m -Xmx1024m -XX:+PrintGC -Xloggc:$REST_SERVICE_LOG_DIR/gc.log"
+$JAVA_HOME/bin/java $JAVA_OPTS -cp .:$HADOOP_CONF_DIR:$LIB_PARENT_DIR/services-api/*:$LIB_PARENT_DIR/slider/* -DREST_SERVICE_LOG_DIR=$REST_SERVICE_LOG_DIR -Dlog4j.configuration=log4j-server.properties -Dslider.libdir=$LIB_PARENT_DIR/slider org.apache.hadoop.yarn.services.webapp.ApplicationApiWebApp 1>>$REST_SERVICE_LOG_DIR/restservice-out.log 2>&1

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/webapp/WEB-INF/web.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/webapp/WEB-INF/web.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/webapp/WEB-INF/web.xml
new file mode 100644
index 0000000..f2f8b5b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/webapp/WEB-INF/web.xml
@@ -0,0 +1,36 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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.
+-->
+<web-app xmlns="http://java.sun.com/xml/ns/javaee" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+        xsi:schemaLocation="http://java.sun.com/xml/ns/javaee http://java.sun.com/xml/ns/javaee/web-app_3_0.xsd"
+        version="3.0">
+
+    <servlet>
+        <servlet-name>Jersey REST API</servlet-name>
+        <servlet-class>com.sun.jersey.spi.container.servlet.ServletContainer</servlet-class>
+        <init-param>
+            <param-name>com.sun.jersey.config.property.packages</param-name>
+            <param-value>org.apache.hadoop.yarn.services.webapp,org.apache.hadoop.yarn.services.api,org.apache.hadoop.yarn.services.resource,org.apache.hadoop.yarn.services.api.impl</param-value>
+        </init-param>
+        <init-param>
+          <param-name>com.sun.jersey.api.json.POJOMappingFeature</param-name>
+          <param-value>true</param-value>
+        </init-param>
+        <load-on-startup>1</load-on-startup>
+    </servlet>
+    <servlet-mapping>
+        <servlet-name>Jersey REST API</servlet-name>
+        <url-pattern>/*</url-pattern>
+    </servlet-mapping>
+</web-app>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/services/api/impl/TestApplicationApiService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/services/api/impl/TestApplicationApiService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/services/api/impl/TestApplicationApiService.java
new file mode 100644
index 0000000..a03ab69
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/services/api/impl/TestApplicationApiService.java
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.services.api.impl;
+
+import static org.apache.hadoop.yarn.services.utils.RestApiConstants.*;
+import static org.apache.hadoop.yarn.services.utils.RestApiErrorMessages.*;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.yarn.services.resource.Application;
+import org.apache.hadoop.yarn.services.resource.Artifact;
+import org.apache.hadoop.yarn.services.resource.Resource;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.core.classloader.annotations.SuppressStaticInitializationFor;
+import org.powermock.modules.junit4.PowerMockRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test class for application life time monitor feature test.
+ */
+@RunWith(PowerMockRunner.class)
+@SuppressStaticInitializationFor("org.apache.hadoop.yarn.services.api.impl.ApplicationApiService")
+public class TestApplicationApiService {
+  private static final Logger logger = LoggerFactory
+      .getLogger(TestApplicationApiService.class);
+  private static String EXCEPTION_PREFIX = "Should have thrown exception: ";
+  private static String NO_EXCEPTION_PREFIX = "Should not have thrown exception: ";
+  private ApplicationApiService appApiService;
+
+  @Before
+  public void setup() throws Exception {
+     appApiService = new ApplicationApiService();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+  }
+
+  @Test(timeout = 90000)
+  public void testValidateApplicationPostPayload() throws Exception {
+    Application app = new Application();
+    Map<String, String> compNameArtifactIdMap = new HashMap<>();
+
+    // no name
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+      Assert.fail(EXCEPTION_PREFIX + "application with no name");
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals(ERROR_APPLICATION_NAME_INVALID, e.getMessage());
+    }
+
+    // bad format name
+    String[] badNames = { "4finance", "Finance", "finance@home" };
+    for (String badName : badNames) {
+      app.setName(badName);
+      try {
+        appApiService.validateApplicationPostPayload(app,
+            compNameArtifactIdMap);
+        Assert.fail(EXCEPTION_PREFIX + "application with bad name " + badName);
+      } catch (IllegalArgumentException e) {
+        Assert.assertEquals(ERROR_APPLICATION_NAME_INVALID_FORMAT,
+            e.getMessage());
+      }
+    }
+
+    // no artifact
+    app.setName("finance_home");
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+      Assert.fail(EXCEPTION_PREFIX + "application with no artifact");
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals(ERROR_ARTIFACT_INVALID, e.getMessage());
+    }
+
+    // no artifact id
+    Artifact artifact = new Artifact();
+    app.setArtifact(artifact);
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+      Assert.fail(EXCEPTION_PREFIX + "application with no artifact id");
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals(ERROR_ARTIFACT_ID_INVALID, e.getMessage());
+    }
+
+    // if artifact is of type APPLICATION then everything is valid here
+    artifact.setType(Artifact.TypeEnum.APPLICATION);
+    artifact.setId("app.io/hbase:facebook_0.2");
+    app.setNumberOfContainers(5l);
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+    } catch (IllegalArgumentException e) {
+      logger.error("application attributes specified should be valid here", e);
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+
+    // default-component, default-lifetime and the property component_type
+    // should get assigned here
+    Assert.assertEquals(app.getComponents().get(0).getName(),
+        DEFAULT_COMPONENT_NAME);
+    Assert.assertEquals(app.getLifetime(), DEFAULT_UNLIMITED_LIFETIME);
+    Assert.assertEquals("Property not set",
+        app.getConfiguration().getProperties().get(PROPERTY_COMPONENT_TYPE),
+        COMPONENT_TYPE_EXTERNAL);
+
+    // unset artifact type, default component and no of containers to test other
+    // validation logic
+    artifact.setType(null);
+    app.setComponents(null);
+    app.setNumberOfContainers(null);
+
+    // resource not specified
+    artifact.setId("docker.io/centos:centos7");
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+      Assert.fail(EXCEPTION_PREFIX + "application with no resource");
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals(ERROR_RESOURCE_INVALID, e.getMessage());
+    }
+
+    // memory not specified
+    Resource res = new Resource();
+    app.setResource(res);
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+      Assert.fail(EXCEPTION_PREFIX + "application with no memory");
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals(ERROR_RESOURCE_MEMORY_INVALID, e.getMessage());
+    }
+
+    // cpus not specified
+    res.setMemory("2gb");
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+      Assert.fail(EXCEPTION_PREFIX + "application with no cpu");
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals(ERROR_RESOURCE_CPUS_INVALID, e.getMessage());
+    }
+
+    // invalid no of cpus
+    res.setCpus(-2);
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+      Assert.fail(
+          EXCEPTION_PREFIX + "application with invalid no of cpups");
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals(ERROR_RESOURCE_CPUS_INVALID_RANGE, e.getMessage());
+    }
+
+    // number of containers not specified
+    res.setCpus(2);
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+      Assert.fail(
+          EXCEPTION_PREFIX + "application with no container count");
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals(ERROR_CONTAINERS_COUNT_INVALID, e.getMessage());
+    }
+
+    // specifying profile along with cpus/memory raises exception
+    res.setProfile("hbase_finance_large");
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+      Assert.fail(EXCEPTION_PREFIX
+          + "application with resource profile along with cpus/memory");
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals(ERROR_RESOURCE_PROFILE_MULTIPLE_VALUES_NOT_SUPPORTED,
+          e.getMessage());
+    }
+
+    // currently resource profile alone is not supported.
+    // TODO: remove the next test once it is supported.
+    res.setCpus(null);
+    res.setMemory(null);
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+      Assert.fail(EXCEPTION_PREFIX
+          + "application with resource profile only - NOT SUPPORTED");
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals(ERROR_RESOURCE_PROFILE_NOT_SUPPORTED_YET,
+          e.getMessage());
+    }
+
+    // unset profile here and add cpus/memory back
+    res.setProfile(null);
+    res.setCpus(2);
+    res.setMemory("2gb");
+
+    // everything valid here
+    app.setNumberOfContainers(5l);
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+    } catch (IllegalArgumentException e) {
+      logger.error("application attributes specified should be valid here", e);
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+
+    // Now test with components
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml
index 493e03a..2cec8bd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml
@@ -37,7 +37,7 @@
     <module>hadoop-yarn-applications-distributedshell</module>
     <module>hadoop-yarn-applications-unmanaged-am-launcher</module>
     <module>hadoop-yarn-slider</module>
-
+    <module>hadoop-yarn-services-api</module>
   </modules>
 
  <profiles>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[32/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
new file mode 100644
index 0000000..8210f4d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
@@ -0,0 +1,4569 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.slider.client;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.io.Files;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.registry.client.api.RegistryConstants;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
+import org.apache.hadoop.registry.client.binding.RegistryUtils;
+import org.apache.hadoop.registry.client.exceptions.NoRecordException;
+import org.apache.hadoop.registry.client.types.Endpoint;
+import org.apache.hadoop.registry.client.types.RegistryPathStatus;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.KerberosDiags;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.alias.CredentialProvider;
+import org.apache.hadoop.security.alias.CredentialProviderFactory;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException;
+import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.slider.api.ClusterDescription;
+import org.apache.slider.api.ClusterNode;
+import org.apache.slider.api.SliderApplicationApi;
+import org.apache.slider.api.SliderClusterProtocol;
+import org.apache.slider.api.StateValues;
+import org.apache.slider.api.proto.Messages;
+import org.apache.slider.api.types.ContainerInformation;
+import org.apache.slider.api.types.NodeInformationList;
+import org.apache.slider.api.types.SliderInstanceDescription;
+import org.apache.slider.client.ipc.SliderApplicationIpcClient;
+import org.apache.slider.client.ipc.SliderClusterOperations;
+import org.apache.slider.common.Constants;
+import org.apache.slider.common.SliderExitCodes;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.params.AbstractActionArgs;
+import org.apache.slider.common.params.AbstractClusterBuildingActionArgs;
+import org.apache.slider.common.params.ActionAMSuicideArgs;
+import org.apache.slider.common.params.ActionClientArgs;
+import org.apache.slider.common.params.ActionCreateArgs;
+import org.apache.slider.common.params.ActionDependencyArgs;
+import org.apache.slider.common.params.ActionDestroyArgs;
+import org.apache.slider.common.params.ActionDiagnosticArgs;
+import org.apache.slider.common.params.ActionEchoArgs;
+import org.apache.slider.common.params.ActionExistsArgs;
+import org.apache.slider.common.params.ActionFlexArgs;
+import org.apache.slider.common.params.ActionFreezeArgs;
+import org.apache.slider.common.params.ActionInstallKeytabArgs;
+import org.apache.slider.common.params.ActionInstallPackageArgs;
+import org.apache.slider.common.params.ActionKDiagArgs;
+import org.apache.slider.common.params.ActionKeytabArgs;
+import org.apache.slider.common.params.ActionKillContainerArgs;
+import org.apache.slider.common.params.ActionListArgs;
+import org.apache.slider.common.params.ActionLookupArgs;
+import org.apache.slider.common.params.ActionNodesArgs;
+import org.apache.slider.common.params.ActionPackageArgs;
+import org.apache.slider.common.params.ActionRegistryArgs;
+import org.apache.slider.common.params.ActionResolveArgs;
+import org.apache.slider.common.params.ActionResourceArgs;
+import org.apache.slider.common.params.ActionStatusArgs;
+import org.apache.slider.common.params.ActionThawArgs;
+import org.apache.slider.common.params.ActionTokensArgs;
+import org.apache.slider.common.params.ActionUpgradeArgs;
+import org.apache.slider.common.params.Arguments;
+import org.apache.slider.common.params.ClientArgs;
+import org.apache.slider.common.params.CommonArgs;
+import org.apache.slider.common.params.LaunchArgsAccessor;
+import org.apache.slider.common.tools.ConfigHelper;
+import org.apache.slider.common.tools.Duration;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.common.tools.SliderVersionInfo;
+import org.apache.slider.core.build.InstanceBuilder;
+import org.apache.slider.core.build.InstanceIO;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTree;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.conf.ResourcesInputPropertiesValidator;
+import org.apache.slider.core.conf.TemplateInputPropertiesValidator;
+import org.apache.slider.core.exceptions.BadClusterStateException;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.core.exceptions.ErrorStrings;
+import org.apache.slider.core.exceptions.NoSuchNodeException;
+import org.apache.slider.core.exceptions.NotFoundException;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.core.exceptions.UnknownApplicationInstanceException;
+import org.apache.slider.core.exceptions.UsageException;
+import org.apache.slider.core.exceptions.WaitTimeoutException;
+import org.apache.slider.core.launch.AppMasterLauncher;
+import org.apache.slider.core.launch.ClasspathConstructor;
+import org.apache.slider.core.launch.CredentialUtils;
+import org.apache.slider.core.launch.JavaCommandLineBuilder;
+import org.apache.slider.core.launch.LaunchedApplication;
+import org.apache.slider.core.launch.RunningApplication;
+import org.apache.slider.core.launch.SerializedApplicationReport;
+import org.apache.slider.core.main.RunService;
+import org.apache.slider.core.persist.AppDefinitionPersister;
+import org.apache.slider.core.persist.ApplicationReportSerDeser;
+import org.apache.slider.core.persist.ConfPersister;
+import org.apache.slider.core.persist.JsonSerDeser;
+import org.apache.slider.core.persist.LockAcquireFailedException;
+import org.apache.slider.core.registry.SliderRegistryUtils;
+import org.apache.slider.core.registry.YarnAppListClient;
+import org.apache.slider.core.registry.docstore.ConfigFormat;
+import org.apache.slider.core.registry.docstore.PublishedConfigSet;
+import org.apache.slider.core.registry.docstore.PublishedConfiguration;
+import org.apache.slider.core.registry.docstore.PublishedConfigurationOutputter;
+import org.apache.slider.core.registry.docstore.PublishedExports;
+import org.apache.slider.core.registry.docstore.PublishedExportsOutputter;
+import org.apache.slider.core.registry.docstore.PublishedExportsSet;
+import org.apache.slider.core.registry.retrieve.RegistryRetriever;
+import org.apache.slider.core.zk.BlockingZKWatcher;
+import org.apache.slider.core.zk.ZKIntegration;
+import org.apache.slider.core.zk.ZKPathBuilder;
+import org.apache.slider.providers.AbstractClientProvider;
+import org.apache.slider.providers.SliderProviderFactory;
+import org.apache.slider.providers.agent.AgentKeys;
+import org.apache.slider.providers.slideram.SliderAMClientProvider;
+import org.apache.slider.server.appmaster.SliderAppMaster;
+import org.apache.slider.server.appmaster.rpc.RpcBinder;
+import org.apache.slider.server.services.security.SecurityStore;
+import org.apache.slider.server.services.utility.AbstractSliderLaunchedService;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.InterruptedIOException;
+import java.io.PrintStream;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.URISyntaxException;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.apache.hadoop.registry.client.binding.RegistryUtils.*;
+import static org.apache.slider.api.InternalKeys.*;
+import static org.apache.slider.api.OptionKeys.*;
+import static org.apache.slider.api.ResourceKeys.*;
+import static org.apache.slider.common.Constants.HADOOP_JAAS_DEBUG;
+import static org.apache.slider.common.params.SliderActions.*;
+import static org.apache.slider.common.tools.SliderUtils.*;
+
+
+/**
+ * Client service for Slider
+ */
+
+public class SliderClient extends AbstractSliderLaunchedService implements RunService,
+    SliderExitCodes, SliderKeys, ErrorStrings, SliderClientAPI {
+  private static final Logger log = LoggerFactory.getLogger(SliderClient.class);
+  public static final String E_MUST_BE_A_VALID_JSON_FILE
+      = "Invalid configuration. Must be a valid json file.";
+  public static final String E_INVALID_INSTALL_LOCATION
+      = "A valid install location must be provided for the client.";
+  public static final String E_UNABLE_TO_READ_SUPPLIED_PACKAGE_FILE
+      = "Unable to read supplied package file";
+  public static final String E_INVALID_APPLICATION_PACKAGE_LOCATION
+      = "A valid application package location required.";
+  public static final String E_INVALID_INSTALL_PATH = "Install path is not a valid directory";
+  public static final String E_INSTALL_PATH_DOES_NOT_EXIST = "Install path does not exist";
+  public static final String E_INVALID_APPLICATION_TYPE_NAME
+      = "A valid application type name is required (e.g. HBASE).";
+  public static final String E_USE_REPLACEPKG_TO_OVERWRITE = "Use --replacepkg to overwrite.";
+  public static final String E_PACKAGE_DOES_NOT_EXIST = "Package does not exist";
+  public static final String E_NO_ZOOKEEPER_QUORUM = "No Zookeeper quorum defined";
+  public static final String E_NO_RESOURCE_MANAGER = "No valid Resource Manager address provided";
+  public static final String E_PACKAGE_EXISTS = "Package exists";
+  private static PrintStream clientOutputStream = System.out;
+
+  // value should not be changed without updating string find in slider.py
+  private static final String PASSWORD_PROMPT = "Enter password for";
+
+  private ClientArgs serviceArgs;
+  public ApplicationId applicationId;
+  
+  private String deployedClusterName;
+  /**
+   * Cluster operations against the deployed cluster -will be null
+   * if no bonding has yet taken place
+   */
+  private SliderClusterOperations sliderClusterOperations;
+
+  protected SliderFileSystem sliderFileSystem;
+
+  /**
+   * Yarn client service
+   */
+  private SliderYarnClientImpl yarnClient;
+  private YarnAppListClient yarnAppListClient;
+  private AggregateConf launchedInstanceDefinition;
+
+  /**
+   * The YARN registry service
+   */
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+  private RegistryOperations registryOperations;
+
+  /**
+   * Constructor
+   */
+  public SliderClient() {
+    super("Slider Client");
+    new HdfsConfiguration();
+    new YarnConfiguration();
+  }
+
+  /**
+   * This is called <i>Before serviceInit is called</i>
+   * @param config the initial configuration build up by the
+   * service launcher.
+   * @param args argument list list of arguments passed to the command line
+   * after any launcher-specific commands have been stripped.
+   * @return the post-binding configuration to pass to the <code>init()</code>
+   * operation.
+   * @throws Exception
+   */
+  @Override
+  public Configuration bindArgs(Configuration config, String... args) throws Exception {
+    config = super.bindArgs(config, args);
+    serviceArgs = new ClientArgs(args);
+    serviceArgs.parse();
+    // add the slider XML config
+    ConfigHelper.injectSliderXMLResource();
+    // yarn-ify
+    YarnConfiguration yarnConfiguration = new YarnConfiguration(config);
+    return patchConfiguration(yarnConfiguration);
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    Configuration clientConf = loadSliderClientXML();
+    ConfigHelper.mergeConfigurations(conf, clientConf, SLIDER_CLIENT_XML, true);
+    serviceArgs.applyDefinitions(conf);
+    serviceArgs.applyFileSystemBinding(conf);
+    AbstractActionArgs coreAction = serviceArgs.getCoreAction();
+    // init security with our conf
+    if (!coreAction.disableSecureLogin() && isHadoopClusterSecure(conf)) {
+      forceLogin();
+      initProcessSecurity(conf);
+    }
+    if (coreAction.getHadoopServicesRequired()) {
+      initHadoopBinding();
+    }
+    super.serviceInit(conf);
+  }
+
+  /**
+   * Launched service execution. This runs {@link #exec()}
+   * then catches some exceptions and converts them to exit codes
+   * @return an exit code
+   * @throws Throwable
+   */
+  @Override
+  public int runService() throws Throwable {
+    try {
+      return exec();
+    } catch (FileNotFoundException | PathNotFoundException nfe) {
+      throw new NotFoundException(nfe, nfe.toString());
+    }
+  }
+
+  /**
+   * Execute the command line
+   * @return an exit code
+   * @throws Throwable on a failure
+   */
+  public int exec() throws Throwable {
+
+    // choose the action
+    String action = serviceArgs.getAction();
+    if (isUnset(action)) {
+      throw new SliderException(EXIT_USAGE, serviceArgs.usage());
+    }
+
+    int exitCode = EXIT_SUCCESS;
+    String clusterName = serviceArgs.getClusterName();
+    // actions
+
+    switch (action) {
+      case ACTION_AM_SUICIDE:
+        exitCode = actionAmSuicide(clusterName,
+            serviceArgs.getActionAMSuicideArgs());
+        break;
+      
+      case ACTION_BUILD:
+        exitCode = actionBuild(clusterName, serviceArgs.getActionBuildArgs());
+        break;
+      
+      case ACTION_CLIENT:
+        exitCode = actionClient(serviceArgs.getActionClientArgs());
+        break;
+
+      case ACTION_CREATE:
+        exitCode = actionCreate(clusterName, serviceArgs.getActionCreateArgs());
+        break;
+
+      case ACTION_DEPENDENCY:
+        exitCode = actionDependency(serviceArgs.getActionDependencyArgs());
+        break;
+
+      case ACTION_DESTROY:
+        exitCode = actionDestroy(clusterName, serviceArgs.getActionDestroyArgs());
+        break;
+
+      case ACTION_DIAGNOSTICS:
+        exitCode = actionDiagnostic(serviceArgs.getActionDiagnosticArgs());
+        break;
+      
+      case ACTION_EXISTS:
+        exitCode = actionExists(clusterName,
+            serviceArgs.getActionExistsArgs());
+        break;
+      
+      case ACTION_FLEX:
+        exitCode = actionFlex(clusterName, serviceArgs.getActionFlexArgs());
+        break;
+      
+      case ACTION_FREEZE:
+        exitCode = actionFreeze(clusterName, serviceArgs.getActionFreezeArgs());
+        break;
+      
+      case ACTION_HELP:
+        log.info(serviceArgs.usage());
+        break;
+
+      case ACTION_KDIAG:
+        exitCode = actionKDiag(serviceArgs.getActionKDiagArgs());
+        break;
+
+      case ACTION_KILL_CONTAINER:
+        exitCode = actionKillContainer(clusterName,
+            serviceArgs.getActionKillContainerArgs());
+        break;
+
+      case ACTION_INSTALL_KEYTAB:
+        exitCode = actionInstallKeytab(serviceArgs.getActionInstallKeytabArgs());
+        break;
+      
+      case ACTION_INSTALL_PACKAGE:
+        exitCode = actionInstallPkg(serviceArgs.getActionInstallPackageArgs());
+        break;
+
+      case ACTION_KEYTAB:
+        exitCode = actionKeytab(serviceArgs.getActionKeytabArgs());
+        break;
+
+      case ACTION_LIST:
+        exitCode = actionList(clusterName, serviceArgs.getActionListArgs());
+        break;
+
+      case ACTION_LOOKUP:
+        exitCode = actionLookup(serviceArgs.getActionLookupArgs());
+        break;
+
+      case ACTION_NODES:
+        exitCode = actionNodes("", serviceArgs.getActionNodesArgs());
+        break;
+
+      case ACTION_PACKAGE:
+        exitCode = actionPackage(serviceArgs.getActionPackageArgs());
+        break;
+
+      case ACTION_REGISTRY:
+        exitCode = actionRegistry(serviceArgs.getActionRegistryArgs());
+        break;
+      
+      case ACTION_RESOLVE:
+        exitCode = actionResolve(serviceArgs.getActionResolveArgs());
+        break;
+
+      case ACTION_RESOURCE:
+        exitCode = actionResource(serviceArgs.getActionResourceArgs());
+        break;
+
+      case ACTION_STATUS:
+        exitCode = actionStatus(clusterName, serviceArgs.getActionStatusArgs());
+        break;
+
+      case ACTION_THAW:
+        exitCode = actionThaw(clusterName, serviceArgs.getActionThawArgs());
+        break;
+
+      case ACTION_TOKENS:
+        exitCode = actionTokens(serviceArgs.getActionTokenArgs());
+        break;
+
+      case ACTION_UPDATE:
+        exitCode = actionUpdate(clusterName, serviceArgs.getActionUpdateArgs());
+        break;
+
+      case ACTION_UPGRADE:
+        exitCode = actionUpgrade(clusterName, serviceArgs.getActionUpgradeArgs());
+        break;
+
+      case ACTION_VERSION:
+        exitCode = actionVersion();
+        break;
+      
+      default:
+        throw new SliderException(EXIT_UNIMPLEMENTED,
+            "Unimplemented: " + action);
+    }
+   
+    return exitCode;
+  }
+
+  /**
+   * Perform everything needed to init the hadoop binding.
+   * This assumes that the service is already  in inited or started state
+   * @throws IOException
+   * @throws SliderException
+   */
+  protected void initHadoopBinding() throws IOException, SliderException {
+    // validate the client
+    validateSliderClientEnvironment(null);
+    //create the YARN client
+    yarnClient = new SliderYarnClientImpl();
+    yarnClient.init(getConfig());
+    if (getServiceState() == STATE.STARTED) {
+      yarnClient.start();
+    }
+    addService(yarnClient);
+    yarnAppListClient =
+        new YarnAppListClient(yarnClient, getUsername(), getConfig());
+    // create the filesystem
+    sliderFileSystem = new SliderFileSystem(getConfig());
+  }
+
+  /**
+   * Delete the zookeeper node associated with the calling user and the cluster
+   * TODO: YARN registry operations
+   **/
+  @VisibleForTesting
+  public boolean deleteZookeeperNode(String clusterName) throws YarnException, IOException {
+    String user = getUsername();
+    String zkPath = ZKIntegration.mkClusterPath(user, clusterName);
+    Exception e = null;
+    try {
+      Configuration config = getConfig();
+      ZKIntegration client = getZkClient(clusterName, user);
+      if (client != null) {
+        if (client.exists(zkPath)) {
+          log.info("Deleting zookeeper path {}", zkPath);
+        }
+        client.deleteRecursive(zkPath);
+        return true;
+      }
+    } catch (InterruptedException | BadConfigException | KeeperException ex) {
+      e = ex;
+    }
+    if (e != null) {
+      log.warn("Unable to recursively delete zk node {}", zkPath, e);
+    }
+
+    return false;
+  }
+
+  /**
+   * Create the zookeeper node associated with the calling user and the cluster
+   *
+   * @param clusterName slider application name
+   * @param nameOnly should the name only be created (i.e. don't create ZK node)
+   * @return the path, using the policy implemented in
+   *   {@link ZKIntegration#mkClusterPath(String, String)}
+   * @throws YarnException
+   * @throws IOException
+   */
+  @VisibleForTesting
+  public String createZookeeperNode(String clusterName, Boolean nameOnly) throws YarnException, IOException {
+    try {
+      return createZookeeperNodeInner(clusterName, nameOnly);
+    } catch (KeeperException.NodeExistsException e) {
+      return null;
+    } catch (KeeperException e) {
+      return null;
+    } catch (InterruptedException e) {
+      throw new InterruptedIOException(e.toString());
+    }
+  }
+
+  /**
+   * Create the zookeeper node associated with the calling user and the cluster
+   * -throwing exceptions on any failure
+   * @param clusterName cluster name
+   * @param nameOnly create the path, not the node
+   * @return the path, using the policy implemented in
+   *   {@link ZKIntegration#mkClusterPath(String, String)}
+   * @throws YarnException
+   * @throws IOException
+   * @throws KeeperException
+   * @throws InterruptedException
+   */
+  @VisibleForTesting
+  public String createZookeeperNodeInner(String clusterName, Boolean nameOnly)
+      throws YarnException, IOException, KeeperException, InterruptedException {
+    String user = getUsername();
+    String zkPath = ZKIntegration.mkClusterPath(user, clusterName);
+    if (nameOnly) {
+      return zkPath;
+    }
+    ZKIntegration client = getZkClient(clusterName, user);
+    if (client != null) {
+      // set up the permissions. This must be done differently on a secure cluster from an insecure
+      // one
+      List<ACL> zkperms = new ArrayList<>();
+      if (UserGroupInformation.isSecurityEnabled()) {
+        zkperms.add(new ACL(ZooDefs.Perms.ALL, ZooDefs.Ids.AUTH_IDS));
+        zkperms.add(new ACL(ZooDefs.Perms.READ, ZooDefs.Ids.ANYONE_ID_UNSAFE));
+      } else {
+        zkperms.add(new ACL(ZooDefs.Perms.ALL, ZooDefs.Ids.ANYONE_ID_UNSAFE));
+      }
+      client.createPath(zkPath, "",
+          zkperms,
+          CreateMode.PERSISTENT);
+      return zkPath;
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Gets a zookeeper client, returns null if it cannot connect to zookeeper
+   **/
+  protected ZKIntegration getZkClient(String clusterName, String user) throws YarnException {
+    String registryQuorum = lookupZKQuorum();
+    ZKIntegration client = null;
+    try {
+      BlockingZKWatcher watcher = new BlockingZKWatcher();
+      client = ZKIntegration.newInstance(registryQuorum, user, clusterName, true, false, watcher,
+          ZKIntegration.SESSION_TIMEOUT);
+      client.init();
+      watcher.waitForZKConnection(2 * 1000);
+    } catch (InterruptedException e) {
+      client = null;
+      log.warn("Unable to connect to zookeeper quorum {}", registryQuorum, e);
+    } catch (IOException e) {
+      log.warn("Unable to connect to zookeeper quorum {}", registryQuorum, e);
+    }
+    return client;
+  }
+
+  /**
+   * Keep this signature for backward compatibility with
+   * force=true by default.
+   */
+  @Override
+  public int actionDestroy(String clustername) throws YarnException,
+                                                      IOException {
+    ActionDestroyArgs destroyArgs = new ActionDestroyArgs();
+    destroyArgs.force = true;
+    return actionDestroy(clustername, destroyArgs);
+  }
+
+  @Override
+  public int actionDestroy(String clustername,
+      ActionDestroyArgs destroyArgs) throws YarnException, IOException {
+    // verify that a live cluster isn't there
+    validateClusterName(clustername);
+    //no=op, it is now mandatory. 
+    verifyBindingsDefined();
+    verifyNoLiveClusters(clustername, "Destroy");
+    boolean forceDestroy = destroyArgs.force;
+    log.debug("actionDestroy({}, force={})", clustername, forceDestroy);
+
+    // create the directory path
+    Path clusterDirectory = sliderFileSystem.buildClusterDirPath(clustername);
+    // delete the directory;
+    FileSystem fs = sliderFileSystem.getFileSystem();
+    boolean exists = fs.exists(clusterDirectory);
+    if (exists) {
+      log.debug("Application Instance {} found at {}: destroying", clustername, clusterDirectory);
+      if (!forceDestroy) {
+        // fail the command if --force is not explicitly specified
+        throw new UsageException("Destroy will permanently delete directories and registries. "
+            + "Reissue this command with the --force option if you want to proceed.");
+      }
+      if (!fs.delete(clusterDirectory, true)) {
+        log.warn("Filesystem returned false from delete() operation");
+      }
+
+      if(!deleteZookeeperNode(clustername)) {
+        log.warn("Unable to perform node cleanup in Zookeeper.");
+      }
+
+      if (fs.exists(clusterDirectory)) {
+        log.warn("Failed to delete {}", clusterDirectory);
+      }
+
+    } else {
+      log.debug("Application Instance {} already destroyed", clustername);
+    }
+
+    // rm the registry entry \u2014do not let this block the destroy operations
+    String registryPath = SliderRegistryUtils.registryPathForInstance(
+        clustername);
+    try {
+      getRegistryOperations().delete(registryPath, true);
+    } catch (IOException e) {
+      log.warn("Error deleting registry entry {}: {} ", registryPath, e, e);
+    } catch (SliderException e) {
+      log.warn("Error binding to registry {} ", e, e);
+    }
+
+    List<ApplicationReport> instances = findAllLiveInstances(clustername);
+    // detect any race leading to cluster creation during the check/destroy process
+    // and report a problem.
+    if (!instances.isEmpty()) {
+      throw new SliderException(EXIT_APPLICATION_IN_USE,
+                              clustername + ": "
+                              + E_DESTROY_CREATE_RACE_CONDITION
+                              + " :" +
+                              instances.get(0));
+    }
+    log.info("Destroyed cluster {}", clustername);
+    return EXIT_SUCCESS;
+  }
+  
+  @Override
+  public int actionAmSuicide(String clustername,
+      ActionAMSuicideArgs args) throws YarnException, IOException {
+    SliderClusterOperations clusterOperations =
+      createClusterOperations(clustername);
+    clusterOperations.amSuicide(args.message, args.exitcode, args.waittime);
+    return EXIT_SUCCESS;
+  }
+
+  @Override
+  public AbstractClientProvider createClientProvider(String provider)
+    throws SliderException {
+    SliderProviderFactory factory =
+      SliderProviderFactory.createSliderProviderFactory(provider);
+    return factory.createClientProvider();
+  }
+
+  /**
+   * Create the cluster -saving the arguments to a specification file first
+   * @param clustername cluster name
+   * @return the status code
+   * @throws YarnException Yarn problems
+   * @throws IOException other problems
+   * @throws BadCommandArgumentsException bad arguments.
+   */
+  public int actionCreate(String clustername, ActionCreateArgs createArgs) throws
+                                               YarnException,
+                                               IOException {
+
+    actionBuild(clustername, createArgs);
+    Path clusterDirectory = sliderFileSystem.buildClusterDirPath(clustername);
+    AggregateConf instanceDefinition = loadInstanceDefinitionUnresolved(
+        clustername, clusterDirectory);
+    try {
+      checkForCredentials(getConfig(), instanceDefinition.getAppConf());
+    } catch (IOException e) {
+      sliderFileSystem.getFileSystem().delete(clusterDirectory, true);
+      throw e;
+    }
+    return startCluster(clustername, createArgs);
+  }
+
+  @Override
+  public int actionUpgrade(String clustername, ActionUpgradeArgs upgradeArgs)
+      throws YarnException, IOException {
+    File template = upgradeArgs.template;
+    File resources = upgradeArgs.resources;
+    List<String> containers = upgradeArgs.containers;
+    List<String> components = upgradeArgs.components;
+
+    // For upgrade spec, let's be little more strict with validation. If either
+    // --template or --resources is specified, then both needs to be specified.
+    // Otherwise the internal app config and resources states of the app will be
+    // unwantedly modified and the change will take effect to the running app
+    // immediately.
+    require(!(template != null && resources == null),
+          "Option %s must be specified with option %s",
+          Arguments.ARG_RESOURCES, Arguments.ARG_TEMPLATE);
+
+    require(!(resources != null && template == null),
+          "Option %s must be specified with option %s",
+          Arguments.ARG_TEMPLATE, Arguments.ARG_RESOURCES);
+
+    // For upgrade spec, both --template and --resources should be specified
+    // and neither of --containers or --components should be used
+    if (template != null && resources != null) {
+      require(CollectionUtils.isEmpty(containers),
+            "Option %s cannot be specified with %s or %s",
+            Arguments.ARG_CONTAINERS, Arguments.ARG_TEMPLATE,
+            Arguments.ARG_RESOURCES);
+      require(CollectionUtils.isEmpty(components),
+              "Option %s cannot be specified with %s or %s",
+              Arguments.ARG_COMPONENTS, Arguments.ARG_TEMPLATE,
+              Arguments.ARG_RESOURCES);
+
+      // not an error to try to upgrade a stopped cluster, just return success
+      // code, appropriate log messages have already been dumped
+      if (!isAppInRunningState(clustername)) {
+        return EXIT_SUCCESS;
+      }
+
+      // Now initiate the upgrade spec flow
+      buildInstanceDefinition(clustername, upgradeArgs, true, true, true);
+      SliderClusterOperations clusterOperations = createClusterOperations(clustername);
+      clusterOperations.amSuicide("AM restarted for application upgrade", 1, 1000);
+      return EXIT_SUCCESS;
+    }
+
+    // Since neither --template or --resources were specified, it is upgrade
+    // containers flow. Here any one or both of --containers and --components
+    // can be specified. If a container is specified with --containers option
+    // and also belongs to a component type specified with --components, it will
+    // be upgraded only once.
+    return actionUpgradeContainers(clustername, upgradeArgs);
+  }
+
+  private int actionUpgradeContainers(String clustername,
+      ActionUpgradeArgs upgradeArgs) throws YarnException, IOException {
+    verifyBindingsDefined();
+    validateClusterName(clustername);
+    int waittime = upgradeArgs.getWaittime(); // ignored for now
+    String text = "Upgrade containers";
+    log.debug("actionUpgradeContainers({}, reason={}, wait={})", clustername,
+        text, waittime);
+
+    // not an error to try to upgrade a stopped cluster, just return success
+    // code, appropriate log messages have already been dumped
+    if (!isAppInRunningState(clustername)) {
+      return EXIT_SUCCESS;
+    }
+
+    // Create sets of containers and components to get rid of duplicates and
+    // for quick lookup during checks below
+    Set<String> containers = new HashSet<>();
+    if (upgradeArgs.containers != null) {
+      containers.addAll(new ArrayList<>(upgradeArgs.containers));
+    }
+    Set<String> components = new HashSet<>();
+    if (upgradeArgs.components != null) {
+      components.addAll(new ArrayList<>(upgradeArgs.components));
+    }
+
+    // check validity of component names and running containers here
+    List<ContainerInformation> liveContainers = getContainers(clustername);
+    Set<String> validContainers = new HashSet<>();
+    Set<String> validComponents = new HashSet<>();
+    for (ContainerInformation liveContainer : liveContainers) {
+      boolean allContainersAndComponentsAccountedFor = true;
+      if (CollectionUtils.isNotEmpty(containers)) {
+        if (containers.contains(liveContainer.containerId)) {
+          containers.remove(liveContainer.containerId);
+          validContainers.add(liveContainer.containerId);
+        }
+        allContainersAndComponentsAccountedFor = false;
+      }
+      if (CollectionUtils.isNotEmpty(components)) {
+        if (components.contains(liveContainer.component)) {
+          components.remove(liveContainer.component);
+          validComponents.add(liveContainer.component);
+        }
+        allContainersAndComponentsAccountedFor = false;
+      }
+      if (allContainersAndComponentsAccountedFor) {
+        break;
+      }
+    }
+
+    // If any item remains in containers or components then they are invalid.
+    // Log warning for them and proceed.
+    if (CollectionUtils.isNotEmpty(containers)) {
+      log.warn("Invalid set of containers provided {}", containers);
+    }
+    if (CollectionUtils.isNotEmpty(components)) {
+      log.warn("Invalid set of components provided {}", components);
+    }
+
+    // If not a single valid container or component is specified do not proceed
+    if (CollectionUtils.isEmpty(validContainers)
+        && CollectionUtils.isEmpty(validComponents)) {
+      log.error("Not a single valid container or component specified. Nothing to do.");
+      return EXIT_NOT_FOUND;
+    }
+
+    SliderClusterProtocol appMaster = connect(findInstance(clustername));
+    Messages.UpgradeContainersRequestProto r =
+      Messages.UpgradeContainersRequestProto
+              .newBuilder()
+              .setMessage(text)
+              .addAllContainer(validContainers)
+              .addAllComponent(validComponents)
+              .build();
+    appMaster.upgradeContainers(r);
+    log.info("Cluster upgrade issued for -");
+    if (CollectionUtils.isNotEmpty(validContainers)) {
+      log.info(" Containers (total {}): {}", validContainers.size(),
+          validContainers);
+    }
+    if (CollectionUtils.isNotEmpty(validComponents)) {
+      log.info(" Components (total {}): {}", validComponents.size(),
+          validComponents);
+    }
+
+    return EXIT_SUCCESS;
+  }
+
+  // returns true if and only if app is in RUNNING state
+  private boolean isAppInRunningState(String clustername) throws YarnException,
+      IOException {
+    // is this actually a known cluster?
+    sliderFileSystem.locateInstanceDefinition(clustername);
+    ApplicationReport app = findInstance(clustername);
+    if (app == null) {
+      // exit early
+      log.info("Cluster {} not running", clustername);
+      return false;
+    }
+    log.debug("App to upgrade was found: {}:\n{}", clustername,
+        new OnDemandReportStringifier(app));
+    if (app.getYarnApplicationState().ordinal() >= YarnApplicationState.FINISHED.ordinal()) {
+      log.info("Cluster {} is in a terminated state {}. Use command '{}' instead.",
+          clustername, app.getYarnApplicationState(), ACTION_UPDATE);
+      return false;
+    }
+
+    // IPC request to upgrade containers is possible if the app is running.
+    if (app.getYarnApplicationState().ordinal() < YarnApplicationState.RUNNING
+        .ordinal()) {
+      log.info("Cluster {} is in a pre-running state {}. To upgrade it needs "
+          + "to be RUNNING.", clustername, app.getYarnApplicationState());
+      return false;
+    }
+
+    return true;
+  }
+
+  protected static void checkForCredentials(Configuration conf,
+      ConfTree tree) throws IOException {
+    if (tree.credentials == null || tree.credentials.isEmpty()) {
+      log.info("No credentials requested");
+      return;
+    }
+
+    BufferedReader br = null;
+    try {
+      for (Entry<String, List<String>> cred : tree.credentials.entrySet()) {
+        String provider = cred.getKey();
+        List<String> aliases = cred.getValue();
+        if (aliases == null || aliases.isEmpty()) {
+          continue;
+        }
+        Configuration c = new Configuration(conf);
+        c.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, provider);
+        CredentialProvider credentialProvider = CredentialProviderFactory.getProviders(c).get(0);
+        Set<String> existingAliases = new HashSet<>(credentialProvider.getAliases());
+        for (String alias : aliases) {
+          if (existingAliases.contains(alias.toLowerCase(Locale.ENGLISH))) {
+            log.info("Credentials for " + alias + " found in " + provider);
+          } else {
+            if (br == null) {
+              br = new BufferedReader(new InputStreamReader(System.in));
+            }
+            char[] pass = readPassword(alias, br);
+            credentialProvider.createCredentialEntry(alias, pass);
+            credentialProvider.flush();
+            Arrays.fill(pass, ' ');
+          }
+        }
+      }
+    } finally {
+      org.apache.hadoop.io.IOUtils.closeStream(br);
+    }
+  }
+
+  private static char[] readOnePassword(String alias) throws IOException {
+    try(BufferedReader br = new BufferedReader(new InputStreamReader(System.in))) {
+      return readPassword(alias, br);
+    }
+  }
+
+  // using a normal reader instead of a secure one,
+  // because stdin is not hooked up to the command line
+  private static char[] readPassword(String alias, BufferedReader br)
+      throws IOException {
+    char[] cred = null;
+
+    boolean noMatch;
+    do {
+      log.info(String.format("%s %s: ", PASSWORD_PROMPT, alias));
+      char[] newPassword1 = br.readLine().toCharArray();
+      log.info(String.format("%s %s again: ", PASSWORD_PROMPT, alias));
+      char[] newPassword2 = br.readLine().toCharArray();
+      noMatch = !Arrays.equals(newPassword1, newPassword2);
+      if (noMatch) {
+        if (newPassword1 != null) Arrays.fill(newPassword1, ' ');
+        log.info(String.format("Passwords don't match. Try again."));
+      } else {
+        cred = newPassword1;
+      }
+      if (newPassword2 != null) Arrays.fill(newPassword2, ' ');
+    } while (noMatch);
+    if (cred == null)
+      throw new IOException("Could not read credentials for " + alias +
+          " from stdin");
+    return cred;
+  }
+
+  @Override
+  public int actionBuild(String clustername,
+      AbstractClusterBuildingActionArgs buildInfo) throws
+                                               YarnException,
+                                               IOException {
+
+    buildInstanceDefinition(clustername, buildInfo, false, false);
+    return EXIT_SUCCESS; 
+  }
+
+  @Override
+  public int actionKeytab(ActionKeytabArgs keytabInfo)
+      throws YarnException, IOException {
+    if (keytabInfo.install) {
+      return actionInstallKeytab(keytabInfo);
+    } else if (keytabInfo.delete) {
+      return actionDeleteKeytab(keytabInfo);
+    } else if (keytabInfo.list) {
+      return actionListKeytab(keytabInfo);
+    } else {
+      throw new BadCommandArgumentsException(
+          "Keytab option specified not found.\n"
+          + CommonArgs.usage(serviceArgs, ACTION_KEYTAB));
+    }
+  }
+
+  private int actionListKeytab(ActionKeytabArgs keytabInfo) throws IOException {
+    String folder = keytabInfo.folder != null ? keytabInfo.folder : StringUtils.EMPTY;
+    Path keytabPath = sliderFileSystem.buildKeytabInstallationDirPath(folder);
+    RemoteIterator<LocatedFileStatus> files =
+        sliderFileSystem.getFileSystem().listFiles(keytabPath, true);
+    log.info("Keytabs:");
+    while (files.hasNext()) {
+      log.info("\t" + files.next().getPath().toString());
+    }
+
+    return EXIT_SUCCESS;
+  }
+
+  private int actionDeleteKeytab(ActionKeytabArgs keytabInfo)
+      throws BadCommandArgumentsException, IOException {
+    if (StringUtils.isEmpty(keytabInfo.folder)) {
+      throw new BadCommandArgumentsException(
+          "A valid destination keytab sub-folder name is required (e.g. 'security').\n"
+          + CommonArgs.usage(serviceArgs, ACTION_KEYTAB));
+    }
+
+    if (StringUtils.isEmpty(keytabInfo.keytab)) {
+      throw new BadCommandArgumentsException("A keytab name is required.");
+    }
+
+    Path pkgPath = sliderFileSystem.buildKeytabInstallationDirPath(keytabInfo.folder);
+
+    Path fileInFs = new Path(pkgPath, keytabInfo.keytab );
+    log.info("Deleting keytab {}", fileInFs);
+    FileSystem sfs = sliderFileSystem.getFileSystem();
+    require(sfs.exists(fileInFs), "No keytab to delete found at %s",
+        fileInFs.toUri());
+    sfs.delete(fileInFs, false);
+
+    return EXIT_SUCCESS;
+  }
+
+  private int actionInstallKeytab(ActionKeytabArgs keytabInfo)
+      throws BadCommandArgumentsException, IOException {
+    Path srcFile = null;
+    require(isSet(keytabInfo.folder),
+        "A valid destination keytab sub-folder name is required (e.g. 'security').\n"
+        + CommonArgs.usage(serviceArgs, ACTION_KEYTAB));
+
+    requireArgumentSet(Arguments.ARG_KEYTAB, keytabInfo.keytab);
+    File keytabFile = new File(keytabInfo.keytab);
+    require(keytabFile.isFile(),
+        "Unable to access supplied keytab file at %s", keytabFile.getAbsolutePath());
+    srcFile = new Path(keytabFile.toURI());
+
+    Path pkgPath = sliderFileSystem.buildKeytabInstallationDirPath(keytabInfo.folder);
+    FileSystem sfs = sliderFileSystem.getFileSystem();
+    sfs.mkdirs(pkgPath);
+    sfs.setPermission(pkgPath, new FsPermission(
+        FsAction.ALL, FsAction.NONE, FsAction.NONE));
+
+    Path fileInFs = new Path(pkgPath, srcFile.getName());
+    log.info("Installing keytab {} at {} and overwrite is {}.",
+        srcFile, fileInFs, keytabInfo.overwrite);
+    require(!(sfs.exists(fileInFs) && !keytabInfo.overwrite),
+        "Keytab exists at %s. Use --overwrite to overwrite.", fileInFs.toUri());
+
+    sfs.copyFromLocalFile(false, keytabInfo.overwrite, srcFile, fileInFs);
+    sfs.setPermission(fileInFs,
+        new FsPermission(FsAction.READ_WRITE, FsAction.NONE, FsAction.NONE));
+
+    return EXIT_SUCCESS;
+  }
+
+  @Override
+  public int actionInstallKeytab(ActionInstallKeytabArgs installKeytabInfo)
+      throws YarnException, IOException {
+    log.warn("The 'install-keytab' option has been deprecated.  Please use 'keytab --install'.");
+    return actionKeytab(new ActionKeytabArgs(installKeytabInfo));
+  }
+
+  @Override
+  public int actionInstallPkg(ActionInstallPackageArgs installPkgInfo) throws
+      YarnException,
+      IOException {
+    log.warn("The " + ACTION_INSTALL_PACKAGE
+        + " option has been deprecated. Please use '"
+        + ACTION_PACKAGE + " " + ClientArgs.ARG_INSTALL + "'.");
+    if (StringUtils.isEmpty(installPkgInfo.name)) {
+      throw new BadCommandArgumentsException(
+          E_INVALID_APPLICATION_TYPE_NAME + "\n"
+              + CommonArgs.usage(serviceArgs, ACTION_INSTALL_PACKAGE));
+    }
+    Path srcFile = extractPackagePath(installPkgInfo.packageURI);
+
+    // Do not provide new options to install-package command as it is in
+    // deprecated mode. So version is kept null here. Use package --install.
+    Path pkgPath = sliderFileSystem.buildPackageDirPath(installPkgInfo.name,
+        null);
+    FileSystem sfs = sliderFileSystem.getFileSystem();
+    sfs.mkdirs(pkgPath);
+
+    Path fileInFs = new Path(pkgPath, srcFile.getName());
+    log.info("Installing package {} at {} and overwrite is {}.",
+        srcFile, fileInFs, installPkgInfo.replacePkg);
+    require(!(sfs.exists(fileInFs) && !installPkgInfo.replacePkg),
+          "Package exists at %s. : %s", fileInFs.toUri(), E_USE_REPLACEPKG_TO_OVERWRITE);
+    sfs.copyFromLocalFile(false, installPkgInfo.replacePkg, srcFile, fileInFs);
+    return EXIT_SUCCESS;
+  }
+
+  @Override
+  public int actionResource(ActionResourceArgs resourceInfo)
+      throws YarnException, IOException {
+    if (resourceInfo.help) {
+      actionHelp(ACTION_RESOURCE);
+      return EXIT_SUCCESS;
+    } else if (resourceInfo.install) {
+      return actionInstallResource(resourceInfo);
+    } else if (resourceInfo.delete) {
+      return actionDeleteResource(resourceInfo);
+    } else if (resourceInfo.list) {
+      return actionListResource(resourceInfo);
+    } else {
+      throw new BadCommandArgumentsException(
+          "Resource option specified not found.\n"
+              + CommonArgs.usage(serviceArgs, ACTION_RESOURCE));
+    }
+  }
+
+  private int actionListResource(ActionResourceArgs resourceInfo) throws IOException {
+    String folder = resourceInfo.folder != null ? resourceInfo.folder : StringUtils.EMPTY;
+    Path path = sliderFileSystem.buildResourcePath(folder);
+    RemoteIterator<LocatedFileStatus> files =
+        sliderFileSystem.getFileSystem().listFiles(path, true);
+    log.info("Resources:");
+    while (files.hasNext()) {
+      log.info("\t" + files.next().getPath().toString());
+    }
+
+    return EXIT_SUCCESS;
+  }
+
+  private int actionDeleteResource(ActionResourceArgs resourceInfo)
+      throws BadCommandArgumentsException, IOException {
+    if (StringUtils.isEmpty(resourceInfo.resource)) {
+      throw new BadCommandArgumentsException("A file name is required.");
+    }
+
+    Path fileInFs;
+    if (resourceInfo.folder == null) {
+      fileInFs = sliderFileSystem.buildResourcePath(resourceInfo.resource);
+    } else {
+      fileInFs = sliderFileSystem.buildResourcePath(resourceInfo.folder,
+          resourceInfo.resource);
+    }
+
+    log.info("Deleting resource {}", fileInFs);
+    FileSystem sfs = sliderFileSystem.getFileSystem();
+    require(sfs.exists(fileInFs), "No resource to delete found at %s", fileInFs.toUri());
+    sfs.delete(fileInFs, true);
+
+    return EXIT_SUCCESS;
+  }
+
+  private int actionInstallResource(ActionResourceArgs resourceInfo)
+      throws BadCommandArgumentsException, IOException {
+    Path srcFile = null;
+    String folder = resourceInfo.folder != null ? resourceInfo.folder : StringUtils.EMPTY;
+
+    requireArgumentSet(Arguments.ARG_RESOURCE, resourceInfo.resource);
+    File file = new File(resourceInfo.resource);
+    require(file.isFile() || file.isDirectory(),
+        "Unable to access supplied file at %s", file.getAbsolutePath());
+
+    File[] files;
+    if (file.isDirectory()) {
+      files = file.listFiles();
+    } else {
+      files = new File[] { file };
+    }
+
+    Path pkgPath = sliderFileSystem.buildResourcePath(folder);
+    FileSystem sfs = sliderFileSystem.getFileSystem();
+
+    if (!sfs.exists(pkgPath)) {
+      sfs.mkdirs(pkgPath);
+      sfs.setPermission(pkgPath, new FsPermission(
+          FsAction.ALL, FsAction.NONE, FsAction.NONE));
+    } else {
+      require(sfs.isDirectory(pkgPath), "Specified folder %s exists and is " +
+          "not a directory", folder);
+    }
+
+    for (File f : files) {
+      srcFile = new Path(f.toURI());
+
+      Path fileInFs = new Path(pkgPath, srcFile.getName());
+      log.info("Installing file {} at {} and overwrite is {}.",
+          srcFile, fileInFs, resourceInfo.overwrite);
+      require(!(sfs.exists(fileInFs) && !resourceInfo.overwrite),
+          "File exists at %s. Use --overwrite to overwrite.", fileInFs.toUri());
+
+      sfs.copyFromLocalFile(false, resourceInfo.overwrite, srcFile, fileInFs);
+      sfs.setPermission(fileInFs,
+          new FsPermission(FsAction.READ_WRITE, FsAction.NONE, FsAction.NONE));
+    }
+
+    return EXIT_SUCCESS;
+  }
+
+  @Override
+  public int actionClient(ActionClientArgs clientInfo) throws
+      YarnException,
+      IOException {
+    if (clientInfo.install) {
+      return doClientInstall(clientInfo);
+    } else if (clientInfo.getCertStore) {
+      return doCertificateStoreRetrieval(clientInfo);
+    } else {
+      throw new BadCommandArgumentsException(
+          "Only install, keystore, and truststore commands are supported for the client.\n"
+          + CommonArgs.usage(serviceArgs, ACTION_CLIENT));
+
+    }
+  }
+
+  private int doCertificateStoreRetrieval(ActionClientArgs clientInfo)
+      throws YarnException, IOException {
+    if (clientInfo.keystore != null && clientInfo.truststore != null) {
+      throw new BadCommandArgumentsException(
+          "Only one of either keystore or truststore can be retrieved at one time.  "
+          + "Retrieval of both should be done separately\n"
+          + CommonArgs.usage(serviceArgs, ACTION_CLIENT));
+    }
+
+    requireArgumentSet(Arguments.ARG_NAME, clientInfo.name);
+
+    File storeFile = null;
+    SecurityStore.StoreType type;
+    if (clientInfo.keystore != null) {
+      storeFile = clientInfo.keystore;
+      type = SecurityStore.StoreType.keystore;
+    } else {
+      storeFile = clientInfo.truststore;
+      type = SecurityStore.StoreType.truststore;
+    }
+
+    require (!storeFile.exists(),
+        "File %s already exists.  Please remove that file or select a different file name.",
+         storeFile.getAbsolutePath());
+    String hostname = null;
+    if (type == SecurityStore.StoreType.keystore) {
+      hostname = clientInfo.hostname;
+      if (hostname == null) {
+        hostname = InetAddress.getLocalHost().getCanonicalHostName();
+        log.info("No hostname specified via command line. Using {}", hostname);
+      }
+    }
+
+    String password = clientInfo.password;
+    if (password == null) {
+      String provider = clientInfo.provider;
+      String alias = clientInfo.alias;
+      if (provider != null && alias != null) {
+        Configuration conf = new Configuration(getConfig());
+        conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, provider);
+        char[] chars = conf.getPassword(alias);
+        if (chars == null) {
+          CredentialProvider credentialProvider =
+              CredentialProviderFactory.getProviders(conf).get(0);
+          chars = readOnePassword(alias);
+          credentialProvider.createCredentialEntry(alias, chars);
+          credentialProvider.flush();
+        }
+        password = String.valueOf(chars);
+        Arrays.fill(chars, ' ');
+      } else {
+        log.info("No password and no provider/alias pair were provided, " +
+            "prompting for password");
+        // get a password
+        password = String.valueOf(readOnePassword(type.name()));
+      }
+    }
+
+    byte[] keystore = createClusterOperations(clientInfo.name)
+        .getClientCertificateStore(hostname, "client", password, type.name());
+    // persist to file
+    FileOutputStream storeFileOutputStream = null;
+    try {
+      storeFileOutputStream = new FileOutputStream(storeFile);
+      IOUtils.write(keystore, storeFileOutputStream);
+    } catch (Exception e) {
+      log.error("Unable to persist to file {}", storeFile);
+      throw e;
+    } finally {
+      if (storeFileOutputStream != null) {
+        storeFileOutputStream.close();
+      }
+    }
+
+    return EXIT_SUCCESS;
+  }
+
+  private int doClientInstall(ActionClientArgs clientInfo)
+      throws IOException, SliderException {
+
+    require(clientInfo.installLocation != null,
+          E_INVALID_INSTALL_LOCATION +"\n"
+          + CommonArgs.usage(serviceArgs, ACTION_CLIENT));
+    require(clientInfo.installLocation.exists(),
+        E_INSTALL_PATH_DOES_NOT_EXIST + ": " + clientInfo.installLocation.getAbsolutePath());
+
+    require(clientInfo.installLocation.isDirectory(),
+        E_INVALID_INSTALL_PATH + ": " + clientInfo.installLocation.getAbsolutePath());
+
+    File pkgFile;
+    File tmpDir = null;
+
+    require(isSet(clientInfo.packageURI) || isSet(clientInfo.name),
+        E_INVALID_APPLICATION_PACKAGE_LOCATION);
+    if (isSet(clientInfo.packageURI)) {
+      pkgFile = new File(clientInfo.packageURI);
+    } else {
+      Path appDirPath = sliderFileSystem.buildAppDefDirPath(clientInfo.name);
+      Path appDefPath = new Path(appDirPath, SliderKeys.DEFAULT_APP_PKG);
+      require(sliderFileSystem.isFile(appDefPath),
+          E_INVALID_APPLICATION_PACKAGE_LOCATION);
+      tmpDir = Files.createTempDir();
+      pkgFile = new File(tmpDir, SliderKeys.DEFAULT_APP_PKG);
+      sliderFileSystem.copyHdfsFileToLocal(appDefPath, pkgFile);
+    }
+    require(pkgFile.isFile(),
+        E_UNABLE_TO_READ_SUPPLIED_PACKAGE_FILE + " at %s", pkgFile.getAbsolutePath());
+
+    JSONObject config = null;
+    if(clientInfo.clientConfig != null) {
+      try {
+        byte[] encoded = Files.toByteArray(clientInfo.clientConfig);
+        config = new JSONObject(new String(encoded, Charset.defaultCharset()));
+      } catch (JSONException jsonEx) {
+        log.error("Unable to read supplied configuration at {}: {}",
+            clientInfo.clientConfig, jsonEx);
+        log.debug("Unable to read supplied configuration at {}: {}",
+            clientInfo.clientConfig, jsonEx, jsonEx);
+        throw new BadConfigException(E_MUST_BE_A_VALID_JSON_FILE, jsonEx);
+      }
+    }
+
+    // Only INSTALL is supported
+    AbstractClientProvider
+        provider = createClientProvider(SliderProviderFactory.DEFAULT_CLUSTER_TYPE);
+    provider.processClientOperation(sliderFileSystem,
+        getRegistryOperations(),
+        getConfig(),
+        "INSTALL",
+        clientInfo.installLocation,
+        pkgFile,
+        config,
+        clientInfo.name);
+    return EXIT_SUCCESS;
+  }
+
+
+  @Override
+  public int actionPackage(ActionPackageArgs actionPackageInfo)
+      throws YarnException, IOException {
+    initializeOutputStream(actionPackageInfo.out);
+    int exitCode = -1;
+    if (actionPackageInfo.help) {
+      exitCode = actionHelp(ACTION_PACKAGE);
+    }
+    if (actionPackageInfo.install) {
+      exitCode = actionPackageInstall(actionPackageInfo);
+    }
+    if (actionPackageInfo.delete) {
+      exitCode = actionPackageDelete(actionPackageInfo);
+    }
+    if (actionPackageInfo.list) {
+      exitCode = actionPackageList();
+    }
+    if (actionPackageInfo.instances) {
+      exitCode = actionPackageInstances();
+    }
+    finalizeOutputStream(actionPackageInfo.out);
+    if (exitCode != -1) {
+      return exitCode;
+    }
+    throw new BadCommandArgumentsException(
+        "Select valid package operation option");
+  }
+
+  private void initializeOutputStream(String outFile)
+      throws FileNotFoundException {
+    if (outFile != null) {
+      clientOutputStream = new PrintStream(new FileOutputStream(outFile));
+    } else {
+      clientOutputStream = System.out;
+    }
+  }
+
+  private void finalizeOutputStream(String outFile) {
+    if (outFile != null && clientOutputStream != null) {
+      clientOutputStream.flush();
+      clientOutputStream.close();
+    }
+    clientOutputStream = System.out;
+  }
+
+  private int actionPackageInstances() throws YarnException, IOException {
+    Map<String, Path> persistentInstances = sliderFileSystem
+        .listPersistentInstances();
+    if (persistentInstances.isEmpty()) {
+      log.info("No slider cluster specification available");
+      return EXIT_SUCCESS;
+    }
+    String pkgPathValue = sliderFileSystem
+        .buildPackageDirPath(StringUtils.EMPTY, StringUtils.EMPTY).toUri()
+        .getPath();
+    FileSystem fs = sliderFileSystem.getFileSystem();
+    Iterator<Map.Entry<String, Path>> instanceItr = persistentInstances
+        .entrySet().iterator();
+    log.info("List of applications with its package name and path");
+    println("%-25s  %15s  %30s  %s", "Cluster Name", "Package Name",
+        "Package Version", "Application Location");
+    while(instanceItr.hasNext()) {
+      Map.Entry<String, Path> entry = instanceItr.next();
+      String clusterName = entry.getKey();
+      Path clusterPath = entry.getValue();
+      AggregateConf instanceDefinition = loadInstanceDefinitionUnresolved(
+          clusterName, clusterPath);
+      Path appDefPath = null;
+      try {
+        appDefPath = new Path(
+            getApplicationDefinitionPath(instanceDefinition
+                .getAppConfOperations()));
+      } catch (BadConfigException e) {
+        // Invalid cluster state, so move on to next. No need to log anything
+        // as this is just listing of instances.
+        continue;
+      }
+      if (!appDefPath.isUriPathAbsolute()) {
+        appDefPath = new Path(fs.getHomeDirectory(), appDefPath);
+      }
+      String appDefPathStr = appDefPath.toUri().toString();
+      try {
+        if (appDefPathStr.contains(pkgPathValue) && fs.isFile(appDefPath)) {
+          String packageName = appDefPath.getParent().getName();
+          String packageVersion = StringUtils.EMPTY;
+          if (instanceDefinition.isVersioned()) {
+            packageVersion = packageName;
+            packageName = appDefPath.getParent().getParent().getName();
+          }
+          println("%-25s  %15s  %30s  %s", clusterName, packageName,
+              packageVersion, appDefPathStr);
+        }
+      } catch (IOException e) {
+        log.debug("{} application definition path {} is not found.", clusterName, appDefPathStr);
+      }
+    }
+    return EXIT_SUCCESS;
+  }
+
+  private int actionPackageList() throws IOException {
+    Path pkgPath = sliderFileSystem.buildPackageDirPath(StringUtils.EMPTY,
+        StringUtils.EMPTY);
+    log.info("Package install path : {}", pkgPath);
+    FileSystem sfs = sliderFileSystem.getFileSystem();
+    if (!sfs.isDirectory(pkgPath)) {
+      log.info("No package(s) installed");
+      return EXIT_SUCCESS;
+    }
+    FileStatus[] fileStatus = sfs.listStatus(pkgPath);
+    boolean hasPackage = false;
+    StringBuilder sb = new StringBuilder();
+    sb.append("List of installed packages:\n");
+    for (FileStatus fstat : fileStatus) {
+      if (fstat.isDirectory()) {
+        sb.append("\t").append(fstat.getPath().getName());
+        sb.append("\n");
+        hasPackage = true;
+      }
+    }
+    if (hasPackage) {
+      println(sb.toString());
+    } else {
+      log.info("No package(s) installed");
+    }
+    return EXIT_SUCCESS;
+  }
+
+  private void createSummaryMetainfoFile(Path srcFile, Path destFile,
+      boolean overwrite) throws IOException {
+    FileSystem srcFs = srcFile.getFileSystem(getConfig());
+    try (InputStream inputStreamJson = SliderUtils
+        .getApplicationResourceInputStream(srcFs, srcFile, "metainfo.json");
+        InputStream inputStreamXml = SliderUtils
+            .getApplicationResourceInputStream(srcFs, srcFile, "metainfo.xml");) {
+      InputStream inputStream = null;
+      Path summaryFileInFs = null;
+      if (inputStreamJson != null) {
+        inputStream = inputStreamJson;
+        summaryFileInFs = new Path(destFile.getParent(), destFile.getName()
+            + ".metainfo.json");
+        log.info("Found JSON metainfo file in package");
+      } else if (inputStreamXml != null) {
+        inputStream = inputStreamXml;
+        summaryFileInFs = new Path(destFile.getParent(), destFile.getName()
+            + ".metainfo.xml");
+        log.info("Found XML metainfo file in package");
+      }
+      if (inputStream != null) {
+        try (FSDataOutputStream dataOutputStream = sliderFileSystem
+            .getFileSystem().create(summaryFileInFs, overwrite)) {
+          log.info("Creating summary metainfo file");
+          IOUtils.copy(inputStream, dataOutputStream);
+        }
+      }
+    }
+  }
+
+  private int actionPackageInstall(ActionPackageArgs actionPackageArgs)
+      throws YarnException, IOException {
+    requireArgumentSet(Arguments.ARG_NAME, actionPackageArgs.name);
+
+    Path srcFile = extractPackagePath(actionPackageArgs.packageURI);
+
+    Path pkgPath = sliderFileSystem.buildPackageDirPath(actionPackageArgs.name,
+        actionPackageArgs.version);
+    FileSystem fs = sliderFileSystem.getFileSystem();
+    if (!fs.exists(pkgPath)) {
+      fs.mkdirs(pkgPath);
+    }
+
+    Path fileInFs = new Path(pkgPath, srcFile.getName());
+    require(actionPackageArgs.replacePkg || !fs.exists(fileInFs),
+        E_PACKAGE_EXISTS +" at  %s. Use --replacepkg to overwrite.", fileInFs.toUri());
+
+    log.info("Installing package {} to {} (overwrite set to {})", srcFile,
+        fileInFs, actionPackageArgs.replacePkg);
+    fs.copyFromLocalFile(false, actionPackageArgs.replacePkg, srcFile, fileInFs);
+    createSummaryMetainfoFile(srcFile, fileInFs, actionPackageArgs.replacePkg);
+
+    String destPathWithHomeDir = Path
+        .getPathWithoutSchemeAndAuthority(fileInFs).toString();
+    String destHomeDir = Path.getPathWithoutSchemeAndAuthority(
+        fs.getHomeDirectory()).toString();
+    // a somewhat contrived approach to stripping out the home directory and any trailing
+    // separator; designed to work on windows and unix
+    String destPathWithoutHomeDir;
+    if (destPathWithHomeDir.startsWith(destHomeDir)) {
+      destPathWithoutHomeDir = destPathWithHomeDir.substring(destHomeDir.length());
+      if (destPathWithoutHomeDir.startsWith("/") || destPathWithoutHomeDir.startsWith("\\")) {
+        destPathWithoutHomeDir = destPathWithoutHomeDir.substring(1);
+      }
+    } else {
+      destPathWithoutHomeDir = destPathWithHomeDir;
+    }
+    log.info("Set " + AgentKeys.APP_DEF + " in your app config JSON to {}",
+        destPathWithoutHomeDir);
+
+    return EXIT_SUCCESS;
+  }
+
+  private Path extractPackagePath(String packageURI)
+      throws BadCommandArgumentsException {
+    require(isSet(packageURI), E_INVALID_APPLICATION_PACKAGE_LOCATION);
+    File pkgFile = new File(packageURI);
+    require(pkgFile.isFile(),
+        E_UNABLE_TO_READ_SUPPLIED_PACKAGE_FILE + ":  " + pkgFile.getAbsolutePath());
+    return new Path(pkgFile.toURI());
+  }
+
+  private int actionPackageDelete(ActionPackageArgs actionPackageArgs) throws
+      YarnException, IOException {
+    requireArgumentSet(Arguments.ARG_NAME, actionPackageArgs.name);
+
+    Path pkgPath = sliderFileSystem.buildPackageDirPath(actionPackageArgs.name,
+        actionPackageArgs.version);
+    FileSystem fs = sliderFileSystem.getFileSystem();
+    require(fs.exists(pkgPath), E_PACKAGE_DOES_NOT_EXIST +": %s ", pkgPath.toUri());
+    log.info("Deleting package {} at {}.", actionPackageArgs.name, pkgPath);
+
+    if(fs.delete(pkgPath, true)) {
+      log.info("Deleted package {} " + actionPackageArgs.name);
+      return EXIT_SUCCESS;
+    } else {
+      log.warn("Package deletion failed.");
+      return EXIT_NOT_FOUND;
+    }
+  }
+
+  @Override
+  public int actionUpdate(String clustername,
+      AbstractClusterBuildingActionArgs buildInfo) throws
+      YarnException, IOException {
+    buildInstanceDefinition(clustername, buildInfo, true, true);
+    return EXIT_SUCCESS; 
+  }
+
+  /**
+   * Build up the AggregateConfiguration for an application instance then
+   * persists it
+   * @param clustername name of the cluster
+   * @param buildInfo the arguments needed to build the cluster
+   * @param overwrite true if existing cluster directory can be overwritten
+   * @param liveClusterAllowed true if live cluster can be modified
+   * @throws YarnException
+   * @throws IOException
+   */
+
+  public void buildInstanceDefinition(String clustername,
+      AbstractClusterBuildingActionArgs buildInfo, boolean overwrite,
+      boolean liveClusterAllowed) throws YarnException, IOException {
+    buildInstanceDefinition(clustername, buildInfo, overwrite,
+        liveClusterAllowed, false);
+  }
+
+  public void buildInstanceDefinition(String clustername,
+      AbstractClusterBuildingActionArgs buildInfo, boolean overwrite,
+      boolean liveClusterAllowed, boolean isUpgradeFlow) throws YarnException,
+      IOException {
+    // verify that a live cluster isn't there
+    validateClusterName(clustername);
+    verifyBindingsDefined();
+    if (!liveClusterAllowed) {
+      verifyNoLiveClusters(clustername, "Create");
+    }
+
+    Configuration conf = getConfig();
+    String registryQuorum = lookupZKQuorum();
+
+    Path appconfdir = buildInfo.getConfdir();
+    // Provider
+    String providerName = buildInfo.getProvider();
+    requireArgumentSet(Arguments.ARG_PROVIDER, providerName);
+    log.debug("Provider is {}", providerName);
+    SliderAMClientProvider sliderAM = new SliderAMClientProvider(conf);
+    AbstractClientProvider provider =
+      createClientProvider(providerName);
+    InstanceBuilder builder =
+      new InstanceBuilder(sliderFileSystem, 
+                          getConfig(),
+                          clustername);
+    
+    AggregateConf instanceDefinition = new AggregateConf();
+    ConfTreeOperations appConf = instanceDefinition.getAppConfOperations();
+    ConfTreeOperations resources = instanceDefinition.getResourceOperations();
+    ConfTreeOperations internal = instanceDefinition.getInternalOperations();
+    //initial definition is set by the providers 
+    sliderAM.prepareInstanceConfiguration(instanceDefinition);
+    provider.prepareInstanceConfiguration(instanceDefinition);
+
+    //load in any specified on the command line
+    if (buildInfo.resources != null) {
+      try {
+        resources.mergeFile(buildInfo.resources,
+                            new ResourcesInputPropertiesValidator());
+
+      } catch (IOException e) {
+        throw new BadConfigException(e,
+               "incorrect argument to %s: \"%s\" : %s ", 
+                                     Arguments.ARG_RESOURCES,
+                                     buildInfo.resources,
+                                     e.toString());
+      }
+    }
+    if (buildInfo.template != null) {
+      try {
+        appConf.mergeFile(buildInfo.template,
+                          new TemplateInputPropertiesValidator());
+      } catch (IOException e) {
+        throw new BadConfigException(e,
+                                     "incorrect argument to %s: \"%s\" : %s ",
+                                     Arguments.ARG_TEMPLATE,
+                                     buildInfo.template,
+                                     e.toString());
+      }
+    }
+
+    if (isUpgradeFlow) {
+      ActionUpgradeArgs upgradeInfo = (ActionUpgradeArgs) buildInfo;
+      if (!upgradeInfo.force) {
+        validateClientAndClusterResource(clustername, resources);
+      }
+    }
+
+    //get the command line options
+    ConfTree cmdLineAppOptions = buildInfo.buildAppOptionsConfTree();
+    ConfTree cmdLineResourceOptions = buildInfo.buildResourceOptionsConfTree();
+
+    appConf.merge(cmdLineAppOptions);
+
+    AppDefinitionPersister appDefinitionPersister = new AppDefinitionPersister(sliderFileSystem);
+    appDefinitionPersister.processSuppliedDefinitions(clustername, buildInfo, appConf);
+
+    // put the role counts into the resources file
+    Map<String, String> argsRoleMap = buildInfo.getComponentMap();
+    for (Map.Entry<String, String> roleEntry : argsRoleMap.entrySet()) {
+      String count = roleEntry.getValue();
+      String key = roleEntry.getKey();
+      log.info("{} => {}", key, count);
+      resources.getOrAddComponent(key).put(COMPONENT_INSTANCES, count);
+    }
+
+    //all CLI role options
+    Map<String, Map<String, String>> appOptionMap =
+      buildInfo.getCompOptionMap();
+    appConf.mergeComponents(appOptionMap);
+
+    //internal picks up core. values only
+    internal.propagateGlobalKeys(appConf, "slider.");
+    internal.propagateGlobalKeys(appConf, "internal.");
+
+    //copy over role. and yarn. values ONLY to the resources
+    if (PROPAGATE_RESOURCE_OPTION) {
+      resources.propagateGlobalKeys(appConf, "component.");
+      resources.propagateGlobalKeys(appConf, "role.");
+      resources.propagateGlobalKeys(appConf, "yarn.");
+      resources.mergeComponentsPrefix(appOptionMap, "component.", true);
+      resources.mergeComponentsPrefix(appOptionMap, "yarn.", true);
+      resources.mergeComponentsPrefix(appOptionMap, "role.", true);
+    }
+
+    // resource component args
+    appConf.merge(cmdLineResourceOptions);
+    resources.merge(cmdLineResourceOptions);
+    resources.mergeComponents(buildInfo.getResourceCompOptionMap());
+
+    builder.init(providerName, instanceDefinition);
+    builder.propagateFilename();
+    builder.propagatePrincipals();
+    builder.setImageDetailsIfAvailable(buildInfo.getImage(),
+                                       buildInfo.getAppHomeDir());
+    builder.setQueue(buildInfo.queue);
+
+    String quorum = buildInfo.getZKhosts();
+    if (isUnset(quorum)) {
+      quorum = registryQuorum;
+    }
+    if (isUnset(quorum)) {
+      throw new BadConfigException(E_NO_ZOOKEEPER_QUORUM);
+    }
+    ZKPathBuilder zkPaths = new ZKPathBuilder(getAppName(),
+        getUsername(),
+        clustername,
+        registryQuorum,
+        quorum);
+    String zookeeperRoot = buildInfo.getAppZKPath();
+
+    if (isSet(zookeeperRoot)) {
+      zkPaths.setAppPath(zookeeperRoot);
+    } else {
+      String createDefaultZkNode = appConf.getGlobalOptions()
+          .getOption(AgentKeys.CREATE_DEF_ZK_NODE, "false");
+      if (createDefaultZkNode.equals("true")) {
+        String defaultZKPath = createZookeeperNode(clustername, false);
+        log.debug("ZK node created for application instance: {}", defaultZKPath);
+        if (defaultZKPath != null) {
+          zkPaths.setAppPath(defaultZKPath);
+        }
+      } else {
+        // create AppPath if default is being used
+        String defaultZKPath = createZookeeperNode(clustername, true);
+        log.debug("ZK node assigned to application instance: {}", defaultZKPath);
+        zkPaths.setAppPath(defaultZKPath);
+      }
+    }
+
+    builder.addZKBinding(zkPaths);
+
+    //then propagate any package URI
+    if (buildInfo.packageURI != null) {
+      appConf.set(AgentKeys.PACKAGE_PATH, buildInfo.packageURI);
+    }
+
+    propagatePythonExecutable(conf, instanceDefinition);
+
+    // make any substitutions needed at this stage
+    replaceTokens(appConf.getConfTree(), getUsername(), clustername);
+
+    // TODO: Refactor the validation code and persistence code
+    try {
+      persistInstanceDefinition(overwrite, appconfdir, builder);
+      appDefinitionPersister.persistPackages();
+
+    } catch (LockAcquireFailedException e) {
+      log.warn("Failed to get a Lock on {} : {}", builder, e, e);
+      throw new BadClusterStateException("Failed to save " + clustername
+                                         + ": " + e);
+    }
+
+    // providers to validate what there is
+    // TODO: Validation should be done before persistence
+    AggregateConf instanceDescription = builder.getInstanceDescription();
+    validateInstanceDefinition(sliderAM, instanceDescription, sliderFileSystem);
+    validateInstanceDefinition(provider, instanceDescription, sliderFileSystem);
+  }
+
+  private void validateClientAndClusterResource(String clustername,
+      ConfTreeOperations clientResources) throws BadClusterStateException,
+      SliderException, IOException {
+    log.info("Validating upgrade resource definition with current cluster "
+        + "state (components and instance count)");
+    Map<String, Integer> clientComponentInstances = new HashMap<>();
+    for (String componentName : clientResources.getComponentNames()) {
+      if (!SliderKeys.COMPONENT_AM.equals(componentName)) {
+        clientComponentInstances.put(componentName, clientResources
+            .getComponentOptInt(componentName,
+                COMPONENT_INSTANCES, -1));
+      }
+    }
+
+    AggregateConf clusterConf = null;
+    try {
+      clusterConf = loadPersistedClusterDescription(clustername);
+    } catch (LockAcquireFailedException e) {
+      log.warn("Failed to get a Lock on cluster resource : {}", e, e);
+      throw new BadClusterStateException(
+          "Failed to load client resource definition " + clustername + ": " + e, e);
+    }
+    Map<String, Integer> clusterComponentInstances = new HashMap<>();
+    for (Map.Entry<String, Map<String, String>> component : clusterConf
+        .getResources().components.entrySet()) {
+      if (!SliderKeys.COMPONENT_AM.equals(component.getKey())) {
+        clusterComponentInstances.put(
+            component.getKey(),
+            Integer.decode(component.getValue().get(
+                COMPONENT_INSTANCES)));
+      }
+    }
+
+    // client and cluster should be an exact match
+    Iterator<Map.Entry<String, Integer>> clientComponentInstanceIt = clientComponentInstances
+        .entrySet().iterator();
+    while (clientComponentInstanceIt.hasNext()) {
+      Map.Entry<String, Integer> clientComponentInstanceEntry = clientComponentInstanceIt.next();
+      if (clusterComponentInstances.containsKey(clientComponentInstanceEntry.getKey())) {
+        // compare instance count now and remove from both maps if they match
+        if (clusterComponentInstances
+            .get(clientComponentInstanceEntry.getKey()).intValue() == clientComponentInstanceEntry
+            .getValue().intValue()) {
+          clusterComponentInstances.remove(clientComponentInstanceEntry
+              .getKey());
+          clientComponentInstanceIt.remove();
+        }
+      }
+    }
+
+    if (!clientComponentInstances.isEmpty()
+        || !clusterComponentInstances.isEmpty()) {
+      log.error("Mismatch found in upgrade resource definition and cluster "
+          + "resource state");
+      if (!clientComponentInstances.isEmpty()) {
+        log.info("The upgrade resource definitions that do not match are:");
+        for (Map.Entry<String, Integer> clientComponentInstanceEntry : clientComponentInstances
+            .entrySet()) {
+          log.info("    Component Name: {}, Instance count: {}",
+              clientComponentInstanceEntry.getKey(),
+              clientComponentInstanceEntry.getValue());
+        }
+      }
+      if (!clusterComponentInstances.isEmpty()) {
+        log.info("The cluster resources that do not match are:");
+        for (Map.Entry<String, Integer> clusterComponentInstanceEntry : clusterComponentInstances
+            .entrySet()) {
+          log.info("    Component Name: {}, Instance count: {}",
+              clusterComponentInstanceEntry.getKey(),
+              clusterComponentInstanceEntry.getValue());
+        }
+      }
+      throw new BadConfigException("Resource definition provided for "
+          + "upgrade does not match with that of the currently running "
+          + "cluster.\nIf you are aware of what you are doing, rerun the "
+          + "command with " + Arguments.ARG_FORCE + " option.");
+    }
+  }
+
+  protected void persistInstanceDefinition(boolean overwrite,
+                                         Path appconfdir,
+                                         InstanceBuilder builder)
+      throws IOException, SliderException, LockAcquireFailedException {
+    builder.persist(appconfdir, overwrite);
+  }
+
+  @VisibleForTesting
+  public static void replaceTokens(ConfTree conf,
+      String userName, String clusterName) throws IOException {
+    Map<String,String> newglobal = new HashMap<>();
+    for (Entry<String,String> entry : conf.global.entrySet()) {
+      newglobal.put(entry.getKey(), replaceTokens(entry.getValue(),
+          userName, clusterName));
+    }
+    conf.global.putAll(newglobal);
+
+    for (String component : conf.components.keySet()) {
+      Map<String,String> newComponent = new HashMap<>();
+      for (Entry<String,String> entry : conf.components.get(component).entrySet()) {
+        newComponent.put(entry.getKey(), replaceTokens(entry.getValue(),
+            userName, clusterName));
+      }
+      conf.components.get(component).putAll(newComponent);
+    }
+
+    Map<String,List<String>> newcred = new HashMap<>();
+    for (Entry<String,List<String>> entry : conf.credentials.entrySet()) {
+      List<String> resultList = new ArrayList<>();
+      for (String v : entry.getValue()) {
+        resultList.add(replaceTokens(v, userName, clusterName));
+      }
+      newcred.put(replaceTokens(entry.getKey(), userName, clusterName),
+          resultList);
+    }
+    conf.credentials.clear();
+    conf.credentials.putAll(newcred);
+  }
+
+  private static String replaceTokens(String s, String userName,
+      String clusterName) throws IOException {
+    return s.replaceAll(Pattern.quote("${USER}"), userName)
+        .replaceAll(Pattern.quote("${USER_NAME}"), userName)
+        .replaceAll(Pattern.quote("${CLUSTER_NAME}"), clusterName);
+  }
+
+  public FsPermission getClusterDirectoryPermissions(Configuration conf) {
+    String clusterDirPermsOct =
+      conf.get(CLUSTER_DIRECTORY_PERMISSIONS, DEFAULT_CLUSTER_DIRECTORY_PERMISSIONS);
+    return new FsPermission(clusterDirPermsOct);
+  }
+
+  /**
+   * Verify that the Resource Manager is configured (on a non-HA cluster).
+   * with a useful error message
+   * @throws BadCommandArgumentsException the exception raised on an invalid config
+   */
+  public void verifyBindingsDefined() throws BadCommandArgumentsException {
+    InetSocketAddress rmAddr = getRmAddress(getConfig());
+    if (!getConfig().getBoolean(YarnConfiguration.RM_HA_ENABLED, false)
+     && !isAddressDefined(rmAddr)) {
+      throw new BadCommandArgumentsException(
+        E_NO_RESOURCE_MANAGER
+        + " in the argument "
+        + Arguments.ARG_MANAGER
+        + " or the configuration property "
+        + YarnConfiguration.RM_ADDRESS 
+        + " value :" + rmAddr);
+    }
+  }
+
+  /**
+   * Load and start a cluster specification.
+   * This assumes that all validation of args and cluster state
+   * have already taken place
+   *
+   * @param clustername name of the cluster.
+   * @param launchArgs launch arguments
+   * @return the exit code
+   * @throws YarnException
+   * @throws IOException
+   */
+  protected int startCluster(String clustername,
+                           LaunchArgsAccessor launchArgs) throws
+                                                          YarnException,
+                                                          IOException {
+    Path clusterDirectory = sliderFileSystem.buildClusterDirPath(clustername);
+    AggregateConf instanceDefinition = loadInstanceDefinitionUnresolved(
+      clustername,
+      clusterDirectory);
+
+    LaunchedApplication launchedApplication =
+      launchApplication(clustername, clusterDirectory, instanceDefinition,
+                        serviceArgs.isDebug());
+
+    if (launchArgs.getOutputFile() != null) {
+      // output file has been requested. Get the app report and serialize it
+      ApplicationReport report =
+          launchedApplication.getApplicationReport();
+      SerializedApplicationReport sar = new SerializedApplicationReport(report);
+      sar.submitTime = System.currentTimeMillis();
+      ApplicationReportSerDeser serDeser = new ApplicationReportSerDeser();
+      serDeser.save(sar, launchArgs.getOutputFile());
+    }
+    int waittime = launchArgs.getWaittime();
+    if (waittime > 0) {
+      return waitForAppRunning(launchedApplication, waittime, waittime);
+    } else {
+      // no waiting
+      return EXIT_SUCCESS;
+    }
+  }
+
+  /**
+   * Load the instance definition. It is not resolved at this point
+   * @param name cluster name
+   * @param clusterDirectory cluster dir
+   * @return the loaded configuration
+   * @throws IOException
+   * @throws SliderException
+   * @throws UnknownApplicationInstanceException if the file is not found
+   */
+  public AggregateConf loadInstanceDefinitionUnresolved(String name,
+            Path clusterDirectory) throws IOException, SliderException {
+
+    try {
+      AggregateConf definition =
+        InstanceIO.loadInstanceDefinitionUnresolved(sliderFileSystem,
+                                                    clusterDirectory);
+      definition.setName(name);
+      return definition;
+    } catch (FileNotFoundException e) {
+      throw UnknownApplicationInstanceException.unknownInstance(name, e);
+    }
+  }
+
+  /**
+   * Load the instance definition. 
+   * @param name cluster name
+   * @param resolved flag to indicate the cluster should be resolved
+   * @return the loaded configuration
+   * @throws IOException IO problems
+   * @throws SliderException slider explicit issues
+   * @throws UnknownApplicationInstanceException if the file is not found
+   */
+    public AggregateConf loadInstanceDefinition(String name,
+        boolean resolved) throws
+        IOException,
+        SliderException {
+
+    Path clusterDirectory = sliderFileSystem.buildClusterDirPath(name);
+    AggregateConf instanceDefinition = loadInstanceDefinitionUnresolved(
+      name,
+      clusterDirectory);
+    if (resolved) {
+      instanceDefinition.resolve();
+    }
+    return instanceDefinition;
+
+  }
+
+  protected AppMasterLauncher setupAppMasterLauncher(String clustername,
+      Path clusterDirectory,
+      AggregateConf instanceDefinition,
+      boolean debugAM)
+    throws YarnException, IOException{
+    deployedClusterName = clustername;
+    validateClusterName(clustername);
+    verifyNoLiveClusters(clustername, "Launch");
+    Configuration config = getConfig();
+    lookupZKQuorum();
+    boolean clusterSecure = isHadoopClusterSecure(config);
+    //create the Slider AM provider -this helps set up the AM
+    SliderAMClientProvider sliderAM = new SliderAMClientProvider(config);
+
+    instanceDefinition.resolve();
+    launchedInstanceDefinition = instanceDefinition;
+
+    ConfTreeOperations internalOperations = instanceDefinition.getInternalOperations();
+    MapOperations internalOptions = internalOperations.getGlobalOptions();
+    ConfTreeOperations resourceOperations = instanceDefinition.getResourceOperations();
+    ConfTreeOperations appOperations = instanceDefinition.getAppConfOperations();
+    Path generatedConfDirPath =
+      createPathThatMustExist(internalOptions.getMandatoryOption(
+        INTERNAL_GENERATED_CONF_PATH));
+    Path snapshotConfPath =
+      createPathThatMustExist(internalOptions.getMandatoryOption(
+        INTERNAL_SNAPSHOT_CONF_PATH));
+
+
+    // cluster Provider
+    AbstractClientProvider provider = createClientProvider(
+      internalOptions.getMandatoryOption(INTERNAL_PROVIDER_NAME));
+    if (log.isDebugEnabled()) {
+      log.debug(instanceDefinition.toString());
+    }
+    MapOperations sliderAMResourceComponent =
+      resourceOperations.getOrAddComponent(SliderKeys.COMPONENT_AM);
+    MapOperations resourceGlobalOptions = resourceOperations.getGlobalOptions();
+
+    // add the tags if available
+    Set<String> applicationTags = provider.getApplicationTags(sliderFileSystem,
+        getApplicationDefinitionPath(appOperations));
+
+    Credentials credentials = null;
+    if (clusterSecure) {
+      // pick up oozie credentials
+      credentials = CredentialUtils.loadTokensFromEnvironment(System.getenv(),
+          config);
+      if (credentials == null) {
+        // nothing from oozie, so build up directly
+        credentials = new Credentials(
+            UserGroupInformation.getCurrentUser().getCredentials());
+        CredentialUtils.addRMRenewableFSDelegationTokens(config,
+            sliderFileSystem.getFileSystem(),
+            credentials);
+        CredentialUtils.addRMDelegationToken(yarnClient, credentials);
+
+      } else {
+        log.info("Using externally supplied credentials to launch AM");
+      }
+    }
+
+    AppMasterLauncher amLauncher = new AppMasterLauncher(clustername,
+        SliderKeys.APP_TYPE,
+        config,
+        sliderFileSystem,
+        yarnClient,
+        clusterSecure,
+        sliderAMResourceComponent,
+        resourceGlobalOptions,
+        applicationTags,
+        credentials);
+
+    ApplicationId appId = amLauncher.getApplicationId();
+    // set the application name;
+    amLauncher.setKeepContainersOverRestarts(true);
+
+    int maxAppAttempts = config.getInt(KEY_AM_RESTART_LIMIT, 0);
+    amLauncher.setMaxAppAttempts(maxAppAttempts);
+
+    sliderFileSystem.purgeAppInstanceTempFiles(clustername);
+    Path tempPath = sliderFileSystem.createAppInstanceTempPath(
+        clustername,
+        appId.toString() + "/am");
+    String libdir = "lib";
+    Path libPath = new Path(tempPath, libdir);
+    sliderFileSystem.getFileSystem().mkdirs(libPath);
+    log.debug("FS={}, tempPath={}, libdir={}", sliderFileSystem, tempPath, libPath);
+ 
+    // set local resources for the application master
+    // local files or archives as needed
+    // In this scenario, the jar file for the application master is part of the local resources
+    Map<String, LocalResource> localResources = amLauncher.getLocalResources();
+    
+    // look for the configuration directory named on the command line
+    boolean hasServerLog4jProperties = false;
+    Path remoteConfPath = null;
+    String relativeConfDir = null;
+    String confdirProp = System.getProperty(SliderKeys.PROPERTY_CONF_DIR);
+    if (isUnset(confdirProp)) {
+      log.debug("No local configuration directory provided as system property");
+    } else {
+      File confDir = new File(confdirProp);
+      if (!confDir.exists()) {
+        throw new BadConfigException(E_CONFIGURATION_DIRECTORY_NOT_FOUND,
+                                     confDir);
+      }
+      Path localConfDirPath = createLocalPath(confDir);
+      remoteConfPath = new Path(clusterDirectory, SliderKeys.SUBMITTED_CONF_DIR);
+      log.debug("Slider configuration directory is {}; remote to be {}", 
+          localConfDirPath, remoteConfPath);
+      copyDirectory(config, localConfDirPath, remoteConfPath, null);
+
+      File log4jserver =
+          new File(confDir, SliderKeys.LOG4J_SERVER_PROP_FILENAME);
+      hasServerLog4jProperties = log4jserver.isFile();
+    }
+    // the assumption here is that minimr cluster => this is a test run
+    // and the classpath can look after itself
+
+    boolean usingMiniMRCluster = getUsingMiniMRCluster();
+    if (!usingMiniMRCluster) {
+
+      log.debug("Destination is not a MiniYARNCluster -copying full classpath");
+
+      // insert conf dir first
+      if (remoteConfPath != null) {
+        relativeConfDir = SliderKeys.SUBMITTED_CONF_DIR;
+        Map<String, LocalResource> submittedConfDir =
+          sliderFileSystem.submitDirectory(remoteConfPath,
+                                         relativeConfDir);
+        mergeMaps(localResources, submittedConfDir);
+      }
+    }
+    // build up the configuration 
+    // IMPORTANT: it is only after this call that site configurations
+    // will be valid.
+
+    propagatePrincipals(config, instanceDefinition);
+    // validate security data
+
+/*
+    // turned off until tested
+    SecurityConfiguration securityConfiguration =
+        new SecurityConfiguration(config,
+            instanceDefinition, clustername);
+    
+*/
+    Configuration clientConfExtras = new Configuration(false);
+    // then build up the generated path.
+    FsPermission clusterPerms = getClusterDirectoryPermissions(config);
+    copyDirectory(config, snapshotConfPath, generatedConfDirPath,
+        clusterPerms);
+
+
+    // standard AM resources
+    sliderAM.prepareAMAnd

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[40/76] [abbrv] hadoop git commit: YARN-5513. Move Java only tests from slider develop to yarn-native-services. Contributed by Gour Saha

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestAgentProviderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestAgentProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestAgentProviderService.java
new file mode 100644
index 0000000..7fceac7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestAgentProviderService.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.publisher;
+
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.providers.agent.AgentProviderService;
+import org.apache.slider.server.appmaster.actions.QueueAccess;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ *
+ */
+public class TestAgentProviderService extends AgentProviderService {
+  protected static final Logger log =
+      LoggerFactory.getLogger(TestAgentProviderService.class);
+
+  public TestAgentProviderService() {
+    super();
+    log.info("TestAgentProviderService created");
+  }
+
+  @Override
+  public void bind(StateAccessForProviders stateAccessor,
+      QueueAccess queueAccess,
+      List<Container> liveContainers) {
+    super.bind(stateAccessor, queueAccess, liveContainers);
+    Map<String,String> dummyProps = new HashMap<String, String>();
+    dummyProps.put("prop1", "val1");
+    dummyProps.put("prop2", "val2");
+    log.info("publishing dummy-site.xml with values {}", dummyProps);
+    publishApplicationInstanceData("dummy-site", "dummy configuration",
+                                   dummyProps.entrySet());
+    // publishing global config for testing purposes
+    publishApplicationInstanceData("global", "global configuration",
+                                   stateAccessor.getAppConfSnapshot()
+                                       .getGlobalOptions().entrySet());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestSliderProviderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestSliderProviderFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestSliderProviderFactory.java
new file mode 100644
index 0000000..f49e15a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/publisher/TestSliderProviderFactory.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.publisher;
+
+import org.apache.slider.providers.ProviderService;
+import org.apache.slider.providers.agent.AgentProviderFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ *
+ */
+public class TestSliderProviderFactory extends AgentProviderFactory{
+  protected static final Logger log =
+      LoggerFactory.getLogger(TestSliderProviderFactory.class);
+
+  public TestSliderProviderFactory() {
+    log.info("Created TestSliderProviderFactory");
+  }
+
+  @Override
+  public ProviderService createServerProvider() {
+    log.info("Creating TestAgentProviderService");
+    return new TestAgentProviderService();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/servicemonitor/TestPortProbe.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/servicemonitor/TestPortProbe.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/servicemonitor/TestPortProbe.java
new file mode 100644
index 0000000..a93ec57
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/servicemonitor/TestPortProbe.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.servicemonitor;
+
+import org.junit.Assert;
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
+
+public class TestPortProbe extends Assert {
+  /**
+   * Assert that a port probe failed if the port is closed
+   * @throws Throwable
+   */
+  @Test
+  public void testPortProbeFailsClosedPort() throws Throwable {
+    PortProbe probe = new PortProbe("127.0.0.1", 65500, 100, "", new Configuration());
+    probe.init();
+    ProbeStatus status = probe.ping(true);
+    assertFalse("Expected a failure but got successful result: " + status,
+      status.isSuccess());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/security/TestCertificateManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/security/TestCertificateManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/security/TestCertificateManager.java
new file mode 100644
index 0000000..7a4a586
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/security/TestCertificateManager.java
@@ -0,0 +1,540 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.services.security;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.alias.CredentialProvider;
+import org.apache.hadoop.security.alias.CredentialProviderFactory;
+import org.apache.hadoop.security.alias.JavaKeyStoreProvider;
+import org.apache.slider.Slider;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.SliderXmlConfKeys;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.exceptions.SliderException;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetAddress;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.Principal;
+import java.security.cert.Certificate;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ *
+ */
+public class TestCertificateManager {
+  @Rule
+  public TemporaryFolder workDir = new TemporaryFolder();
+  private File secDir;
+  private CertificateManager certMan;
+
+  @Before
+  public void setup() throws Exception {
+    certMan = new CertificateManager();
+    MapOperations compOperations = new MapOperations();
+    secDir = new File(workDir.getRoot(), SliderKeys.SECURITY_DIR);
+    File keystoreFile = new File(secDir, SliderKeys.KEYSTORE_FILE_NAME);
+    compOperations.put(SliderXmlConfKeys.KEY_KEYSTORE_LOCATION,
+                       keystoreFile.getAbsolutePath());
+    certMan.initialize(compOperations, "cahost", null, null);
+  }
+
+  @Test
+  public void testServerCertificateGenerated() throws Exception {
+    File serverCrt = new File(secDir, SliderKeys.CRT_FILE_NAME);
+    Assert.assertTrue("Server CRD does not exist:" + serverCrt,
+                      serverCrt.exists());
+  }
+
+  @Test
+  public void testAMKeystoreGenerated() throws Exception {
+    File keystoreFile = new File(secDir, SliderKeys.KEYSTORE_FILE_NAME);
+    Assert.assertTrue("Keystore does not exist: " + keystoreFile,
+                      keystoreFile.exists());
+    InputStream is = null;
+    try {
+
+      is = new FileInputStream(keystoreFile);
+      KeyStore keystore = KeyStore.getInstance("pkcs12");
+      String password = SecurityUtils.getKeystorePass();
+      keystore.load(is, password.toCharArray());
+
+      Certificate certificate = keystore.getCertificate(
+          keystore.aliases().nextElement());
+      Assert.assertNotNull(certificate);
+
+      if (certificate instanceof X509Certificate) {
+        X509Certificate x509cert = (X509Certificate) certificate;
+
+        // Get subject
+        Principal principal = x509cert.getSubjectDN();
+        String subjectDn = principal.getName();
+        Assert.assertEquals("wrong DN",
+                            "CN=cahost",
+                            subjectDn);
+
+        // Get issuer
+        principal = x509cert.getIssuerDN();
+        String issuerDn = principal.getName();
+        Assert.assertEquals("wrong Issuer DN",
+                            "CN=cahost",
+                            issuerDn);
+      }
+    } finally {
+      if(null != is) {
+        is.close();
+      }
+    }
+  }
+
+  @Test
+  public void testContainerCertificateGeneration() throws Exception {
+    certMan.generateContainerCertificate("testhost", "container1");
+    Assert.assertTrue("container certificate not generated",
+                      new File(secDir, "container1.crt").exists());
+  }
+
+  @Test
+  public void testContainerKeystoreGeneration() throws Exception {
+    SecurityStore keystoreFile = certMan.generateContainerKeystore("testhost",
+                                                                   "container1",
+                                                                   "component1",
+                                                                   "password");
+    validateKeystore(keystoreFile.getFile(), "testhost", "cahost");
+  }
+
+  private void validateKeystore(File keystoreFile, String certHostname,
+                                String issuerHostname)
+      throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException {
+    Assert.assertTrue("container keystore not generated",
+                      keystoreFile.exists());
+
+    InputStream is = null;
+    try {
+
+      is = new FileInputStream(keystoreFile);
+      KeyStore keystore = KeyStore.getInstance("pkcs12");
+      String password = "password";
+      keystore.load(is, password.toCharArray());
+
+      Certificate certificate = keystore.getCertificate(
+          keystore.aliases().nextElement());
+      Assert.assertNotNull(certificate);
+
+      if (certificate instanceof X509Certificate) {
+        X509Certificate x509cert = (X509Certificate) certificate;
+
+        // Get subject
+        Principal principal = x509cert.getSubjectDN();
+        String subjectDn = principal.getName();
+        Assert.assertEquals("wrong DN", "CN=" + certHostname + ", OU=container1",
+                            subjectDn);
+
+        // Get issuer
+        principal = x509cert.getIssuerDN();
+        String issuerDn = principal.getName();
+        Assert.assertEquals("wrong Issuer DN",
+                            "CN=" + issuerHostname,
+                            issuerDn);
+      }
+    } finally {
+      if(null != is) {
+        is.close();
+      }
+    }
+  }
+
+  @Test
+  public void testContainerKeystoreGenerationViaStoresGenerator() throws Exception {
+    AggregateConf instanceDefinition = new AggregateConf();
+    MapOperations compOps = new MapOperations();
+    instanceDefinition.getAppConf().components.put("component1", compOps);
+    compOps.put(SliderKeys.COMP_KEYSTORE_PASSWORD_PROPERTY_KEY,
+                "app1.component1.password.property");
+    compOps.put(SliderKeys.COMP_STORES_REQUIRED_KEY, "true");
+    instanceDefinition.getAppConf().global.put(
+        "app1.component1.password.property", "password");
+    instanceDefinition.resolve();
+    SecurityStore[]
+        files = StoresGenerator.generateSecurityStores("testhost",
+                                                       "container1",
+                                                       "component1",
+                                                       instanceDefinition,
+                                                       compOps);
+    assertEquals("wrong number of stores", 1, files.length);
+    validateKeystore(files[0].getFile(), "testhost", "cahost");
+  }
+
+  @Test
+  public void testContainerKeystoreGenerationViaStoresGeneratorUsingGlobalProps() throws Exception {
+    AggregateConf instanceDefinition = new AggregateConf();
+    MapOperations compOps = new MapOperations();
+    instanceDefinition.getAppConf().components.put("component1", compOps);
+    compOps.put(SliderKeys.COMP_KEYSTORE_PASSWORD_PROPERTY_KEY,
+                "app1.component1.password.property");
+    instanceDefinition.getAppConf().global.put(SliderKeys.COMP_STORES_REQUIRED_KEY, "true");
+    compOps.put(
+        "app1.component1.password.property", "password");
+    instanceDefinition.resolve();
+    SecurityStore[]
+        files = StoresGenerator.generateSecurityStores("testhost",
+                                                       "container1",
+                                                       "component1",
+                                                       instanceDefinition,
+                                                       compOps);
+    assertEquals("wrong number of stores", 1, files.length);
+    validateKeystore(files[0].getFile(), "testhost", "cahost");
+  }
+
+  @Test
+  public void testContainerKeystoreGenerationViaStoresGeneratorOverrideGlobalSetting() throws Exception {
+    AggregateConf instanceDefinition = new AggregateConf();
+    MapOperations compOps = setupComponentOptions(true, null,
+                                                  "app1.component1.password.property",
+                                                  null, null);
+    instanceDefinition.getAppConf().components.put("component1", compOps);
+    instanceDefinition.getAppConf().global.put(
+        "app1.component1.password.property", "password");
+    instanceDefinition.getAppConf().global.put(SliderKeys.COMP_STORES_REQUIRED_KEY, "false");
+    instanceDefinition.resolve();
+    SecurityStore[]
+        files = StoresGenerator.generateSecurityStores("testhost",
+                                                       "container1",
+                                                       "component1",
+                                                       instanceDefinition,
+                                                       compOps);
+    assertEquals("wrong number of stores", 1, files.length);
+    validateKeystore(files[0].getFile(), "testhost", "cahost");
+  }
+
+  @Test
+  public void testContainerTrusttoreGeneration() throws Exception {
+    SecurityStore keystoreFile =
+        certMan.generateContainerKeystore("testhost",
+                                          "container1",
+                                          "component1",
+                                          "keypass");
+    Assert.assertTrue("container keystore not generated",
+                      keystoreFile.getFile().exists());
+    SecurityStore truststoreFile =
+        certMan.generateContainerTruststore("container1",
+                                            "component1", "trustpass"
+        );
+    Assert.assertTrue("container truststore not generated",
+                      truststoreFile.getFile().exists());
+
+    validateTruststore(keystoreFile.getFile(), truststoreFile.getFile());
+  }
+
+  @Test
+  public void testContainerGenerationUsingStoresGeneratorNoTruststore() throws Exception {
+    AggregateConf instanceDefinition = new AggregateConf();
+    MapOperations compOps = new MapOperations();
+    compOps.put(SliderKeys.COMP_STORES_REQUIRED_KEY, "true");
+    compOps.put(SliderKeys.COMP_KEYSTORE_PASSWORD_ALIAS_KEY,
+                "test.keystore.password");
+
+    setupCredentials(instanceDefinition, "test.keystore.password", null);
+
+    SecurityStore[]
+        files = StoresGenerator.generateSecurityStores("testhost",
+                                                       "container1",
+                                                       "component1",
+                                                       instanceDefinition,
+                                                       compOps);
+    assertEquals("wrong number of stores", 1, files.length);
+    File keystoreFile = CertificateManager.getContainerKeystoreFilePath(
+        "container1", "component1");
+    Assert.assertTrue("container keystore not generated",
+                      keystoreFile.exists());
+
+    Assert.assertTrue("keystore not in returned list",
+                      Arrays.asList(files).contains(new SecurityStore(keystoreFile,
+                                                    SecurityStore.StoreType.keystore)));
+    File truststoreFile =
+        CertificateManager.getContainerTruststoreFilePath("component1",
+                                                          "container1");
+    Assert.assertFalse("container truststore generated",
+                      truststoreFile.exists());
+    Assert.assertFalse("truststore in returned list",
+                      Arrays.asList(files).contains(new SecurityStore(truststoreFile,
+                                                    SecurityStore.StoreType.truststore)));
+
+  }
+
+  @Test
+  public void testContainerGenerationUsingStoresGeneratorJustTruststoreWithDefaultAlias() throws Exception {
+    AggregateConf instanceDefinition = new AggregateConf();
+    MapOperations compOps = setupComponentOptions(true);
+
+    setupCredentials(instanceDefinition, null,
+                     SliderKeys.COMP_TRUSTSTORE_PASSWORD_ALIAS_DEFAULT);
+
+    SecurityStore[]
+        files = StoresGenerator.generateSecurityStores("testhost",
+                                                       "container1",
+                                                       "component1",
+                                                       instanceDefinition,
+                                                       compOps);
+    assertEquals("wrong number of stores", 1, files.length);
+    File keystoreFile = CertificateManager.getContainerKeystoreFilePath(
+        "container1", "component1");
+    Assert.assertFalse("container keystore generated",
+                       keystoreFile.exists());
+    Assert.assertFalse("keystore in returned list",
+                       Arrays.asList(files).contains(keystoreFile));
+    File truststoreFile =
+        CertificateManager.getContainerTruststoreFilePath("component1",
+                                                          "container1");
+    Assert.assertTrue("container truststore not generated",
+                      truststoreFile.exists());
+    Assert.assertTrue("truststore not in returned list",
+                      Arrays.asList(files).contains(new SecurityStore(truststoreFile,
+                                                                      SecurityStore.StoreType.truststore)));
+
+  }
+
+  @Test
+  public void testContainerTrusttoreGenerationUsingStoresGenerator() throws Exception {
+    AggregateConf instanceDefinition = new AggregateConf();
+    MapOperations compOps = setupComponentOptions(true,
+                                                  "test.keystore.password",
+                                                  null,
+                                                  "test.truststore.password",
+                                                  null);
+
+    setupCredentials(instanceDefinition, "test.keystore.password",
+                     "test.truststore.password");
+
+    SecurityStore[]
+        files = StoresGenerator.generateSecurityStores("testhost",
+                                                       "container1",
+                                                       "component1",
+                                                       instanceDefinition,
+                                                       compOps);
+    assertEquals("wrong number of stores", 2, files.length);
+    File keystoreFile = CertificateManager.getContainerKeystoreFilePath(
+        "container1", "component1");
+    Assert.assertTrue("container keystore not generated",
+                      keystoreFile.exists());
+    Assert.assertTrue("keystore not in returned list",
+                      Arrays.asList(files).contains(new SecurityStore(keystoreFile,
+                                                                      SecurityStore.StoreType.keystore)));
+    File truststoreFile =
+        CertificateManager.getContainerTruststoreFilePath("component1",
+                                                          "container1");
+    Assert.assertTrue("container truststore not generated",
+                      truststoreFile.exists());
+    Assert.assertTrue("truststore not in returned list",
+                      Arrays.asList(files).contains(new SecurityStore(truststoreFile,
+                                                                      SecurityStore.StoreType.truststore)));
+
+    validateTruststore(keystoreFile, truststoreFile);
+  }
+
+  private void setupCredentials(AggregateConf instanceDefinition,
+                                String keyAlias, String trustAlias)
+      throws Exception {
+    Configuration conf = new Configuration();
+    final Path jksPath = new Path(SecurityUtils.getSecurityDir(), "test.jks");
+    final String ourUrl =
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri();
+
+    File file = new File(SecurityUtils.getSecurityDir(), "test.jks");
+    file.delete();
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, ourUrl);
+
+    instanceDefinition.getAppConf().credentials.put(ourUrl, new ArrayList<String>());
+
+    CredentialProvider provider =
+        CredentialProviderFactory.getProviders(conf).get(0);
+
+    // create new aliases
+    try {
+
+      if (keyAlias != null) {
+        char[] storepass = {'k', 'e', 'y', 'p', 'a', 's', 's'};
+        provider.createCredentialEntry(
+            keyAlias, storepass);
+      }
+
+      if (trustAlias != null) {
+        char[] trustpass = {'t', 'r', 'u', 's', 't', 'p', 'a', 's', 's'};
+        provider.createCredentialEntry(
+            trustAlias, trustpass);
+      }
+
+      // write out so that it can be found in checks
+      provider.flush();
+    } catch (Exception e) {
+      e.printStackTrace();
+      throw e;
+    }
+  }
+
+  private MapOperations setupComponentOptions(boolean storesRequired) {
+    return this.setupComponentOptions(storesRequired, null, null, null, null);
+  }
+
+  private MapOperations setupComponentOptions(boolean storesRequired,
+                                              String keyAlias,
+                                              String keyPwd,
+                                              String trustAlias,
+                                              String trustPwd) {
+    MapOperations compOps = new MapOperations();
+    compOps.put(SliderKeys.COMP_STORES_REQUIRED_KEY,
+                Boolean.toString(storesRequired));
+    if (keyAlias != null) {
+      compOps.put(SliderKeys.COMP_KEYSTORE_PASSWORD_ALIAS_KEY,
+                  "test.keystore.password");
+    }
+    if (trustAlias != null) {
+      compOps.put(SliderKeys.COMP_TRUSTSTORE_PASSWORD_ALIAS_KEY,
+                  "test.truststore.password");
+    }
+    if (keyPwd != null) {
+      compOps.put(SliderKeys.COMP_KEYSTORE_PASSWORD_PROPERTY_KEY,
+                  keyPwd);
+    }
+    if (trustPwd != null) {
+      compOps.put(SliderKeys.COMP_TRUSTSTORE_PASSWORD_PROPERTY_KEY,
+                  trustPwd);
+    }
+    return compOps;
+  }
+
+  @Test
+  public void testContainerStoresGenerationKeystoreOnly() throws Exception {
+    AggregateConf instanceDefinition = new AggregateConf();
+    MapOperations compOps = new MapOperations();
+    compOps.put(SliderKeys.COMP_STORES_REQUIRED_KEY, "true");
+
+    setupCredentials(instanceDefinition,
+                     SliderKeys.COMP_KEYSTORE_PASSWORD_ALIAS_DEFAULT, null);
+
+    SecurityStore[]
+        files = StoresGenerator.generateSecurityStores("testhost",
+                                                       "container1",
+                                                       "component1",
+                                                       instanceDefinition,
+                                                       compOps);
+    assertEquals("wrong number of stores", 1, files.length);
+    File keystoreFile = CertificateManager.getContainerKeystoreFilePath(
+        "container1", "component1");
+    Assert.assertTrue("container keystore not generated",
+                      keystoreFile.exists());
+    Assert.assertTrue("keystore not in returned list",
+                      Arrays.asList(files).contains(new SecurityStore(keystoreFile,
+                                                                      SecurityStore.StoreType.keystore)));
+    File truststoreFile =
+        CertificateManager.getContainerTruststoreFilePath("component1",
+                                                          "container1");
+    Assert.assertFalse("container truststore generated",
+                       truststoreFile.exists());
+    Assert.assertFalse("truststore in returned list",
+                       Arrays.asList(files).contains(new SecurityStore(truststoreFile,
+                                                                       SecurityStore.StoreType.truststore)));
+
+  }
+
+  @Test
+  public void testContainerStoresGenerationMisconfiguration() throws Exception {
+    AggregateConf instanceDefinition = new AggregateConf();
+    MapOperations compOps = new MapOperations();
+    compOps.put(SliderKeys.COMP_STORES_REQUIRED_KEY, "true");
+
+    setupCredentials(instanceDefinition, "cant.be.found", null);
+
+    try {
+      StoresGenerator.generateSecurityStores("testhost", "container1",
+                                                            "component1", instanceDefinition,
+                                                            compOps);
+      Assert.fail("SliderException should have been generated");
+    } catch (SliderException e) {
+      // ignore - should be thrown
+    }
+  }
+
+  private void validateTruststore(File keystoreFile, File truststoreFile)
+      throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException {
+    InputStream keyis = null;
+    InputStream trustis = null;
+    try {
+
+      // create keystore
+      keyis = new FileInputStream(keystoreFile);
+      KeyStore keystore = KeyStore.getInstance("pkcs12");
+      String password = "keypass";
+      keystore.load(keyis, password.toCharArray());
+
+      // obtain server cert
+      Certificate certificate = keystore.getCertificate(
+          keystore.aliases().nextElement());
+      Assert.assertNotNull(certificate);
+
+      // create trust store from generated trust store file
+      trustis = new FileInputStream(truststoreFile);
+      KeyStore truststore = KeyStore.getInstance("pkcs12");
+      password = "trustpass";
+      truststore.load(trustis, password.toCharArray());
+
+      // validate keystore cert using trust store
+      TrustManagerFactory
+          trustManagerFactory =
+          TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
+      trustManagerFactory.init(truststore);
+
+      for (TrustManager trustManager: trustManagerFactory.getTrustManagers()) {
+        if (trustManager instanceof X509TrustManager) {
+          X509TrustManager x509TrustManager = (X509TrustManager)trustManager;
+          x509TrustManager.checkServerTrusted(
+              new X509Certificate[] {(X509Certificate) certificate},
+              "RSA_EXPORT");
+        }
+      }
+
+    } finally {
+      if(null != keyis) {
+        keyis.close();
+      }
+      if(null != trustis) {
+        trustis.close();
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/security/TestMultiThreadedStoreGeneration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/security/TestMultiThreadedStoreGeneration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/security/TestMultiThreadedStoreGeneration.java
new file mode 100644
index 0000000..2e2ffce
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/security/TestMultiThreadedStoreGeneration.java
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.services.security;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.alias.CredentialProvider;
+import org.apache.hadoop.security.alias.CredentialProviderFactory;
+import org.apache.hadoop.security.alias.JavaKeyStoreProvider;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.SliderXmlConfKeys;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.exceptions.SliderException;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ *
+ */
+public class TestMultiThreadedStoreGeneration {
+
+  public static final int NUM_THREADS = 30;
+  @Rule
+  public TemporaryFolder workDir = new TemporaryFolder();;
+
+  private void setupCredentials(AggregateConf instanceDefinition,
+                                String keyAlias, String trustAlias)
+      throws Exception {
+    Configuration conf = new Configuration();
+    final Path jksPath = new Path(SecurityUtils.getSecurityDir(), "test.jks");
+    final String ourUrl =
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri();
+
+    File file = new File(SecurityUtils.getSecurityDir(), "test.jks");
+    file.delete();
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, ourUrl);
+
+    instanceDefinition.getAppConf().credentials.put(ourUrl, new ArrayList<String>());
+
+    CredentialProvider provider =
+        CredentialProviderFactory.getProviders(conf).get(0);
+
+    // create new aliases
+    try {
+
+      if (keyAlias != null) {
+        char[] storepass = {'k', 'e', 'y', 'p', 'a', 's', 's'};
+        provider.createCredentialEntry(
+            keyAlias, storepass);
+      }
+
+      if (trustAlias != null) {
+        char[] trustpass = {'t', 'r', 'u', 's', 't', 'p', 'a', 's', 's'};
+        provider.createCredentialEntry(
+            trustAlias, trustpass);
+      }
+
+      // write out so that it can be found in checks
+      provider.flush();
+    } catch (Exception e) {
+      e.printStackTrace();
+      throw e;
+    }
+  }
+
+
+  @Test
+  public void testMultiThreadedStoreGeneration() throws Exception {
+
+    CertificateManager certMan = new CertificateManager();
+    MapOperations compOperations = new MapOperations();
+    File secDir = new File(workDir.getRoot(), SliderKeys.SECURITY_DIR);
+    File keystoreFile = new File(secDir, SliderKeys.KEYSTORE_FILE_NAME);
+    compOperations.put(SliderXmlConfKeys.KEY_KEYSTORE_LOCATION,
+                       keystoreFile.getAbsolutePath());
+    certMan.initialize(compOperations, "cahost", null, null);
+
+    final CountDownLatch latch = new CountDownLatch(1);
+    final List<SecurityStore> stores = new ArrayList<>();
+    List<Thread> threads = new ArrayList<>();
+    final AggregateConf instanceDefinition = new AggregateConf();
+
+    setupCredentials(instanceDefinition,
+                     SliderKeys.COMP_KEYSTORE_PASSWORD_ALIAS_DEFAULT, null);
+    final MapOperations compOps = new MapOperations();
+    compOps.put(SliderKeys.COMP_STORES_REQUIRED_KEY, "true");
+
+    for (int i=0; i<NUM_THREADS; ++i) {
+      final int finalI = i;
+      Runnable runner = new Runnable() {
+        public void run() {
+          System.out.println ("----> In run");
+          try {
+            latch.await();
+            SecurityStore[] stores1 = StoresGenerator.generateSecurityStores(
+                "testhost",
+                "container" + finalI,
+                "component" + finalI,
+                instanceDefinition,
+                compOps);
+            System.out.println ("----> stores1" + stores1);
+            List<SecurityStore>
+                securityStores =
+                Arrays.asList(stores1);
+            stores.addAll(securityStores);
+          } catch (InterruptedException e) {
+            e.printStackTrace();
+          } catch (SliderException e) {
+            e.printStackTrace();
+          } catch (IOException e) {
+            e.printStackTrace();
+          } catch (Exception e) {
+            e.printStackTrace();
+          }
+        }
+      };
+      Thread thread = new Thread(runner, "TestThread" + i);
+      threads.add(thread);
+      thread.start();
+    }
+    latch.countDown();
+    for (Thread t : threads) {
+      t.join();
+    }
+
+    for (int i=0; i < NUM_THREADS; i++) {
+      assertTrue("keystore " + i + " not generated", stores.get(i).getFile().exists());
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/MockService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/MockService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/MockService.java
new file mode 100644
index 0000000..588f621
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/MockService.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.service.ServiceStateException;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+public class MockService extends AbstractService {
+  private final boolean fail;
+  private final int lifespan;
+  private final ExecutorService executorService =
+      Executors.newSingleThreadExecutor();
+
+  MockService() {
+    this("mock", false, -1);
+  }
+
+  MockService(String name, boolean fail, int lifespan) {
+    super(name);
+    this.fail = fail;
+    this.lifespan = lifespan;
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    //act on the lifespan here
+    if (lifespan > 0) {
+      executorService.submit(new Runnable() {
+        @Override
+        public void run() {
+          try {
+            Thread.sleep(lifespan);
+          } catch (InterruptedException ignored) {
+
+          }
+          finish();
+        }
+      });
+    } else {
+      if (lifespan == 0) {
+        finish();
+      } else {
+        //continue until told not to
+      }
+    }
+  }
+
+  void finish() {
+    if (fail) {
+      ServiceStateException e =
+          new ServiceStateException(getName() + " failed");
+
+      noteFailure(e);
+      stop();
+      throw e;
+    } else {
+      stop();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/ParentWorkflowTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/ParentWorkflowTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/ParentWorkflowTestBase.java
new file mode 100644
index 0000000..a11a1cf
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/ParentWorkflowTestBase.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+import org.apache.hadoop.service.Service;
+
+/**
+ * Extends {@link WorkflowServiceTestBase} with parent-specific operations
+ * and logic to build up and run the parent service
+ */
+public abstract class ParentWorkflowTestBase extends WorkflowServiceTestBase {
+
+  /**
+   * Wait a second for the service parent to stop
+   * @param parent the service to wait for
+   */
+  protected void waitForParentToStop(ServiceParent parent) {
+    waitForParentToStop(parent, 1000);
+  }
+
+  /**
+   * Wait for the service parent to stop
+   * @param parent the service to wait for
+   * @param timeout time in milliseconds
+   */
+  protected void waitForParentToStop(ServiceParent parent, int timeout) {
+    boolean stop = parent.waitForServiceToStop(timeout);
+    if (!stop) {
+      logState(parent);
+      fail("Service failed to stop : after " + timeout + " millis " + parent);
+    }
+  }
+
+  /**
+   * Subclasses are require to implement this and return an instance of a
+   * ServiceParent
+   * @param services a possibly empty list of services
+   * @return an inited -but -not-started- service parent instance
+   */
+  protected abstract ServiceParent buildService(Service... services);
+
+  /**
+   * Use {@link #buildService(Service...)} to create service and then start it
+   * @param services
+   * @return
+   */
+  protected ServiceParent startService(Service... services) {
+    ServiceParent parent = buildService(services);
+    //expect service to start and stay started
+    parent.start();
+    return parent;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/ProcessCommandFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/ProcessCommandFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/ProcessCommandFactory.java
new file mode 100644
index 0000000..4a19417
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/ProcessCommandFactory.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+import org.apache.hadoop.util.Shell;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * A source of commands, with the goal being to allow for adding different
+ * implementations for different platforms
+ */
+public class ProcessCommandFactory {
+
+  protected ProcessCommandFactory() {
+  }
+
+  /**
+   * The command to list a directory
+   * @param dir directory
+   * @return commands
+   */
+  public List<String> ls(File dir) {
+    List<String> commands;
+    if (!Shell.WINDOWS) {
+      commands = Arrays.asList("ls","-1", dir.getAbsolutePath());
+    } else {
+      commands = Arrays.asList("cmd", "/c", "dir", dir.getAbsolutePath());
+    }
+    return commands;
+  }
+
+  /**
+   * Echo some text to stdout
+   * @param text text
+   * @return commands
+   */
+  public List<String> echo(String text) {
+    List<String> commands = new ArrayList<String>(5);
+    commands.add("echo");
+    commands.add(text);
+    return commands;
+  }
+
+  /**
+   * print env variables
+   * @return commands
+   */
+  public List<String> env() {
+    List<String> commands;
+    if (!Shell.WINDOWS) {
+      commands = Arrays.asList("env");
+    } else {
+      commands = Arrays.asList("cmd", "/c", "set");
+    }
+    return commands;
+  }
+
+  /**
+   * execute a command that returns with an error code that will
+   * be converted into a number
+   * @return commands
+   */
+  public List<String> exitFalse() {
+    List<String> commands = new ArrayList<String>(2);
+    commands.add("false");
+    return commands;
+  }
+
+  /**
+   * Create a process command factory for this OS
+   * @return
+   */
+  public static ProcessCommandFactory createProcessCommandFactory() {
+    return new ProcessCommandFactory();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/SimpleRunnable.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/SimpleRunnable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/SimpleRunnable.java
new file mode 100644
index 0000000..1f330f4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/SimpleRunnable.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+/**
+ * Test runnable that can be made to exit, or throw an exception
+ * during its run
+ */
+class SimpleRunnable implements Runnable {
+  boolean throwException = false;
+
+
+  SimpleRunnable() {
+  }
+
+  SimpleRunnable(boolean throwException) {
+    this.throwException = throwException;
+  }
+
+  @Override
+  public synchronized void run() {
+    try {
+      if (throwException) {
+        throw new RuntimeException("SimpleRunnable");
+      }
+    } finally {
+      this.notify();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowClosingService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowClosingService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowClosingService.java
new file mode 100644
index 0000000..39516b7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowClosingService.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+public class TestWorkflowClosingService extends WorkflowServiceTestBase {
+
+  @Test
+  public void testSimpleClose() throws Throwable {
+    ClosingService<OpenClose> svc = instance(false);
+    OpenClose openClose = svc.getCloseable();
+    assertFalse(openClose.closed);
+    svc.stop();
+    assertTrue(openClose.closed);
+  }
+
+  @Test
+  public void testNullClose() throws Throwable {
+    ClosingService<OpenClose> svc = new ClosingService<OpenClose>("", null);
+    svc.init(new Configuration());
+    svc.start();
+    assertNull(svc.getCloseable());
+    svc.stop();
+  }
+
+  @Test
+  public void testFailingClose() throws Throwable {
+    ClosingService<OpenClose> svc = instance(false);
+    OpenClose openClose = svc.getCloseable();
+    openClose.raiseExceptionOnClose = true;
+    svc.stop();
+    assertTrue(openClose.closed);
+    Throwable cause = svc.getFailureCause();
+    assertNotNull(cause);
+
+    //retry should be a no-op
+    svc.close();
+  }
+
+  @Test
+  public void testDoubleClose() throws Throwable {
+    ClosingService<OpenClose> svc = instance(false);
+    OpenClose openClose = svc.getCloseable();
+    openClose.raiseExceptionOnClose = true;
+    svc.stop();
+    assertTrue(openClose.closed);
+    Throwable cause = svc.getFailureCause();
+    assertNotNull(cause);
+    openClose.closed = false;
+    svc.stop();
+    assertEquals(cause, svc.getFailureCause());
+  }
+
+  /**
+   * This does not recurse forever, as the service has already entered the
+   * STOPPED state before the inner close tries to stop it -that operation
+   * is a no-op
+   * @throws Throwable
+   */
+  @Test
+  public void testCloseSelf() throws Throwable {
+    ClosingService<ClosingService> svc =
+        new ClosingService<ClosingService>("");
+    svc.setCloseable(svc);
+    svc.stop();
+  }
+
+
+  private ClosingService<OpenClose> instance(boolean raiseExceptionOnClose) {
+    ClosingService<OpenClose> svc = new ClosingService<OpenClose>(new OpenClose(
+        raiseExceptionOnClose));
+    svc.init(new Configuration());
+    svc.start();
+    return svc;
+  }
+
+  private static class OpenClose implements Closeable {
+    public boolean closed = false;
+    public boolean raiseExceptionOnClose;
+
+    private OpenClose(boolean raiseExceptionOnClose) {
+      this.raiseExceptionOnClose = raiseExceptionOnClose;
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (!closed) {
+        closed = true;
+        if (raiseExceptionOnClose) {
+          throw new IOException("OpenClose");
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowCompositeService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowCompositeService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowCompositeService.java
new file mode 100644
index 0000000..5780149
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowCompositeService.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.Service;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestWorkflowCompositeService extends ParentWorkflowTestBase {
+  private static final Logger
+      log = LoggerFactory.getLogger(TestWorkflowCompositeService.class);
+
+  @Test
+  public void testSingleChild() throws Throwable {
+    Service parent = startService(new MockService());
+    parent.stop();
+  }
+
+  @Test
+  public void testSingleChildTerminating() throws Throwable {
+    ServiceParent parent =
+        startService(new MockService("1", false, 100));
+    waitForParentToStop(parent);
+  }
+
+  @Test
+  public void testSingleChildFailing() throws Throwable {
+    ServiceParent parent =
+        startService(new MockService("1", true, 100));
+    waitForParentToStop(parent);
+    assert parent.getFailureCause() != null;
+  }
+
+  @Test
+  public void testTwoChildren() throws Throwable {
+    MockService one = new MockService("one", false, 100);
+    MockService two = new MockService("two", false, 100);
+    ServiceParent parent = startService(one, two);
+    waitForParentToStop(parent);
+    assertStopped(one);
+    assertStopped(two);
+  }
+
+  @Test
+  public void testCallableChild() throws Throwable {
+
+    MockService one = new MockService("one", false, 100);
+    CallableHandler handler = new CallableHandler("hello");
+    WorkflowCallbackService<String> ens =
+        new WorkflowCallbackService<String>("handler", handler, 100, true);
+    MockService two = new MockService("two", false, 100);
+    ServiceParent parent = startService(one, ens, two);
+    waitForParentToStop(parent);
+    assertStopped(one);
+    assertStopped(ens);
+    assertStopped(two);
+    assertTrue(handler.notified);
+    String s = ens.getScheduledFuture().get();
+    assertEquals("hello", s);
+  }
+
+  @Test
+  public void testNestedComposite() throws Throwable {
+    MockService one = new MockService("one", false, 100);
+    MockService two = new MockService("two", false, 100);
+    ServiceParent parent = buildService(one, two);
+    ServiceParent outer = startService(parent);
+    assertTrue(outer.waitForServiceToStop(1000));
+    assertStopped(one);
+    assertStopped(two);
+  }
+
+  @Test
+  public void testFailingComposite() throws Throwable {
+    MockService one = new MockService("one", true, 10);
+    MockService two = new MockService("two", false, 1000);
+    ServiceParent parent = startService(one, two);
+    waitForParentToStop(parent);
+    assertStopped(one);
+    assertStopped(two);
+    assertNotNull(one.getFailureCause());
+    assertNotNull(parent.getFailureCause());
+    assertEquals(one.getFailureCause(), parent.getFailureCause());
+  }
+
+  @Override
+  public ServiceParent buildService(Service... services) {
+    ServiceParent parent =
+        new WorkflowCompositeService("test", services);
+    parent.init(new Configuration());
+    return parent;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowExecutorService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowExecutorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowExecutorService.java
new file mode 100644
index 0000000..dc160d9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowExecutorService.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+import org.junit.Test;
+
+import java.util.concurrent.ExecutorService;
+
+
+/**
+ * Basic tests for executor service
+ */
+public class TestWorkflowExecutorService extends WorkflowServiceTestBase {
+
+  @Test
+  public void testAsyncRun() throws Throwable {
+
+    ExecutorSvc svc = run(new ExecutorSvc());
+    ServiceTerminatingRunnable runnable = new ServiceTerminatingRunnable(svc,
+        new SimpleRunnable());
+
+    // synchronous in-thread execution
+    svc.execute(runnable);
+    Thread.sleep(1000);
+    assertStopped(svc);
+  }
+
+  @Test
+  public void testFailureRun() throws Throwable {
+
+    ExecutorSvc svc = run(new ExecutorSvc());
+    ServiceTerminatingRunnable runnable = new ServiceTerminatingRunnable(svc,
+        new SimpleRunnable(true));
+
+    // synchronous in-thread execution
+    svc.execute(runnable);
+    Thread.sleep(1000);
+    assertStopped(svc);
+    assertNotNull(runnable.getException());
+  }
+
+  private static class ExecutorSvc
+      extends WorkflowExecutorService<ExecutorService> {
+    private ExecutorSvc() {
+      super("ExecutorService",
+          ServiceThreadFactory.singleThreadExecutor("test", true));
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowRpcService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowRpcService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowRpcService.java
new file mode 100644
index 0000000..38cd9e1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowRpcService.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.Server;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+public class TestWorkflowRpcService extends WorkflowServiceTestBase {
+
+  @Test
+  public void testCreateMockRPCService() throws Throwable {
+    MockRPC rpc = new MockRPC();
+    rpc.start();
+    assertTrue(rpc.started);
+    rpc.getListenerAddress();
+    rpc.stop();
+    assertTrue(rpc.stopped);
+  }
+
+  @Test
+  public void testLifecycle() throws Throwable {
+    MockRPC rpc = new MockRPC();
+    WorkflowRpcService svc = new WorkflowRpcService("test", rpc);
+    run(svc);
+    assertTrue(rpc.started);
+    svc.getConnectAddress();
+    svc.stop();
+    assertTrue(rpc.stopped);
+  }
+
+  @Test
+  public void testStartFailure() throws Throwable {
+    MockRPC rpc = new MockRPC();
+    rpc.failOnStart = true;
+    WorkflowRpcService svc = new WorkflowRpcService("test", rpc);
+    svc.init(new Configuration());
+    try {
+      svc.start();
+      fail("expected an exception");
+    } catch (RuntimeException e) {
+      assertEquals("failOnStart", e.getMessage());
+    }
+    svc.stop();
+    assertTrue(rpc.stopped);
+  }
+
+  private static class MockRPC extends Server {
+
+    public boolean stopped;
+    public boolean started;
+    public boolean failOnStart;
+
+    private MockRPC() throws IOException {
+      super("localhost", 0, null, 1, new Configuration());
+    }
+
+    @Override
+    public synchronized void start() {
+      if (failOnStart) {
+        throw new RuntimeException("failOnStart");
+      }
+      started = true;
+      super.start();
+    }
+
+    @Override
+    public synchronized void stop() {
+      stopped = true;
+      super.stop();
+    }
+
+    @Override
+    public synchronized InetSocketAddress getListenerAddress() {
+      return super.getListenerAddress();
+    }
+
+    @Override
+    public Writable call(RPC.RpcKind rpcKind,
+        String protocol,
+        Writable param,
+        long receiveTime) throws Exception {
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowSequenceService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowSequenceService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowSequenceService.java
new file mode 100644
index 0000000..581e3ed
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowSequenceService.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.Service;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestWorkflowSequenceService extends ParentWorkflowTestBase {
+  private static final Logger
+      log = LoggerFactory.getLogger(TestWorkflowSequenceService.class);
+
+  @Test
+  public void testSingleSequence() throws Throwable {
+    ServiceParent parent = startService(new MockService());
+    parent.stop();
+  }
+
+  @Test
+  public void testEmptySequence() throws Throwable {
+    ServiceParent parent = startService();
+    waitForParentToStop(parent);
+  }
+
+  @Test
+  public void testSequence() throws Throwable {
+    MockService one = new MockService("one", false, 100);
+    MockService two = new MockService("two", false, 100);
+    ServiceParent parent = startService(one, two);
+    waitForParentToStop(parent);
+    assertStopped(one);
+    assertStopped(two);
+    assert ((WorkflowSequenceService) parent).getPreviousService().equals(two);
+  }
+
+  @Test
+  public void testCallableChild() throws Throwable {
+
+    MockService one = new MockService("one", false, 100);
+    CallableHandler handler = new CallableHandler("hello");
+    WorkflowCallbackService<String> ens =
+        new WorkflowCallbackService<String>("handler", handler, 100, true);
+    MockService two = new MockService("two", false, 100);
+    ServiceParent parent = startService(one, ens, two);
+    waitForParentToStop(parent);
+    assertStopped(one);
+    assertStopped(ens);
+    assertStopped(two);
+    assertTrue(handler.notified);
+    String s = ens.getScheduledFuture().get();
+    assertEquals("hello", s);
+  }
+
+
+  @Test
+  public void testFailingSequence() throws Throwable {
+    MockService one = new MockService("one", true, 100);
+    MockService two = new MockService("two", false, 100);
+    WorkflowSequenceService parent =
+        (WorkflowSequenceService) startService(one, two);
+    waitForParentToStop(parent);
+    assertStopped(one);
+    assertInState(two, Service.STATE.NOTINITED);
+    assertEquals(one, parent.getPreviousService());
+  }
+
+
+  @Test
+  public void testFailInStartNext() throws Throwable {
+    MockService one = new MockService("one", false, 100);
+    MockService two = new MockService("two", true, 0);
+    MockService three = new MockService("3", false, 0);
+    ServiceParent parent = startService(one, two, three);
+    waitForParentToStop(parent);
+    assertStopped(one);
+    assertStopped(two);
+    Throwable failureCause = two.getFailureCause();
+    assertNotNull(failureCause);
+    Throwable parentFailureCause = parent.getFailureCause();
+    assertNotNull(parentFailureCause);
+    assertEquals(parentFailureCause, failureCause);
+    assertInState(three, Service.STATE.NOTINITED);
+  }
+
+  @Test
+  public void testSequenceInSequence() throws Throwable {
+    MockService one = new MockService("one", false, 100);
+    MockService two = new MockService("two", false, 100);
+    ServiceParent parent = buildService(one, two);
+    ServiceParent outer = startService(parent);
+    waitForParentToStop(parent);
+    assertStopped(one);
+    assertStopped(two);
+  }
+
+  @Test
+  public void testVarargsConstructor() throws Throwable {
+    MockService one = new MockService("one", false, 100);
+    MockService two = new MockService("two", false, 100);
+    ServiceParent parent = new WorkflowSequenceService("test", one, two);
+    parent.init(new Configuration());
+    parent.start();
+    waitForParentToStop(parent);
+    assertStopped(one);
+    assertStopped(two);
+  }
+
+
+  @Test
+  public void testAddChild() throws Throwable {
+    MockService one = new MockService("one", false, 5000);
+    MockService two = new MockService("two", false, 100);
+    ServiceParent parent = startService(one, two);
+    CallableHandler handler = new CallableHandler("hello");
+    WorkflowCallbackService<String> ens =
+        new WorkflowCallbackService<String>("handler", handler, 100, true);
+    parent.addService(ens);
+    waitForParentToStop(parent, 10000);
+    assertStopped(one);
+    assertStopped(two);
+    assertStopped(ens);
+    assertStopped(two);
+    assertEquals("hello", ens.getScheduledFuture().get());
+  }
+
+  public WorkflowSequenceService buildService(Service... services) {
+    WorkflowSequenceService parent =
+        new WorkflowSequenceService("test", services);
+    parent.init(new Configuration());
+    return parent;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowServiceTerminatingRunnable.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowServiceTerminatingRunnable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowServiceTerminatingRunnable.java
new file mode 100644
index 0000000..5b7a6f9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/TestWorkflowServiceTerminatingRunnable.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+import org.junit.Test;
+
+
+public class TestWorkflowServiceTerminatingRunnable extends WorkflowServiceTestBase {
+
+  @Test
+  public void testNoservice() throws Throwable {
+
+    try {
+      new ServiceTerminatingRunnable(null, new SimpleRunnable());
+      fail("unexpected ");
+    } catch (IllegalArgumentException e) {
+
+      // expected
+    }
+  }
+
+
+  @Test
+  public void testBasicRun() throws Throwable {
+
+    WorkflowCompositeService svc = run(new WorkflowCompositeService());
+    ServiceTerminatingRunnable runnable = new ServiceTerminatingRunnable(svc,
+        new SimpleRunnable());
+
+    // synchronous in-thread execution
+    runnable.run();
+    assertStopped(svc);
+  }
+
+  @Test
+  public void testFailureRun() throws Throwable {
+
+    WorkflowCompositeService svc = run(new WorkflowCompositeService());
+    ServiceTerminatingRunnable runnable =
+        new ServiceTerminatingRunnable(svc, new SimpleRunnable(true));
+
+    // synchronous in-thread execution
+    runnable.run();
+    assertStopped(svc);
+    assertNotNull(runnable.getException());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/WorkflowServiceTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/WorkflowServiceTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/WorkflowServiceTestBase.java
new file mode 100644
index 0000000..f38bd9d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/services/workflow/WorkflowServiceTestBase.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.Service;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.rules.TestName;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.concurrent.Callable;
+
+/**
+ * Test base for workflow service tests.
+ */
+public abstract class WorkflowServiceTestBase extends Assert {
+  private static final Logger
+      log = LoggerFactory.getLogger(WorkflowServiceTestBase.class);
+
+  /**
+   * Set the timeout for every test
+   */
+  @Rule
+  public Timeout testTimeout = new Timeout(15000);
+
+  @Rule
+  public TestName name = new TestName();
+
+  @Before
+  public void nameThread() {
+    Thread.currentThread().setName("JUnit");
+  }
+
+
+  protected void assertInState(Service service, Service.STATE expected) {
+    Service.STATE actual = service.getServiceState();
+    if (actual != expected) {
+      fail("Service " + service.getName() + " in state " + actual
+           + " -expected " + expected);
+    }
+  }
+
+  protected void assertStopped(Service service) {
+    assertInState(service, Service.STATE.STOPPED);
+  }
+
+  protected void logState(ServiceParent p) {
+    logService(p);
+    for (Service s : p.getServices()) {
+      logService(s);
+    }
+  }
+
+  protected void logService(Service s) {
+    log.info(s.toString());
+    Throwable failureCause = s.getFailureCause();
+    if (failureCause != null) {
+      log.info("Failed in state {} with {}", s.getFailureState(),
+          failureCause);
+    }
+  }
+
+  /**
+   * Init and start a service
+   * @param svc the service
+   * @return the service
+   */
+  protected <S extends Service> S run(S svc) {
+    svc.init(new Configuration());
+    svc.start();
+    return svc;
+  }
+
+  /**
+   * Handler for callable events
+   */
+  public static class CallableHandler implements Callable<String> {
+    public volatile boolean notified = false;
+    public final String result;
+
+    public CallableHandler(String result) {
+      this.result = result;
+    }
+
+    @Override
+    public String call() throws Exception {
+      log.info("CallableHandler::call");
+      notified = true;
+      return result;
+    }
+  }
+
+  /**
+   * Assert that a string is in an output list. Fails fast if the output
+   * list is empty
+   * @param text text to scan for
+   * @param output list of output lines.
+   */
+  public void assertStringInOutput(String text, List<String> output) {
+    assertTrue("Empty output list", !output.isEmpty());
+    boolean found = false;
+    StringBuilder builder = new StringBuilder();
+    for (String s : output) {
+      builder.append(s.toLowerCase(Locale.ENGLISH)).append('\n');
+      if (s.contains(text)) {
+        found = true;
+        break;
+      }
+    }
+
+    if (!found) {
+      String message =
+          "Text \"" + text + "\" not found in " + output.size() + " lines\n";
+      fail(message + builder.toString());
+    }
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[66/76] [abbrv] hadoop git commit: YARN-5944. Native services AM should remain up if RM is down. Contributed by Billie Rinaldi

Posted by ji...@apache.org.
YARN-5944. Native services AM should remain up if RM is down. Contributed by Billie Rinaldi


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

Branch: refs/heads/yarn-native-services
Commit: 0c1ad96b9eee1811c3cc95bd2224bbb27f625c2f
Parents: 4f3033b
Author: Gour Saha <go...@apache.org>
Authored: Thu Dec 1 00:30:01 2016 -0800
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 .../org/apache/slider/server/appmaster/SliderAppMaster.java     | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0c1ad96b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
index 34b6a7d..8c39343 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
@@ -705,6 +705,11 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     synchronized (appState) {
       int heartbeatInterval = HEARTBEAT_INTERVAL;
 
+      // configure AM to wait forever for RM
+      getConfig().setLong(YarnConfiguration.RESOURCEMANAGER_CONNECT_MAX_WAIT_MS,
+          -1);
+      getConfig().unset(YarnConfiguration.CLIENT_FAILOVER_MAX_ATTEMPTS);
+
       // add the RM client -this brings the callbacks in
       asyncRMClient = AMRMClientAsync.createAMRMClientAsync(heartbeatInterval, this);
       addService(asyncRMClient);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[11/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
new file mode 100644
index 0000000..16c2435
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
@@ -0,0 +1,2489 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricRegistry;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
+import org.apache.hadoop.yarn.client.api.AMRMClient;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.apache.slider.api.ClusterDescription;
+import org.apache.slider.api.ClusterDescriptionKeys;
+import org.apache.slider.api.ClusterDescriptionOperations;
+import org.apache.slider.api.ClusterNode;
+import org.apache.slider.api.InternalKeys;
+import org.apache.slider.api.ResourceKeys;
+import org.apache.slider.api.StatusKeys;
+import org.apache.slider.api.types.ApplicationLivenessInformation;
+import org.apache.slider.api.types.ComponentInformation;
+import org.apache.slider.api.types.RoleStatistics;
+import org.apache.slider.common.SliderExitCodes;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.tools.ConfigHelper;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTree;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.exceptions.BadClusterStateException;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.core.exceptions.ErrorStrings;
+import org.apache.slider.core.exceptions.NoSuchNodeException;
+import org.apache.slider.core.exceptions.SliderInternalStateException;
+import org.apache.slider.core.exceptions.TriggerClusterTeardownException;
+import org.apache.slider.core.persist.AggregateConfSerDeser;
+import org.apache.slider.core.persist.ConfTreeSerDeser;
+import org.apache.slider.providers.PlacementPolicy;
+import org.apache.slider.providers.ProviderRole;
+import org.apache.slider.server.appmaster.management.LongGauge;
+import org.apache.slider.server.appmaster.management.MetricsAndMonitoring;
+import org.apache.slider.server.appmaster.management.MetricsConstants;
+import org.apache.slider.server.appmaster.operations.AbstractRMOperation;
+import org.apache.slider.server.appmaster.operations.ContainerReleaseOperation;
+import org.apache.slider.server.appmaster.operations.ContainerRequestOperation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.slider.api.ResourceKeys.*;
+import static org.apache.slider.api.RoleKeys.*;
+import static org.apache.slider.api.StateValues.*;
+
+/**
+ * The model of all the ongoing state of a Slider AM.
+ *
+ * concurrency rules: any method which begins with <i>build</i>
+ * is not synchronized and intended to be used during
+ * initialization.
+ */
+public class AppState {
+  protected static final Logger log =
+    LoggerFactory.getLogger(AppState.class);
+  
+  private final AbstractClusterServices recordFactory;
+
+  private final MetricsAndMonitoring metricsAndMonitoring;
+
+  /**
+   * Flag set to indicate the application is live -this only happens
+   * after the buildInstance operation
+   */
+  private boolean applicationLive = false;
+
+  /**
+   * The definition of the instance. Flexing updates the resources section
+   * This is used as a synchronization point on activities that update
+   * the CD, and also to update some of the structures that
+   * feed in to the CD
+   */
+  private AggregateConf instanceDefinition;
+
+  /**
+   * Time the instance definition snapshots were created
+   */
+  private long snapshotTime;
+
+  /**
+   * Snapshot of the instance definition. This is fully
+   * resolved.
+   */
+  private AggregateConf instanceDefinitionSnapshot;
+
+  /**
+   * Snapshot of the raw instance definition; unresolved and
+   * without any patch of an AM into it.
+   */
+  private AggregateConf unresolvedInstanceDefinition;
+
+  /**
+   * snapshot of resources as of last update time
+   */
+  private ConfTreeOperations resourcesSnapshot;
+  private ConfTreeOperations appConfSnapshot;
+  private ConfTreeOperations internalsSnapshot;
+
+  /**
+   * This is the status, the live model
+   */
+  private ClusterDescription clusterStatus = new ClusterDescription();
+
+  /**
+   * Metadata provided by the AM for use in filling in status requests
+   */
+  private Map<String, String> applicationInfo;
+
+  /**
+   * Client properties created via the provider -static for the life
+   * of the application
+   */
+  private Map<String, String> clientProperties = new HashMap<>();
+
+  /**
+   * This is a template of the cluster status
+   */
+  private ClusterDescription clusterStatusTemplate = new ClusterDescription();
+
+  private final Map<Integer, RoleStatus> roleStatusMap =
+    new ConcurrentSkipListMap<>();
+
+  private final Map<String, ProviderRole> roles =
+    new ConcurrentHashMap<>();
+
+  private final ConcurrentSkipListMap<Integer, ProviderRole> rolePriorityMap =
+    new ConcurrentSkipListMap<>();
+
+  /**
+   * The master node.
+   */
+  private RoleInstance appMasterNode;
+
+  /**
+   * Hash map of the containers we have. This includes things that have
+   * been allocated but are not live; it is a superset of the live list
+   */
+  private final ConcurrentMap<ContainerId, RoleInstance> ownedContainers =
+    new ConcurrentHashMap<>();
+
+  /**
+   * Hash map of the containers we have released, but we
+   * are still awaiting acknowledgements on. Any failure of these
+   * containers is treated as a successful outcome
+   */
+  private final ConcurrentMap<ContainerId, Container> containersBeingReleased =
+    new ConcurrentHashMap<>();
+  
+  /**
+   * Counter for completed containers ( complete denotes successful or failed )
+   */
+  private final LongGauge completedContainerCount = new LongGauge();
+
+  /**
+   *   Count of failed containers
+   */
+  private final LongGauge failedContainerCount = new LongGauge();
+
+  /**
+   * # of started containers
+   */
+  private final LongGauge startedContainers = new LongGauge();
+
+  /**
+   * # of containers that failed to start 
+   */
+  private final LongGauge startFailedContainerCount = new LongGauge();
+
+  /**
+   * Track the number of surplus containers received and discarded
+   */
+  private final LongGauge surplusContainers = new LongGauge();
+
+  /**
+   * Track the number of requested containers.
+   * Important: this does not include AA requests which are yet to be issued.
+   */
+  private final LongGauge outstandingContainerRequests = new LongGauge();
+
+  /**
+   * Map of requested nodes. This records the command used to start it,
+   * resources, etc. When container started callback is received,
+   * the node is promoted from here to the containerMap
+   */
+  private final Map<ContainerId, RoleInstance> startingContainers =
+    new ConcurrentHashMap<>();
+
+  /**
+   * List of completed nodes. This isn't kept in the CD as it gets too
+   * big for the RPC responses. Indeed, we should think about how deep to get this
+   */
+  private final Map<ContainerId, RoleInstance> completedContainers
+    = new ConcurrentHashMap<>();
+
+  /**
+   * Nodes that failed to start.
+   * Again, kept out of the CD
+   */
+  private final Map<ContainerId, RoleInstance> failedContainers =
+    new ConcurrentHashMap<>();
+
+  /**
+   * Nodes that came assigned to a role above that
+   * which were asked for -this appears to happen
+   */
+  private final Set<ContainerId> surplusNodes = new HashSet<>();
+
+  /**
+   * Map of containerID to cluster nodes, for status reports.
+   * Access to this should be synchronized on the clusterDescription
+   */
+  private final Map<ContainerId, RoleInstance> liveNodes =
+    new ConcurrentHashMap<>();
+  private final AtomicInteger completionOfNodeNotInLiveListEvent =
+    new AtomicInteger();
+  private final AtomicInteger completionOfUnknownContainerEvent =
+    new AtomicInteger();
+
+
+  /**
+   * limits of container core numbers in this queue
+   */
+  private int containerMaxCores;
+  private int containerMinCores;
+
+  /**
+   * limits of container memory in this queue
+   */
+  private int containerMaxMemory;
+  private int containerMinMemory;
+
+  private RoleHistory roleHistory;
+  private Configuration publishedProviderConf;
+  private long startTimeThreshold;
+
+  private int failureThreshold = 10;
+  private int nodeFailureThreshold = 3;
+
+  private String logServerURL = "";
+
+  /**
+   * Selector of containers to release; application wide.
+   */
+  private ContainerReleaseSelector containerReleaseSelector;
+  private Resource minResource;
+  private Resource maxResource;
+
+  /**
+   * Create an instance
+   * @param recordFactory factory for YARN records
+   * @param metricsAndMonitoring metrics and monitoring services
+   */
+  public AppState(AbstractClusterServices recordFactory,
+      MetricsAndMonitoring metricsAndMonitoring) {
+    Preconditions.checkArgument(recordFactory != null, "null recordFactory");
+    Preconditions.checkArgument(metricsAndMonitoring != null, "null metricsAndMonitoring");
+    this.recordFactory = recordFactory;
+    this.metricsAndMonitoring = metricsAndMonitoring;
+
+    // register any metrics
+    register(MetricsConstants.CONTAINERS_OUTSTANDING_REQUESTS, outstandingContainerRequests);
+    register(MetricsConstants.CONTAINERS_SURPLUS, surplusContainers);
+    register(MetricsConstants.CONTAINERS_STARTED, startedContainers);
+    register(MetricsConstants.CONTAINERS_COMPLETED, completedContainerCount);
+    register(MetricsConstants.CONTAINERS_FAILED, failedContainerCount);
+    register(MetricsConstants.CONTAINERS_START_FAILED, startFailedContainerCount);
+  }
+
+  private void register(String name, Metric counter) {
+    this.metricsAndMonitoring.getMetrics().register(
+        MetricRegistry.name(AppState.class, name), counter);
+  }
+
+  public long getFailedCountainerCount() {
+    return failedContainerCount.getCount();
+  }
+
+  /**
+   * Increment the count
+   */
+  public void incFailedCountainerCount() {
+    failedContainerCount.inc();
+  }
+
+  public long getStartFailedCountainerCount() {
+    return startFailedContainerCount.getCount();
+  }
+
+  /**
+   * Increment the count and return the new value
+   */
+  public void incStartedCountainerCount() {
+    startedContainers.inc();
+  }
+
+  public long getStartedCountainerCount() {
+    return startedContainers.getCount();
+  }
+
+  /**
+   * Increment the count and return the new value
+   */
+  public void incStartFailedCountainerCount() {
+    startFailedContainerCount.inc();
+  }
+
+  public AtomicInteger getCompletionOfNodeNotInLiveListEvent() {
+    return completionOfNodeNotInLiveListEvent;
+  }
+
+  public AtomicInteger getCompletionOfUnknownContainerEvent() {
+    return completionOfUnknownContainerEvent;
+  }
+
+
+  public Map<Integer, RoleStatus> getRoleStatusMap() {
+    return roleStatusMap;
+  }
+  
+  protected Map<String, ProviderRole> getRoleMap() {
+    return roles;
+  }
+
+  public Map<Integer, ProviderRole> getRolePriorityMap() {
+    return rolePriorityMap;
+  }
+
+  private Map<ContainerId, RoleInstance> getStartingContainers() {
+    return startingContainers;
+  }
+
+  private Map<ContainerId, RoleInstance> getCompletedContainers() {
+    return completedContainers;
+  }
+
+  public Map<ContainerId, RoleInstance> getFailedContainers() {
+    return failedContainers;
+  }
+
+  public Map<ContainerId, RoleInstance> getLiveContainers() {
+    return liveNodes;
+  }
+
+  /**
+   * Get the current view of the cluster status.
+   * <p>
+   *   Calls to {@link #refreshClusterStatus()} trigger a
+   *   refresh of this field.
+   * <p>
+   * This is read-only
+   * to the extent that changes here do not trigger updates in the
+   * application state. 
+   * @return the cluster status
+   */
+  public synchronized ClusterDescription getClusterStatus() {
+    return clusterStatus;
+  }
+
+  @VisibleForTesting
+  protected synchronized void setClusterStatus(ClusterDescription clusterDesc) {
+    this.clusterStatus = clusterDesc;
+  }
+
+  /**
+   * Set the instance definition -this also builds the (now obsolete)
+   * cluster specification from it.
+   * 
+   * Important: this is for early binding and must not be used after the build
+   * operation is complete. 
+   * @param definition initial definition
+   * @throws BadConfigException
+   */
+  public synchronized void setInitialInstanceDefinition(AggregateConf definition)
+      throws BadConfigException, IOException {
+    log.debug("Setting initial instance definition");
+    // snapshot the definition
+    AggregateConfSerDeser serDeser = new AggregateConfSerDeser();
+
+    unresolvedInstanceDefinition = serDeser.fromInstance(definition);
+    
+    this.instanceDefinition = serDeser.fromInstance(definition);
+    onInstanceDefinitionUpdated();
+  }
+
+  public synchronized AggregateConf getInstanceDefinition() {
+    return instanceDefinition;
+  }
+
+  /**
+   * Get the role history of the application
+   * @return the role history
+   */
+  @VisibleForTesting
+  public RoleHistory getRoleHistory() {
+    return roleHistory;
+  }
+
+  /**
+   * Get the path used for history files
+   * @return the directory used for history files
+   */
+  @VisibleForTesting
+  public Path getHistoryPath() {
+    return roleHistory.getHistoryPath();
+  }
+
+  /**
+   * Set the container limits -the min and max values for
+   * resource requests. All requests must be multiples of the min
+   * values.
+   * @param minMemory min memory MB
+   * @param maxMemory maximum memory
+   * @param minCores min v core count
+   * @param maxCores maximum cores
+   */
+  public void setContainerLimits(int minMemory,int maxMemory, int minCores, int maxCores) {
+    containerMinCores = minCores;
+    containerMaxCores = maxCores;
+    containerMinMemory = minMemory;
+    containerMaxMemory = maxMemory;
+    minResource = recordFactory.newResource(containerMinMemory, containerMinCores);
+    maxResource = recordFactory.newResource(containerMaxMemory, containerMaxCores);
+  }
+
+  public ConfTreeOperations getResourcesSnapshot() {
+    return resourcesSnapshot;
+  }
+
+  public ConfTreeOperations getAppConfSnapshot() {
+    return appConfSnapshot;
+  }
+
+  public ConfTreeOperations getInternalsSnapshot() {
+    return internalsSnapshot;
+  }
+
+  public boolean isApplicationLive() {
+    return applicationLive;
+  }
+
+  public long getSnapshotTime() {
+    return snapshotTime;
+  }
+
+  public synchronized AggregateConf getInstanceDefinitionSnapshot() {
+    return instanceDefinitionSnapshot;
+  }
+
+  public AggregateConf getUnresolvedInstanceDefinition() {
+    return unresolvedInstanceDefinition;
+  }
+
+  public synchronized void buildInstance(AppStateBindingInfo binding)
+      throws BadClusterStateException, BadConfigException, IOException {
+    binding.validate();
+
+    log.debug("Building application state");
+    publishedProviderConf = binding.publishedProviderConf;
+    applicationInfo = binding.applicationInfo != null ? binding.applicationInfo
+                        : new HashMap<String, String>();
+
+    clientProperties = new HashMap<>();
+    containerReleaseSelector = binding.releaseSelector;
+
+
+    Set<String> confKeys = ConfigHelper.sortedConfigKeys(publishedProviderConf);
+
+    //  Add the -site configuration properties
+    for (String key : confKeys) {
+      String val = publishedProviderConf.get(key);
+      clientProperties.put(key, val);
+    }
+
+    // set the cluster specification (once its dependency the client properties
+    // is out the way
+    setInitialInstanceDefinition(binding.instanceDefinition);
+
+    //build the initial role list
+    List<ProviderRole> roleList = new ArrayList<>(binding.roles);
+    for (ProviderRole providerRole : roleList) {
+      buildRole(providerRole);
+    }
+
+    ConfTreeOperations resources = instanceDefinition.getResourceOperations();
+
+    Set<String> roleNames = resources.getComponentNames();
+    for (String name : roleNames) {
+      if (roles.containsKey(name)) {
+        continue;
+      }
+      if (hasUniqueNames(resources, name)) {
+        log.info("Skipping group {}", name);
+        continue;
+      }
+      // this is a new value
+      log.info("Adding role {}", name);
+      MapOperations resComponent = resources.getComponent(name);
+      ProviderRole dynamicRole = createDynamicProviderRole(name, resComponent);
+      buildRole(dynamicRole);
+      roleList.add(dynamicRole);
+    }
+    //then pick up the requirements
+    buildRoleRequirementsFromResources();
+
+    //set the livespan
+    MapOperations globalResOpts = instanceDefinition.getResourceOperations().getGlobalOptions();
+
+    startTimeThreshold = globalResOpts.getOptionInt(
+        InternalKeys.INTERNAL_CONTAINER_FAILURE_SHORTLIFE,
+        InternalKeys.DEFAULT_INTERNAL_CONTAINER_FAILURE_SHORTLIFE);
+
+    failureThreshold = globalResOpts.getOptionInt(
+        CONTAINER_FAILURE_THRESHOLD,
+        DEFAULT_CONTAINER_FAILURE_THRESHOLD);
+    nodeFailureThreshold = globalResOpts.getOptionInt(
+        NODE_FAILURE_THRESHOLD,
+        DEFAULT_NODE_FAILURE_THRESHOLD);
+    initClusterStatus();
+
+
+    // set up the role history
+    roleHistory = new RoleHistory(roleStatusMap.values(), recordFactory);
+    roleHistory.register(metricsAndMonitoring);
+    roleHistory.onStart(binding.fs, binding.historyPath);
+    // trigger first node update
+    roleHistory.onNodesUpdated(binding.nodeReports);
+
+
+    //rebuild any live containers
+    rebuildModelFromRestart(binding.liveContainers);
+
+    // any am config options to pick up
+    logServerURL = binding.serviceConfig.get(YarnConfiguration.YARN_LOG_SERVER_URL, "");
+    //mark as live
+    applicationLive = true;
+  }
+
+  public void initClusterStatus() {
+    //copy into cluster status. 
+    ClusterDescription status = ClusterDescription.copy(clusterStatusTemplate);
+    status.state = STATE_CREATED;
+    MapOperations infoOps = new MapOperations("info", status.info);
+    infoOps.mergeWithoutOverwrite(applicationInfo);
+    SliderUtils.addBuildInfo(infoOps, "status");
+
+    long now = now();
+    status.setInfoTime(StatusKeys.INFO_LIVE_TIME_HUMAN,
+                              StatusKeys.INFO_LIVE_TIME_MILLIS,
+                              now);
+    SliderUtils.setInfoTime(infoOps,
+        StatusKeys.INFO_LIVE_TIME_HUMAN,
+        StatusKeys.INFO_LIVE_TIME_MILLIS,
+        now);
+    if (0 == status.createTime) {
+      status.createTime = now;
+      SliderUtils.setInfoTime(infoOps,
+          StatusKeys.INFO_CREATE_TIME_HUMAN,
+          StatusKeys.INFO_CREATE_TIME_MILLIS,
+          now);
+    }
+    status.state = STATE_LIVE;
+
+      //set the app state to this status
+    setClusterStatus(status);
+  }
+
+  /**
+   * Build a dynamic provider role
+   * @param name name of role
+   * @return a new provider role
+   * @throws BadConfigException bad configuration
+   */
+  public ProviderRole createDynamicProviderRole(String name, MapOperations component)
+      throws BadConfigException {
+    return createDynamicProviderRole(name, name, component);
+  }
+
+  /**
+   * Build a dynamic provider role
+   * @param name name of role
+   * @param group group of role
+   * @return a new provider role
+   * @throws BadConfigException bad configuration
+   */
+  public ProviderRole createDynamicProviderRole(String name, String group, MapOperations component)
+      throws BadConfigException {
+    String priOpt = component.getMandatoryOption(COMPONENT_PRIORITY);
+    int priority = SliderUtils.parseAndValidate(
+        "value of " + name + " " + COMPONENT_PRIORITY, priOpt, 0, 1, -1);
+
+    String placementOpt = component.getOption(COMPONENT_PLACEMENT_POLICY,
+        Integer.toString(PlacementPolicy.DEFAULT));
+
+    int placement = SliderUtils.parseAndValidate(
+        "value of " + name + " " + COMPONENT_PLACEMENT_POLICY, placementOpt, 0, 0, -1);
+
+    int placementTimeout = component.getOptionInt(PLACEMENT_ESCALATE_DELAY,
+            DEFAULT_PLACEMENT_ESCALATE_DELAY_SECONDS);
+
+    ProviderRole newRole = new ProviderRole(name,
+        group,
+        priority,
+        placement,
+        getNodeFailureThresholdForRole(group),
+        placementTimeout,
+        component.getOption(YARN_LABEL_EXPRESSION, DEF_YARN_LABEL_EXPRESSION));
+    log.info("New {} ", newRole);
+    return newRole;
+  }
+
+  /**
+   * Actions to perform when an instance definition is updated
+   * Currently: 
+   * <ol>
+   *   <li>
+   *     resolve the configuration
+   *   </li>
+   *   <li>
+   *     update the cluster spec derivative
+   *   </li>
+   * </ol>
+   *  
+   * @throws BadConfigException
+   */
+  private synchronized void onInstanceDefinitionUpdated()
+      throws BadConfigException, IOException {
+
+    log.debug("Instance definition updated");
+    //note the time 
+    snapshotTime = now();
+
+    for (String component : instanceDefinition.getResourceOperations().getComponentNames()) {
+      instanceDefinition.getAppConfOperations().getOrAddComponent(component);
+    }
+
+    // resolve references if not already done
+    instanceDefinition.resolve();
+
+    // force in the AM desired state values
+    ConfTreeOperations resources = instanceDefinition.getResourceOperations();
+
+    if (resources.getComponent(SliderKeys.COMPONENT_AM) != null) {
+      resources.setComponentOpt(
+          SliderKeys.COMPONENT_AM, COMPONENT_INSTANCES, "1");
+    }
+
+
+    //snapshot all three sectons
+    resourcesSnapshot = ConfTreeOperations.fromInstance(instanceDefinition.getResources());
+    appConfSnapshot = ConfTreeOperations.fromInstance(instanceDefinition.getAppConf());
+    internalsSnapshot = ConfTreeOperations.fromInstance(instanceDefinition.getInternal());
+    //build a new aggregate from the snapshots
+    instanceDefinitionSnapshot = new AggregateConf(resourcesSnapshot.confTree,
+                                                   appConfSnapshot.confTree,
+                                                   internalsSnapshot.confTree);
+    instanceDefinitionSnapshot.setName(instanceDefinition.getName());
+
+    clusterStatusTemplate = ClusterDescriptionOperations.buildFromInstanceDefinition(
+          instanceDefinition);
+
+    // Add the -site configuration properties
+    for (Map.Entry<String, String> prop : clientProperties.entrySet()) {
+      clusterStatusTemplate.clientProperties.put(prop.getKey(), prop.getValue());
+    }
+
+  }
+
+  /**
+   * The resource configuration is updated -review and update state.
+   * @param resources updated resources specification
+   * @return a list of any dynamically added provider roles
+   * (purely for testing purposes)
+   */
+  @VisibleForTesting
+  public synchronized List<ProviderRole> updateResourceDefinitions(ConfTree resources)
+      throws BadConfigException, IOException {
+    log.debug("Updating resources to {}", resources);
+    // snapshot the (possibly unresolved) values
+    ConfTreeSerDeser serDeser = new ConfTreeSerDeser();
+    unresolvedInstanceDefinition.setResources(
+        serDeser.fromInstance(resources));
+    // assign another copy under the instance definition for resolving
+    // and then driving application size
+    instanceDefinition.setResources(serDeser.fromInstance(resources));
+    onInstanceDefinitionUpdated();
+
+    // propagate the role table
+    Map<String, Map<String, String>> updated = resources.components;
+    getClusterStatus().roles = SliderUtils.deepClone(updated);
+    getClusterStatus().updateTime = now();
+    return buildRoleRequirementsFromResources();
+  }
+
+  /**
+   * build the role requirements from the cluster specification
+   * @return a list of any dynamically added provider roles
+   */
+  private List<ProviderRole> buildRoleRequirementsFromResources() throws BadConfigException {
+
+    List<ProviderRole> newRoles = new ArrayList<>(0);
+
+    // now update every role's desired count.
+    // if there are no instance values, that role count goes to zero
+
+    ConfTreeOperations resources =
+        instanceDefinition.getResourceOperations();
+
+    // Add all the existing roles
+    Map<String, Integer> groupCounts = new HashMap<>();
+    for (RoleStatus roleStatus : getRoleStatusMap().values()) {
+      if (roleStatus.isExcludeFromFlexing()) {
+        // skip inflexible roles, e.g AM itself
+        continue;
+      }
+      long currentDesired = roleStatus.getDesired();
+      String role = roleStatus.getName();
+      String roleGroup = roleStatus.getGroup();
+      int desiredInstanceCount = getDesiredInstanceCount(resources, roleGroup);
+
+      int newDesired = desiredInstanceCount;
+      if (hasUniqueNames(resources, roleGroup)) {
+        Integer groupCount = 0;
+        if (groupCounts.containsKey(roleGroup)) {
+          groupCount = groupCounts.get(roleGroup);
+        }
+
+        newDesired = desiredInstanceCount - groupCount;
+
+        if (newDesired > 0) {
+          newDesired = 1;
+          groupCounts.put(roleGroup, groupCount + newDesired);
+        } else {
+          newDesired = 0;
+        }
+      }
+
+      if (newDesired == 0) {
+        log.info("Role {} has 0 instances specified", role);
+      }
+      if (currentDesired != newDesired) {
+        log.info("Role {} flexed from {} to {}", role, currentDesired,
+            newDesired);
+        roleStatus.setDesired(newDesired);
+      }
+    }
+
+    // now the dynamic ones. Iterate through the the cluster spec and
+    // add any role status entries not in the role status
+    Set<String> roleNames = resources.getComponentNames();
+    for (String name : roleNames) {
+      if (roles.containsKey(name)) {
+        continue;
+      }
+      if (hasUniqueNames(resources, name)) {
+        // THIS NAME IS A GROUP
+        int desiredInstanceCount = getDesiredInstanceCount(resources, name);
+        Integer groupCount = 0;
+        if (groupCounts.containsKey(name)) {
+          groupCount = groupCounts.get(name);
+        }
+        for (int i = groupCount + 1; i <= desiredInstanceCount; i++) {
+          int priority = resources.getComponentOptInt(name, COMPONENT_PRIORITY, i);
+          // this is a new instance of an existing group
+          String newName = String.format("%s%d", name, i);
+          int newPriority = getNewPriority(priority + i - 1);
+          log.info("Adding new role {}", newName);
+          MapOperations component = resources.getComponent(name,
+              Collections.singletonMap(COMPONENT_PRIORITY,
+                  Integer.toString(newPriority)));
+          if (component == null) {
+            throw new BadConfigException("Component is null for name = " + name
+                + ", newPriority =" + newPriority);
+          }
+          ProviderRole dynamicRole = createDynamicProviderRole(newName, name, component);
+          RoleStatus roleStatus = buildRole(dynamicRole);
+          roleStatus.setDesired(1);
+          log.info("New role {}", roleStatus);
+          if (roleHistory != null) {
+            roleHistory.addNewRole(roleStatus);
+          }
+          newRoles.add(dynamicRole);
+        }
+      } else {
+        // this is a new value
+        log.info("Adding new role {}", name);
+        MapOperations component = resources.getComponent(name);
+        ProviderRole dynamicRole = createDynamicProviderRole(name, component);
+        RoleStatus roleStatus = buildRole(dynamicRole);
+        roleStatus.setDesired(getDesiredInstanceCount(resources, name));
+        log.info("New role {}", roleStatus);
+        if (roleHistory != null) {
+          roleHistory.addNewRole(roleStatus);
+        }
+        newRoles.add(dynamicRole);
+      }
+    }
+    // and fill in all those roles with their requirements
+    buildRoleResourceRequirements();
+
+    return newRoles;
+  }
+
+  private int getNewPriority(int start) {
+    if (!rolePriorityMap.containsKey(start)) {
+      return start;
+    }
+    return rolePriorityMap.lastKey() + 1;
+  }
+
+  /**
+   * Get the desired instance count of a role, rejecting negative values
+   * @param resources resource map
+   * @param roleGroup role group
+   * @return the instance count
+   * @throws BadConfigException if the count is negative
+   */
+  private int getDesiredInstanceCount(ConfTreeOperations resources,
+      String roleGroup) throws BadConfigException {
+    int desiredInstanceCount =
+      resources.getComponentOptInt(roleGroup, COMPONENT_INSTANCES, 0);
+
+    if (desiredInstanceCount < 0) {
+      log.error("Role {} has negative desired instances : {}", roleGroup,
+          desiredInstanceCount);
+      throw new BadConfigException(
+          "Negative instance count (%) requested for component %s",
+          desiredInstanceCount, roleGroup);
+    }
+    return desiredInstanceCount;
+  }
+
+  private Boolean hasUniqueNames(ConfTreeOperations resources, String group) {
+    MapOperations component = resources.getComponent(group);
+    if (component == null) {
+      log.info("Component was null for {} when checking unique names", group);
+      return Boolean.FALSE;
+    }
+    return component.getOptionBool(UNIQUE_NAMES, Boolean.FALSE);
+  }
+
+  /**
+   * Add knowledge of a role.
+   * This is a build-time operation that is not synchronized, and
+   * should be used while setting up the system state -before servicing
+   * requests.
+   * @param providerRole role to add
+   * @return the role status built up
+   * @throws BadConfigException if a role of that priority already exists
+   */
+  public RoleStatus buildRole(ProviderRole providerRole) throws BadConfigException {
+    // build role status map
+    int priority = providerRole.id;
+    if (roleStatusMap.containsKey(priority)) {
+      throw new BadConfigException("Duplicate Provider Key: %s and %s",
+                                   providerRole,
+                                   roleStatusMap.get(priority));
+    }
+    RoleStatus roleStatus = new RoleStatus(providerRole);
+    roleStatusMap.put(priority, roleStatus);
+    String name = providerRole.name;
+    roles.put(name, providerRole);
+    rolePriorityMap.put(priority, providerRole);
+    // register its entries
+    metricsAndMonitoring.addMetricSet(MetricsConstants.PREFIX_SLIDER_ROLES + name, roleStatus);
+    return roleStatus;
+  }
+
+  /**
+   * Build up the requirements of every resource
+   */
+  private void buildRoleResourceRequirements() {
+    for (RoleStatus role : roleStatusMap.values()) {
+      role.setResourceRequirements(
+          buildResourceRequirements(role, recordFactory.newResource()));
+    }
+  }
+
+  /**
+   * build up the special master node, which lives
+   * in the live node set but has a lifecycle bonded to the AM
+   * @param containerId the AM master
+   * @param host hostname
+   * @param amPort port
+   * @param nodeHttpAddress http address: may be null
+   */
+  public void buildAppMasterNode(ContainerId containerId,
+                                 String host,
+                                 int amPort,
+                                 String nodeHttpAddress) {
+    Container container = new ContainerPBImpl();
+    container.setId(containerId);
+    NodeId nodeId = NodeId.newInstance(host, amPort);
+    container.setNodeId(nodeId);
+    container.setNodeHttpAddress(nodeHttpAddress);
+    RoleInstance am = new RoleInstance(container);
+    am.role = SliderKeys.COMPONENT_AM;
+    am.group = SliderKeys.COMPONENT_AM;
+    am.roleId = SliderKeys.ROLE_AM_PRIORITY_INDEX;
+    am.createTime =now();
+    am.startTime = am.createTime;
+    appMasterNode = am;
+    //it is also added to the set of live nodes
+    getLiveContainers().put(containerId, am);
+    putOwnedContainer(containerId, am);
+
+    // patch up the role status
+    RoleStatus roleStatus = roleStatusMap.get(SliderKeys.ROLE_AM_PRIORITY_INDEX);
+    roleStatus.setDesired(1);
+    roleStatus.incActual();
+    roleStatus.incStarted();
+  }
+
+  /**
+   * Note that the master node has been launched,
+   * though it isn't considered live until any forked
+   * processes are running. It is NOT registered with
+   * the role history -the container is incomplete
+   * and it will just cause confusion
+   */
+  public void noteAMLaunched() {
+    getLiveContainers().put(appMasterNode.getContainerId(), appMasterNode);
+  }
+
+  /**
+   * AM declares ourselves live in the cluster description.
+   * This is meant to be triggered from the callback
+   * indicating the spawned process is up and running.
+   */
+  public void noteAMLive() {
+    appMasterNode.state = STATE_LIVE;
+  }
+
+  /**
+   * Look up the status entry of a role or raise an exception
+   * @param key role ID
+   * @return the status entry
+   * @throws RuntimeException if the role cannot be found
+   */
+  public RoleStatus lookupRoleStatus(int key) {
+    RoleStatus rs = getRoleStatusMap().get(key);
+    if (rs == null) {
+      throw new RuntimeException("Cannot find role for role ID " + key);
+    }
+    return rs;
+  }
+
+  /**
+   * Look up the status entry of a container or raise an exception
+   *
+   * @param c container
+   * @return the status entry
+   * @throws RuntimeException if the role cannot be found
+   */
+  public RoleStatus lookupRoleStatus(Container c) {
+    return lookupRoleStatus(ContainerPriority.extractRole(c));
+  }
+
+  /**
+   * Get a deep clone of the role status list. Concurrent events may mean this
+   * list (or indeed, some of the role status entries) may be inconsistent
+   * @return a snapshot of the role status entries
+   */
+  public List<RoleStatus> cloneRoleStatusList() {
+    Collection<RoleStatus> statuses = roleStatusMap.values();
+    List<RoleStatus> statusList = new ArrayList<>(statuses.size());
+    try {
+      for (RoleStatus status : statuses) {
+        statusList.add((RoleStatus)(status.clone()));
+      }
+    } catch (CloneNotSupportedException e) {
+      log.warn("Unexpected cloning failure: {}", e, e);
+    }
+    return statusList;
+  }
+
+
+  /**
+   * Look up a role in the map
+   * @param name role name
+   * @return the instance
+   * @throws YarnRuntimeException if not found
+   */
+  public RoleStatus lookupRoleStatus(String name) throws YarnRuntimeException {
+    ProviderRole providerRole = roles.get(name);
+    if (providerRole == null) {
+      throw new YarnRuntimeException("Unknown role " + name);
+    }
+    return lookupRoleStatus(providerRole.id);
+  }
+
+
+  /**
+   * Clone the list of active (==owned) containers
+   * @return the list of role instances representing all owned containers
+   */
+  public synchronized List<RoleInstance> cloneOwnedContainerList() {
+    Collection<RoleInstance> values = ownedContainers.values();
+    return new ArrayList<>(values);
+  }
+
+  /**
+   * Get the number of active (==owned) containers
+   * @return
+   */
+  public int getNumOwnedContainers() {
+    return ownedContainers.size();
+  }
+  
+  /**
+   * Look up an active container: any container that the AM has, even
+   * if it is not currently running/live
+   */
+  public RoleInstance getOwnedContainer(ContainerId id) {
+    return ownedContainers.get(id);
+  }
+
+  /**
+   * Remove an owned container
+   * @param id container ID
+   * @return the instance removed
+   */
+  private RoleInstance removeOwnedContainer(ContainerId id) {
+    return ownedContainers.remove(id);
+  }
+
+  /**
+   * set/update an owned container
+   * @param id container ID
+   * @param instance
+   * @return
+   */
+  private RoleInstance putOwnedContainer(ContainerId id,
+      RoleInstance instance) {
+    return ownedContainers.put(id, instance);
+  }
+
+  /**
+   * Clone the live container list. This is synchronized.
+   * @return a snapshot of the live node list
+   */
+  public synchronized List<RoleInstance> cloneLiveContainerInfoList() {
+    List<RoleInstance> allRoleInstances;
+    Collection<RoleInstance> values = getLiveContainers().values();
+    allRoleInstances = new ArrayList<>(values);
+    return allRoleInstances;
+  }
+
+  /**
+   * Lookup live instance by string value of container ID
+   * @param containerId container ID as a string
+   * @return the role instance for that container
+   * @throws NoSuchNodeException if it does not exist
+   */
+  public synchronized RoleInstance getLiveInstanceByContainerID(String containerId)
+      throws NoSuchNodeException {
+    Collection<RoleInstance> nodes = getLiveContainers().values();
+    return findNodeInCollection(containerId, nodes);
+  }
+
+  /**
+   * Lookup owned instance by string value of container ID
+   * @param containerId container ID as a string
+   * @return the role instance for that container
+   * @throws NoSuchNodeException if it does not exist
+   */
+  public synchronized RoleInstance getOwnedInstanceByContainerID(String containerId)
+      throws NoSuchNodeException {
+    Collection<RoleInstance> nodes = ownedContainers.values();
+    return findNodeInCollection(containerId, nodes);
+  }
+
+  /**
+   * Iterate through a collection of role instances to find one with a
+   * specific (string) container ID
+   * @param containerId container ID as a string
+   * @param nodes collection
+   * @return the found node 
+   * @throws NoSuchNodeException if there was no match
+   */
+  private RoleInstance findNodeInCollection(String containerId,
+      Collection<RoleInstance> nodes) throws NoSuchNodeException {
+    RoleInstance found = null;
+    for (RoleInstance node : nodes) {
+      if (containerId.equals(node.id)) {
+        found = node;
+        break;
+      }
+    }
+    if (found != null) {
+      return found;
+    } else {
+      //at this point: no node
+      throw new NoSuchNodeException("Unknown node: " + containerId);
+    }
+  }
+
+  public synchronized List<RoleInstance> getLiveInstancesByContainerIDs(
+    Collection<String> containerIDs) {
+    //first, a hashmap of those containerIDs is built up
+    Set<String> uuidSet = new HashSet<String>(containerIDs);
+    List<RoleInstance> nodes = new ArrayList<RoleInstance>(uuidSet.size());
+    Collection<RoleInstance> clusterNodes = getLiveContainers().values();
+
+    for (RoleInstance node : clusterNodes) {
+      if (uuidSet.contains(node.id)) {
+        nodes.add(node);
+      }
+    }
+    //at this point: a possibly empty list of nodes
+    return nodes;
+  }
+
+  /**
+   * Enum all nodes by role. 
+   * @param role role, or "" for all roles
+   * @return a list of nodes, may be empty
+   */
+  public synchronized List<RoleInstance> enumLiveNodesInRole(String role) {
+    List<RoleInstance> nodes = new ArrayList<RoleInstance>();
+    Collection<RoleInstance> allRoleInstances = getLiveContainers().values();
+    for (RoleInstance node : allRoleInstances) {
+      if (role.isEmpty() || role.equals(node.role)) {
+        nodes.add(node);
+      }
+    }
+    return nodes;
+  }
+
+ 
+  /**
+   * enum nodes by role ID, from either the owned or live node list
+   * @param roleId role the container must be in
+   * @param owned flag to indicate "use owned list" rather than the smaller
+   * "live" list
+   * @return a list of nodes, may be empty
+   */
+  public synchronized List<RoleInstance> enumNodesWithRoleId(int roleId,
+      boolean owned) {
+    List<RoleInstance> nodes = new ArrayList<RoleInstance>();
+    Collection<RoleInstance> allRoleInstances;
+    allRoleInstances = owned ? ownedContainers.values() : liveNodes.values();
+    for (RoleInstance node : allRoleInstances) {
+      if (node.roleId == roleId) {
+        nodes.add(node);
+      }
+    }
+    return nodes;
+  }
+
+  /**
+   * Build an instance map.
+   * @return the map of Role name to list of role instances
+   */
+  private synchronized Map<String, List<String>> createRoleToInstanceMap() {
+    Map<String, List<String>> map = new HashMap<String, List<String>>();
+    for (RoleInstance node : getLiveContainers().values()) {
+      List<String> containers = map.get(node.role);
+      if (containers == null) {
+        containers = new ArrayList<String>();
+        map.put(node.role, containers);
+      }
+      containers.add(node.id);
+    }
+    return map;
+  }
+
+  /**
+   * Build a map of role->nodename->node-info
+   * 
+   * @return the map of Role name to list of Cluster Nodes
+   */
+  public synchronized Map<String, Map<String, ClusterNode>> createRoleToClusterNodeMap() {
+    Map<String, Map<String, ClusterNode>> map = new HashMap<>();
+    for (RoleInstance node : getLiveContainers().values()) {
+      
+      Map<String, ClusterNode> containers = map.get(node.role);
+      if (containers == null) {
+        containers = new HashMap<String, ClusterNode>();
+        map.put(node.role, containers);
+      }
+      ClusterNode clusterNode = node.toClusterNode();
+      containers.put(clusterNode.name, clusterNode);
+    }
+    return map;
+  }
+
+  /**
+   * Notification called just before the NM is asked to 
+   * start a container
+   * @param container container to start
+   * @param instance clusterNode structure
+   */
+  public void containerStartSubmitted(Container container,
+                                      RoleInstance instance) {
+    instance.state = STATE_SUBMITTED;
+    instance.container = container;
+    instance.createTime = now();
+    getStartingContainers().put(container.getId(), instance);
+    putOwnedContainer(container.getId(), instance);
+    roleHistory.onContainerStartSubmitted(container, instance);
+  }
+
+  /**
+   * Note that a container has been submitted for release; update internal state
+   * and mark the associated ContainerInfo released field to indicate that
+   * while it is still in the active list, it has been queued for release.
+   *
+   * @param container container
+   * @throws SliderInternalStateException if there is no container of that ID
+   * on the active list
+   */
+  public synchronized void containerReleaseSubmitted(Container container)
+      throws SliderInternalStateException {
+    ContainerId id = container.getId();
+    //look up the container
+    RoleInstance instance = getOwnedContainer(id);
+    if (instance == null) {
+      throw new SliderInternalStateException(
+        "No active container with ID " + id);
+    }
+    //verify that it isn't already released
+    if (containersBeingReleased.containsKey(id)) {
+      throw new SliderInternalStateException(
+        "Container %s already queued for release", id);
+    }
+    instance.released = true;
+    containersBeingReleased.put(id, instance.container);
+    RoleStatus role = lookupRoleStatus(instance.roleId);
+    role.incReleasing();
+    roleHistory.onContainerReleaseSubmitted(container);
+  }
+
+  /**
+   * Create a container request.
+   * Update internal state, such as the role request count. 
+   * Anti-Affine: the {@link RoleStatus#outstandingAArequest} is set here.
+   * This is where role history information will be used for placement decisions.
+   * @param role role
+   * @return the container request to submit or null if there is none
+   */
+  private AMRMClient.ContainerRequest createContainerRequest(RoleStatus role) {
+    if (role.isAntiAffinePlacement()) {
+      return createAAContainerRequest(role);
+    } else {
+      incrementRequestCount(role);
+      OutstandingRequest request = roleHistory.requestContainerForRole(role);
+      if (request != null) {
+        return request.getIssuedRequest();
+      } else {
+        return null;
+      }
+    }
+  }
+
+  /**
+   * Create a container request.
+   * Update internal state, such as the role request count.
+   * Anti-Affine: the {@link RoleStatus#outstandingAArequest} is set here.
+   * This is where role history information will be used for placement decisions.
+   * @param role role
+   * @return the container request to submit or null if there is none
+   */
+  private AMRMClient.ContainerRequest createAAContainerRequest(RoleStatus role) {
+    OutstandingRequest request = roleHistory.requestContainerForAARole(role);
+    if (request == null) {
+      return null;
+    }
+    incrementRequestCount(role);
+    role.setOutstandingAArequest(request);
+    return request.getIssuedRequest();
+  }
+
+  /**
+   * Increment the request count of a role.
+   * <p>
+   *   Also updates application state counters
+   * @param role role being requested.
+   */
+  protected void incrementRequestCount(RoleStatus role) {
+    role.incRequested();
+    incOutstandingContainerRequests();
+  }
+
+  /**
+   * Inc #of outstanding requests.
+   */
+  private void incOutstandingContainerRequests() {
+     outstandingContainerRequests.inc();
+  }
+
+  /**
+   * Decrement the number of outstanding requests. This never goes below zero.
+   */
+  private void decOutstandingContainerRequests() {
+    synchronized (outstandingContainerRequests) {
+      if (outstandingContainerRequests.getCount() > 0) {
+        // decrement but never go below zero
+        outstandingContainerRequests.dec();
+      }
+    }
+  }
+
+
+  /**
+   * Get the value of a YARN requirement (cores, RAM, etc).
+   * These are returned as integers, but there is special handling of the 
+   * string {@link ResourceKeys#YARN_RESOURCE_MAX}, which triggers
+   * the return of the maximum value.
+   * @param group component to get from
+   * @param option option name
+   * @param defVal default value
+   * @param maxVal value to return if the max val is requested
+   * @return parsed value
+   * @throws NumberFormatException if the role could not be parsed.
+   */
+  private int getResourceRequirement(ConfTreeOperations resources,
+                                     String group,
+                                     String option,
+                                     int defVal,
+                                     int maxVal) {
+
+    String val = resources.getComponentOpt(group, option,
+        Integer.toString(defVal));
+    Integer intVal;
+    if (YARN_RESOURCE_MAX.equals(val)) {
+      intVal = maxVal;
+    } else {
+      intVal = Integer.decode(val);
+    }
+    return intVal;
+  }
+
+  /**
+   * Build up the resource requirements for this role from the
+   * cluster specification, including substituing max allowed values
+   * if the specification asked for it.
+   * @param role role
+   * @param capability capability to set up. A new one may be created
+   * during normalization
+   */
+  public Resource buildResourceRequirements(RoleStatus role, Resource capability) {
+    // Set up resource requirements from role values
+    String name = role.getName();
+    String group = role.getGroup();
+    ConfTreeOperations resources = getResourcesSnapshot();
+    int cores = getResourceRequirement(resources,
+                                       group,
+                                       YARN_CORES,
+                                       DEF_YARN_CORES,
+                                       containerMaxCores);
+    capability.setVirtualCores(cores);
+    int ram = getResourceRequirement(resources, group,
+                                     YARN_MEMORY,
+                                     DEF_YARN_MEMORY,
+                                     containerMaxMemory);
+    capability.setMemory(ram);
+    log.debug("Component {} has RAM={}, vCores ={}", name, ram, cores);
+    Resource normalized = recordFactory.normalize(capability, minResource,
+        maxResource);
+    if (!Resources.equals(normalized, capability)) {
+      // resource requirements normalized to something other than asked for.
+      // LOG @ WARN so users can see why this is happening.
+      log.warn("Resource requirements of {} normalized" +
+              " from {} to {}", name, capability, normalized);
+    }
+    return normalized;
+  }
+
+  /**
+   * add a launched container to the node map for status responses
+   * @param container id
+   * @param node node details
+   */
+  private void addLaunchedContainer(Container container, RoleInstance node) {
+    node.container = container;
+    if (node.role == null) {
+      throw new RuntimeException(
+        "Unknown role for node " + node);
+    }
+    getLiveContainers().put(node.getContainerId(), node);
+    //tell role history
+    roleHistory.onContainerStarted(container);
+  }
+
+  /**
+   * container start event
+   * @param containerId container that is to be started
+   * @return the role instance, or null if there was a problem
+   */
+  public synchronized RoleInstance onNodeManagerContainerStarted(ContainerId containerId) {
+    try {
+      return innerOnNodeManagerContainerStarted(containerId);
+    } catch (YarnRuntimeException e) {
+      log.error("NodeManager callback on started container {} failed",
+                containerId,
+                e);
+      return null;
+    }
+  }
+
+   /**
+   * container start event handler -throwing an exception on problems
+   * @param containerId container that is to be started
+   * @return the role instance
+   * @throws RuntimeException on problems
+   */
+  @VisibleForTesting
+  public RoleInstance innerOnNodeManagerContainerStarted(ContainerId containerId) {
+    incStartedCountainerCount();
+    RoleInstance instance = getOwnedContainer(containerId);
+    if (instance == null) {
+      //serious problem
+      throw new YarnRuntimeException("Container not in active containers start "+
+                containerId);
+    }
+    if (instance.role == null) {
+      throw new YarnRuntimeException("Component instance has no instance name " +
+                                     instance);
+    }
+    instance.startTime = now();
+    RoleInstance starting = getStartingContainers().remove(containerId);
+    if (null == starting) {
+      throw new YarnRuntimeException(
+        "Container "+ containerId +" is already started");
+    }
+    instance.state = STATE_LIVE;
+    RoleStatus roleStatus = lookupRoleStatus(instance.roleId);
+    roleStatus.incStarted();
+    Container container = instance.container;
+    addLaunchedContainer(container, instance);
+    return instance;
+  }
+
+  /**
+   * update the application state after a failure to start a container.
+   * This is perhaps where blacklisting could be most useful: failure
+   * to start a container is a sign of a more serious problem
+   * than a later exit.
+   *
+   * -relayed from NMClientAsync.CallbackHandler 
+   * @param containerId failing container
+   * @param thrown what was thrown
+   */
+  public synchronized void onNodeManagerContainerStartFailed(ContainerId containerId,
+                                                             Throwable thrown) {
+    removeOwnedContainer(containerId);
+    incFailedCountainerCount();
+    incStartFailedCountainerCount();
+    RoleInstance instance = getStartingContainers().remove(containerId);
+    if (null != instance) {
+      RoleStatus roleStatus = lookupRoleStatus(instance.roleId);
+      String text;
+      if (null != thrown) {
+        text = SliderUtils.stringify(thrown);
+      } else {
+        text = "container start failure";
+      }
+      instance.diagnostics = text;
+      roleStatus.noteFailed(true, text, ContainerOutcome.Failed);
+      getFailedContainers().put(containerId, instance);
+      roleHistory.onNodeManagerContainerStartFailed(instance.container);
+    }
+  }
+
+  /**
+   * Handle node update from the RM. This syncs up the node map with the RM's view
+   * @param updatedNodes updated nodes
+   */
+  public synchronized NodeUpdatedOutcome onNodesUpdated(List<NodeReport> updatedNodes) {
+    boolean changed = roleHistory.onNodesUpdated(updatedNodes);
+    if (changed) {
+      log.info("YARN cluster changed \u2014cancelling current AA requests");
+      List<AbstractRMOperation> operations = cancelOutstandingAARequests();
+      log.debug("Created {} cancel requests", operations.size());
+      return new NodeUpdatedOutcome(true, operations);
+    }
+    return new NodeUpdatedOutcome(false, new ArrayList<AbstractRMOperation>(0));
+  }
+
+  /**
+   * Return value of the {@link #onNodesUpdated(List)} call.
+   */
+  public static class NodeUpdatedOutcome {
+    public final boolean clusterChanged;
+    public final List<AbstractRMOperation> operations;
+
+    public NodeUpdatedOutcome(boolean clusterChanged,
+        List<AbstractRMOperation> operations) {
+      this.clusterChanged = clusterChanged;
+      this.operations = operations;
+    }
+  }
+  /**
+   * Is a role short lived by the threshold set for this application
+   * @param instance instance
+   * @return true if the instance is considered short lived
+   */
+  @VisibleForTesting
+  public boolean isShortLived(RoleInstance instance) {
+    long time = now();
+    long started = instance.startTime;
+    boolean shortlived;
+    if (started > 0) {
+      long duration = time - started;
+      shortlived = duration < (startTimeThreshold * 1000);
+      log.info("Duration {} and startTimeThreshold {}", duration, startTimeThreshold);
+    } else {
+      // never even saw a start event
+      shortlived = true;
+    }
+    return shortlived;
+  }
+
+  /**
+   * Current time in milliseconds. Made protected for
+   * the option to override it in tests.
+   * @return the current time.
+   */
+  protected long now() {
+    return System.currentTimeMillis();
+  }
+
+  /**
+   * This is a very small class to send a multiple result back from 
+   * the completion operation
+   */
+  public static class NodeCompletionResult {
+    public boolean surplusNode = false;
+    public RoleInstance roleInstance;
+    // did the container fail for *any* reason?
+    public boolean containerFailed = false;
+    // detailed outcome on the container failure
+    public ContainerOutcome outcome = ContainerOutcome.Completed;
+    public int exitStatus = 0;
+    public boolean unknownNode = false;
+
+    public String toString() {
+      final StringBuilder sb =
+        new StringBuilder("NodeCompletionResult{");
+      sb.append("surplusNode=").append(surplusNode);
+      sb.append(", roleInstance=").append(roleInstance);
+      sb.append(", exitStatus=").append(exitStatus);
+      sb.append(", containerFailed=").append(containerFailed);
+      sb.append(", outcome=").append(outcome);
+      sb.append(", unknownNode=").append(unknownNode);
+      sb.append('}');
+      return sb.toString();
+    }
+  }
+
+  /**
+   * handle completed node in the CD -move something from the live
+   * server list to the completed server list.
+   * @param status the node that has just completed
+   * @return NodeCompletionResult
+   */
+  public synchronized NodeCompletionResult onCompletedNode(ContainerStatus status) {
+    ContainerId containerId = status.getContainerId();
+    NodeCompletionResult result = new NodeCompletionResult();
+    RoleInstance roleInstance;
+
+    int exitStatus = status.getExitStatus();
+    result.exitStatus = exitStatus;
+    if (containersBeingReleased.containsKey(containerId)) {
+      log.info("Container was queued for release : {}", containerId);
+      Container container = containersBeingReleased.remove(containerId);
+      RoleStatus roleStatus = lookupRoleStatus(container);
+      long releasing = roleStatus.decReleasing();
+      long actual = roleStatus.decActual();
+      long completedCount = roleStatus.incCompleted();
+      log.info("decrementing role count for role {} to {}; releasing={}, completed={}",
+          roleStatus.getName(),
+          actual,
+          releasing,
+          completedCount);
+      result.outcome = ContainerOutcome.Completed;
+      roleHistory.onReleaseCompleted(container);
+
+    } else if (surplusNodes.remove(containerId)) {
+      //its a surplus one being purged
+      result.surplusNode = true;
+    } else {
+      // a container has failed or been killed
+      // use the exit code to determine the outcome
+      result.containerFailed = true;
+      result.outcome = ContainerOutcome.fromExitStatus(exitStatus);
+
+      roleInstance = removeOwnedContainer(containerId);
+      if (roleInstance != null) {
+        //it was active, move it to failed 
+        incFailedCountainerCount();
+        failedContainers.put(containerId, roleInstance);
+      } else {
+        // the container may have been noted as failed already, so look
+        // it up
+        roleInstance = failedContainers.get(containerId);
+      }
+      if (roleInstance != null) {
+        int roleId = roleInstance.roleId;
+        String rolename = roleInstance.role;
+        log.info("Failed container in role[{}] : {}", roleId, rolename);
+        try {
+          RoleStatus roleStatus = lookupRoleStatus(roleId);
+          roleStatus.decActual();
+          boolean shortLived = isShortLived(roleInstance);
+          String message;
+          Container failedContainer = roleInstance.container;
+
+          //build the failure message
+          if (failedContainer != null) {
+            String completedLogsUrl = getLogsURLForContainer(failedContainer);
+            message = String.format("Failure %s on host %s (%d): %s",
+                roleInstance.getContainerId(),
+                failedContainer.getNodeId().getHost(),
+                exitStatus,
+                completedLogsUrl);
+          } else {
+            message = String.format("Failure %s (%d)", containerId, exitStatus);
+          }
+          roleStatus.noteFailed(shortLived, message, result.outcome);
+          long failed = roleStatus.getFailed();
+          log.info("Current count of failed role[{}] {} =  {}",
+              roleId, rolename, failed);
+          if (failedContainer != null) {
+            roleHistory.onFailedContainer(failedContainer, shortLived, result.outcome);
+          }
+
+        } catch (YarnRuntimeException e1) {
+          log.error("Failed container of unknown role {}", roleId);
+        }
+      } else {
+        //this isn't a known container.
+
+        log.error("Notified of completed container {} that is not in the list" +
+            " of active or failed containers", containerId);
+        completionOfUnknownContainerEvent.incrementAndGet();
+        result.unknownNode = true;
+      }
+    }
+
+    if (result.surplusNode) {
+      //a surplus node
+      return result;
+    }
+
+    //record the complete node's details; this pulls it from the livenode set 
+    //remove the node
+    ContainerId id = status.getContainerId();
+    log.info("Removing node ID {}", id);
+    RoleInstance node = getLiveContainers().remove(id);
+    if (node != null) {
+      node.state = STATE_DESTROYED;
+      node.exitCode = exitStatus;
+      node.diagnostics = status.getDiagnostics();
+      getCompletedContainers().put(id, node);
+      result.roleInstance = node;
+    } else {
+      // not in the list
+      log.warn("Received notification of completion of unknown node {}", id);
+      completionOfNodeNotInLiveListEvent.incrementAndGet();
+    }
+
+    // and the active node list if present
+    removeOwnedContainer(containerId);
+
+    // finally, verify the node doesn't exist any more
+    assert !containersBeingReleased.containsKey(
+        containerId) : "container still in release queue";
+    assert !getLiveContainers().containsKey(
+        containerId) : " container still in live nodes";
+    assert getOwnedContainer(containerId) ==
+           null : "Container still in active container list";
+
+    return result;
+  }
+
+  /**
+   * Get the URL log for a container
+   * @param c container
+   * @return the URL or "" if it cannot be determined
+   */
+  protected String getLogsURLForContainer(Container c) {
+    if (c==null) {
+      return null;
+    }
+    String user = null;
+    try {
+      user = SliderUtils.getCurrentUser().getShortUserName();
+    } catch (IOException ignored) {
+    }
+    String completedLogsUrl = "";
+    String url = logServerURL;
+    if (user != null && SliderUtils.isSet(url)) {
+      completedLogsUrl = url
+          + "/" + c.getNodeId() + "/" + c.getId() + "/ctx/" + user;
+    }
+    return completedLogsUrl;
+  }
+
+  /**
+   * Return the percentage done that Slider is to have YARN display in its
+   * Web UI
+   * @return an number from 0 to 100
+   */
+  public synchronized float getApplicationProgressPercentage() {
+    float percentage;
+    long desired = 0;
+    float actual = 0;
+    for (RoleStatus role : getRoleStatusMap().values()) {
+      desired += role.getDesired();
+      actual += role.getActual();
+    }
+    if (desired == 0) {
+      percentage = 100;
+    } else {
+      percentage = actual / desired;
+    }
+    return percentage;
+  }
+
+  /**
+   * Update the cluster description with the current application state
+   */
+
+  public ClusterDescription refreshClusterStatus() {
+    return refreshClusterStatus(null);
+  }
+
+  /**
+   * Update the cluster description with the current application state
+   * @param providerStatus status from the provider for the cluster info section
+   */
+  public synchronized ClusterDescription refreshClusterStatus(Map<String, String> providerStatus) {
+    ClusterDescription cd = getClusterStatus();
+    long now = now();
+    cd.setInfoTime(StatusKeys.INFO_STATUS_TIME_HUMAN,
+                   StatusKeys.INFO_STATUS_TIME_MILLIS,
+                   now);
+    if (providerStatus != null) {
+      for (Map.Entry<String, String> entry : providerStatus.entrySet()) {
+        cd.setInfo(entry.getKey(), entry.getValue());
+      }
+    }
+    MapOperations infoOps = new MapOperations("info", cd.info);
+    infoOps.mergeWithoutOverwrite(applicationInfo);
+    SliderUtils.addBuildInfo(infoOps, "status");
+    cd.statistics = new HashMap<>();
+
+    // build the map of node -> container IDs
+    Map<String, List<String>> instanceMap = createRoleToInstanceMap();
+    cd.instances = instanceMap;
+    
+    //build the map of node -> containers
+    Map<String, Map<String, ClusterNode>> clusterNodes =
+      createRoleToClusterNodeMap();
+    log.info("app state clusterNodes {} ", clusterNodes.toString());
+    cd.status = new HashMap<>();
+    cd.status.put(ClusterDescriptionKeys.KEY_CLUSTER_LIVE, clusterNodes);
+
+
+    for (RoleStatus role : getRoleStatusMap().values()) {
+      String rolename = role.getName();
+      if (hasUniqueNames(instanceDefinition.getResourceOperations(),
+          role.getGroup())) {
+        cd.setRoleOpt(rolename, COMPONENT_PRIORITY, role.getPriority());
+        cd.setRoleOpt(rolename, ROLE_GROUP, role.getGroup());
+        MapOperations groupOptions = instanceDefinition.getResourceOperations()
+            .getComponent(role.getGroup());
+        SliderUtils.mergeMapsIgnoreDuplicateKeys(cd.getRole(rolename),
+            groupOptions.options);
+      }
+      List<String> instances = instanceMap.get(rolename);
+      int nodeCount = instances != null ? instances.size(): 0;
+      cd.setRoleOpt(rolename, COMPONENT_INSTANCES,
+                    role.getDesired());
+      cd.setRoleOpt(rolename, ROLE_ACTUAL_INSTANCES, nodeCount);
+      cd.setRoleOpt(rolename, ROLE_REQUESTED_INSTANCES, role.getRequested());
+      cd.setRoleOpt(rolename, ROLE_RELEASING_INSTANCES, role.getReleasing());
+      cd.setRoleOpt(rolename, ROLE_FAILED_INSTANCES, role.getFailed());
+      cd.setRoleOpt(rolename, ROLE_FAILED_STARTING_INSTANCES, role.getStartFailed());
+      cd.setRoleOpt(rolename, ROLE_FAILED_RECENTLY_INSTANCES, role.getFailedRecently());
+      cd.setRoleOpt(rolename, ROLE_NODE_FAILED_INSTANCES, role.getNodeFailed());
+      cd.setRoleOpt(rolename, ROLE_PREEMPTED_INSTANCES, role.getPreempted());
+      if (role.isAntiAffinePlacement()) {
+        cd.setRoleOpt(rolename, ROLE_PENDING_AA_INSTANCES, role.getPendingAntiAffineRequests());
+      }
+      Map<String, Integer> stats = role.buildStatistics();
+      cd.statistics.put(rolename, stats);
+    }
+
+    Map<String, Integer> sliderstats = getLiveStatistics();
+    cd.statistics.put(SliderKeys.COMPONENT_AM, sliderstats);
+
+    // liveness
+    cd.liveness = getApplicationLivenessInformation();
+
+    return cd;
+  }
+
+  /**
+   * get application liveness information
+   * @return a snapshot of the current liveness information
+   */  
+  public ApplicationLivenessInformation getApplicationLivenessInformation() {
+    ApplicationLivenessInformation li = new ApplicationLivenessInformation();
+    RoleStatistics stats = getRoleStatistics();
+    int outstanding = (int)(stats.desired - stats.actual);
+    li.requestsOutstanding = outstanding;
+    li.allRequestsSatisfied = outstanding <= 0;
+    li.activeRequests = (int)stats.requested;
+    return li;
+  }
+
+  /**
+   * Get the live statistics map
+   * @return a map of statistics values, defined in the {@link StatusKeys}
+   * keylist.
+   */
+  protected Map<String, Integer> getLiveStatistics() {
+    Map<String, Integer> sliderstats = new HashMap<>();
+    sliderstats.put(StatusKeys.STATISTICS_CONTAINERS_LIVE,
+        liveNodes.size());
+    sliderstats.put(StatusKeys.STATISTICS_CONTAINERS_COMPLETED,
+        completedContainerCount.intValue());
+    sliderstats.put(StatusKeys.STATISTICS_CONTAINERS_FAILED,
+        failedContainerCount.intValue());
+    sliderstats.put(StatusKeys.STATISTICS_CONTAINERS_STARTED,
+        startedContainers.intValue());
+    sliderstats.put(StatusKeys.STATISTICS_CONTAINERS_START_FAILED,
+         startFailedContainerCount.intValue());
+    sliderstats.put(StatusKeys.STATISTICS_CONTAINERS_SURPLUS,
+        surplusContainers.intValue());
+    sliderstats.put(StatusKeys.STATISTICS_CONTAINERS_UNKNOWN_COMPLETED,
+        completionOfUnknownContainerEvent.get());
+    return sliderstats;
+  }
+
+  /**
+   * Get the aggregate statistics across all roles
+   * @return role statistics
+   */
+  public RoleStatistics getRoleStatistics() {
+    RoleStatistics stats = new RoleStatistics();
+    for (RoleStatus role : getRoleStatusMap().values()) {
+      stats.add(role.getStatistics());
+    }
+    return stats;
+  }
+
+  /**
+   * Get a snapshot of component information.
+   * <p>
+   *   This does <i>not</i> include any container list, which 
+   *   is more expensive to create.
+   * @return a map of current role status values.
+   */
+  public Map<String, ComponentInformation> getComponentInfoSnapshot() {
+
+    Map<Integer, RoleStatus> statusMap = getRoleStatusMap();
+    Map<String, ComponentInformation> results = new HashMap<>(
+            statusMap.size());
+
+    for (RoleStatus status : statusMap.values()) {
+      String name = status.getName();
+      ComponentInformation info = status.serialize();
+      results.put(name, info);
+    }
+    return results;
+  }
+
+  /**
+   * Look at where the current node state is -and whether it should be changed
+   */
+  public synchronized List<AbstractRMOperation> reviewRequestAndReleaseNodes()
+      throws SliderInternalStateException, TriggerClusterTeardownException {
+    log.debug("in reviewRequestAndReleaseNodes()");
+    List<AbstractRMOperation> allOperations = new ArrayList<>();
+    for (RoleStatus roleStatus : getRoleStatusMap().values()) {
+      if (!roleStatus.isExcludeFromFlexing()) {
+        List<AbstractRMOperation> operations = reviewOneRole(roleStatus);
+        allOperations.addAll(operations);
+      }
+    }
+    return allOperations;
+  }
+
+  /**
+   * Check the "recent" failure threshold for a role
+   * @param role role to examine
+   * @throws TriggerClusterTeardownException if the role
+   * has failed too many times
+   */
+  private void checkFailureThreshold(RoleStatus role)
+      throws TriggerClusterTeardownException {
+    long failures = role.getFailedRecently();
+    int threshold = getFailureThresholdForRole(role);
+    if (log.isDebugEnabled() && failures > 0) {
+      log.debug("Failure count of component: {}: {}, threshold={}",
+          role.getName(), failures, threshold);
+    }
+
+    if (failures > threshold) {
+      throw new TriggerClusterTeardownException(
+        SliderExitCodes.EXIT_DEPLOYMENT_FAILED,
+          FinalApplicationStatus.FAILED, ErrorStrings.E_UNSTABLE_CLUSTER +
+        " - failed with component %s failed 'recently' %d times (%d in startup);" +
+        " threshold is %d - last failure: %s",
+          role.getName(),
+        role.getFailed(),
+        role.getStartFailed(),
+          threshold,
+        role.getFailureMessage());
+    }
+  }
+
+  /**
+   * Get the failure threshold for a specific role, falling back to
+   * the global one if not
+   * @param roleStatus role
+   * @return the threshold for failures
+   */
+  private int getFailureThresholdForRole(RoleStatus roleStatus) {
+    ConfTreeOperations resources =
+        instanceDefinition.getResourceOperations();
+    return resources.getComponentOptInt(roleStatus.getGroup(),
+        CONTAINER_FAILURE_THRESHOLD,
+        failureThreshold);
+  }
+
+  /**
+   * Get the node failure threshold for a specific role, falling back to
+   * the global one if not
+   * @param roleGroup role group
+   * @return the threshold for failures
+   */
+  private int getNodeFailureThresholdForRole(String roleGroup) {
+    ConfTreeOperations resources =
+        instanceDefinition.getResourceOperations();
+    return resources.getComponentOptInt(roleGroup,
+                                        NODE_FAILURE_THRESHOLD,
+                                        nodeFailureThreshold);
+  }
+
+  /**
+   * Reset the "recent" failure counts of all roles
+   */
+  public void resetFailureCounts() {
+    for (RoleStatus roleStatus : getRoleStatusMap().values()) {
+      long failed = roleStatus.resetFailedRecently();
+      log.info("Resetting failure count of {}; was {}",
+               roleStatus.getName(),
+          failed);
+    }
+    roleHistory.resetFailedRecently();
+  }
+
+  /**
+   * Escalate operation as triggered by external timer.
+   * @return a (usually empty) list of cancel/request operations.
+   */
+  public List<AbstractRMOperation> escalateOutstandingRequests() {
+    return roleHistory.escalateOutstandingRequests();
+  }
+
+  /**
+   * Cancel any outstanding AA Requests, building up the list of ops to
+   * cancel, removing them from RoleHistory structures and the RoleStatus
+   * entries.
+   * @return a (usually empty) list of cancel/request operations.
+   */
+  public synchronized List<AbstractRMOperation> cancelOutstandingAARequests() {
+    // get the list of cancel operations
+    List<AbstractRMOperation> operations = roleHistory.cancelOutstandingAARequests();
+    for (RoleStatus roleStatus : roleStatusMap.values()) {
+      if (roleStatus.isAARequestOutstanding()) {
+        log.info("Cancelling outstanding AA request for {}", roleStatus);
+        roleStatus.cancelOutstandingAARequest();
+      }
+    }
+    return operations;
+  }
+
+  /**
+   * Look at the allocation status of one role, and trigger add/release
+   * actions if the number of desired role instances doesn't equal 
+   * (actual + pending).
+   * <p>
+   * MUST be executed from within a synchronized method
+   * <p>
+   * @param role role
+   * @return a list of operations
+   * @throws SliderInternalStateException if the operation reveals that
+   * the internal state of the application is inconsistent.
+   */
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  private List<AbstractRMOperation> reviewOneRole(RoleStatus role)
+      throws SliderInternalStateException, TriggerClusterTeardownException {
+    List<AbstractRMOperation> operations = new ArrayList<>();
+    long delta;
+    long expected;
+    String name = role.getName();
+    synchronized (role) {
+      delta = role.getDelta();
+      expected = role.getDesired();
+    }
+
+    log.info("Reviewing {} : ", role);
+    log.debug("Expected {}, Delta: {}", expected, delta);
+    checkFailureThreshold(role);
+
+    if (expected < 0 ) {
+      // negative value: fail
+      throw new TriggerClusterTeardownException(
+          SliderExitCodes.EXIT_DEPLOYMENT_FAILED,
+          FinalApplicationStatus.FAILED,
+          "Negative component count of %d desired for component %s",
+          expected, role);
+    }
+
+    if (delta > 0) {
+      // more workers needed than we have -ask for more
+      log.info("{}: Asking for {} more nodes(s) for a total of {} ", name, delta, expected);
+
+      if (role.isAntiAffinePlacement()) {
+        long pending = delta;
+        if (roleHistory.canPlaceAANodes()) {
+          // build one only if there is none outstanding, the role history knows
+          // enough about the cluster to ask, and there is somewhere to place
+          // the node
+          if (!role.isAARequestOutstanding()) {
+            // no outstanding AA; try to place things
+            AMRMClient.ContainerRequest request = createAAContainerRequest(role);
+            if (request != null) {
+              pending--;
+              log.info("Starting an anti-affine request sequence for {} nodes; pending={}",
+                delta, pending);
+              addContainerRequest(operations, request);
+            } else {
+              log.info("No location for anti-affine request");
+            }
+          }
+        } else {
+          log.warn("Awaiting node map before generating anti-affinity requests");
+        }
+        log.info("Setting pending to {}", pending);
+        role.setPendingAntiAffineRequests(pending);
+      } else {
+
+        for (int i = 0; i < delta; i++) {
+          //get the role history to select a suitable node, if available
+          addContainerRequest(operations, createContainerRequest(role));
+        }
+      }
+    } else if (delta < 0) {
+      log.info("{}: Asking for {} fewer node(s) for a total of {}", name,
+               -delta,
+               expected);
+      // reduce the number expected (i.e. subtract the delta)
+      long excess = -delta;
+
+      // how many requests are outstanding? for AA roles, this includes pending
+      long outstandingRequests = role.getRequested() + role.getPendingAntiAffineRequests();
+      if (outstandingRequests > 0) {
+        // outstanding requests.
+        int toCancel = (int)Math.min(outstandingRequests, excess);
+
+        // Delegate to Role History
+        List<AbstractRMOperation> cancellations = roleHistory.cancelRequestsForRole(role, toCancel);
+        log.info("Found {} outstanding requests to cancel", cancellations.size());
+        operations.addAll(cancellations);
+        if (toCancel != cancellations.size()) {
+          log.error("Tracking of outstanding requests is not in sync with the summary statistics:" +
+              " expected to be able to cancel {} requests, but got {}",
+              toCancel, cancellations.size());
+        }
+
+        role.cancel(toCancel);
+        excess -= toCancel;
+        assert excess >= 0 : "Attempted to cancel too many requests";
+        log.info("Submitted {} cancellations, leaving {} to release",
+            toCancel, excess);
+        if (excess == 0) {
+          log.info("After cancelling requests, application is now at desired size");
+        }
+      }
+
+      // after the cancellation there may be no excess
+      if (excess > 0) {
+
+        // there's an excess, so more to cancel
+        // get the nodes to release
+        int roleId = role.getKey();
+
+        // enum all active nodes that aren't being released
+        List<RoleInstance> containersToRelease = enumNodesWithRoleId(roleId, true);
+        if (containersToRelease.isEmpty()) {
+          log.info("No containers for component {}", roleId);
+        }
+
+        // filter out all release-in-progress nodes
+        ListIterator<RoleInstance> li = containersToRelease.listIterator();
+        while (li.hasNext()) {
+          RoleInstance next = li.next();
+          if (next.released) {
+            li.remove();
+          }
+        }
+
+        // warn if the desired state can't be reached
+        int numberAvailableForRelease = containersToRelease.size();
+        if (numberAvailableForRelease < excess) {
+          log.warn("Not enough containers to release, have {} and need {} more",
+              numberAvailableForRelease,
+              excess - numberAvailableForRelease);
+        }
+
+        // ask the release selector to sort the targets
+        containersToRelease =  containerReleaseSelector.sortCandidates(
+            roleId,
+            containersToRelease);
+
+        // crop to the excess
+        List<RoleInstance> finalCandidates = (excess < numberAvailableForRelease)
+            ? containersToRelease.subList(0, (int)excess)
+            : containersToRelease;
+
+        // then build up a release operation, logging each container as released
+        for (RoleInstance possible : finalCandidates) {
+          log.info("Targeting for release: {}", possible);
+          containerReleaseSubmitted(possible.container);
+          operations.add(new ContainerReleaseOperation(possible.getId()));
+        }
+      }
+
+    } else {
+      // actual + requested == desired
+      // there's a special case here: clear all pending AA requests
+      if (role.getPendingAntiAffineRequests() > 0) {
+        log.debug("Clearing outstanding pending AA requests");
+        role.setPendingAntiAffineRequests(0);
+      }
+    }
+
+    // there's now a list of operations to execute
+    log.debug("operations scheduled: {}; updated role: {}", operations.size(), role);
+    return operations;
+  }
+
+  /**
+   * Add a container request if the request is non-null
+   * @param operations operations to add the entry to
+   * @param containerAsk what to ask for
+   * @return true if a request was added
+   */
+  private boolean addContainerRequest(List<AbstractRMOperation> operations,
+      AMRMClient.ContainerRequest containerAsk) {
+    if (containerAsk != null) {
+      log.info("Container ask is {} and label = {}", containerAsk,
+          containerAsk.getNodeLabelExpression());
+      int askMemory = containerAsk.getCapability().getMemory();
+      if (askMemory > this.containerMaxMemory) {
+        log.warn("Memory requested: {} > max of {}", askMemory, containerMaxMemory);
+      }
+      operations.add(new ContainerRequestOperation(containerAsk));
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Releases a container based on container id
+   * @param containerId
+   * @return
+   * @throws SliderInternalStateException
+   */
+  public List<AbstractRMOperation> releaseContainer(ContainerId containerId)
+      throws SliderInternalStateException {
+    List<AbstractRMOperation> operations = new ArrayList<AbstractRMOperation>();
+    List<RoleInstance> activeRoleInstances = cloneOwnedContainerList();
+    for (RoleInstance role : activeRoleInstances) {
+      if (role.container.getId().equals(containerId)) {
+        containerReleaseSubmitted(role.container);
+        operations.add(new ContainerReleaseOperation(role.getId()));
+      }
+    }
+
+    return operations;
+  }
+
+  /**
+   * Find a container running on a specific host -looking
+   * into the node ID to determine this.
+   *
+   * @param node node
+   * @param roleId role the container must be in
+   * @return a container or null if there are no containers on this host
+   * that can be released.
+   */
+  private RoleInstance findRoleInstanceOnHost(NodeInstance node, int roleId) {
+    Collection<RoleInstance> targets = cloneOwnedContainerList();
+    String hostname = node.hostname;
+    for (RoleInstance ri : targets) {
+      if (hostname.equals(RoleHistoryUtils.hostnameOf(ri.container))
+                         && ri.roleId == roleId
+        && containersBeingReleased.get(ri.getContainerId()) == null) {
+        return ri;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Release all containers.
+   * @return a list of operations to execute
+   */
+  public synchronized List<AbstractRMOperation> releaseAllContainers() {
+
+    Collection<RoleInstance> targets = cloneOwnedContainerList();
+    log.info("Releasing {} containers", targets.size());
+    List<AbstractRMOperation> operations =
+      new ArrayList<>(targets.size());
+    for (RoleInstance instance : targets) {
+      if (instance.roleId == SliderKeys.ROLE_AM_PRIORITY_INDEX) {
+        // don't worry about the AM
+        continue;
+      }
+      Container possible = instance.container;
+      ContainerId id = possible.getId();
+      if (!instance.released) {
+        String url = getLogsURLForContainer(possible);
+        log.info("Releasing container. Log: " + url);
+        try {
+          containerReleaseSubmitted(possible);
+        } catch (SliderInternalStateException e) {
+          log.warn("when releasing container {} :", possible, e);
+        }
+        operations.add(new ContainerReleaseOperation(id));
+      }
+    }
+    return operations;
+  }
+
+  /**
+   * Event handler for allocated containers: builds up the lists
+   * of assignment actions (what to run where), and possibly
+   * a list of operations to perform
+   * @param allocatedContainers the containers allocated
+   * @param assignments the assignments of roles to containers
+   * @param operations any allocation or release operations
+   */
+  public synchronized void onContainersAllocated(List<Container> allocatedContainers,
+                                    List<ContainerAssignment> assignments,
+                                    List<AbstractRMOperation> operations) {
+    assignments.clear();
+    operations.clear();
+    List<Container> ordered = roleHistory.prepareAllocationList(allocatedContainers);
+    log.debug("onContainersAllocated(): Total containers allocated = {}", ordered.size());
+    for (Container container : ordered) {
+      final NodeId nodeId = container.getNodeId();
+      String containerHostInfo = nodeId.getHost() + ":" + nodeId.getPort();
+      //get the role
+      final ContainerId cid = container.getId();
+      final RoleStatus role = lookupRoleStatus(container);
+
+      //dec requested count
+      role.decRequested();
+
+      //inc allocated count -this may need to be dropped in a moment,
+      // but us needed to update the logic below
+      final long allocated = role.incActual();
+      final long desired = role.getDesired();
+
+      final String roleName = role.getName();
+      final ContainerAllocationResults allocation =
+          roleHistory.onContainerAllocated(container, desired, allocated);
+      final ContainerAllocationOutcome outcome = allocation.outcome;
+
+      // add all requests to the operations list
+      operations.addAll(allocation.operations);
+
+      //look for condition where we get more back than we asked
+      if (allocated > desired) {
+        log.info("Discarding surplus {} container {} on {}", roleName,  cid, containerHostInfo);
+        operations.add(new ContainerReleaseOperation(cid));
+        //register as a surplus node
+        surplusNodes.add(cid);
+        surplusContainers.inc();
+        //and, as we aren't binding it to role, dec that role's actual count
+        role.decActual();
+      } else {
+
+        // Allocation being accepted -so decrement the number of outstanding requests
+        decOutstandingContainerRequests();
+
+        log.info("Assigning role {} to container" +
+                 " {}," +
+                 " on {}:{},",
+                 roleName,
+                 cid,
+                 nodeId.getHost(),
+                 nodeId.getPort());
+
+        assignments.add(new ContainerAssignment(container, role, outcome));
+        //add to the history
+        roleHistory.onContainerAssigned(container);
+        // now for AA requests, add some more
+        if (role.isAntiAffinePlacement()) {
+          role.completeOutstandingAARequest();
+          // check invariants. The new node must become unavailable.
+          NodeInstance node = roleHistory.getOrCreateNodeInstance(container);
+          if (node.canHost(role.getKey(), role.getLabelExpression())) {
+            log.error("Assigned node still declares as available {}", node.toFullString() );
+          }
+          if (role.getPendingAntiAffineRequests() > 0) {
+            // still an outstanding AA request: need to issue a new one.
+            log.info("Asking for next container for AA role {}", roleName);
+            if (!addContainerRequest(operations, createAAContainerRequest(role))) {
+              log.info("No capacity in cluster for new requests");
+            } else {
+              role.decPendingAntiAffineRequests();
+            }
+            log.debug("Current AA role status {}", role);
+          } else {
+            log.info("AA request sequence completed for role {}", role);
+          }
+        }
+
+      }
+    }
+  }
+
+  /**
+   * Get diagnostics info about containers
+   */
+  public String getContainerDiagnosticInfo() {
+    StringBuilder builder = new StringBuilder();
+    for (RoleStatus roleStatus : getRoleStatusMap().values()) {
+      builder.append(roleStatus).append('\n');
+    }
+    return builder.toString();
+  }
+
+  /**
+   * Event handler for the list of active containers on restart.
+   * Sets the info key {@link StatusKeys#INFO_CONTAINERS_AM_RESTART}
+   * to the size of the list passed down (and does not set it if none were)
+   * @param liveContainers the containers allocated
+   * @return true if a rebuild took place (even if size 0)
+   * @throws RuntimeException on problems
+   */
+  private boolean rebuildModelFromRestart(List<Container> liveContainers)
+      throws BadClusterStateException {
+    if (liveContainers == null) {
+      return false;
+    }
+    for (Container container : liveContainers) {
+      addRestartedContainer(container);
+    }
+    clusterStatus.setInfo(StatusKeys.INFO_CONTAINERS_AM_RESTART,
+                               Integer.toString(liveContainers.size()));
+    return true;
+  }
+
+  /**
+   * Add a restarted container by walking it through the create/submit/start
+   * lifecycle, so building up the internal structures
+   * @param container container that was running before the AM restarted
+   * @throws RuntimeException on problems
+   */
+  private void addRestartedContainer(Container container)
+      throws BadClusterStateException {
+    String containerHostInfo = container.getNodeId().getHost()
+                               + ":" +
+                               container.getNodeId().getPort();
+    // get the container ID
+    ContainerId cid = container.getId();
+    
+    // get the role
+    int roleId = ContainerPriority.extractRole(container);
+    RoleStatus role =
+      lookupRoleStatus(roleId);
+    // increment

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[38/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
YARN-5461. Initial code ported from slider-core module. (jianhe)


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

Branch: refs/heads/yarn-native-services
Commit: d8cab88d2b3829299e2da4e3a6b8c26805ffdd46
Parents: 9f5d2c4
Author: Jian He <ji...@apache.org>
Authored: Wed Aug 3 00:18:01 2016 +0800
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 .../hadoop-yarn-slider-core/pom.xml             |   397 +
 .../src/assembly/executable-jar.xml             |    47 +
 .../src/license/THIRD-PARTY.properties          |    33 +
 .../slider/server/avro/RoleHistoryRecord.avsc   |   114 +
 .../apache/hadoop/security/KerberosDiags.java   |   676 +
 .../src/main/java/org/apache/slider/Slider.java |    52 +
 .../apache/slider/api/ClusterDescription.java   |   795 +
 .../slider/api/ClusterDescriptionKeys.java      |    25 +
 .../api/ClusterDescriptionOperations.java       |    93 +
 .../java/org/apache/slider/api/ClusterNode.java |   220 +
 .../org/apache/slider/api/InternalKeys.java     |   199 +
 .../java/org/apache/slider/api/OptionKeys.java  |    58 +
 .../org/apache/slider/api/ResourceKeys.java     |   201 +
 .../java/org/apache/slider/api/RoleKeys.java    |   116 +
 .../apache/slider/api/SliderApplicationApi.java |   167 +
 .../slider/api/SliderClusterProtocol.java       |   179 +
 .../java/org/apache/slider/api/StateValues.java |    53 +
 .../java/org/apache/slider/api/StatusKeys.java  |   117 +
 .../org/apache/slider/api/proto/Messages.java   | 34473 +++++++++++++++++
 .../slider/api/proto/RestTypeMarshalling.java   |   319 +
 .../slider/api/proto/SliderClusterAPI.java      |  2293 ++
 .../types/ApplicationLivenessInformation.java   |    53 +
 .../slider/api/types/ComponentInformation.java  |   107 +
 .../slider/api/types/ContainerInformation.java  |    58 +
 .../slider/api/types/NodeEntryInformation.java  |    78 +
 .../slider/api/types/NodeInformation.java       |    59 +
 .../slider/api/types/NodeInformationList.java   |    41 +
 .../slider/api/types/PingInformation.java       |    47 +
 .../apache/slider/api/types/RoleStatistics.java |    66 +
 .../api/types/SliderInstanceDescription.java    |    54 +
 .../slider/client/ClientRegistryBinder.java     |   201 +
 .../org/apache/slider/client/ClientUtils.java   |   111 +
 .../org/apache/slider/client/SliderClient.java  |  4569 +++
 .../apache/slider/client/SliderClientAPI.java   |   368 +
 .../slider/client/SliderYarnClientImpl.java     |   410 +
 .../apache/slider/client/TokensOperation.java   |   109 +
 .../client/ipc/SliderApplicationIpcClient.java  |   245 +
 .../client/ipc/SliderClusterOperations.java     |   529 +
 .../slider/client/rest/BaseRestClient.java      |   152 +
 .../slider/client/rest/RestClientFactory.java   |    89 +
 .../rest/SliderApplicationApiRestClient.java    |   326 +
 .../org/apache/slider/common/Constants.java     |    35 +
 .../apache/slider/common/SliderExitCodes.java   |    88 +
 .../org/apache/slider/common/SliderKeys.java    |   278 +
 .../common/SliderXMLConfKeysForTesting.java     |    83 +
 .../apache/slider/common/SliderXmlConfKeys.java |   206 +
 .../common/params/AbstractActionArgs.java       |   178 +
 .../common/params/AbstractArgsDelegate.java     |    26 +
 .../AbstractClusterBuildingActionArgs.java      |   217 +
 .../common/params/ActionAMSuicideArgs.java      |    44 +
 .../slider/common/params/ActionBuildArgs.java   |    32 +
 .../slider/common/params/ActionClientArgs.java  |    98 +
 .../slider/common/params/ActionCreateArgs.java  |    59 +
 .../common/params/ActionDependencyArgs.java     |    65 +
 .../slider/common/params/ActionDestroyArgs.java |    37 +
 .../common/params/ActionDiagnosticArgs.java     |    73 +
 .../slider/common/params/ActionEchoArgs.java    |    33 +
 .../slider/common/params/ActionExistsArgs.java  |    47 +
 .../slider/common/params/ActionFlexArgs.java    |    54 +
 .../slider/common/params/ActionFreezeArgs.java  |    56 +
 .../slider/common/params/ActionHelpArgs.java    |    51 +
 .../common/params/ActionInstallKeytabArgs.java  |    57 +
 .../common/params/ActionInstallPackageArgs.java |    58 +
 .../slider/common/params/ActionKDiagArgs.java   |    86 +
 .../slider/common/params/ActionKeytabArgs.java  |    81 +
 .../common/params/ActionKillContainerArgs.java  |    37 +
 .../slider/common/params/ActionListArgs.java    |    74 +
 .../slider/common/params/ActionLookupArgs.java  |    76 +
 .../slider/common/params/ActionNodesArgs.java   |    71 +
 .../slider/common/params/ActionPackageArgs.java |    81 +
 .../common/params/ActionRegistryArgs.java       |   218 +
 .../slider/common/params/ActionResolveArgs.java |   153 +
 .../common/params/ActionResourceArgs.java       |    68 +
 .../slider/common/params/ActionStatusArgs.java  |    45 +
 .../slider/common/params/ActionThawArgs.java    |    61 +
 .../slider/common/params/ActionTokensArgs.java  |    78 +
 .../slider/common/params/ActionUpdateArgs.java  |    32 +
 .../slider/common/params/ActionUpgradeArgs.java |    73 +
 .../slider/common/params/ActionVersionArgs.java |    46 +
 .../slider/common/params/AddonArgsDelegate.java |    54 +
 .../params/AppAndResouceOptionArgsDelegate.java |   111 +
 .../org/apache/slider/common/params/ArgOps.java |   157 +
 .../apache/slider/common/params/Arguments.java  |   162 +
 .../apache/slider/common/params/ClientArgs.java |   385 +
 .../apache/slider/common/params/CommonArgs.java |   303 +
 .../common/params/ComponentArgsDelegate.java    |    52 +
 .../common/params/DontSplitArguments.java       |    34 +
 .../common/params/LaunchArgsAccessor.java       |    30 +
 .../common/params/LaunchArgsDelegate.java       |    51 +
 .../common/params/PathArgumentConverter.java    |    34 +
 .../slider/common/params/SliderAMArgs.java      |    57 +
 .../common/params/SliderAMCreateAction.java     |    74 +
 .../slider/common/params/SliderActions.java     |   113 +
 .../common/params/URIArgumentConverter.java     |    40 +
 .../common/params/URLArgumentConverter.java     |    40 +
 .../slider/common/params/WaitArgsDelegate.java  |    42 +
 .../slider/common/params/WaitTimeAccessor.java  |    24 +
 .../apache/slider/common/tools/Comparators.java |    69 +
 .../slider/common/tools/ConfigHelper.java       |   658 +
 .../slider/common/tools/CoreFileSystem.java     |   915 +
 .../apache/slider/common/tools/Duration.java    |   109 +
 .../apache/slider/common/tools/PortScanner.java |   113 +
 .../slider/common/tools/SliderFileSystem.java   |    42 +
 .../apache/slider/common/tools/SliderUtils.java |  2548 ++
 .../slider/common/tools/SliderVersionInfo.java  |   108 +
 .../conf/AbstractInputPropertiesValidator.java  |    49 +
 .../apache/slider/core/conf/AggregateConf.java  |   198 +
 .../org/apache/slider/core/conf/ConfTree.java   |   114 +
 .../slider/core/conf/ConfTreeOperations.java    |   477 +
 .../core/conf/InputPropertiesValidator.java     |    27 +
 .../apache/slider/core/conf/MapOperations.java  |   344 +
 .../conf/ResourcesInputPropertiesValidator.java |    41 +
 .../conf/TemplateInputPropertiesValidator.java  |    38 +
 .../exceptions/BadClusterStateException.java    |    35 +
 .../BadCommandArgumentsException.java           |    30 +
 .../core/exceptions/BadConfigException.java     |    39 +
 .../slider/core/exceptions/ErrorStrings.java    |    57 +
 .../core/exceptions/ExceptionConverter.java     |   128 +
 .../core/exceptions/NoSuchNodeException.java    |    32 +
 .../core/exceptions/NotFoundException.java      |    35 +
 .../exceptions/ServiceNotReadyException.java    |    43 +
 .../slider/core/exceptions/SliderException.java |    67 +
 .../SliderInternalStateException.java           |    34 +
 .../TriggerClusterTeardownException.java        |    41 +
 .../UnknownApplicationInstanceException.java    |    51 +
 .../slider/core/exceptions/UsageException.java  |    34 +
 .../core/exceptions/WaitTimeoutException.java   |    34 +
 .../slider/core/launch/AbstractLauncher.java    |   528 +
 .../slider/core/launch/AppMasterLauncher.java   |   233 +
 .../core/launch/ClasspathConstructor.java       |   172 +
 .../slider/core/launch/CommandLineBuilder.java  |   104 +
 .../slider/core/launch/ContainerLauncher.java   |    72 +
 .../slider/core/launch/CredentialUtils.java     |   379 +
 .../core/launch/JavaCommandLineBuilder.java     |   182 +
 .../slider/core/launch/LaunchedApplication.java |   108 +
 .../slider/core/launch/RunningApplication.java  |    76 +
 .../launch/SerializedApplicationReport.java     |    98 +
 .../slider/core/main/ExitCodeProvider.java      |    32 +
 .../org/apache/slider/core/main/IrqHandler.java |   103 +
 .../slider/core/main/LauncherExitCodes.java     |   196 +
 .../org/apache/slider/core/main/RunService.java |    62 +
 .../core/main/ServiceLaunchException.java       |    73 +
 .../slider/core/main/ServiceLauncher.java       |   642 +
 .../slider/core/main/ServiceShutdownHook.java   |    80 +
 .../core/persist/AggregateConfSerDeser.java     |    55 +
 .../core/persist/AppDefinitionPersister.java    |   260 +
 .../core/persist/ApplicationReportSerDeser.java |    57 +
 .../slider/core/persist/ConfPersister.java      |   286 +
 .../slider/core/persist/ConfTreeSerDeser.java   |    54 +
 .../apache/slider/core/persist/Filenames.java   |    28 +
 .../slider/core/persist/InstancePaths.java      |    58 +
 .../slider/core/persist/JsonSerDeser.java       |   243 +
 .../persist/LockAcquireFailedException.java     |    40 +
 .../slider/core/persist/LockHeldAction.java     |    38 +
 .../apache/slider/core/persist/PersistKeys.java |    25 +
 .../core/registry/SliderRegistryUtils.java      |    62 +
 .../slider/core/registry/YarnAppListClient.java |   189 +
 .../core/registry/docstore/ConfigFormat.java    |    60 +
 .../core/registry/docstore/ConfigUtils.java     |    96 +
 .../docstore/ConfigurationResolver.java         |    24 +
 .../core/registry/docstore/ExportEntry.java     |   120 +
 .../registry/docstore/PublishedConfigSet.java   |   100 +
 .../docstore/PublishedConfiguration.java        |   196 +
 .../PublishedConfigurationOutputter.java        |   210 +
 .../registry/docstore/PublishedExports.java     |   140 +
 .../docstore/PublishedExportsOutputter.java     |   104 +
 .../registry/docstore/PublishedExportsSet.java  |    98 +
 .../slider/core/registry/docstore/UriMap.java   |    38 +
 .../registry/info/CustomRegistryConstants.java  |    57 +
 .../core/registry/retrieve/AMWebClient.java     |   158 +
 .../registry/retrieve/RegistryRetriever.java    |   183 +
 .../core/restclient/HttpOperationResponse.java  |    34 +
 .../apache/slider/core/restclient/HttpVerb.java |    57 +
 .../restclient/SliderURLConnectionFactory.java  |   176 +
 .../core/restclient/UgiJerseyBinding.java       |   154 +
 .../restclient/UrlConnectionOperations.java     |   210 +
 .../slider/core/zk/BlockingZKWatcher.java       |    67 +
 .../slider/core/zk/MiniZooKeeperCluster.java    |   423 +
 .../org/apache/slider/core/zk/ZKCallback.java   |    31 +
 .../apache/slider/core/zk/ZKIntegration.java    |   323 +
 .../apache/slider/core/zk/ZKPathBuilder.java    |    82 +
 .../apache/slider/core/zk/ZookeeperUtils.java   |   147 +
 .../providers/AbstractClientProvider.java       |   248 +
 .../providers/AbstractProviderService.java      |   424 +
 .../apache/slider/providers/MonitorDetail.java  |    43 +
 .../slider/providers/PlacementPolicy.java       |    64 +
 .../providers/PlacementPolicyOptions.java       |    26 +
 .../slider/providers/ProviderCompleted.java     |    29 +
 .../providers/ProviderCompletedCallable.java    |    38 +
 .../apache/slider/providers/ProviderCore.java   |    43 +
 .../apache/slider/providers/ProviderRole.java   |   135 +
 .../slider/providers/ProviderService.java       |   217 +
 .../apache/slider/providers/ProviderUtils.java  |   530 +
 .../slider/providers/SliderProviderFactory.java |   110 +
 .../providers/agent/AgentClientProvider.java    |   701 +
 .../slider/providers/agent/AgentKeys.java       |   109 +
 .../providers/agent/AgentLaunchParameter.java   |   130 +
 .../providers/agent/AgentProviderFactory.java   |    47 +
 .../providers/agent/AgentProviderService.java   |  3212 ++
 .../slider/providers/agent/AgentRoles.java      |    38 +
 .../slider/providers/agent/AgentUtils.java      |   134 +
 .../apache/slider/providers/agent/Command.java  |    59 +
 .../slider/providers/agent/CommandResult.java   |    40 +
 .../providers/agent/ComponentCommandOrder.java  |   181 +
 .../providers/agent/ComponentInstanceState.java |   340 +
 .../providers/agent/ComponentTagProvider.java   |   127 +
 .../slider/providers/agent/ContainerState.java  |    41 +
 .../providers/agent/HeartbeatMonitor.java       |   130 +
 .../apache/slider/providers/agent/State.java    |   199 +
 .../application/metadata/AbstractComponent.java |    80 +
 .../metadata/AbstractMetainfoParser.java        |   130 +
 .../metadata/AbstractMetainfoSchema.java        |    69 +
 .../metadata/AddonPackageMetainfoParser.java    |    53 +
 .../agent/application/metadata/Application.java |   193 +
 .../metadata/ApplicationPackage.java            |    69 +
 .../application/metadata/CommandOrder.java      |    61 +
 .../application/metadata/CommandScript.java     |    72 +
 .../agent/application/metadata/Component.java   |   217 +
 .../application/metadata/ComponentCommand.java  |    85 +
 .../application/metadata/ComponentExport.java   |    54 +
 .../metadata/ComponentsInAddonPackage.java      |    26 +
 .../agent/application/metadata/ConfigFile.java  |    59 +
 .../application/metadata/DefaultConfig.java     |    39 +
 .../metadata/DefaultConfigParser.java           |    54 +
 .../application/metadata/DockerContainer.java   |   187 +
 .../metadata/DockerContainerInputFile.java      |    50 +
 .../metadata/DockerContainerMount.java          |    60 +
 .../metadata/DockerContainerPort.java           |    66 +
 .../agent/application/metadata/Export.java      |    61 +
 .../agent/application/metadata/ExportGroup.java |    71 +
 .../agent/application/metadata/Metainfo.java    |   118 +
 .../application/metadata/MetainfoParser.java    |    97 +
 .../agent/application/metadata/OSPackage.java   |    51 +
 .../agent/application/metadata/OSSpecific.java  |    57 +
 .../agent/application/metadata/Package.java     |    60 +
 .../application/metadata/PropertyInfo.java      |    54 +
 .../agent/application/metadata/Validate.java    |    27 +
 .../org/apache/slider/providers/agent/todo.md   |    22 +
 .../slideram/SliderAMClientProvider.java        |   304 +
 .../slideram/SliderAMProviderService.java       |   189 +
 .../appmaster/AppMasterActionOperations.java    |    29 +
 .../server/appmaster/PrivilegedConnectToCM.java |    48 +
 .../appmaster/ProtobufClusterServices.java      |    36 +
 .../server/appmaster/PublishedArtifacts.java    |    31 +
 .../server/appmaster/RoleLaunchService.java     |   257 +
 .../server/appmaster/SliderAppMaster.java       |  2450 ++
 .../appmaster/actions/ActionFlexCluster.java    |    44 +
 .../server/appmaster/actions/ActionHalt.java    |    53 +
 .../appmaster/actions/ActionKillContainer.java  |    86 +
 .../actions/ActionRegisterServiceInstance.java  |    59 +
 .../appmaster/actions/ActionStartContainer.java |    62 +
 .../appmaster/actions/ActionStopQueue.java      |    56 +
 .../appmaster/actions/ActionStopSlider.java     |   162 +
 .../actions/ActionUpgradeContainers.java        |   106 +
 .../server/appmaster/actions/AsyncAction.java   |   138 +
 .../actions/EscalateOutstandingRequests.java    |    45 +
 .../actions/ProviderReportedContainerLoss.java  |    53 +
 .../actions/ProviderStartupCompleted.java       |    36 +
 .../server/appmaster/actions/QueueAccess.java   |    72 +
 .../server/appmaster/actions/QueueExecutor.java |    90 +
 .../server/appmaster/actions/QueueService.java  |   202 +
 .../actions/RegisterComponentInstance.java      |    59 +
 .../appmaster/actions/RenewingAction.java       |   141 +
 .../appmaster/actions/ResetFailureWindow.java   |    39 +
 .../actions/ReviewAndFlexApplicationSize.java   |    43 +
 .../actions/UnregisterComponentInstance.java    |    51 +
 .../server/appmaster/management/BoolMetric.java |    87 +
 .../management/BoolMetricPredicate.java         |    44 +
 .../server/appmaster/management/LongGauge.java  |    98 +
 .../management/LongMetricFunction.java          |    44 +
 .../appmaster/management/MeterAndCounter.java   |   109 +
 .../management/MetricsAndMonitoring.java        |   195 +
 .../management/MetricsBindingService.java       |   151 +
 .../appmaster/management/MetricsConstants.java  |    58 +
 .../appmaster/management/MetricsKeys.java       |    92 +
 .../management/PrefixedMetricsSet.java          |    53 +
 .../management/RangeLimitedCounter.java         |    85 +
 .../appmaster/management/RecordedEvent.java     |    58 +
 .../server/appmaster/management/Timestamp.java  |    33 +
 .../management/YarnServiceHealthCheck.java      |    38 +
 .../server/appmaster/monkey/ChaosEntry.java     |    85 +
 .../server/appmaster/monkey/ChaosKillAM.java    |    48 +
 .../appmaster/monkey/ChaosKillContainer.java    |    84 +
 .../appmaster/monkey/ChaosMonkeyService.java    |   138 +
 .../server/appmaster/monkey/ChaosTarget.java    |    24 +
 .../appmaster/monkey/MonkeyPlayAction.java      |    48 +
 .../operations/AbstractRMOperation.java         |    30 +
 .../operations/AsyncRMOperationHandler.java     |   110 +
 .../operations/CancelSingleRequest.java         |    54 +
 .../operations/ContainerReleaseOperation.java   |    47 +
 .../operations/ContainerRequestOperation.java   |    62 +
 .../ProviderNotifyingOperationHandler.java      |    55 +
 .../operations/RMOperationHandler.java          |    32 +
 .../operations/RMOperationHandlerActions.java   |    60 +
 .../slider/server/appmaster/rpc/RpcBinder.java  |   310 +
 .../appmaster/rpc/SliderAMPolicyProvider.java   |    41 +
 .../appmaster/rpc/SliderClusterProtocolPB.java  |    27 +
 .../rpc/SliderClusterProtocolPBImpl.java        |   318 +
 .../rpc/SliderClusterProtocolProxy.java         |   358 +
 .../server/appmaster/rpc/SliderIPCService.java  |   551 +
 .../appmaster/rpc/SliderRPCSecurityInfo.java    |    87 +
 .../security/SecurityConfiguration.java         |   162 +
 .../state/AbstractClusterServices.java          |    61 +
 .../slider/server/appmaster/state/AppState.java |  2489 ++
 .../appmaster/state/AppStateBindingInfo.java    |    63 +
 .../state/ContainerAllocationOutcome.java       |    44 +
 .../state/ContainerAllocationResults.java       |    50 +
 .../appmaster/state/ContainerAssignment.java    |    60 +
 .../appmaster/state/ContainerOutcome.java       |    61 +
 .../appmaster/state/ContainerPriority.java      |   109 +
 .../state/ContainerReleaseSelector.java         |    37 +
 .../MostRecentContainerReleaseSelector.java     |    51 +
 .../server/appmaster/state/NodeEntry.java       |   325 +
 .../server/appmaster/state/NodeInstance.java    |   409 +
 .../slider/server/appmaster/state/NodeMap.java  |   174 +
 .../appmaster/state/OutstandingRequest.java     |   428 +
 .../state/OutstandingRequestTracker.java        |   482 +
 .../appmaster/state/ProviderAppState.java       |   307 +
 .../server/appmaster/state/RoleHistory.java     |  1101 +
 .../appmaster/state/RoleHistoryUtils.java       |    50 +
 .../appmaster/state/RoleHostnamePair.java       |    75 +
 .../server/appmaster/state/RoleInstance.java    |   323 +
 .../server/appmaster/state/RoleStatus.java      |   563 +
 .../appmaster/state/SimpleReleaseSelector.java  |    33 +
 .../state/StateAccessForProviders.java          |   313 +
 .../server/appmaster/web/AgentService.java      |    37 +
 .../server/appmaster/web/HttpCacheHeaders.java  |    35 +
 .../appmaster/web/SliderAMController.java       |    69 +
 .../server/appmaster/web/SliderAMWebApp.java    |   108 +
 .../web/SliderDefaultWrapperServlet.java        |    48 +
 .../slider/server/appmaster/web/WebAppApi.java  |    95 +
 .../server/appmaster/web/WebAppApiImpl.java     |   128 +
 .../server/appmaster/web/layout/AppLayout.java  |    32 +
 .../web/layout/ClusterSpecificationView.java    |    32 +
 .../web/layout/ContainerStatsView.java          |    33 +
 .../appmaster/web/layout/WebUILayout.java       |    43 +
 .../appmaster/web/rest/AMWadlGenerator.java     |    72 +
 .../web/rest/AMWadlGeneratorConfig.java         |    34 +
 .../appmaster/web/rest/AMWebServices.java       |    72 +
 .../web/rest/AbstractSliderResource.java        |   157 +
 .../appmaster/web/rest/InsecureAmFilter.java    |   105 +
 .../web/rest/InsecureAmFilterInitializer.java   |   102 +
 .../server/appmaster/web/rest/RestPaths.java    |   172 +
 .../web/rest/SliderJacksonJaxbJsonProvider.java |    58 +
 .../web/rest/agent/AgentCommandType.java        |    23 +
 .../appmaster/web/rest/agent/AgentEnv.java      |   376 +
 .../appmaster/web/rest/agent/AgentResource.java |   118 +
 .../web/rest/agent/AgentRestOperations.java     |    28 +
 .../appmaster/web/rest/agent/AgentWebApp.java   |   258 +
 .../web/rest/agent/AgentWebServices.java        |    40 +
 .../appmaster/web/rest/agent/CommandReport.java |   207 +
 .../web/rest/agent/ComponentStatus.java         |   129 +
 .../appmaster/web/rest/agent/DiskInfo.java      |   128 +
 .../web/rest/agent/ExecutionCommand.java        |   310 +
 .../appmaster/web/rest/agent/HeartBeat.java     |   149 +
 .../web/rest/agent/HeartBeatResponse.java       |   147 +
 .../appmaster/web/rest/agent/HostInfo.java      |   398 +
 .../appmaster/web/rest/agent/HostStatus.java    |    63 +
 .../appmaster/web/rest/agent/Register.java      |   193 +
 .../web/rest/agent/RegistrationCommand.java     |    43 +
 .../web/rest/agent/RegistrationResponse.java    |   133 +
 .../web/rest/agent/RegistrationStatus.java      |    22 +
 .../appmaster/web/rest/agent/StatusCommand.java |   152 +
 .../ApplicationResouceContentCacheFactory.java  |    63 +
 .../rest/application/ApplicationResource.java   |   516 +
 .../application/actions/RestActionPing.java     |    50 +
 .../application/actions/RestActionStop.java     |    67 +
 .../rest/application/actions/StopResponse.java  |    29 +
 .../web/rest/application/package-info.java      |    24 +
 .../resources/AggregateModelRefresher.java      |    47 +
 .../application/resources/AppconfRefresher.java |    55 +
 .../application/resources/CachedContent.java    |   121 +
 .../application/resources/ContentCache.java     |    67 +
 .../resources/LiveComponentsRefresher.java      |    39 +
 .../resources/LiveContainersRefresher.java      |    52 +
 .../resources/LiveNodesRefresher.java           |    41 +
 .../resources/LiveResourcesRefresher.java       |    68 +
 .../resources/LiveStatisticsRefresher.java      |    39 +
 .../resources/ResourceRefresher.java            |    31 +
 .../resources/ResourceSnapshotRefresher.java    |    40 +
 .../web/rest/management/ManagementResource.java |    93 +
 .../management/resources/ActionsResource.java   |    22 +
 .../resources/AggregateConfResource.java        |    90 +
 .../management/resources/ComponentResource.java |    53 +
 .../management/resources/ConfTreeResource.java  |    69 +
 .../management/resources/ResourceFactory.java   |    47 +
 .../web/rest/publisher/PublisherResource.java   |   273 +
 .../web/rest/registry/PathEntryResource.java    |    45 +
 .../web/rest/registry/RegistryResource.java     |   151 +
 .../web/view/ClusterSpecificationBlock.java     |    55 +
 .../appmaster/web/view/ContainerStatsBlock.java |   282 +
 .../server/appmaster/web/view/IndexBlock.java   |   305 +
 .../server/appmaster/web/view/NavBlock.java     |    62 +
 .../appmaster/web/view/SliderHamletBlock.java   |    56 +
 .../slider/server/avro/LoadedRoleHistory.java   |    92 +
 .../slider/server/avro/NewerFilesFirst.java     |    43 +
 .../slider/server/avro/OlderFilesFirst.java     |    43 +
 .../slider/server/avro/RoleHistoryWriter.java   |   449 +
 .../slider/server/servicemonitor/HttpProbe.java |    82 +
 .../server/servicemonitor/LogEntryBuilder.java  |    76 +
 .../server/servicemonitor/MonitorKeys.java      |   279 +
 .../server/servicemonitor/MonitorUtils.java     |   109 +
 .../slider/server/servicemonitor/PortProbe.java |   107 +
 .../slider/server/servicemonitor/Probe.java     |   107 +
 .../servicemonitor/ProbeFailedException.java    |    32 +
 .../ProbeInterruptedException.java              |    29 +
 .../server/servicemonitor/ProbePhase.java       |    56 +
 .../servicemonitor/ProbeReportHandler.java      |    79 +
 .../server/servicemonitor/ProbeStatus.java      |   173 +
 .../server/servicemonitor/ProbeWorker.java      |   446 +
 .../server/servicemonitor/ReportingLoop.java    |   265 +
 .../servicemonitor/YarnApplicationProbe.java    |    90 +
 .../AbstractSecurityStoreGenerator.java         |    98 +
 .../services/security/CertificateManager.java   |   495 +
 .../services/security/KeystoreGenerator.java    |    64 +
 .../server/services/security/SecurityStore.java |    66 +
 .../security/SecurityStoreGenerator.java        |    40 +
 .../server/services/security/SecurityUtils.java |   256 +
 .../services/security/SignCertResponse.java     |    67 +
 .../server/services/security/SignMessage.java   |    54 +
 .../services/security/StoresGenerator.java      |    68 +
 .../services/security/TruststoreGenerator.java  |    62 +
 .../utility/AbstractSliderLaunchedService.java  |   120 +
 .../services/utility/EndOfServiceWaiter.java    |    87 +
 .../LaunchedWorkflowCompositeService.java       |   117 +
 .../services/utility/PatternValidator.java      |    61 +
 .../server/services/utility/WebAppService.java  |    69 +
 .../services/workflow/ClosingService.java       |    94 +
 .../services/workflow/ForkedProcessService.java |   301 +
 .../services/workflow/LongLivedProcess.java     |   598 +
 .../LongLivedProcessLifecycleEvent.java         |    41 +
 .../server/services/workflow/ServiceParent.java |    44 +
 .../workflow/ServiceTerminatingCallable.java    |    92 +
 .../workflow/ServiceTerminatingRunnable.java    |    72 +
 .../services/workflow/ServiceThreadFactory.java |   102 +
 .../workflow/WorkflowCallbackService.java       |   113 +
 .../workflow/WorkflowCompositeService.java      |   167 +
 .../workflow/WorkflowExecutorService.java       |   113 +
 .../services/workflow/WorkflowRpcService.java   |    76 +
 .../WorkflowScheduledExecutorService.java       |    38 +
 .../workflow/WorkflowSequenceService.java       |   306 +
 .../server/services/workflow/package-info.java  |   172 +
 .../YarnRegistryViewForProviders.java           |   270 +
 .../src/main/proto/SliderClusterMessages.proto  |   396 +
 .../src/main/proto/SliderClusterProtocol.proto  |   184 +
 .../org.apache.hadoop.security.SecurityInfo     |    15 +
 .../org/apache/slider/log4j.properties          |    52 +
 .../slider/providers/agent/conf/agent.txt       |    19 +
 .../slider/providers/agent/conf/command.json    |   168 +
 .../providers/agent/conf/command_template.json  |   168 +
 .../apache/slider/providers/agent/role-node.xml |    65 +
 .../providers/dynamic/application.properties    |    25 +
 .../providers/slideram/instance/appconf.json    |    19 +
 .../providers/slideram/instance/internal.json   |    17 +
 .../providers/slideram/instance/resources.json  |    18 +
 .../main/resources/org/apache/slider/slider.xml |    30 +
 .../src/scripts/slider_keytabs.sh               |    67 +
 .../src/scripts/yarnservice.py                  |   383 +
 .../hadoop-yarn-slider-core/src/site/site.xml   |    26 +
 .../hadoop-yarn-slider/pom.xml                  |    39 +
 .../hadoop-yarn-applications/pom.xml            |     2 +
 461 files changed, 109486 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
new file mode 100644
index 0000000..14130c5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
@@ -0,0 +1,397 @@
+<!--
+   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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-yarn-applications</artifactId>
+    <version>3.0.0-alpha2-SNAPSHOT</version>
+  </parent>
+  <groupId>org.apache.hadoop</groupId>
+  <artifactId>hadoop-yarn-slider-core</artifactId>
+  <packaging>jar</packaging>
+  <name>Apache Hadoop YARN Slider Core</name>
+
+  <build>
+    <!-- resources are filtered for dynamic updates. This gets build info in-->
+    <resources>
+      <resource>
+        <directory>src/main/resources</directory>
+        <filtering>true</filtering>
+      </resource>
+    </resources>
+    
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <environmentVariables>
+            <JAVA_HOME>${java.home}</JAVA_HOME>
+          </environmentVariables>
+        </configuration>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.avro</groupId>
+        <artifactId>avro-maven-plugin</artifactId>
+        <version>${avro.version}</version>
+        <executions>
+          <execution>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>schema</goal>
+            </goals>
+            <configuration>
+              <sourceDirectory>${project.basedir}/src/main/avro/
+              </sourceDirectory>
+              <outputDirectory>${project.build.directory}/generated-sources/java
+              </outputDirectory>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+
+    </plugins>
+  </build>
+  <dependencies>
+    <dependency>
+      <groupId>com.beust</groupId>
+      <artifactId>jcommander</artifactId>
+      <version>1.30</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-core-asl</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-jaxrs</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-mapper-asl</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-xc</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-client</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-web-proxy</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-registry</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.avro</groupId>
+      <artifactId>avro</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-compress</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>commons-digester</groupId>
+      <artifactId>commons-digester</artifactId>
+      <version>1.8</version>
+    </dependency>
+
+    <dependency>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>commons-lang</groupId>
+      <artifactId>commons-lang</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>commons-logging</groupId>
+      <artifactId>commons-logging</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.codahale.metrics</groupId>
+      <artifactId>metrics-core</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.codahale.metrics</groupId>
+      <artifactId>metrics-servlets</artifactId>
+      <version>3.0.1</version>
+    </dependency>
+
+    <!-- ======================================================== -->
+    <!-- service registry -->
+    <!-- ======================================================== -->
+
+    <dependency>
+      <groupId>org.apache.zookeeper</groupId>
+      <artifactId>zookeeper</artifactId>
+    </dependency>
+
+    <!-- ======================================================== -->
+    <!-- Jersey and webapp support -->
+    <!-- ======================================================== -->
+
+    <dependency>
+      <groupId>javax.servlet</groupId>
+      <artifactId>servlet-api</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>javax.xml.bind</groupId>
+      <artifactId>jaxb-api</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.sun.jersey</groupId>
+      <artifactId>jersey-client</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.sun.jersey</groupId>
+      <artifactId>jersey-json</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.sun.jersey</groupId>
+      <artifactId>jersey-server</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.inject</groupId>
+      <artifactId>guice</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.code.gson</groupId>
+      <artifactId>gson</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.inject.extensions</groupId>
+      <artifactId>guice-servlet</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.sun.jersey.contribs</groupId>
+      <artifactId>jersey-guice</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.mortbay.jetty</groupId>
+      <artifactId>jetty</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.mortbay.jetty</groupId>
+      <artifactId>jetty-util</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.mortbay.jetty</groupId>
+      <artifactId>jetty-sslengine</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>javax.servlet.jsp</groupId>
+      <artifactId>jsp-api</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.codehaus.jettison</groupId>
+      <artifactId>jettison</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.mortbay.jetty</groupId>
+      <artifactId>jetty-sslengine</artifactId>
+      <version>6.1.26</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.yaml</groupId>
+      <artifactId>snakeyaml</artifactId>
+      <version>1.16</version>
+      <scope>compile</scope>
+    </dependency>
+
+  </dependencies>
+
+
+  <profiles>
+    <profile>
+      <id>compile-protobuf</id>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-maven-plugins</artifactId>
+            <executions>
+              <execution>
+                <id>compile-protoc</id>
+                <phase>generate-sources</phase>
+                <goals>
+                  <goal>protoc</goal>
+                </goals>
+                <configuration>
+                  <protocVersion>${protobuf.version}</protocVersion>
+                  <protocCommand>protoc</protocCommand>
+                  <imports>
+                    <param>${basedir}/src/main/proto</param>
+                  </imports>
+                  <source>
+                    <directory>${basedir}/src/main/proto</directory>
+                    <includes>
+                      <include>SliderClusterMessages.proto</include>
+                      <include>SliderClusterProtocol.proto</include>
+                    </includes>
+                  </source>
+                  <output>${basedir}/src/main/java</output>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+
+
+    <profile>
+      <id>rat</id>
+      <build>
+        <plugins>
+
+          <plugin>
+            <groupId>org.apache.rat</groupId>
+            <artifactId>apache-rat-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>check-licenses</id>
+                <goals>
+                  <goal>check</goal>
+                </goals>
+              </execution>
+            </executions>
+            <configuration>
+              <excludes>
+                <exclude>**/*.json</exclude>
+                <exclude>src/test/python/agent.ini</exclude>
+                <exclude>src/test/python/version</exclude>
+                <exclude>**/THIRD-PARTY.properties</exclude>
+                <exclude>src/main/resources/webapps/slideram/.keep</exclude>
+                <exclude>src/main/resources/webapps/slideragent/.keep</exclude>
+                <exclude>src/main/resources/webapps/static/yarn.dt.plugins.js</exclude>
+                <!-- jQuery DataTables files (BSD license) -->
+                <exclude>src/main/resources/webapps/static/dt-1.9.4/**</exclude>
+                <!-- jQuery (MIT license) -->
+                <exclude>src/main/resources/webapps/static/jquery/jquery-1.8.2.min.js</exclude>
+                <!-- jQuery UI (MIT license) -->
+                <exclude>src/main/resources/webapps/static/jquery/jquery-ui-1.9.1.custom.min.js</exclude>
+                <exclude>src/main/resources/webapps/static/jquery/themes-1.9.1/base/jquery-ui.css</exclude>
+                <!-- jQuery jsTree (MIT license) -->
+                <exclude>src/main/resources/webapps/static/jt/jquery.jstree.js</exclude>
+                <!-- protobuf generated classes -->
+                <exclude>src/main/java/org/apache/slider/api/proto/Messages.java</exclude>
+                <exclude>src/main/java/org/apache/slider/api/proto/SliderClusterAPI.java</exclude>
+                <exclude>src/test/app_packages/test_am_config/resources/test.template</exclude>
+                <exclude>src/test/app_packages/test_am_config/test_archive/testfile</exclude>
+              </excludes>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+
+  </profiles>
+
+</project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/assembly/executable-jar.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/assembly/executable-jar.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/assembly/executable-jar.xml
new file mode 100644
index 0000000..23383c8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/assembly/executable-jar.xml
@@ -0,0 +1,47 @@
+<!--
+  ~ 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.
+  -->
+
+<assembly
+  xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0 http://maven.apache.org/xsd/assembly-1.1.0.xsd">
+  <id>distribution</id>
+  <formats>
+    <format>zip</format>
+  </formats>
+  <includeBaseDirectory>true</includeBaseDirectory>
+  <fileSets>
+    <fileSet>
+      <directory>${project.build.directory}</directory>
+      <outputDirectory>/</outputDirectory>
+      <includes>
+        <include>*.jar</include>
+      </includes>
+    </fileSet>
+  </fileSets>
+  <dependencySets>
+    <dependencySet>
+      <scope>runtime</scope>
+      <outputDirectory>/lib</outputDirectory>
+      <!-- dont copy JAR into /lib-->
+      <useProjectArtifact>false</useProjectArtifact>
+   <!--   <includeBaseDirectory>false</includeBaseDirectory>-->
+      <unpack>false</unpack>
+    </dependencySet>
+  </dependencySets>
+</assembly>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/license/THIRD-PARTY.properties
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/license/THIRD-PARTY.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/license/THIRD-PARTY.properties
new file mode 100644
index 0000000..1abd56e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/license/THIRD-PARTY.properties
@@ -0,0 +1,33 @@
+# Generated by org.codehaus.mojo.license.AddThirdPartyMojo
+#-------------------------------------------------------------------------------
+# Already used licenses in project :
+# - Apache License
+# - BSD
+# - CDDL + GPLv2 with classpath exception
+# - CDDL 1.1
+# - CDDL License
+# - CDDL+GPL
+# - Common Public License Version 1.0
+# - Eclipse Public License - Version 1.0
+# - GNU Lesser General Public License (LGPL), Version 2.1
+# - GNU Lesser General Public License, Version 2.1
+# - GPL2 w/ CPE
+# - MIT License
+# - MPL 1.1
+# - New BSD License
+# - Public Domain
+# - Revised BSD
+# - The Apache Software License, Version 2.0
+# - The BSD 3-Clause License
+# - The BSD License
+# - The MIT License
+#-------------------------------------------------------------------------------
+# Please fill the missing licenses for dependencies :
+#
+#
+#Thu Oct 15 16:45:02 EDT 2015
+commons-beanutils--commons-beanutils--1.7.0=The Apache Software License, Version 2.0
+javax.servlet--servlet-api--2.5=CDDL License
+javax.servlet.jsp--jsp-api--2.1=CDDL License
+org.apache.zookeeper--zookeeper--3.4.6=The Apache Software License, Version 2.0
+org.codehaus.jettison--jettison--1.1=The Apache Software License, Version 2.0

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/avro/org/apache/slider/server/avro/RoleHistoryRecord.avsc
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/avro/org/apache/slider/server/avro/RoleHistoryRecord.avsc b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/avro/org/apache/slider/server/avro/RoleHistoryRecord.avsc
new file mode 100644
index 0000000..3667c01
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/avro/org/apache/slider/server/avro/RoleHistoryRecord.avsc
@@ -0,0 +1,114 @@
+// 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.
+
+[
+
+  {
+    "type": "record",
+    "name": "NodeEntryRecord",
+    "namespace": "org.apache.slider.server.avro",
+    "fields": [
+      {
+        "name": "host",
+        "type": "string"
+      },
+      {
+        "name": "role",
+        "type": "int"
+      },
+      {
+        "name": "active",
+        "type": "boolean"
+      },
+      {
+        "name": "last_used",
+        "type": "long"
+      }
+    ]
+  },
+
+  {
+    "type": "record",
+    "name": "RoleHistoryHeader",
+    "namespace": "org.apache.slider.server.avro",
+    "fields": [
+      {
+        "name": "version",
+        "type": "int"
+      },
+      {
+        "name": "saved",
+        "type": "long"
+      },
+      {
+        "name": "savedx",
+        "type": "string"
+      },
+      {
+        "name": "savedate",
+        "type": "string",
+        "default": ""
+      },
+      {
+        "name": "roles",
+        "type": "int"
+      }
+    ]
+  },
+  {
+    "type": "record",
+    "name": "RoleHistoryMapping",
+    "namespace": "org.apache.slider.server.avro",
+    "fields": [
+      {
+        "name": "rolemap",
+        "type": {
+          "type": "map",
+          "values": "int"
+         }
+      }
+    ]
+  },
+  {
+    "type": "record",
+    "name": "RoleHistoryFooter",
+    "namespace": "org.apache.slider.server.avro",
+    "fields": [
+      {
+        "name": "count",
+        "type": "long"
+      }
+    ]
+  },
+
+  {
+    "type": "record",
+    "name": "RoleHistoryRecord",
+    "namespace": "org.apache.slider.server.avro",
+    "fields": [
+      {
+        "name": "entry",
+        "type": [
+          "org.apache.slider.server.avro.NodeEntryRecord",
+          "org.apache.slider.server.avro.RoleHistoryHeader",
+          "org.apache.slider.server.avro.RoleHistoryFooter",
+          "org.apache.slider.server.avro.RoleHistoryMapping"
+        ]
+      }
+    ]
+  }
+
+]

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/security/KerberosDiags.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/security/KerberosDiags.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/security/KerberosDiags.java
new file mode 100644
index 0000000..8c572b3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/hadoop/security/KerberosDiags.java
@@ -0,0 +1,676 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.security;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.util.ExitUtil;
+import org.apache.hadoop.util.Shell;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.crypto.Cipher;
+import java.io.Closeable;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.security.NoSuchAlgorithmException;
+import java.util.Collection;
+import java.util.Date;
+import java.util.List;
+import java.util.regex.Pattern;
+
+import static org.apache.hadoop.security.UserGroupInformation.*;
+import static org.apache.hadoop.security.authentication.util.KerberosUtil.*;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.*;
+
+/**
+ * Kerberos diagnostics
+ * At some point this may move to hadoop core, so please keep use of slider
+ * methods and classes to ~0.
+ *
+ * This operation expands some of the diagnostic output of the security code,
+ * but not all. For completeness
+ *
+ * Set the environment variable {@code HADOOP_JAAS_DEBUG=true}
+ * Set the log level for {@code org.apache.hadoop.security=DEBUG}
+ */
+public class KerberosDiags implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(KerberosDiags.class);
+  public static final String KRB5_CCNAME = "KRB5CCNAME";
+  public static final String JAVA_SECURITY_KRB5_CONF
+    = "java.security.krb5.conf";
+  public static final String JAVA_SECURITY_KRB5_REALM
+    = "java.security.krb5.realm";
+  public static final String SUN_SECURITY_KRB5_DEBUG
+    = "sun.security.krb5.debug";
+  public static final String SUN_SECURITY_SPNEGO_DEBUG
+    = "sun.security.spnego.debug";
+  public static final String SUN_SECURITY_JAAS_FILE
+    = "java.security.auth.login.config";
+  public static final String KERBEROS_KINIT_COMMAND
+    = "hadoop.kerberos.kinit.command";
+  public static final String HADOOP_AUTHENTICATION_IS_DISABLED
+      = "Hadoop authentication is disabled";
+  public static final String UNSET = "(unset)";
+  public static final String NO_DEFAULT_REALM = "Cannot locate default realm";
+
+  private final Configuration conf;
+  private final List<String> services;
+  private final PrintWriter out;
+  private final File keytab;
+  private final String principal;
+  private final long minKeyLength;
+  private final boolean securityRequired;
+
+  public static final String CAT_JVM = "JVM";
+  public static final String CAT_JAAS = "JAAS";
+  public static final String CAT_CONFIG = "CONFIG";
+  public static final String CAT_LOGIN = "LOGIN";
+  public static final String CAT_KERBEROS = "KERBEROS";
+  public static final String CAT_SASL = "SASL";
+
+  @SuppressWarnings("IOResourceOpenedButNotSafelyClosed")
+  public KerberosDiags(Configuration conf,
+      PrintWriter out,
+      List<String> services,
+      File keytab,
+      String principal,
+      long minKeyLength,
+      boolean securityRequired) {
+    this.conf = conf;
+    this.services = services;
+    this.keytab = keytab;
+    this.principal = principal;
+    this.out = out;
+    this.minKeyLength = minKeyLength;
+    this.securityRequired = securityRequired;
+  }
+
+  @Override
+  public void close() throws IOException {
+    flush();
+  }
+
+  /**
+   * Execute diagnostics.
+   * <p>
+   * Things it would be nice if UGI made accessible
+   * <ol>
+   *   <li>A way to enable JAAS debug programatically</li>
+   *   <li>Access to the TGT</li>
+   * </ol>
+   * @return true if security was enabled and all probes were successful
+   * @throws KerberosDiagsFailure explicitly raised failure
+   * @throws Exception other security problems
+   */
+  @SuppressWarnings("deprecation")
+  public boolean execute() throws Exception {
+
+    title("Kerberos Diagnostics scan at %s",
+        new Date(System.currentTimeMillis()));
+
+    // check that the machine has a name
+    println("Hostname: %s",
+        InetAddress.getLocalHost().getCanonicalHostName());
+
+    // Fail fast on a JVM without JCE installed.
+    validateKeyLength();
+
+    // look at realm
+    println("JVM Kerberos Login Module = %s", getKrb5LoginModuleName());
+    printDefaultRealm();
+
+    title("System Properties");
+    for (String prop : new String[]{
+      JAVA_SECURITY_KRB5_CONF,
+      JAVA_SECURITY_KRB5_REALM,
+      SUN_SECURITY_KRB5_DEBUG,
+      SUN_SECURITY_SPNEGO_DEBUG,
+      SUN_SECURITY_JAAS_FILE
+    }) {
+      printSysprop(prop);
+    }
+
+    title("Environment Variables");
+    for (String env : new String[]{
+      "HADOOP_JAAS_DEBUG",
+      KRB5_CCNAME,
+      HADOOP_USER_NAME,
+      HADOOP_PROXY_USER,
+      HADOOP_TOKEN_FILE_LOCATION,
+    }) {
+      printEnv(env);
+    }
+
+    for (String prop : new String[]{
+      KERBEROS_KINIT_COMMAND,
+      HADOOP_SECURITY_AUTHENTICATION,
+      HADOOP_SECURITY_AUTHORIZATION,
+      "hadoop.kerberos.min.seconds.before.relogin",    // not in 2.6
+      "hadoop.security.dns.interface",   // not in 2.6
+      "hadoop.security.dns.nameserver",  // not in 2.6
+      HADOOP_RPC_PROTECTION,
+      HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS,
+      HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX,
+      HADOOP_SECURITY_GROUP_MAPPING,
+      "hadoop.security.impersonation.provider.class",    // not in 2.6
+      "dfs.data.transfer.protection" // HDFS
+    }) {
+      printConfOpt(prop);
+    }
+
+    // check that authentication is enabled
+    if (SecurityUtil.getAuthenticationMethod(conf)
+        .equals(AuthenticationMethod.SIMPLE)) {
+      println(HADOOP_AUTHENTICATION_IS_DISABLED);
+      failif(securityRequired, CAT_CONFIG, HADOOP_AUTHENTICATION_IS_DISABLED);
+      // no security, skip rest of test
+      return false;
+    }
+
+    validateKrb5File();
+    validateSasl(HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS);
+    validateSasl("dfs.data.transfer.saslproperties.resolver.class");
+    validateKinitExecutable();
+    validateJAAS();
+    // now the big test: login, then try again
+    boolean krb5Debug = getAndSet(SUN_SECURITY_KRB5_DEBUG);
+    boolean spnegoDebug = getAndSet(SUN_SECURITY_SPNEGO_DEBUG);
+    try {
+      title("Logging in");
+
+      if (keytab != null) {
+        dumpKeytab(keytab);
+        loginFromKeytab();
+      } else {
+        UserGroupInformation loginUser = getLoginUser();
+        dumpUGI("Log in user", loginUser);
+        validateUGI("Login user", loginUser);
+        println("Ticket based login: %b", isLoginTicketBased());
+        println("Keytab based login: %b", isLoginKeytabBased());
+      }
+
+      return true;
+    } finally {
+      // restore original system properties
+      System.setProperty(SUN_SECURITY_KRB5_DEBUG,
+        Boolean.toString(krb5Debug));
+      System.setProperty(SUN_SECURITY_SPNEGO_DEBUG,
+        Boolean.toString(spnegoDebug));
+    }
+  }
+
+  /**
+   * Fail fast on a JVM without JCE installed.
+   *
+   * This is a recurrent problem
+   * (that is: it keeps creeping back with JVM updates);
+   * a fast failure is the best tactic
+   * @throws NoSuchAlgorithmException
+   */
+
+  protected void validateKeyLength() throws NoSuchAlgorithmException {
+    int aesLen = Cipher.getMaxAllowedKeyLength("AES");
+    println("Maximum AES encryption key length %d bits", aesLen);
+    failif (aesLen < minKeyLength,
+        CAT_JVM,
+        "Java Cryptography Extensions are not installed on this JVM."
+        +" Maximum supported key length %s - minimum required %d",
+        aesLen, minKeyLength);
+  }
+
+  /**
+   * Get the default realm.
+   * <p>
+   * Not having a default realm may be harmless, so is noted at info.
+   * All other invocation failures are downgraded to warn, as
+   * follow-on actions may still work.
+   * failure to invoke the method via introspection is rejected,
+   * as it's a sign of JVM compatibility issues that may have other
+   * consequences
+   */
+  protected void printDefaultRealm() {
+    try {
+      println("Default Realm = %s",
+          getDefaultRealm());
+    } catch (ClassNotFoundException
+        | IllegalAccessException
+        | NoSuchMethodException e) {
+
+      throw new KerberosDiagsFailure(CAT_JVM, e,
+          "Failed to invoke krb5.Config.getDefaultRealm: %s", e);
+    } catch (InvocationTargetException e) {
+      Throwable cause = e.getCause() != null ? e.getCause() : e;
+      if (cause.toString().contains(NO_DEFAULT_REALM)) {
+        // exception raised if there is no default realm. This is not
+        // always a problem, so downgrade to a message.
+        println("Host has no default realm");
+        LOG.debug(cause.toString(), cause);
+      } else {
+        println("Kerberos.getDefaultRealm() failed: %s\n%s",
+            cause,
+            org.apache.hadoop.util.StringUtils.stringifyException(cause));
+      }
+    }
+  }
+
+  /**
+   * Locate the krb5.conf file and dump it.
+   * No-op on windows.
+   * @throws IOException
+   */
+  private void validateKrb5File() throws IOException {
+    if (!Shell.WINDOWS) {
+      title("Locating Kerberos configuration file");
+      String krbPath = "/etc/krb5.conf";
+      String jvmKrbPath = System.getProperty(JAVA_SECURITY_KRB5_CONF);
+      if (jvmKrbPath != null) {
+        println("Setting kerberos path from sysprop %s: %s",
+          JAVA_SECURITY_KRB5_CONF, jvmKrbPath);
+        krbPath = jvmKrbPath;
+      }
+
+      String krb5name = System.getenv(KRB5_CCNAME);
+      if (krb5name != null) {
+        println("Setting kerberos path from environment variable %s: %s",
+          KRB5_CCNAME, krb5name);
+        krbPath = krb5name;
+        if (jvmKrbPath != null) {
+          println("Warning - both %s and %s were set - %s takes priority",
+            JAVA_SECURITY_KRB5_CONF, KRB5_CCNAME, KRB5_CCNAME);
+        }
+      }
+
+      File krbFile = new File(krbPath);
+      println("Kerberos configuration file = %s", krbFile);
+      failif(!krbFile.exists(),
+          CAT_KERBEROS,
+          "Kerberos configuration file %s not found", krbFile);
+      dump(krbFile);
+    }
+  }
+
+  /**
+   * Dump a keytab: list all principals.
+   * @param keytabFile the keytab file
+   * @throws IOException IO problems
+   */
+  public void dumpKeytab(File keytabFile) throws IOException {
+    title("Examining keytab %s", keytabFile);
+    File kt = keytabFile.getCanonicalFile();
+    failif(!kt.exists(), CAT_CONFIG, "Keytab not found: %s", kt);
+    failif(!kt.isFile(), CAT_CONFIG, "Keytab is not a valid file: %s", kt);
+
+    String[] names = getPrincipalNames(keytabFile.getCanonicalPath(),
+        Pattern.compile(".*"));
+    println("keytab entry count: %d", names.length);
+    for (String name : names) {
+      println("    %s", name);
+    }
+    println("-----");
+  }
+
+  /**
+   * Log in from a keytab, dump the UGI, validate it, then try and log in again.
+   * That second-time login catches JVM/Hadoop compatibility problems.
+   * @throws IOException
+   */
+  private void loginFromKeytab() throws IOException {
+    UserGroupInformation ugi;
+    String identity;
+    if (keytab != null) {
+      File kt = keytab.getCanonicalFile();
+      println("Using keytab %s principal %s", kt, principal);
+      identity = principal;
+
+      failif(StringUtils.isEmpty(principal), CAT_KERBEROS,
+          "No principal defined");
+      ugi = loginUserFromKeytabAndReturnUGI(principal, kt.getPath());
+      dumpUGI(identity, ugi);
+      validateUGI(principal, ugi);
+
+      title("Attempting to log in from keytab again");
+      // package scoped -hence the reason why this class must be in the
+      // hadoop.security package
+      setShouldRenewImmediatelyForTests(true);
+      // attempt a new login
+      ugi.reloginFromKeytab();
+    } else {
+      println("No keytab: logging is as current user");
+    }
+  }
+
+  /**
+   * Dump a UGI.
+   * @param title title of this section
+   * @param ugi UGI to dump
+   * @throws IOException
+   */
+  private void dumpUGI(String title, UserGroupInformation ugi)
+    throws IOException {
+    title(title);
+    println("UGI instance = %s", ugi);
+    println("Has kerberos credentials: %b", ugi.hasKerberosCredentials());
+    println("Authentication method: %s", ugi.getAuthenticationMethod());
+    println("Real Authentication method: %s",
+      ugi.getRealAuthenticationMethod());
+    title("Group names");
+    for (String name : ugi.getGroupNames()) {
+      println(name);
+    }
+    title("Credentials");
+    Credentials credentials = ugi.getCredentials();
+    List<Text> secretKeys = credentials.getAllSecretKeys();
+    title("Secret keys");
+    if (!secretKeys.isEmpty()) {
+      for (Text secret: secretKeys) {
+        println("%s", secret);
+      }
+    } else {
+      println("(none)");
+    }
+
+    dumpTokens(ugi);
+  }
+
+  /**
+   * Validate the UGI: verify it is kerberized.
+   * @param messagePrefix message in exceptions
+   * @param user user to validate
+   */
+  private void validateUGI(String messagePrefix, UserGroupInformation user) {
+    failif(!user.hasKerberosCredentials(),
+        CAT_LOGIN, "%s: No kerberos credentials for %s", messagePrefix, user);
+    failif(user.getAuthenticationMethod() == null,
+        CAT_LOGIN, "%s: Null AuthenticationMethod for %s", messagePrefix, user);
+  }
+
+  /**
+   * A cursory look at the {@code kinit} executable.
+   * If it is an absolute path: it must exist with a size > 0.
+   * If it is just a command, it has to be on the path. There's no check
+   * for that -but the PATH is printed out.
+   */
+  private void validateKinitExecutable() {
+    String kinit = conf.getTrimmed(KERBEROS_KINIT_COMMAND, "");
+    if (!kinit.isEmpty()) {
+      File kinitPath = new File(kinit);
+      println("%s = %s", KERBEROS_KINIT_COMMAND, kinitPath);
+      if (kinitPath.isAbsolute()) {
+        failif(!kinitPath.exists(), CAT_KERBEROS,
+            "%s executable does not exist: %s",
+            KERBEROS_KINIT_COMMAND, kinitPath);
+        failif(!kinitPath.isFile(), CAT_KERBEROS,
+            "%s path does not refer to a file: %s",
+            KERBEROS_KINIT_COMMAND, kinitPath);
+        failif(kinitPath.length() == 0, CAT_KERBEROS,
+            "%s file is empty: %s",
+            KERBEROS_KINIT_COMMAND, kinitPath);
+      } else {
+        println("Executable %s is relative -must be on the PATH", kinit);
+        printEnv("PATH");
+      }
+    }
+  }
+
+  /**
+   * Try to load the SASL resolver.
+   * @param saslPropsResolverKey key for the SASL resolver
+   */
+  private void validateSasl(String saslPropsResolverKey) {
+    title("Resolving SASL property %s", saslPropsResolverKey);
+    String saslPropsResolver = conf.getTrimmed(saslPropsResolverKey);
+    try {
+      Class<? extends SaslPropertiesResolver> resolverClass = conf.getClass(
+          saslPropsResolverKey,
+          SaslPropertiesResolver.class, SaslPropertiesResolver.class);
+      println("Resolver is %s", resolverClass);
+    } catch (RuntimeException e) {
+      throw new KerberosDiagsFailure(CAT_SASL, e,
+          "Failed to load %s class %s",
+          saslPropsResolverKey, saslPropsResolver);
+    }
+  }
+
+  /**
+   * Validate any JAAS entry referenced in the {@link #SUN_SECURITY_JAAS_FILE}
+   * property.
+   */
+  private void validateJAAS() {
+    String jaasFilename = System.getProperty(SUN_SECURITY_JAAS_FILE);
+    if (jaasFilename != null) {
+      title("JAAS");
+      File jaasFile = new File(jaasFilename);
+      println("JAAS file is defined in %s: %s",
+          SUN_SECURITY_JAAS_FILE, jaasFile);
+      failif(!jaasFile.exists(), CAT_JAAS,
+          "JAAS file does not exist: %s", jaasFile);
+      failif(!jaasFile.isFile(), CAT_JAAS,
+          "Specified JAAS file is not a file: %s", jaasFile);
+    }
+  }
+
+  /**
+   * Dump all tokens of a user
+   * @param user user
+   */
+  public void dumpTokens(UserGroupInformation user) {
+    Collection<Token<? extends TokenIdentifier>> tokens
+      = user.getCredentials().getAllTokens();
+    title("Token Count: %d", tokens.size());
+    for (Token<? extends TokenIdentifier> token : tokens) {
+      println("Token %s", token.getKind());
+    }
+  }
+
+  /**
+   * Set the System property to true; return the old value for caching
+   * @param sysprop property
+   * @return the previous value
+   */
+  private boolean getAndSet(String sysprop) {
+    boolean old = Boolean.getBoolean(sysprop);
+    System.setProperty(sysprop, "true");
+    return old;
+  }
+
+  /**
+   * Flush all active output channels, including {@Code System.err},
+   * so as to stay in sync with any JRE log messages.
+   */
+  private void flush() {
+    if (out != null) {
+      out.flush();
+    } else {
+      System.out.flush();
+    }
+    System.err.flush();
+  }
+
+  /**
+   * Format and print a line of output.
+   * This goes to any output file, or
+   * is logged at info. The output is flushed before and after, to
+   * try and stay in sync with JRE logging.
+   * @param format format string
+   * @param args any arguments
+   */
+  @VisibleForTesting
+  public void println(String format, Object... args) {
+    println(format(format, args));
+  }
+
+  /**
+   * Print a line of output. This goes to any output file, or
+   * is logged at info. The output is flushed before and after, to
+   * try and stay in sync with JRE logging.
+   * @param msg message string
+   */
+  @VisibleForTesting
+  private void println(String msg) {
+    flush();
+    if (out != null) {
+      out.println(msg);
+    } else {
+      LOG.info(msg);
+    }
+    flush();
+  }
+
+  /**
+   * Print a title entry
+   * @param format format string
+   * @param args any arguments
+   */
+  private void title(String format, Object... args) {
+    println("");
+    println("");
+    String msg = "== " + format(format, args) + " ==";
+    println(msg);
+    println("");
+  }
+
+  /**
+   * Print a system property, or {@link #UNSET} if unset.
+   * @param property property to print
+   */
+  private void printSysprop(String property) {
+    println("%s = \"%s\"", property,
+        System.getProperty(property, UNSET));
+  }
+
+  /**
+   * Print a configuration option, or {@link #UNSET} if unset.
+   * @param option option to print
+   */
+  private void printConfOpt(String option) {
+    println("%s = \"%s\"", option, conf.get(option, UNSET));
+  }
+
+  /**
+   * Print an environment variable's name and value; printing
+   * {@link #UNSET} if it is not set
+   * @param variable environment variable
+   */
+  private void printEnv(String variable) {
+    String env = System.getenv(variable);
+    println("%s = \"%s\"", variable, env != null ? env : UNSET);
+  }
+
+  /**
+   * Dump any file to standard out; add a trailing newline
+   * @param file file to dump
+   * @throws IOException IO problems
+   */
+  public void dump(File file) throws IOException {
+    try (FileInputStream in = new FileInputStream(file)) {
+      for (String line : IOUtils.readLines(in)) {
+        println("%s", line);
+      }
+    }
+    println("");
+  }
+
+  /**
+   * Format and raise a failure
+   *
+   * @param category category for exception
+   * @param message string formatting message
+   * @param args any arguments for the formatting
+   * @throws KerberosDiagsFailure containing the formatted text
+   */
+  private void fail(String category, String message, Object... args)
+    throws KerberosDiagsFailure {
+    throw new KerberosDiagsFailure(category, message, args);
+  }
+
+  /**
+   * Conditional failure with string formatted arguments
+   * @param condition failure condition
+   * @param category category for exception
+   * @param message string formatting message
+   * @param args any arguments for the formatting
+   * @throws KerberosDiagsFailure containing the formatted text
+   *         if the condition was met
+   */
+  private void failif(boolean condition,
+      String category,
+      String message,
+      Object... args)
+    throws KerberosDiagsFailure {
+    if (condition) {
+      fail(category, message, args);
+    }
+  }
+
+  /**
+   * Format a string, treating a call where there are no varags values
+   * as a string to pass through unformatted.
+   * @param message message, which is either a format string + args, or
+   * a general string
+   * @param args argument array
+   * @return a string for printing.
+   */
+  public static String format(String message, Object... args) {
+    if (args.length == 0) {
+      return message;
+    } else {
+      return String.format(message, args);
+    }
+  }
+
+  /**
+   * Diagnostics failures return the exit code 41, "unauthorized".
+   *
+   * They have a category, initially for testing: the category can be
+   * validated without having to match on the entire string.
+   */
+  public static class KerberosDiagsFailure extends ExitUtil.ExitException {
+    private final String category;
+
+    public KerberosDiagsFailure(String category, String message) {
+      super(41, category + ": " + message);
+      this.category = category;
+    }
+
+    public KerberosDiagsFailure(String category, String message, Object... args) {
+      this(category, format(message, args));
+    }
+
+    public KerberosDiagsFailure(String category, Throwable throwable,
+        String message, Object... args) {
+      this(category, message, args);
+      initCause(throwable);
+    }
+
+    public String getCategory() {
+      return category;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/Slider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/Slider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/Slider.java
new file mode 100644
index 0000000..5fc8618
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/Slider.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider;
+
+import org.apache.slider.client.SliderClient;
+import org.apache.slider.core.main.ServiceLauncher;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * This is just the entry point class
+ */
+public class Slider extends SliderClient {
+
+
+  public static final String SERVICE_CLASSNAME = "org.apache.slider.Slider";
+
+  /**
+   * This is the main entry point for the service launcher.
+   * @param args command line arguments.
+   */
+  public static void main(String[] args) {
+    
+    //turn the args to a list
+    List<String> argsList = Arrays.asList(args);
+    //create a new list, as the ArrayList type doesn't push() on an insert
+    List<String> extendedArgs = new ArrayList<String>(argsList);
+    //insert the service name
+    extendedArgs.add(0, SERVICE_CLASSNAME);
+    //now have the service launcher do its work
+    ServiceLauncher.serviceMain(extendedArgs);
+  }
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[51/76] [abbrv] hadoop git commit: YARN-5828. Native services client errors out when config formats are uppercase. Contributed by Billie Rinaldi

Posted by ji...@apache.org.
YARN-5828. Native services client errors out when config formats are uppercase. Contributed by Billie Rinaldi


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

Branch: refs/heads/yarn-native-services
Commit: 2d644dfbddb601c4576634cbe6e92c7ae1cfc3f6
Parents: fc6901a
Author: Gour Saha <go...@apache.org>
Authored: Thu Nov 3 18:15:44 2016 -0700
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 .../org/apache/slider/core/registry/docstore/ConfigFormat.java   | 4 +++-
 .../src/main/java/org/apache/slider/providers/ProviderUtils.java | 2 +-
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d644dfb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigFormat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigFormat.java
index ddab606..723b975 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigFormat.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigFormat.java
@@ -18,6 +18,8 @@
 
 package org.apache.slider.core.registry.docstore;
 
+import java.util.Locale;
+
 public enum ConfigFormat {
 
   JSON("json"),
@@ -51,7 +53,7 @@ public enum ConfigFormat {
    */
   public static ConfigFormat resolve(String type) {
     for (ConfigFormat format: values()) {
-      if (format.getSuffix().equals(type)) {
+      if (format.getSuffix().equals(type.toLowerCase(Locale.ENGLISH))) {
         return format;
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d644dfb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java
index c5e6782..39986c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java
@@ -611,7 +611,7 @@ public class ProviderUtils implements RoleKeys, SliderKeys {
       }
       ConfigFormat configFormat = ConfigFormat.resolve(configFileType);
       if (configFormat == null) {
-        throw new BadConfigException("Config format " + configFormat +
+        throw new BadConfigException("Config format " + configFileType +
             " doesn't exist");
       }
       localizeConfigFile(launcher, roleName, roleGroup, configEntry.getKey(),


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[16/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
new file mode 100644
index 0000000..67d3647
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers.slideram;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.registry.client.binding.RegistryTypeUtils;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.tools.ConfigHelper;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.core.launch.ContainerLauncher;
+import org.apache.slider.core.registry.docstore.PublishedConfiguration;
+import org.apache.slider.core.registry.info.CustomRegistryConstants;
+import org.apache.slider.providers.AbstractProviderService;
+import org.apache.slider.providers.ProviderCore;
+import org.apache.slider.providers.ProviderRole;
+import org.apache.slider.providers.agent.AgentKeys;
+import org.apache.slider.server.appmaster.PublishedArtifacts;
+import org.apache.slider.server.appmaster.web.rest.RestPaths;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.*;
+
+/**
+ * Exists just to move some functionality out of AppMaster into a peer class
+ * of the actual service provider doing the real work
+ */
+public class SliderAMProviderService extends AbstractProviderService implements
+    ProviderCore,
+    AgentKeys,
+    SliderKeys {
+
+  public SliderAMProviderService() {
+    super("SliderAMProviderService");
+  }
+
+  @Override
+  public String getHumanName() {
+    return "Slider Application";
+  }
+  
+  @Override
+  public Configuration loadProviderConfigurationInformation(File confDir) throws
+      BadCommandArgumentsException,
+      IOException {
+    return null;
+  }
+
+  @Override
+  public void buildContainerLaunchContext(ContainerLauncher containerLauncher,
+      AggregateConf instanceDefinition,
+      Container container,
+      ProviderRole role,
+      SliderFileSystem sliderFileSystem,
+      Path generatedConfPath,
+      MapOperations resourceComponent,
+      MapOperations appComponent,
+      Path containerTmpDirPath) throws IOException, SliderException {
+  }
+
+  @Override
+  public List<ProviderRole> getRoles() {
+    return new ArrayList<>(0);
+  }
+
+  @Override
+  public void validateInstanceDefinition(AggregateConf instanceDefinition) throws
+      SliderException {
+
+  }
+
+  @Override
+  public void applyInitialRegistryDefinitions(URL amWebURI,
+      URL agentOpsURI,
+      URL agentStatusURI,
+      ServiceRecord serviceRecord)
+      throws IOException {
+    super.applyInitialRegistryDefinitions(amWebURI,
+        agentOpsURI,
+        agentStatusURI,
+        serviceRecord);
+    // now publish site.xml files
+    YarnConfiguration defaultYarnConfig = new YarnConfiguration();
+    amState.getPublishedSliderConfigurations().put(
+        PublishedArtifacts.COMPLETE_CONFIG,
+        new PublishedConfiguration(
+            "Complete slider application settings",
+            getConfig(), getConfig()));
+    amState.getPublishedSliderConfigurations().put(
+        PublishedArtifacts.YARN_SITE_CONFIG,
+        new PublishedConfiguration(
+            "YARN site settings",
+            ConfigHelper.loadFromResource("yarn-site.xml"),
+            defaultYarnConfig) );
+
+    amState.getPublishedSliderConfigurations().put(
+        PublishedArtifacts.CORE_SITE_CONFIG,
+        new PublishedConfiguration(
+            "Core site settings",
+            ConfigHelper.loadFromResource("core-site.xml"),
+            defaultYarnConfig) );
+    amState.getPublishedSliderConfigurations().put(
+        PublishedArtifacts.HDFS_SITE_CONFIG,
+        new PublishedConfiguration(
+            "HDFS site settings",
+            ConfigHelper.loadFromResource("hdfs-site.xml"),
+            new HdfsConfiguration(true)) );
+
+
+    try {
+
+      URL managementAPI = new URL(amWebURI, RELATIVE_PATH_MANAGEMENT);
+      URL registryREST = new URL(amWebURI, RELATIVE_PATH_REGISTRY);
+
+      URL publisherURL = new URL(amWebURI, RELATIVE_PATH_PUBLISHER);
+
+      // Set the configurations URL.
+
+      String configurationsURL = SliderUtils.appendToURL(
+          publisherURL.toExternalForm(), RestPaths.SLIDER_CONFIGSET);
+      String exportsURL = SliderUtils.appendToURL(
+          publisherURL.toExternalForm(), RestPaths.SLIDER_EXPORTS);
+
+      serviceRecord.addExternalEndpoint(
+          RegistryTypeUtils.webEndpoint(
+              CustomRegistryConstants.WEB_UI, amWebURI.toURI()));
+      
+      serviceRecord.addExternalEndpoint(
+          RegistryTypeUtils.webEndpoint(
+              CustomRegistryConstants.AM_REST_BASE, amWebURI.toURI()));
+      
+      serviceRecord.addExternalEndpoint(
+          RegistryTypeUtils.restEndpoint(
+              CustomRegistryConstants.MANAGEMENT_REST_API,
+              managementAPI.toURI()));
+      serviceRecord.addExternalEndpoint(
+          RegistryTypeUtils.restEndpoint(
+              CustomRegistryConstants.PUBLISHER_REST_API,
+              publisherURL.toURI()));
+      serviceRecord.addExternalEndpoint(
+          RegistryTypeUtils.restEndpoint(
+              CustomRegistryConstants.REGISTRY_REST_API,
+              registryREST.toURI()));
+      serviceRecord.addExternalEndpoint(
+          RegistryTypeUtils.restEndpoint(
+              CustomRegistryConstants.PUBLISHER_CONFIGURATIONS_API,
+              new URI(configurationsURL)));
+      serviceRecord.addExternalEndpoint(
+          RegistryTypeUtils.restEndpoint(
+              CustomRegistryConstants.PUBLISHER_EXPORTS_API,
+              new URI(exportsURL)));
+
+    } catch (URISyntaxException e) {
+      throw new IOException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/AppMasterActionOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/AppMasterActionOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/AppMasterActionOperations.java
new file mode 100644
index 0000000..288f25a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/AppMasterActionOperations.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster;
+
+import org.apache.slider.server.appmaster.operations.RMOperationHandlerActions;
+
+/**
+ * Interface of AM operations
+ */
+public interface AppMasterActionOperations extends RMOperationHandlerActions {
+
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/PrivilegedConnectToCM.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/PrivilegedConnectToCM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/PrivilegedConnectToCM.java
new file mode 100644
index 0000000..65b88cf
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/PrivilegedConnectToCM.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster;
+
+
+import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
+
+import java.net.InetSocketAddress;
+import java.security.PrivilegedAction;
+
+/**
+ * Implement privileged connection to the CM
+ *
+ */
+public class PrivilegedConnectToCM implements PrivilegedAction<ContainerManagementProtocol> {
+  final SliderAppMaster appMaster;
+  final InetSocketAddress cmAddress;
+
+  public PrivilegedConnectToCM(SliderAppMaster appMaster,
+                               InetSocketAddress cmAddress) {
+    this.appMaster = appMaster;
+    this.cmAddress = cmAddress;
+  }
+
+
+  @Override //PrivilegedAction
+  public ContainerManagementProtocol run() {
+    return ((ContainerManagementProtocol) appMaster.getProxy(
+      ContainerManagementProtocol.class,
+      cmAddress));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/ProtobufClusterServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/ProtobufClusterServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/ProtobufClusterServices.java
new file mode 100644
index 0000000..5d52441
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/ProtobufClusterServices.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster;
+
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.Records;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.apache.slider.server.appmaster.state.AbstractClusterServices;
+
+public class ProtobufClusterServices extends AbstractClusterServices {
+
+  public Resource newResource() {
+    return Records.newRecord(Resource.class);
+  }
+
+  @Override
+  public Resource newResource(int memory, int cores) {
+    return Resources.createResource(memory, cores);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/PublishedArtifacts.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/PublishedArtifacts.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/PublishedArtifacts.java
new file mode 100644
index 0000000..fdc386f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/PublishedArtifacts.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster;
+
+/**
+ * This is the name of YARN artifacts that are published
+ */
+public interface PublishedArtifacts {
+
+  String COMPLETE_CONFIG = "complete-config";
+  String CORE_SITE_CONFIG = "core-site";
+  String HDFS_SITE_CONFIG = "hdfs-site";
+  String YARN_SITE_CONFIG = "yarn-site";
+  String LOG4J = "log4j";
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/RoleLaunchService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/RoleLaunchService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/RoleLaunchService.java
new file mode 100644
index 0000000..3cfe167
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/RoleLaunchService.java
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.launch.ContainerLauncher;
+import org.apache.slider.providers.ProviderRole;
+import org.apache.slider.providers.ProviderService;
+import org.apache.slider.providers.agent.AgentKeys;
+import org.apache.slider.server.appmaster.actions.ActionStartContainer;
+import org.apache.slider.server.appmaster.actions.QueueAccess;
+import org.apache.slider.server.appmaster.state.ContainerAssignment;
+import org.apache.slider.server.appmaster.state.RoleInstance;
+import org.apache.slider.server.appmaster.state.RoleStatus;
+import org.apache.slider.server.services.workflow.WorkflowExecutorService;
+import org.apache.slider.server.services.workflow.ServiceThreadFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A service for launching containers
+ */
+public class RoleLaunchService
+    extends WorkflowExecutorService<ExecutorService> {
+  protected static final Logger log =
+    LoggerFactory.getLogger(RoleLaunchService.class);
+
+  public static final String ROLE_LAUNCH_SERVICE = "RoleLaunchService";
+
+
+  /**
+   * Queue submission API
+   */
+  private final QueueAccess actionQueue;
+
+  /**
+   * Provider building up the command
+   */
+  private final ProviderService provider;
+  
+  /**
+   * Filesystem to use for the launch
+   */
+  private final SliderFileSystem fs;
+
+  /**
+   * Path in the launch filesystem that refers to a configuration directory
+   * -the interpretation of it is left to the Provider
+   */
+  private final Path generatedConfDirPath;
+  /**
+   * Path in the launch filesystem that refers to a temp directory
+   * which will be cleaned up at (some) time in the future
+   */
+  private final Path launcherTmpDirPath;
+
+  private Map<String, String> envVars;
+
+  /**
+   * Construct an instance of the launcher
+   * @param queueAccess
+   * @param provider the provider
+   * @param fs filesystem
+   * @param generatedConfDirPath path in the FS for the generated dir
+   * @param envVars environment variables
+   * @param launcherTmpDirPath path for a temporary data in the launch process
+   */
+  public RoleLaunchService(QueueAccess queueAccess,
+      ProviderService provider,
+      SliderFileSystem fs,
+      Path generatedConfDirPath,
+      Map<String, String> envVars,
+      Path launcherTmpDirPath) {
+    super(ROLE_LAUNCH_SERVICE);
+    this.actionQueue = queueAccess;
+    this.fs = fs;
+    this.generatedConfDirPath = generatedConfDirPath;
+    this.launcherTmpDirPath = launcherTmpDirPath;
+    this.provider = provider;
+    this.envVars = envVars;
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    super.init(conf);
+    setExecutor(Executors.newCachedThreadPool(
+        new ServiceThreadFactory(ROLE_LAUNCH_SERVICE, true)));
+  }
+
+  /**
+   * Start an asychronous launch operation
+   * @param assignment container assignment
+   * @param clusterSpec cluster spec to use for template
+   * @param credentials credentials to use
+   */
+  public void launchRole(ContainerAssignment assignment,
+      AggregateConf clusterSpec,
+      Credentials credentials) {
+    RoleStatus role = assignment.role;
+    String roleName = role.getName();
+    String roleGroup = role.getGroup();
+    // prelaunch safety check
+    Preconditions.checkArgument(provider.isSupportedRole(roleName));
+    RoleLaunchService.RoleLauncher launcher =
+      new RoleLaunchService.RoleLauncher(assignment,
+         clusterSpec,
+         clusterSpec.getResourceOperations().getOrAddComponent(roleGroup),
+         clusterSpec.getAppConfOperations().getOrAddComponent(roleGroup),
+         credentials);
+    execute(launcher);
+  }
+
+  /**
+   * Thread that runs on the AM to launch a container
+   */
+  private class RoleLauncher implements Runnable {
+
+    private final ContainerAssignment assignment;
+    // Allocated container
+    public final Container container;
+    private final MapOperations resourceComponent;
+    private final MapOperations appComponent;
+    private final AggregateConf instanceDefinition;
+    public final ProviderRole role;
+    private final Credentials credentials;
+    private Exception raisedException;
+
+    public RoleLauncher(ContainerAssignment assignment,
+        AggregateConf instanceDefinition,
+        MapOperations resourceComponent,
+        MapOperations appComponent,
+        Credentials credentials) {
+      this.assignment = assignment;
+      this.credentials = credentials;
+      this.container = assignment.container;
+      RoleStatus roleStatus = assignment.role;
+
+      assert resourceComponent != null;
+      assert appComponent != null;
+      ProviderRole providerRole = roleStatus.getProviderRole();
+      assert providerRole != null;
+      this.role = providerRole;
+      this.resourceComponent = resourceComponent;
+      this.appComponent = appComponent;
+      this.instanceDefinition = instanceDefinition;
+    }
+
+    public Exception getRaisedException() {
+      return raisedException;
+    }
+
+    @Override
+    public String toString() {
+      return "RoleLauncher{" +
+             "container=" + container.getId() +
+             ", containerRole='" + role.name + '\'' +
+             ", containerGroup='" + role.group + '\'' +
+             '}';
+    }
+
+    @Override
+    public void run() {
+      try {
+        ContainerLauncher containerLauncher =
+            new ContainerLauncher(getConfig(), fs, container, credentials);
+        containerLauncher.setupUGI();
+        containerLauncher.putEnv(envVars);
+
+        log.debug("Launching container {} into role {}",
+                  container.getId(),
+                  role.name);
+
+        //now build up the configuration data
+        Path containerTmpDirPath =
+          new Path(launcherTmpDirPath, container.getId().toString());
+        provider.buildContainerLaunchContext(containerLauncher,
+            instanceDefinition,
+            container,
+            role,
+            fs,
+            generatedConfDirPath,
+            resourceComponent,
+            appComponent,
+            containerTmpDirPath);
+
+        RoleInstance instance = new RoleInstance(container);
+        String[] envDescription = containerLauncher.dumpEnvToString();
+
+        String commandsAsString = containerLauncher.getCommandsAsString();
+        log.info("Starting container with command: {}",
+                 commandsAsString);
+
+        instance.command = commandsAsString;
+        instance.role = role.name;
+        instance.group = role.group;
+        instance.roleId = role.id;
+        instance.appVersion = instanceDefinition.getAppConfOperations()
+            .getGlobalOptions().get(SliderKeys.APP_VERSION);
+        instance.environment = envDescription;
+        int delay = appComponent.getOptionInt(
+            AgentKeys.KEY_CONTAINER_LAUNCH_DELAY, 0);
+        int maxDelay =
+            getConfig().getInt(YarnConfiguration.RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS,
+                               YarnConfiguration.DEFAULT_RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS);
+        if (delay > maxDelay/1000) {
+          log.warn("Container launch delay of {} exceeds the maximum allowed of"
+                   + " {} seconds.  Delay will not be utilized.",
+                   delay, maxDelay/1000);
+          delay = 0;
+        }
+        log.info("Container launch delay for {} set to {} seconds",
+                 role.name, delay);
+        actionQueue.schedule(new ActionStartContainer("starting " + role.name,
+                                                      container,
+                                                      containerLauncher.completeContainerLaunch(),
+                                                      instance,
+                                                      delay,
+                                                      TimeUnit.SECONDS));
+      } catch (Exception e) {
+        log.error("Exception thrown while trying to start {}: {}",
+            role.name, e, e);
+        raisedException = e;
+      }
+    }
+
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[06/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationResponse.java
new file mode 100644
index 0000000..80b7a5e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationResponse.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.agent;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.annotate.JsonProperty;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.List;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class RegistrationResponse {
+
+  @JsonProperty("response")
+  private RegistrationStatus response;
+
+  /**
+   * exitstatus is a code of error which was rised on server side. exitstatus
+   * = 0 (OK - Default) exitstatus = 1 (Registration failed because different
+   * version of agent and server)
+   */
+  @JsonProperty("exitstatus")
+  private int exitstatus;
+
+  /** log - message, which will be printed to agents log */
+  @JsonProperty("log")
+  private String log;
+
+  /** tags - tags associated with the container */
+  @JsonProperty("tags")
+  private String tags;
+  
+  @JsonProperty("package")
+  private String pkg;
+
+  //Response id to start with, usually zero.
+  @JsonProperty("responseId")
+  private long responseId;
+
+  @JsonProperty("statusCommands")
+  private List<StatusCommand> statusCommands = null;
+
+  public RegistrationResponse() {
+  }
+
+  public RegistrationStatus getResponse() {
+    return response;
+  }
+
+  public void setResponse(RegistrationStatus response) {
+    this.response = response;
+  }
+
+  public int getExitstatus() {
+    return exitstatus;
+  }
+
+  public void setExitstatus(int exitstatus) {
+    this.exitstatus = exitstatus;
+  }
+
+  public RegistrationStatus getResponseStatus() {
+    return response;
+  }
+
+  public void setResponseStatus(RegistrationStatus response) {
+    this.response = response;
+  }
+
+  public List<StatusCommand> getStatusCommands() {
+    return statusCommands;
+  }
+
+  public void setStatusCommands(List<StatusCommand> statusCommands) {
+    this.statusCommands = statusCommands;
+  }
+
+  public long getResponseId() {
+    return responseId;
+  }
+
+  public void setResponseId(long responseId) {
+    this.responseId = responseId;
+  }
+
+  public String getTags() {
+    return tags;
+  }
+
+  public void setTags(String tags) {
+    this.tags = tags;
+  }
+
+  public String getLog() {
+    return log;
+  }
+
+  public void setLog(String log) {
+    this.log = log;
+  }
+
+  public String getPkg() {
+    return pkg;
+  }
+
+  public void setPkg(String pkg) {
+    this.pkg = pkg;
+  }
+
+  @Override
+  public String toString() {
+    return "RegistrationResponse{" +
+           "response=" + response +
+           ", responseId=" + responseId +
+           ", statusCommands=" + statusCommands +
+           '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationStatus.java
new file mode 100644
index 0000000..8374710
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationStatus.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.agent;
+
+public enum RegistrationStatus {
+    OK,
+    FAILED
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/StatusCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/StatusCommand.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/StatusCommand.java
new file mode 100644
index 0000000..5b205b5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/StatusCommand.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.agent;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.annotate.JsonProperty;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Command to report the status of a list of services in roles.
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class StatusCommand {
+  public static String STATUS_COMMAND = "STATUS";
+  public static String GET_CONFIG_COMMAND = "GET_CONFIG";
+
+  AgentCommandType agentCommandType;
+
+  private String clusterName;
+  private String serviceName;
+  private String componentName;
+  private Map<String, Map<String, String>> configurations;
+  private Map<String, String> commandParams = new HashMap<String, String>();
+  private Map<String, String> hostLevelParams = new HashMap<String, String>();
+  private String roleCommand;
+  private boolean yarnDockerMode;
+
+  public StatusCommand() {
+    this.agentCommandType = AgentCommandType.STATUS_COMMAND;
+  }
+
+  @JsonProperty("clusterName")
+  public String getClusterName() {
+    return clusterName;
+  }
+
+  @JsonProperty("clusterName")
+  public void setClusterName(String clusterName) {
+    this.clusterName = clusterName;
+  }
+
+  @JsonProperty("serviceName")
+  public String getServiceName() {
+    return serviceName;
+  }
+
+  @JsonProperty("serviceName")
+  public void setServiceName(String serviceName) {
+    this.serviceName = serviceName;
+  }
+
+  @JsonProperty("componentName")
+  public String getComponentName() {
+    return componentName;
+  }
+
+  @JsonProperty("componentName")
+  public void setComponentName(String componentName) {
+    this.componentName = componentName;
+  }
+
+  @JsonProperty("configurations")
+  public Map<String, Map<String, String>> getConfigurations() {
+    return configurations;
+  }
+
+  @JsonProperty("configurations")
+  public void setConfigurations(Map<String, Map<String, String>> configurations) {
+    this.configurations = configurations;
+  }
+
+  @JsonProperty("hostLevelParams")
+  public Map<String, String> getHostLevelParams() {
+    return hostLevelParams;
+  }
+
+  @JsonProperty("hostLevelParams")
+  public void setHostLevelParams(Map<String, String> params) {
+    this.hostLevelParams = params;
+  }
+
+  @JsonProperty("commandParams")
+  public Map<String, String> getCommandParams() {
+    return commandParams;
+  }
+
+  @JsonProperty("commandParams")
+  public void setCommandParams(Map<String, String> commandParams) {
+    this.commandParams = commandParams;
+  }
+
+  @JsonProperty("commandType")
+  public AgentCommandType getCommandType() {
+    return agentCommandType;
+  }
+
+  @JsonProperty("commandType")
+  public void setCommandType(AgentCommandType commandType) {
+    this.agentCommandType = commandType;
+  }
+
+  @JsonProperty("roleCommand")
+  public String getRoleCommand() {
+    return roleCommand;
+  }
+
+  @JsonProperty("roleCommand")
+  public void setRoleCommand(String roleCommand) {
+    this.roleCommand = roleCommand;
+  }
+  
+  @JsonProperty("yarnDockerMode")
+  public boolean isYarnDockerMode() {
+    return yarnDockerMode;
+  }
+
+  @JsonProperty("yarnDockerMode")
+  public void setYarnDockerMode(boolean yarnDockerMode) {
+    this.yarnDockerMode = yarnDockerMode;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append("StatusCommand [agentCommandType=").append(agentCommandType)
+        .append(", clusterName=").append(clusterName).append(", serviceName=")
+        .append(serviceName).append(", componentName=").append(componentName)
+        .append(", configurations=").append(configurations)
+        .append(", commandParams=").append(commandParams)
+        .append(", hostLevelParams=").append(hostLevelParams)
+        .append(", roleCommand=").append(roleCommand).append("]");
+    return builder.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResouceContentCacheFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResouceContentCacheFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResouceContentCacheFactory.java
new file mode 100644
index 0000000..2facf16
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResouceContentCacheFactory.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest.application;
+
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+import org.apache.slider.server.appmaster.web.rest.application.resources.AggregateModelRefresher;
+import org.apache.slider.server.appmaster.web.rest.application.resources.AppconfRefresher;
+import org.apache.slider.server.appmaster.web.rest.application.resources.CachedContent;
+import org.apache.slider.server.appmaster.web.rest.application.resources.ContentCache;
+import org.apache.slider.server.appmaster.web.rest.application.resources.LiveComponentsRefresher;
+import org.apache.slider.server.appmaster.web.rest.application.resources.LiveContainersRefresher;
+import org.apache.slider.server.appmaster.web.rest.application.resources.LiveNodesRefresher;
+import org.apache.slider.server.appmaster.web.rest.application.resources.LiveResourcesRefresher;
+import org.apache.slider.server.appmaster.web.rest.application.resources.LiveStatisticsRefresher;
+
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.*;
+
+public class ApplicationResouceContentCacheFactory {
+  public static final int LIFESPAN = 500;
+
+  /**
+   * Build the content cache
+   * @param state state view
+   */
+  public static ContentCache createContentCache(
+      StateAccessForProviders state) {
+    ContentCache cache = new ContentCache();
+    cache.put(LIVE_RESOURCES, new CachedContent<>(LIFESPAN, new LiveResourcesRefresher(state)));
+    cache.put(LIVE_CONTAINERS, new CachedContent<>(LIFESPAN, new LiveContainersRefresher(state)));
+    cache.put(LIVE_COMPONENTS, new CachedContent<>(LIFESPAN, new LiveComponentsRefresher(state)));
+    cache.put(LIVE_NODES, new CachedContent<>(LIFESPAN, new LiveNodesRefresher(state)));
+    cache.put(MODEL_DESIRED,
+        new CachedContent<>(LIFESPAN, new AggregateModelRefresher(state, false)));
+    cache.put(MODEL_RESOLVED,
+        new CachedContent<>(LIFESPAN, new AggregateModelRefresher(state, true)));
+    cache.put(MODEL_RESOLVED_APPCONF,
+        new CachedContent<>(LIFESPAN, new AppconfRefresher(state, false, false)));
+    cache.put(MODEL_RESOLVED_RESOURCES,
+        new CachedContent<>(LIFESPAN, new AppconfRefresher(state, false, true)));
+    cache.put(MODEL_DESIRED_APPCONF,
+        new CachedContent<>(LIFESPAN, new AppconfRefresher(state, true, false)));
+    cache.put(MODEL_DESIRED_RESOURCES,
+        new CachedContent<>(LIFESPAN, new AppconfRefresher(state, true, true)));
+    cache.put(LIVE_STATISTICS, new CachedContent<>(LIFESPAN, new LiveStatisticsRefresher(state)));
+    return cache;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResource.java
new file mode 100644
index 0000000..52068d6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/ApplicationResource.java
@@ -0,0 +1,516 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest.application;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.webapp.BadRequestException;
+import org.apache.hadoop.yarn.webapp.NotFoundException;
+import org.apache.slider.api.types.ApplicationLivenessInformation;
+import org.apache.slider.api.types.ComponentInformation;
+import org.apache.slider.api.types.ContainerInformation;
+import org.apache.slider.api.types.NodeInformation;
+import org.apache.slider.api.types.NodeInformationList;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTree;
+import org.apache.slider.core.exceptions.NoSuchNodeException;
+import org.apache.slider.core.persist.ConfTreeSerDeser;
+import org.apache.slider.server.appmaster.actions.ActionFlexCluster;
+import org.apache.slider.server.appmaster.actions.AsyncAction;
+import org.apache.slider.server.appmaster.actions.QueueAccess;
+import org.apache.slider.server.appmaster.state.RoleInstance;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.apache.slider.server.appmaster.web.rest.AbstractSliderResource;
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.*;
+
+import org.apache.slider.server.appmaster.web.rest.application.actions.RestActionStop;
+import org.apache.slider.server.appmaster.web.rest.application.actions.StopResponse;
+import org.apache.slider.server.appmaster.web.rest.application.resources.ContentCache;
+import org.apache.slider.server.appmaster.web.rest.application.actions.RestActionPing;
+import org.apache.slider.api.types.PingInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.inject.Singleton;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.HEAD;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Context;
+
+import static javax.ws.rs.core.MediaType.*;
+
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.UriInfo;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+@Singleton
+@SuppressWarnings("unchecked")
+public class ApplicationResource extends AbstractSliderResource {
+  private static final Logger log =
+      LoggerFactory.getLogger(ApplicationResource.class);
+
+  public static final List<String> LIVE_ENTRIES = toJsonList("resources",
+      "containers",
+      "components",
+      "nodes",
+      "statistics",
+      "internal");
+
+  public static final List<String> ROOT_ENTRIES =
+      toJsonList("model", "live", "actions");
+
+  public static final List<String> MODEL_ENTRIES =
+      toJsonList("desired", "resolved");
+
+  /**
+   * This is the cache of all content ... each entry is
+   * designed to be self-refreshing on get operations, 
+   * so is never very out of date, yet many GETs don't
+   * overload the rest of the system.
+   */
+  private final ContentCache cache;
+  private final StateAccessForProviders state;
+  private final QueueAccess actionQueues;
+
+  public ApplicationResource(WebAppApi slider) {
+    super(slider);
+    state = slider.getAppState();
+    cache = slider.getContentCache();
+    actionQueues = slider.getQueues();
+  }
+
+  /**
+   * Build a new JSON-marshallable list of string elements
+   * @param elements elements
+   * @return something that can be returned
+   */
+  private static List<String> toJsonList(String... elements) {
+    return Lists.newArrayList(elements);
+  }
+
+  @GET
+  @Path("/")
+  @Produces({APPLICATION_JSON})
+  public List<String> getRoot() {
+    markGet(SLIDER_SUBPATH_APPLICATION);
+    return ROOT_ENTRIES;
+  }
+
+  /**
+   * Enum model values: desired and resolved
+   * @return the desired and resolved model
+   */
+  @GET
+  @Path(MODEL)
+  @Produces({APPLICATION_JSON})
+  public List<String> getModel() {
+    markGet(SLIDER_SUBPATH_APPLICATION, MODEL);
+    return MODEL_ENTRIES;
+  }
+
+  @GET
+  @Path(MODEL_DESIRED)
+  @Produces({APPLICATION_JSON})
+  public AggregateConf getModelDesired() {
+    markGet(SLIDER_SUBPATH_APPLICATION, MODEL_DESIRED);
+    return lookupAggregateConf(MODEL_DESIRED);
+  }
+  
+  @GET
+  @Path(MODEL_DESIRED_APPCONF)
+  @Produces({APPLICATION_JSON})
+  public ConfTree getModelDesiredAppconf() {
+    markGet(SLIDER_SUBPATH_APPLICATION, MODEL_DESIRED_APPCONF);
+    return lookupConfTree(MODEL_DESIRED_APPCONF);
+  }
+
+  @GET
+  @Path(MODEL_DESIRED_RESOURCES)
+  @Produces({APPLICATION_JSON})
+  public ConfTree getModelDesiredResources() {
+    markGet(SLIDER_SUBPATH_APPLICATION, MODEL_DESIRED_RESOURCES);
+    return lookupConfTree(MODEL_DESIRED_RESOURCES);
+  }
+
+/*
+  @PUT
+  @Path(MODEL_DESIRED_RESOURCES)
+//  @Consumes({APPLICATION_JSON, TEXT_PLAIN})
+  @Consumes({TEXT_PLAIN})
+  @Produces({APPLICATION_JSON})
+*/
+  public ConfTree setModelDesiredResources(
+      String json) {
+    markPut(SLIDER_SUBPATH_APPLICATION, MODEL_DESIRED_RESOURCES);
+    int size = json != null ? json.length() : 0;
+    log.info("PUT {} {} bytes:\n{}", MODEL_DESIRED_RESOURCES,
+        size,
+        json);
+    if (size == 0) {
+      log.warn("No JSON in PUT request; rejecting");
+      throw new BadRequestException("No JSON in PUT");
+    }
+    
+    try {
+      ConfTreeSerDeser serDeser = new ConfTreeSerDeser();
+      ConfTree updated = serDeser.fromJson(json);
+      queue(new ActionFlexCluster("flex",
+          1, TimeUnit.MILLISECONDS,
+          updated));
+      // return the updated value, even though it potentially hasn't yet
+      // been executed
+      return updated;
+    } catch (Exception e) {
+      throw buildException("PUT to "+ MODEL_DESIRED_RESOURCES , e);
+    }
+  }
+  @PUT
+  @Path(MODEL_DESIRED_RESOURCES)
+  @Consumes({APPLICATION_JSON})
+  @Produces({APPLICATION_JSON})
+  public ConfTree setModelDesiredResources(
+      ConfTree updated) {
+    try {
+      queue(new ActionFlexCluster("flex",
+          1, TimeUnit.MILLISECONDS,
+          updated));
+      // return the updated value, even though it potentially hasn't yet
+      // been executed
+      return updated;
+    } catch (Exception e) {
+      throw buildException("PUT to "+ MODEL_DESIRED_RESOURCES , e);
+    }
+  }
+  
+  
+
+  @GET
+  @Path(MODEL_RESOLVED)
+  @Produces({APPLICATION_JSON})
+  public AggregateConf getModelResolved() {
+    markGet(SLIDER_SUBPATH_APPLICATION, MODEL_RESOLVED);
+    return lookupAggregateConf(MODEL_RESOLVED);
+  }
+
+  @GET
+  @Path(MODEL_RESOLVED_APPCONF)
+  @Produces({APPLICATION_JSON})
+  public ConfTree getModelResolvedAppconf() {
+    markGet(SLIDER_SUBPATH_APPLICATION, MODEL_RESOLVED_APPCONF);
+    return lookupConfTree(MODEL_RESOLVED_APPCONF);
+  }
+
+  @GET
+  @Path(MODEL_RESOLVED_RESOURCES)
+  @Produces({APPLICATION_JSON})
+  public ConfTree getModelResolvedResources() {
+    markGet(SLIDER_SUBPATH_APPLICATION, MODEL_RESOLVED_RESOURCES);
+    return lookupConfTree(MODEL_RESOLVED_RESOURCES);
+  }
+  
+  @GET
+  @Path(LIVE)
+  @Produces({APPLICATION_JSON})
+  public List<String> getLive() {
+    markGet(SLIDER_SUBPATH_APPLICATION, LIVE);
+    return LIVE_ENTRIES;
+  }
+
+  @GET
+  @Path(LIVE_RESOURCES)
+  @Produces({APPLICATION_JSON})
+  public ConfTree getLiveResources() {
+    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_RESOURCES);
+    return lookupConfTree(LIVE_RESOURCES);
+  }
+  
+  @GET
+  @Path(LIVE_CONTAINERS)
+  @Produces({APPLICATION_JSON})
+  public Map<String, ContainerInformation> getLiveContainers() {
+    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_CONTAINERS);
+    try {
+      return (Map<String, ContainerInformation>)cache.lookup(
+          LIVE_CONTAINERS);
+    } catch (Exception e) {
+      throw buildException(LIVE_CONTAINERS, e);
+    }
+  }
+
+  @GET
+  @Path(LIVE_CONTAINERS + "/{containerId}")
+  @Produces({APPLICATION_JSON})
+  public ContainerInformation getLiveContainer(
+      @PathParam("containerId") String containerId) {
+    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_CONTAINERS);
+    try {
+      RoleInstance id = state.getLiveInstanceByContainerID(containerId);
+      return id.serialize();
+    } catch (NoSuchNodeException e) {
+      throw new NotFoundException("Unknown container: " + containerId);
+    } catch (Exception e) {
+      throw buildException(LIVE_CONTAINERS + "/"+ containerId, e);
+    }
+  }
+
+  @GET
+  @Path(LIVE_COMPONENTS)
+  @Produces({APPLICATION_JSON})
+  public Map<String, ComponentInformation> getLiveComponents() {
+    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_COMPONENTS);
+    try {
+      return (Map<String, ComponentInformation>) cache.lookup(LIVE_COMPONENTS);
+    } catch (Exception e) {
+      throw buildException(LIVE_COMPONENTS, e);
+    }
+  }
+  
+  @GET
+  @Path(LIVE_COMPONENTS + "/{component}")
+  @Produces({APPLICATION_JSON})
+  public ComponentInformation getLiveComponent(
+      @PathParam("component") String component) {
+    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_COMPONENTS);
+    try {
+      return state.getComponentInformation(component);
+    } catch (YarnRuntimeException e) {
+      throw new NotFoundException("Unknown component: " + component);
+    } catch (Exception e) {
+      throw buildException(LIVE_CONTAINERS +"/" + component, e);
+    }
+  }
+
+  /**
+   * Liveness information for the application as a whole
+   * @return snapshot of liveness
+   */
+  @GET
+  @Path(LIVE_LIVENESS)
+  @Produces({APPLICATION_JSON})
+  public ApplicationLivenessInformation getLivenessInformation() {
+    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_LIVENESS);
+    try {
+      return state.getApplicationLivenessInformation();
+    } catch (Exception e) {
+      throw buildException(LIVE_CONTAINERS, e);
+    }
+  }
+
+/*
+TODO: decide what structure to return here, then implement
+
+  @GET
+  @Path(LIVE_LIVENESS + "/{component}")
+  @Produces({APPLICATION_JSON})
+  public ApplicationLivenessInformation getLivenessForComponent(
+      @PathParam("component") String component) {
+    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_COMPONENTS);
+    try {
+      RoleStatus roleStatus = state.lookupRoleStatus(component);
+      ApplicationLivenessInformation info = new ApplicationLivenessInformation();
+      info.requested = roleStatus.getRequested();
+      info.allRequestsSatisfied = info.requested == 0;
+      return info;
+    } catch (YarnRuntimeException e) {
+      throw new NotFoundException("Unknown component: " + component);
+    } catch (Exception e) {
+      throw buildException(LIVE_LIVENESS + "/" + component, e);
+    }
+  }
+*/
+
+
+  @GET
+  @Path(LIVE_NODES)
+  @Produces({APPLICATION_JSON})
+  public NodeInformationList getLiveNodes() {
+    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_COMPONENTS);
+    try {
+      return (NodeInformationList) cache.lookup(LIVE_NODES);
+    } catch (Exception e) {
+      throw buildException(LIVE_COMPONENTS, e);
+    }
+  }
+
+  @GET
+  @Path(LIVE_NODES + "/{hostname}")
+  @Produces({APPLICATION_JSON})
+  public NodeInformation getLiveNode(@PathParam("hostname") String hostname) {
+    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_COMPONENTS);
+    try {
+      NodeInformation ni = state.getNodeInformation(hostname);
+      if (ni != null) {
+        return ni;
+      } else {
+        throw new NotFoundException("Unknown node: " + hostname);
+      }
+    } catch (NotFoundException e) {
+      throw e;
+    } catch (Exception e) {
+      throw buildException(LIVE_COMPONENTS + "/" + hostname, e);
+    }
+  }
+
+  /**
+   * Statistics of the application
+   * @return snapshot statistics
+   */
+  @GET
+  @Path(LIVE_STATISTICS)
+  @Produces({APPLICATION_JSON})
+  public Map<String, Integer> getLiveStatistics() {
+    markGet(SLIDER_SUBPATH_APPLICATION, LIVE_LIVENESS);
+    try {
+      return (Map<String, Integer>) cache.lookup(LIVE_STATISTICS);
+    } catch (Exception e) {
+      throw buildException(LIVE_STATISTICS, e);
+    }
+  }
+
+  /**
+   * Helper method; look up an aggregate configuration in the cache from
+   * a key, or raise an exception
+   * @param key key to resolve
+   * @return the configuration
+   * @throws WebApplicationException on a failure
+   */
+  protected AggregateConf lookupAggregateConf(String key) {
+    try {
+      return (AggregateConf) cache.lookup(key);
+    } catch (Exception e) {
+      throw buildException(key, e);
+    }
+  }
+
+
+  /**
+   * Helper method; look up an conf tree in the cache from
+   * a key, or raise an exception
+   * @param key key to resolve
+   * @return the configuration
+   * @throws WebApplicationException on a failure
+   */
+  protected ConfTree lookupConfTree(String key) {
+    try {
+      return (ConfTree) cache.lookup(key);
+    } catch (Exception e) {
+      throw buildException(key, e);
+    }
+  }
+
+  /* ************************************************************************
+  
+  ACTION PING
+  
+  **************************************************************************/
+  
+  @GET
+  @Path(ACTION_PING)
+  @Produces({APPLICATION_JSON})
+  public PingInformation actionPingGet(@Context HttpServletRequest request,
+      @Context UriInfo uriInfo) {
+    markGet(SLIDER_SUBPATH_APPLICATION, ACTION_PING);
+    return new RestActionPing().ping(request, uriInfo, "");
+  }
+  
+  @POST
+  @Path(ACTION_PING)
+  @Produces({APPLICATION_JSON})
+  public PingInformation actionPingPost(@Context HttpServletRequest request,
+      @Context UriInfo uriInfo,
+      String body) {
+    markPost(SLIDER_SUBPATH_APPLICATION, ACTION_PING);
+    return new RestActionPing().ping(request, uriInfo, body);
+  }
+  
+  @PUT
+  @Path(ACTION_PING)
+  @Consumes({TEXT_PLAIN})
+  @Produces({APPLICATION_JSON})
+  public PingInformation actionPingPut(@Context HttpServletRequest request,
+      @Context UriInfo uriInfo,
+      String body) {
+    markPut(SLIDER_SUBPATH_APPLICATION, ACTION_PING);
+    return new RestActionPing().ping(request, uriInfo, body);
+  }
+  
+  @DELETE
+  @Path(ACTION_PING)
+  @Consumes({APPLICATION_JSON})
+  @Produces({APPLICATION_JSON})
+  public PingInformation actionPingDelete(@Context HttpServletRequest request,
+      @Context UriInfo uriInfo) {
+    markDelete(SLIDER_SUBPATH_APPLICATION, ACTION_PING);
+    return new RestActionPing().ping(request, uriInfo, "");
+  }
+  
+  @HEAD
+  @Path(ACTION_PING)
+  public Object actionPingHead(@Context HttpServletRequest request,
+      @Context UriInfo uriInfo) {
+    mark("HEAD", SLIDER_SUBPATH_APPLICATION, ACTION_PING);
+    return new RestActionPing().ping(request, uriInfo, "");
+  }
+  
+  /* ************************************************************************
+  
+  ACTION STOP
+  
+  **************************************************************************/
+
+
+  @POST
+  @Path(ACTION_STOP)
+  @Produces({APPLICATION_JSON})
+  public StopResponse actionStop(@Context HttpServletRequest request,
+      @Context UriInfo uriInfo,
+      String body) {
+    markPost(SLIDER_SUBPATH_APPLICATION, ACTION_STOP);
+    return new RestActionStop(slider).stop(request, uriInfo, body);
+  }
+
+  /**
+   * Schedule an action
+   * @param action for delayed execution
+   */
+  public void schedule(AsyncAction action) {
+    actionQueues.schedule(action);
+  }
+
+  /**
+   * Put an action on the immediate queue -to be executed when the queue
+   * reaches it.
+   * @param action action to queue
+   */
+  public void queue(AsyncAction action) {
+    actionQueues.put(action);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionPing.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionPing.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionPing.java
new file mode 100644
index 0000000..96b4f75
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionPing.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest.application.actions;
+
+import org.apache.slider.api.types.PingInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.core.UriInfo;
+import java.util.Locale;
+
+public class RestActionPing {
+  private static final Logger log =
+      LoggerFactory.getLogger(RestActionPing.class);
+
+  public RestActionPing() {
+  }
+  
+  public PingInformation ping(HttpServletRequest request, UriInfo uriInfo, String body) {
+    String verb = request.getMethod();
+    log.info("Ping {}", verb);
+    PingInformation pingInformation = new PingInformation();
+    pingInformation.time = System.currentTimeMillis();
+    pingInformation.verb = verb;
+    pingInformation.body = body;
+    String text = 
+        String.format(Locale.ENGLISH,
+            "Ping verb %s received at %tc",
+            verb, pingInformation.time);
+    pingInformation.text = text;
+    return pingInformation;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionStop.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionStop.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionStop.java
new file mode 100644
index 0000000..544f589
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/RestActionStop.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest.application.actions;
+
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.slider.core.main.LauncherExitCodes;
+import org.apache.slider.server.appmaster.actions.ActionStopSlider;
+import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.core.UriInfo;
+import java.util.Locale;
+import java.util.concurrent.TimeUnit;
+
+public class RestActionStop {
+  private static final Logger log =
+      LoggerFactory.getLogger(RestActionStop.class);
+
+  private final WebAppApi slider;
+  
+  public RestActionStop(WebAppApi slider) {
+    this.slider = slider;
+  }
+  
+  public StopResponse stop(HttpServletRequest request, UriInfo uriInfo, String body) {
+    String verb = request.getMethod();
+    log.info("Ping {}", verb);
+    StopResponse response = new StopResponse();
+    response.verb = verb;
+    long time = System.currentTimeMillis();
+    String text = 
+        String.format(Locale.ENGLISH,
+            "Stopping action %s received at %tc",
+            verb, time);
+    response.text = text;
+    log.info(text);
+    ActionStopSlider stopSlider =
+        new ActionStopSlider(text,
+            1000,
+            TimeUnit.MILLISECONDS,
+            LauncherExitCodes.EXIT_SUCCESS,
+            FinalApplicationStatus.SUCCEEDED,
+            text);
+    log.info("SliderAppMasterApi.stopCluster: {}", stopSlider);
+    slider.getQueues().schedule(stopSlider);
+    
+    return response;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/StopResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/StopResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/StopResponse.java
new file mode 100644
index 0000000..9af6a22
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/actions/StopResponse.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest.application.actions;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class StopResponse {
+  String verb;
+  public String text;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/package-info.java
new file mode 100644
index 0000000..34eb82c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/package-info.java
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+
+/**
+ * This package contains resources related to the application
+ * REST API
+ */
+package org.apache.slider.server.appmaster.web.rest.application;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AggregateModelRefresher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AggregateModelRefresher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AggregateModelRefresher.java
new file mode 100644
index 0000000..ee28abf
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AggregateModelRefresher.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest.application.resources;
+
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+
+/**
+ * Refresh the aggregate desired model via
+ * {@link StateAccessForProviders#getInstanceDefinitionSnapshot()}
+ */
+public class AggregateModelRefresher
+    implements ResourceRefresher<AggregateConf> {
+
+  private final StateAccessForProviders state;
+  private final boolean resolved;
+
+  public AggregateModelRefresher(StateAccessForProviders state,
+      boolean resolved) {
+    this.state = state;
+    this.resolved = resolved;
+  }
+
+  @Override
+  public AggregateConf refresh() throws Exception {
+    return
+        resolved ?
+          state.getInstanceDefinitionSnapshot()
+          : state.getUnresolvedInstanceDefinition();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AppconfRefresher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AppconfRefresher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AppconfRefresher.java
new file mode 100644
index 0000000..06460cc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/AppconfRefresher.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest.application.resources;
+
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTree;
+import org.apache.slider.core.persist.ConfTreeSerDeser;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+
+/**
+ * refresher for resources and application configuration
+ */
+public class AppconfRefresher
+    implements ResourceRefresher<ConfTree> {
+
+  private final StateAccessForProviders state;
+  private final boolean unresolved;
+  private final boolean resources;
+
+  public AppconfRefresher(StateAccessForProviders state,
+      boolean unresolved,
+      boolean resources) {
+    this.state = state;
+    this.unresolved = unresolved;
+    this.resources = resources;
+  }
+
+
+  @Override
+  public ConfTree refresh() throws Exception {
+    AggregateConf aggregateConf =
+        unresolved ?
+        state.getUnresolvedInstanceDefinition():
+        state.getInstanceDefinitionSnapshot();
+    ConfTree ct = resources ? aggregateConf.getResources() 
+                            : aggregateConf.getAppConf();
+    return new ConfTreeSerDeser().fromInstance(ct);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/CachedContent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/CachedContent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/CachedContent.java
new file mode 100644
index 0000000..22fd0fe
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/CachedContent.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest.application.resources;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A cached resource is one that can be stored and served up, with a refresh 
+ * only taking place when the expiry happens.
+ * 
+ * The refresh check/refresh is synchronized.
+ * @param <T> type to return
+ */
+public class CachedContent<T> {
+  private static final Logger log =
+      LoggerFactory.getLogger(CachedContent.class);
+  private T cachedValue;
+  private long expires;
+  private final long lifespan;
+  private final ResourceRefresher<T> refresh;
+  private int refreshCounter;
+
+  public CachedContent(long lifespan,
+      ResourceRefresher<T> refresh) {
+    this.lifespan = lifespan;
+    this.refresh = refresh;
+  }
+
+  /**
+   * Get the value, triggering a refresh if needed
+   * @return the cached or latest value.
+   */
+  public T get() throws Exception {
+    maybeRefresh();
+    return getCachedValue();
+  }
+
+  /**
+   * Get the cached value without any expiry check
+   * @return the last value set. May be null.
+   */
+  public synchronized T getCachedValue() {
+    return cachedValue;
+  }
+
+  public synchronized int getRefreshCounter() {
+    return refreshCounter;
+  }
+
+  /**
+   * Get the lifespan in millis of the cached value
+   * @return the lifespan
+   */
+  public long getLifespan() {
+    return lifespan;
+  }
+
+  /**
+   * Maybe refresh the content
+   * @return true if a refresh took place.
+   */
+  public synchronized boolean maybeRefresh() throws Exception {
+    long now = now();
+    if (cachedValue == null || now >= expires) {
+      log.debug("Refreshing at time {}", now);
+      forceRefresh();
+      log.debug("Refreshed value now {}", cachedValue);
+      return true;
+    }
+    return false;
+  }
+
+  protected long now() {
+    return Time.monotonicNow();
+  }
+
+  /**
+   * Force a refresh and reset the expiry counter
+   * @return the new value
+   */
+  protected synchronized T forceRefresh() throws Exception {
+    refreshCounter ++;
+    T updated = refresh.refresh();
+    Preconditions.checkNotNull(updated);
+    cachedValue = updated;
+    expires = now() + lifespan;
+    return cachedValue;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+        new StringBuilder("CachedContent{");
+    sb.append("  expires=").append(expires);
+    sb.append(", lifespan=").append(lifespan);
+    sb.append(", refresh=").append(refresh);
+    sb.append(", refreshCounter=").append(refreshCounter);
+    sb.append(", cached=").append(cachedValue);
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ContentCache.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ContentCache.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ContentCache.java
new file mode 100644
index 0000000..8f026a1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ContentCache.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest.application.resources;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Cache of content
+ */
+public class ContentCache extends ConcurrentHashMap<String, CachedContent> {
+
+  public ContentCache(int initialCapacity) {
+    super(initialCapacity);
+  }
+
+  public ContentCache() {
+  }
+
+
+  public Object lookup(String key) throws Exception {
+    CachedContent content = get(key);
+    if (content == null) {
+      throw new FileNotFoundException("no content for path " + key);
+    }
+    return content.get();
+  }
+
+
+  /**
+   * Lookup a cached item. If an exception is raised on the refresh...
+   * <ol>
+   *   <li>IOExceptions are thrown directly</li>
+   *   <li>Other exceptions are wrapped with an IOExceptions</li>
+   * </ol>
+   * @param key
+   * @return
+   * @throws IOException
+   */
+  public Object lookupWithIOE(String key) throws IOException {
+    try {
+      return lookup(key);
+    } catch (IOException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new IOException("Looking up " + key + ": " + e, e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveComponentsRefresher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveComponentsRefresher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveComponentsRefresher.java
new file mode 100644
index 0000000..b6627a7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveComponentsRefresher.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest.application.resources;
+
+import org.apache.slider.api.types.ComponentInformation;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+
+import java.util.Map;
+
+public class LiveComponentsRefresher
+    implements ResourceRefresher<Map<String, ComponentInformation>> {
+
+  private final StateAccessForProviders state;
+
+  public LiveComponentsRefresher(StateAccessForProviders state) {
+    this.state = state;
+  }
+
+  @Override
+  public Map<String, ComponentInformation> refresh() {
+    return state.getComponentInfoSnapshot();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveContainersRefresher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveContainersRefresher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveContainersRefresher.java
new file mode 100644
index 0000000..68bd8a2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveContainersRefresher.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest.application.resources;
+
+import org.apache.slider.api.types.ContainerInformation;
+import org.apache.slider.server.appmaster.state.RoleInstance;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Refresh the container list.
+ */
+public class LiveContainersRefresher implements ResourceRefresher<Map<String, ContainerInformation>> {
+
+  private final StateAccessForProviders state;
+
+  public LiveContainersRefresher(StateAccessForProviders state) {
+    this.state = state;
+  }
+
+  @Override
+  public Map<String, ContainerInformation> refresh() throws
+      Exception {
+    List<RoleInstance> containerList = state.cloneOwnedContainerList();
+
+    Map<String, ContainerInformation> map = new HashMap<>();
+    for (RoleInstance instance : containerList) {
+      ContainerInformation serialized = instance.serialize();
+      map.put(serialized.containerId, serialized);
+    }
+    return map;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveNodesRefresher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveNodesRefresher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveNodesRefresher.java
new file mode 100644
index 0000000..aeb7a11
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveNodesRefresher.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest.application.resources;
+
+import org.apache.slider.api.types.NodeInformationList;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+
+/**
+ * Update the live nodes map
+ */
+public class LiveNodesRefresher
+    implements ResourceRefresher<NodeInformationList> {
+
+  private final StateAccessForProviders state;
+
+  public LiveNodesRefresher(StateAccessForProviders state) {
+    this.state = state;
+  }
+
+  @Override
+  public NodeInformationList refresh() {
+
+    return new NodeInformationList(state.getNodeInformationSnapshot().values());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveResourcesRefresher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveResourcesRefresher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveResourcesRefresher.java
new file mode 100644
index 0000000..f988297
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveResourcesRefresher.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest.application.resources;
+
+import org.apache.slider.api.StatusKeys;
+import org.apache.slider.core.conf.ConfTree;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.server.appmaster.state.RoleStatus;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+
+import java.util.Map;
+
+public class LiveResourcesRefresher implements ResourceRefresher<ConfTree> {
+
+  private final StateAccessForProviders state;
+
+  public LiveResourcesRefresher(StateAccessForProviders state) {
+    this.state = state;
+  }
+
+  @Override
+  public ConfTree refresh() throws Exception {
+
+    // snapshot resources
+    ConfTreeOperations resources = state.getResourcesSnapshot();
+    // then add actual values
+    Map<Integer, RoleStatus> roleStatusMap = state.getRoleStatusMap();
+    
+    for (RoleStatus status : roleStatusMap.values()) {
+      String name = status.getName();
+      resources.setComponentOpt(name,
+          StatusKeys.COMPONENT_INSTANCES_REQUESTING,
+          status.getRequested());
+      resources.setComponentOpt(name,
+          StatusKeys.COMPONENT_INSTANCES_ACTUAL,
+          status.getActual());
+      resources.setComponentOpt(name,
+          StatusKeys.COMPONENT_INSTANCES_RELEASING,
+          status.getReleasing());
+      resources.setComponentOpt(name,
+          StatusKeys.COMPONENT_INSTANCES_FAILED,
+          status.getFailed());
+      resources.setComponentOpt(name,
+          StatusKeys.COMPONENT_INSTANCES_COMPLETED,
+          status.getCompleted());
+      resources.setComponentOpt(name,
+          StatusKeys.COMPONENT_INSTANCES_STARTED,
+          status.getStarted());
+    }
+    return resources.getConfTree();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveStatisticsRefresher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveStatisticsRefresher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveStatisticsRefresher.java
new file mode 100644
index 0000000..d31b455
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/LiveStatisticsRefresher.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest.application.resources;
+
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+
+import java.util.Map;
+
+public class LiveStatisticsRefresher implements ResourceRefresher<Map<String,Integer>> {
+
+  private final StateAccessForProviders state;
+
+  public LiveStatisticsRefresher(StateAccessForProviders state) {
+    this.state = state;
+  }
+
+  @Override
+  public Map<String, Integer> refresh() throws Exception {
+
+    // snapshot resources
+    return state.getLiveStatistics();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceRefresher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceRefresher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceRefresher.java
new file mode 100644
index 0000000..35f0367
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceRefresher.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest.application.resources;
+
+/**
+ * Interface which must be implemented to act as a source for cached content.
+ * @param <T> type to return
+ */
+public interface ResourceRefresher<T> {
+  /**
+   * Build an up to date version of the data
+   * @return a new instance of the (JSON serializable) data
+   */
+  T refresh() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceSnapshotRefresher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceSnapshotRefresher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceSnapshotRefresher.java
new file mode 100644
index 0000000..c16912a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/application/resources/ResourceSnapshotRefresher.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest.application.resources;
+
+import org.apache.slider.core.conf.ConfTree;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+
+public class ResourceSnapshotRefresher implements ResourceRefresher<ConfTree> {
+
+  private final StateAccessForProviders state;
+
+  public ResourceSnapshotRefresher(StateAccessForProviders state) {
+    this.state = state;
+  }
+
+  @Override
+  public ConfTree refresh() throws Exception {
+
+    // snapshot resources
+    ConfTreeOperations resources = state.getResourcesSnapshot();
+      return resources.getConfTree();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java
new file mode 100644
index 0000000..f27711a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/ManagementResource.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.management;
+
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.apache.slider.server.appmaster.web.rest.AbstractSliderResource;
+import org.apache.slider.server.appmaster.web.rest.RestPaths;
+import org.apache.slider.server.appmaster.web.rest.management.resources.AggregateConfResource;
+import org.apache.slider.server.appmaster.web.rest.management.resources.ConfTreeResource;
+import org.apache.slider.server.appmaster.web.rest.management.resources.ResourceFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.UriInfo;
+import java.net.URI;
+import java.net.URL;
+
+/**
+ *
+ */
+public class ManagementResource extends AbstractSliderResource {
+  protected static final Logger log =
+      LoggerFactory.getLogger(ManagementResource.class);
+  public static final String CONFIG = "config";
+  public static final String APP_UNDER_MANAGEMENT = "/app";
+
+  public ManagementResource(WebAppApi slider) {
+    super(slider);
+  }
+
+  private void init(HttpServletResponse res) {
+    res.setContentType(null);
+  }
+
+  @GET
+  public Response getWadl (@Context HttpServletRequest request) {
+    return redirectToAppWadl(request);
+  }
+
+  @GET
+  @Path("/app")
+  @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+  public AggregateConfResource getAggregateConfiguration(@Context UriInfo uriInfo,
+                                                         @Context HttpServletResponse res) {
+    init(res);
+    return ResourceFactory.createAggregateConfResource(getAggregateConf(),
+                                                       uriInfo.getAbsolutePathBuilder());
+  }
+
+  @GET
+  @Path(APP_UNDER_MANAGEMENT+"/configurations/{config}")
+  @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+  public ConfTreeResource getConfTreeResource(@PathParam(CONFIG) String config,
+                                              @Context UriInfo uriInfo,
+                                              @Context HttpServletResponse res) {
+    init(res);
+    AggregateConfResource aggregateConf =
+        ResourceFactory.createAggregateConfResource(getAggregateConf(),
+      uriInfo.getBaseUriBuilder()
+      .path(RestPaths.SLIDER_CONTEXT_ROOT)
+      .path(RestPaths.MANAGEMENT + APP_UNDER_MANAGEMENT));
+    return aggregateConf.getConfTree(config);
+  }
+
+  protected AggregateConf getAggregateConf() {
+    return slider.getAppState().getInstanceDefinitionSnapshot();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/ActionsResource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/ActionsResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/ActionsResource.java
new file mode 100644
index 0000000..9b340fa
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/ActionsResource.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest.management.resources;
+
+public class ActionsResource {
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/AggregateConfResource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/AggregateConfResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/AggregateConfResource.java
new file mode 100644
index 0000000..794daf9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/management/resources/AggregateConfResource.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.management.resources;
+
+import org.apache.slider.core.conf.AggregateConf;
+import org.codehaus.jackson.annotate.JsonIgnore;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import javax.ws.rs.core.UriBuilder;
+import java.util.HashMap;
+import java.util.Map;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class AggregateConfResource {
+  private String href;
+  private final ConfTreeResource resources;
+  private final ConfTreeResource internal;
+  private final ConfTreeResource appConf;
+  @JsonIgnore
+  private Map<String, ConfTreeResource> confMap;
+
+  public AggregateConfResource(AggregateConf conf, UriBuilder uriBuilder) {
+    if (uriBuilder != null) {
+      this.href = uriBuilder.build().toASCIIString();
+      resources = ResourceFactory.createConfTreeResource(conf.getResources(),
+                   uriBuilder.clone().path("configurations").path("resources"));
+      internal = ResourceFactory.createConfTreeResource(conf.getInternal(),
+                   uriBuilder.clone().path("configurations").path("internal"));
+      appConf = ResourceFactory.createConfTreeResource(conf.getAppConf(),
+                   uriBuilder.clone().path("configurations").path("appConf"));
+      initConfMap();
+    } else {
+      resources = null;
+      internal = null;
+      appConf = null;
+    }
+  }
+
+  private void initConfMap() {
+    confMap = new HashMap<>();
+    confMap.put("internal", internal);
+    confMap.put("resources", resources);
+    confMap.put("appConf", appConf);
+  }
+
+  public AggregateConfResource() {
+    this(null, null);
+  }
+
+  public ConfTreeResource getConfTree(String name) {
+    return confMap.get(name);
+  }
+
+  public String getHref() {
+    return href;
+  }
+
+  public void setHref(String href) {
+    this.href = href;
+  }
+
+  public ConfTreeResource getResources() {
+    return resources;
+  }
+
+  public ConfTreeResource getInternal() {
+    return internal;
+  }
+
+  public ConfTreeResource getAppConf() {
+    return appConf;
+  }
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[44/76] [abbrv] hadoop git commit: YARN-5623. Apply SLIDER-1166 to yarn-native-services branch. Contributed by Gour Saha

Posted by ji...@apache.org.
YARN-5623. Apply SLIDER-1166 to yarn-native-services branch. Contributed by Gour Saha


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

Branch: refs/heads/yarn-native-services
Commit: cc7b200b37a099cd1503c050968abb0541cf0536
Parents: cb61fe3
Author: Jian He <ji...@apache.org>
Authored: Mon Sep 12 10:44:39 2016 +0800
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 .../org/apache/slider/client/SliderClient.java  |  9 ++++--
 .../apache/slider/core/zk/ZKIntegration.java    | 34 +++++++++++++++++---
 2 files changed, 36 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc7b200b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
index 5096bb7..fe4f1d2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
@@ -604,11 +604,14 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
       BlockingZKWatcher watcher = new BlockingZKWatcher();
       client = ZKIntegration.newInstance(registryQuorum, user, clusterName, true, false, watcher,
           ZKIntegration.SESSION_TIMEOUT);
-      client.init();
-      watcher.waitForZKConnection(2 * 1000);
+      boolean fromCache = client.init();
+      if (!fromCache) {
+        watcher.waitForZKConnection(2 * 1000);
+      }
     } catch (InterruptedException e) {
       client = null;
-      log.warn("Unable to connect to zookeeper quorum {}", registryQuorum, e);
+      log.warn("Interrupted - unable to connect to zookeeper quorum {}",
+          registryQuorum, e);
     } catch (IOException e) {
       log.warn("Unable to connect to zookeeper quorum {}", registryQuorum, e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc7b200b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKIntegration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKIntegration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKIntegration.java
index ca41e4b..6ed58d5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKIntegration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKIntegration.java
@@ -33,6 +33,8 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 
@@ -65,6 +67,8 @@ public class ZKIntegration implements Watcher, Closeable {
   private final String clustername;
   private final String userPath;
   private int sessionTimeout = SESSION_TIMEOUT;
+  private static final Map<String, ZooKeeper> ZK_SESSIONS = new HashMap<>();
+
 /**
  flag to set to indicate that the user path should be created if
  it is not already there
@@ -93,10 +97,32 @@ public class ZKIntegration implements Watcher, Closeable {
     this.userPath = mkSliderUserPath(username);
   }
 
-  public void init() throws IOException {
-    assert zookeeper == null;
-    log.debug("Binding ZK client to {}", zkConnection);
-    zookeeper = new ZooKeeper(zkConnection, sessionTimeout, this, canBeReadOnly);
+  /**
+   * Returns true only if an active ZK session is available and retrieved from
+   * cache, false when it has to create a new one.
+   *
+   * @return true if from cache, false when new session created
+   * @throws IOException
+   */
+  public synchronized boolean init() throws IOException {
+    if (zookeeper != null && getAlive()) {
+      return true;
+    }
+
+    synchronized (ZK_SESSIONS) {
+      if (ZK_SESSIONS.containsKey(zkConnection)) {
+        zookeeper = ZK_SESSIONS.get(zkConnection);
+      }
+      if (zookeeper == null || !getAlive()) {
+        log.info("Binding ZK client to {}", zkConnection);
+        zookeeper = new ZooKeeper(zkConnection, sessionTimeout, this,
+            canBeReadOnly);
+        ZK_SESSIONS.put(zkConnection, zookeeper);
+        return false;
+      } else {
+        return true;
+      }
+    }
   }
 
   /**


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[53/76] [abbrv] hadoop git commit: YARN-5769. Integrate update app lifetime using feature implemented in YARN-5611. Contributed by Jian He

Posted by ji...@apache.org.
YARN-5769. Integrate update app lifetime using feature implemented in YARN-5611. Contributed by Jian He


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

Branch: refs/heads/yarn-native-services
Commit: 4f3033b494fc723fafcfa5d0a901d39150539730
Parents: 427fe37
Author: Gour Saha <go...@apache.org>
Authored: Wed Nov 30 23:24:45 2016 -0800
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 .../api/impl/ApplicationApiService.java         | 25 +++++++++++++--
 .../org/apache/slider/client/SliderClient.java  | 33 +++++++++++++++++---
 .../AbstractClusterBuildingActionArgs.java      |  3 +-
 3 files changed, 53 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f3033b4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
index 6db69ac..c4f5d43 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
@@ -77,6 +77,7 @@ import org.apache.slider.common.params.ActionFreezeArgs;
 import org.apache.slider.common.params.ActionListArgs;
 import org.apache.slider.common.params.ActionRegistryArgs;
 import org.apache.slider.common.params.ActionThawArgs;
+import org.apache.slider.common.params.ActionUpdateArgs;
 import org.apache.slider.common.params.ComponentArgsDelegate;
 import org.apache.slider.common.tools.SliderUtils;
 import org.apache.slider.common.tools.SliderVersionInfo;
@@ -1398,14 +1399,34 @@ public class ApplicationApiService implements ApplicationApi {
     }
 
     // If new lifetime value specified then update it
-    if (updateAppData.getLifetime() != null) {
-      // TODO: Once YARN-3813 and YARN-4205 are available
+    if (updateAppData.getLifetime() != null
+        && updateAppData.getLifetime() > 0) {
+      try {
+        updateAppLifetime(appName, updateAppData.getLifetime());
+      } catch (Exception e) {
+        logger.error("Failed to update application (" + appName + ") lifetime ("
+            + updateAppData.getLifetime() + ")", e);
+        return Response.status(Status.INTERNAL_SERVER_ERROR).build();
+      }
     }
 
     // If nothing happens consider it a no-op
     return Response.status(Status.NO_CONTENT).build();
   }
 
+  private Void updateAppLifetime(String appName, long lifetime)
+      throws InterruptedException, YarnException, IOException {
+    return invokeSliderClientRunnable(new SliderClientContextRunnable<Void>() {
+      @Override public Void run(SliderClient sliderClient)
+          throws YarnException, IOException, InterruptedException {
+        ActionUpdateArgs args = new ActionUpdateArgs();
+        args.lifetime = lifetime;
+        sliderClient.actionUpdate(appName, args);
+        return null;
+      }
+    });
+  }
+
   // create default component and initialize with app level global values
   private List<Component> getDefaultComponentAsList(Application app) {
     List<Component> comps = getDefaultComponentAsList();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f3033b4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
index 3f2df0a..12f7870 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.security.alias.CredentialProvider;
 import org.apache.hadoop.security.alias.CredentialProviderFactory;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
@@ -65,6 +66,7 @@ import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException;
 import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hadoop.yarn.util.Times;
 import org.apache.slider.api.ClusterDescription;
 import org.apache.slider.api.ClusterNode;
 import org.apache.slider.api.SliderApplicationApi;
@@ -177,14 +179,12 @@ import org.codehaus.jettison.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.BufferedReader;
 import java.io.Console;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.InputStreamReader;
 import java.io.InterruptedIOException;
 import java.io.PrintStream;
 import java.io.PrintWriter;
@@ -1602,8 +1602,33 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   public int actionUpdate(String clustername,
       AbstractClusterBuildingActionArgs buildInfo) throws
       YarnException, IOException {
-    buildInstanceDefinition(clustername, buildInfo, true, true);
-    return EXIT_SUCCESS; 
+    if (buildInfo.lifetime > 0) {
+      updateLifetime(clustername, buildInfo.lifetime);
+    } else {
+      buildInstanceDefinition(clustername, buildInfo, true, true);
+    }
+    return EXIT_SUCCESS;
+  }
+
+  public void updateLifetime(String appName, long lifetime)
+      throws YarnException, IOException {
+    ApplicationReport report = findInstance(appName);
+    if (report == null) {
+      throw new YarnException("Application not found for " + appName);
+    }
+    ApplicationId appId = report.getApplicationId();
+    log.info("Updating lifetime of an application: appName = " + appName
+        + ", appId = " + appId+ ", lifetime = " + lifetime);
+    Map<ApplicationTimeoutType, String> map = new HashMap<>();
+    String newTimeout =
+        Times.formatISO8601(System.currentTimeMillis() + lifetime * 1000);
+    map.put(ApplicationTimeoutType.LIFETIME, newTimeout);
+    UpdateApplicationTimeoutsRequest request =
+        UpdateApplicationTimeoutsRequest.newInstance(appId, map);
+    yarnClient.updateApplicationTimeouts(request);
+    log.info("Successfully updated lifetime for an application: appName = "
+        + appName + ", appId = " + appId
+        + ". New expiry time in ISO8601 format is " + newTimeout);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f3033b4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractClusterBuildingActionArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractClusterBuildingActionArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractClusterBuildingActionArgs.java
index 3cb75e1..20a9989 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractClusterBuildingActionArgs.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractClusterBuildingActionArgs.java
@@ -103,8 +103,7 @@ public abstract class AbstractClusterBuildingActionArgs extends
   public String queue;
 
   @Parameter(names = {ARG_LIFETIME},
-      description = "Life time of the application since application started at"
-          + " running state")
+      description = "Lifetime of the application from the time of request")
   public long lifetime;
 
   @ParametersDelegate


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[36/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/proto/Messages.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/proto/Messages.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/proto/Messages.java
new file mode 100644
index 0000000..373d64d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/proto/Messages.java
@@ -0,0 +1,34473 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: SliderClusterMessages.proto
+
+package org.apache.slider.api.proto;
+
+public final class Messages {
+  private Messages() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface RoleInstanceStateOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required string name = 1;
+    /**
+     * <code>required string name = 1;</code>
+     */
+    boolean hasName();
+    /**
+     * <code>required string name = 1;</code>
+     */
+    java.lang.String getName();
+    /**
+     * <code>required string name = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getNameBytes();
+
+    // optional string role = 2;
+    /**
+     * <code>optional string role = 2;</code>
+     */
+    boolean hasRole();
+    /**
+     * <code>optional string role = 2;</code>
+     */
+    java.lang.String getRole();
+    /**
+     * <code>optional string role = 2;</code>
+     */
+    com.google.protobuf.ByteString
+        getRoleBytes();
+
+    // required uint32 state = 4;
+    /**
+     * <code>required uint32 state = 4;</code>
+     */
+    boolean hasState();
+    /**
+     * <code>required uint32 state = 4;</code>
+     */
+    int getState();
+
+    // required uint32 exitCode = 5;
+    /**
+     * <code>required uint32 exitCode = 5;</code>
+     */
+    boolean hasExitCode();
+    /**
+     * <code>required uint32 exitCode = 5;</code>
+     */
+    int getExitCode();
+
+    // optional string command = 6;
+    /**
+     * <code>optional string command = 6;</code>
+     */
+    boolean hasCommand();
+    /**
+     * <code>optional string command = 6;</code>
+     */
+    java.lang.String getCommand();
+    /**
+     * <code>optional string command = 6;</code>
+     */
+    com.google.protobuf.ByteString
+        getCommandBytes();
+
+    // optional string diagnostics = 7;
+    /**
+     * <code>optional string diagnostics = 7;</code>
+     */
+    boolean hasDiagnostics();
+    /**
+     * <code>optional string diagnostics = 7;</code>
+     */
+    java.lang.String getDiagnostics();
+    /**
+     * <code>optional string diagnostics = 7;</code>
+     */
+    com.google.protobuf.ByteString
+        getDiagnosticsBytes();
+
+    // repeated string output = 8;
+    /**
+     * <code>repeated string output = 8;</code>
+     */
+    java.util.List<java.lang.String>
+    getOutputList();
+    /**
+     * <code>repeated string output = 8;</code>
+     */
+    int getOutputCount();
+    /**
+     * <code>repeated string output = 8;</code>
+     */
+    java.lang.String getOutput(int index);
+    /**
+     * <code>repeated string output = 8;</code>
+     */
+    com.google.protobuf.ByteString
+        getOutputBytes(int index);
+
+    // repeated string environment = 9;
+    /**
+     * <code>repeated string environment = 9;</code>
+     */
+    java.util.List<java.lang.String>
+    getEnvironmentList();
+    /**
+     * <code>repeated string environment = 9;</code>
+     */
+    int getEnvironmentCount();
+    /**
+     * <code>repeated string environment = 9;</code>
+     */
+    java.lang.String getEnvironment(int index);
+    /**
+     * <code>repeated string environment = 9;</code>
+     */
+    com.google.protobuf.ByteString
+        getEnvironmentBytes(int index);
+
+    // required uint32 roleId = 10;
+    /**
+     * <code>required uint32 roleId = 10;</code>
+     */
+    boolean hasRoleId();
+    /**
+     * <code>required uint32 roleId = 10;</code>
+     */
+    int getRoleId();
+
+    // required bool released = 11;
+    /**
+     * <code>required bool released = 11;</code>
+     */
+    boolean hasReleased();
+    /**
+     * <code>required bool released = 11;</code>
+     */
+    boolean getReleased();
+
+    // required int64 createTime = 12;
+    /**
+     * <code>required int64 createTime = 12;</code>
+     */
+    boolean hasCreateTime();
+    /**
+     * <code>required int64 createTime = 12;</code>
+     */
+    long getCreateTime();
+
+    // required int64 startTime = 13;
+    /**
+     * <code>required int64 startTime = 13;</code>
+     */
+    boolean hasStartTime();
+    /**
+     * <code>required int64 startTime = 13;</code>
+     */
+    long getStartTime();
+
+    // required string host = 14;
+    /**
+     * <code>required string host = 14;</code>
+     */
+    boolean hasHost();
+    /**
+     * <code>required string host = 14;</code>
+     */
+    java.lang.String getHost();
+    /**
+     * <code>required string host = 14;</code>
+     */
+    com.google.protobuf.ByteString
+        getHostBytes();
+
+    // required string hostURL = 15;
+    /**
+     * <code>required string hostURL = 15;</code>
+     */
+    boolean hasHostURL();
+    /**
+     * <code>required string hostURL = 15;</code>
+     */
+    java.lang.String getHostURL();
+    /**
+     * <code>required string hostURL = 15;</code>
+     */
+    com.google.protobuf.ByteString
+        getHostURLBytes();
+
+    // optional string appVersion = 16;
+    /**
+     * <code>optional string appVersion = 16;</code>
+     */
+    boolean hasAppVersion();
+    /**
+     * <code>optional string appVersion = 16;</code>
+     */
+    java.lang.String getAppVersion();
+    /**
+     * <code>optional string appVersion = 16;</code>
+     */
+    com.google.protobuf.ByteString
+        getAppVersionBytes();
+  }
+  /**
+   * Protobuf type {@code org.apache.slider.api.RoleInstanceState}
+   */
+  public static final class RoleInstanceState extends
+      com.google.protobuf.GeneratedMessage
+      implements RoleInstanceStateOrBuilder {
+    // Use RoleInstanceState.newBuilder() to construct.
+    private RoleInstanceState(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private RoleInstanceState(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final RoleInstanceState defaultInstance;
+    public static RoleInstanceState getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public RoleInstanceState getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private RoleInstanceState(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              name_ = input.readBytes();
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              role_ = input.readBytes();
+              break;
+            }
+            case 32: {
+              bitField0_ |= 0x00000004;
+              state_ = input.readUInt32();
+              break;
+            }
+            case 40: {
+              bitField0_ |= 0x00000008;
+              exitCode_ = input.readUInt32();
+              break;
+            }
+            case 50: {
+              bitField0_ |= 0x00000010;
+              command_ = input.readBytes();
+              break;
+            }
+            case 58: {
+              bitField0_ |= 0x00000020;
+              diagnostics_ = input.readBytes();
+              break;
+            }
+            case 66: {
+              if (!((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
+                output_ = new com.google.protobuf.LazyStringArrayList();
+                mutable_bitField0_ |= 0x00000040;
+              }
+              output_.add(input.readBytes());
+              break;
+            }
+            case 74: {
+              if (!((mutable_bitField0_ & 0x00000080) == 0x00000080)) {
+                environment_ = new com.google.protobuf.LazyStringArrayList();
+                mutable_bitField0_ |= 0x00000080;
+              }
+              environment_.add(input.readBytes());
+              break;
+            }
+            case 80: {
+              bitField0_ |= 0x00000040;
+              roleId_ = input.readUInt32();
+              break;
+            }
+            case 88: {
+              bitField0_ |= 0x00000080;
+              released_ = input.readBool();
+              break;
+            }
+            case 96: {
+              bitField0_ |= 0x00000100;
+              createTime_ = input.readInt64();
+              break;
+            }
+            case 104: {
+              bitField0_ |= 0x00000200;
+              startTime_ = input.readInt64();
+              break;
+            }
+            case 114: {
+              bitField0_ |= 0x00000400;
+              host_ = input.readBytes();
+              break;
+            }
+            case 122: {
+              bitField0_ |= 0x00000800;
+              hostURL_ = input.readBytes();
+              break;
+            }
+            case 130: {
+              bitField0_ |= 0x00001000;
+              appVersion_ = input.readBytes();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
+          output_ = new com.google.protobuf.UnmodifiableLazyStringList(output_);
+        }
+        if (((mutable_bitField0_ & 0x00000080) == 0x00000080)) {
+          environment_ = new com.google.protobuf.UnmodifiableLazyStringList(environment_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.slider.api.proto.Messages.internal_static_org_apache_slider_api_RoleInstanceState_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.slider.api.proto.Messages.internal_static_org_apache_slider_api_RoleInstanceState_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.slider.api.proto.Messages.RoleInstanceState.class, org.apache.slider.api.proto.Messages.RoleInstanceState.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<RoleInstanceState> PARSER =
+        new com.google.protobuf.AbstractParser<RoleInstanceState>() {
+      public RoleInstanceState parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new RoleInstanceState(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<RoleInstanceState> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required string name = 1;
+    public static final int NAME_FIELD_NUMBER = 1;
+    private java.lang.Object name_;
+    /**
+     * <code>required string name = 1;</code>
+     */
+    public boolean hasName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required string name = 1;</code>
+     */
+    public java.lang.String getName() {
+      java.lang.Object ref = name_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          name_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string name = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getNameBytes() {
+      java.lang.Object ref = name_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        name_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional string role = 2;
+    public static final int ROLE_FIELD_NUMBER = 2;
+    private java.lang.Object role_;
+    /**
+     * <code>optional string role = 2;</code>
+     */
+    public boolean hasRole() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional string role = 2;</code>
+     */
+    public java.lang.String getRole() {
+      java.lang.Object ref = role_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          role_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string role = 2;</code>
+     */
+    public com.google.protobuf.ByteString
+        getRoleBytes() {
+      java.lang.Object ref = role_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        role_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // required uint32 state = 4;
+    public static final int STATE_FIELD_NUMBER = 4;
+    private int state_;
+    /**
+     * <code>required uint32 state = 4;</code>
+     */
+    public boolean hasState() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>required uint32 state = 4;</code>
+     */
+    public int getState() {
+      return state_;
+    }
+
+    // required uint32 exitCode = 5;
+    public static final int EXITCODE_FIELD_NUMBER = 5;
+    private int exitCode_;
+    /**
+     * <code>required uint32 exitCode = 5;</code>
+     */
+    public boolean hasExitCode() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>required uint32 exitCode = 5;</code>
+     */
+    public int getExitCode() {
+      return exitCode_;
+    }
+
+    // optional string command = 6;
+    public static final int COMMAND_FIELD_NUMBER = 6;
+    private java.lang.Object command_;
+    /**
+     * <code>optional string command = 6;</code>
+     */
+    public boolean hasCommand() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    /**
+     * <code>optional string command = 6;</code>
+     */
+    public java.lang.String getCommand() {
+      java.lang.Object ref = command_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          command_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string command = 6;</code>
+     */
+    public com.google.protobuf.ByteString
+        getCommandBytes() {
+      java.lang.Object ref = command_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        command_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional string diagnostics = 7;
+    public static final int DIAGNOSTICS_FIELD_NUMBER = 7;
+    private java.lang.Object diagnostics_;
+    /**
+     * <code>optional string diagnostics = 7;</code>
+     */
+    public boolean hasDiagnostics() {
+      return ((bitField0_ & 0x00000020) == 0x00000020);
+    }
+    /**
+     * <code>optional string diagnostics = 7;</code>
+     */
+    public java.lang.String getDiagnostics() {
+      java.lang.Object ref = diagnostics_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          diagnostics_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string diagnostics = 7;</code>
+     */
+    public com.google.protobuf.ByteString
+        getDiagnosticsBytes() {
+      java.lang.Object ref = diagnostics_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        diagnostics_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // repeated string output = 8;
+    public static final int OUTPUT_FIELD_NUMBER = 8;
+    private com.google.protobuf.LazyStringList output_;
+    /**
+     * <code>repeated string output = 8;</code>
+     */
+    public java.util.List<java.lang.String>
+        getOutputList() {
+      return output_;
+    }
+    /**
+     * <code>repeated string output = 8;</code>
+     */
+    public int getOutputCount() {
+      return output_.size();
+    }
+    /**
+     * <code>repeated string output = 8;</code>
+     */
+    public java.lang.String getOutput(int index) {
+      return output_.get(index);
+    }
+    /**
+     * <code>repeated string output = 8;</code>
+     */
+    public com.google.protobuf.ByteString
+        getOutputBytes(int index) {
+      return output_.getByteString(index);
+    }
+
+    // repeated string environment = 9;
+    public static final int ENVIRONMENT_FIELD_NUMBER = 9;
+    private com.google.protobuf.LazyStringList environment_;
+    /**
+     * <code>repeated string environment = 9;</code>
+     */
+    public java.util.List<java.lang.String>
+        getEnvironmentList() {
+      return environment_;
+    }
+    /**
+     * <code>repeated string environment = 9;</code>
+     */
+    public int getEnvironmentCount() {
+      return environment_.size();
+    }
+    /**
+     * <code>repeated string environment = 9;</code>
+     */
+    public java.lang.String getEnvironment(int index) {
+      return environment_.get(index);
+    }
+    /**
+     * <code>repeated string environment = 9;</code>
+     */
+    public com.google.protobuf.ByteString
+        getEnvironmentBytes(int index) {
+      return environment_.getByteString(index);
+    }
+
+    // required uint32 roleId = 10;
+    public static final int ROLEID_FIELD_NUMBER = 10;
+    private int roleId_;
+    /**
+     * <code>required uint32 roleId = 10;</code>
+     */
+    public boolean hasRoleId() {
+      return ((bitField0_ & 0x00000040) == 0x00000040);
+    }
+    /**
+     * <code>required uint32 roleId = 10;</code>
+     */
+    public int getRoleId() {
+      return roleId_;
+    }
+
+    // required bool released = 11;
+    public static final int RELEASED_FIELD_NUMBER = 11;
+    private boolean released_;
+    /**
+     * <code>required bool released = 11;</code>
+     */
+    public boolean hasReleased() {
+      return ((bitField0_ & 0x00000080) == 0x00000080);
+    }
+    /**
+     * <code>required bool released = 11;</code>
+     */
+    public boolean getReleased() {
+      return released_;
+    }
+
+    // required int64 createTime = 12;
+    public static final int CREATETIME_FIELD_NUMBER = 12;
+    private long createTime_;
+    /**
+     * <code>required int64 createTime = 12;</code>
+     */
+    public boolean hasCreateTime() {
+      return ((bitField0_ & 0x00000100) == 0x00000100);
+    }
+    /**
+     * <code>required int64 createTime = 12;</code>
+     */
+    public long getCreateTime() {
+      return createTime_;
+    }
+
+    // required int64 startTime = 13;
+    public static final int STARTTIME_FIELD_NUMBER = 13;
+    private long startTime_;
+    /**
+     * <code>required int64 startTime = 13;</code>
+     */
+    public boolean hasStartTime() {
+      return ((bitField0_ & 0x00000200) == 0x00000200);
+    }
+    /**
+     * <code>required int64 startTime = 13;</code>
+     */
+    public long getStartTime() {
+      return startTime_;
+    }
+
+    // required string host = 14;
+    public static final int HOST_FIELD_NUMBER = 14;
+    private java.lang.Object host_;
+    /**
+     * <code>required string host = 14;</code>
+     */
+    public boolean hasHost() {
+      return ((bitField0_ & 0x00000400) == 0x00000400);
+    }
+    /**
+     * <code>required string host = 14;</code>
+     */
+    public java.lang.String getHost() {
+      java.lang.Object ref = host_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          host_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string host = 14;</code>
+     */
+    public com.google.protobuf.ByteString
+        getHostBytes() {
+      java.lang.Object ref = host_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        host_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // required string hostURL = 15;
+    public static final int HOSTURL_FIELD_NUMBER = 15;
+    private java.lang.Object hostURL_;
+    /**
+     * <code>required string hostURL = 15;</code>
+     */
+    public boolean hasHostURL() {
+      return ((bitField0_ & 0x00000800) == 0x00000800);
+    }
+    /**
+     * <code>required string hostURL = 15;</code>
+     */
+    public java.lang.String getHostURL() {
+      java.lang.Object ref = hostURL_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          hostURL_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string hostURL = 15;</code>
+     */
+    public com.google.protobuf.ByteString
+        getHostURLBytes() {
+      java.lang.Object ref = hostURL_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        hostURL_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional string appVersion = 16;
+    public static final int APPVERSION_FIELD_NUMBER = 16;
+    private java.lang.Object appVersion_;
+    /**
+     * <code>optional string appVersion = 16;</code>
+     */
+    public boolean hasAppVersion() {
+      return ((bitField0_ & 0x00001000) == 0x00001000);
+    }
+    /**
+     * <code>optional string appVersion = 16;</code>
+     */
+    public java.lang.String getAppVersion() {
+      java.lang.Object ref = appVersion_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          appVersion_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string appVersion = 16;</code>
+     */
+    public com.google.protobuf.ByteString
+        getAppVersionBytes() {
+      java.lang.Object ref = appVersion_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        appVersion_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    private void initFields() {
+      name_ = "";
+      role_ = "";
+      state_ = 0;
+      exitCode_ = 0;
+      command_ = "";
+      diagnostics_ = "";
+      output_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      environment_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      roleId_ = 0;
+      released_ = false;
+      createTime_ = 0L;
+      startTime_ = 0L;
+      host_ = "";
+      hostURL_ = "";
+      appVersion_ = "";
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasState()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasExitCode()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasRoleId()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasReleased()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasCreateTime()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasStartTime()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasHost()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasHostURL()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getNameBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, getRoleBytes());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeUInt32(4, state_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeUInt32(5, exitCode_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeBytes(6, getCommandBytes());
+      }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        output.writeBytes(7, getDiagnosticsBytes());
+      }
+      for (int i = 0; i < output_.size(); i++) {
+        output.writeBytes(8, output_.getByteString(i));
+      }
+      for (int i = 0; i < environment_.size(); i++) {
+        output.writeBytes(9, environment_.getByteString(i));
+      }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        output.writeUInt32(10, roleId_);
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        output.writeBool(11, released_);
+      }
+      if (((bitField0_ & 0x00000100) == 0x00000100)) {
+        output.writeInt64(12, createTime_);
+      }
+      if (((bitField0_ & 0x00000200) == 0x00000200)) {
+        output.writeInt64(13, startTime_);
+      }
+      if (((bitField0_ & 0x00000400) == 0x00000400)) {
+        output.writeBytes(14, getHostBytes());
+      }
+      if (((bitField0_ & 0x00000800) == 0x00000800)) {
+        output.writeBytes(15, getHostURLBytes());
+      }
+      if (((bitField0_ & 0x00001000) == 0x00001000)) {
+        output.writeBytes(16, getAppVersionBytes());
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getNameBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, getRoleBytes());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(4, state_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(5, exitCode_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(6, getCommandBytes());
+      }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(7, getDiagnosticsBytes());
+      }
+      {
+        int dataSize = 0;
+        for (int i = 0; i < output_.size(); i++) {
+          dataSize += com.google.protobuf.CodedOutputStream
+            .computeBytesSizeNoTag(output_.getByteString(i));
+        }
+        size += dataSize;
+        size += 1 * getOutputList().size();
+      }
+      {
+        int dataSize = 0;
+        for (int i = 0; i < environment_.size(); i++) {
+          dataSize += com.google.protobuf.CodedOutputStream
+            .computeBytesSizeNoTag(environment_.getByteString(i));
+        }
+        size += dataSize;
+        size += 1 * getEnvironmentList().size();
+      }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(10, roleId_);
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(11, released_);
+      }
+      if (((bitField0_ & 0x00000100) == 0x00000100)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt64Size(12, createTime_);
+      }
+      if (((bitField0_ & 0x00000200) == 0x00000200)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt64Size(13, startTime_);
+      }
+      if (((bitField0_ & 0x00000400) == 0x00000400)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(14, getHostBytes());
+      }
+      if (((bitField0_ & 0x00000800) == 0x00000800)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(15, getHostURLBytes());
+      }
+      if (((bitField0_ & 0x00001000) == 0x00001000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(16, getAppVersionBytes());
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.slider.api.proto.Messages.RoleInstanceState)) {
+        return super.equals(obj);
+      }
+      org.apache.slider.api.proto.Messages.RoleInstanceState other = (org.apache.slider.api.proto.Messages.RoleInstanceState) obj;
+
+      boolean result = true;
+      result = result && (hasName() == other.hasName());
+      if (hasName()) {
+        result = result && getName()
+            .equals(other.getName());
+      }
+      result = result && (hasRole() == other.hasRole());
+      if (hasRole()) {
+        result = result && getRole()
+            .equals(other.getRole());
+      }
+      result = result && (hasState() == other.hasState());
+      if (hasState()) {
+        result = result && (getState()
+            == other.getState());
+      }
+      result = result && (hasExitCode() == other.hasExitCode());
+      if (hasExitCode()) {
+        result = result && (getExitCode()
+            == other.getExitCode());
+      }
+      result = result && (hasCommand() == other.hasCommand());
+      if (hasCommand()) {
+        result = result && getCommand()
+            .equals(other.getCommand());
+      }
+      result = result && (hasDiagnostics() == other.hasDiagnostics());
+      if (hasDiagnostics()) {
+        result = result && getDiagnostics()
+            .equals(other.getDiagnostics());
+      }
+      result = result && getOutputList()
+          .equals(other.getOutputList());
+      result = result && getEnvironmentList()
+          .equals(other.getEnvironmentList());
+      result = result && (hasRoleId() == other.hasRoleId());
+      if (hasRoleId()) {
+        result = result && (getRoleId()
+            == other.getRoleId());
+      }
+      result = result && (hasReleased() == other.hasReleased());
+      if (hasReleased()) {
+        result = result && (getReleased()
+            == other.getReleased());
+      }
+      result = result && (hasCreateTime() == other.hasCreateTime());
+      if (hasCreateTime()) {
+        result = result && (getCreateTime()
+            == other.getCreateTime());
+      }
+      result = result && (hasStartTime() == other.hasStartTime());
+      if (hasStartTime()) {
+        result = result && (getStartTime()
+            == other.getStartTime());
+      }
+      result = result && (hasHost() == other.hasHost());
+      if (hasHost()) {
+        result = result && getHost()
+            .equals(other.getHost());
+      }
+      result = result && (hasHostURL() == other.hasHostURL());
+      if (hasHostURL()) {
+        result = result && getHostURL()
+            .equals(other.getHostURL());
+      }
+      result = result && (hasAppVersion() == other.hasAppVersion());
+      if (hasAppVersion()) {
+        result = result && getAppVersion()
+            .equals(other.getAppVersion());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasName()) {
+        hash = (37 * hash) + NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getName().hashCode();
+      }
+      if (hasRole()) {
+        hash = (37 * hash) + ROLE_FIELD_NUMBER;
+        hash = (53 * hash) + getRole().hashCode();
+      }
+      if (hasState()) {
+        hash = (37 * hash) + STATE_FIELD_NUMBER;
+        hash = (53 * hash) + getState();
+      }
+      if (hasExitCode()) {
+        hash = (37 * hash) + EXITCODE_FIELD_NUMBER;
+        hash = (53 * hash) + getExitCode();
+      }
+      if (hasCommand()) {
+        hash = (37 * hash) + COMMAND_FIELD_NUMBER;
+        hash = (53 * hash) + getCommand().hashCode();
+      }
+      if (hasDiagnostics()) {
+        hash = (37 * hash) + DIAGNOSTICS_FIELD_NUMBER;
+        hash = (53 * hash) + getDiagnostics().hashCode();
+      }
+      if (getOutputCount() > 0) {
+        hash = (37 * hash) + OUTPUT_FIELD_NUMBER;
+        hash = (53 * hash) + getOutputList().hashCode();
+      }
+      if (getEnvironmentCount() > 0) {
+        hash = (37 * hash) + ENVIRONMENT_FIELD_NUMBER;
+        hash = (53 * hash) + getEnvironmentList().hashCode();
+      }
+      if (hasRoleId()) {
+        hash = (37 * hash) + ROLEID_FIELD_NUMBER;
+        hash = (53 * hash) + getRoleId();
+      }
+      if (hasReleased()) {
+        hash = (37 * hash) + RELEASED_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getReleased());
+      }
+      if (hasCreateTime()) {
+        hash = (37 * hash) + CREATETIME_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getCreateTime());
+      }
+      if (hasStartTime()) {
+        hash = (37 * hash) + STARTTIME_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getStartTime());
+      }
+      if (hasHost()) {
+        hash = (37 * hash) + HOST_FIELD_NUMBER;
+        hash = (53 * hash) + getHost().hashCode();
+      }
+      if (hasHostURL()) {
+        hash = (37 * hash) + HOSTURL_FIELD_NUMBER;
+        hash = (53 * hash) + getHostURL().hashCode();
+      }
+      if (hasAppVersion()) {
+        hash = (37 * hash) + APPVERSION_FIELD_NUMBER;
+        hash = (53 * hash) + getAppVersion().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.slider.api.proto.Messages.RoleInstanceState parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.slider.api.proto.Messages.RoleInstanceState parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.slider.api.proto.Messages.RoleInstanceState parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.slider.api.proto.Messages.RoleInstanceState parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.slider.api.proto.Messages.RoleInstanceState parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.slider.api.proto.Messages.RoleInstanceState parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.slider.api.proto.Messages.RoleInstanceState parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.slider.api.proto.Messages.RoleInstanceState parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.slider.api.proto.Messages.RoleInstanceState parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.slider.api.proto.Messages.RoleInstanceState parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.slider.api.proto.Messages.RoleInstanceState prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code org.apache.slider.api.RoleInstanceState}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.slider.api.proto.Messages.RoleInstanceStateOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.slider.api.proto.Messages.internal_static_org_apache_slider_api_RoleInstanceState_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.slider.api.proto.Messages.internal_static_org_apache_slider_api_RoleInstanceState_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.slider.api.proto.Messages.RoleInstanceState.class, org.apache.slider.api.proto.Messages.RoleInstanceState.Builder.class);
+      }
+
+      // Construct using org.apache.slider.api.proto.Messages.RoleInstanceState.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        name_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        role_ = "";
+        bitField0_ = (bitField0_ & ~0x00000002);
+        state_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        exitCode_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000008);
+        command_ = "";
+        bitField0_ = (bitField0_ & ~0x00000010);
+        diagnostics_ = "";
+        bitField0_ = (bitField0_ & ~0x00000020);
+        output_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000040);
+        environment_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000080);
+        roleId_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000100);
+        released_ = false;
+        bitField0_ = (bitField0_ & ~0x00000200);
+        createTime_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000400);
+        startTime_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000800);
+        host_ = "";
+        bitField0_ = (bitField0_ & ~0x00001000);
+        hostURL_ = "";
+        bitField0_ = (bitField0_ & ~0x00002000);
+        appVersion_ = "";
+        bitField0_ = (bitField0_ & ~0x00004000);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.slider.api.proto.Messages.internal_static_org_apache_slider_api_RoleInstanceState_descriptor;
+      }
+
+      public org.apache.slider.api.proto.Messages.RoleInstanceState getDefaultInstanceForType() {
+        return org.apache.slider.api.proto.Messages.RoleInstanceState.getDefaultInstance();
+      }
+
+      public org.apache.slider.api.proto.Messages.RoleInstanceState build() {
+        org.apache.slider.api.proto.Messages.RoleInstanceState result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.slider.api.proto.Messages.RoleInstanceState buildPartial() {
+        org.apache.slider.api.proto.Messages.RoleInstanceState result = new org.apache.slider.api.proto.Messages.RoleInstanceState(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.name_ = name_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.role_ = role_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.state_ = state_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.exitCode_ = exitCode_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.command_ = command_;
+        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
+          to_bitField0_ |= 0x00000020;
+        }
+        result.diagnostics_ = diagnostics_;
+        if (((bitField0_ & 0x00000040) == 0x00000040)) {
+          output_ = new com.google.protobuf.UnmodifiableLazyStringList(
+              output_);
+          bitField0_ = (bitField0_ & ~0x00000040);
+        }
+        result.output_ = output_;
+        if (((bitField0_ & 0x00000080) == 0x00000080)) {
+          environment_ = new com.google.protobuf.UnmodifiableLazyStringList(
+              environment_);
+          bitField0_ = (bitField0_ & ~0x00000080);
+        }
+        result.environment_ = environment_;
+        if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
+          to_bitField0_ |= 0x00000040;
+        }
+        result.roleId_ = roleId_;
+        if (((from_bitField0_ & 0x00000200) == 0x00000200)) {
+          to_bitField0_ |= 0x00000080;
+        }
+        result.released_ = released_;
+        if (((from_bitField0_ & 0x00000400) == 0x00000400)) {
+          to_bitField0_ |= 0x00000100;
+        }
+        result.createTime_ = createTime_;
+        if (((from_bitField0_ & 0x00000800) == 0x00000800)) {
+          to_bitField0_ |= 0x00000200;
+        }
+        result.startTime_ = startTime_;
+        if (((from_bitField0_ & 0x00001000) == 0x00001000)) {
+          to_bitField0_ |= 0x00000400;
+        }
+        result.host_ = host_;
+        if (((from_bitField0_ & 0x00002000) == 0x00002000)) {
+          to_bitField0_ |= 0x00000800;
+        }
+        result.hostURL_ = hostURL_;
+        if (((from_bitField0_ & 0x00004000) == 0x00004000)) {
+          to_bitField0_ |= 0x00001000;
+        }
+        result.appVersion_ = appVersion_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.slider.api.proto.Messages.RoleInstanceState) {
+          return mergeFrom((org.apache.slider.api.proto.Messages.RoleInstanceState)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.slider.api.proto.Messages.RoleInstanceState other) {
+        if (other == org.apache.slider.api.proto.Messages.RoleInstanceState.getDefaultInstance()) return this;
+        if (other.hasName()) {
+          bitField0_ |= 0x00000001;
+          name_ = other.name_;
+          onChanged();
+        }
+        if (other.hasRole()) {
+          bitField0_ |= 0x00000002;
+          role_ = other.role_;
+          onChanged();
+        }
+        if (other.hasState()) {
+          setState(other.getState());
+        }
+        if (other.hasExitCode()) {
+          setExitCode(other.getExitCode());
+        }
+        if (other.hasCommand()) {
+          bitField0_ |= 0x00000010;
+          command_ = other.command_;
+          onChanged();
+        }
+        if (other.hasDiagnostics()) {
+          bitField0_ |= 0x00000020;
+          diagnostics_ = other.diagnostics_;
+          onChanged();
+        }
+        if (!other.output_.isEmpty()) {
+          if (output_.isEmpty()) {
+            output_ = other.output_;
+            bitField0_ = (bitField0_ & ~0x00000040);
+          } else {
+            ensureOutputIsMutable();
+            output_.addAll(other.output_);
+          }
+          onChanged();
+        }
+        if (!other.environment_.isEmpty()) {
+          if (environment_.isEmpty()) {
+            environment_ = other.environment_;
+            bitField0_ = (bitField0_ & ~0x00000080);
+          } else {
+            ensureEnvironmentIsMutable();
+            environment_.addAll(other.environment_);
+          }
+          onChanged();
+        }
+        if (other.hasRoleId()) {
+          setRoleId(other.getRoleId());
+        }
+        if (other.hasReleased()) {
+          setReleased(other.getReleased());
+        }
+        if (other.hasCreateTime()) {
+          setCreateTime(other.getCreateTime());
+        }
+        if (other.hasStartTime()) {
+          setStartTime(other.getStartTime());
+        }
+        if (other.hasHost()) {
+          bitField0_ |= 0x00001000;
+          host_ = other.host_;
+          onChanged();
+        }
+        if (other.hasHostURL()) {
+          bitField0_ |= 0x00002000;
+          hostURL_ = other.hostURL_;
+          onChanged();
+        }
+        if (other.hasAppVersion()) {
+          bitField0_ |= 0x00004000;
+          appVersion_ = other.appVersion_;
+          onChanged();
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasName()) {
+          
+          return false;
+        }
+        if (!hasState()) {
+          
+          return false;
+        }
+        if (!hasExitCode()) {
+          
+          return false;
+        }
+        if (!hasRoleId()) {
+          
+          return false;
+        }
+        if (!hasReleased()) {
+          
+          return false;
+        }
+        if (!hasCreateTime()) {
+          
+          return false;
+        }
+        if (!hasStartTime()) {
+          
+          return false;
+        }
+        if (!hasHost()) {
+          
+          return false;
+        }
+        if (!hasHostURL()) {
+          
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.slider.api.proto.Messages.RoleInstanceState parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.slider.api.proto.Messages.RoleInstanceState) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required string name = 1;
+      private java.lang.Object name_ = "";
+      /**
+       * <code>required string name = 1;</code>
+       */
+      public boolean hasName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string name = 1;</code>
+       */
+      public java.lang.String getName() {
+        java.lang.Object ref = name_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          name_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string name = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getNameBytes() {
+        java.lang.Object ref = name_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          name_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string name = 1;</code>
+       */
+      public Builder setName(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        name_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string name = 1;</code>
+       */
+      public Builder clearName() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        name_ = getDefaultInstance().getName();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string name = 1;</code>
+       */
+      public Builder setNameBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        name_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional string role = 2;
+      private java.lang.Object role_ = "";
+      /**
+       * <code>optional string role = 2;</code>
+       */
+      public boolean hasRole() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional string role = 2;</code>
+       */
+      public java.lang.String getRole() {
+        java.lang.Object ref = role_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          role_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string role = 2;</code>
+       */
+      public com.google.protobuf.ByteString
+          getRoleBytes() {
+        java.lang.Object ref = role_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          role_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string role = 2;</code>
+       */
+      public Builder setRole(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        role_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string role = 2;</code>
+       */
+      public Builder clearRole() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        role_ = getDefaultInstance().getRole();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string role = 2;</code>
+       */
+      public Builder setRoleBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        role_ = value;
+        onChanged();
+        return this;
+      }
+
+      // required uint32 state = 4;
+      private int state_ ;
+      /**
+       * <code>required uint32 state = 4;</code>
+       */
+      public boolean hasState() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>required uint32 state = 4;</code>
+       */
+      public int getState() {
+        return state_;
+      }
+      /**
+       * <code>required uint32 state = 4;</code>
+       */
+      public Builder setState(int value) {
+        bitField0_ |= 0x00000004;
+        state_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint32 state = 4;</code>
+       */
+      public Builder clearState() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        state_ = 0;
+        onChanged();
+        return this;
+      }
+
+      // required uint32 exitCode = 5;
+      private int exitCode_ ;
+      /**
+       * <code>required uint32 exitCode = 5;</code>
+       */
+      public boolean hasExitCode() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>required uint32 exitCode = 5;</code>
+       */
+      public int getExitCode() {
+        return exitCode_;
+      }
+      /**
+       * <code>required uint32 exitCode = 5;</code>
+       */
+      public Builder setExitCode(int value) {
+        bitField0_ |= 0x00000008;
+        exitCode_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint32 exitCode = 5;</code>
+       */
+      public Builder clearExitCode() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        exitCode_ = 0;
+        onChanged();
+        return this;
+      }
+
+      // optional string command = 6;
+      private java.lang.Object command_ = "";
+      /**
+       * <code>optional string command = 6;</code>
+       */
+      public boolean hasCommand() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      /**
+       * <code>optional string command = 6;</code>
+       */
+      public java.lang.String getCommand() {
+        java.lang.Object ref = command_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          command_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string command = 6;</code>
+       */
+      public com.google.protobuf.ByteString
+          getCommandBytes() {
+        java.lang.Object ref = command_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          command_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string command = 6;</code>
+       */
+      public Builder setCommand(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000010;
+        command_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string command = 6;</code>
+       */
+      public Builder clearCommand() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        command_ = getDefaultInstance().getCommand();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string command = 6;</code>
+       */
+      public Builder setCommandBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000010;
+        command_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional string diagnostics = 7;
+      private java.lang.Object diagnostics_ = "";
+      /**
+       * <code>optional string diagnostics = 7;</code>
+       */
+      public boolean hasDiagnostics() {
+        return ((bitField0_ & 0x00000020) == 0x00000020);
+      }
+      /**
+       * <code>optional string diagnostics = 7;</code>
+       */
+      public java.lang.String getDiagnostics() {
+        java.lang.Object ref = diagnostics_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          diagnostics_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string diagnostics = 7;</code>
+       */
+      public com.google.protobuf.ByteString
+          getDiagnosticsBytes() {
+        java.lang.Object ref = diagnostics_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          diagnostics_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string diagnostics = 7;</code>
+       */
+      public Builder setDiagnostics(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000020;
+        diagnostics_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string diagnostics = 7;</code>
+       */
+      public Builder clearDiagnostics() {
+        bitField0_ = (bitField0_ & ~0x00000020);
+        diagnostics_ = getDefaultInstance().getDiagnostics();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string diagnostics = 7;</code>
+       */
+      public Builder setDiagnosticsBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000020;
+        diagnostics_ = value;
+        onChanged();
+        return this;
+      }
+
+      // repeated string output = 8;
+      private com.google.protobuf.LazyStringList output_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      private void ensureOutputIsMutable() {
+        if (!((bitField0_ & 0x00000040) == 0x00000040)) {
+          output_ = new com.google.protobuf.LazyStringArrayList(output_);
+          bitField0_ |= 0x00000040;
+         }
+      }
+      /**
+       * <code>repeated string output = 8;</code>
+       */
+      public java.util.List<java.lang.String>
+          getOutputList() {
+        return java.util.Collections.unmodifiableList(output_);
+      }
+      /**
+       * <code>repeated string output = 8;</code>
+       */
+      public int getOutputCount() {
+        return output_.size();
+      }
+      /**
+       * <code>repeated string output = 8;</code>
+       */
+      public java.lang.String getOutput(int index) {
+        return output_.get(index);
+      }
+      /**
+       * <code>repeated string output = 8;</code>
+       */
+      public com.google.protobuf.ByteString
+          getOutputBytes(int index) {
+        return output_.getByteString(index);
+      }
+      /**
+       * <code>repeated string output = 8;</code>
+       */
+      public Builder setOutput(
+          int index, java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureOutputIsMutable();
+        output_.set(index, value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string output = 8;</code>
+       */
+      public Builder addOutput(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureOutputIsMutable();
+        output_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string output = 8;</code>
+       */
+      public Builder addAllOutput(
+          java.lang.Iterable<java.lang.String> values) {
+        ensureOutputIsMutable();
+        super.addAll(values, output_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string output = 8;</code>
+       */
+      public Builder clearOutput() {
+        output_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000040);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string output = 8;</code>
+       */
+      public Builder addOutputBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureOutputIsMutable();
+        output_.add(value);
+        onChanged();
+        return this;
+      }
+
+      // repeated string environment = 9;
+      private com.google.protobuf.LazyStringList environment_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      private void ensureEnvironmentIsMutable() {
+        if (!((bitField0_ & 0x00000080) == 0x00000080)) {
+          environment_ = new com.google.protobuf.LazyStringArrayList(environment_);
+          bitField0_ |= 0x00000080;
+         }
+      }
+      /**
+       * <code>repeated string environment = 9;</code>
+       */
+      public java.util.List<java.lang.String>
+          getEnvironmentList() {
+        return java.util.Collections.unmodifiableList(environment_);
+      }
+      /**
+       * <code>repeated string environment = 9;</code>
+       */
+      public int getEnvironmentCount() {
+        return environment_.size();
+      }
+      /**
+       * <code>repeated string environment = 9;</code>
+       */
+      public java.lang.String getEnvironment(int index) {
+        return environment_.get(index);
+      }
+      /**
+       * <code>repeated string environment = 9;</code>
+       */
+      public com.google.protobuf.ByteString
+          getEnvironmentBytes(int index) {
+        return environment_.getByteString(index);
+      }
+      /**
+       * <code>repeated string environment = 9;</code>
+       */
+      public Builder setEnvironment(
+          int index, java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureEnvironmentIsMutable();
+        environment_.set(index, value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string environment = 9;</code>
+       */
+      public Builder addEnvironment(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureEnvironmentIsMutable();
+        environment_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string environment = 9;</code>
+       */
+      public Builder addAllEnvironment(
+          java.lang.Iterable<java.lang.String> values) {
+        ensureEnvironmentIsMutable();
+        super.addAll(values, environment_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string environment = 9;</code>
+       */
+      public Builder clearEnvironment() {
+        environment_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000080);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string environment = 9;</code>
+       */
+      public Builder addEnvironmentBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureEnvironmentIsMutable();
+        environment_.add(value);
+        onChanged();
+        return this;
+      }
+
+      // required uint32 roleId = 10;
+      private int roleId_ ;
+      /**
+       * <code>required uint32 roleId = 10;</code>
+       */
+      public boolean hasRoleId() {
+        return ((bitField0_ & 0x00000100) == 0x00000100);
+      }
+      /**
+       * <code>required uint32 roleId = 10;</code>
+       */
+      public int getRoleId() {
+        return roleId_;
+      }
+      /**
+       * <code>required uint32 roleId = 10;</code>
+       */
+      public Builder setRoleId(int value) {
+        bitField0_ |= 0x00000100;
+        roleId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint32 roleId = 10;</code>
+       */
+      public Builder clearRoleId() {
+        bitField0_ = (bitField0_ & ~0x00000100);
+        roleId_ = 0;
+        onChanged();
+        return this;
+      }
+
+      // required bool released = 11;
+      private boolean released_ ;
+      /**
+       * <code>required bool released = 11;</code>
+       */
+      public boolean hasReleased() {
+        return ((bitField0_ & 0x00000200) == 0x00000200);
+      }
+      /**
+       * <code>required bool released = 11;</code>
+       */
+      public boolean getReleased() {
+        return released_;
+      }
+      /**
+       * <code>required bool released = 11;</code>
+       */
+      public Builder setReleased(boolean value) {
+        bitField0_ |= 0x00000200;
+        released_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required bool released = 11;</code>
+       */
+      public Builder clearReleased() {
+        bitField0_ = (bitField0_ & ~0x00000200);
+        released_ = false;
+        onChanged();
+        return this;
+      }
+
+      // required int64 createTime = 12;
+      private long createTime_ ;
+      /**
+       * <code>required int64 createTime = 12;</code>
+       */
+      public boolean hasCreateTime() {
+        return ((bitField0_ & 0x00000400) == 0x00000400);
+      }
+      /**
+       * <code>required int64 createTime = 12;</code>
+       */
+      public long getCreateTime() {
+        return createTime_;
+      }
+      /**
+       * <code>required int64 createTime = 12;</code>
+       */
+      public Builder setCreateTime(long value) {
+        bitField0_ |= 0x00000400;
+        createTime_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required int64 createTime = 12;</code>
+       */
+      public Builder clearCreateTime() {
+        bitField0_ = (bitField0_ & ~0x00000400);
+        createTime_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // required int64 startTime = 13;
+      private long startTime_ ;
+      /**
+       * <code>required int64 startTime = 13;</code>
+       */
+      public boolean hasStartTime() {
+        return ((bitField0_ & 0x00000800) == 0x00000800);
+      }
+      /**
+       * <code>required int64 startTime = 13;</code>
+       */
+      public long getStartTime() {
+        return startTime_;
+      }
+      /**
+       * <code>required int64 startTime = 13;</code>
+       */
+      public Builder setStartTime(long value) {
+        bitField0_ |= 0x00000800;
+        startTime_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required int64 startTime = 13;</code>
+       */
+      public Builder clearStartTime() {
+        bitField0_ = (bitField0_ & ~0x00000800);
+        startTime_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // required string host = 14;
+      private java.lang.Object host_ = "";
+      /**
+       * <code>required string host = 14;</code>
+       */
+      public boolean hasHost() {
+        return ((bitField0_ & 0x00001000) == 0x00001000);
+      }
+      /**
+       * <code>required string host = 14;</code>
+       */
+      public java.lang.String getHost() {
+        java.lang.Object ref = host_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          host_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string host = 14;</code>
+       */
+      public com.google.protobuf.ByteString
+          getHostBytes() {
+        java.lang.Object ref = host_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          host_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string host = 14;</code>
+       */
+      public Builder setHost(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00001000;
+        host_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string host = 14;</code>
+       */
+      public Builder clearHost() {
+        bitField0_ = (bitField0_ & ~0x00001000);
+        host_ = getDefaultInstance().getHost();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string host = 14;</code>
+       */
+      public Builder setHostBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00001000;
+        host_ = value;
+        onChanged();
+        return this;
+      }
+
+      // required string hostURL = 15;
+      private java.lang.Object hostURL_ = "";
+      /**
+       * <code>required string hostURL = 15;</code>
+       */
+      public boolean hasHostURL() {
+        return ((bitField0_ & 0x00002000) == 0x00002000);
+      }
+      /**
+       * <code>required string hostURL = 15;</code>
+       */
+      public java.lang.String getHostURL() {
+        java.lang.Object ref = hostURL_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          hostURL_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string hostURL = 15;</code>
+       */
+      public com.google.protobuf.ByteString
+          getHostURLBytes() {
+        java.lang.Object ref = hostURL_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          hostURL_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string hostURL = 15;</code>
+       */
+      public Builder setHostURL(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00002000;
+        hostURL_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string hostURL = 15;</code>
+       */
+      public Builder clearHostURL() {
+        bitField0_ = (bitField0_ & ~0x00002000);
+        hostURL_ = getDefaultInstance().getHostURL();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string hostURL = 15;</code>
+       */
+      public Builder setHostURLBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00002000;
+        hostURL_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional string appVersion = 16;
+      private java.lang.Object appVersion_ = "";
+      /**
+       * <code>optional string appVersion = 16;</code>
+       */
+      public boolean hasAppVersion() {
+        return ((bitField0_ & 0x00004000) == 0x00004000);
+      }
+      /**
+       * <code>optional string appVersion = 16;</code>
+       */
+      public java.lang.String getAppVersion() {
+        java.lang.Object ref = appVersion_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          appVersion_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string appVersion = 16;</code>
+       */
+      public com.google.protobuf.ByteString
+          getAppVersionBytes() {
+        java.lang.Object ref = appVersion_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          appVersion_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string appVersion = 16;</code>
+       */
+      public Builder setAppVersion(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00004000;
+        appVersion_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string appVersion = 16;</code>
+       */
+      public Builder clearAppVersion() {
+        bitField0_ = (bitField0_ & ~0x00004000);
+        appVersion_ = getDefaultInstance().getAppVersion();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string appVersion = 16;</code>
+       */
+      public Builder setAppVersionBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00004000;
+        appVersion_ = value;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:org.apache.slider.api.RoleInstanceState)
+    }
+
+    static {
+      defaultInstance = new RoleInstanceState(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:org.apache.slider.api.RoleInstanceState)
+  }
+
+  public interface StopClusterRequestProtoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required string message = 1;
+    /**
+     * <code>required string message = 1;</code>
+     *
+     * <pre>
+     **
+     *message to include
+     * </pre>
+     */
+    boolean hasMessage();
+    /**
+     * <code>required string message = 1;</code>
+     *
+     * <pre>
+     **
+     *message to include
+     * </pre>
+     */
+    java.lang.String getMessage();
+    /**
+     * <code>required string message = 1;</code>
+     *
+     * <pre>
+     **
+     *message to include
+     * </pre>
+     */
+    com.google.protobuf.ByteString
+        getMessageBytes();
+  }
+  /**
+   * Protobuf type {@code org.apache.slider.api.StopClusterRequestProto}
+   *
+   * <pre>
+   **
+   * stop the cluster
+   * </pre>
+   */
+  public static final class StopClusterRequestProto extends
+      com.google.protobuf.GeneratedMessage
+      implements StopClusterRequestProtoOrBuilder {
+    // Use StopClusterRequestProto.newBuilder() to construct.
+    private StopClusterRequestProto(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private StopClusterRequestProto(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final StopClusterRequestProto defaultInstance;
+    public static StopClusterRequestProto getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public StopClusterRequestProto getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private StopClusterRequestProto(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              message_ = input.readBytes();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.slider.api.proto.Messages.internal_static_org_apache_slider_api_StopClusterRequestProto_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.slider.api.proto.Messages.internal_static_org_apache_slider_api_StopClusterRequestProto_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.slider.api.proto.Messages.StopClusterRequestProto.class, org.apache.slider.api.proto.Messages.StopClusterRequestProto.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<StopClusterRequestProto> PARSER =
+        new com.google.protobuf.AbstractParser<StopClusterRequestProto>() {
+      public StopClusterRequestProto parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new StopClusterRequestProto(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<StopClusterRequestProto> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required string message = 1;
+    public static final int MESSAGE_FIELD_NUMBER = 1;
+    private java.lang.Object message_;
+    /**
+     * <code>required string message = 1;</code>
+     *
+     * <pre>
+     **
+     *message to include
+     * </pre>
+     */
+    public boolean hasMessage() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required string message = 1;</code>
+     *
+     * <pre>
+     **
+     *message to include
+     * </pre>
+     */
+    public java.lang.String getMessage() {
+      java.lang.Object ref = message_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          message_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string message = 1;</code>
+     *
+     * <pre>
+     **
+     *message to include
+     * </pre>
+     */
+    public com.google.protobuf.ByteString
+        getMessageBytes() {
+      java.lang.Object ref = message_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        message_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    private void initFields() {
+      message_ = "";
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasMessage()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getMessageBytes());
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getMessageBytes());
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.slider.api.proto.Messages.StopClusterRequestProto)) {
+        return super.equals(obj);
+      }
+      org.apache.slider.api.proto.Messages.StopClusterRequestProto other = (org.apache.slider.api.proto.Messages.StopClusterRequestProto) obj;
+
+      boolean result = true;
+      result = result && (hasMessage() == other.hasMessage());
+      if (hasMessage()) {
+        result = result && getMessage()
+            .equals(other.getMessage());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasMessage()) {
+        hash = (37 * hash) + MESSAGE_FIELD_NUMBER;
+        hash = (53 * hash) + getMessage().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.slider.api.proto.Messages.StopClusterRequestProto parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.slider.api.proto.Messages.StopClusterRequestProto parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.slider.api.proto.Messages.StopClusterRequestProto parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.slider.api.proto.Messages.StopClusterRequestProto parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.slider.api.proto.Messages.StopClusterRequestProto parseFrom(java.io.InputStream

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[65/76] [abbrv] hadoop git commit: YARN-5505. Create an agent-less docker provider in the native-services framework. Contributed by Billie Rinaldi

Posted by ji...@apache.org.
YARN-5505. Create an agent-less docker provider in the native-services framework. Contributed by Billie Rinaldi


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

Branch: refs/heads/yarn-native-services
Commit: cb61fe3fa6784cc72f10f90456987bbaa55c8914
Parents: 86a29d4
Author: Jian He <ji...@apache.org>
Authored: Thu Sep 1 22:38:42 2016 +0800
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/slider/api/OptionKeys.java  |   15 +-
 .../org/apache/slider/client/SliderClient.java  |   17 +-
 .../org/apache/slider/common/SliderKeys.java    |   22 +-
 .../apache/slider/common/tools/SliderUtils.java |    4 +
 .../slider/core/launch/AbstractLauncher.java    |   18 +-
 .../PublishedConfigurationOutputter.java        |    6 +-
 .../providers/AbstractClientProvider.java       |    4 +-
 .../providers/AbstractProviderService.java      |   22 +-
 .../slider/providers/ProviderService.java       |   12 +-
 .../apache/slider/providers/ProviderUtils.java  | 1391 ++++++++++++++----
 .../providers/agent/AgentClientProvider.java    |   36 +-
 .../slider/providers/agent/AgentKeys.java       |   12 +-
 .../providers/agent/AgentProviderService.java   |  705 ++-------
 .../providers/docker/DockerClientProvider.java  |   96 ++
 .../slider/providers/docker/DockerKeys.java     |   32 +
 .../providers/docker/DockerProviderFactory.java |   43 +
 .../providers/docker/DockerProviderService.java |  355 +++++
 .../slideram/SliderAMProviderService.java       |    4 -
 .../server/appmaster/SliderAppMaster.java       |   39 +-
 .../main/resources/org/apache/slider/slider.xml |    4 +
 .../slider/providers/docker/appConfig.json      |   42 +
 .../slider/providers/docker/resources.json      |   16 +
 .../slider/providers/docker/test.template       |   16 +
 23 files changed, 1971 insertions(+), 940 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb61fe3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/OptionKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/OptionKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/OptionKeys.java
index a035a99..434b1d9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/OptionKeys.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/OptionKeys.java
@@ -41,7 +41,20 @@ public interface OptionKeys extends InternalKeys {
    * Prefix for site.xml options: {@value}
    */
   String SITE_XML_PREFIX = "site.";
-
+  /**
+   * Prefix for config file options: {@value}
+   */
+  String CONF_FILE_PREFIX = "conf.";
+  /**
+   * Prefix for package options: {@value}
+   */
+  String PKG_FILE_PREFIX = "pkg.";
+  /**
+   * Prefix for export options: {@value}
+   */
+  String EXPORT_PREFIX = "export.";
+  String TYPE_SUFFIX = ".type";
+  String NAME_SUFFIX = ".name";
 
   /**
    * Zookeeper quorum host list: {@value}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb61fe3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
index 3129f6f..5096bb7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
@@ -151,7 +151,6 @@ import org.apache.slider.core.registry.YarnAppListClient;
 import org.apache.slider.core.registry.docstore.ConfigFormat;
 import org.apache.slider.core.registry.docstore.PublishedConfigSet;
 import org.apache.slider.core.registry.docstore.PublishedConfiguration;
-import org.apache.slider.core.registry.docstore.PublishedConfigurationOutputter;
 import org.apache.slider.core.registry.docstore.PublishedExports;
 import org.apache.slider.core.registry.docstore.PublishedExportsOutputter;
 import org.apache.slider.core.registry.docstore.PublishedExportsSet;
@@ -162,6 +161,7 @@ import org.apache.slider.core.zk.ZKPathBuilder;
 import org.apache.slider.providers.AbstractClientProvider;
 import org.apache.slider.providers.SliderProviderFactory;
 import org.apache.slider.providers.agent.AgentKeys;
+import org.apache.slider.providers.docker.DockerClientProvider;
 import org.apache.slider.providers.slideram.SliderAMClientProvider;
 import org.apache.slider.server.appmaster.SliderAppMaster;
 import org.apache.slider.server.appmaster.rpc.RpcBinder;
@@ -2081,7 +2081,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
 
     // add the tags if available
     Set<String> applicationTags = provider.getApplicationTags(sliderFileSystem,
-        getApplicationDefinitionPath(appOperations));
+        appOperations);
 
     Credentials credentials = null;
     if (clusterSecure) {
@@ -2242,11 +2242,12 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
                                                   );
 
 
-    // TODO: consider supporting apps that don't have an image path
-    Path imagePath =
-        extractImagePath(sliderFileSystem, internalOptions);
-    if (sliderFileSystem.maybeAddImagePath(localResources, imagePath)) {
-      log.debug("Registered image path {}", imagePath);
+    if (!(provider instanceof DockerClientProvider)) {
+      Path imagePath =
+          extractImagePath(sliderFileSystem, internalOptions);
+      if (sliderFileSystem.maybeAddImagePath(localResources, imagePath)) {
+        log.debug("Registered image path {}", imagePath);
+      }
     }
 
     // build the environment
@@ -3814,7 +3815,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
           Path subPath = new Path(path1, appReport.getApplicationId()
               .toString() + "/agent");
           imagePath = subPath.toString();
-          String pathStr = imagePath + "/" + AGENT_TAR;
+          String pathStr = imagePath + "/" + AgentKeys.AGENT_TAR;
           try {
             validateHDFSFile(sliderFileSystem, pathStr);
             log.info("Slider agent package is properly installed at " + pathStr);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb61fe3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
index 120b1fc..1484ee3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
@@ -81,6 +81,10 @@ public interface SliderKeys extends SliderXmlConfKeys {
   String COMPONENT_SEPARATOR = "-";
   String[] COMPONENT_KEYS_TO_SKIP = {"zookeeper.", "env.MALLOC_ARENA_MAX",
       "site.fs.", "site.dfs."};
+  /**
+   * A component type for a client component
+   */
+  String COMPONENT_TYPE_CLIENT = "client";
 
   /**
    * Key for application version. This must be set in app_config/global {@value}
@@ -222,7 +226,6 @@ public interface SliderKeys extends SliderXmlConfKeys {
   String SLIDER_JAR = "slider.jar";
   String JCOMMANDER_JAR = "jcommander.jar";
   String GSON_JAR = "gson.jar";
-  String AGENT_TAR = "slider-agent.tar.gz";
   String DEFAULT_APP_PKG = "appPkg.zip";
 
   String DEFAULT_JVM_HEAP = "256M";
@@ -288,4 +291,21 @@ public interface SliderKeys extends SliderXmlConfKeys {
 
   String SLIDER_CLASSPATH_EXTRA = "SLIDER_CLASSPATH_EXTRA";
   String YARN_CONTAINER_PATH = "/node/container/";
+
+  String GLOBAL_CONFIG_TAG = "global";
+  String SYSTEM_CONFIGS = "system_configs";
+  String JAVA_HOME = "java_home";
+  String TWO_WAY_SSL_ENABLED = "ssl.server.client.auth";
+  String INFRA_RUN_SECURITY_DIR = "infra/run/security/";
+  String CERT_FILE_LOCALIZATION_PATH = INFRA_RUN_SECURITY_DIR + "ca.crt";
+
+  String AM_CONFIG_GENERATION = "am.config.generation";
+  String APP_CONF_DIR = "app/conf";
+
+  String APP_RESOURCES = "application.resources";
+  String APP_RESOURCES_DIR = "app/resources";
+  String PER_COMPONENT = "per.component";
+  String PER_GROUP = "per.group";
+
+  String APP_PACKAGES_DIR = "app/packages";
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb61fe3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
index e9f65ba..f773982 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
@@ -183,6 +183,10 @@ public final class SliderUtils {
     return !isUnset(s);
   }
 
+  public static boolean isEmpty(List l) {
+    return l == null || l.isEmpty();
+  }
+
   /**
    * Probe for a list existing and not being empty
    * @param l list

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb61fe3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/AbstractLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/AbstractLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/AbstractLauncher.java
index 5a3eb3d..aefc0de 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/AbstractLauncher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/launch/AbstractLauncher.java
@@ -52,6 +52,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
+import static org.apache.slider.providers.docker.DockerKeys.DEFAULT_DOCKER_NETWORK;
+
 /**
  * Launcher of applications: base class
  */
@@ -79,6 +81,7 @@ public abstract class AbstractLauncher extends Configured {
   protected LogAggregationContext logAggregationContext;
   protected boolean yarnDockerMode = false;
   protected String dockerImage;
+  protected String dockerNetwork = DEFAULT_DOCKER_NETWORK;
   protected String yarnContainerMountPoints;
   protected String runPrivilegedContainer;
 
@@ -232,7 +235,8 @@ public abstract class AbstractLauncher extends Configured {
     if(yarnDockerMode){
       Map<String, String> env = containerLaunchContext.getEnvironment();
       env.put("YARN_CONTAINER_RUNTIME_TYPE", "docker");
-      env.put("YARN_CONTAINER_RUNTIME_DOCKER_IMAGE", dockerImage);//if yarnDockerMode, then dockerImage is set
+      env.put("YARN_CONTAINER_RUNTIME_DOCKER_IMAGE", dockerImage);
+      env.put("YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK", dockerNetwork);
       env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER", runPrivilegedContainer);
       StringBuilder sb = new StringBuilder();
       for (Entry<String,String> mount : mountPaths.entrySet()) {
@@ -517,6 +521,10 @@ public abstract class AbstractLauncher extends Configured {
     this.dockerImage = dockerImage;
   }
 
+  public void setDockerNetwork(String dockerNetwork) {
+    this.dockerNetwork = dockerNetwork;
+  }
+
   public void setYarnContainerMountPoints(String yarnContainerMountPoints) {
     this.yarnContainerMountPoints = yarnContainerMountPoints;
   }
@@ -525,4 +533,12 @@ public abstract class AbstractLauncher extends Configured {
     this.runPrivilegedContainer = runPrivilegedContainer;
   }
 
+  public void setRunPrivilegedContainer(boolean runPrivilegedContainer) {
+    if (runPrivilegedContainer) {
+      this.runPrivilegedContainer = Boolean.toString(true);
+    } else {
+      this.runPrivilegedContainer = Boolean.toString(false);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb61fe3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigurationOutputter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigurationOutputter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigurationOutputter.java
index 9bdcfcb..4ec513c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigurationOutputter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigurationOutputter.java
@@ -39,6 +39,8 @@ import java.util.Properties;
  */
 public abstract class PublishedConfigurationOutputter {
 
+  private static final String COMMENTS = "Generated by Apache Slider";
+
   protected final PublishedConfiguration owner;
 
   protected PublishedConfigurationOutputter(PublishedConfiguration owner) {
@@ -143,13 +145,13 @@ public abstract class PublishedConfigurationOutputter {
 
     @Override
     public void save(OutputStream out) throws IOException {
-      properties.store(out, "");
+      properties.store(out, COMMENTS);
     }
 
 
     public String asString() throws IOException {
       StringWriter sw = new StringWriter();
-      properties.store(sw, "");
+      properties.store(sw, COMMENTS);
       return sw.toString();
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb61fe3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractClientProvider.java
index 510de5d..f59c347 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractClientProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractClientProvider.java
@@ -216,8 +216,8 @@ public abstract class AbstractClientProvider extends Configured {
    * Return a set of application specific string tags.
    * @return the set of tags.
    */
-  public Set<String> getApplicationTags (SliderFileSystem fileSystem,
-                                         String appDef) throws SliderException {
+  public Set<String> getApplicationTags(SliderFileSystem fileSystem,
+                                        ConfTreeOperations appConf) throws SliderException {
     return Collections.emptySet();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb61fe3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
index 92766f5..19fa07b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.Service;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.registry.client.binding.RegistryTypeUtils;
@@ -139,6 +140,19 @@ public abstract class AbstractProviderService
   }
 
   /**
+   * Load default Configuration
+   * @param confDir configuration directory
+   * @return configuration
+   * @throws BadCommandArgumentsException
+   * @throws IOException
+   */
+  @Override
+  public Configuration loadProviderConfigurationInformation(File confDir)
+      throws BadCommandArgumentsException, IOException {
+    return new Configuration(false);
+  }
+
+  /**
    * Load a specific XML configuration file for the provider config
    * @param confDir configuration directory
    * @param siteXMLFilename provider-specific filename
@@ -369,8 +383,6 @@ public abstract class AbstractProviderService
 
   @Override
   public void applyInitialRegistryDefinitions(URL amWebURI,
-      URL agentOpsURI,
-      URL agentStatusURI,
       ServiceRecord serviceRecord)
     throws IOException {
       this.amWebAPI = amWebURI;
@@ -422,4 +434,10 @@ public abstract class AbstractProviderService
   public void rebuildContainerDetails(List<Container> liveContainers,
       String applicationId, Map<Integer, ProviderRole> providerRoles) {
   }
+
+  @Override
+  public boolean processContainerStatus(ContainerId containerId,
+      ContainerStatus status) {
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb61fe3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
index 3f24665..b62510a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.service.Service;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.slider.api.ClusterDescription;
 import org.apache.slider.common.tools.SliderFileSystem;
 import org.apache.slider.core.conf.AggregateConf;
@@ -189,13 +190,9 @@ public interface ProviderService extends ProviderCore,
   /**
    * Prior to going live -register the initial service registry data
    * @param amWebURI URL to the AM. This may be proxied, so use relative paths
-   * @param agentOpsURI URI for agent operations. This will not be proxied
-   * @param agentStatusURI URI For agent status. Again: no proxy
    * @param serviceRecord service record to build up
    */
   void applyInitialRegistryDefinitions(URL amWebURI,
-      URL agentOpsURI,
-      URL agentStatusURI,
       ServiceRecord serviceRecord)
       throws IOException;
 
@@ -216,4 +213,11 @@ public interface ProviderService extends ProviderCore,
    */
   void rebuildContainerDetails(List<Container> liveContainers,
       String applicationId, Map<Integer, ProviderRole> providerRoles);
+
+  /**
+   * Process container status
+   * @return true if status needs to be requested again, false otherwise
+   */
+  boolean processContainerStatus(ContainerId containerId,
+      ContainerStatus status);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb61fe3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java
index 07d106b..47556f0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java
@@ -18,16 +18,29 @@
 
 package org.apache.slider.providers;
 
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.hadoop.registry.client.types.yarn.PersistencePolicies;
+import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.slider.api.ClusterDescription;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.slider.api.ClusterNode;
 import org.apache.slider.api.InternalKeys;
 import org.apache.slider.api.OptionKeys;
 import org.apache.slider.api.ResourceKeys;
 import org.apache.slider.api.RoleKeys;
+import org.apache.slider.common.SliderExitCodes;
 import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.SliderXmlConfKeys;
 import org.apache.slider.common.tools.SliderFileSystem;
 import org.apache.slider.common.tools.SliderUtils;
 import org.apache.slider.core.conf.AggregateConf;
@@ -35,28 +48,50 @@ import org.apache.slider.core.conf.ConfTreeOperations;
 import org.apache.slider.core.conf.MapOperations;
 import org.apache.slider.core.exceptions.BadCommandArgumentsException;
 import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.core.exceptions.NoSuchNodeException;
 import org.apache.slider.core.exceptions.SliderException;
-import org.apache.slider.core.exceptions.SliderInternalStateException;
+import org.apache.slider.core.launch.ContainerLauncher;
+import org.apache.slider.core.registry.docstore.ConfigFormat;
+import org.apache.slider.core.registry.docstore.ConfigUtils;
+import org.apache.slider.core.registry.docstore.ExportEntry;
+import org.apache.slider.core.registry.docstore.PublishedConfiguration;
+import org.apache.slider.core.registry.docstore.PublishedConfigurationOutputter;
+import org.apache.slider.core.registry.docstore.PublishedExports;
+import org.apache.slider.server.appmaster.state.RoleInstance;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+import org.apache.slider.server.services.security.CertificateManager;
+import org.apache.slider.server.services.security.SecurityStore;
+import org.apache.slider.server.services.security.StoresGenerator;
+import org.apache.slider.server.services.yarnregistry.YarnRegistryViewForProviders;
 import org.slf4j.Logger;
 
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.util.LinkedList;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeMap;
 import java.util.regex.Pattern;
 
 /**
- * this is a factoring out of methods handy for providers. It's bonded to a log at
- * construction time
+ * This is a factoring out of methods handy for providers. It's bonded to a log
+ * at construction time.
  */
-public class ProviderUtils implements RoleKeys {
+public class ProviderUtils implements RoleKeys, SliderKeys {
 
   protected final Logger log;
 
   /**
-   * Create an instace
+   * Create an instance
    * @param log log directory to use -usually the provider
    */
   
@@ -66,14 +101,14 @@ public class ProviderUtils implements RoleKeys {
 
   /**
    * Add oneself to the classpath. This does not work
-   * on minicluster test runs where the JAR is not built up
+   * on minicluster test runs where the JAR is not built up.
    * @param providerResources map of provider resources to add these entries to
    * @param provider provider to add
    * @param jarName name of the jar to use
    * @param sliderFileSystem target filesystem
    * @param tempPath path in the cluster FS for temp files
    * @param libdir relative directory to place resources
-   * @param miniClusterTestRun
+   * @param miniClusterTestRun true if minicluster is being used
    * @return true if the class was found in a JAR
    * 
    * @throws FileNotFoundException if the JAR was not found and this is NOT
@@ -81,7 +116,8 @@ public class ProviderUtils implements RoleKeys {
    * @throws IOException IO problems
    * @throws SliderException any Slider problem
    */
-  public static boolean addProviderJar(Map<String, LocalResource> providerResources,
+  public static boolean addProviderJar(
+      Map<String, LocalResource> providerResources,
       Object provider,
       String jarName,
       SliderFileSystem sliderFileSystem,
@@ -108,13 +144,14 @@ public class ProviderUtils implements RoleKeys {
   }
 
   /**
-   * Add/overwrite the agent tarball (overwritten every time application is restarted)
-   * @param provider
-   * @param tarName
-   * @param sliderFileSystem
-   * @param agentDir
+   * Add/overwrite the agent tarball (overwritten every time application is
+   * restarted).
+   * @param provider an instance of a provider class
+   * @param tarName name of the tarball to upload
+   * @param sliderFileSystem the file system
+   * @param agentDir directory to upload to
    * @return true the location could be determined and the file added
-   * @throws IOException
+   * @throws IOException if the upload fails
    */
   public static boolean addAgentTar(Object provider,
                                     String tarName,
@@ -125,100 +162,58 @@ public class ProviderUtils implements RoleKeys {
     if(localFile != null) {
       String parentDir = localFile.getParent();
       Path agentTarPath = new Path(parentDir, tarName);
-      sliderFileSystem.getFileSystem().copyFromLocalFile(false, true, agentTarPath, agentDir);
+      sliderFileSystem.getFileSystem().copyFromLocalFile(false, true,
+          agentTarPath, agentDir);
       return true;
     }
     return false;
   }
   
   /**
-   * Add a set of dependencies to the provider resources being built up,
-   * by copying them from the local classpath to the remote one, then
-   * registering them
-   * @param providerResources map of provider resources to add these entries to
-   * @param sliderFileSystem target filesystem
-   * @param tempPath path in the cluster FS for temp files
-   * @param libdir relative directory to place resources
-   * @param resources list of resource names (e.g. "hbase.jar"
-   * @param classes list of classes where classes[i] refers to a class in
-   * resources[i]
-   * @throws IOException IO problems
-   * @throws SliderException any Slider problem
-   */
-  public static void addDependencyJars(Map<String, LocalResource> providerResources,
-                                       SliderFileSystem sliderFileSystem,
-                                       Path tempPath,
-                                       String libdir,
-                                       String[] resources,
-                                       Class[] classes
-                                      ) throws
-                                        IOException,
-      SliderException {
-    if (resources.length != classes.length) {
-      throw new SliderInternalStateException(
-        "mismatch in Jar names [%d] and classes [%d]",
-        resources.length,
-        classes.length);
-    }
-    int size = resources.length;
-    for (int i = 0; i < size; i++) {
-      String jarName = resources[i];
-      Class clazz = classes[i];
-      SliderUtils.putJar(providerResources,
-          sliderFileSystem,
-          clazz,
-          tempPath,
-          libdir,
-          jarName);
-    }
-    
-  }
-
-  /**
-   * Loads all dependency jars from the default path
+   * Loads all dependency jars from the default path.
    * @param providerResources map of provider resources to add these entries to
    * @param sliderFileSystem target filesystem
    * @param tempPath path in the cluster FS for temp files
    * @param libDir relative directory to place resources
    * @param libLocalSrcDir explicitly supplied local libs dir
-   * @throws IOException
-   * @throws SliderException
-   */
-  public static void addAllDependencyJars(Map<String, LocalResource> providerResources,
-                                          SliderFileSystem sliderFileSystem,
-                                          Path tempPath,
-                                          String libDir,
-                                          String libLocalSrcDir)
+   * @throws IOException trouble copying to HDFS
+   * @throws SliderException trouble copying to HDFS
+   */
+  public static void addAllDependencyJars(
+      Map<String, LocalResource> providerResources,
+      SliderFileSystem sliderFileSystem,
+      Path tempPath,
+      String libDir,
+      String libLocalSrcDir)
       throws IOException, SliderException {
-    String libSrcToUse = libLocalSrcDir;
     if (SliderUtils.isSet(libLocalSrcDir)) {
       File file = new File(libLocalSrcDir);
       if (!file.exists() || !file.isDirectory()) {
-        throw new BadCommandArgumentsException("Supplied lib src dir %s is not valid", libLocalSrcDir);
+        throw new BadCommandArgumentsException(
+            "Supplied lib src dir %s is not valid", libLocalSrcDir);
       }
     }
-    SliderUtils.putAllJars(providerResources, sliderFileSystem, tempPath, libDir, libSrcToUse);
+    SliderUtils.putAllJars(providerResources, sliderFileSystem, tempPath,
+        libDir, libLocalSrcDir);
   }
 
+
   /**
-   * build the log directory
-   * @return the log dir
+   * Validate the requested number of instances of a component.
+   * <p>
+   * If max &lt;= 0:  min &lt;= count
+   * If max &gt; 0:  min &lt;= count &lt;= max
+   * @param instanceDescription configuration
+   * @param name node class name
+   * @param min requested heap size
+   * @param max maximum value.
+   * @throws BadCommandArgumentsException if the values are out of range
    */
-  public String getLogdir() throws IOException {
-    String logdir = System.getenv("LOGDIR");
-    if (logdir == null) {
-      logdir =
-        SliderKeys.TMP_LOGDIR_PREFIX + UserGroupInformation.getCurrentUser().getShortUserName();
-    }
-    return logdir;
-  }
-
-
   public void validateNodeCount(AggregateConf instanceDescription,
-                                String name, int min, int max) throws
-                                                               BadCommandArgumentsException {
+                                String name, int min, int max)
+      throws BadCommandArgumentsException {
     MapOperations component =
-      instanceDescription.getResourceOperations().getComponent(name);
+        instanceDescription.getResourceOperations().getComponent(name);
     int count;
     if (component == null) {
       count = 0;
@@ -229,7 +224,7 @@ public class ProviderUtils implements RoleKeys {
   }
   
   /**
-   * Validate the node count and heap size values of a node class 
+   * Validate the count is between min and max.
    * <p>
    * If max &lt;= 0:  min &lt;= count
    * If max &gt; 0:  min &lt;= count &lt;= max
@@ -256,33 +251,36 @@ public class ProviderUtils implements RoleKeys {
   }
 
   /**
-   * copy all options beginning site. into the site.xml
-   * @param clusterSpec cluster specification
-   * @param sitexml map for XML file to build up
+   * Copy options beginning with "site.configName." prefix from options map
+   * to sitexml map, removing the prefix and substituting the tokens
+   * specified in the tokenMap.
+   * @param options source map
+   * @param sitexml destination map
+   * @param configName optional ".configName" portion of the prefix
+   * @param tokenMap key/value pairs to substitute into the option values
    */
-  public void propagateSiteOptions(ClusterDescription clusterSpec,
-                                    Map<String, String> sitexml) {
-    Map<String, String> options = clusterSpec.options;
-    propagateSiteOptions(options, sitexml);
-  }
-
-  public void propagateSiteOptions(Map<String, String> options,
-                                   Map<String, String> sitexml) {
-    propagateSiteOptions(options, sitexml, "");
-  }
-
   public void propagateSiteOptions(Map<String, String> options,
-                                   Map<String, String> sitexml,
-                                   String configName) {
-    propagateSiteOptions(options, sitexml, configName, null);
+      Map<String, String> sitexml,
+      String configName,
+      Map<String,String> tokenMap) {
+    String prefix = OptionKeys.SITE_XML_PREFIX +
+        (!configName.isEmpty() ? configName + "." : "");
+    propagateOptions(options, sitexml, tokenMap, prefix);
   }
 
-  public void propagateSiteOptions(Map<String, String> options,
+  /**
+   * Copy options beginning with prefix from options map
+   * to sitexml map, removing the prefix and substituting the tokens
+   * specified in the tokenMap.
+   * @param options source map
+   * @param sitexml destination map
+   * @param tokenMap key/value pairs to substitute into the option values
+   * @param prefix which options to copy to destination map
+   */
+  public void propagateOptions(Map<String, String> options,
                                    Map<String, String> sitexml,
-                                   String configName,
-                                   Map<String,String> tokenMap) {
-    String prefix = OptionKeys.SITE_XML_PREFIX +
-                    (!configName.isEmpty() ? configName + "." : "");
+                                   Map<String,String> tokenMap,
+                                   String prefix) {
     for (Map.Entry<String, String> entry : options.entrySet()) {
       String key = entry.getKey();
       if (key.startsWith(prefix)) {
@@ -302,229 +300,1038 @@ public class ProviderUtils implements RoleKeys {
   }
 
   /**
-   * Propagate an option from the cluster specification option map
-   * to the site XML map, using the site key for the name
-   * @param global global config spec
-   * @param optionKey key in the option map
-   * @param sitexml  map for XML file to build up
-   * @param siteKey key to assign the value to in the site XML
-   * @throws BadConfigException if the option is missing from the cluster spec
-   */
-  public void propagateOption(MapOperations global,
-                              String optionKey,
-                              Map<String, String> sitexml,
-                              String siteKey) throws BadConfigException {
-    sitexml.put(siteKey, global.getMandatoryOption(optionKey));
-  }
-
-
-  /**
-   * Build the image dir. This path is relative and only valid at the far end
-   * @param instanceDefinition instance definition
-   * @param bindir bin subdir
-   * @param script script in bin subdir
-   * @return the path to the script
-   * @throws FileNotFoundException if a file is not found, or it is not a directory* 
-   */
-  public String buildPathToHomeDir(AggregateConf instanceDefinition,
-                                  String bindir,
-                                  String script) throws
-                                                 FileNotFoundException,
-                                                 BadConfigException {
-    MapOperations globalOptions =
-      instanceDefinition.getInternalOperations().getGlobalOptions();
-    String applicationHome =
-      globalOptions.get(InternalKeys.INTERNAL_APPLICATION_HOME);
-    String imagePath =
-      globalOptions.get(InternalKeys.INTERNAL_APPLICATION_IMAGE_PATH);
-    return buildPathToHomeDir(imagePath, applicationHome, bindir, script);
-  }
-
-  public String buildPathToHomeDir(String imagePath,
-                                   String applicationHome,
-                                   String bindir, String script) throws
-                                                                 FileNotFoundException {
-    String path;
-    File scriptFile;
-    if (imagePath != null) {
-      File tarball = new File(SliderKeys.LOCAL_TARBALL_INSTALL_SUBDIR);
-      scriptFile = findBinScriptInExpandedArchive(tarball, bindir, script);
-      // now work back from the script to build the relative path
-      // to the binary which will be valid remote or local
-      StringBuilder builder = new StringBuilder();
-      builder.append(SliderKeys.LOCAL_TARBALL_INSTALL_SUBDIR);
-      builder.append("/");
-      //for the script, we want the name of ../..
-      File archive = scriptFile.getParentFile().getParentFile();
-      builder.append(archive.getName());
-      path = builder.toString();
+   * Substitute tokens into option map values, returning a new map.
+   * @param options source map
+   * @param tokenMap key/value pairs to substitute into the option values
+   * @return map with substituted values
+   */
+  public Map<String, String> filterSiteOptions(Map<String, String> options,
+      Map<String, String> tokenMap) {
+    String prefix = OptionKeys.SITE_XML_PREFIX;
+    Map<String, String> filteredOptions = new HashMap<>();
+    for (Map.Entry<String, String> entry : options.entrySet()) {
+      String key = entry.getKey();
+      if (key.startsWith(prefix)) {
+        String value = entry.getValue();
+        if (tokenMap != null) {
+          for (Map.Entry<String,String> token : tokenMap.entrySet()) {
+            value = value.replaceAll(Pattern.quote(token.getKey()),
+                token.getValue());
+          }
+        }
+        filteredOptions.put(key, value);
+      }
+    }
+    return filteredOptions;
+  }
+
+  /**
+   * Get resource requirements from a String value. If value isn't specified,
+   * use the default value. If value is greater than max, use the max value.
+   * @param val string value
+   * @param defVal default value
+   * @param maxVal maximum value
+   * @return int resource requirement
+   */
+  public int getRoleResourceRequirement(String val,
+                                        int defVal,
+                                        int maxVal) {
+    if (val==null) {
+      val = Integer.toString(defVal);
+    }
+    Integer intVal;
+    if (ResourceKeys.YARN_RESOURCE_MAX.equals(val)) {
+      intVal = maxVal;
+    } else {
+      intVal = Integer.decode(val);
+    }
+    return intVal;
+  }
+
+  /**
+   * Localize the service keytabs for the application.
+   * @param launcher container launcher
+   * @param instanceDefinition app specification
+   * @param fileSystem file system
+   * @param clusterName app name
+   * @throws IOException trouble uploading to HDFS
+   */
+  public void localizeServiceKeytabs(ContainerLauncher launcher,
+      AggregateConf instanceDefinition, SliderFileSystem fileSystem,
+      String clusterName) throws IOException {
+    ConfTreeOperations appConf = instanceDefinition.getAppConfOperations();
+    String keytabPathOnHost = appConf.getComponent(COMPONENT_AM).get(
+            SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH);
+    if (SliderUtils.isUnset(keytabPathOnHost)) {
+      String amKeytabName = appConf.getComponent(COMPONENT_AM).get(
+              SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME);
+      String keytabDir = appConf.getComponent(COMPONENT_AM).get(
+              SliderXmlConfKeys.KEY_HDFS_KEYTAB_DIR);
+      // we need to localize the keytab files in the directory
+      Path keytabDirPath = fileSystem.buildKeytabPath(keytabDir, null,
+          clusterName);
+      boolean serviceKeytabsDeployed = false;
+      if (fileSystem.getFileSystem().exists(keytabDirPath)) {
+        FileStatus[] keytabs = fileSystem.getFileSystem().listStatus(
+            keytabDirPath);
+        LocalResource keytabRes;
+        for (FileStatus keytab : keytabs) {
+          if (!amKeytabName.equals(keytab.getPath().getName())
+              && keytab.getPath().getName().endsWith(".keytab")) {
+            serviceKeytabsDeployed = true;
+            log.info("Localizing keytab {}", keytab.getPath().getName());
+            keytabRes = fileSystem.createAmResource(keytab.getPath(),
+                LocalResourceType.FILE);
+            launcher.addLocalResource(KEYTAB_DIR + "/" +
+                    keytab.getPath().getName(),
+                keytabRes);
+          }
+        }
+      }
+      if (!serviceKeytabsDeployed) {
+        log.warn("No service keytabs for the application have been localized.  "
+            + "If the application requires keytabs for secure operation, "
+            + "please ensure that the required keytabs have been uploaded "
+            + "to the folder {}", keytabDirPath);
+      }
+    }
+  }
+
+  /**
+   * Return whether two-way SSL is enabled for Agent / AM communication.
+   * @param amComponent component specification
+   * @return true if enabled
+   */
+  public boolean hasTwoWaySSLEnabled(MapOperations amComponent) {
+    return amComponent != null ?
+        amComponent.getOptionBool(TWO_WAY_SSL_ENABLED, false) : false;
+  }
+
+  /**
+   * Generate and localize SSL certs for Agent / AM communication
+   * @param launcher container launcher
+   * @param container allocated container information
+   * @param fileSystem file system
+   * @param clusterName app name
+   * @throws SliderException certs cannot be generated/uploaded
+   */
+  public void localizeContainerSSLResources(ContainerLauncher launcher,
+      Container container, SliderFileSystem fileSystem, String clusterName)
+      throws SliderException {
+    try {
+      // localize server cert
+      Path certsDir = fileSystem.buildClusterSecurityDirPath(clusterName);
+      LocalResource certResource = fileSystem.createAmResource(
+          new Path(certsDir, CRT_FILE_NAME),
+          LocalResourceType.FILE);
+      launcher.addLocalResource(CERT_FILE_LOCALIZATION_PATH, certResource);
+
+      // generate and localize agent cert
+      CertificateManager certMgr = new CertificateManager();
+      String hostname = container.getNodeId().getHost();
+      String containerId = container.getId().toString();
+      certMgr.generateContainerCertificate(hostname, containerId);
+      LocalResource agentCertResource = fileSystem.createAmResource(
+          uploadSecurityResource(
+              CertificateManager.getAgentCertficateFilePath(containerId),
+              fileSystem, clusterName), LocalResourceType.FILE);
+      // still using hostname as file name on the agent side, but the files
+      // do end up under the specific container's file space
+      launcher.addLocalResource(INFRA_RUN_SECURITY_DIR + hostname +
+          ".crt", agentCertResource);
+      LocalResource agentKeyResource = fileSystem.createAmResource(
+          uploadSecurityResource(
+              CertificateManager.getAgentKeyFilePath(containerId), fileSystem,
+              clusterName),
+          LocalResourceType.FILE);
+      launcher.addLocalResource(INFRA_RUN_SECURITY_DIR + hostname +
+          ".key", agentKeyResource);
+
+    } catch (Exception e) {
+      throw new SliderException(SliderExitCodes.EXIT_DEPLOYMENT_FAILED, e,
+          "Unable to localize certificates.  Two-way SSL cannot be enabled");
+    }
+  }
+
+  /**
+   * Upload a local file to the cluster security dir in HDFS. If the file
+   * already exists, it is not replaced.
+   * @param resource file to upload
+   * @param fileSystem file system
+   * @param clusterName app name
+   * @return Path of the uploaded file
+   * @throws IOException file cannot be uploaded
+   */
+  private Path uploadSecurityResource(File resource,
+      SliderFileSystem fileSystem, String clusterName) throws IOException {
+    Path certsDir = fileSystem.buildClusterSecurityDirPath(clusterName);
+    return uploadResource(resource, fileSystem, certsDir);
+  }
+
+  /**
+   * Upload a local file to the cluster resources dir in HDFS. If the file
+   * already exists, it is not replaced.
+   * @param resource file to upload
+   * @param fileSystem file system
+   * @param roleName optional subdirectory (for component-specific resources)
+   * @param clusterName app name
+   * @return Path of the uploaded file
+   * @throws IOException file cannot be uploaded
+   */
+  private Path uploadResource(File resource, SliderFileSystem fileSystem,
+      String roleName, String clusterName) throws IOException {
+    Path dir;
+    if (roleName == null) {
+      dir = fileSystem.buildClusterResourcePath(clusterName);
+    } else {
+      dir = fileSystem.buildClusterResourcePath(clusterName, roleName);
+    }
+    return uploadResource(resource, fileSystem, dir);
+  }
 
+  /**
+   * Upload a local file to a specified HDFS directory. If the file already
+   * exists, it is not replaced.
+   * @param resource file to upload
+   * @param fileSystem file system
+   * @param parentDir destination directory in HDFS
+   * @return Path of the uploaded file
+   * @throws IOException file cannot be uploaded
+   */
+  private synchronized Path uploadResource(File resource,
+      SliderFileSystem fileSystem, Path parentDir) throws IOException {
+    if (!fileSystem.getFileSystem().exists(parentDir)) {
+      fileSystem.getFileSystem().mkdirs(parentDir,
+          new FsPermission(FsAction.ALL, FsAction.NONE, FsAction.NONE));
+    }
+    Path destPath = new Path(parentDir, resource.getName());
+    if (!fileSystem.getFileSystem().exists(destPath)) {
+      FSDataOutputStream os = null;
+      try {
+        os = fileSystem.getFileSystem().create(destPath);
+        byte[] contents = FileUtils.readFileToByteArray(resource);
+        os.write(contents, 0, contents.length);
+        os.flush();
+      } finally {
+        IOUtils.closeStream(os);
+      }
+      log.info("Uploaded {} to localization path {}", resource, destPath);
     } else {
-      // using a home directory which is required to be present on 
-      // the local system -so will be absolute and resolvable
-      File homedir = new File(applicationHome);
-      path = homedir.getAbsolutePath();
+      log.info("Resource {} already existed at localization path {}", resource,
+          destPath);
+    }
 
-      //this is absolute, resolve its entire path
-      SliderUtils.verifyIsDir(homedir, log);
-      File bin = new File(homedir, bindir);
-      SliderUtils.verifyIsDir(bin, log);
-      scriptFile = new File(bin, script);
-      SliderUtils.verifyFileExists(scriptFile, log);
+    while (!fileSystem.getFileSystem().exists(destPath)) {
+      try {
+        Thread.sleep(500);
+      } catch (InterruptedException e) {
+        // ignore
+      }
     }
-    return path;
+
+    fileSystem.getFileSystem().setPermission(destPath,
+        new FsPermission(FsAction.READ, FsAction.NONE, FsAction.NONE));
+
+    return destPath;
   }
 
-  
   /**
-   * Build the image dir. This path is relative and only valid at the far end
-   * @param instance instance options
-   * @param bindir bin subdir
-   * @param script script in bin subdir
-   * @return the path to the script
-   * @throws FileNotFoundException if a file is not found, or it is not a directory* 
-   */
-  public String buildPathToScript(AggregateConf instance,
-                                String bindir,
-                                String script) throws FileNotFoundException {
-    return buildPathToScript(instance.getInternalOperations(), bindir, script);
-  }
-  /**
-   * Build the image dir. This path is relative and only valid at the far end
-   * @param internal internal options
-   * @param bindir bin subdir
-   * @param script script in bin subdir
-   * @return the path to the script
-   * @throws FileNotFoundException if a file is not found, or it is not a directory* 
-   */
-  public String buildPathToScript(ConfTreeOperations internal,
-                                String bindir,
-                                String script) throws FileNotFoundException {
-    
-    String homedir = buildPathToHomeDir(
-      internal.get(InternalKeys.INTERNAL_APPLICATION_IMAGE_PATH),
-      internal.get(InternalKeys.INTERNAL_APPLICATION_HOME),
-      bindir,
-      script);
-    return buildScriptPath(bindir, script, homedir);
+   * Write a configuration property map to a local file in a specified format.
+   * @param fileSystem file system
+   * @param file destination file
+   * @param configFormat file format
+   * @param configFileDN file description
+   * @param config properties to save to the file
+   * @param clusterName app name
+   * @throws IOException file cannot be created
+   */
+  private void createConfigFile(SliderFileSystem fileSystem, File file,
+      ConfigFormat configFormat, String configFileDN,
+      Map<String, String> config, String clusterName) throws IOException {
+    log.info("Writing {} file {}", configFormat, file);
+
+    ConfigUtils.prepConfigForTemplateOutputter(configFormat, config,
+        fileSystem, clusterName, file.getName());
+    PublishedConfiguration publishedConfiguration =
+        new PublishedConfiguration(configFileDN,
+            config.entrySet());
+    PublishedConfigurationOutputter configurationOutputter =
+        PublishedConfigurationOutputter.createOutputter(configFormat,
+            publishedConfiguration);
+    configurationOutputter.save(file);
   }
-  
-  
 
-  public String buildScriptPath(String bindir, String script, String homedir) {
-    StringBuilder builder = new StringBuilder(homedir);
-    builder.append("/");
-    builder.append(bindir);
-    builder.append("/");
-    builder.append(script);
-    return builder.toString();
+  /**
+   * Determine config files requested in the appConf, generate the files, and
+   * localize them.
+   * @param launcher container launcher
+   * @param roleName component name
+   * @param roleGroup component group
+   * @param appConf app configurations
+   * @param configs configurations grouped by config name
+   * @param env environment variables
+   * @param fileSystem file system
+   * @param clusterName app name
+   * @throws IOException file(s) cannot be uploaded
+   * @throws BadConfigException file name not specified or file format not
+   * supported
+   */
+  public void localizeConfigFiles(ContainerLauncher launcher,
+      String roleName, String roleGroup,
+      ConfTreeOperations appConf,
+      Map<String, Map<String, String>> configs,
+      MapOperations env,
+      SliderFileSystem fileSystem,
+      String clusterName)
+      throws IOException, BadConfigException {
+    for (Entry<String, Map<String, String>> configEntry : configs.entrySet()) {
+      String configFileName = appConf.getComponentOpt(roleGroup,
+          OptionKeys.CONF_FILE_PREFIX + configEntry.getKey() + OptionKeys
+              .NAME_SUFFIX, null);
+      String configFileType = appConf.getComponentOpt(roleGroup,
+          OptionKeys.CONF_FILE_PREFIX + configEntry.getKey() + OptionKeys
+              .TYPE_SUFFIX, null);
+      if (configFileName == null && configFileType == null) {
+        // config file not requested, so continue
+        continue;
+      }
+      if (configFileName == null) {
+        throw new BadConfigException("Config file name null for " +
+            configEntry.getKey());
+      }
+      if (configFileType == null) {
+        throw new BadConfigException("Config file type null for " +
+            configEntry.getKey());
+      }
+      ConfigFormat configFormat = ConfigFormat.resolve(configFileType);
+      if (configFormat == null) {
+        throw new BadConfigException("Config format " + configFormat +
+            " doesn't exist");
+      }
+      localizeConfigFile(launcher, roleName, roleGroup, configEntry.getKey(),
+          configFormat, configFileName, configs, env, fileSystem, clusterName);
+    }
   }
 
+  /**
+   * Create and localize a config file.
+   * @param launcher container launcher
+   * @param roleName component name
+   * @param roleGroup component group
+   * @param configFileDN config description/name
+   * @param configFormat config format
+   * @param configFileName config file name
+   * @param configs configs grouped by config description/name
+   * @param env environment variables
+   * @param fileSystem file system
+   * @param clusterName app name
+   * @throws IOException file cannot be uploaded
+   */
+  public void localizeConfigFile(ContainerLauncher launcher,
+      String roleName, String roleGroup,
+      String configFileDN, ConfigFormat configFormat, String configFileName,
+      Map<String, Map<String, String>> configs,
+      MapOperations env,
+      SliderFileSystem fileSystem,
+      String clusterName)
+      throws IOException {
+    if (launcher == null) {
+      return;
+    }
+    Map<String, String> config = ConfigUtils.replacePropsInConfig(
+        configs.get(configFileDN), env.options);
+    String fileName = ConfigUtils.replaceProps(config, configFileName);
+    File localFile = new File(RESOURCE_DIR);
+    if (!localFile.exists()) {
+      if (!localFile.mkdir()) {
+        throw new IOException(RESOURCE_DIR + " could not be created!");
+      }
+    }
+    localFile = new File(localFile, new File(fileName).getName());
 
-  public static String convertToAppRelativePath(File file) {
-    return convertToAppRelativePath(file.getPath());
+    String folder = null;
+    if ("true".equals(config.get(PER_COMPONENT))) {
+      folder = roleName;
+    } else if ("true".equals(config.get(PER_GROUP))) {
+      folder = roleGroup;
+    }
+
+    log.info("Localizing {} configs to config file {} (destination {}) " +
+            "based on {} configs", config.size(), localFile, fileName,
+        configFileDN);
+    createConfigFile(fileSystem, localFile, configFormat, configFileDN, config,
+        clusterName);
+    Path destPath = uploadResource(localFile, fileSystem, folder, clusterName);
+    LocalResource configResource = fileSystem.createAmResource(destPath,
+        LocalResourceType.FILE);
+
+    File destFile = new File(fileName);
+    if (destFile.isAbsolute()) {
+      launcher.addLocalResource(
+          RESOURCE_DIR + "/" + destFile.getName(),
+          configResource, fileName);
+    } else {
+      launcher.addLocalResource(APP_CONF_DIR + "/" + fileName,
+          configResource);
+    }
   }
 
-  public static String convertToAppRelativePath(String path) {
-    return ApplicationConstants.Environment.PWD.$() + "/" + path;
+  /**
+   * Generate and localize security stores requested by the app. Also perform
+   * last-minute substitution of cluster name into credentials strings.
+   * @param launcher container launcher
+   * @param container allocated container information
+   * @param role component name
+   * @param fileSystem file system
+   * @param instanceDefinition app specification
+   * @param compOps component specification
+   * @param clusterName app name
+   * @throws SliderException stores cannot be generated/uploaded
+   * @throws IOException stores cannot be generated/uploaded
+   */
+  public void localizeContainerSecurityStores(ContainerLauncher launcher,
+      Container container,
+      String role,
+      SliderFileSystem fileSystem,
+      AggregateConf instanceDefinition,
+      MapOperations compOps,
+      String clusterName)
+      throws SliderException, IOException {
+    // substitute CLUSTER_NAME into credentials
+    Map<String,List<String>> newcred = new HashMap<>();
+    for (Entry<String,List<String>> entry :
+        instanceDefinition.getAppConf().credentials.entrySet()) {
+      List<String> resultList = new ArrayList<>();
+      for (String v : entry.getValue()) {
+        resultList.add(v.replaceAll(Pattern.quote("${CLUSTER_NAME}"),
+            clusterName).replaceAll(Pattern.quote("${CLUSTER}"),
+            clusterName));
+      }
+      newcred.put(entry.getKey().replaceAll(Pattern.quote("${CLUSTER_NAME}"),
+          clusterName).replaceAll(Pattern.quote("${CLUSTER}"),
+          clusterName),
+          resultList);
+    }
+    instanceDefinition.getAppConf().credentials = newcred;
+
+    // generate and localize security stores
+    SecurityStore[] stores = generateSecurityStores(container, role,
+        instanceDefinition, compOps);
+    for (SecurityStore store : stores) {
+      LocalResource keystoreResource = fileSystem.createAmResource(
+          uploadSecurityResource(store.getFile(), fileSystem, clusterName),
+          LocalResourceType.FILE);
+      launcher.addLocalResource(String.format("secstores/%s-%s.p12",
+          store.getType(), role),
+          keystoreResource);
+    }
+  }
+
+  /**
+   * Generate security stores requested by the app.
+   * @param container allocated container information
+   * @param role component name
+   * @param instanceDefinition app specification
+   * @param compOps component specification
+   * @return security stores
+   * @throws SliderException stores cannot be generated
+   * @throws IOException stores cannot be generated
+   */
+  private SecurityStore[] generateSecurityStores(Container container,
+      String role,
+      AggregateConf instanceDefinition,
+      MapOperations compOps)
+      throws SliderException, IOException {
+    return StoresGenerator.generateSecurityStores(
+        container.getNodeId().getHost(), container.getId().toString(),
+        role, instanceDefinition, compOps);
   }
 
+  /**
+   * Return whether security stores are requested by the app.
+   * @param compOps component specification
+   * @return true if stores are requested
+   */
+  public boolean areStoresRequested(MapOperations compOps) {
+    return compOps != null ? compOps.
+        getOptionBool(COMP_STORES_REQUIRED_KEY, false) : false;
+  }
 
-  public static void validatePathReferencesLocalDir(String meaning, String path)
-      throws BadConfigException {
-    File file = new File(path);
-    if (!file.exists()) {
-      throw new BadConfigException("%s directory %s not found", meaning, file);
+  /**
+   * Localize application tarballs and other resources requested by the app.
+   * @param launcher container launcher
+   * @param fileSystem file system
+   * @param appConf app configurations
+   * @param roleGroup component group
+   * @param clusterName app name
+   * @throws IOException resources cannot be uploaded
+   * @throws BadConfigException package name or type is not specified
+   */
+  public void localizePackages(ContainerLauncher launcher,
+      SliderFileSystem fileSystem, ConfTreeOperations appConf, String roleGroup,
+      String clusterName) throws IOException, BadConfigException {
+    for (Entry<String, Map<String, String>> pkg :
+        getPackages(roleGroup, appConf).entrySet()) {
+      String pkgName = pkg.getValue().get(OptionKeys.NAME_SUFFIX);
+      String pkgType = pkg.getValue().get(OptionKeys.TYPE_SUFFIX);
+      Path pkgPath = fileSystem.buildResourcePath(pkgName);
+      if (!fileSystem.isFile(pkgPath)) {
+        pkgPath = fileSystem.buildResourcePath(clusterName,
+            pkgName);
+      }
+      if (!fileSystem.isFile(pkgPath)) {
+        throw new IOException("Package doesn't exist as a resource: " +
+            pkgName);
+      }
+      log.info("Adding resource {}", pkgName);
+      LocalResourceType type = LocalResourceType.FILE;
+      if ("archive".equals(pkgType)) {
+        type = LocalResourceType.ARCHIVE;
+      }
+      LocalResource packageResource = fileSystem.createAmResource(
+          pkgPath, type);
+      launcher.addLocalResource(APP_PACKAGES_DIR, packageResource);
     }
-    if (!file.isDirectory()) {
-      throw new BadConfigException("%s is not a directory: %s", meaning, file);
+  }
+
+  /**
+   * Build a map of configuration description/name to configuration key/value
+   * properties, with all known tokens substituted into the property values.
+   * @param appConf app configurations
+   * @param internalsConf internal configurations
+   * @param containerId container ID
+   * @param roleName component name
+   * @param roleGroup component group
+   * @param amState access to AM state
+   * @return configuration properties grouped by config description/name
+   */
+  public Map<String, Map<String, String>> buildConfigurations(
+      ConfTreeOperations appConf, ConfTreeOperations internalsConf,
+      String containerId, String roleName, String roleGroup,
+      StateAccessForProviders amState) {
+
+    Map<String, Map<String, String>> configurations = new TreeMap<>();
+    Map<String, String> tokens = getStandardTokenMap(appConf,
+        internalsConf, roleName, roleGroup, containerId);
+
+    Set<String> configs = new HashSet<>();
+    configs.addAll(getApplicationConfigurationTypes(roleGroup, appConf));
+    configs.addAll(getSystemConfigurationsRequested(appConf));
+
+    for (String configType : configs) {
+      addNamedConfiguration(configType, appConf.getGlobalOptions().options,
+          configurations, tokens, amState);
+      if (appConf.getComponent(roleGroup) != null) {
+        addNamedConfiguration(configType,
+            appConf.getComponent(roleGroup).options, configurations, tokens,
+            amState);
+      }
     }
+
+    //do a final replacement of re-used configs
+    dereferenceAllConfigs(configurations);
+
+    return configurations;
   }
 
   /**
-   * get the user name
-   * @return the user name
+   * Substitute "site." prefixed configuration values into other configuration
+   * values where needed. The format for these substitutions is that
+   * {@literal ${@//site/configDN/key}} will be replaced by the value for the
+   * "site.configDN.key" property.
+   * @param configurations configuration properties grouped by config
+   *                       description/name
    */
-  public String getUserName() throws IOException {
-    return UserGroupInformation.getCurrentUser().getShortUserName();
+  public void dereferenceAllConfigs(
+      Map<String, Map<String, String>> configurations) {
+    Map<String, String> allConfigs = new HashMap<>();
+    String lookupFormat = "${@//site/%s/%s}";
+    for (String configType : configurations.keySet()) {
+      Map<String, String> configBucket = configurations.get(configType);
+      for (String configName : configBucket.keySet()) {
+        allConfigs.put(String.format(lookupFormat, configType, configName),
+            configBucket.get(configName));
+      }
+    }
+
+    boolean finished = false;
+    while (!finished) {
+      finished = true;
+      for (Map.Entry<String, String> entry : allConfigs.entrySet()) {
+        String configValue = entry.getValue();
+        for (Map.Entry<String, String> lookUpEntry : allConfigs.entrySet()) {
+          String lookUpValue = lookUpEntry.getValue();
+          if (lookUpValue.contains("${@//site/")) {
+            continue;
+          }
+          String lookUpKey = lookUpEntry.getKey();
+          if (configValue != null && configValue.contains(lookUpKey)) {
+            configValue = configValue.replace(lookUpKey, lookUpValue);
+          }
+        }
+        if (!configValue.equals(entry.getValue())) {
+          finished = false;
+          allConfigs.put(entry.getKey(), configValue);
+        }
+      }
+    }
+
+    for (String configType : configurations.keySet()) {
+      Map<String, String> configBucket = configurations.get(configType);
+      for (Map.Entry<String, String> entry: configBucket.entrySet()) {
+        String configName = entry.getKey();
+        String configValue = entry.getValue();
+        for (Map.Entry<String, String> lookUpEntry : allConfigs.entrySet()) {
+          String lookUpValue = lookUpEntry.getValue();
+          if (lookUpValue.contains("${@//site/")) {
+            continue;
+          }
+          String lookUpKey = lookUpEntry.getKey();
+          if (configValue != null && configValue.contains(lookUpKey)) {
+            configValue = configValue.replace(lookUpKey, lookUpValue);
+          }
+        }
+        configBucket.put(configName, configValue);
+      }
+    }
   }
 
   /**
-   * Find a script in an expanded archive
-   * @param base base directory
-   * @param bindir bin subdir
-   * @param script script in bin subdir
-   * @return the path to the script
-   * @throws FileNotFoundException if a file is not found, or it is not a directory
+   * Return a set of configuration description/names represented in the app.
+   * configuration
+   * @param roleGroup component group
+   * @param appConf app configurations
+   * @return set of configuration description/names
    */
-  public File findBinScriptInExpandedArchive(File base,
-                                             String bindir,
-                                             String script)
-      throws FileNotFoundException {
-    
-    SliderUtils.verifyIsDir(base, log);
-    File[] ls = base.listFiles();
-    if (ls == null) {
-      //here for the IDE to be happy, as the previous check will pick this case
-      throw new FileNotFoundException("Failed to list directory " + base);
+  public Set<String> getApplicationConfigurationTypes(String roleGroup,
+      ConfTreeOperations appConf) {
+    Set<String> configList = new HashSet<>();
+
+    String prefix = OptionKeys.CONF_FILE_PREFIX;
+    String suffix = OptionKeys.TYPE_SUFFIX;
+    MapOperations component = appConf.getComponent(roleGroup);
+    if (component != null) {
+      addConfsToList(component, configList, prefix, suffix);
     }
+    addConfsToList(appConf.getGlobalOptions(), configList, prefix, suffix);
 
-    log.debug("Found {} entries in {}", ls.length, base);
-    List<File> directories = new LinkedList<File>();
-    StringBuilder dirs = new StringBuilder();
-    for (File file : ls) {
-      log.debug("{}", false);
-      if (file.isDirectory()) {
-        directories.add(file);
-        dirs.append(file.getPath()).append(" ");
+    return configList;
+  }
+
+  /**
+   * Finds all configuration description/names of the form
+   * prefixconfigDNsuffix in the configuration (e.g. conf.configDN.type).
+   * @param confMap configuration properties
+   * @param confList set containing configuration description/names
+   * @param prefix configuration key prefix to match
+   * @param suffix configuration key suffix to match
+   */
+  private void addConfsToList(Map<String, String> confMap,
+      Set<String> confList, String prefix, String suffix) {
+    for (String key : confMap.keySet()) {
+      if (key.startsWith(prefix) && key.endsWith(suffix)) {
+        String confName = key.substring(prefix.length(),
+            key.length() - suffix.length());
+        if (!confName.isEmpty()) {
+          confList.add(confName);
+        }
       }
     }
-    if (directories.size() > 1) {
-      throw new FileNotFoundException(
-        "Too many directories in archive to identify binary: " + dirs);
+  }
+
+  /**
+   * Build a map of package description/name to package key/value properties
+   * (there should be two properties, type and name).
+   * @param roleGroup component group
+   * @param appConf app configurations
+   * @return map of package description/name to package key/value properties
+   * @throws BadConfigException package name or type is not specified
+   */
+  public Map<String, Map<String, String>> getPackages(String roleGroup,
+      ConfTreeOperations appConf) throws BadConfigException {
+    Map<String, Map<String, String>> packages = new HashMap<>();
+    String prefix = OptionKeys.PKG_FILE_PREFIX;
+    String typeSuffix = OptionKeys.TYPE_SUFFIX;
+    String nameSuffix = OptionKeys.NAME_SUFFIX;
+    MapOperations component = appConf.getComponent(roleGroup);
+    if (component == null) {
+      component = appConf.getGlobalOptions();
+    }
+    for (Map.Entry<String, String> entry : component.entrySet()) {
+      String key = entry.getKey();
+      if (key.startsWith(prefix)) {
+        String confName;
+        String type;
+        if (key.endsWith(typeSuffix)) {
+          confName = key.substring(prefix.length(), key.length() - typeSuffix.length());
+          type = typeSuffix;
+        } else if (key.endsWith(nameSuffix)) {
+          confName = key.substring(prefix.length(), key.length() - nameSuffix.length());
+          type = nameSuffix;
+        } else {
+          continue;
+        }
+        if (!packages.containsKey(confName)) {
+          packages.put(confName, new HashMap<String, String>());
+        }
+        packages.get(confName).put(type, entry.getValue());
+      }
     }
-    if (directories.isEmpty()) {
-      throw new FileNotFoundException(
-        "No directory found in archive " + base);
+
+    for (Entry<String, Map<String, String>> pkg : packages.entrySet()) {
+      if (!pkg.getValue().containsKey(OptionKeys.TYPE_SUFFIX)) {
+        throw new BadConfigException("Package " + pkg.getKey() + " doesn't " +
+            "have a package type");
+      }
+      if (!pkg.getValue().containsKey(OptionKeys.NAME_SUFFIX)) {
+        throw new BadConfigException("Package " + pkg.getKey() + " doesn't " +
+            "have a package name");
+      }
     }
-    File archive = directories.get(0);
-    File bin = new File(archive, bindir);
-    SliderUtils.verifyIsDir(bin, log);
-    File scriptFile = new File(bin, script);
-    SliderUtils.verifyFileExists(scriptFile, log);
-    return scriptFile;
+
+    return packages;
   }
 
   /**
-   * Return any additional arguments (argv) to provide when starting this role
-   * 
-   * @param roleOptions
-   *          The options for this role
-   * @return A non-null String which contains command line arguments for this role, or the empty string.
+   * Return system configurations requested by the app.
+   * @param appConf app configurations
+   * @return set of system configurations
    */
-  public static String getAdditionalArgs(Map<String,String> roleOptions) {
-    if (roleOptions.containsKey(RoleKeys.ROLE_ADDITIONAL_ARGS)) {
-      String additionalArgs = roleOptions.get(RoleKeys.ROLE_ADDITIONAL_ARGS);
-      if (null != additionalArgs) {
-        return additionalArgs;
+  public Set<String> getSystemConfigurationsRequested(
+      ConfTreeOperations appConf) {
+    Set<String> configList = new HashSet<>();
+
+    String configTypes = appConf.get(SYSTEM_CONFIGS);
+    if (configTypes != null && configTypes.length() > 0) {
+      String[] configs = configTypes.split(",");
+      for (String config : configs) {
+        configList.add(config.trim());
       }
     }
 
-    return "";
+    return configList;
   }
-  
-  public int getRoleResourceRequirement(String val,
-                                        int defVal,
-                                        int maxVal) {
-    if (val==null) {
-      val = Integer.toString(defVal);
+
+  /**
+   * For a given config description/name, pull out its site configs from the
+   * source config map, remove the site.configDN. prefix from them, and place
+   * them into a new config map using the {@link #propagateSiteOptions} method
+   * (with tokens substituted). This new k/v map is put as the value for the
+   * configDN key in the configurations map.
+   * @param configName config description/name
+   * @param sourceConfig config containing site.* properties
+   * @param configurations configuration map to be populated
+   * @param tokens initial substitution tokens
+   * @param amState access to AM state
+   */
+  private void addNamedConfiguration(String configName,
+      Map<String, String> sourceConfig,
+      Map<String, Map<String, String>> configurations,
+      Map<String, String> tokens, StateAccessForProviders amState) {
+    Map<String, String> config = new HashMap<>();
+    if (configName.equals(GLOBAL_CONFIG_TAG)) {
+      addDefaultGlobalConfig(config);
     }
-    Integer intVal;
-    if (ResourceKeys.YARN_RESOURCE_MAX.equals(val)) {
-      intVal = maxVal;
+    // add role hosts to tokens
+    addRoleRelatedTokens(tokens, amState);
+    propagateSiteOptions(sourceConfig, config, configName, tokens);
+
+    configurations.put(configName, config);
+  }
+
+  /**
+   * Get initial token map to be substituted into config values.
+   * @param appConf app configurations
+   * @param internals internal configurations
+   * @param componentName component name
+   * @param componentGroup component group
+   * @param clusterName app name
+   * @return tokens to replace
+   */
+  public Map<String, String> getStandardTokenMap(ConfTreeOperations appConf,
+      ConfTreeOperations internals, String componentName,
+      String componentGroup, String clusterName) {
+    return getStandardTokenMap(appConf, internals, componentName,
+        componentGroup, null, clusterName);
+  }
+
+  /**
+   * Get initial token map to be substituted into config values.
+   * @param appConf app configurations
+   * @param internals internal configurations
+   * @param componentName component name
+   * @param componentGroup component group
+   * @param containerId container ID
+   * @param clusterName app name
+   * @return tokens to replace
+   */
+  public Map<String, String> getStandardTokenMap(ConfTreeOperations appConf,
+      ConfTreeOperations internals, String componentName,
+      String componentGroup, String containerId, String clusterName) {
+
+    Map<String, String> tokens = new HashMap<>();
+    if (containerId != null) {
+      tokens.put("${CONTAINER_ID}", containerId);
+    }
+    String nnuri = appConf.get("site.fs.defaultFS");
+    tokens.put("${NN_URI}", nnuri);
+    tokens.put("${NN_HOST}", URI.create(nnuri).getHost());
+    tokens.put("${ZK_HOST}", appConf.get(OptionKeys.ZOOKEEPER_HOSTS));
+    tokens.put("${DEFAULT_ZK_PATH}", appConf.get(OptionKeys.ZOOKEEPER_PATH));
+    String prefix = appConf.getComponentOpt(componentGroup, ROLE_PREFIX,
+        null);
+    String dataDirSuffix = "";
+    if (prefix == null) {
+      prefix = "";
     } else {
-      intVal = Integer.decode(val);
+      dataDirSuffix = "_" + SliderUtils.trimPrefix(prefix);
     }
-    return intVal;
+    tokens.put("${DEFAULT_DATA_DIR}", internals.getGlobalOptions()
+        .getOption(InternalKeys.INTERNAL_DATA_DIR_PATH, null) + dataDirSuffix);
+    tokens.put("${JAVA_HOME}", appConf.get(JAVA_HOME));
+    tokens.put("${COMPONENT_NAME}", componentName);
+    tokens.put("${COMPONENT_NAME.lc}", componentName.toLowerCase());
+    tokens.put("${COMPONENT_PREFIX}", prefix);
+    tokens.put("${COMPONENT_PREFIX.lc}", prefix.toLowerCase());
+    if (!componentName.equals(componentGroup) &&
+        componentName.startsWith(componentGroup)) {
+      tokens.put("${COMPONENT_ID}",
+          componentName.substring(componentGroup.length()));
+    }
+    if (clusterName != null) {
+      tokens.put("${CLUSTER_NAME}", clusterName);
+      tokens.put("${CLUSTER_NAME.lc}", clusterName.toLowerCase());
+      tokens.put("${APP_NAME}", clusterName);
+      tokens.put("${APP_NAME.lc}", clusterName.toLowerCase());
+    }
+    tokens.put("${APP_COMPONENT_NAME}", componentName);
+    tokens.put("${APP_COMPONENT_NAME.lc}", componentName.toLowerCase());
+    return tokens;
+  }
+
+  /**
+   * Add ROLE_HOST tokens for substitution into config values.
+   * @param tokens existing tokens
+   * @param amState access to AM state
+   */
+  public void addRoleRelatedTokens(Map<String, String> tokens,
+      StateAccessForProviders amState) {
+    if (amState == null) {
+      return;
+    }
+    for (Map.Entry<String, Map<String, ClusterNode>> entry :
+        amState.getRoleClusterNodeMapping().entrySet()) {
+      String tokenName = entry.getKey().toUpperCase(Locale.ENGLISH) + "_HOST";
+      String hosts = StringUtils .join(",",
+          getHostsList(entry.getValue().values(), true));
+      tokens.put("${" + tokenName + "}", hosts);
+    }
+  }
+
+  /**
+   * Add global configuration properties.
+   * @param config map where default global properties will be added
+   */
+  private void addDefaultGlobalConfig(Map<String, String> config) {
+    config.put("app_log_dir", "${LOG_DIR}");
+    config.put("app_pid_dir", "${WORK_DIR}/app/run");
+    config.put("app_install_dir", "${WORK_DIR}/app/install");
+    config.put("app_conf_dir", "${WORK_DIR}/" + APP_CONF_DIR);
+    config.put("app_input_conf_dir", "${WORK_DIR}/" + PROPAGATED_CONF_DIR_NAME);
+
+    // add optional parameters only if they are not already provided
+    if (!config.containsKey("pid_file")) {
+      config.put("pid_file", "${WORK_DIR}/app/run/component.pid");
+    }
+    if (!config.containsKey("app_root")) {
+      config.put("app_root", "${WORK_DIR}/app/install");
+    }
+  }
+
+  /**
+   * Return a list of hosts based on current ClusterNodes.
+   * @param values cluster nodes
+   * @param hostOnly whether host or host/server name will be added to list
+   * @return list of hosts
+   */
+  public Iterable<String> getHostsList(Collection<ClusterNode> values,
+      boolean hostOnly) {
+    List<String> hosts = new ArrayList<>();
+    for (ClusterNode cn : values) {
+      hosts.add(hostOnly ? cn.host : cn.host + "/" + cn.name);
+    }
+    return hosts;
+  }
+
+  /**
+   * Update ServiceRecord in Registry with IP and hostname.
+   * @param amState access to AM state
+   * @param yarnRegistry acces to YARN registry
+   * @param containerId container ID
+   * @param roleName component name
+   * @param ip list of IPs
+   * @param hostname hostname
+   */
+  public void updateServiceRecord(StateAccessForProviders amState,
+      YarnRegistryViewForProviders yarnRegistry,
+      String containerId, String roleName, List<String> ip, String hostname) {
+    try {
+      RoleInstance role = null;
+      if(ip != null && !ip.isEmpty()){
+        role = amState.getOwnedContainer(containerId);
+        role.ip = ip.get(0);
+      }
+      if(hostname != null && !hostname.isEmpty()){
+        role = amState.getOwnedContainer(containerId);
+        role.hostname = hostname;
+      }
+      if (role != null) {
+        // create and publish updated service record (including hostname & ip)
+        ServiceRecord record = new ServiceRecord();
+        record.set(YarnRegistryAttributes.YARN_ID, containerId);
+        record.description = roleName.replaceAll("_", "-");
+        record.set(YarnRegistryAttributes.YARN_PERSISTENCE,
+            PersistencePolicies.CONTAINER);
+        // TODO: use constants from YarnRegistryAttributes
+        if (role.ip != null) {
+          record.set("yarn:ip", role.ip);
+        }
+        if (role.hostname != null) {
+          record.set("yarn:hostname", role.hostname);
+        }
+        yarnRegistry.putComponent(
+            RegistryPathUtils.encodeYarnID(containerId), record);
+      }
+    } catch (NoSuchNodeException e) {
+      // ignore - there is nothing to do if we don't find a container
+      log.warn("Owned container {} not found - {}", containerId, e);
+    } catch (IOException e) {
+      log.warn("Error updating container {} service record in registry",
+          containerId, e);
+    }
+  }
+
+  /**
+   * Publish a named property bag that may contain name-value pairs for app
+   * configurations such as hbase-site.
+   * @param name config file identifying name
+   * @param description config file description
+   * @param entries config file properties
+   * @param amState access to AM state
+   */
+  public void publishApplicationInstanceData(String name, String description,
+      Iterable<Map.Entry<String, String>> entries,
+      StateAccessForProviders amState) {
+    PublishedConfiguration pubconf = new PublishedConfiguration(description,
+        entries);
+    log.info("publishing {}", pubconf);
+    amState.getPublishedSliderConfigurations().put(name, pubconf);
+  }
+
+  /**
+   * Publish an export group.
+   * @param exportGroup export groups
+   * @param amState access to AM state
+   * @param roleGroup component group
+   */
+  public void publishExportGroup(Map<String, List<ExportEntry>> exportGroup,
+      StateAccessForProviders amState, String roleGroup) {
+    // Publish in old format for the time being
+    Map<String, String> simpleEntries = new HashMap<>();
+    for (Entry<String, List<ExportEntry>> entry : exportGroup.entrySet()) {
+      List<ExportEntry> exports = entry.getValue();
+      if (SliderUtils.isNotEmpty(exports)) {
+        // there is no support for multiple exports per name, so extract only
+        // the first one
+        simpleEntries.put(entry.getKey(), entry.getValue().get(0).getValue());
+      }
+    }
+    publishApplicationInstanceData(roleGroup, roleGroup,
+        simpleEntries.entrySet(), amState);
+
+    PublishedExports exports = new PublishedExports(roleGroup);
+    exports.setUpdated(new Date().getTime());
+    exports.putValues(exportGroup.entrySet());
+    amState.getPublishedExportsSet().put(roleGroup, exports);
+  }
+
+  public Map<String, String> getExports(ConfTreeOperations appConf,
+      String roleGroup) {
+    Map<String, String> exports = new HashMap<>();
+    propagateOptions(appConf.getComponent(roleGroup).options, exports,
+        null, OptionKeys.EXPORT_PREFIX);
+    return exports;
+  }
+
+  private static final String COMPONENT_TAG = "component";
+  private static final String HOST_FOLDER_FORMAT = "%s:%s";
+  private static final String CONTAINER_LOGS_TAG = "container_log_dirs";
+  private static final String CONTAINER_PWDS_TAG = "container_work_dirs";
+
+  /**
+   * Format the folder locations and publish in the registry service.
+   * @param folders folder information
+   * @param containerId container ID
+   * @param hostFqdn host FQDN
+   * @param componentName component name
+   */
+  public void publishFolderPaths(Map<String, String> folders,
+      String containerId, String componentName, String hostFqdn,
+      StateAccessForProviders amState,
+      Map<String, ExportEntry> logFolderExports,
+      Map<String, ExportEntry> workFolderExports) {
+    Date now = new Date();
+    for (Map.Entry<String, String> entry : folders.entrySet()) {
+      ExportEntry exportEntry = new ExportEntry();
+      exportEntry.setValue(String.format(HOST_FOLDER_FORMAT, hostFqdn,
+          entry.getValue()));
+      exportEntry.setContainerId(containerId);
+      exportEntry.setLevel(COMPONENT_TAG);
+      exportEntry.setTag(componentName);
+      exportEntry.setUpdatedTime(now.toString());
+      if (entry.getKey().equals("AGENT_LOG_ROOT") ||
+          entry.getKey().equals("LOG_DIR")) {
+        synchronized (logFolderExports) {
+          logFolderExports.put(containerId, exportEntry);
+        }
+      } else {
+        synchronized (workFolderExports) {
+          workFolderExports.put(containerId, exportEntry);
+        }
+      }
+      log.info("Updating log and pwd folders for container {}", containerId);
+    }
+
+    PublishedExports exports = new PublishedExports(CONTAINER_LOGS_TAG);
+    exports.setUpdated(now.getTime());
+    synchronized (logFolderExports) {
+      updateExportsFromList(exports, logFolderExports);
+    }
+    amState.getPublishedExportsSet().put(CONTAINER_LOGS_TAG, exports);
+
+    exports = new PublishedExports(CONTAINER_PWDS_TAG);
+    exports.setUpdated(now.getTime());
+    synchronized (workFolderExports) {
+      updateExportsFromList(exports, workFolderExports);
+    }
+    amState.getPublishedExportsSet().put(CONTAINER_PWDS_TAG, exports);
+  }
+
+  /**
+   * Update the export data from the map.
+   * @param exports published exports
+   * @param folderExports folder exports
+   */
+  private void updateExportsFromList(PublishedExports exports,
+      Map<String, ExportEntry> folderExports) {
+    Map<String, List<ExportEntry>> perComponentList = new HashMap<>();
+    for(Map.Entry<String, ExportEntry> logEntry : folderExports.entrySet()) {
+      String componentName = logEntry.getValue().getTag();
+      if (!perComponentList.containsKey(componentName)) {
+        perComponentList.put(componentName, new ArrayList<ExportEntry>());
+      }
+      perComponentList.get(componentName).add(logEntry.getValue());
+    }
+    exports.putValues(perComponentList.entrySet());
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[68/76] [abbrv] hadoop git commit: YARN-5729. Bug fixes for the service Rest API. Contributed by Gour Saha

Posted by ji...@apache.org.
YARN-5729. Bug fixes for the service Rest API. Contributed by Gour Saha


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

Branch: refs/heads/yarn-native-services
Commit: b279e669cf4e9b096130f290243d23864f86d880
Parents: 5b72893
Author: Jian He <ji...@apache.org>
Authored: Fri Oct 14 13:47:38 2016 -0700
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 .../services/api/impl/ApplicationApiService.java    | 16 ++++++++++++++--
 .../hadoop/yarn/services/resource/Application.java  |  6 +++---
 .../hadoop/yarn/services/resource/Artifact.java     |  4 +++-
 .../hadoop/yarn/services/resource/Component.java    |  4 +++-
 .../hadoop/yarn/services/resource/ConfigFile.java   |  4 +++-
 .../yarn/services/resource/Configuration.java       |  4 +++-
 .../hadoop/yarn/services/resource/Container.java    |  6 +++---
 .../yarn/services/resource/PlacementPolicy.java     |  4 +++-
 .../yarn/services/resource/ReadinessCheck.java      |  4 +++-
 .../hadoop/yarn/services/resource/Resource.java     |  2 +-
 10 files changed, 39 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b279e669/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
index 0a62629..21cf113 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
@@ -276,7 +276,7 @@ public class ApplicationApiService implements ApplicationApi {
 
     // If it is a simple app with no components, then create a default component
     if (application.getComponents() == null) {
-      application.setComponents(getDefaultComponentAsList());
+      application.setComponents(getDefaultComponentAsList(application));
     }
 
     // Application lifetime if not specified, is set to unlimited lifetime
@@ -1029,7 +1029,8 @@ public class ApplicationApiService implements ApplicationApi {
     // end-users point of view, is out of scope of the REST API. Also, this
     // readiness has nothing to do with readiness-check defined at the component
     // level (which is used for dependency resolution of component DAG).
-    if (totalNumberOfIpAssignedContainers == totalExpectedNumberOfRunningContainers) {
+    if (totalNumberOfIpAssignedContainers
+        .longValue() == totalExpectedNumberOfRunningContainers.longValue()) {
       app.setState(ApplicationState.READY);
     }
     logger.info("Application = {}", app);
@@ -1389,6 +1390,17 @@ public class ApplicationApiService implements ApplicationApi {
     return Response.status(Status.NO_CONTENT).build();
   }
 
+  // create default component and initialize with app level global values
+  private List<Component> getDefaultComponentAsList(Application app) {
+    List<Component> comps = getDefaultComponentAsList();
+    Component comp = comps.get(0);
+    comp.setArtifact(app.getArtifact());
+    comp.setResource(app.getResource());
+    comp.setNumberOfContainers(app.getNumberOfContainers());
+    comp.setLaunchCommand(app.getLaunchCommand());
+    return comps;
+  }
+
   private List<Component> getDefaultComponentAsList() {
     Component comp = new Component();
     comp.setName(DEFAULT_COMPONENT_NAME);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b279e669/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Application.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Application.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Application.java
index 719bf95..ed65ad2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Application.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Application.java
@@ -168,19 +168,19 @@ public class Application extends BaseResource {
    * The time when the application was created, e.g. 2016-03-16T01:01:49.000Z.
    **/
   public Application launchTime(Date launchTime) {
-    this.launchTime = launchTime;
+    this.launchTime = launchTime == null ? null : (Date) launchTime.clone();
     return this;
   }
 
   @ApiModelProperty(example = "null", value = "The time when the application was created, e.g. 2016-03-16T01:01:49.000Z.")
   @JsonProperty("launch_time")
   public String getLaunchTime() {
-    return launchTime.toString();
+    return launchTime == null ? null : launchTime.toString();
   }
 
   @XmlElement(name = "launch_time")
   public void setLaunchTime(Date launchTime) {
-    this.launchTime = launchTime;
+    this.launchTime = launchTime == null ? null : (Date) launchTime.clone();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b279e669/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Artifact.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Artifact.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Artifact.java
index af0ad12..9ac2bc7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Artifact.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Artifact.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.services.resource;
 import io.swagger.annotations.ApiModel;
 import io.swagger.annotations.ApiModelProperty;
 
+import java.io.Serializable;
 import java.util.Objects;
 
 import com.fasterxml.jackson.annotation.JsonInclude;
@@ -33,7 +34,8 @@ import com.fasterxml.jackson.annotation.JsonValue;
 @ApiModel(description = "Artifact of an application component")
 @javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
 @JsonInclude(JsonInclude.Include.NON_NULL)
-public class Artifact {
+public class Artifact implements Serializable {
+  private static final long serialVersionUID = 3608929500111099035L;
 
   private String id = null;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b279e669/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Component.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Component.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Component.java
index 1246aa8..75f579a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Component.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Component.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.services.resource;
 import io.swagger.annotations.ApiModel;
 import io.swagger.annotations.ApiModelProperty;
 
+import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Objects;
@@ -42,7 +43,8 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 @javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
 @XmlRootElement
 @JsonInclude(JsonInclude.Include.NON_NULL)
-public class Component {
+public class Component implements Serializable {
+  private static final long serialVersionUID = -8430058381509087805L;
 
   private String name = null;
   private List<String> dependencies = new ArrayList<String>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b279e669/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ConfigFile.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ConfigFile.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ConfigFile.java
index d06c1b8..3ced153 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ConfigFile.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ConfigFile.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.services.resource;
 import io.swagger.annotations.ApiModel;
 import io.swagger.annotations.ApiModelProperty;
 
+import java.io.Serializable;
 import java.util.Objects;
 
 import javax.xml.bind.annotation.XmlElement;
@@ -38,7 +39,8 @@ import com.fasterxml.jackson.annotation.JsonValue;
 @javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
 @XmlRootElement
 @JsonInclude(JsonInclude.Include.NON_NULL)
-public class ConfigFile {
+public class ConfigFile implements Serializable {
+  private static final long serialVersionUID = -7009402089417704612L;
 
   public enum TypeEnum {
     XML("xml"), PROPERTIES("properties"), JSON("json"), YAML("yaml"), TEMPLATE(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b279e669/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Configuration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Configuration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Configuration.java
index 05983db..908220a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Configuration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Configuration.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.services.resource;
 import io.swagger.annotations.ApiModel;
 import io.swagger.annotations.ApiModelProperty;
 
+import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -39,7 +40,8 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 @ApiModel(description = "Set of configuration properties that can be injected into the application components via envs, files and custom pluggable helper docker containers. Files of several standard formats like xml, properties, json, yaml and templates will be supported.")
 @javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
 @JsonInclude(JsonInclude.Include.NON_NULL)
-public class Configuration {
+public class Configuration implements Serializable {
+  private static final long serialVersionUID = -4330788704981074466L;
 
   private Map<String, String> properties = new HashMap<String, String>();
   private Map<String, String> env = new HashMap<String, String>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b279e669/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Container.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Container.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Container.java
index 4e40102..a4efdf3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Container.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Container.java
@@ -73,19 +73,19 @@ public class Container extends BaseResource {
    * This will most likely be different from cluster launch time.
    **/
   public Container launchTime(Date launchTime) {
-    this.launchTime = launchTime;
+    this.launchTime = launchTime == null ? null : (Date) launchTime.clone();
     return this;
   }
 
   @ApiModelProperty(example = "null", value = "The time when the container was created, e.g. 2016-03-16T01:01:49.000Z. This will most likely be different from cluster launch time.")
   @JsonProperty("launch_time")
   public String getLaunchTime() {
-    return launchTime.toString();
+    return launchTime == null ? null : launchTime.toString();
   }
 
   @XmlElement(name = "launch_time")
   public void setLaunchTime(Date launchTime) {
-    this.launchTime = launchTime;
+    this.launchTime = launchTime == null ? null : (Date) launchTime.clone();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b279e669/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/PlacementPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/PlacementPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/PlacementPolicy.java
index 7541e2f..5df00a0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/PlacementPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/PlacementPolicy.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.services.resource;
 import io.swagger.annotations.ApiModel;
 import io.swagger.annotations.ApiModelProperty;
 
+import java.io.Serializable;
 import java.util.Objects;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -31,7 +32,8 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 
 @ApiModel(description = "Placement policy of an instance of an application. This feature is in the works in YARN-4902.")
 @javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
-public class PlacementPolicy {
+public class PlacementPolicy implements Serializable {
+  private static final long serialVersionUID = 4341110649551172231L;
 
   private String label = null;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b279e669/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ReadinessCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ReadinessCheck.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ReadinessCheck.java
index 10c951a..f549746 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ReadinessCheck.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ReadinessCheck.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.services.resource;
 import io.swagger.annotations.ApiModel;
 import io.swagger.annotations.ApiModelProperty;
 
+import java.io.Serializable;
 import java.util.Objects;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -34,7 +35,8 @@ import com.fasterxml.jackson.annotation.JsonValue;
 
 @ApiModel(description = "A custom command or a pluggable helper container to determine the readiness of a container of a component. Readiness for every application is different. Hence the need for a simple interface, with scope to support advanced usecases.")
 @javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
-public class ReadinessCheck {
+public class ReadinessCheck implements Serializable {
+  private static final long serialVersionUID = -3836839816887186801L;
 
   public enum TypeEnum {
     HTTP("http");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b279e669/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Resource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Resource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Resource.java
index a3780cc..234ccb3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Resource.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Resource.java
@@ -35,7 +35,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 
 @ApiModel(description = "Resource determines the amount of resources (vcores, memory, network, etc.) usable by a container. This field determines the resource to be applied for all the containers of a component or application. The resource specified at the app (or global) level can be overriden at the component level. Only one of profile OR cpu & memory are exepected. It raises a validation exception otherwise.")
 @javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
-public class Resource extends BaseResource {
+public class Resource extends BaseResource implements Cloneable {
   private static final long serialVersionUID = -6431667797380250037L;
 
   private String profile = null;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[63/76] [abbrv] hadoop git commit: YARN-5505. Create an agent-less docker provider in the native-services framework. Contributed by Billie Rinaldi

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb61fe3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerProviderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerProviderService.java
new file mode 100644
index 0000000..bebb5f0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerProviderService.java
@@ -0,0 +1,355 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.docker;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.slider.api.ClusterDescription;
+import org.apache.slider.api.ClusterNode;
+import org.apache.slider.api.OptionKeys;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.core.launch.CommandLineBuilder;
+import org.apache.slider.core.launch.ContainerLauncher;
+import org.apache.slider.core.registry.docstore.ConfigFormat;
+import org.apache.slider.core.registry.docstore.ConfigUtils;
+import org.apache.slider.core.registry.docstore.ExportEntry;
+import org.apache.slider.providers.AbstractProviderService;
+import org.apache.slider.providers.ProviderCore;
+import org.apache.slider.providers.ProviderRole;
+import org.apache.slider.providers.ProviderUtils;
+import org.apache.slider.server.appmaster.state.RoleInstance;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Scanner;
+
+public class DockerProviderService extends AbstractProviderService implements
+    ProviderCore,
+    DockerKeys,
+    SliderKeys {
+
+  protected static final Logger log =
+      LoggerFactory.getLogger(DockerProviderService.class);
+  private static final ProviderUtils providerUtils = new ProviderUtils(log);
+  private static final String EXPORT_GROUP = "quicklinks";
+  private static final String APPLICATION_TAG = "application";
+
+  private String clusterName = null;
+  private SliderFileSystem fileSystem = null;
+
+  protected DockerProviderService() {
+    super("DockerProviderService");
+  }
+
+  @Override
+  public List<ProviderRole> getRoles() {
+    return Collections.emptyList();
+  }
+
+  @Override
+  public boolean isSupportedRole(String role) {
+    return true;
+  }
+
+  @Override
+  public void validateInstanceDefinition(AggregateConf instanceDefinition)
+      throws SliderException {
+  }
+
+  private String getClusterName() {
+    if (SliderUtils.isUnset(clusterName)) {
+      clusterName = getAmState().getInternalsSnapshot().get(OptionKeys.APPLICATION_NAME);
+    }
+    return clusterName;
+  }
+
+  @Override
+  public void buildContainerLaunchContext(ContainerLauncher launcher,
+      AggregateConf instanceDefinition, Container container,
+      ProviderRole providerRole, SliderFileSystem fileSystem,
+      Path generatedConfPath, MapOperations resourceComponent,
+      MapOperations appComponent, Path containerTmpDirPath)
+      throws IOException, SliderException {
+
+    String roleName = providerRole.name;
+    String roleGroup = providerRole.group;
+
+    initializeApplicationConfiguration(instanceDefinition, fileSystem,
+        roleGroup);
+
+    log.info("Build launch context for Docker");
+    log.debug(instanceDefinition.toString());
+
+    ConfTreeOperations appConf = instanceDefinition.getAppConfOperations();
+    launcher.setYarnDockerMode(true);
+    launcher.setDockerImage(appConf.getComponentOpt(roleGroup, DOCKER_IMAGE,
+        null));
+    launcher.setDockerNetwork(appConf.getComponentOpt(roleGroup, DOCKER_NETWORK,
+        DEFAULT_DOCKER_NETWORK));
+    launcher.setRunPrivilegedContainer(appConf.getComponentOptBool(roleGroup,
+        DOCKER_USE_PRIVILEGED, false));
+
+    // Set the environment
+    launcher.putEnv(SliderUtils.buildEnvMap(appComponent,
+        providerUtils.getStandardTokenMap(getAmState().getAppConfSnapshot(),
+            getAmState().getInternalsSnapshot(), roleName, roleGroup,
+            getClusterName())));
+
+    String workDir = ApplicationConstants.Environment.PWD.$();
+    launcher.setEnv("WORK_DIR", workDir);
+    log.info("WORK_DIR set to {}", workDir);
+    String logDir = ApplicationConstants.LOG_DIR_EXPANSION_VAR;
+    launcher.setEnv("LOG_DIR", logDir);
+    log.info("LOG_DIR set to {}", logDir);
+    if (System.getenv(HADOOP_USER_NAME) != null) {
+      launcher.setEnv(HADOOP_USER_NAME, System.getenv(HADOOP_USER_NAME));
+    }
+    //add english env
+    launcher.setEnv("LANG", "en_US.UTF-8");
+    launcher.setEnv("LC_ALL", "en_US.UTF-8");
+    launcher.setEnv("LANGUAGE", "en_US.UTF-8");
+
+    //local resources
+    providerUtils.localizePackages(launcher, fileSystem, appConf, roleGroup,
+        getClusterName());
+
+    if (SliderUtils.isHadoopClusterSecure(getConfig())) {
+      providerUtils.localizeServiceKeytabs(launcher, instanceDefinition,
+          fileSystem, getClusterName());
+    }
+
+    if (providerUtils.areStoresRequested(appComponent)) {
+      providerUtils.localizeContainerSecurityStores(launcher, container,
+          roleName, fileSystem, instanceDefinition, appComponent, getClusterName());
+    }
+
+    if (appComponent.getOptionBool(AM_CONFIG_GENERATION, false)) {
+      // build and localize configuration files
+      Map<String, Map<String, String>> configurations =
+          providerUtils.buildConfigurations(
+              instanceDefinition.getAppConfOperations(),
+              instanceDefinition.getInternalOperations(),
+              container.getId().toString(), roleName, roleGroup,
+              getAmState());
+      providerUtils.localizeConfigFiles(launcher, roleName, roleGroup,
+          appConf, configurations, launcher.getEnv(), fileSystem,
+          getClusterName());
+    }
+
+    //add the configuration resources
+    launcher.addLocalResources(fileSystem.submitDirectory(
+        generatedConfPath,
+        PROPAGATED_CONF_DIR_NAME));
+
+    CommandLineBuilder operation = new CommandLineBuilder();
+    operation.add(appConf.getComponentOpt(roleGroup, DOCKER_START_COMMAND,
+        "/bin/bash"));
+
+    operation.add("> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/"
+        + OUT_FILE + " 2>" + ERR_FILE);
+
+    launcher.addCommand(operation.build());
+
+    // Additional files to localize
+    String appResourcesString = instanceDefinition.getAppConfOperations()
+        .getGlobalOptions().getOption(APP_RESOURCES, null);
+    log.info("Configuration value for extra resources to localize: {}", appResourcesString);
+    if (null != appResourcesString) {
+      try (Scanner scanner = new Scanner(appResourcesString).useDelimiter(",")) {
+        while (scanner.hasNext()) {
+          String resource = scanner.next();
+          Path resourcePath = new Path(resource);
+          LocalResource extraResource = fileSystem.createAmResource(
+              fileSystem.getFileSystem().resolvePath(resourcePath),
+              LocalResourceType.FILE);
+          String destination = APP_RESOURCES_DIR + "/" + resourcePath.getName();
+          log.info("Localizing {} to {}", resourcePath, destination);
+          // TODO Can we try harder to avoid collisions?
+          launcher.addLocalResource(destination, extraResource);
+        }
+      }
+    }
+  }
+
+  @Override
+  public void initializeApplicationConfiguration(
+      AggregateConf instanceDefinition, SliderFileSystem fileSystem,
+      String roleGroup)
+      throws IOException, SliderException {
+        this.fileSystem = fileSystem;
+  }
+
+  @Override
+  public void applyInitialRegistryDefinitions(URL amWebURI,
+      ServiceRecord serviceRecord)
+      throws IOException {
+    super.applyInitialRegistryDefinitions(amWebURI, serviceRecord);
+
+    // identify client component
+    String clientName = null;
+    ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
+    for (String component : appConf.getComponentNames()) {
+      if (COMPONENT_TYPE_CLIENT.equals(appConf.getComponentOpt(component,
+          COMPONENT_TYPE_KEY, null))) {
+        clientName = component;
+        break;
+      }
+    }
+    if (clientName == null) {
+      log.info("No client component specified, not publishing client configs");
+      return;
+    }
+
+    // register AM-generated client configs
+    // appConf should already be resolved!
+    MapOperations clientOperations = appConf.getComponent(clientName);
+    if (!clientOperations.getOptionBool(AM_CONFIG_GENERATION, false)) {
+      log.info("AM config generation is false, not publishing client configs");
+      return;
+    }
+
+    // build and localize configuration files
+    Map<String, Map<String, String>> configurations =
+        providerUtils.buildConfigurations(appConf, getAmState()
+            .getInternalsSnapshot(), null, clientName, clientName,
+            getAmState());
+
+    for (String configFileDN : configurations.keySet()) {
+      String configFileName = appConf.getComponentOpt(clientName,
+          OptionKeys.CONF_FILE_PREFIX + configFileDN + OptionKeys
+              .NAME_SUFFIX, null);
+      String configFileType = appConf.getComponentOpt(clientName,
+          OptionKeys.CONF_FILE_PREFIX + configFileDN + OptionKeys
+              .TYPE_SUFFIX, null);
+      if (configFileName == null && configFileType == null) {
+        continue;
+      }
+      ConfigFormat configFormat = ConfigFormat.resolve(configFileType);
+
+      Map<String, String> config = configurations.get(configFileDN);
+      ConfigUtils.prepConfigForTemplateOutputter(configFormat, config,
+          fileSystem, getClusterName(),
+          new File(configFileName).getName());
+      providerUtils.publishApplicationInstanceData(configFileDN, configFileDN,
+          config.entrySet(), getAmState());
+    }
+  }
+
+  @Override
+  public boolean processContainerStatus(ContainerId containerId,
+      ContainerStatus status) {
+    log.debug("Handling container status: {}", status);
+    if (SliderUtils.isEmpty(status.getIPs()) ||
+        SliderUtils.isUnset(status.getHost())) {
+      return true;
+    }
+    RoleInstance instance = getAmState().getOwnedContainer(containerId);
+    if (instance == null) {
+      // container is completed?
+      return false;
+    }
+
+    String roleName = instance.role;
+    String roleGroup = instance.group;
+    String containerIdStr = containerId.toString();
+
+    providerUtils.updateServiceRecord(getAmState(), yarnRegistry,
+        containerIdStr, roleName, status.getIPs(), status.getHost());
+
+    publishExportGroups(containerIdStr, roleName, roleGroup,
+        status.getHost());
+    return false;
+  }
+
+  /**
+   * This method looks for configuration properties of the form
+   * export.key,value and publishes the key,value pair. Standard tokens are
+   * substituted into the value, and COMPONENTNAME_HOST and THIS_HOST tokens
+   * are substituted with the actual hostnames of the containers.
+   */
+  protected void publishExportGroups(String containerId,
+      String roleName, String roleGroup, String thisHost) {
+    ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
+    ConfTreeOperations internalsConf = getAmState().getInternalsSnapshot();
+
+    Map<String, String> exports = providerUtils.getExports(
+        getAmState().getAppConfSnapshot(), roleGroup);
+
+    String hostKeyFormat = "${%s_HOST}";
+
+    // publish export groups if any
+    Map<String, String> replaceTokens =
+        providerUtils.filterSiteOptions(
+            appConf.getComponent(roleGroup).options,
+            providerUtils.getStandardTokenMap(appConf, internalsConf, roleName,
+                roleGroup, containerId, getClusterName()));
+    for (Map.Entry<String, Map<String, ClusterNode>> entry :
+        getAmState().getRoleClusterNodeMapping().entrySet()) {
+      String hostName = providerUtils.getHostsList(
+          entry.getValue().values(), true).iterator().next();
+      replaceTokens.put(String.format(hostKeyFormat,
+          entry.getKey().toUpperCase(Locale.ENGLISH)), hostName);
+    }
+    replaceTokens.put("${THIS_HOST}", thisHost);
+
+    Map<String, List<ExportEntry>> entries = new HashMap<>();
+    for (Entry<String, String> export : exports.entrySet()) {
+      String value = export.getValue();
+      // replace host names and site properties
+      for (String token : replaceTokens.keySet()) {
+        if (value.contains(token)) {
+          value = value.replace(token, replaceTokens.get(token));
+        }
+      }
+      ExportEntry entry = new ExportEntry();
+      entry.setLevel(APPLICATION_TAG);
+      entry.setValue(value);
+      entry.setUpdatedTime(new Date().toString());
+      // over-write, app exports are singletons
+      entries.put(export.getKey(), new ArrayList(Arrays.asList(entry)));
+      log.info("Preparing to publish. Key {} and Value {}",
+          export.getKey(), value);
+    }
+    providerUtils.publishExportGroup(entries, getAmState(), EXPORT_GROUP);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb61fe3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
index 67d3647..c021b80 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
@@ -103,13 +103,9 @@ public class SliderAMProviderService extends AbstractProviderService implements
 
   @Override
   public void applyInitialRegistryDefinitions(URL amWebURI,
-      URL agentOpsURI,
-      URL agentStatusURI,
       ServiceRecord serviceRecord)
       throws IOException {
     super.applyInitialRegistryDefinitions(amWebURI,
-        agentOpsURI,
-        agentStatusURI,
         serviceRecord);
     // now publish site.xml files
     YarnConfiguration defaultYarnConfig = new YarnConfiguration();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb61fe3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
index 983b5ba..57ec218 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
@@ -784,8 +784,10 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
               contentCache);
       initAMFilterOptions(serviceConf);
 
-      // start the agent web app
-      startAgentWebApp(appInformation, serviceConf, webAppApi);
+      if (providerService instanceof AgentProviderService) {
+        // start the agent web app
+        startAgentWebApp(appInformation, serviceConf, webAppApi);
+      }
       int webAppPort = deployWebApplication(webAppApi);
 
       String scheme = WebAppUtils.HTTP_PREFIX;
@@ -1296,8 +1298,6 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     
     // the registry is running, so register services
     URL amWebURI = new URL(appMasterProxiedUrl);
-    URL agentOpsURI = new URL(agentOpsUrl);
-    URL agentStatusURI = new URL(agentStatusUrl);
 
     //Give the provider restricted access to the state, registry
     setupInitialRegistryPaths();
@@ -1324,15 +1324,20 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
             
     // internal services
     sliderAMProvider.applyInitialRegistryDefinitions(amWebURI,
-        agentOpsURI,
-        agentStatusURI,
         serviceRecord);
 
     // provider service dynamic definitions.
-    providerService.applyInitialRegistryDefinitions(amWebURI,
-        agentOpsURI,
-        agentStatusURI,
-        serviceRecord);
+    if (providerService instanceof AgentProviderService) {
+      URL agentOpsURI = new URL(agentOpsUrl);
+      URL agentStatusURI = new URL(agentStatusUrl);
+      ((AgentProviderService)providerService).applyInitialRegistryDefinitions(
+          amWebURI,
+          agentOpsURI,
+          agentStatusURI,
+          serviceRecord);
+    } else {
+      providerService.applyInitialRegistryDefinitions(amWebURI, serviceRecord);
+    }
 
     // set any provided attributes
     setProvidedServiceRecordAttributes(
@@ -2285,6 +2290,20 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
       ContainerStatus containerStatus) {
     LOG_YARN.debug("Container Status: id={}, status={}", containerId,
         containerStatus);
+    if (providerService.processContainerStatus(containerId, containerStatus)) {
+      try {
+        Thread.sleep(1000);
+      } catch (InterruptedException e) {
+      }
+      RoleInstance cinfo = appState.getOwnedContainer(containerId);
+      if (cinfo != null) {
+        LOG_YARN.info("Re-requesting status for role {}, {}",
+            cinfo.role, containerId);
+        //trigger another async container status
+        nmClientAsync.getContainerStatusAsync(containerId,
+            cinfo.container.getNodeId());
+      }
+    }
   }
 
   @Override //  NMClientAsync.CallbackHandler 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb61fe3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/slider.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/slider.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/slider.xml
index 37ac65c..a2517d5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/slider.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/resources/org/apache/slider/slider.xml
@@ -27,4 +27,8 @@
     <name>slider.provider.agent</name>
     <value>org.apache.slider.providers.agent.AgentProviderFactory</value>
   </property>
+  <property>
+    <name>slider.provider.docker</name>
+    <value>org.apache.slider.providers.docker.DockerProviderFactory</value>
+  </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb61fe3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/providers/docker/appConfig.json
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/providers/docker/appConfig.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/providers/docker/appConfig.json
new file mode 100644
index 0000000..c87f77c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/providers/docker/appConfig.json
@@ -0,0 +1,42 @@
+{
+  "schema": "http://example.org/specification/v2.0.0",
+  "metadata": {},
+  "global": {
+    "am.config.generation": "true",
+    "component.unique.names": "true",
+
+    "export.app.monitor": "${COMPONENT1_HOST} : ${@//site/test-xml/xmlkey}",
+    "export.other.key": "exportvalue",
+
+    "docker.image": "docker.io/centos:centos6",
+    "docker.startCommand": "sleep 600",
+
+    "conf.test-json.type": "json",
+    "conf.test-json.name": "/tmp/test.json",
+    "conf.test-xml.type": "xml",
+    "conf.test-xml.name": "/tmp/test.xml",
+    "conf.test-properties.type": "properties",
+    "conf.test-properties.name": "/tmp/test.xml",
+    "conf.test-yaml.type": "yaml",
+    "conf.test-yaml.name": "/tmp/test.yaml",
+    "conf.test-env.type": "env",
+    "conf.test-env.name": "/tmp/testenv",
+    "conf.test-template.type": "template",
+    "conf.test-template.name": "/tmp/test.template",
+    "conf.test-hadoop-xml.type": "hadoop-xml",
+    "conf.test-hadoop-xml.name": "/tmp/test-hadoop.xml",
+
+    "site.test-json.jsonkey": "val1",
+    "site.test-xml.xmlkey": "val2",
+    "site.test-hadoop-xml.xmlkey": "val3",
+    "site.test-properties.propkey": "val4",
+    "site.test-yaml.yamlkey": "val5",
+    "site.test-env.content": "test ${envkey1} {{envkey2}} content",
+    "site.test-env.envkey1": "envval1",
+    "site.test-env.envkey2": "envval2",
+    "site.test-template.templatekey1": "templateval1",
+    "site.test-template.templatekey2": "templateval2"
+  },
+  "components": {
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb61fe3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/providers/docker/resources.json
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/providers/docker/resources.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/providers/docker/resources.json
new file mode 100644
index 0000000..1b06224
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/providers/docker/resources.json
@@ -0,0 +1,16 @@
+{
+  "schema": "http://example.org/specification/v2.0.0",
+  "metadata": {},
+  "global": {},
+  "components": {
+    "slider-appmaster": {
+      "yarn.memory": "384"
+    },
+    "COMPONENT": {
+      "yarn.role.priority": "1",
+      "yarn.component.instances": 2,
+      "yarn.memory": "512",
+      "yarn.vcores": "2"
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb61fe3f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/providers/docker/test.template
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/providers/docker/test.template b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/providers/docker/test.template
new file mode 100644
index 0000000..2922655
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/providers/docker/test.template
@@ -0,0 +1,16 @@
+# 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.
+
+test ${templatekey1} {{templatekey2}} content


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[09/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ProviderAppState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ProviderAppState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ProviderAppState.java
new file mode 100644
index 0000000..37e9a7f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ProviderAppState.java
@@ -0,0 +1,307 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.slider.api.ClusterDescription;
+import org.apache.slider.api.ClusterNode;
+import org.apache.slider.api.types.ApplicationLivenessInformation;
+import org.apache.slider.api.types.ComponentInformation;
+import org.apache.slider.api.types.NodeInformation;
+import org.apache.slider.api.types.RoleStatistics;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.core.exceptions.NoSuchNodeException;
+import org.apache.slider.core.registry.docstore.PublishedConfigSet;
+import org.apache.slider.core.registry.docstore.PublishedExportsSet;
+import org.apache.slider.server.appmaster.web.rest.RestPaths;
+import org.apache.slider.server.services.utility.PatternValidator;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Implementation of {@link StateAccessForProviders}, which means
+ * state access for providers, web UI and IPC/REST views.
+ */
+public class ProviderAppState implements StateAccessForProviders {
+
+
+  private final Map<String, PublishedConfigSet> publishedConfigSets =
+      new ConcurrentHashMap<>(5);
+  private final PublishedExportsSet publishedExportsSets = new PublishedExportsSet();
+  private static final PatternValidator validator = new PatternValidator(
+      RestPaths.PUBLISHED_CONFIGURATION_SET_REGEXP);
+  private String applicationName;
+
+  private final AppState appState;
+
+  public ProviderAppState(String applicationName, AppState appState) {
+    this.appState = appState;
+    this.applicationName = applicationName;
+  }
+
+  public void setApplicationName(String applicationName) {
+    this.applicationName = applicationName;
+  }
+
+  @Override
+  public String getApplicationName() {
+    return applicationName;
+  }
+
+  @Override
+  public PublishedConfigSet getPublishedSliderConfigurations() {
+    return getOrCreatePublishedConfigSet(RestPaths.SLIDER_CONFIGSET);
+  }
+
+  @Override
+  public PublishedExportsSet getPublishedExportsSet() {
+    return publishedExportsSets;
+  }
+
+  @Override
+  public PublishedConfigSet getPublishedConfigSet(String name) {
+    return publishedConfigSets.get(name);
+  }
+
+  @Override
+  public PublishedConfigSet getOrCreatePublishedConfigSet(String name) {
+    PublishedConfigSet set = publishedConfigSets.get(name);
+    if (set == null) {
+      validator.validate(name);
+      synchronized (publishedConfigSets) {
+        // synchronized double check to ensure that there is never an overridden
+        // config set created
+        set = publishedConfigSets.get(name);
+        if (set == null) {
+          set = new PublishedConfigSet();
+          publishedConfigSets.put(name, set);
+        }
+      }
+    }
+    return set;
+  }
+
+  @Override
+  public List<String> listConfigSets() {
+
+    synchronized (publishedConfigSets) {
+      List<String> sets = new ArrayList<>(publishedConfigSets.keySet());
+      return sets;
+    }
+  }
+
+  @Override
+  public Map<Integer, RoleStatus> getRoleStatusMap() {
+    return appState.getRoleStatusMap();
+  }
+
+
+  @Override
+  public Map<ContainerId, RoleInstance> getFailedContainers() {
+    return appState.getFailedContainers();
+  }
+
+  @Override
+  public Map<ContainerId, RoleInstance> getLiveContainers() {
+    return appState.getLiveContainers();
+  }
+
+  @Override
+  public ClusterDescription getClusterStatus() {
+    return appState.getClusterStatus();
+  }
+
+  @Override
+  public ConfTreeOperations getResourcesSnapshot() {
+    return appState.getResourcesSnapshot();
+  }
+
+  @Override
+  public ConfTreeOperations getAppConfSnapshot() {
+    return appState.getAppConfSnapshot();
+  }
+
+  @Override
+  public ConfTreeOperations getInternalsSnapshot() {
+    return appState.getInternalsSnapshot();
+  }
+
+  @Override
+  public boolean isApplicationLive() {
+    return appState.isApplicationLive();
+  }
+
+  @Override
+  public long getSnapshotTime() {
+    return appState.getSnapshotTime();
+  }
+
+  @Override
+  public AggregateConf getInstanceDefinitionSnapshot() {
+    return appState.getInstanceDefinitionSnapshot();
+  }
+  
+  @Override
+  public AggregateConf getUnresolvedInstanceDefinition() {
+    return appState.getUnresolvedInstanceDefinition();
+  }
+
+  @Override
+  public RoleStatus lookupRoleStatus(int key) {
+    return appState.lookupRoleStatus(key);
+  }
+
+  @Override
+  public RoleStatus lookupRoleStatus(Container c) throws YarnRuntimeException {
+    return appState.lookupRoleStatus(c);
+  }
+
+  @Override
+  public RoleStatus lookupRoleStatus(String name) throws YarnRuntimeException {
+    return appState.lookupRoleStatus(name);
+  }
+
+  @Override
+  public List<RoleInstance> cloneOwnedContainerList() {
+    return appState.cloneOwnedContainerList();
+  }
+
+  @Override
+  public int getNumOwnedContainers() {
+    return appState.getNumOwnedContainers();
+  }
+
+  @Override
+  public RoleInstance getOwnedContainer(ContainerId id) {
+    return appState.getOwnedContainer(id);
+  }
+
+  @Override
+  public RoleInstance getOwnedContainer(String id) throws NoSuchNodeException {
+    return appState.getOwnedInstanceByContainerID(id);
+  }
+
+  @Override
+  public List<RoleInstance> cloneLiveContainerInfoList() {
+    return appState.cloneLiveContainerInfoList();
+  }
+
+  @Override
+  public RoleInstance getLiveInstanceByContainerID(String containerId) throws
+      NoSuchNodeException {
+    return appState.getLiveInstanceByContainerID(containerId);
+  }
+
+  @Override
+  public List<RoleInstance> getLiveInstancesByContainerIDs(Collection<String> containerIDs) {
+    return appState.getLiveInstancesByContainerIDs(containerIDs);
+  }
+
+  @Override
+  public ClusterDescription refreshClusterStatus() {
+    return appState.refreshClusterStatus();
+  }
+
+  @Override
+  public List<RoleStatus> cloneRoleStatusList() {
+    return appState.cloneRoleStatusList();
+  }
+
+  @Override
+  public ApplicationLivenessInformation getApplicationLivenessInformation() {
+    return appState.getApplicationLivenessInformation();
+  }
+
+  @Override
+  public Map<String, Integer> getLiveStatistics() {
+    return appState.getLiveStatistics();
+  }
+
+  @Override
+  public Map<String, ComponentInformation> getComponentInfoSnapshot() {
+    return appState.getComponentInfoSnapshot();
+  }
+
+  @Override
+  public Map<String, Map<String, ClusterNode>> getRoleClusterNodeMapping() {
+    return appState.createRoleToClusterNodeMap();
+  }
+
+  @Override
+  public List<RoleInstance> enumLiveInstancesInRole(String role) {
+    List<RoleInstance> nodes = new ArrayList<>();
+    Collection<RoleInstance> allRoleInstances = cloneLiveContainerInfoList();
+    for (RoleInstance node : allRoleInstances) {
+      if (role.isEmpty() || role.equals(node.role)) {
+        nodes.add(node);
+      }
+    }
+    return nodes;
+  }
+
+  @Override
+  public List<RoleInstance> lookupRoleContainers(String component) {
+    RoleStatus roleStatus = lookupRoleStatus(component);
+    List<RoleInstance> ownedContainerList = cloneOwnedContainerList();
+    List<RoleInstance> matching = new ArrayList<>(ownedContainerList.size());
+    int roleId = roleStatus.getPriority();
+    for (RoleInstance instance : ownedContainerList) {
+      if (instance.roleId == roleId) {
+        matching.add(instance);
+      }
+    }
+    return matching;
+  }
+  
+  @Override
+  public ComponentInformation getComponentInformation(String component) {
+    RoleStatus roleStatus = lookupRoleStatus(component);
+    ComponentInformation info = roleStatus.serialize();
+    List<RoleInstance> containers = lookupRoleContainers(component);
+    info.containers = new ArrayList<>(containers.size());
+    for (RoleInstance container : containers) {
+      info.containers.add(container.id);
+    }
+    return info;
+  }
+
+  @Override
+  public Map<String, NodeInformation> getNodeInformationSnapshot() {
+    return appState.getRoleHistory()
+      .getNodeInformationSnapshot(appState.buildNamingMap());
+  }
+
+  @Override
+  public NodeInformation getNodeInformation(String hostname) {
+    return appState.getRoleHistory()
+      .getNodeInformation(hostname, appState.buildNamingMap());
+  }
+
+  @Override
+  public RoleStatistics getRoleStatistics() {
+    return appState.getRoleStatistics();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
new file mode 100644
index 0000000..4e8a4d7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistory.java
@@ -0,0 +1,1101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.slider.api.types.NodeInformation;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.providers.ProviderRole;
+import org.apache.slider.server.appmaster.management.BoolMetric;
+import org.apache.slider.server.appmaster.management.MetricsAndMonitoring;
+import org.apache.slider.server.appmaster.management.Timestamp;
+import org.apache.slider.server.appmaster.operations.AbstractRMOperation;
+import org.apache.slider.server.avro.LoadedRoleHistory;
+import org.apache.slider.server.avro.NodeEntryRecord;
+import org.apache.slider.server.avro.RoleHistoryHeader;
+import org.apache.slider.server.avro.RoleHistoryWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The Role History.
+ * <p>
+ * Synchronization policy: all public operations are synchronized.
+ * Protected methods are in place for testing -no guarantees are made.
+ * <p>
+ * Inner classes have no synchronization guarantees; they should be manipulated 
+ * in these classes and not externally.
+ * <p>
+ * Note that as well as some methods marked visible for testing, there
+ * is the option for the time generator method, {@link #now()} to
+ * be overridden so that a repeatable time series can be used.
+ * 
+ */
+public class RoleHistory {
+  protected static final Logger log =
+    LoggerFactory.getLogger(RoleHistory.class);
+  private final List<ProviderRole> providerRoles;
+  /** the roles in here are shared with App State */
+  private final Map<Integer, RoleStatus> roleStatusMap = new HashMap<>();
+  private final AbstractClusterServices recordFactory;
+
+  private long startTime;
+
+  /** Time when saved */
+  private final Timestamp saveTime = new Timestamp(0);
+
+  /** If the history was loaded, the time at which the history was saved.
+   * That is: the time the data was valid */
+  private final Timestamp thawedDataTime = new Timestamp(0);
+  
+  private NodeMap nodemap;
+  private int roleSize;
+  private final BoolMetric dirty = new BoolMetric(false);
+  private FileSystem filesystem;
+  private Path historyPath;
+  private RoleHistoryWriter historyWriter = new RoleHistoryWriter();
+
+  /**
+   * When were the nodes updated in a {@link #onNodesUpdated(List)} call?
+   * If zero: never.
+   */
+  private final Timestamp nodesUpdatedTime = new Timestamp(0);
+  private final BoolMetric nodeUpdateReceived = new BoolMetric(false);
+
+  private OutstandingRequestTracker outstandingRequests =
+    new OutstandingRequestTracker();
+
+  /**
+   * For each role, lists nodes that are available for data-local allocation,
+   * ordered by more recently released - to accelerate node selection.
+   * That is, they are "recently used nodes"
+   */
+  private Map<Integer, LinkedList<NodeInstance>> recentNodes;
+
+  /**
+   * Instantiate
+   * @param roles initial role list
+   * @param recordFactory yarn record factory
+   * @throws BadConfigException
+   */
+  public RoleHistory(Collection<RoleStatus> roles, AbstractClusterServices recordFactory) throws BadConfigException {
+    this.recordFactory = recordFactory;
+    roleSize = roles.size();
+    providerRoles = new ArrayList<>(roleSize);
+    for (RoleStatus role : roles) {
+      addNewRole(role);
+    }
+    reset();
+  }
+
+  /**
+   * Reset the variables -this does not adjust the fixed attributes
+   * of the history, but the nodemap and failed node map are cleared.
+   */
+  protected synchronized void reset() throws BadConfigException {
+
+    nodemap = new NodeMap(roleSize);
+    resetAvailableNodeLists();
+    outstandingRequests = new OutstandingRequestTracker();
+  }
+
+  /**
+   * Register all metrics with the metrics infra
+   * @param metrics metrics
+   */
+  public void register(MetricsAndMonitoring metrics) {
+    metrics.register(RoleHistory.class, dirty, "dirty");
+    metrics.register(RoleHistory.class, nodesUpdatedTime, "nodes-updated.time");
+    metrics.register(RoleHistory.class, nodeUpdateReceived, "nodes-updated.flag");
+    metrics.register(RoleHistory.class, thawedDataTime, "thawed.time");
+    metrics.register(RoleHistory.class, saveTime, "saved.time");
+  }
+
+  /**
+   * safety check: make sure the role is unique amongst
+   * the role stats...which is extended with the new role
+   * @param roleStatus role
+   * @throws ArrayIndexOutOfBoundsException
+   * @throws BadConfigException
+   */
+  protected void putRole(RoleStatus roleStatus) throws BadConfigException {
+    int index = roleStatus.getKey();
+    if (index < 0) {
+      throw new BadConfigException("Provider " + roleStatus + " id is out of range");
+    }
+    if (roleStatusMap.get(index) != null) {
+      throw new BadConfigException(
+        roleStatus.toString() + " id duplicates that of " +
+            roleStatusMap.get(index));
+    }
+    roleStatusMap.put(index, roleStatus);
+  }
+
+  /**
+   * Add a new role
+   * @param roleStatus new role
+   */
+  public void addNewRole(RoleStatus roleStatus) throws BadConfigException {
+    log.debug("Validating/adding new role to role history: {} ", roleStatus);
+    putRole(roleStatus);
+    this.providerRoles.add(roleStatus.getProviderRole());
+  }
+
+  /**
+   * Lookup a role by ID
+   * @param roleId role Id
+   * @return role or null if not found
+   */
+  public ProviderRole lookupRole(int roleId) {
+    for (ProviderRole role : providerRoles) {
+      if (role.id == roleId) {
+        return role;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Clear the lists of available nodes
+   */
+  private synchronized void resetAvailableNodeLists() {
+    recentNodes = new ConcurrentHashMap<>(roleSize);
+  }
+
+  /**
+   * Prepare the history for re-reading its state.
+   * <p>
+   * This intended for use by the RoleWriter logic.
+   * @throws BadConfigException if there is a problem rebuilding the state
+   */
+  private void prepareForReading(RoleHistoryHeader header)
+      throws BadConfigException {
+    reset();
+
+    int roleCountInSource = header.getRoles();
+    if (roleCountInSource != roleSize) {
+      log.warn("Number of roles in source {}"
+              +" does not match the expected number of {}",
+          roleCountInSource,
+          roleSize);
+    }
+    //record when the data was loaded
+    setThawedDataTime(header.getSaved());
+  }
+
+  /**
+   * rebuild the placement history from the loaded role history
+   * @param loadedRoleHistory loaded history
+   * @return the number of entries discarded
+   * @throws BadConfigException if there is a problem rebuilding the state
+   */
+  @VisibleForTesting
+  public synchronized int rebuild(LoadedRoleHistory loadedRoleHistory) throws BadConfigException {
+    RoleHistoryHeader header = loadedRoleHistory.getHeader();
+    prepareForReading(header);
+    int discarded = 0;
+    Long saved = header.getSaved();
+    for (NodeEntryRecord nodeEntryRecord : loadedRoleHistory.records) {
+      Integer roleId = nodeEntryRecord.getRole();
+      NodeEntry nodeEntry = new NodeEntry(roleId);
+      nodeEntry.setLastUsed(nodeEntryRecord.getLastUsed());
+      if (nodeEntryRecord.getActive()) {
+        //if active at the time of save, make the last used time the save time
+        nodeEntry.setLastUsed(saved);
+      }
+      String hostname = SliderUtils.sequenceToString(nodeEntryRecord.getHost());
+      ProviderRole providerRole = lookupRole(roleId);
+      if (providerRole == null) {
+        // discarding entry
+        log.info("Discarding history entry with unknown role: {} on host {}",
+            roleId, hostname);
+        discarded ++;
+      } else {
+        NodeInstance instance = getOrCreateNodeInstance(hostname);
+        instance.set(roleId, nodeEntry);
+      }
+    }
+    return discarded;
+  }
+
+  public synchronized long getStartTime() {
+    return startTime;
+  }
+
+  public synchronized long getSaveTime() {
+    return saveTime.get();
+  }
+
+  public long getThawedDataTime() {
+    return thawedDataTime.get();
+  }
+
+  public void setThawedDataTime(long thawedDataTime) {
+    this.thawedDataTime.set(thawedDataTime);
+  }
+
+  public synchronized int getRoleSize() {
+    return roleSize;
+  }
+
+  /**
+   * Get the total size of the cluster -the number of NodeInstances
+   * @return a count
+   */
+  public synchronized int getClusterSize() {
+    return nodemap.size();
+  }
+
+  public synchronized boolean isDirty() {
+    return dirty.get();
+  }
+
+  public synchronized void setDirty(boolean dirty) {
+    this.dirty.set(dirty);
+  }
+
+  /**
+   * Tell the history that it has been saved; marks itself as clean
+   * @param timestamp timestamp -updates the savetime field
+   */
+  public synchronized void saved(long timestamp) {
+    setDirty(false);
+    saveTime.set(timestamp);
+  }
+
+  /**
+   * Get a clone of the nodemap.
+   * The instances inside are not cloned
+   * @return the map
+   */
+  public synchronized NodeMap cloneNodemap() {
+    return (NodeMap) nodemap.clone();
+  }
+
+  /**
+   * Get snapshot of the node map
+   * @return a snapshot of the current node state
+   * @param naming naming map of priority to enty name; entries must be unique.
+   * It's OK to be incomplete, for those the list falls back to numbers.
+   */
+  public synchronized Map<String, NodeInformation> getNodeInformationSnapshot(
+    Map<Integer, String> naming) {
+    Map<String, NodeInformation> result = new HashMap<>(nodemap.size());
+    for (Map.Entry<String, NodeInstance> entry : nodemap.entrySet()) {
+      result.put(entry.getKey(), entry.getValue().serialize(naming));
+    }
+    return result;
+  }
+
+  /**
+   * Get the information on a node
+   * @param hostname hostname
+   * @param naming naming map of priority to enty name; entries must be unique.
+   * It's OK to be incomplete, for those the list falls back to numbers.
+   * @return the information about that host, or null if there is none
+   */
+  public synchronized NodeInformation getNodeInformation(String hostname,
+    Map<Integer, String> naming) {
+    NodeInstance nodeInstance = nodemap.get(hostname);
+    return nodeInstance != null ? nodeInstance.serialize(naming) : null;
+  }
+
+  /**
+   * Get the node instance for the specific node -creating it if needed
+   * @param hostname node address
+   * @return the instance
+   */
+  public synchronized NodeInstance getOrCreateNodeInstance(String hostname) {
+    //convert to a string
+    return nodemap.getOrCreate(hostname);
+  }
+
+  /**
+   * Insert a list of nodes into the map; overwrite any with that name.
+   * This is a bulk operation for testing.
+   * Important: this does not update the available node lists, these
+   * must be rebuilt afterwards.
+   * @param nodes collection of nodes.
+   */
+  @VisibleForTesting
+  public synchronized void insert(Collection<NodeInstance> nodes) {
+    nodemap.insert(nodes);
+  }
+
+  /**
+   * Get current time. overrideable for test subclasses
+   * @return current time in millis
+   */
+  protected long now() {
+    return System.currentTimeMillis();
+  }
+
+  /**
+   * Mark ourselves as dirty
+   */
+  public void touch() {
+    setDirty(true);
+    try {
+      saveHistoryIfDirty();
+    } catch (IOException e) {
+      log.warn("Failed to save history file ", e);
+    }
+  }
+
+  /**
+   * reset the failed recently counters
+   */
+  public synchronized void resetFailedRecently() {
+    log.info("Resetting failure history");
+    nodemap.resetFailedRecently();
+  }
+
+  /**
+   * Get the path used for history files
+   * @return the directory used for history files
+   */
+  public Path getHistoryPath() {
+    return historyPath;
+  }
+
+  /**
+   * Save the history to its location using the timestamp as part of
+   * the filename. The saveTime and dirty fields are updated
+   * @param time timestamp timestamp to use as the save time
+   * @return the path saved to
+   * @throws IOException IO problems
+   */
+  @VisibleForTesting
+  public synchronized Path saveHistory(long time) throws IOException {
+    Path filename = historyWriter.createHistoryFilename(historyPath, time);
+    historyWriter.write(filesystem, filename, true, this, time);
+    saved(time);
+    return filename;
+  }
+
+  /**
+   * Save the history with the current timestamp if it is dirty;
+   * return the path saved to if this is the case
+   * @return the path or null if the history was not saved
+   * @throws IOException failed to save for some reason
+   */
+  public synchronized Path saveHistoryIfDirty() throws IOException {
+    if (isDirty()) {
+      return saveHistory(now());
+    } else {
+      return null;
+    }
+  } 
+
+  /**
+   * Start up
+   * @param fs filesystem 
+   * @param historyDir path in FS for history
+   * @return true if the history was thawed
+   */
+  public boolean onStart(FileSystem fs, Path historyDir) throws BadConfigException {
+    assert filesystem == null;
+    filesystem = fs;
+    historyPath = historyDir;
+    startTime = now();
+    //assume the history is being thawed; this will downgrade as appropriate
+    return onThaw();
+    }
+  
+  /**
+   * Handler for bootstrap event: there was no history to thaw
+   */
+  public void onBootstrap() {
+    log.debug("Role history bootstrapped");
+  }
+
+  /**
+   * Handle the start process <i>after the history has been rebuilt</i>,
+   * and after any gc/purge
+   */
+  public synchronized boolean onThaw() throws BadConfigException {
+    assert filesystem != null;
+    assert historyPath != null;
+    boolean thawSuccessful = false;
+    //load in files from data dir
+
+    LoadedRoleHistory loadedRoleHistory = null;
+    try {
+      loadedRoleHistory = historyWriter.loadFromHistoryDir(filesystem, historyPath);
+    } catch (IOException e) {
+      log.warn("Exception trying to load history from {}", historyPath, e);
+    }
+    if (loadedRoleHistory != null) {
+      rebuild(loadedRoleHistory);
+      thawSuccessful = true;
+      Path loadPath = loadedRoleHistory.getPath();
+      log.debug("loaded history from {}", loadPath);
+      // delete any old entries
+      try {
+        int count = historyWriter.purgeOlderHistoryEntries(filesystem, loadPath);
+        log.debug("Deleted {} old history entries", count);
+      } catch (IOException e) {
+        log.info("Ignoring exception raised while trying to delete old entries",
+                 e);
+      }
+
+      //start is then completed
+      buildRecentNodeLists();
+    } else {
+      //fallback to bootstrap procedure
+      onBootstrap();
+    }
+    return thawSuccessful;
+  }
+
+
+  /**
+   * (After the start), rebuild the availability data structures
+   */
+  @VisibleForTesting
+  public synchronized void buildRecentNodeLists() {
+    resetAvailableNodeLists();
+    // build the list of available nodes
+    for (Map.Entry<String, NodeInstance> entry : nodemap.entrySet()) {
+      NodeInstance ni = entry.getValue();
+      for (int i = 0; i < roleSize; i++) {
+        NodeEntry nodeEntry = ni.get(i);
+        if (nodeEntry != null && nodeEntry.isAvailable()) {
+          log.debug("Adding {} for role {}", ni, i);
+          listRecentNodesForRoleId(i).add(ni);
+        }
+      }
+    }
+    // sort the resulting arrays
+    for (int i = 0; i < roleSize; i++) {
+      sortRecentNodeList(i);
+    }
+  }
+
+  /**
+   * Get the nodes for an ID -may be null
+   * @param id role ID
+   * @return potentially null list
+   */
+  @VisibleForTesting
+  public List<NodeInstance> getRecentNodesForRoleId(int id) {
+    return recentNodes.get(id);
+  }
+
+  /**
+   * Get a possibly empty list of suggested nodes for a role.
+   * @param id role ID
+   * @return list
+   */
+  private LinkedList<NodeInstance> listRecentNodesForRoleId(int id) {
+    LinkedList<NodeInstance> instances = recentNodes.get(id);
+    if (instances == null) {
+      synchronized (this) {
+        // recheck in the synchronized block and recreate
+        if (recentNodes.get(id) == null) {
+          recentNodes.put(id, new LinkedList<NodeInstance>());
+        }
+        instances = recentNodes.get(id);
+      }
+    }
+    return instances;
+  }
+
+  /**
+   * Sort a the recent node list for a single role
+   * @param role role to sort
+   */
+  private void sortRecentNodeList(int role) {
+    List<NodeInstance> nodesForRoleId = getRecentNodesForRoleId(role);
+    if (nodesForRoleId != null) {
+      Collections.sort(nodesForRoleId, new NodeInstance.Preferred(role));
+    }
+  }
+
+  /**
+   * Find a node for use
+   * @param role role
+   * @return the instance, or null for none
+   */
+  @VisibleForTesting
+  public synchronized NodeInstance findRecentNodeForNewInstance(RoleStatus role) {
+    if (!role.isPlacementDesired()) {
+      // no data locality policy
+      return null;
+    }
+    int roleId = role.getKey();
+    boolean strictPlacement = role.isStrictPlacement();
+    NodeInstance nodeInstance = null;
+    // Get the list of possible targets.
+    // This is a live list: changes here are preserved
+    List<NodeInstance> targets = getRecentNodesForRoleId(roleId);
+    if (targets == null) {
+      // nothing to allocate on
+      return null;
+    }
+
+    int cnt = targets.size();
+    log.debug("There are {} node(s) to consider for {}", cnt, role.getName());
+    for (int i = 0; i < cnt  && nodeInstance == null; i++) {
+      NodeInstance candidate = targets.get(i);
+      if (candidate.getActiveRoleInstances(roleId) == 0) {
+        // no active instances: check failure statistics
+        if (strictPlacement
+            || (candidate.isOnline() && !candidate.exceedsFailureThreshold(role))) {
+          targets.remove(i);
+          // exit criteria for loop is now met
+          nodeInstance = candidate;
+        } else {
+          // too many failures for this node
+          log.info("Recent node failures is higher than threshold {}. Not requesting host {}",
+              role.getNodeFailureThreshold(), candidate.hostname);
+        }
+      }
+    }
+
+    if (nodeInstance == null) {
+      log.info("No node found for {}", role.getName());
+    }
+    return nodeInstance;
+  }
+
+  /**
+   * Find a node for use
+   * @param role role
+   * @return the instance, or null for none
+   */
+  @VisibleForTesting
+  public synchronized List<NodeInstance> findNodeForNewAAInstance(RoleStatus role) {
+    // all nodes that are live and can host the role; no attempt to exclude ones
+    // considered failing
+    return nodemap.findAllNodesForRole(role.getKey(), role.getLabelExpression());
+  }
+
+  /**
+   * Request an instance on a given node.
+   * An outstanding request is created & tracked, with the 
+   * relevant node entry for that role updated.
+   *<p>
+   * The role status entries will also be tracked
+   * <p>
+   * Returns the request that is now being tracked.
+   * If the node instance is not null, it's details about the role is incremented
+   *
+   * @param node node to target or null for "any"
+   * @param role role to request
+   * @return the request
+   */
+  public synchronized OutstandingRequest requestInstanceOnNode(
+      NodeInstance node, RoleStatus role, Resource resource) {
+    OutstandingRequest outstanding = outstandingRequests.newRequest(node, role.getKey());
+    outstanding.buildContainerRequest(resource, role, now());
+    return outstanding;
+  }
+
+  /**
+   * Find a node for a role and request an instance on that (or a location-less
+   * instance)
+   * @param role role status
+   * @return a request ready to go, or null if this is an AA request and no
+   * location can be found.
+   */
+  public synchronized OutstandingRequest requestContainerForRole(RoleStatus role) {
+
+    if (role.isAntiAffinePlacement()) {
+      return requestContainerForAARole(role);
+    } else {
+      Resource resource = recordFactory.newResource();
+      role.copyResourceRequirements(resource);
+      NodeInstance node = findRecentNodeForNewInstance(role);
+      return requestInstanceOnNode(node, role, resource);
+    }
+  }
+
+  /**
+   * Find a node for an AA role and request an instance on that (or a location-less
+   * instance)
+   * @param role role status
+   * @return a request ready to go, or null if no location can be found.
+   */
+  public synchronized OutstandingRequest requestContainerForAARole(RoleStatus role) {
+    List<NodeInstance> nodes = findNodeForNewAAInstance(role);
+    if (!nodes.isEmpty()) {
+      OutstandingRequest outstanding = outstandingRequests.newAARequest(
+          role.getKey(), nodes, role.getLabelExpression());
+      Resource resource = recordFactory.newResource();
+      role.copyResourceRequirements(resource);
+      outstanding.buildContainerRequest(resource, role, now());
+      return outstanding;
+    } else {
+      log.warn("No suitable location for {}", role.getName());
+      return null;
+    }
+  }
+  /**
+   * Get the list of active nodes ... walks the node map so
+   * is {@code O(nodes)}
+   * @param role role index
+   * @return a possibly empty list of nodes with an instance of that node
+   */
+  public synchronized List<NodeInstance> listActiveNodes(int role) {
+    return nodemap.listActiveNodes(role);
+  }
+
+  /**
+   * Get the node entry of a container
+   * @param container container to look up
+   * @return the entry
+   * @throws RuntimeException if the container has no hostname
+   */
+  public NodeEntry getOrCreateNodeEntry(Container container) {
+    return getOrCreateNodeInstance(container).getOrCreate(container);
+  }
+
+  /**
+   * Get the node instance of a container -always returns something
+   * @param container container to look up
+   * @return a (possibly new) node instance
+   * @throws RuntimeException if the container has no hostname
+   */
+  public synchronized NodeInstance getOrCreateNodeInstance(Container container) {
+    return nodemap.getOrCreate(RoleHistoryUtils.hostnameOf(container));
+  }
+
+  /**
+   * Get the node instance of a host if defined
+   * @param hostname hostname to look up
+   * @return a node instance or null
+   * @throws RuntimeException if the container has no hostname
+   */
+  public synchronized NodeInstance getExistingNodeInstance(String hostname) {
+    return nodemap.get(hostname);
+  }
+
+  /**
+   * Get the node instance of a container <i>if there's an entry in the history</i>
+   * @param container container to look up
+   * @return a node instance or null
+   * @throws RuntimeException if the container has no hostname
+   */
+  public synchronized NodeInstance getExistingNodeInstance(Container container) {
+    return nodemap.get(RoleHistoryUtils.hostnameOf(container));
+  }
+
+  /**
+   * Perform any pre-allocation operations on the list of allocated containers
+   * based on knowledge of system state. 
+   * Currently this places requested hosts ahead of unrequested ones.
+   * @param allocatedContainers list of allocated containers
+   * @return list of containers potentially reordered
+   */
+  public synchronized List<Container> prepareAllocationList(List<Container> allocatedContainers) {
+
+    //partition into requested and unrequested
+    List<Container> requested =
+      new ArrayList<>(allocatedContainers.size());
+    List<Container> unrequested =
+      new ArrayList<>(allocatedContainers.size());
+    outstandingRequests.partitionRequests(this, allocatedContainers, requested, unrequested);
+
+    //give the unrequested ones lower priority
+    requested.addAll(unrequested);
+    return requested;
+  }
+
+  /**
+   * A container has been allocated on a node -update the data structures
+   * @param container container
+   * @param desiredCount desired #of instances
+   * @param actualCount current count of instances
+   * @return The allocation outcome
+   */
+  public synchronized ContainerAllocationResults onContainerAllocated(Container container,
+      long desiredCount,
+      long actualCount) {
+    int role = ContainerPriority.extractRole(container);
+
+    String hostname = RoleHistoryUtils.hostnameOf(container);
+    List<NodeInstance> nodeInstances = listRecentNodesForRoleId(role);
+    ContainerAllocationResults outcome =
+        outstandingRequests.onContainerAllocated(role, hostname, container);
+    if (desiredCount <= actualCount) {
+      // all outstanding requests have been satisfied
+      // clear all the lists, so returning nodes to the available set
+      List<NodeInstance> hosts = outstandingRequests.resetOutstandingRequests(role);
+      if (!hosts.isEmpty()) {
+        //add the list
+        log.info("Adding {} hosts for role {}", hosts.size(), role);
+        nodeInstances.addAll(hosts);
+        sortRecentNodeList(role);
+      }
+    }
+    return outcome;
+  }
+
+  /**
+   * A container has been assigned to a role instance on a node -update the data structures
+   * @param container container
+   */
+  public void onContainerAssigned(Container container) {
+    NodeInstance node = getOrCreateNodeInstance(container);
+    NodeEntry nodeEntry = node.getOrCreate(container);
+    nodeEntry.onStarting();
+    log.debug("Node {} has updated NodeEntry {}", node, nodeEntry);
+  }
+
+  /**
+   * Event: a container start has been submitted
+   * @param container container being started
+   * @param instance instance bound to the container
+   */
+  public void onContainerStartSubmitted(Container container,
+                                        RoleInstance instance) {
+    // no actions here
+  }
+
+  /**
+   * Container start event
+   * @param container container that just started
+   */
+  public void onContainerStarted(Container container) {
+    NodeEntry nodeEntry = getOrCreateNodeEntry(container);
+    nodeEntry.onStartCompleted();
+    touch();
+  }
+
+  /**
+   * A container failed to start: update the node entry state
+   * and return the container to the queue
+   * @param container container that failed
+   * @return true if the node was queued
+   */
+  public boolean onNodeManagerContainerStartFailed(Container container) {
+    return markContainerFinished(container, false, true, ContainerOutcome.Failed);
+  }
+
+  /**
+   * Does the RoleHistory have enough information about the YARN cluster
+   * to start placing AA requests? That is: has it the node map and
+   * any label information needed?
+   * @return true if the caller can start requesting AA nodes
+   */
+  public boolean canPlaceAANodes() {
+    return nodeUpdateReceived.get();
+  }
+
+  /**
+   * Get the last time the nodes were updated from YARN
+   * @return the update time or zero if never updated.
+   */
+  public long getNodesUpdatedTime() {
+    return nodesUpdatedTime.get();
+  }
+
+  /**
+   * Update failedNodes and nodemap based on the node state
+   *
+   * @param updatedNodes list of updated nodes
+   * @return true if a review should be triggered.
+   */
+  public synchronized boolean onNodesUpdated(List<NodeReport> updatedNodes) {
+    log.debug("Updating {} nodes", updatedNodes.size());
+    nodesUpdatedTime.set(now());
+    nodeUpdateReceived.set(true);
+    int printed = 0;
+    boolean triggerReview = false;
+    for (NodeReport updatedNode : updatedNodes) {
+      String hostname = updatedNode.getNodeId() == null
+          ? ""
+          : updatedNode.getNodeId().getHost();
+      NodeState nodeState = updatedNode.getNodeState();
+      if (hostname.isEmpty() || nodeState == null) {
+        log.warn("Ignoring incomplete update");
+        continue;
+      }
+      if (log.isDebugEnabled() && printed++ < 10) {
+        // log the first few, but avoid overloading the logs for a full cluster
+        // update
+        log.debug("Node \"{}\" is in state {}", hostname, nodeState);
+      }
+      // update the node; this also creates an instance if needed
+      boolean updated = nodemap.updateNode(hostname, updatedNode);
+      triggerReview |= updated;
+    }
+    return triggerReview;
+  }
+
+  /**
+   * A container release request was issued
+   * @param container container submitted
+   */
+  public void onContainerReleaseSubmitted(Container container) {
+    NodeEntry nodeEntry = getOrCreateNodeEntry(container);
+    nodeEntry.release();
+  }
+
+  /**
+   * App state notified of a container completed 
+   * @param container completed container
+   * @return true if the node was queued
+   */
+  public boolean onReleaseCompleted(Container container) {
+    return markContainerFinished(container, true, false, ContainerOutcome.Failed);
+  }
+
+  /**
+   * App state notified of a container completed -but as
+   * it wasn't being released it is marked as failed
+   *
+   * @param container completed container
+   * @param shortLived was the container short lived?
+   * @param outcome
+   * @return true if the node is considered available for work
+   */
+  public boolean onFailedContainer(Container container,
+      boolean shortLived,
+      ContainerOutcome outcome) {
+    return markContainerFinished(container, false, shortLived, outcome);
+  }
+
+  /**
+   * Mark a container finished; if it was released then that is treated
+   * differently. history is {@code touch()}-ed
+   *
+   *
+   * @param container completed container
+   * @param wasReleased was the container released?
+   * @param shortLived was the container short lived?
+   * @param outcome
+   * @return true if the node was queued
+   */
+  protected synchronized boolean markContainerFinished(Container container,
+      boolean wasReleased,
+      boolean shortLived,
+      ContainerOutcome outcome) {
+    NodeEntry nodeEntry = getOrCreateNodeEntry(container);
+    log.info("Finished container for node {}, released={}, shortlived={}",
+        nodeEntry.rolePriority, wasReleased, shortLived);
+    boolean available;
+    if (shortLived) {
+      nodeEntry.onStartFailed();
+      available = false;
+    } else {
+      available = nodeEntry.containerCompleted(wasReleased, outcome);
+      maybeQueueNodeForWork(container, nodeEntry, available);
+    }
+    touch();
+    return available;
+  }
+
+  /**
+   * If the node is marked as available; queue it for assignments.
+   * Unsynced: requires caller to be in a sync block.
+   * @param container completed container
+   * @param nodeEntry node
+   * @param available available flag
+   * @return true if the node was queued
+   */
+  private boolean maybeQueueNodeForWork(Container container,
+                                        NodeEntry nodeEntry,
+                                        boolean available) {
+    if (available) {
+      //node is free
+      nodeEntry.setLastUsed(now());
+      NodeInstance ni = getOrCreateNodeInstance(container);
+      int roleId = ContainerPriority.extractRole(container);
+      log.debug("Node {} is now available for role id {}", ni, roleId);
+      listRecentNodesForRoleId(roleId).addFirst(ni);
+    }
+    return available;
+  }
+
+  /**
+   * Print the history to the log. This is for testing and diagnostics 
+   */
+  public synchronized void dump() {
+    for (ProviderRole role : providerRoles) {
+      log.info(role.toString());
+      List<NodeInstance> instances = listRecentNodesForRoleId(role.id);
+      log.info("  available: " + instances.size()
+               + " " + SliderUtils.joinWithInnerSeparator(" ", instances));
+    }
+
+    log.info("Nodes in Cluster: {}", getClusterSize());
+    for (NodeInstance node : nodemap.values()) {
+      log.info(node.toFullString());
+    }
+  }
+
+  /**
+   * Build the mapping entry for persisting to the role history
+   * @return a mapping object
+   */
+  public synchronized Map<CharSequence, Integer> buildMappingForHistoryFile() {
+    Map<CharSequence, Integer> mapping = new HashMap<>(getRoleSize());
+    for (ProviderRole role : providerRoles) {
+      mapping.put(role.name, role.id);
+    }
+    return mapping;
+  }
+
+  /**
+   * Get a clone of the available list
+   * @param role role index
+   * @return a clone of the list
+   */
+  @VisibleForTesting
+  public List<NodeInstance> cloneRecentNodeList(int role) {
+    return new LinkedList<>(listRecentNodesForRoleId(role));
+  }
+
+  /**
+   * Get a snapshot of the outstanding placed request list
+   * @return a list of the requests outstanding at the time of requesting
+   */
+  @VisibleForTesting
+  public List<OutstandingRequest> listPlacedRequests() {
+    return outstandingRequests.listPlacedRequests();
+  }
+
+  /**
+   * Get a snapshot of the outstanding placed request list
+   * @return a list of the requests outstanding at the time of requesting
+   */
+  @VisibleForTesting
+  public List<OutstandingRequest> listOpenRequests() {
+    return outstandingRequests.listOpenRequests();
+  }
+
+  /**
+   * Escalate operation as triggered by external timer.
+   * @return a (usually empty) list of cancel/request operations.
+   */
+  public synchronized List<AbstractRMOperation> escalateOutstandingRequests() {
+    return outstandingRequests.escalateOutstandingRequests(now());
+  }
+  /**
+   * Escalate operation as triggered by external timer.
+   * @return a (usually empty) list of cancel/request operations.
+   */
+  public List<AbstractRMOperation> cancelOutstandingAARequests() {
+    return outstandingRequests.cancelOutstandingAARequests();
+  }
+
+  /**
+   * Cancel a number of outstanding requests for a role -that is, not
+   * actual containers, just requests for new ones.
+   * @param role role
+   * @param toCancel number to cancel
+   * @return a list of cancellable operations.
+   */
+  public List<AbstractRMOperation> cancelRequestsForRole(RoleStatus role, int toCancel) {
+    return role.isAntiAffinePlacement() ?
+        cancelRequestsForAARole(role, toCancel)
+        : cancelRequestsForSimpleRole(role, toCancel);
+  }
+
+  /**
+   * Build the list of requests to cancel from the outstanding list.
+   * @param role role
+   * @param toCancel number to cancel
+   * @return a list of cancellable operations.
+   */
+  private synchronized List<AbstractRMOperation> cancelRequestsForSimpleRole(RoleStatus role, int toCancel) {
+    Preconditions.checkArgument(toCancel > 0,
+        "trying to cancel invalid number of requests: " + toCancel);
+    List<AbstractRMOperation> results = new ArrayList<>(toCancel);
+    // first scan through the unplaced request list to find all of a role
+    int roleId = role.getKey();
+    List<OutstandingRequest> requests =
+        outstandingRequests.extractOpenRequestsForRole(roleId, toCancel);
+
+    // are there any left?
+    int remaining = toCancel - requests.size();
+    // ask for some placed nodes
+    requests.addAll(outstandingRequests.extractPlacedRequestsForRole(roleId, remaining));
+
+    // build cancellations
+    for (OutstandingRequest request : requests) {
+      results.add(request.createCancelOperation());
+    }
+    return results;
+  }
+
+  /**
+   * Build the list of requests to cancel for an AA role. This reduces the number
+   * of outstanding pending requests first, then cancels any active request,
+   * before finally asking for any placed containers
+   * @param role role
+   * @param toCancel number to cancel
+   * @return a list of cancellable operations.
+   */
+  private synchronized List<AbstractRMOperation> cancelRequestsForAARole(RoleStatus role, int toCancel) {
+    List<AbstractRMOperation> results = new ArrayList<>(toCancel);
+    int roleId = role.getKey();
+    List<OutstandingRequest> requests = new ArrayList<>(toCancel);
+    // there may be pending requests which can be cancelled here
+    long pending = role.getPendingAntiAffineRequests();
+    if (pending > 0) {
+      // there are some pending ones which can be cancelled first
+      long pendingToCancel = Math.min(pending, toCancel);
+      log.info("Cancelling {} pending AA allocations, leaving {}", toCancel,
+          pendingToCancel);
+      role.setPendingAntiAffineRequests(pending - pendingToCancel);
+      toCancel -= pendingToCancel;
+    }
+    if (toCancel > 0 && role.isAARequestOutstanding()) {
+      // not enough
+      log.info("Cancelling current AA request");
+      // find the single entry which may be running
+      requests = outstandingRequests.extractOpenRequestsForRole(roleId, toCancel);
+      role.cancelOutstandingAARequest();
+      toCancel--;
+    }
+
+    // ask for some excess nodes
+    if (toCancel > 0) {
+      requests.addAll(outstandingRequests.extractPlacedRequestsForRole(roleId, toCancel));
+    }
+
+    // build cancellations
+    for (OutstandingRequest request : requests) {
+      results.add(request.createCancelOperation());
+    }
+    return results;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistoryUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistoryUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistoryUtils.java
new file mode 100644
index 0000000..ea6197b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHistoryUtils.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.slider.common.tools.SliderUtils;
+
+public class RoleHistoryUtils {
+
+  public static String hostnameOf(Container container) {
+    NodeId nodeId = container.getNodeId();
+    if (nodeId== null) {
+      throw new RuntimeException("Container has no node ID: %s" +
+         SliderUtils.containerToString(container));
+    }
+    return nodeId.getHost();
+  }
+
+  /**
+   * Decrement a value but hold it at zero. Usually a sanity check
+   * on counters tracking outstanding operations
+   * @param val value
+   * @return decremented value
+   */
+  public static int decToFloor(int val) {
+    int v = val-1;
+    if (v < 0) {
+      v = 0;
+    }
+    return v;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHostnamePair.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHostnamePair.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHostnamePair.java
new file mode 100644
index 0000000..920887a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleHostnamePair.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+import java.util.Objects;
+
+public class RoleHostnamePair {
+
+  /**
+   * requested role
+   */
+  public final int roleId;
+
+  /**
+   * hostname -will be null if node==null
+   */
+  public final String hostname;
+
+  public RoleHostnamePair(int roleId, String hostname) {
+    this.roleId = roleId;
+    this.hostname = hostname;
+  }
+
+  public int getRoleId() {
+    return roleId;
+  }
+
+  public String getHostname() {
+    return hostname;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof RoleHostnamePair)) {
+      return false;
+    }
+    RoleHostnamePair that = (RoleHostnamePair) o;
+    return Objects.equals(roleId, that.roleId) &&
+        Objects.equals(hostname, that.hostname);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(roleId, hostname);
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "RoleHostnamePair{");
+    sb.append("roleId=").append(roleId);
+    sb.append(", hostname='").append(hostname).append('\'');
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleInstance.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleInstance.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleInstance.java
new file mode 100644
index 0000000..30cfec9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleInstance.java
@@ -0,0 +1,323 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.registry.client.binding.RegistryTypeUtils;
+import org.apache.hadoop.registry.client.types.Endpoint;
+import org.apache.hadoop.registry.client.types.ProtocolTypes;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.slider.api.ClusterNode;
+import org.apache.slider.api.proto.Messages;
+import org.apache.slider.api.types.ContainerInformation;
+import org.apache.slider.common.tools.SliderUtils;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Tracking information about a container
+ */
+public final class RoleInstance implements Cloneable {
+
+  public Container container;
+  /**
+   * Container ID
+   */
+  public final String id;
+  public long createTime;
+  public long startTime;
+  /**
+   * flag set when it is released, to know if it has
+   * already been targeted for termination
+   */
+  public boolean released;
+
+  /**
+   * Name of the role
+   */
+  public String role;
+  public String group;
+
+  /**
+   * Version of the app
+   */
+  public String appVersion;
+
+  /**
+   * Role Id; matches priority in resources.json
+   */
+  public int roleId;
+
+  /**
+   * state from StateValues
+   */
+  public int state;
+
+  /**
+   * Exit code: only valid if the state >= STOPPED
+   */
+  public int exitCode;
+
+  /**
+   * what was the command executed?
+   */
+  public String command;
+
+  /**
+   * Any diagnostics
+   */
+  public String diagnostics;
+
+  /**
+   * What is the tail output from the executed process (or [] if not started
+   * or the log cannot be picked up
+   */
+  public String[] output;
+
+  /**
+   * Any environment details
+   */
+  public String[] environment;
+  
+  public String ip;
+  public String hostname;
+  public String host;
+  public String hostURL;
+  public ContainerAllocationOutcome placement;
+
+
+  /**
+   * A list of registered endpoints.
+   */
+  private List<Endpoint> endpoints =
+      new ArrayList<>(2);
+
+  public RoleInstance(ContainerAssignment assignment) {
+    this(assignment.container);
+    placement = assignment.placement;
+  }
+  /**
+   * Create an instance to track an allocated container
+   * @param container a container which must be non null, and have a non-null Id field.
+   */
+  public RoleInstance(Container container) {
+    Preconditions.checkNotNull(container, "Null container");
+    Preconditions.checkState(container.getId() != null, 
+      "Null container ID");
+
+    this.container = container;
+    id = container.getId().toString();
+    if (container.getNodeId() != null) {
+      host = container.getNodeId().getHost();
+    }
+    if (container.getNodeHttpAddress() != null) {
+      hostURL = "http://" + container.getNodeHttpAddress();
+    }
+  }
+
+  public ContainerId getId() {
+    return container.getId();
+  }
+  
+  public NodeId getHost() {
+    return container.getNodeId();
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb =
+      new StringBuilder("RoleInstance{");
+    sb.append("role='").append(role).append('\'');
+    sb.append(", id='").append(id).append('\'');
+    sb.append(", container=").append(SliderUtils.containerToString(container));
+    sb.append(", createTime=").append(createTime);
+    sb.append(", startTime=").append(startTime);
+    sb.append(", released=").append(released);
+    sb.append(", roleId=").append(roleId);
+    sb.append(", host=").append(host);
+    sb.append(", hostURL=").append(hostURL);
+    sb.append(", state=").append(state);
+    sb.append(", placement=").append(placement);
+    sb.append(", exitCode=").append(exitCode);
+    sb.append(", command='").append(command).append('\'');
+    sb.append(", diagnostics='").append(diagnostics).append('\'');
+    sb.append(", output=").append(Arrays.toString(output));
+    sb.append(", environment=").append(Arrays.toString(environment));
+    sb.append('}');
+    return sb.toString();
+  }
+
+  public ContainerId getContainerId() {
+    return container != null ? container.getId() : null;
+  }
+
+  /**
+   * Generate the protobuf format of a request
+   * @return protobuf format. This excludes the Container info
+   */
+  public Messages.RoleInstanceState toProtobuf() {
+    Messages.RoleInstanceState.Builder builder =
+      Messages.RoleInstanceState.newBuilder();
+    if (container != null) {
+      builder.setName(container.getId().toString());
+    } else {
+      builder.setName("unallocated instance");
+    }
+    if (command != null) {
+      builder.setCommand(command);
+    }
+    if (environment != null) {
+      builder.addAllEnvironment(Arrays.asList(environment));
+    }
+    if (diagnostics != null) {
+      builder.setDiagnostics(diagnostics);
+    }
+    builder.setExitCode(exitCode);
+
+    if (output != null) {
+      builder.addAllOutput(Arrays.asList(output));
+    }
+    if (role != null) {
+      builder.setRole(role);
+    }
+    builder.setRoleId(roleId);
+    builder.setState(state);
+
+    builder.setReleased(released);
+    builder.setCreateTime(createTime);
+    builder.setStartTime(startTime);
+    builder.setHost(host);
+    builder.setHostURL(hostURL);
+    if (appVersion != null) {
+      builder.setAppVersion(appVersion);
+    }
+    return builder.build();
+  }
+
+  /**
+   * Build a serializable ClusterNode structure from this instance.
+   * This operation is unsynchronized.
+   * @return a serialized value.
+   */
+  public ClusterNode toClusterNode() {
+    ClusterNode node;
+    if (container != null) {
+      node = new ClusterNode(container.getId());
+    } else {
+      node = new ClusterNode();
+      node.name = "unallocated instance";
+    }
+    node.command = command;
+    node.createTime = createTime;
+    node.diagnostics = diagnostics;
+    if (environment != null) {
+      node.environment = Arrays.copyOf(environment, environment.length);
+    }
+    node.exitCode = exitCode;
+    node.ip = ip;
+    node.hostname = hostname;
+    node.host = host;
+    node.hostUrl = hostURL;
+    if (output != null) {
+      node.output = Arrays.copyOf(output, output.length);
+    }
+    node.released = released;
+    node.role = role;
+    node.roleId = roleId;
+    node.startTime = startTime ;
+    node.state = state;
+    
+    return node;
+  }
+  
+  /**
+   * Clone operation clones all the simple values but shares the 
+   * Container object into the cloned copy -same with the output,
+   * diagnostics and env arrays.
+   * @return a clone of the object
+   * @throws CloneNotSupportedException
+   */
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    RoleInstance cloned = (RoleInstance) super.clone();
+    // clone the endpoint list, but not the values
+    cloned.endpoints = new ArrayList<Endpoint>(this.endpoints);
+    return cloned;
+  }
+
+  /**
+   * Get the list of endpoints. 
+   * @return the endpoint list.
+   */
+  public List<Endpoint> getEndpoints() {
+    return endpoints;
+  }
+
+  /**
+   * Add an endpoint registration
+   * @param endpoint endpoint (non-null)
+   */
+  public void addEndpoint(Endpoint endpoint) {
+    Preconditions.checkArgument(endpoint != null);
+    endpoints.add(endpoint);
+  }
+
+  /**
+   * Register a port endpoint as an inet-addr formatted endpoint, using the
+   * hostname as the first part of the address
+   * @param port port port
+   * @param api  API API name
+   */
+  public void registerPortEndpoint(int port, String api) {
+    Endpoint epr =
+        RegistryTypeUtils.inetAddrEndpoint(api,
+            ProtocolTypes.PROTOCOL_TCP, host, port);
+    addEndpoint(epr);
+  }
+
+  /**
+   * Serialize. Some data structures (e.g output)
+   * may be shared
+   * @return a serialized form for marshalling as JSON
+   */
+  public ContainerInformation serialize() {
+    ContainerInformation info = new ContainerInformation();
+    info.containerId = id;
+    info.component = role;
+    info.appVersion = appVersion;
+    info.startTime = startTime;
+    info.createTime = createTime;
+    info.diagnostics = diagnostics;
+    info.state = state;
+    info.host = host;
+    info.hostURL = hostURL;
+    info.released = released ? Boolean.TRUE : null;
+    if (placement != null) {
+      info.placement = placement.toString();
+    }
+    if (output != null) {
+      info.output = output;
+    }
+    return info;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
new file mode 100644
index 0000000..0a3a3c9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/RoleStatus.java
@@ -0,0 +1,563 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricSet;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.slider.api.types.ComponentInformation;
+import org.apache.slider.api.types.RoleStatistics;
+import org.apache.slider.providers.PlacementPolicy;
+import org.apache.slider.providers.ProviderRole;
+import org.apache.slider.server.appmaster.management.BoolMetricPredicate;
+import org.apache.slider.server.appmaster.management.LongGauge;
+
+import java.io.Serializable;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Models the ongoing status of all nodes in an application.
+ *
+ * These structures are shared across the {@link AppState} and {@link RoleHistory} structures,
+ * and must be designed for synchronous access. Atomic counters are preferred to anything which
+ * requires synchronization. Where synchronized access is good is that it allows for
+ * the whole instance to be locked, for updating multiple entries.
+ */
+public final class RoleStatus implements Cloneable, MetricSet {
+
+  private final String name;
+  private final String group;
+
+  /**
+   * Role priority
+   */
+  private final int key;
+  private final ProviderRole providerRole;
+
+  private final LongGauge actual = new LongGauge();
+  private final LongGauge completed = new LongGauge();
+  private final LongGauge desired = new LongGauge();
+  private final LongGauge failed = new LongGauge();
+  private final LongGauge failedRecently = new LongGauge(0);
+  private final LongGauge limitsExceeded = new LongGauge(0);
+  private final LongGauge nodeFailed = new LongGauge(0);
+  /** Number of AA requests queued. */
+  private final LongGauge pendingAntiAffineRequests = new LongGauge(0);
+  private final LongGauge preempted = new LongGauge(0);
+  private final LongGauge releasing = new LongGauge();
+  private final LongGauge requested = new LongGauge();
+  private final LongGauge started = new LongGauge();
+  private final LongGauge startFailed = new LongGauge();
+  private final LongGauge totalRequested = new LongGauge();
+
+  /** resource requirements */
+  private Resource resourceRequirements;
+
+
+  /** any pending AA request */
+  private volatile OutstandingRequest outstandingAArequest = null;
+
+
+  private String failureMessage = "";
+
+  public RoleStatus(ProviderRole providerRole) {
+    this.providerRole = providerRole;
+    this.name = providerRole.name;
+    this.group = providerRole.group;
+    this.key = providerRole.id;
+  }
+
+  @Override
+  public Map<String, Metric> getMetrics() {
+    Map<String, Metric> metrics = new HashMap<>(15);
+    metrics.put("actual", actual);
+    metrics.put("completed", completed );
+    metrics.put("desired", desired);
+    metrics.put("failed", failed);
+    metrics.put("limitsExceeded", limitsExceeded);
+    metrics.put("nodeFailed", nodeFailed);
+    metrics.put("preempted", preempted);
+    metrics.put("pendingAntiAffineRequests", pendingAntiAffineRequests);
+    metrics.put("releasing", releasing);
+    metrics.put("requested", requested);
+    metrics.put("preempted", preempted);
+    metrics.put("releasing", releasing );
+    metrics.put("requested", requested);
+    metrics.put("started", started);
+    metrics.put("startFailed", startFailed);
+    metrics.put("totalRequested", totalRequested);
+
+    metrics.put("outstandingAArequest",
+      new BoolMetricPredicate(new BoolMetricPredicate.Eval() {
+        @Override
+        public boolean eval() {
+          return isAARequestOutstanding();
+        }
+      }));
+    return metrics;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public String getGroup() {
+    return group;
+  }
+
+  public int getKey() {
+    return key;
+  }
+
+  public int getPriority() {
+    return getKey();
+  }
+
+  /**
+   * Get the placement policy enum, from the values in
+   * {@link PlacementPolicy}
+   * @return the placement policy for this role
+   */
+  public int getPlacementPolicy() {
+    return providerRole.placementPolicy;
+  }
+
+  public long getPlacementTimeoutSeconds() {
+    return providerRole.placementTimeoutSeconds;
+  }
+  
+  /**
+   * The number of failures on a specific node that can be tolerated
+   * before selecting a different node for placement
+   * @return
+   */
+  public int getNodeFailureThreshold() {
+    return providerRole.nodeFailureThreshold;
+  }
+
+  public boolean isExcludeFromFlexing() {
+    return hasPlacementPolicy(PlacementPolicy.EXCLUDE_FROM_FLEXING);
+  }
+
+  public boolean isStrictPlacement() {
+    return hasPlacementPolicy(PlacementPolicy.STRICT);
+  }
+
+  public boolean isAntiAffinePlacement() {
+    return hasPlacementPolicy(PlacementPolicy.ANTI_AFFINITY_REQUIRED);
+  }
+
+  public boolean hasPlacementPolicy(int policy) {
+    return 0 != (getPlacementPolicy() & policy);
+  }
+
+  public boolean isPlacementDesired() {
+    return !hasPlacementPolicy(PlacementPolicy.ANYWHERE);
+  }
+
+  public long getDesired() {
+    return desired.get();
+  }
+
+  public void setDesired(long desired) {
+    this.desired.set(desired);
+  }
+
+  public long getActual() {
+    return actual.get();
+  }
+
+  public long incActual() {
+    return actual.incrementAndGet();
+  }
+
+  public long decActual() {
+    return actual.decToFloor(1);
+  }
+
+  /**
+   * Get the request count.
+   * @return a count of requested containers
+   */
+  public long getRequested() {
+    return requested.get();
+  }
+
+  public long incRequested() {
+    totalRequested.incrementAndGet();
+    return requested.incrementAndGet();
+  }
+
+  public void cancel(long count) {
+    requested.decToFloor(count);
+  }
+
+  public void decRequested() {
+    cancel(1);
+  }
+
+  public long getReleasing() {
+    return releasing.get();
+  }
+
+  public long incReleasing() {
+    return releasing.incrementAndGet();
+  }
+
+  public long decReleasing() {
+    return releasing.decToFloor(1);
+  }
+
+  public long getFailed() {
+    return failed.get();
+  }
+
+  public long getFailedRecently() {
+    return failedRecently.get();
+  }
+
+  /**
+   * Reset the recent failure
+   * @return the number of failures in the "recent" window
+   */
+  public long resetFailedRecently() {
+    return failedRecently.getAndSet(0);
+  }
+
+  public long getLimitsExceeded() {
+    return limitsExceeded.get();
+  }
+
+  public long incPendingAntiAffineRequests(long v) {
+    return pendingAntiAffineRequests.addAndGet(v);
+  }
+
+  /**
+   * Probe for an outstanding AA request being true
+   * @return true if there is an outstanding AA Request
+   */
+  public boolean isAARequestOutstanding() {
+    return outstandingAArequest != null;
+  }
+
+  /**
+   * expose the predicate {@link #isAARequestOutstanding()} as an integer,
+   * which is very convenient in tests
+   * @return 1 if there is an outstanding request; 0 if not
+   */
+  public int getOutstandingAARequestCount() {
+    return isAARequestOutstanding()? 1: 0;
+  }
+  /**
+   * Note that a role failed, text will
+   * be used in any diagnostics if an exception
+   * is later raised.
+   * @param startupFailure flag to indicate this was a startup event
+   * @param text text about the failure
+   * @param outcome outcome of the container
+   */
+  public synchronized void noteFailed(boolean startupFailure, String text,
+      ContainerOutcome outcome) {
+    if (text != null) {
+      failureMessage = text;
+    }
+    switch (outcome) {
+      case Preempted:
+        preempted.incrementAndGet();
+        break;
+
+      case Node_failure:
+        nodeFailed.incrementAndGet();
+        failed.incrementAndGet();
+        break;
+
+      case Failed_limits_exceeded: // exceeded memory or CPU; app/configuration related
+        limitsExceeded.incrementAndGet();
+        // fall through
+      case Failed: // application failure, possibly node related, possibly not
+      default: // anything else (future-proofing)
+        failed.incrementAndGet();
+        failedRecently.incrementAndGet();
+        //have a look to see if it short lived
+        if (startupFailure) {
+          incStartFailed();
+        }
+        break;
+    }
+  }
+
+  public long getStartFailed() {
+    return startFailed.get();
+  }
+
+  public synchronized void incStartFailed() {
+    startFailed.getAndIncrement();
+  }
+
+  public synchronized String getFailureMessage() {
+    return failureMessage;
+  }
+
+  public long getCompleted() {
+    return completed.get();
+  }
+
+  public synchronized void setCompleted(int completed) {
+    this.completed.set(completed);
+  }
+
+  public long incCompleted() {
+    return completed.incrementAndGet();
+  }
+  public long getStarted() {
+    return started.get();
+  }
+
+  public synchronized void incStarted() {
+    started.incrementAndGet();
+  }
+
+  public long getTotalRequested() {
+    return totalRequested.get();
+  }
+
+  public long getPreempted() {
+    return preempted.get();
+  }
+
+  public long getNodeFailed() {
+    return nodeFailed.get();
+  }
+
+  public long getPendingAntiAffineRequests() {
+    return pendingAntiAffineRequests.get();
+  }
+
+  public void setPendingAntiAffineRequests(long pendingAntiAffineRequests) {
+    this.pendingAntiAffineRequests.set(pendingAntiAffineRequests);
+  }
+
+  public long decPendingAntiAffineRequests() {
+    return pendingAntiAffineRequests.decToFloor(1);
+  }
+
+  public OutstandingRequest getOutstandingAArequest() {
+    return outstandingAArequest;
+  }
+
+  public void setOutstandingAArequest(OutstandingRequest outstandingAArequest) {
+    this.outstandingAArequest = outstandingAArequest;
+  }
+
+  /**
+   * Complete the outstanding AA request (there's no check for one in progress, caller
+   * expected to have done that).
+   */
+  public void completeOutstandingAARequest() {
+    setOutstandingAArequest(null);
+  }
+
+  /**
+   * Cancel any outstanding AA request. Harmless if the role is non-AA, or
+   * if there are no outstanding requests.
+   */
+  public void cancelOutstandingAARequest() {
+    if (outstandingAArequest != null) {
+      setOutstandingAArequest(null);
+      setPendingAntiAffineRequests(0);
+      decRequested();
+    }
+  }
+
+  /**
+   * Get the number of roles we are short of.
+   * nodes released are ignored.
+   * @return the positive or negative number of roles to add/release.
+   * 0 means "do nothing".
+   */
+  public long getDelta() {
+    long inuse = getActualAndRequested();
+    long delta = desired.get() - inuse;
+    if (delta < 0) {
+      //if we are releasing, remove the number that are already released.
+      delta += releasing.get();
+      //but never switch to a positive
+      delta = Math.min(delta, 0);
+    }
+    return delta;
+  }
+
+  /**
+   * Get count of actual and requested containers. This includes pending ones
+   * @return the size of the application when outstanding requests are included.
+   */
+  public long getActualAndRequested() {
+    return actual.get() + requested.get();
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder("RoleStatus{");
+    sb.append("name='").append(name).append('\'');
+    sb.append(", group=").append(group);
+    sb.append(", key=").append(key);
+    sb.append(", desired=").append(desired);
+    sb.append(", actual=").append(actual);
+    sb.append(", requested=").append(requested);
+    sb.append(", releasing=").append(releasing);
+    sb.append(", failed=").append(failed);
+    sb.append(", startFailed=").append(startFailed);
+    sb.append(", started=").append(started);
+    sb.append(", completed=").append(completed);
+    sb.append(", totalRequested=").append(totalRequested);
+    sb.append(", preempted=").append(preempted);
+    sb.append(", nodeFailed=").append(nodeFailed);
+    sb.append(", failedRecently=").append(failedRecently);
+    sb.append(", limitsExceeded=").append(limitsExceeded);
+    sb.append(", resourceRequirements=").append(resourceRequirements);
+    sb.append(", isAntiAffinePlacement=").append(isAntiAffinePlacement());
+    if (isAntiAffinePlacement()) {
+      sb.append(", pendingAntiAffineRequests=").append(pendingAntiAffineRequests);
+      sb.append(", outstandingAArequest=").append(outstandingAArequest);
+    }
+    sb.append(", failureMessage='").append(failureMessage).append('\'');
+    sb.append(", providerRole=").append(providerRole);
+    sb.append('}');
+    return sb.toString();
+  }
+
+  @Override
+  public synchronized  Object clone() throws CloneNotSupportedException {
+    return super.clone();
+  }
+
+  /**
+   * Get the provider role
+   * @return the provider role
+   */
+  public ProviderRole getProviderRole() {
+    return providerRole;
+  }
+
+  /**
+   * Build the statistics map from the current data
+   * @return a map for use in statistics reports
+   */
+  public Map<String, Integer> buildStatistics() {
+    ComponentInformation componentInformation = serialize();
+    return componentInformation.buildStatistics();
+  }
+
+  /**
+   * Produced a serialized form which can be served up as JSON
+   * @return a summary of the current role status.
+   */
+  public synchronized ComponentInformation serialize() {
+    ComponentInformation info = new ComponentInformation();
+    info.name = name;
+    info.priority = getPriority();
+    info.desired = desired.intValue();
+    info.actual = actual.intValue();
+    info.requested = requested.intValue();
+    info.releasing = releasing.intValue();
+    info.failed = failed.intValue();
+    info.startFailed = startFailed.intValue();
+    info.placementPolicy = getPlacementPolicy();
+    info.failureMessage = failureMessage;
+    info.totalRequested = totalRequested.intValue();
+    info.failedRecently = failedRecently.intValue();
+    info.nodeFailed = nodeFailed.intValue();
+    info.preempted = preempted.intValue();
+    info.pendingAntiAffineRequestCount = pendingAntiAffineRequests.intValue();
+    info.isAARequestOutstanding = isAARequestOutstanding();
+    return info;
+  }
+
+  /**
+   * Get the (possibly null) label expression for this role
+   * @return a string or null
+   */
+  public String getLabelExpression() {
+    return providerRole.labelExpression;
+  }
+
+  public Resource getResourceRequirements() {
+    return resourceRequirements;
+  }
+
+  public void setResourceRequirements(Resource resourceRequirements) {
+    this.resourceRequirements = resourceRequirements;
+  }
+
+  /**
+   * Compare two role status entries by name
+   */
+  public static class CompareByName implements Comparator<RoleStatus>,
+      Serializable {
+    @Override
+    public int compare(RoleStatus o1, RoleStatus o2) {
+      return o1.getName().compareTo(o2.getName());
+    }
+  }
+  
+  /**
+   * Compare two role status entries by key
+   */
+  public static class CompareByKey implements Comparator<RoleStatus>,
+      Serializable {
+    @Override
+    public int compare(RoleStatus o1, RoleStatus o2) {
+      return (o1.getKey() < o2.getKey() ? -1 : (o1.getKey() == o2.getKey() ? 0 : 1));
+    }
+  }
+
+  /**
+   * Given a resource, set its requirements to those this role needs
+   * @param resource resource to configure
+   * @return the resource
+   */
+  public Resource copyResourceRequirements(Resource resource) {
+    Preconditions.checkNotNull(resourceRequirements,
+        "Role resource requirements have not been set");
+    resource.setMemory(resourceRequirements.getMemory());
+    resource.setVirtualCores(resourceRequirements.getVirtualCores());
+    return resource;
+  }
+
+  public synchronized RoleStatistics getStatistics() {
+    RoleStatistics stats = new RoleStatistics();
+    stats.activeAA = getOutstandingAARequestCount();
+    stats.actual = actual.get();
+    stats.desired = desired.get();
+    stats.failed = failed.get();
+    stats.limitsExceeded = limitsExceeded.get();
+    stats.nodeFailed = nodeFailed.get();
+    stats.preempted = preempted.get();
+    stats.releasing = releasing.get();
+    stats.requested = requested.get();
+    stats.started = started.get();
+    stats.startFailed = startFailed.get();
+    stats.totalRequested = totalRequested.get();
+    return stats;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/SimpleReleaseSelector.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/SimpleReleaseSelector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/SimpleReleaseSelector.java
new file mode 100644
index 0000000..b848096
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/SimpleReleaseSelector.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+import java.util.List;
+
+/**
+ * Simplest release selector simply returns the list
+ */
+public class SimpleReleaseSelector implements ContainerReleaseSelector {
+
+  @Override
+  public List<RoleInstance> sortCandidates(int roleId,
+      List<RoleInstance> candidates) {
+    return candidates;
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[04/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/HttpProbe.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/HttpProbe.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/HttpProbe.java
new file mode 100644
index 0000000..9c14ca7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/HttpProbe.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.servicemonitor;
+
+import org.apache.hadoop.conf.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URL;
+
+public class HttpProbe extends Probe {
+  protected static final Logger log = LoggerFactory.getLogger(HttpProbe.class);
+
+  private final URL url;
+  private final int timeout;
+  private final int min, max;
+
+
+  public HttpProbe(URL url, int timeout, int min, int max, Configuration conf) throws IOException {
+    super("Http probe of " + url + " [" + min + "-" + max + "]", conf);
+    this.url = url;
+    this.timeout = timeout;
+    this.min = min;
+    this.max = max;
+  }
+
+  public static HttpURLConnection getConnection(URL url, int timeout) throws IOException {
+    HttpURLConnection connection = (HttpURLConnection) url.openConnection();
+    connection.setInstanceFollowRedirects(true);
+    connection.setConnectTimeout(timeout);
+    return connection;
+  }
+  
+  @Override
+  public ProbeStatus ping(boolean livePing) {
+    ProbeStatus status = new ProbeStatus();
+    HttpURLConnection connection = null;
+    try {
+      if (log.isDebugEnabled()) {
+        // LOG.debug("Fetching " + url + " with timeout " + timeout);
+      }
+      connection = getConnection(url, this.timeout);
+      int rc = connection.getResponseCode();
+      if (rc < min || rc > max) {
+        String error = "Probe " + url + " error code: " + rc;
+        log.info(error);
+        status.fail(this,
+                    new IOException(error));
+      } else {
+        status.succeed(this);
+      }
+    } catch (IOException e) {
+      String error = "Probe " + url + " failed: " + e;
+      log.info(error, e);
+      status.fail(this,
+                  new IOException(error, e));
+    } finally {
+      if (connection != null) {
+        connection.disconnect();
+      }
+    }
+    return status;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/LogEntryBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/LogEntryBuilder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/LogEntryBuilder.java
new file mode 100644
index 0000000..a1ad44f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/LogEntryBuilder.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.servicemonitor;
+
+/**
+ * Build up log entries for ease of splunk
+ */
+public class LogEntryBuilder {
+
+  private final StringBuilder builder = new StringBuilder();
+
+  public LogEntryBuilder() {
+  }
+
+  public LogEntryBuilder(String text) {
+    elt(text);
+  }
+
+
+  public LogEntryBuilder(String name, Object value) {
+    entry(name, value);
+  }
+
+  public LogEntryBuilder elt(String text) {
+    addComma();
+    builder.append(text);
+    return this;
+  }
+
+  public LogEntryBuilder elt(String name, Object value) {
+    addComma();
+    entry(name, value);
+    return this;
+  }
+
+  private void addComma() {
+    if (!isEmpty()) {
+      builder.append(", ");
+    }
+  }
+
+  private void entry(String name, Object value) {
+    builder.append(name).append('=');
+    if (value != null) {
+      builder.append('"').append(value.toString()).append('"');
+    } else {
+      builder.append("null");
+    }
+  }
+
+  @Override
+  public String toString() {
+    return builder.toString();
+  }
+
+  private boolean isEmpty() {
+    return builder.length() == 0;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/MonitorKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/MonitorKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/MonitorKeys.java
new file mode 100644
index 0000000..f7bdd4a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/MonitorKeys.java
@@ -0,0 +1,279 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.servicemonitor;
+
+/**
+ * Config keys for monitoring
+ */
+public interface MonitorKeys {
+
+  /**
+   * Prefix of all other configuration options: {@value}
+   */
+  String MONITOR_KEY_PREFIX = "service.monitor.";
+
+
+  /**
+   * Classname of the reporter Key: {@value}
+   */
+  String MONITOR_REPORTER =
+    MONITOR_KEY_PREFIX + "report.classname";
+
+  /**
+   * Interval in milliseconds between reporting health status to the reporter
+   * Key: {@value}
+   */
+  String MONITOR_REPORT_INTERVAL =
+    MONITOR_KEY_PREFIX + "report.interval";
+
+  /**
+   * Time in millis between the last probing cycle ending and the new one
+   * beginning. Key: {@value}
+   */
+  String MONITOR_PROBE_INTERVAL =
+    MONITOR_KEY_PREFIX + "probe.interval";
+
+  /**
+   * How long in milliseconds does the probing loop have to be blocked before
+   * that is considered a liveness failure Key: {@value}
+   */
+  String MONITOR_PROBE_TIMEOUT =
+    MONITOR_KEY_PREFIX + "probe.timeout";
+
+  /**
+   * How long in milliseconds does the probing loop have to be blocked before
+   * that is considered a liveness failure Key: {@value}
+   */
+  String MONITOR_BOOTSTRAP_TIMEOUT =
+    MONITOR_KEY_PREFIX + "bootstrap.timeout";
+
+
+  /**
+   * does the monitor depend on DFS being live
+   */
+  String MONITOR_DEPENDENCY_DFSLIVE =
+    MONITOR_KEY_PREFIX + "dependency.dfslive";
+
+
+  /**
+   * default timeout for the entire bootstrap phase {@value}
+   */
+
+  int BOOTSTRAP_TIMEOUT_DEFAULT = 60000;
+
+
+  /**
+   * Default value if the key is not in the config file: {@value}
+   */
+  int REPORT_INTERVAL_DEFAULT = 10000;
+  /**
+   * Default value if the key is not in the config file: {@value}
+   */
+  int PROBE_INTERVAL_DEFAULT = 10000;
+  /**
+   * Default value if the key is not in the config file: {@value}
+   */
+  int PROBE_TIMEOUT_DEFAULT = 60000;
+
+  /**
+   * Port probe enabled/disabled flag Key: {@value}
+   */
+  String PORT_PROBE_ENABLED =
+    MONITOR_KEY_PREFIX + "portprobe.enabled";
+
+
+  /**
+   * Port probing key : port to attempt to create a TCP connection to {@value}
+   */
+  String PORT_PROBE_PORT =
+    MONITOR_KEY_PREFIX + "portprobe.port";
+
+  /**
+   * Port probing key : port to attempt to create a TCP connection to {@value}
+   */
+  String PORT_PROBE_HOST =
+    MONITOR_KEY_PREFIX + "portprobe.host";
+
+
+  /**
+   * Port probing key : timeout of the connection attempt {@value}
+   */
+  String PORT_PROBE_CONNECT_TIMEOUT =
+    MONITOR_KEY_PREFIX + "portprobe.connect.timeout";
+
+  /**
+   * Port probing key : bootstrap timeout -how long in milliseconds should the
+   * port probing take to connect before the failure to connect is considered a
+   * liveness failure. That is: how long should the IPC port take to come up?
+   * {@value}
+   */
+  String PORT_PROBE_BOOTSTRAP_TIMEOUT =
+    MONITOR_KEY_PREFIX + "portprobe.bootstrap.timeout";
+
+
+  /**
+   * default timeout for port probes {@value}
+   */
+  int PORT_PROBE_BOOTSTRAP_TIMEOUT_DEFAULT = 60000;
+
+  /**
+   * default value for port probe connection attempts {@value}
+   */
+
+  int PORT_PROBE_CONNECT_TIMEOUT_DEFAULT = 1000;
+
+
+  /**
+   * default port for probes {@value}
+   */
+  int DEFAULT_PROBE_PORT = 8020;
+
+
+  /**
+   * default host for probes {@value}
+   */
+  String DEFAULT_PROBE_HOST = "localhost";
+
+
+  /**
+   * Probe enabled/disabled flag Key: {@value}
+   */
+  String LS_PROBE_ENABLED =
+    MONITOR_KEY_PREFIX + "lsprobe.enabled";
+
+  /**
+   * Probe path for LS operation Key: {@value}
+   */
+  String LS_PROBE_PATH =
+    MONITOR_KEY_PREFIX + "lsprobe.path";
+
+  /**
+   * Default path for LS operation Key: {@value}
+   */
+  String LS_PROBE_DEFAULT = "/";
+
+  /**
+   * Port probing key : bootstrap timeout -how long in milliseconds should the
+   * port probing take to connect before the failure to connect is considered a
+   * liveness failure. That is: how long should the IPC port take to come up?
+   * {@value}
+   */
+  String LS_PROBE_BOOTSTRAP_TIMEOUT =
+    MONITOR_KEY_PREFIX + "lsprobe.bootstrap.timeout";
+
+
+  /**
+   * default timeout for port probes {@value}
+   */
+
+  int LS_PROBE_BOOTSTRAP_TIMEOUT_DEFAULT = PORT_PROBE_BOOTSTRAP_TIMEOUT_DEFAULT;
+
+
+  /**
+   * Probe enabled/disabled flag Key: {@value}
+   */
+  String WEB_PROBE_ENABLED =
+    MONITOR_KEY_PREFIX + "webprobe.enabled";
+
+  /**
+   * Probe URL Key: {@value}
+   */
+  String WEB_PROBE_URL =
+    MONITOR_KEY_PREFIX + "webprobe.url";
+
+  /**
+   * Default path for web probe Key: {@value}
+   */
+  String WEB_PROBE_DEFAULT_URL = "http://localhost:50070/";
+
+  /**
+   * min error code Key: {@value}
+   */
+  String WEB_PROBE_MIN =
+    MONITOR_KEY_PREFIX + "webprobe.min";
+  /**
+   * min error code Key: {@value}
+   */
+  String WEB_PROBE_MAX =
+    MONITOR_KEY_PREFIX + "webprobe.max";
+
+
+  /**
+   * Port probing key : timeout of the connection attempt {@value}
+   */
+  String WEB_PROBE_CONNECT_TIMEOUT =
+    MONITOR_KEY_PREFIX + "webprobe.connect.timeout";
+
+  /**
+   * Default HTTP response code expected from the far end for
+   * the endpoint to be considered live.
+   */
+  int WEB_PROBE_DEFAULT_CODE = 200;
+
+  /**
+   * Port probing key : bootstrap timeout -how long in milliseconds should the
+   * port probing take to connect before the failure to connect is considered a
+   * liveness failure. That is: how long should the IPC port take to come up?
+   * {@value}
+   */
+  String WEB_PROBE_BOOTSTRAP_TIMEOUT =
+    MONITOR_KEY_PREFIX + "webprobe.bootstrap.timeout";
+
+
+  /**
+   * default timeout for port probes {@value}
+   */
+
+  int WEB_PROBE_BOOTSTRAP_TIMEOUT_DEFAULT = PORT_PROBE_BOOTSTRAP_TIMEOUT_DEFAULT;
+
+  /**
+   * Probe enabled/disabled flag Key: {@value}
+   */
+  String JT_PROBE_ENABLED =
+    MONITOR_KEY_PREFIX + "jtprobe.enabled";
+
+  /**
+   * Port probing key : bootstrap timeout -how long in milliseconds should the
+   * port probing take to connect before the failure to connect is considered a
+   * liveness failure. That is: how long should the IPC port take to come up?
+   * {@value}
+   */
+  String JT_PROBE_BOOTSTRAP_TIMEOUT =
+    MONITOR_KEY_PREFIX + "jtprobe.bootstrap.timeout";
+
+
+  /**
+   * default timeout for port probes {@value}
+   */
+
+  int JT_PROBE_BOOTSTRAP_TIMEOUT_DEFAULT = PORT_PROBE_BOOTSTRAP_TIMEOUT_DEFAULT;
+
+
+  /**
+   * Probe enabled/disabled flag Key: {@value}
+   */
+  String PID_PROBE_ENABLED =
+    MONITOR_KEY_PREFIX + "pidprobe.enabled";
+
+  /**
+   * PID probing key : pid to attempt to create a TCP connection to {@value}
+   */
+  String PID_PROBE_PIDFILE =
+    MONITOR_KEY_PREFIX + "pidprobe.pidfile";
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/MonitorUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/MonitorUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/MonitorUtils.java
new file mode 100644
index 0000000..a4447e3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/MonitorUtils.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.servicemonitor;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Formatter;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.TreeSet;
+
+/**
+ * Various utils to work with the monitor
+ */
+public final class MonitorUtils {
+  protected static final Logger log = LoggerFactory.getLogger(MonitorUtils.class);
+
+  private MonitorUtils() {
+  }
+
+  public static String toPlural(int val) {
+    return val != 1 ? "s" : "";
+  }
+
+  /**
+   * Convert the arguments -including dropping any empty strings that creep in
+   * @param args arguments
+   * @return a list view with no empty strings
+   */
+  public static List<String> prepareArgs(String[] args) {
+    List<String> argsList = new ArrayList<String>(args.length);
+    StringBuilder argsStr = new StringBuilder("Arguments: [");
+    for (String arg : args) {
+      argsStr.append('"').append(arg).append("\" ");
+      if (!arg.isEmpty()) {
+        argsList.add(arg);
+      }
+    }
+    argsStr.append(']');
+    log.debug(argsStr.toString());
+    return argsList;
+  }
+
+  /**
+   * Convert milliseconds to human time -the exact format is unspecified
+   * @param milliseconds a time in milliseconds
+   * @return a time that is converted to human intervals
+   */
+  public static String millisToHumanTime(long milliseconds) {
+    StringBuilder sb = new StringBuilder();
+    // Send all output to the Appendable object sb
+    Formatter formatter = new Formatter(sb, Locale.US);
+
+    long s = Math.abs(milliseconds / 1000);
+    long m = Math.abs(milliseconds % 1000);
+    if (milliseconds > 0) {
+      formatter.format("%d.%03ds", s, m);
+    } else if (milliseconds == 0) {
+      formatter.format("0");
+    } else {
+      formatter.format("-%d.%03ds", s, m);
+    }
+    return sb.toString();
+  }
+
+  public static InetSocketAddress getURIAddress(URI uri) {
+    String host = uri.getHost();
+    int port = uri.getPort();
+    return new InetSocketAddress(host, port);
+  }
+
+
+  /**
+   * Get the localhost -may be null
+   * @return the localhost if known
+   */
+  public static InetAddress getLocalHost() {
+    InetAddress localHost;
+    try {
+      localHost = InetAddress.getLocalHost();
+    } catch (UnknownHostException e) {
+      localHost = null;
+    }
+    return localHost;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/PortProbe.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/PortProbe.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/PortProbe.java
new file mode 100644
index 0000000..b1ff792
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/PortProbe.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.servicemonitor;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+
+/**
+ * Probe for a port being open
+ */
+public class PortProbe extends Probe {
+  protected static final Logger log = LoggerFactory.getLogger(PortProbe.class);
+  private final String host;
+  private final int port;
+  private final int timeout;
+
+  public PortProbe(String host, int port, int timeout, String name, Configuration conf)
+      throws IOException {
+    super("Port probe " + name + " " + host + ":" + port + " for " + timeout + "ms",
+          conf);
+    this.host = host;
+    this.port = port;
+    this.timeout = timeout;
+  }
+
+  public static PortProbe createPortProbe(Configuration conf,
+                                          String hostname,
+                                          int port) throws IOException {
+    PortProbe portProbe = new PortProbe(hostname,
+                                        port,
+                                        conf.getInt(
+                                          PORT_PROBE_CONNECT_TIMEOUT,
+                                          PORT_PROBE_CONNECT_TIMEOUT_DEFAULT),
+                                        "",
+                                        conf);
+
+    return portProbe;
+  }
+
+  @Override
+  public void init() throws IOException {
+    if (port >= 65536) {
+      throw new IOException("Port is out of range: " + port);
+    }
+    InetAddress target;
+    if (host != null) {
+      log.debug("looking up host " + host);
+      target = InetAddress.getByName(host);
+    } else {
+      log.debug("Host is null, retrieving localhost address");
+      target = InetAddress.getLocalHost();
+    }
+    log.info("Checking " + target + ":" + port);
+  }
+
+  /**
+   * Try to connect to the (host,port); a failure to connect within
+   * the specified timeout is a failure
+   * @param livePing is the ping live: true for live; false for boot time
+   * @return the outcome
+   */
+  @Override
+  public ProbeStatus ping(boolean livePing) {
+    ProbeStatus status = new ProbeStatus();
+    InetSocketAddress sockAddr = new InetSocketAddress(host, port);
+    Socket socket = new Socket();
+    try {
+      if (log.isDebugEnabled()) {
+        log.debug("Connecting to " + sockAddr.toString() + " connection-timeout=" +
+                  MonitorUtils.millisToHumanTime(timeout));
+      }
+      socket.connect(sockAddr, timeout);
+      status.succeed(this);
+    } catch (IOException e) {
+      String error = "Probe " + sockAddr + " failed: " + e;
+      log.debug(error, e);
+      status.fail(this,
+                  new IOException(error, e));
+    } finally {
+      IOUtils.closeSocket(socket);
+    }
+    return status;
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/Probe.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/Probe.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/Probe.java
new file mode 100644
index 0000000..be4b5ef
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/Probe.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.servicemonitor;
+
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.IOException;
+
+/**
+ * Base class of all probes.
+ */
+public abstract class Probe implements MonitorKeys {
+
+  protected final Configuration conf;
+  private String name;
+
+  // =======================================================
+  /*
+   * These fields are all used by the probe loops
+   * to maintain state. Please Leave them alone.
+   */
+  public int successCount;
+  public int failureCount;
+  public long bootstrapStarted;
+  public long bootstrapFinished;
+  private boolean booted = false;
+
+  // =======================================================
+
+  /**
+   * Create a probe of a specific name
+   *
+   * @param name probe name
+   * @param conf configuration being stored.
+   */
+  public Probe(String name, Configuration conf) {
+    this.name = name;
+    this.conf = conf;
+  }
+
+
+  protected void setName(String name) {
+    this.name = name;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+
+  @Override
+  public String toString() {
+    return getName() +
+           " {" +
+           "successCount=" + successCount +
+           ", failureCount=" + failureCount +
+           '}';
+  }
+
+  /**
+   * perform any prelaunch initialization
+   */
+  public void init() throws IOException {
+
+  }
+
+  /**
+   * Ping the endpoint. All exceptions must be caught and included in the
+   * (failure) status.
+   *
+   * @param livePing is the ping live: true for live; false for boot time
+   * @return the status
+   */
+  public abstract ProbeStatus ping(boolean livePing);
+
+  public void beginBootstrap() {
+    bootstrapStarted = System.currentTimeMillis();
+  }
+
+  public void endBootstrap() {
+    setBooted(true);
+    bootstrapFinished = System.currentTimeMillis();
+  }
+
+  public boolean isBooted() {
+    return booted;
+  }
+
+  public void setBooted(boolean booted) {
+    this.booted = booted;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbeFailedException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbeFailedException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbeFailedException.java
new file mode 100644
index 0000000..f09b848
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbeFailedException.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.servicemonitor;
+
+/**
+ * An exception to raise on a probe failure
+ */
+public class ProbeFailedException extends Exception {
+
+  public final ProbeStatus status;
+
+  public ProbeFailedException(String text, ProbeStatus status) {
+    super((text == null ? "Probe Failed" : (text + ": ")) + status, status.getThrown());
+    this.status = status;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbeInterruptedException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbeInterruptedException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbeInterruptedException.java
new file mode 100644
index 0000000..5a02f46
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbeInterruptedException.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.servicemonitor;
+
+/**
+ * This exception is raised when the probe loop detects that it has been requested to stop
+ *
+ */
+public class ProbeInterruptedException extends Exception {
+
+  public ProbeInterruptedException() {
+    super("Probe Interrupted");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbePhase.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbePhase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbePhase.java
new file mode 100644
index 0000000..d87c81b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbePhase.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.servicemonitor;
+
+/**
+ * Probe phases. The names are for strings; the index is the order in which things happen;
+ * -any state can got to terminating directly.
+ */
+public enum ProbePhase {
+  INIT("Initializing", 0),
+  DEPENDENCY_CHECKING("Dependencies", 1),
+  BOOTSTRAPPING("Bootstrapping", 2),
+  LIVE("Live", 3),
+  TERMINATING("Terminating", 4);
+
+  private final String name;
+  private final int index;
+
+  ProbePhase(String name, int index) {
+    this.name = name;
+    this.index = index;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public int getIndex() {
+    return index;
+  }
+
+  /**
+   * How many phases are there?
+   */
+  public static final int PHASE_COUNT = TERMINATING.index + 1;
+
+  @Override
+  public String toString() {
+    return name;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbeReportHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbeReportHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbeReportHandler.java
new file mode 100644
index 0000000..36c20c8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbeReportHandler.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.servicemonitor;
+
+/**
+ * This interface is for use by the Poll Workers to send events to the reporters.
+ *
+ * It is up the reporters what to do with the specific events.
+ */
+public interface ProbeReportHandler {
+
+  /**
+   * The probe process has changed state. 
+   * @param probePhase the new process phrase
+   */
+  void probeProcessStateChange(ProbePhase probePhase);
+
+  /**
+   * Report a probe outcome
+   * @param phase the current phase of probing
+   * @param status the probe status
+   */
+  void probeResult(ProbePhase phase, ProbeStatus status);
+
+  /**
+   * A probe has failed
+   */
+  void probeFailure(ProbeFailedException exception);
+
+  /**
+   * A probe has just booted
+   * @param status probe status
+   */
+  void probeBooted(ProbeStatus status);
+
+  boolean commence(String name, String description);
+
+  void unregister();
+
+  /**
+   * A heartbeat event should be raised
+   * @param status the probe status
+   */
+  void heartbeat(ProbeStatus status);
+
+  /**
+   * A probe has timed out
+   * @param currentPhase the current execution phase
+   * @param probe the probe that timed out
+   * @param lastStatus the last status that was successfully received -which is implicitly 
+   * not the status of the timed out probe
+   * @param currentTime the current time
+   */
+  void probeTimedOut(ProbePhase currentPhase,
+                     Probe probe,
+                     ProbeStatus lastStatus,
+                     long currentTime);
+
+  /**
+   * Event to say that the live probe cycle completed so the entire
+   * system can be considered functional.
+   */
+  void liveProbeCycleCompleted();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbeStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbeStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbeStatus.java
new file mode 100644
index 0000000..653f479
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbeStatus.java
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.servicemonitor;
+
+import java.io.Serializable;
+import java.util.Date;
+
+/**
+ * Status message of a probe. This is designed to be sent over the wire, though the exception
+ * Had better be unserializable at the far end if that is to work.
+ */
+public final class ProbeStatus implements Serializable {
+
+  private long timestamp;
+  private String timestampText;
+  private boolean success;
+  private boolean realOutcome;
+  private String message;
+  private Throwable thrown;
+  private transient Probe originator;
+  private ProbePhase probePhase;
+
+  public ProbeStatus() {
+  }
+
+  public ProbeStatus(long timestamp, String message, Throwable thrown) {
+    this.success = false;
+    this.message = message;
+    this.thrown = thrown;
+    setTimestamp(timestamp);
+  }
+
+  public ProbeStatus(long timestamp, String message) {
+    this.success = true;
+    setTimestamp(timestamp);
+    this.message = message;
+    this.thrown = null;
+  }
+
+  public long getTimestamp() {
+    return timestamp;
+  }
+
+  public void setTimestamp(long timestamp) {
+    this.timestamp = timestamp;
+    timestampText = new Date(timestamp).toString();
+  }
+
+  public boolean isSuccess() {
+    return success;
+  }
+
+  /**
+   * Set both the success and the real outcome bits to the same value
+   * @param success the new value
+   */
+  public void setSuccess(boolean success) {
+    this.success = success;
+    realOutcome = success;
+  }
+
+  public String getTimestampText() {
+    return timestampText;
+  }
+
+  public boolean getRealOutcome() {
+    return realOutcome;
+  }
+
+  public String getMessage() {
+    return message;
+  }
+
+  public void setMessage(String message) {
+    this.message = message;
+  }
+
+  public Throwable getThrown() {
+    return thrown;
+  }
+
+  public void setThrown(Throwable thrown) {
+    this.thrown = thrown;
+  }
+
+  public ProbePhase getProbePhase() {
+    return probePhase;
+  }
+
+  public void setProbePhase(ProbePhase probePhase) {
+    this.probePhase = probePhase;
+  }
+
+  /**
+   * Get the probe that generated this result. May be null
+   * @return a possibly null reference to a probe
+   */
+  public Probe getOriginator() {
+    return originator;
+  }
+
+  /**
+   * The probe has succeeded -capture the current timestamp, set
+   * success to true, and record any other data needed.
+   * @param probe probe
+   */
+  public void succeed(Probe probe) {
+    finish(probe, true, probe.getName(), null);
+  }
+
+  /**
+   * A probe has failed either because the test returned false, or an exception
+   * was thrown. The {@link #success} field is set to false, any exception 
+   * thrown is recorded.
+   * @param probe probe that failed
+   * @param thrown an exception that was thrown.
+   */
+  public void fail(Probe probe, Throwable thrown) {
+    finish(probe, false, "Failure in " + probe, thrown);
+  }
+
+  public void finish(Probe probe, boolean succeeded, String text, Throwable thrown) {
+    setTimestamp(System.currentTimeMillis());
+    setSuccess(succeeded);
+    originator = probe;
+    message = text;
+    this.thrown = thrown;
+  }
+
+  @Override
+  public String toString() {
+    LogEntryBuilder builder = new LogEntryBuilder("Probe Status");
+    builder.elt("time", timestampText)
+           .elt("phase", probePhase)
+           .elt("outcome", (success ? "success" : "failure"));
+
+    if (success != realOutcome) {
+      builder.elt("originaloutcome", (realOutcome ? "success" : "failure"));
+    }
+    builder.elt("message", message);
+    if (thrown != null) {
+      builder.elt("exception", thrown);
+    }
+
+    return builder.toString();
+  }
+
+  public boolean inPhase(ProbePhase phase) {
+    return getProbePhase().equals(phase);
+  }
+
+  /**
+   * Flip the success bit on while the real outcome bit is kept false
+   */
+  public void markAsSuccessful() {
+    success = true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbeWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbeWorker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbeWorker.java
new file mode 100644
index 0000000..f64ec8d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ProbeWorker.java
@@ -0,0 +1,446 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.servicemonitor;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This is the entry point to do work. A list of probes is taken in, in order of
+ * booting. Once live they go to the live probes list.
+ *
+ * The dependency probes are a set of probes for dependent services, all of which
+ * must be live before boot probes commence.
+ *
+ * The boot probes are executed and are allowed to fail; failure is interpreted as "not yet live"
+ *
+ * Once all boot probes are live, the live list is used for probes; these must not fail.
+ *
+ * There is no timeout on dependency probe bootstrap time, because of the notion that
+ * restarting this service will have no effect on the dependencies. 
+ */
+
+public class ProbeWorker implements Runnable {
+  protected static final Logger log = LoggerFactory.getLogger(ProbeWorker.class);
+
+  public static final String FAILED_TO_BOOT = "Monitored service failed to bootstrap after ";
+  public static final String FAILURE_OF_A_LIVE_PROBE_DURING_BOOTSTRAPPING = "Failure of a live probe during bootstrapping";
+  private final List<Probe> monitorProbes;
+  private final List<Probe> dependencyProbes;
+  public final int interval;
+  protected volatile ProbeStatus lastStatus;
+  protected volatile ProbeStatus lastFailingBootstrapProbe;
+  protected volatile Probe currentProbe;
+  private volatile boolean mustExit;
+  private final int bootstrapTimeout;
+  private long bootstrapEndtime;
+
+  private ProbeReportHandler reportHandler;
+  private volatile ProbePhase probePhase = ProbePhase.INIT;
+
+  /**
+   * Create a probe worker
+   * @param monitorProbes list of probes that must boot and then go live -after which
+   * they must stay live.
+   * @param dependencyProbes the list of dependency probes that must all succeed before
+   * any attempt to probe the direct probe list is performed. Once the 
+   * dependency phase has completed, these probes are never checked again.
+   * @param interval probe interval in milliseconds.
+   * @param bootstrapTimeout timeout for bootstrap in milliseconds
+   */
+  public ProbeWorker(List<Probe> monitorProbes, List<Probe> dependencyProbes, int interval, int bootstrapTimeout) {
+    this.monitorProbes = monitorProbes;
+    this.dependencyProbes = dependencyProbes != null ? dependencyProbes : new ArrayList<Probe>(0);
+    this.interval = interval;
+    lastStatus = new ProbeStatus(now(),
+                                 "Initial status");
+    lastStatus.setProbePhase(ProbePhase.INIT);
+    this.bootstrapTimeout = bootstrapTimeout;
+  }
+
+  public void init() throws IOException {
+    for (Probe probe : monitorProbes) {
+      probe.init();
+    }
+    for (Probe probe : dependencyProbes) {
+      probe.init();
+    }
+  }
+
+  public void setReportHandler(ProbeReportHandler reportHandler) {
+    this.reportHandler = reportHandler;
+  }
+
+  public void setMustExit() {
+    this.mustExit = true;
+  }
+
+  public ProbeStatus getLastStatus() {
+    return lastStatus;
+  }
+
+  public synchronized Probe getCurrentProbe() {
+    return currentProbe;
+  }
+
+  public ProbePhase getProbePhase() {
+    return probePhase;
+  }
+
+  /**
+   * Enter the new process state, and report it to the report handler.
+   * This is synchronized just to make sure there isn't more than one
+   * invocation at the same time.
+   * @param status the new process status
+   */
+  private synchronized void enterProbePhase(ProbePhase status) {
+    this.probePhase = status;
+    if (reportHandler != null) {
+      reportHandler.probeProcessStateChange(status);
+    }
+  }
+
+  /**
+   * Report the probe status to the listener -setting the probe phase field
+   * before doing so.
+   * The value is also stored in the {@link #lastStatus} field
+   * @param status the new status
+   */
+  private void reportProbeStatus(ProbeStatus status) {
+    ProbePhase phase = getProbePhase();
+    status.setProbePhase(phase);
+    lastStatus = status;
+    reportHandler.probeResult(phase, status);
+  }
+
+  /**
+   * Ping one probe. Logs the operation at debug level; sets the field <code>currentProbe</code>
+   * to the probe for the duration of the operation -this is used when identifying the
+   * cause of a hung reporting loop
+   * @param probe probe to ping
+   * @param live flag to indicate whether or not the operation is live or bootstrapping
+   * @return the status of the ping
+   * @throws ProbeInterruptedException if the probe has been told to exit
+   */
+  private ProbeStatus ping(Probe probe, boolean live) throws ProbeInterruptedException {
+    if (log.isDebugEnabled()) {
+      log.debug("Executing " + probe);
+    }
+    checkForExitRequest();
+    currentProbe = probe;
+    try {
+      return probe.ping(live);
+    } finally {
+      currentProbe = null;
+    }
+  }
+
+  /**
+   * Check for an exit request -and convert it to an exception if made
+   * @throws ProbeInterruptedException iff {@link #mustExit} is true
+   */
+  private void checkForExitRequest() throws ProbeInterruptedException {
+    if (mustExit) {
+      throw new ProbeInterruptedException();
+    }
+  }
+
+  /**
+   * Check the dependencies. 
+   * The moment a failing test is reached the call returns without
+   * any reporting.
+   *
+   * All successful probes are reported, so as to keep the heartbeats happy.
+   *
+   * @return the status of the last dependency check. If this is a success
+   * them every probe passed.
+   */
+  private ProbeStatus checkDependencyProbes() throws ProbeInterruptedException {
+    ProbeStatus status = null;
+    for (Probe dependency : dependencyProbes) {
+      //ping them, making clear they are not to run any bootstrap logic
+      status = ping(dependency, true);
+
+      if (!status.isSuccess()) {
+        //the first failure means the rest of the list can be skipped
+        break;
+      }
+      reportProbeStatus(status);
+    }
+    //return the last status
+    return status;
+  }
+
+  /**
+   * Run through all the dependency probes and report their outcomes successes (even if they fail)
+   * @return true iff all the probes have succeeded.
+   * @throws ProbeInterruptedException if the process was interrupted.
+   */
+  public boolean checkAndReportDependencyProbes() throws ProbeInterruptedException {
+    ProbeStatus status;
+    status = checkDependencyProbes();
+    if (status != null && !status.isSuccess()) {
+      //during dependency checking, a failure is still reported as a success
+      status.markAsSuccessful();
+      reportProbeStatus(status);
+      //then return without checking anything else
+      return false;
+    }
+    //all dependencies are done.
+    return true;
+  }
+
+  /**
+   * Begin bootstrapping by telling each probe that they have started.
+   * This sets the timeouts up, as well as permits any other set-up actions
+   * to begin.
+   */
+  private void beginBootstrapProbes() {
+    synchronized (this) {
+      bootstrapEndtime = now() + bootstrapTimeout;
+    }
+    for (Probe probe : monitorProbes) {
+      probe.beginBootstrap();
+    }
+  }
+
+  private long now() {
+    return System.currentTimeMillis();
+  }
+
+
+  /**
+   * Check the bootstrap probe list. All successful probes get reported.
+   * The first unsuccessful probe will be returned and not reported (left for policy upstream).
+   * If the failing probe has timed out, that is turned into a {@link ProbeFailedException}
+   * @return the last (unsuccessful) probe, or null if they all succeeded
+   * @throws ProbeInterruptedException interrupts
+   * @throws ProbeFailedException on a boot timeout
+   */
+  private boolean checkBootstrapProbes() throws ProbeInterruptedException, ProbeFailedException {
+    verifyBootstrapHasNotTimedOut();
+
+    boolean probeFailed = false;
+    //now run through all the bootstrap probes
+    for (Probe probe : monitorProbes) {
+      //ping them
+      ProbeStatus status = ping(probe, false);
+      if (!status.isSuccess()) {
+        probeFailed = true;
+        lastFailingBootstrapProbe = status;
+        probe.failureCount++;
+        if (log.isDebugEnabled()) {
+          log.debug("Booting probe failed: " + status);
+        }
+        //at this point check to see if the timeout has occurred -and if so, force in the last probe status.
+
+        //this is a failure but not a timeout
+        //during boot, a failure of a probe that hasn't booted is still reported as a success
+        if (!probe.isBooted()) {
+          //so the success bit is flipped
+          status.markAsSuccessful();
+          reportProbeStatus(status);
+        } else {
+          //the probe had booted but then it switched to failing
+
+          //update the status unedited
+          reportProbeStatus(status);
+          //then fail
+          throw raiseProbeFailure(status, FAILURE_OF_A_LIVE_PROBE_DURING_BOOTSTRAPPING);
+        }
+      } else {
+        //this probe is working
+        if (!probe.isBooted()) {
+          //if it is new, mark it as live
+          if (log.isDebugEnabled()) {
+            log.debug("Booting probe is now live: " + probe);
+          }
+          probe.endBootstrap();
+          //tell the report handler that another probe has booted
+          reportHandler.probeBooted(status);
+        }
+        //push out its status
+        reportProbeStatus(status);
+        probe.successCount++;
+      }
+    }
+    return !probeFailed;
+  }
+
+
+  public int getBootstrapTimeout() {
+    return bootstrapTimeout;
+  }
+
+  /**
+   * This checks that bootstrap operations have not timed out
+   * @throws ProbeFailedException if the bootstrap has failed
+   */
+  public void verifyBootstrapHasNotTimedOut() throws ProbeFailedException {
+    //first step -look for a timeout
+    if (isBootstrapTimeExceeded()) {
+      String text = FAILED_TO_BOOT
+                    + MonitorUtils.millisToHumanTime(bootstrapTimeout);
+
+      ProbeStatus status;
+      if (lastFailingBootstrapProbe != null) {
+        status = lastFailingBootstrapProbe;
+        status.setSuccess(false);
+      } else {
+        status = new ProbeStatus();
+        status.finish(null, false, text, null);
+      }
+
+      throw raiseProbeFailure(status,
+                              text);
+    }
+  }
+
+  /**
+   * predicate that gets current time and checks for its time being exceeded.
+   * @return true iff the current time is > the end time
+   */
+  public synchronized boolean isBootstrapTimeExceeded() {
+    return now() > bootstrapEndtime;
+  }
+
+  /**
+   * run through all the bootstrap probes and see if they are live.
+   * @return true iff all boot probes succeeded
+   * @throws ProbeInterruptedException the probe interruption flags
+   * @throws ProbeFailedException if a probe failed.
+   */
+  public boolean checkAndReportBootstrapProbes() throws ProbeInterruptedException,
+                                                        ProbeFailedException {
+    if (bootstrapTimeout <= 0) {
+      //there is no period of grace for bootstrapping probes, so return true saying
+      //this phase is complete
+      return true;
+    }
+    //now the bootstrapping probes
+    return checkBootstrapProbes();
+  }
+
+
+  /**
+   * run through all the live probes, pinging and reporting them.
+   * A single probe failure is turned into an exception
+   * @throws ProbeFailedException a probe failed
+   * @throws ProbeInterruptedException the probe process was explicitly interrupted
+   */
+  protected void checkAndReportLiveProbes() throws ProbeFailedException, ProbeInterruptedException {
+    ProbeStatus status = null;
+    //go through the live list
+    if (log.isDebugEnabled()) {
+      log.debug("Checking live probes");
+    }
+    for (Probe probe : monitorProbes) {
+      status = ping(probe, true);
+      reportProbeStatus(status);
+      if (!status.isSuccess()) {
+        throw raiseProbeFailure(status, "Failure of probe in \"live\" monitor");
+      }
+      probe.successCount++;
+    }
+    //here all is well, so notify the reporter
+    reportHandler.liveProbeCycleCompleted();
+  }
+
+  /**
+   * Run the set of probes relevant for this phase of the probe lifecycle.
+   * @throws ProbeFailedException a probe failed
+   * @throws ProbeInterruptedException the probe process was explicitly interrupted
+   */
+  protected void executeProbePhases() throws ProbeFailedException, ProbeInterruptedException {
+    switch (probePhase) {
+      case INIT:
+        enterProbePhase(ProbePhase.DEPENDENCY_CHECKING);
+        //fall through straight into the dependency check
+      case DEPENDENCY_CHECKING:
+        if (checkAndReportDependencyProbes()) {
+          enterProbePhase(ProbePhase.BOOTSTRAPPING);
+          beginBootstrapProbes();
+        }
+        break;
+      case BOOTSTRAPPING:
+        if (checkAndReportBootstrapProbes()) {
+          enterProbePhase(ProbePhase.LIVE);
+        }
+        break;
+      case LIVE:
+        checkAndReportLiveProbes();
+        break;
+
+      case TERMINATING:
+      default:
+        //do nothing.
+        break;
+    }
+  }
+
+
+  /**
+   * Raise a probe failure; injecting the phase into the status result first
+   *
+   * @param status ping result
+   * @param text optional text -null or "" means "none"
+   * @return an exception ready to throw
+   */
+  private ProbeFailedException raiseProbeFailure(ProbeStatus status, String text) {
+    status.setProbePhase(probePhase);
+    log.info("Probe failed: " + status);
+    return new ProbeFailedException(text, status);
+  }
+
+  @Override
+  public void run() {
+    int size = monitorProbes.size();
+    log.info("Probe Worker Starting; " + size + " probe" + MonitorUtils.toPlural(size) + ":");
+    enterProbePhase(ProbePhase.DEPENDENCY_CHECKING);
+    for (Probe probe : monitorProbes) {
+      log.info(probe.getName());
+    }
+    while (!mustExit) {
+      try {
+        Thread.sleep(interval);
+        executeProbePhases();
+      } catch (ProbeFailedException e) {
+        //relay to the inner loop handler
+        probeFailed(e);
+      } catch (InterruptedException interrupted) {
+        break;
+      } catch (ProbeInterruptedException e) {
+        //exit raised.
+        //this will be true, just making extra-sure
+        break;
+      }
+    }
+    log.info("Probe Worker Exiting");
+    enterProbePhase(ProbePhase.TERMINATING);
+  }
+
+
+  protected void probeFailed(ProbeFailedException e) {
+    reportHandler.probeFailure(e);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ReportingLoop.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ReportingLoop.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ReportingLoop.java
new file mode 100644
index 0000000..096838d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/ReportingLoop.java
@@ -0,0 +1,265 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.servicemonitor;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * This is the monitor service
+ */
+public final class ReportingLoop implements Runnable, ProbeReportHandler, MonitorKeys, Closeable {
+  protected static final Logger log = LoggerFactory.getLogger(ReportingLoop.class);
+  private final ProbeWorker worker;
+  private final Thread workerThread;
+  private final int reportInterval;
+  private final int probeTimeout;
+  private final int bootstrapTimeout;
+  private ProbeReportHandler reporter;
+  private final String name;
+  private volatile boolean mustExit;
+
+  public ReportingLoop(String name,
+                       ProbeReportHandler reporter,
+                       List<Probe> probes,
+                       List<Probe> dependencyProbes,
+                       int probeInterval,
+                       int reportInterval,
+                       int probeTimeout,
+                       int bootstrapTimeout) throws IOException {
+    this(name,
+         reporter,
+         new ProbeWorker(probes, dependencyProbes, probeInterval, bootstrapTimeout),
+         reportInterval,
+         probeTimeout);
+  }
+
+  /**
+   * Create a new reporting loop -and bond the worker's ProbeReportHandler
+   * to us
+   * @param name
+   * @param reporter
+   * @param worker
+   * @param reportInterval
+   * @param probeTimeout
+   */
+  public ReportingLoop(String name,
+                       ProbeReportHandler reporter,
+                       ProbeWorker worker,
+                       int reportInterval,
+                       int probeTimeout) throws IOException {
+    this.name = name;
+    this.reporter = reporter;
+    this.reportInterval = reportInterval;
+    this.probeTimeout = probeTimeout;
+    this.worker = worker;
+    this.bootstrapTimeout = worker.getBootstrapTimeout();
+    worker.setReportHandler(this);
+    workerThread = new Thread(worker, "probe thread - " + name);
+    worker.init();
+  }
+  
+  public int getBootstrapTimeout() {
+    return bootstrapTimeout;
+  }
+
+  public ReportingLoop withReporter(ProbeReportHandler reporter) {
+    assert this.reporter == null : "attempting to reassign reporter ";
+    assert reporter != null : "new reporter is null";
+    this.reporter = reporter;
+    return this;
+  }
+
+  /**
+   * Start the monitoring.
+   *
+   * @return false if the monitoring did not start and that the worker threads
+   *         should be run up.
+   */
+  public boolean startReporting() {
+    String description = "Service Monitor for " + name + ", probe-interval= "
+                         + MonitorUtils.millisToHumanTime(worker.interval)
+                         + ", report-interval=" + MonitorUtils.millisToHumanTime(reportInterval)
+                         + ", probe-timeout=" + timeoutToStr(probeTimeout)
+                         + ", bootstrap-timeout=" + timeoutToStr(bootstrapTimeout);
+    log.info("Starting reporting"
+             + " to " + reporter
+             + description);
+    return reporter.commence(name, description);
+  }
+
+  private String timeoutToStr(int timeout) {
+    return timeout >= 0 ? MonitorUtils.millisToHumanTime(timeout) : "not set";
+  }
+
+  private void startWorker() {
+    log.info("Starting reporting worker thread ");
+    workerThread.setDaemon(true);
+    workerThread.start();
+  }
+
+
+  /**
+   * This exits the process cleanly
+   */
+  @Override
+  public void close() {
+    log.info("Stopping reporting");
+    mustExit = true;
+    if (worker != null) {
+      worker.setMustExit();
+      workerThread.interrupt();
+    }
+    if (reporter != null) {
+      reporter.unregister();
+    }
+  }
+
+  @Override
+  public void probeFailure(ProbeFailedException exception) {
+    reporter.probeFailure(exception);
+  }
+
+  @Override
+  public void probeProcessStateChange(ProbePhase probePhase) {
+    reporter.probeProcessStateChange(probePhase);
+  }
+
+  @Override
+  public void probeBooted(ProbeStatus status) {
+    reporter.probeBooted(status);
+  }
+
+  private long now() {
+    return System.currentTimeMillis();
+  }
+
+  @Override
+  public void probeResult(ProbePhase phase, ProbeStatus status) {
+    reporter.probeResult(phase, status);
+  }
+
+  @Override
+  public boolean commence(String n, String description) {
+    return true;
+  }
+
+  @Override
+  public void unregister() {
+  }
+
+  @Override
+  public void heartbeat(ProbeStatus status) {
+  }
+
+  @Override
+  public void probeTimedOut(ProbePhase currentPhase, Probe probe, ProbeStatus lastStatus,
+      long currentTime) {
+  }
+
+  @Override
+  public void liveProbeCycleCompleted() {
+    //delegate to the reporter
+    reporter.liveProbeCycleCompleted();
+  }
+
+  /**
+   * The reporting loop
+   */
+  void reportingLoop() {
+
+    while (!mustExit) {
+      try {
+        ProbeStatus workerStatus = worker.getLastStatus();
+        long now = now();
+        long lastStatusIssued = workerStatus.getTimestamp();
+        long timeSinceLastStatusIssued = now - lastStatusIssued;
+        //two actions can occur here: a heartbeat is issued or a timeout reported. 
+        //this flag decides which
+        boolean heartbeat;
+
+        //based on phase, decide whether to heartbeat or timeout
+        ProbePhase probePhase = worker.getProbePhase();
+        switch (probePhase) {
+          case DEPENDENCY_CHECKING:
+            //no timeouts in dependency phase
+            heartbeat = true;
+            break;
+
+          case BOOTSTRAPPING:
+            //the timeout here is fairly straightforward: heartbeats are
+            //raised while the worker hasn't timed out
+            heartbeat = bootstrapTimeout < 0 || timeSinceLastStatusIssued < bootstrapTimeout;
+
+            break;
+
+          case LIVE:
+            //use the probe timeout interval between the current time
+            //and the time the last status event was received.
+            heartbeat = timeSinceLastStatusIssued < probeTimeout;
+            break;
+
+          case INIT:
+          case TERMINATING:
+          default:
+            //send a heartbeat, because this isn't the time to be failing
+            heartbeat = true;
+        }
+        if (heartbeat) {
+          //a heartbeat is sent to the reporter
+          reporter.heartbeat(workerStatus);
+        } else {
+          //no response from the worker -it is hung.
+          reporter.probeTimedOut(probePhase,
+                                 worker.getCurrentProbe(),
+                                 workerStatus,
+                                 now
+                                );
+        }
+
+        //now sleep
+        Thread.sleep(reportInterval);
+
+      } catch (InterruptedException e) {
+        //interrupted -always exit the loop.
+        break;
+      }
+    }
+    //this point is reached if and only if a clean exit was requested or something failed.
+  }
+
+  /**
+   * This can be run in a separate thread, or it can be run directly from the caller.
+   * Test runs do the latter, HAM runs multiple reporting threads.
+   */
+  @Override
+  public void run() {
+    try {
+      startWorker();
+      reportingLoop();
+    } catch (RuntimeException e) {
+      log.warn("Failure in the reporting loop: " + e, e);
+      //rethrow so that inline code can pick it up (e.g. test runs)
+      throw e;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/YarnApplicationProbe.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/YarnApplicationProbe.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/YarnApplicationProbe.java
new file mode 100644
index 0000000..adf613c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/servicemonitor/YarnApplicationProbe.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.servicemonitor;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.slider.client.SliderYarnClientImpl;
+import org.apache.slider.core.exceptions.UnknownApplicationInstanceException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Probe for YARN application
+ */
+public class YarnApplicationProbe extends Probe {
+  protected static final Logger log = LoggerFactory.getLogger(
+    YarnApplicationProbe.class);
+
+  /**
+   * Yarn client service
+   */
+  private SliderYarnClientImpl yarnClient;
+  private final String clustername;
+  private final String username;
+
+  public YarnApplicationProbe(String clustername,
+                              SliderYarnClientImpl yarnClient,
+                              String name,
+                              Configuration conf, String username)
+      throws IOException {
+    super("Port probe " + name + " " + clustername,
+          conf);
+    this.clustername = clustername;
+    this.yarnClient = yarnClient;
+    this.username = username;
+  }
+
+
+  @Override
+  public void init() throws IOException {
+   
+    log.info("Checking " + clustername );
+  }
+
+  /**
+   * Try to connect to the (host,port); a failure to connect within
+   * the specified timeout is a failure
+   * @param livePing is the ping live: true for live; false for boot time
+   * @return the outcome
+   */
+  @Override
+  public ProbeStatus ping(boolean livePing) {
+    
+    ProbeStatus status = new ProbeStatus();
+    try {
+
+      List<ApplicationReport> instances =
+        yarnClient.listDeployedInstances(username);
+      ApplicationReport instance =
+        yarnClient.findClusterInInstanceList(instances, clustername);
+      if (null == instance) {
+        throw UnknownApplicationInstanceException.unknownInstance(clustername);
+      }
+
+      status.succeed(this);
+    } catch (Exception e) {
+      status.fail(this, e);
+    }
+    return status;
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/AbstractSecurityStoreGenerator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/AbstractSecurityStoreGenerator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/AbstractSecurityStoreGenerator.java
new file mode 100644
index 0000000..11d3aa1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/security/AbstractSecurityStoreGenerator.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.services.security;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.alias.CredentialProvider;
+import org.apache.hadoop.security.alias.CredentialProviderFactory;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.exceptions.SliderException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+/**
+ *
+ */
+public abstract class AbstractSecurityStoreGenerator implements
+    SecurityStoreGenerator {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractSecurityStoreGenerator.class);
+
+  protected CertificateManager certificateMgr;
+
+  public AbstractSecurityStoreGenerator(CertificateManager certificateMgr) {
+    this.certificateMgr = certificateMgr;
+  }
+
+  protected String getStorePassword(Map<String, List<String>> credentials,
+                                    MapOperations compOps, String role)
+      throws SliderException, IOException {
+    String password = getPassword(compOps);
+    if (password == null) {
+      // need to leverage credential provider
+      String alias = getAlias(compOps);
+      LOG.debug("Alias {} found for role {}", alias, role);
+      if (alias == null) {
+        throw new SliderException("No store password or credential provider "
+                                  + "alias found");
+      }
+      if (credentials.isEmpty()) {
+        LOG.info("Credentials can not be retrieved for store generation since "
+                 + "no CP paths are configured");
+      }
+      synchronized (this) {
+        for (Map.Entry<String, List<String>> cred : credentials.entrySet()) {
+          String provider = cred.getKey();
+          Configuration c = new Configuration();
+          c.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, provider);
+          LOG.debug("Configured provider {}", provider);
+          CredentialProvider cp =
+              CredentialProviderFactory.getProviders(c).get(0);
+          LOG.debug("Aliases: {}", cp.getAliases());
+          char[] credential = c.getPassword(alias);
+          if (credential != null) {
+            LOG.info("Credential found for role {}", role);
+            return String.valueOf(credential);
+          }
+        }
+      }
+
+      if (password == null) {
+        LOG.info("No store credential found for alias {}.  "
+                 + "Generation of store for {} is not possible.", alias, role);
+
+      }
+    }
+
+    return password;
+
+  }
+
+  @Override
+  public boolean isStoreRequested(MapOperations compOps) {
+    return compOps.getOptionBool(SliderKeys.COMP_STORES_REQUIRED_KEY, false);
+  }
+
+  abstract String getPassword(MapOperations compOps);
+
+  abstract String getAlias(MapOperations compOps);
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[41/76] [abbrv] hadoop git commit: YARN-5513. Move Java only tests from slider develop to yarn-native-services. Contributed by Gour Saha

Posted by ji...@apache.org.
YARN-5513. Move Java only tests from slider develop to yarn-native-services. Contributed by Gour Saha


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

Branch: refs/heads/yarn-native-services
Commit: a4beaeb897721f2dfc6540b6bd3073b026b764fd
Parents: 0437f8a
Author: Jian He <ji...@apache.org>
Authored: Wed Aug 17 00:42:24 2016 +0800
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 .../dev-support/findbugs-exclude.xml            |  20 +
 .../hadoop-yarn-slider-core/pom.xml             |  22 +
 .../slider/common/tools/TestSliderUtils.java    | 159 ++++
 .../core/launch/TestAppMasterLauncher.java      | 157 ++++
 .../TestAppMasterLauncherWithAmReset.java       |  92 ++
 .../TestPublishedConfigurationOutputter.java    | 222 +++++
 .../agent/TestAgentClientProvider.java          |  77 ++
 .../agent/TestAgentLaunchParameter.java         |  76 ++
 .../slider/providers/agent/TestAgentUtils.java  |  94 ++
 .../agent/TestAppDefinitionPersister.java       | 264 ++++++
 .../agent/TestComponentTagProvider.java         | 115 +++
 .../slider/providers/agent/TestState.java       |  33 +
 .../application/metadata/TestConfigParser.java  | 107 +++
 .../metadata/TestMetainfoParser.java            | 177 ++++
 .../appmaster/TestServiceRecordAttributes.java  |  68 ++
 .../publisher/TestAgentProviderService.java     |  60 ++
 .../publisher/TestSliderProviderFactory.java    |  40 +
 .../server/servicemonitor/TestPortProbe.java    |  37 +
 .../security/TestCertificateManager.java        | 540 +++++++++++
 .../TestMultiThreadedStoreGeneration.java       | 156 ++++
 .../server/services/workflow/MockService.java   |  80 ++
 .../workflow/ParentWorkflowTestBase.java        |  70 ++
 .../workflow/ProcessCommandFactory.java         |  96 ++
 .../services/workflow/SimpleRunnable.java       |  46 +
 .../workflow/TestWorkflowClosingService.java    | 116 +++
 .../workflow/TestWorkflowCompositeService.java  | 113 +++
 .../workflow/TestWorkflowExecutorService.java   |  66 ++
 .../workflow/TestWorkflowRpcService.java        | 107 +++
 .../workflow/TestWorkflowSequenceService.java   | 151 ++++
 .../TestWorkflowServiceTerminatingRunnable.java |  64 ++
 .../workflow/WorkflowServiceTestBase.java       | 139 +++
 .../apache/slider/test/ContractTestUtils.java   | 901 +++++++++++++++++++
 .../slider/test/MiniZooKeeperCluster.java       | 395 ++++++++
 .../org/apache/slider/tools/TestUtility.java    | 181 ++++
 .../slider/common/tools/test/metainfo.txt       |  16 +
 .../slider/common/tools/test/metainfo.xml       |  98 ++
 .../slider/common/tools/test/someOtherFile.txt  |  16 +
 .../slider/common/tools/test/someOtherFile.xml  |  17 +
 .../agent/application/metadata/metainfo.xml     | 180 ++++
 39 files changed, 5368 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/dev-support/findbugs-exclude.xml
new file mode 100644
index 0000000..b89146a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/dev-support/findbugs-exclude.xml
@@ -0,0 +1,20 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+   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.
+-->
+<FindBugsFilter>
+
+</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
index 591a5ca..d778f44 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
@@ -129,6 +129,13 @@
 
     <dependency>
       <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-hdfs</artifactId>
     </dependency>
 
@@ -268,6 +275,20 @@
     </dependency>
 
     <dependency>
+      <groupId>org.easymock</groupId>
+      <artifactId>easymock</artifactId>
+      <version>3.1</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.powermock</groupId>
+      <artifactId>powermock-api-easymock</artifactId>
+      <version>1.5</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
       <groupId>org.mortbay.jetty</groupId>
       <artifactId>jetty</artifactId>
     </dependency>
@@ -292,6 +313,7 @@
       <groupId>org.codehaus.jettison</groupId>
       <artifactId>jettison</artifactId>
     </dependency>
+
     <dependency>
       <groupId>org.mortbay.jetty</groupId>
       <artifactId>jetty-sslengine</artifactId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestSliderUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestSliderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestSliderUtils.java
new file mode 100644
index 0000000..20e72c0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/common/tools/TestSliderUtils.java
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.common.tools;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationReportPBImpl;
+import org.apache.slider.tools.TestUtility;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+
+/** Test slider util methods. */
+public class TestSliderUtils {
+  protected static final Logger log =
+      LoggerFactory.getLogger(TestSliderUtils.class);
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  @Test
+  public void testGetMetaInfoStreamFromZip() throws Exception {
+    String zipFileName = TestUtility.createAppPackage(
+        folder,
+        "testpkg",
+        "test.zip",
+        "target/test-classes/org/apache/slider/common/tools/test");
+    Configuration configuration = new Configuration();
+    FileSystem fs = FileSystem.getLocal(configuration);
+    log.info("fs working dir is {}", fs.getWorkingDirectory().toString());
+    SliderFileSystem sliderFileSystem = new SliderFileSystem(fs, configuration);
+
+    InputStream stream = SliderUtils.getApplicationResourceInputStream(
+        sliderFileSystem.getFileSystem(),
+        new Path(zipFileName),
+        "metainfo.xml");
+    Assert.assertTrue(stream != null);
+    Assert.assertTrue(stream.available() > 0);
+  }
+
+  @Test
+  public void testTruncate() {
+    Assert.assertEquals(SliderUtils.truncate(null, 5), null);
+    Assert.assertEquals(SliderUtils.truncate("323", -1), "323");
+    Assert.assertEquals(SliderUtils.truncate("3232", 5), "3232");
+    Assert.assertEquals(SliderUtils.truncate("1234567890", 0), "1234567890");
+    Assert.assertEquals(SliderUtils.truncate("123456789012345", 15), "123456789012345");
+    Assert.assertEquals(SliderUtils.truncate("123456789012345", 14), "12345678901...");
+    Assert.assertEquals(SliderUtils.truncate("1234567890", 1), "1");
+    Assert.assertEquals(SliderUtils.truncate("1234567890", 10), "1234567890");
+    Assert.assertEquals(SliderUtils.truncate("", 10), "");
+  }
+
+  @Test
+  public void testApplicationReportComparison() {
+    List<ApplicationReport> instances = getApplicationReports();
+
+    SliderUtils.sortApplicationsByMostRecent(instances);
+
+    Assert.assertEquals(1000, instances.get(0).getStartTime());
+    Assert.assertEquals(1000, instances.get(1).getStartTime());
+    Assert.assertEquals(1000, instances.get(2).getStartTime());
+    Assert.assertEquals(1000, instances.get(3).getStartTime());
+
+    instances = getApplicationReports();
+
+    SliderUtils.sortApplicationReport(instances);
+    Assert.assertEquals(1000, instances.get(0).getStartTime());
+    Assert.assertEquals(1000, instances.get(1).getStartTime());
+    Assert.assertEquals(1000, instances.get(2).getStartTime());
+    Assert.assertEquals(1000, instances.get(3).getStartTime());
+
+    Assert.assertTrue(instances.get(0).getYarnApplicationState() == YarnApplicationState.ACCEPTED ||
+                      instances.get(0).getYarnApplicationState() == YarnApplicationState.RUNNING);
+    Assert.assertTrue(instances.get(1).getYarnApplicationState() == YarnApplicationState.ACCEPTED ||
+                      instances.get(1).getYarnApplicationState() == YarnApplicationState.RUNNING);
+    Assert.assertTrue(instances.get(2).getYarnApplicationState() == YarnApplicationState.ACCEPTED ||
+                      instances.get(2).getYarnApplicationState() == YarnApplicationState.RUNNING);
+    Assert.assertTrue(instances.get(3).getYarnApplicationState() == YarnApplicationState.KILLED);
+  }
+
+  private List<ApplicationReport> getApplicationReports() {
+    List<ApplicationReport> instances = new ArrayList<ApplicationReport>();
+    instances.add(getApplicationReport(1000, 0, "app1", YarnApplicationState.ACCEPTED));
+    instances.add(getApplicationReport(900, 998, "app1", YarnApplicationState.KILLED));
+    instances.add(getApplicationReport(900, 998, "app2", YarnApplicationState.FAILED));
+    instances.add(getApplicationReport(1000, 0, "app2", YarnApplicationState.RUNNING));
+    instances.add(getApplicationReport(800, 837, "app3", YarnApplicationState.FINISHED));
+    instances.add(getApplicationReport(1000, 0, "app3", YarnApplicationState.RUNNING));
+    instances.add(getApplicationReport(900, 998, "app3", YarnApplicationState.KILLED));
+    instances.add(getApplicationReport(800, 837, "app4", YarnApplicationState.FINISHED));
+    instances.add(getApplicationReport(1000, 1050, "app4", YarnApplicationState.KILLED));
+    instances.add(getApplicationReport(900, 998, "app4", YarnApplicationState.FINISHED));
+
+    Assert.assertEquals("app1", instances.get(0).getApplicationType());
+    Assert.assertEquals("app1", instances.get(1).getApplicationType());
+    Assert.assertEquals("app2", instances.get(2).getApplicationType());
+    Assert.assertEquals("app2", instances.get(3).getApplicationType());
+    return instances;
+  }
+
+  private ApplicationReportPBImpl getApplicationReport(long startTime,
+                                                       long finishTime,
+                                                       String name,
+                                                       YarnApplicationState state) {
+    ApplicationReportPBImpl ar = new ApplicationReportPBImpl();
+    ar.setFinishTime(finishTime);
+    ar.setStartTime(startTime);
+    ar.setApplicationType(name);
+    ar.setYarnApplicationState(state);
+    return ar;
+  }
+
+
+  @Test
+  public void testGetHdpVersion() {
+    String hdpVersion = "2.3.2.0-2766";
+    Assert.assertEquals("Version should be empty", null,
+        SliderUtils.getHdpVersion());
+  }
+
+  @Test
+  public void testIsHdp() {
+    Assert.assertFalse("Should be false", SliderUtils.isHdp());
+  }
+
+  @Test
+  public void testWrite() throws IOException {
+    File testWriteFile = folder.newFile("testWrite");
+    SliderUtils.write(testWriteFile, "test".getBytes("UTF-8"), true);
+    Assert.assertTrue(FileUtils.readFileToString(testWriteFile, "UTF-8").equals("test"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/launch/TestAppMasterLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/launch/TestAppMasterLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/launch/TestAppMasterLauncher.java
new file mode 100644
index 0000000..b955931
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/launch/TestAppMasterLauncher.java
@@ -0,0 +1,157 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.launch;
+
+import java.lang.reflect.Method;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.LogAggregationContext;
+import org.apache.hadoop.yarn.client.api.YarnClientApplication;
+import org.apache.slider.api.ResourceKeys;
+import org.apache.slider.client.SliderYarnClientImpl;
+import org.apache.slider.common.SliderKeys;
+import org.easymock.EasyMock;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestAppMasterLauncher {
+  SliderYarnClientImpl mockYarnClient;
+  YarnClientApplication yarnClientApp;
+  ApplicationSubmissionContext appSubmissionContext;
+  Set<String> tags = Collections.emptySet();
+  AppMasterLauncher appMasterLauncher = null;
+  boolean isOldApi = true;
+  Method rolledLogsIncludeMethod = null;
+  Method rolledLogsExcludeMethod = null;
+
+  @Before
+  public void initialize() throws Exception {
+    mockYarnClient = EasyMock.createNiceMock(SliderYarnClientImpl.class);
+    yarnClientApp = EasyMock.createNiceMock(YarnClientApplication.class);
+    appSubmissionContext = EasyMock
+        .createNiceMock(ApplicationSubmissionContext.class);
+    EasyMock.expect(yarnClientApp.getApplicationSubmissionContext())
+        .andReturn(appSubmissionContext).once();
+    EasyMock.expect(mockYarnClient.createApplication())
+        .andReturn(yarnClientApp).once();
+
+    try {
+      LogAggregationContext.class.getMethod("newInstance", String.class,
+          String.class, String.class, String.class);
+      isOldApi = false;
+      rolledLogsIncludeMethod = LogAggregationContext.class
+          .getMethod("getRolledLogsIncludePattern");
+      rolledLogsExcludeMethod = LogAggregationContext.class
+          .getMethod("getRolledLogsExcludePattern");
+    } catch (Exception e) {
+      isOldApi = true;
+    }
+  }
+
+  /**
+   * These tests will probably fail when compiled against hadoop 2.7+. Please
+   * refer to SLIDER-810. It has been purposely not modified so that it fails
+   * and that someone needs to modify the code in
+   * {@code AbstractLauncher#extractLogAggregationContext(Map)}. Comments are
+   * provided in that method as to what needs to be done.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testExtractLogAggregationContext() throws Exception {
+    Map<String, String> options = new HashMap<String, String>();
+    options.put(ResourceKeys.YARN_LOG_INCLUDE_PATTERNS,
+        " | slider*.txt  |agent.out| |");
+    options.put(ResourceKeys.YARN_LOG_EXCLUDE_PATTERNS,
+        "command*.json|  agent.log*        |     ");
+
+    EasyMock.replay(mockYarnClient, appSubmissionContext, yarnClientApp);
+    appMasterLauncher = new AppMasterLauncher("cl1", SliderKeys.APP_TYPE, null,
+        null, mockYarnClient, false, null, options, tags, null);
+
+    // Verify the include/exclude patterns
+    String expectedInclude = "slider*.txt|agent.out";
+    String expectedExclude = "command*.json|agent.log*";
+    assertPatterns(expectedInclude, expectedExclude);
+
+    EasyMock.verify(mockYarnClient, appSubmissionContext, yarnClientApp);
+
+  }
+
+  @Test
+  public void testExtractLogAggregationContextEmptyIncludePattern()
+      throws Exception {
+    Map<String, String> options = new HashMap<String, String>();
+    options.put(ResourceKeys.YARN_LOG_INCLUDE_PATTERNS, " ");
+    options.put(ResourceKeys.YARN_LOG_EXCLUDE_PATTERNS,
+        "command*.json|  agent.log*        |     ");
+
+    EasyMock.replay(mockYarnClient, appSubmissionContext, yarnClientApp);
+    appMasterLauncher = new AppMasterLauncher("cl1", SliderKeys.APP_TYPE, null,
+        null, mockYarnClient, false, null, options, tags, null);
+
+    // Verify the include/exclude patterns
+    String expectedInclude = isOldApi ? "" : ".*";
+    String expectedExclude = "command*.json|agent.log*";
+    assertPatterns(expectedInclude, expectedExclude);
+
+    EasyMock.verify(mockYarnClient, appSubmissionContext, yarnClientApp);
+  }
+
+  @Test
+  public void testExtractLogAggregationContextEmptyIncludeAndExcludePattern()
+      throws Exception {
+    Map<String, String> options = new HashMap<String, String>();
+    options.put(ResourceKeys.YARN_LOG_INCLUDE_PATTERNS, "");
+    options.put(ResourceKeys.YARN_LOG_EXCLUDE_PATTERNS, "  ");
+
+    EasyMock.replay(mockYarnClient, appSubmissionContext, yarnClientApp);
+    appMasterLauncher = new AppMasterLauncher("cl1", SliderKeys.APP_TYPE, null,
+        null, mockYarnClient, false, null, options, tags, null);
+
+    // Verify the include/exclude patterns
+    String expectedInclude = isOldApi ? "" : ".*";
+    String expectedExclude = "";
+    assertPatterns(expectedInclude, expectedExclude);
+
+    EasyMock.verify(mockYarnClient, appSubmissionContext, yarnClientApp);
+  }
+
+  private void assertPatterns(String expectedIncludePattern,
+      String expectedExcludePattern) throws Exception {
+    if (isOldApi) {
+      Assert.assertEquals(expectedIncludePattern,
+          appMasterLauncher.logAggregationContext.getIncludePattern());
+      Assert.assertEquals(expectedExcludePattern,
+          appMasterLauncher.logAggregationContext.getExcludePattern());
+    } else {
+      Assert.assertEquals(expectedIncludePattern,
+          (String) rolledLogsIncludeMethod
+              .invoke(appMasterLauncher.logAggregationContext));
+      Assert.assertEquals(expectedExcludePattern,
+          (String) rolledLogsExcludeMethod
+              .invoke(appMasterLauncher.logAggregationContext));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/launch/TestAppMasterLauncherWithAmReset.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/launch/TestAppMasterLauncherWithAmReset.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/launch/TestAppMasterLauncherWithAmReset.java
new file mode 100644
index 0000000..a8f6b26
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/launch/TestAppMasterLauncherWithAmReset.java
@@ -0,0 +1,92 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.launch;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.client.api.YarnClientApplication;
+import org.apache.hadoop.yarn.util.Records;
+import org.apache.slider.api.ResourceKeys;
+import org.apache.slider.client.SliderYarnClientImpl;
+import org.apache.slider.common.SliderKeys;
+import org.easymock.EasyMock;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestAppMasterLauncherWithAmReset {
+  SliderYarnClientImpl mockYarnClient;
+  YarnClientApplication yarnClientApp;
+  ApplicationSubmissionContext appSubmissionContext;
+  GetNewApplicationResponse newApp;
+  Set<String> tags = Collections.emptySet();
+  AppMasterLauncher appMasterLauncher = null;
+  boolean isOldApi = true;
+
+  @Before
+  public void initialize() throws Exception {
+    mockYarnClient = EasyMock.createNiceMock(SliderYarnClientImpl.class);
+    yarnClientApp = EasyMock.createNiceMock(YarnClientApplication.class);
+    newApp = EasyMock.createNiceMock(GetNewApplicationResponse.class);
+    EasyMock.expect(mockYarnClient.createApplication())
+        .andReturn(new YarnClientApplication(newApp,
+        Records.newRecord(ApplicationSubmissionContext.class)));
+  }
+
+  @Test
+  public void testExtractYarnResourceManagerAmRetryCountWindowMs() throws
+      Exception {
+    Map<String, String> options = new HashMap<String, String>();
+    final String expectedInterval = Integer.toString (120000);
+    options.put(ResourceKeys.YARN_RESOURCEMANAGER_AM_RETRY_COUNT_WINDOW_MS,
+        expectedInterval);
+    EasyMock.replay(mockYarnClient, yarnClientApp);
+
+    appMasterLauncher = new AppMasterLauncher("am1", SliderKeys.APP_TYPE, null,
+        null, mockYarnClient, false, null, options, tags, null);
+
+    ApplicationSubmissionContext ctx = appMasterLauncher.application
+        .getApplicationSubmissionContext();
+    String retryIntervalWindow = Long.toString(ctx
+        .getAttemptFailuresValidityInterval());
+    Assert.assertEquals(expectedInterval, retryIntervalWindow);
+  }
+
+  @Test
+  public void testExtractYarnResourceManagerAmRetryCountWindowMsDefaultValue()
+      throws Exception {
+    Map<String, String> options = new HashMap<String, String>();
+    EasyMock.replay(mockYarnClient, yarnClientApp);
+
+    appMasterLauncher = new AppMasterLauncher("am1", SliderKeys.APP_TYPE, null,
+        null, mockYarnClient, false, null, options, tags, null);
+
+    ApplicationSubmissionContext ctx = appMasterLauncher.application
+        .getApplicationSubmissionContext();
+    long retryIntervalWindow = ctx.getAttemptFailuresValidityInterval();
+    Assert.assertEquals(ResourceKeys.DEFAULT_AM_RETRY_COUNT_WINDOW_MS,
+        retryIntervalWindow);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/registry/docstore/TestPublishedConfigurationOutputter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/registry/docstore/TestPublishedConfigurationOutputter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/registry/docstore/TestPublishedConfigurationOutputter.java
new file mode 100644
index 0000000..3706182
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/core/registry/docstore/TestPublishedConfigurationOutputter.java
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.core.registry.docstore;
+
+import com.google.common.base.Charsets;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.fs.Path;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.powermock.api.easymock.PowerMock;
+import org.yaml.snakeyaml.Yaml;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.easymock.EasyMock.anyObject;
+import static org.easymock.EasyMock.expect;
+import static org.mockito.Matchers.anyString;
+import static org.powermock.api.easymock.PowerMock.createNiceMock;
+
+public class TestPublishedConfigurationOutputter {
+  private static HashMap<String, String> config = new HashMap<>();
+
+  @Rule
+  public TemporaryFolder tmpDir = new TemporaryFolder();
+
+  @Before
+  public void setup() {
+    config.put("key1", "val1");
+  }
+
+  @Test
+  public void testJson() throws IOException {
+    PublishedConfigurationOutputter configurationOutputter =
+        PublishedConfigurationOutputter.createOutputter(ConfigFormat.JSON,
+            new PublishedConfiguration("description",
+                config.entrySet()));
+
+    String output = configurationOutputter.asString().replaceAll("( |\\r|\\n)",
+        "");
+    assert "{\"key1\":\"val1\"}".equals(output);
+
+    File file = tmpDir.newFile();
+    configurationOutputter.save(file);
+
+    ObjectMapper mapper = new ObjectMapper();
+    @SuppressWarnings("unchecked")
+    Map<String, String> read = mapper.readValue(file, Map.class);
+    assert 1 == read.size();
+    assert "val1".equals(read.get("key1"));
+  }
+
+  @Test
+  public void testXml() throws IOException {
+    PublishedConfigurationOutputter configurationOutputter =
+        PublishedConfigurationOutputter.createOutputter(ConfigFormat.XML,
+            new PublishedConfiguration("description",
+                config.entrySet()));
+
+    String output = configurationOutputter.asString().replaceAll("( |\\r|\\n)",
+        "");
+    assert output.contains(
+        "<configuration><property><name>key1</name><value>val1</value><source/></property></configuration>");
+
+    File file = tmpDir.newFile();
+    configurationOutputter.save(file);
+
+    assert FileUtils.readFileToString(file, Charsets.UTF_8)
+        .replaceAll("( |\\r|\\n)", "")
+        .contains(
+            "<configuration><property><name>key1</name><value>val1</value><source/></property></configuration>");
+  }
+
+  @Test
+  public void testHadoopXml() throws IOException {
+    PublishedConfigurationOutputter configurationOutputter =
+        PublishedConfigurationOutputter.createOutputter(ConfigFormat.HADOOP_XML,
+            new PublishedConfiguration("description",
+                config.entrySet()));
+
+    String output = configurationOutputter.asString().replaceAll("( |\\r|\\n)",
+        "");
+    assert output.contains("<configuration><property><name>key1</name><value>val1</value><source/></property></configuration>");
+
+    File file = tmpDir.newFile();
+    configurationOutputter.save(file);
+
+    assert FileUtils.readFileToString(file, Charsets.UTF_8)
+        .replaceAll("( |\\r|\\n)", "")
+        .contains( "<configuration><property><name>key1</name><value>val1</value><source/></property></configuration>");
+  }
+
+  @Test
+  public void testProperties() throws IOException {
+    PublishedConfigurationOutputter configurationOutputter =
+        PublishedConfigurationOutputter.createOutputter(ConfigFormat.PROPERTIES,
+            new PublishedConfiguration("description",
+                config.entrySet()));
+
+    String output = configurationOutputter.asString();
+    assert output.contains("key1=val1");
+
+    File file = tmpDir.newFile();
+    configurationOutputter.save(file);
+
+    Properties properties = new Properties();
+    FileInputStream fis = null;
+    try {
+      fis = new FileInputStream(file);
+      properties.load(fis);
+    } finally {
+      if (fis != null) {
+        fis.close();
+      }
+    }
+    assert 1 == properties.size();
+    assert "val1".equals(properties.getProperty("key1"));
+  }
+
+  @Test
+  public void testYaml() throws IOException {
+    PublishedConfigurationOutputter configurationOutputter =
+        PublishedConfigurationOutputter.createOutputter(ConfigFormat.YAML,
+            new PublishedConfiguration("description",
+                config.entrySet()));
+
+    String output = configurationOutputter.asString().replaceAll("(\\r|\\n)",
+        "");
+    assert "key1: val1".equals(output);
+
+    File file = tmpDir.newFile();
+    configurationOutputter.save(file);
+
+    Yaml yaml = new Yaml();
+    FileInputStream fis = null;
+    Map<String, String> read;
+    try {
+      fis = new FileInputStream(file);
+      read = (Map<String, String>) yaml.load(fis);
+    } finally {
+      if (fis != null) {
+        fis.close();
+      }
+    }
+    assert 1 == read.size();
+    assert "val1".equals(read.get("key1"));
+  }
+
+  @Test
+  public void testEnv() throws IOException {
+    HashMap<String, String> envConfig = new HashMap<>(config);
+    envConfig.put("content", "content {{key1}} ");
+
+    PublishedConfigurationOutputter configurationOutputter =
+        PublishedConfigurationOutputter.createOutputter(ConfigFormat.ENV,
+            new PublishedConfiguration("description",
+                envConfig.entrySet()));
+
+    String output = configurationOutputter.asString();
+    assert "content val1 ".equals(output);
+
+    File file = tmpDir.newFile();
+    configurationOutputter.save(file);
+
+    assert "content val1 ".equals(FileUtils.readFileToString(file,
+        Charsets.UTF_8));
+  }
+
+  @Test
+  public void testTemplate1() throws IOException {
+    HashMap<String, String> templateConfig = new HashMap<>(config);
+    templateConfig.put(ConfigUtils.TEMPLATE_FILE, "templateFileName");
+
+    SliderFileSystem fileSystem = createNiceMock(SliderFileSystem.class);
+    expect(fileSystem.buildResourcePath(anyString())).andReturn(new Path("path")).anyTimes();
+    expect(fileSystem.isFile(anyObject(Path.class))).andReturn(true).anyTimes();
+    expect(fileSystem.cat(anyObject(Path.class))).andReturn("content {{key1}}\n more ${key1} content").anyTimes();
+
+    PowerMock.replay(fileSystem);
+
+    ConfigUtils.prepConfigForTemplateOutputter(ConfigFormat.TEMPLATE,
+        templateConfig, fileSystem, "clusterName", null);
+    PublishedConfigurationOutputter configurationOutputter =
+        PublishedConfigurationOutputter.createOutputter(ConfigFormat.TEMPLATE,
+            new PublishedConfiguration("description",
+                templateConfig.entrySet()));
+
+    String output = configurationOutputter.asString();
+    assert "content val1\n more val1 content".equals(output);
+
+    File file = tmpDir.newFile();
+    configurationOutputter.save(file);
+
+    PowerMock.verify(fileSystem);
+
+    assert "content val1\n more val1 content".equals(
+        FileUtils.readFileToString(file, Charsets.UTF_8));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider.java
new file mode 100644
index 0000000..0bea8fa
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.tools.TestUtility;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Set;
+
+/**
+ *
+ */
+public class TestAgentClientProvider {
+  protected static final Logger log =
+      LoggerFactory.getLogger(TestAgentClientProvider.class);
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  @Test
+  public void testGetApplicationTags() throws Exception {
+    Configuration configuration = new Configuration();
+    FileSystem fs = FileSystem.getLocal(configuration);
+    SliderFileSystem sliderFileSystem = new SliderFileSystem(fs, configuration);
+
+    AgentClientProvider provider = new AgentClientProvider(null);
+    String zipFileName = TestUtility.createAppPackage(
+        folder,
+        "testpkg",
+        "test.zip",
+        "target/test-classes/org/apache/slider/common/tools/test");
+    Set<String> tags = provider.getApplicationTags(sliderFileSystem, zipFileName);
+    assert tags != null;
+    assert !tags.isEmpty();
+    assert tags.contains("Name: STORM");
+    assert tags.contains("Description: Apache Hadoop Stream processing framework");
+    assert tags.contains("Version: 0.9.1.2.1");
+
+  }
+
+  @Test
+  public void testValidateInstanceDefinition() throws Exception {
+    AgentClientProvider provider = new AgentClientProvider(null);
+    AggregateConf instanceDefinition = new AggregateConf();
+
+    try {
+      provider.validateInstanceDefinition(instanceDefinition, null);
+      Assert.assertFalse("Should fail with BadConfigException", true);
+    } catch (BadConfigException e) {
+      log.info(e.toString());
+      Assert.assertTrue(e.getMessage().contains("Application definition must be provided"));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentLaunchParameter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentLaunchParameter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentLaunchParameter.java
new file mode 100644
index 0000000..ec62b54
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentLaunchParameter.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ *
+ */
+public class TestAgentLaunchParameter {
+  protected static final Logger log =
+      LoggerFactory.getLogger(TestAgentLaunchParameter.class);
+
+  @Test
+  public void testTestAgentLaunchParameter() throws Exception {
+    AgentLaunchParameter alp = new AgentLaunchParameter("");
+    Assert.assertEquals("", alp.getNextLaunchParameter("abc"));
+    Assert.assertEquals("", alp.getNextLaunchParameter("HBASE_MASTER"));
+
+    alp = new AgentLaunchParameter("a:1:2:3|b:5:6:NONE");
+    Assert.assertEquals("1", alp.getNextLaunchParameter("a"));
+    Assert.assertEquals("2", alp.getNextLaunchParameter("a"));
+    Assert.assertEquals("3", alp.getNextLaunchParameter("a"));
+    Assert.assertEquals("3", alp.getNextLaunchParameter("a"));
+
+    Assert.assertEquals("5", alp.getNextLaunchParameter("b"));
+    Assert.assertEquals("6", alp.getNextLaunchParameter("b"));
+    Assert.assertEquals("", alp.getNextLaunchParameter("b"));
+    Assert.assertEquals("", alp.getNextLaunchParameter("b"));
+    Assert.assertEquals("", alp.getNextLaunchParameter("c"));
+
+    alp = new AgentLaunchParameter("|a:1:3|b::5:NONE:");
+    Assert.assertEquals("1", alp.getNextLaunchParameter("a"));
+    Assert.assertEquals("3", alp.getNextLaunchParameter("a"));
+    Assert.assertEquals("3", alp.getNextLaunchParameter("a"));
+
+    Assert.assertEquals("", alp.getNextLaunchParameter("b"));
+    Assert.assertEquals("5", alp.getNextLaunchParameter("b"));
+    Assert.assertEquals("", alp.getNextLaunchParameter("b"));
+    Assert.assertEquals("", alp.getNextLaunchParameter("b"));
+
+    alp = new AgentLaunchParameter("|:");
+    Assert.assertEquals("", alp.getNextLaunchParameter("b"));
+    Assert.assertEquals("", alp.getNextLaunchParameter("a"));
+
+    alp = new AgentLaunchParameter("HBASE_MASTER:a,b:DO_NOT_REGISTER:");
+    Assert.assertEquals("a,b", alp.getNextLaunchParameter("HBASE_MASTER"));
+    Assert.assertEquals("DO_NOT_REGISTER", alp.getNextLaunchParameter("HBASE_MASTER"));
+    Assert.assertEquals("DO_NOT_REGISTER", alp.getNextLaunchParameter("HBASE_MASTER"));
+
+    alp = new AgentLaunchParameter("HBASE_MASTER:a,b:DO_NOT_REGISTER::c:::");
+    Assert.assertEquals("a,b", alp.getNextLaunchParameter("HBASE_MASTER"));
+    Assert.assertEquals("DO_NOT_REGISTER", alp.getNextLaunchParameter("HBASE_MASTER"));
+    Assert.assertEquals("", alp.getNextLaunchParameter("HBASE_MASTER"));
+    Assert.assertEquals("c", alp.getNextLaunchParameter("HBASE_MASTER"));
+    Assert.assertEquals("c", alp.getNextLaunchParameter("HBASE_MASTER"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentUtils.java
new file mode 100644
index 0000000..5e1dc7f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentUtils.java
@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers.agent;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.providers.agent.application.metadata.Metainfo;
+import org.apache.slider.tools.TestUtility;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestAgentUtils {
+  protected static final Logger log =
+      LoggerFactory.getLogger(TestAgentUtils.class);
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private static final String metainfo_str = "<metainfo>\n"
+      + "  <schemaVersion>2.0</schemaVersion>\n"
+      + "  <application>\n"
+      + "      <name>MYTESTAPPLICATION</name>\n"
+      + "      <comment>\n"
+      + "        My Test Application\n"
+      + "      </comment>\n"
+      + "      <version>1.0</version>\n"
+      + "      <type>YARN-APP</type>\n"
+      + "      <components>\n"
+      + "        <component>\n"
+      + "          <name>REST</name>\n"
+      + "          <category>MASTER</category>\n"
+      + "          <commandScript>\n"
+      + "            <script>scripts/rest.py</script>\n"
+      + "            <scriptType>PYTHON</scriptType>\n"
+      + "            <timeout>600</timeout>\n"
+      + "          </commandScript>\n"
+      + "        </component>\n"
+      + "      </components>\n"
+      + "  </application>\n"
+      + "</metainfo>";
+
+  @Test
+  public void testGetApplicationMetainfo() throws Exception {
+    String zipFileName = TestUtility.createAppPackage(
+        folder,
+        "testpkg",
+        "test.zip",
+        "target/test-classes/org/apache/slider/common/tools/test");
+    Configuration configuration = new Configuration();
+    FileSystem fs = FileSystem.getLocal(configuration);
+    log.info("fs working dir is {}", fs.getWorkingDirectory().toString());
+    SliderFileSystem sliderFileSystem = new SliderFileSystem(fs, configuration);
+
+    // Without accompany metainfo file, read metainfo from the zip file
+    Metainfo metainfo = AgentUtils.getApplicationMetainfo(
+        sliderFileSystem, zipFileName, false);
+    Assert.assertNotNull(metainfo.getApplication());
+    Assert.assertEquals("STORM", metainfo.getApplication().getName());
+
+    // With accompany metainfo file, read metainfo from the accompany file
+    String acompanyFileName = zipFileName + ".metainfo.xml";
+    File f = new File(acompanyFileName);
+    try (BufferedWriter writer = new BufferedWriter(new FileWriter(f))) {
+      writer.write(metainfo_str);
+    }
+    metainfo = AgentUtils.getApplicationMetainfo(
+        sliderFileSystem, zipFileName, false);
+    Assert.assertNotNull(metainfo.getApplication());
+    Assert.assertEquals("MYTESTAPPLICATION", metainfo.getApplication().getName());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAppDefinitionPersister.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAppDefinitionPersister.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAppDefinitionPersister.java
new file mode 100644
index 0000000..dedf4f6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestAppDefinitionPersister.java
@@ -0,0 +1,264 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent;
+
+import com.google.common.io.Files;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.slider.common.params.ActionCreateArgs;
+import org.apache.slider.common.params.AddonArgsDelegate;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.core.conf.ConfTree;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.core.persist.AppDefinitionPersister;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ *
+ */
+public class TestAppDefinitionPersister {
+  protected static final Logger log =
+      LoggerFactory.getLogger(TestAppDefinitionPersister.class);
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  /**
+   * @BeforeClass public static void initialize() { BasicConfigurator.resetConfiguration();
+   * BasicConfigurator.configure(); }*
+   */
+
+
+  @Test
+  public void testAppDefinitionPersister() throws Exception {
+    Configuration configuration = new Configuration();
+    FileSystem fs = FileSystem.getLocal(configuration);
+    log.info("fs working dir is {}", fs.getWorkingDirectory().toString());
+    SliderFileSystem sliderFileSystem = new SliderFileSystem(fs, configuration);
+
+    AppDefinitionPersister adp = new AppDefinitionPersister(sliderFileSystem);
+    String clustername = "c1";
+    ActionCreateArgs buildInfo = new ActionCreateArgs();
+    buildInfo.appMetaInfo = null;
+    buildInfo.appDef = null;
+    buildInfo.addonDelegate = new AddonArgsDelegate();
+
+    // nothing to do
+    adp.processSuppliedDefinitions(clustername, buildInfo, null);
+    adp.persistPackages();
+    List<AppDefinitionPersister.AppDefinition> appDefinitions = adp.getAppDefinitions();
+    Assert.assertTrue(appDefinitions.size() == 0);
+
+    ConfTree ct = new ConfTree();
+    ConfTreeOperations appConf = new ConfTreeOperations(ct);
+    final File tempDir = Files.createTempDir();
+    final File metainfo = new File(tempDir, "metainfo.json");
+
+    // unreadable metainfo
+    buildInfo.appMetaInfo = metainfo;
+
+    try {
+      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    } catch (BadConfigException bce) {
+      log.info(bce.getMessage());
+      Assert.assertTrue(bce.getMessage().contains(
+          "Path specified with "
+              + "--metainfo either cannot be read or is not a file"));
+    }
+
+    try (PrintWriter writer = new PrintWriter(metainfo.getAbsolutePath(), "UTF-8")) {
+      writer.println("{");
+      writer.println("}");
+    }
+    buildInfo.appDef = metainfo;
+
+    try {
+      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    } catch (BadConfigException bce) {
+      log.info(bce.getMessage());
+      Assert.assertTrue(bce.getMessage().contains(
+          "Both --metainfo and --appdef cannot be specified"));
+    }
+
+    // both --metainfojson and --appdef cannot be specified
+    buildInfo.appMetaInfo = null;
+    buildInfo.appMetaInfoJson = "{}";
+    try {
+      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    } catch (BadConfigException bce) {
+      log.info(bce.getMessage());
+      Assert.assertTrue(bce.getMessage().contains(
+          "Both --metainfojson and --appdef cannot be specified"));
+    }
+
+    buildInfo.appDef = null;
+
+    buildInfo.appMetaInfoJson = "";
+    try {
+      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    } catch (BadConfigException bce) {
+      log.info(bce.getMessage());
+      Assert.assertTrue(bce.getMessage().contains(
+          "Empty string specified with --metainfojson"));
+    }
+    buildInfo.appMetaInfo = metainfo;
+
+    // both --metainfo and --metainfojson cannot be specified
+    buildInfo.appMetaInfoJson = "{}";
+    try {
+      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    } catch (BadConfigException bce) {
+      log.info(bce.getMessage());
+      Assert.assertTrue(bce.getMessage().contains(
+          "Both --metainfo and --metainfojson cannot be specified"));
+    }
+    buildInfo.appMetaInfoJson = null;
+
+    appConf.getGlobalOptions().set(AgentKeys.APP_DEF, metainfo.getAbsolutePath());
+
+    try {
+      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    } catch (BadConfigException bce) {
+      log.info(bce.getMessage());
+      Assert.assertTrue(bce.getMessage().contains(
+          "application.def cannot "
+              + "not be set if --metainfo is specified in the cmd line"));
+    }
+
+    appConf.getGlobalOptions().remove(AgentKeys.APP_DEF);
+
+    adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    appDefinitions = adp.getAppDefinitions();
+    Assert.assertTrue(appDefinitions.size() == 1);
+    Assert.assertTrue(appConf.getGlobalOptions().get(AgentKeys.APP_DEF).contains("appdef/appPkg.zip"));
+    log.info(appDefinitions.get(0).toString());
+    Assert.assertTrue(appDefinitions.get(0).appDefPkgOrFolder.toString().endsWith("default"));
+    Assert.assertTrue(appDefinitions.get(0).targetFolderInFs.toString().contains("cluster/c1/appdef"));
+    Assert.assertEquals("appPkg.zip", appDefinitions.get(0).pkgName);
+
+    buildInfo.appDef = tempDir;
+    buildInfo.appMetaInfo = null;
+
+    appConf.getGlobalOptions().set(AgentKeys.APP_DEF, metainfo.getAbsolutePath());
+
+    try {
+      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    } catch (BadConfigException bce) {
+      log.info(bce.getMessage());
+      Assert.assertTrue(bce.getMessage().contains("application.def must not be set if --appdef is provided"));
+    }
+
+    adp.getAppDefinitions().clear();
+    appConf.getGlobalOptions().remove(AgentKeys.APP_DEF);
+    adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    appDefinitions = adp.getAppDefinitions();
+    Assert.assertTrue(appDefinitions.size() == 1);
+    Assert.assertTrue(appConf.getGlobalOptions().get(AgentKeys.APP_DEF).contains("appdef/appPkg.zip"));
+    log.info(appDefinitions.get(0).toString());
+    Assert.assertTrue(appDefinitions.get(0).appDefPkgOrFolder.toString().endsWith(tempDir.toString()));
+    Assert.assertTrue(appDefinitions.get(0).targetFolderInFs.toString().contains("cluster/c1/appdef"));
+    Assert.assertEquals("appPkg.zip", appDefinitions.get(0).pkgName);
+
+    adp.getAppDefinitions().clear();
+    buildInfo.appDef = null;
+    buildInfo.appMetaInfo = null;
+    appConf.getGlobalOptions().remove(AgentKeys.APP_DEF);
+
+    ArrayList<String> list = new ArrayList<String>() {{
+      add("addon1");
+      add("");
+      add("addon2");
+      add(metainfo.getAbsolutePath());
+    }};
+
+    buildInfo.addonDelegate.addonTuples = list;
+    try {
+      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    } catch (BadConfigException bce) {
+      log.info(bce.getMessage());
+      Assert.assertTrue(bce.getMessage().contains("addon package can only be specified if main app package is specified"));
+    }
+
+    buildInfo.appMetaInfo = metainfo;
+
+    try {
+      adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    } catch (BadConfigException bce) {
+      log.info(bce.getMessage());
+      Assert.assertTrue(bce.getMessage().contains("Invalid path for addon package addon1"));
+    }
+
+    appConf.getGlobalOptions().remove(AgentKeys.APP_DEF);
+
+    list = new ArrayList<String>() {{
+      add("addon1");
+      add(tempDir.getAbsolutePath());
+      add("addon2");
+      add(metainfo.getAbsolutePath());
+    }};
+
+    buildInfo.addonDelegate.addonTuples = list;
+    adp.getAppDefinitions().clear();
+
+    adp.processSuppliedDefinitions(clustername, buildInfo, appConf);
+    appDefinitions = adp.getAppDefinitions();
+
+    Assert.assertTrue(appDefinitions.size() == 3);
+    Assert.assertTrue(appConf.getGlobalOptions().get(AgentKeys.APP_DEF).contains("appdef/appPkg.zip"));
+    Assert.assertTrue(appConf.getGlobalOptions().get("application.addon.addon1").contains(
+        "addons/addon1/addon_addon1.zip"));
+    Assert.assertTrue(appConf.getGlobalOptions().get("application.addon.addon2").contains(
+        "addons/addon2/addon_addon2.zip"));
+    log.info(appConf.getGlobalOptions().get("application.addons"));
+    Assert.assertTrue(appConf.getGlobalOptions().get("application.addons").contains(
+        "application.addon.addon2,application.addon.addon1")
+                      || appConf.getGlobalOptions().get("application.addons").contains(
+        "application.addon.addon1,application.addon.addon2"));
+    int seen = 0;
+    for (AppDefinitionPersister.AppDefinition adp_ad : appDefinitions) {
+      if (adp_ad.pkgName.equals("appPkg.zip")) {
+        log.info(adp_ad.toString());
+        Assert.assertTrue(adp_ad.appDefPkgOrFolder.toString().endsWith("default"));
+        Assert.assertTrue(adp_ad.targetFolderInFs.toString().contains("cluster/c1/appdef"));
+        seen++;
+      }
+      if (adp_ad.pkgName.equals("addon_addon1.zip")) {
+        log.info(adp_ad.toString());
+        Assert.assertTrue(adp_ad.appDefPkgOrFolder.toString().endsWith(tempDir.toString()));
+        Assert.assertTrue(adp_ad.targetFolderInFs.toString().contains("addons/addon1"));
+        seen++;
+      }
+      if (adp_ad.pkgName.equals("addon_addon2.zip")) {
+        log.info(adp_ad.toString());
+        Assert.assertTrue(adp_ad.appDefPkgOrFolder.toString().endsWith("metainfo.json"));
+        Assert.assertTrue(adp_ad.targetFolderInFs.toString().contains("addons/addon2"));
+        seen++;
+      }
+    }
+    Assert.assertEquals(3, seen);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestComponentTagProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestComponentTagProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestComponentTagProvider.java
new file mode 100644
index 0000000..7b38ee3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestComponentTagProvider.java
@@ -0,0 +1,115 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers.agent;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class TestComponentTagProvider {
+  protected static final Logger log =
+      LoggerFactory.getLogger(TestComponentTagProvider.class);
+
+  @Test
+  public void testTagProvider() throws Exception {
+    ComponentTagProvider ctp = new ComponentTagProvider();
+    Assert.assertEquals("", ctp.getTag(null, null));
+    Assert.assertEquals("", ctp.getTag(null, "cid"));
+    Assert.assertEquals("", ctp.getTag("comp1", null));
+
+    Assert.assertEquals("1", ctp.getTag("comp1", "cid1"));
+    Assert.assertEquals("2", ctp.getTag("comp1", "cid2"));
+    Assert.assertEquals("3", ctp.getTag("comp1", "cid3"));
+    ctp.releaseTag("comp1", "cid2");
+    Assert.assertEquals("2", ctp.getTag("comp1", "cid22"));
+
+    ctp.releaseTag("comp1", "cid4");
+    ctp.recordAssignedTag("comp1", "cid5", "5");
+    Assert.assertEquals("4", ctp.getTag("comp1", "cid4"));
+    Assert.assertEquals("4", ctp.getTag("comp1", "cid4"));
+    Assert.assertEquals("6", ctp.getTag("comp1", "cid6"));
+
+    ctp.recordAssignedTag("comp1", "cid55", "5");
+    Assert.assertEquals("5", ctp.getTag("comp1", "cid55"));
+
+    ctp.recordAssignedTag("comp2", "cidb3", "3");
+    Assert.assertEquals("1", ctp.getTag("comp2", "cidb1"));
+    Assert.assertEquals("2", ctp.getTag("comp2", "cidb2"));
+    Assert.assertEquals("4", ctp.getTag("comp2", "cidb4"));
+
+    ctp.recordAssignedTag("comp2", "cidb5", "six");
+    ctp.recordAssignedTag("comp2", "cidb5", "-55");
+    ctp.recordAssignedTag("comp2", "cidb5", "tags");
+    ctp.recordAssignedTag("comp2", "cidb5", null);
+    ctp.recordAssignedTag("comp2", "cidb5", "");
+    ctp.recordAssignedTag("comp2", "cidb5", "5");
+    Assert.assertEquals("6", ctp.getTag("comp2", "cidb6"));
+
+    ctp.recordAssignedTag("comp2", null, "5");
+    ctp.recordAssignedTag(null, null, "5");
+    ctp.releaseTag("comp1", null);
+    ctp.releaseTag(null, "cid4");
+    ctp.releaseTag(null, null);
+  }
+
+  @Test
+  public void testTagProviderWithThread() throws Exception {
+    ComponentTagProvider ctp = new ComponentTagProvider();
+    Thread thread = new Thread(new Taggged(ctp));
+    Thread thread2 = new Thread(new Taggged(ctp));
+    Thread thread3 = new Thread(new Taggged(ctp));
+    thread.start();
+    thread2.start();
+    thread3.start();
+    ctp.getTag("comp1", "cid50");
+    thread.join();
+    thread2.join();
+    thread3.join();
+    Assert.assertEquals("101", ctp.getTag("comp1", "cid101"));
+  }
+
+  public class Taggged implements Runnable {
+    private final ComponentTagProvider ctp;
+
+    public Taggged(ComponentTagProvider ctp) {
+      this.ctp = ctp;
+    }
+
+    public void run() {
+      for (int i = 0; i < 100; i++) {
+        String containerId = "cid" + (i + 1);
+        this.ctp.getTag("comp1", containerId);
+      }
+      for (int i = 0; i < 100; i++) {
+        String containerId = "cid" + (i + 1);
+        this.ctp.getTag("comp1", containerId);
+      }
+      for (int i = 0; i < 100; i += 2) {
+        String containerId = "cid" + (i + 1);
+        this.ctp.releaseTag("comp1", containerId);
+      }
+      for (int i = 0; i < 100; i += 2) {
+        String containerId = "cid" + (i + 1);
+        this.ctp.getTag("comp1", containerId);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestState.java
new file mode 100644
index 0000000..6a2e5ab5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/TestState.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestState {
+  protected static final Logger log = LoggerFactory.getLogger(TestState.class);
+
+  @Test
+  public void testState() throws Exception {
+    State state = State.STARTED;
+    Assert.assertEquals(Command.STOP, state.getSupportedCommand(false, true));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestConfigParser.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestConfigParser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestConfigParser.java
new file mode 100644
index 0000000..3aa44a1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestConfigParser.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ *
+ */
+public class TestConfigParser {
+  protected static final Logger log =
+      LoggerFactory.getLogger(TestConfigParser.class);
+  private static final String config_1_str = "<configuration>\n"
+                                             + "  <property>\n"
+                                             + "    <name>security.client.protocol.acl</name>\n"
+                                             + "    <value>*</value>\n"
+                                             + "    <description>ACL for HRegionInterface protocol implementations (ie. \n"
+                                             + "    clients talking to HRegionServers)\n"
+                                             + "    The ACL is a comma-separated list of user and group names. The user and \n"
+                                             + "    group list is separated by a blank. For e.g. \"alice,bob users,wheel\". \n"
+                                             + "    A special value of \"*\" means all users are allowed.</description>\n"
+                                             + "  </property>\n"
+                                             + "\n"
+                                             + "  <property>\n"
+                                             + "    <name>security.admin.protocol.acl</name>\n"
+                                             + "    <value>*</value>\n"
+                                             + "    <description>ACL for HMasterInterface protocol implementation (ie. \n"
+                                             + "    clients talking to HMaster for admin operations).\n"
+                                             + "    The ACL is a comma-separated list of user and group names. The user and \n"
+                                             + "    group list is separated by a blank. For e.g. \"alice,bob users,wheel\". \n"
+                                             + "    A special value of \"*\" means all users are allowed.</description>\n"
+                                             + "  </property>\n"
+                                             + "\n"
+                                             + "  <property>\n"
+                                             + "    <name>security.masterregion.protocol.acl</name>\n"
+                                             + "    <value>*</value>\n"
+                                             + "    <description>ACL for HMasterRegionInterface protocol implementations\n"
+                                             + "    (for HRegionServers communicating with HMaster)\n"
+                                             + "    The ACL is a comma-separated list of user and group names. The user and \n"
+                                             + "    group list is separated by a blank. For e.g. \"alice,bob users,wheel\". \n"
+                                             + "    A special value of \"*\" means all users are allowed.</description>\n"
+                                             + "  </property>\n"
+                                             + "  <property>\n"
+                                             + "    <name>emptyVal</name>\n"
+                                             + "    <value></value>\n"
+                                             + "    <description>non-empty-desc</description>\n"
+                                             + "  </property>\n"
+                                             + "  <property>\n"
+                                             + "    <name>emptyDesc</name>\n"
+                                             + "    <value></value>\n"
+                                             + "    <description></description>\n"
+                                             + "  </property>\n"
+                                             + "  <property>\n"
+                                             + "    <name>noDesc</name>\n"
+                                             + "    <value></value>\n"
+                                             + "  </property>\n"
+                                             + "</configuration>";
+
+  @Test
+  public void testParse() throws IOException {
+
+    InputStream config_1 = new ByteArrayInputStream(config_1_str.getBytes());
+    DefaultConfig config = new DefaultConfigParser().parse(config_1);
+    Assert.assertNotNull(config);
+    Assert.assertNotNull(config.getPropertyInfos());
+    Assert.assertEquals(6, config.getPropertyInfos().size());
+    for (PropertyInfo pInfo : config.getPropertyInfos()) {
+      if (pInfo.getName().equals("security.client.protocol.acl")) {
+        Assert.assertEquals("*", pInfo.getValue());
+        Assert.assertTrue(pInfo.getDescription().startsWith("ACL for HRegionInterface "));
+      }
+      if (pInfo.getName().equals("emptyVal")) {
+        Assert.assertEquals("", pInfo.getValue());
+        Assert.assertEquals("non-empty-desc", pInfo.getDescription());
+      }
+      if (pInfo.getName().equals("emptyDesc")) {
+        Assert.assertEquals("", pInfo.getValue());
+        Assert.assertEquals("", pInfo.getDescription());
+      }
+      if (pInfo.getName().equals("noDesc")) {
+        Assert.assertEquals("", pInfo.getValue());
+        Assert.assertNull(pInfo.getDescription());
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestMetainfoParser.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestMetainfoParser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestMetainfoParser.java
new file mode 100644
index 0000000..ba1912a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestMetainfoParser.java
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers.agent.application.metadata;
+
+import org.apache.slider.providers.agent.AgentProviderService;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.List;
+
+import static org.mockito.Mockito.doReturn;
+
+/**
+ *
+ */
+public class TestMetainfoParser {
+  protected static final Logger log =
+      LoggerFactory.getLogger(TestMetainfoParser.class);
+  public static final String METAINFO_XML =
+      "/org/apache/slider/providers/agent/application/metadata/metainfo.xml";
+
+  @Test
+  public void testParse() throws IOException {
+
+    InputStream resStream = this.getClass().getResourceAsStream(
+        METAINFO_XML);
+    MetainfoParser parser = new MetainfoParser();
+    Metainfo metainfo = parser.fromXmlStream(resStream);
+    Assert.assertNotNull(metainfo);
+    Assert.assertNotNull(metainfo.getApplication());
+    Application application = metainfo.getApplication();
+    assert "STORM".equals(application.getName());
+    assert 6 == application.getComponents().size();
+    OSPackage pkg = application.getOSSpecifics().get(0).getPackages().get(0);
+    assert "tarball".equals(pkg.getType());
+    assert "files/apache-storm-0.9.1.2.1.1.0-237.tar.gz".equals(pkg.getName());
+    boolean found = false;
+    for (Component comp : application.getComponents()) {
+      if (comp != null && comp.getName().equals("NIMBUS")) {
+        found = true;
+        Assert.assertEquals(0, comp.getComponentExports().size());
+      }
+      if (comp != null && comp.getName().equals("SUPERVISOR")) {
+        Assert.assertEquals(1, comp.getComponentExports().size());
+      }
+      if (comp != null && comp.getName().equals("ANOTHER_COMPONENT")) {
+        assert 2 == comp.getCommands().size();
+        assert "start command".equals(comp.getCommands().get(0).getExec());
+        assert "START".equals(comp.getCommands().get(0).getName());
+        assert "stop command".equals(comp.getCommands().get(1).getExec());
+        assert "STOP".equals(comp.getCommands().get(1).getName());
+      }
+    }
+    assert found;
+    Assert.assertEquals(0, application.getConfigFiles().size());
+    assert 1 == application.getPackages().size();
+    Package p = application.getPackages().get(0);
+    assert "tarball".equals(p.getType());
+    assert "test-tarball-name.tgz".equals(p.getName());
+  }
+
+  @Test
+  public void testJsonParse() throws IOException {
+    String metaInfo1_json = "{\n"
+                            + "\"schemaVersion\":\"2.2\",\n"
+                            + "\"application\":{\n"
+                            +     "\"name\": \"MEMCACHED\","
+                            +     "\"exportGroups\": ["
+                            +        "{"
+                            +          "\"name\": \"Servers\","
+                            +          "\"exports\": ["
+                            +            "{"
+                            +               "\"name\": \"host_port\","
+                            +               "\"value\": \"${MEMCACHED_HOST}:${site.global.port}\""
+                            +            "}"
+                            +          "]"
+                            +        "}"
+                            +      "],"
+                            +     "\"components\": ["
+                            +        "{"
+                            +          "\"name\": \"MEMCACHED\","
+                            +          "\"compExports\": \"Servers-host_port\","
+                            +          "\"commands\": ["
+                            +            "{"
+                            +               "\"exec\": \"java -classpath /usr/myapps/memcached/*:/usr/lib/hadoop/lib/* com.thimbleware.jmemcached.Main\""
+                            +            "}"
+                            +          "]"
+                            +        "},"
+                            +        "{"
+                            +          "\"name\": \"MEMCACHED2\","
+                            +          "\"commands\": ["
+                            +            "{"
+                            +               "\"exec\": \"scripts/config.py\","
+                            +               "\"type\": \"PYTHON\","
+                            +               "\"name\": \"CONFIGURE\""
+                            +            "}"
+                            +          "],"
+                            +          "\"dockerContainers\": ["
+                            +            "{"
+                            +               "\"name\": \"redis\","
+                            +               "\"image\": \"dockerhub/redis\","
+                            +               "\"options\": \"-net=bridge\","
+                            +               "\"mounts\": ["
+                            +                 "{"
+                            +                   "\"containerMount\": \"/tmp/conf\","
+                            +                   "\"hostMount\": \"{$conf:@//site/global/app_root}/conf\""
+                            +                 "}"
+                            +               "]"
+                            +            "}"
+                            +          "]"
+                            +        "}"
+                            +      "]"
+                            +   "}"
+                            + "}";
+
+    MetainfoParser parser = new MetainfoParser();
+    Metainfo mInfo = parser.fromJsonString(metaInfo1_json);
+    Assert.assertEquals("2.2", mInfo.getSchemaVersion());
+
+    Application app = mInfo.getApplication();
+    Assert.assertNotNull(app);
+
+    Assert.assertEquals("MEMCACHED", app.getName());
+    List<ExportGroup> egs = app.getExportGroups();
+    Assert.assertEquals(1, egs.size());
+    ExportGroup eg = egs.get(0);
+    Assert.assertEquals("Servers", eg.getName());
+    List<Export> exports = eg.getExports();
+    Assert.assertEquals(1, exports.size());
+    Export export = exports.get(0);
+    Assert.assertEquals("host_port", export.getName());
+    Assert.assertEquals("${MEMCACHED_HOST}:${site.global.port}", export.getValue());
+
+    List<Component> components = app.getComponents();
+    Assert.assertEquals(2, components.size());
+
+    Component c1 = mInfo.getApplicationComponent("MEMCACHED");
+    Assert.assertNotNull(c1);
+    Assert.assertEquals("MEMCACHED", c1.getName());
+    Assert.assertEquals("Servers-host_port", c1.getCompExports());
+    Assert.assertEquals(1, c1.getCommands().size());
+    ComponentCommand cmd = c1.getCommands().get(0);
+    Assert.assertEquals("START", cmd.getName());
+    Assert.assertEquals("SHELL", cmd.getType());
+    Assert.assertEquals("java -classpath /usr/myapps/memcached/*:/usr/lib/hadoop/lib/* com.thimbleware.jmemcached.Main",
+                        cmd.getExec());
+
+    Component c2 = mInfo.getApplicationComponent("MEMCACHED2");
+    Assert.assertNotNull(c2);
+    Assert.assertEquals("MEMCACHED2", c2.getName());
+    Assert.assertEquals(1, c2.getCommands().size());
+    cmd = c2.getCommands().get(0);
+    Assert.assertEquals("CONFIGURE", cmd.getName());
+    Assert.assertEquals("PYTHON", cmd.getType());
+    Assert.assertEquals("scripts/config.py", cmd.getExec());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/TestServiceRecordAttributes.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/TestServiceRecordAttributes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/TestServiceRecordAttributes.java
new file mode 100644
index 0000000..a1986cd
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/server/appmaster/TestServiceRecordAttributes.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster;
+
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.core.conf.MapOperations;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ *
+ */
+public class TestServiceRecordAttributes extends Assert {
+
+  @Test
+  public void testAppConfigProvidedServiceRecordAttributes() throws Exception {
+    Map<String, String> options = new HashMap<>();
+    options.put("slider.some.arbitrary.option", "arbitrary value");
+    options.put("service.record.attribute.one_attribute", "one_attribute_value");
+    options.put("service.record.attribute.second_attribute", "second_attribute_value");
+    MapOperations serviceProps = new MapOperations(SliderKeys.COMPONENT_AM, options);
+    options = new HashMap<>();
+    options.put("some.component.attribute", "component_attribute_value");
+    options.put("service.record.attribute.component_attribute", "component_attribute_value");
+    MapOperations compProps = new MapOperations("TEST_COMP", options);
+
+    SliderAppMaster appMaster = new SliderAppMaster();
+
+    ServiceRecord appServiceRecord = new ServiceRecord();
+
+    appMaster.setProvidedServiceRecordAttributes(serviceProps, appServiceRecord);
+
+    assertNull("property should not be attribute",
+               appServiceRecord.get("slider.some.arbitrary.option"));
+    assertEquals("wrong value", "one_attribute_value",
+                 appServiceRecord.get("one_attribute"));
+    assertEquals("wrong value", "second_attribute_value",
+                 appServiceRecord.get("second_attribute"));
+
+    ServiceRecord compServiceRecord = new ServiceRecord();
+
+    appMaster.setProvidedServiceRecordAttributes(compProps, compServiceRecord);
+
+    assertNull("should not be attribute",
+               compServiceRecord.get("some.component.attribute"));
+    assertEquals("wrong value", "component_attribute_value",
+                 compServiceRecord.get("component_attribute"));
+
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[37/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ClusterDescription.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ClusterDescription.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ClusterDescription.java
new file mode 100644
index 0000000..f8e5e7c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ClusterDescription.java
@@ -0,0 +1,795 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.api;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.slider.api.types.ApplicationLivenessInformation;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.providers.SliderProviderFactory;
+import org.codehaus.jackson.JsonGenerationException;
+import org.codehaus.jackson.JsonParseException;
+import org.codehaus.jackson.annotate.JsonIgnore;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.JsonMappingException;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.SerializationConfig;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.slider.api.OptionKeys.INTERNAL_APPLICATION_HOME;
+import static org.apache.slider.api.OptionKeys.INTERNAL_APPLICATION_IMAGE_PATH;
+import static org.apache.slider.api.OptionKeys.ZOOKEEPER_PATH;
+import static org.apache.slider.api.OptionKeys.ZOOKEEPER_QUORUM;
+
+/**
+ * Represents a cluster specification; designed to be sendable over the wire
+ * and persisted in JSON by way of Jackson.
+ * 
+ * When used in cluster status operations the <code>info</code>
+ * and <code>statistics</code> maps contain information about the cluster.
+ * 
+ * As a wire format it is less efficient in both xfer and ser/deser than 
+ * a binary format, but by having one unified format for wire and persistence,
+ * the code paths are simplified.
+ *
+ * This was the original single-file specification/model used in the Hoya
+ * precursor to Slider. Its now retained primarily as a way to publish
+ * the current state of the application, or at least a fraction thereof ...
+ * the larger set of information from the REST API is beyond the scope of
+ * this structure.
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+
+public class ClusterDescription implements Cloneable {
+  protected static final Logger
+    log = LoggerFactory.getLogger(ClusterDescription.class);
+
+  private static final String UTF_8 = "UTF-8";
+
+  /**
+   * version counter
+   */
+  public String version = "1.0";
+
+  /**
+   * Name of the cluster
+   */
+  public String name;
+
+  /**
+   * Type of cluster
+   */
+  public String type = SliderProviderFactory.DEFAULT_CLUSTER_TYPE;
+
+  /**
+   * State of the cluster
+   */
+  public int state;
+  
+  /*
+   State list for both clusters and nodes in them. Ordered so that destroyed follows
+   stopped.
+   
+   Some of the states are only used for recording
+   the persistent state of the cluster and are not
+   seen in node descriptions
+   */
+
+  /**
+   * Specification is incomplete & cannot
+   * be used: {@value}
+   */
+  public static final int STATE_INCOMPLETE = StateValues.STATE_INCOMPLETE;
+
+  /**
+   * Spec has been submitted: {@value}
+   */
+  public static final int STATE_SUBMITTED = StateValues.STATE_SUBMITTED;
+  /**
+   * Cluster created: {@value}
+   */
+  public static final int STATE_CREATED = StateValues.STATE_CREATED;
+  /**
+   * Live: {@value}
+   */
+  public static final int STATE_LIVE = StateValues.STATE_LIVE;
+  /**
+   * Stopped
+   */
+  public static final int STATE_STOPPED = StateValues.STATE_STOPPED;
+  /**
+   * destroyed
+   */
+  public static final int STATE_DESTROYED = StateValues.STATE_DESTROYED;
+  
+  /**
+   * When was the cluster specification created?
+   * This is not the time a cluster was thawed; that will
+   * be in the <code>info</code> section.
+   */
+  public long createTime;
+
+  /**
+   * When was the cluster specification last updated
+   */
+  public long updateTime;
+
+  /**
+   * URL path to the original configuration
+   * files; these are re-read when 
+   * restoring a cluster
+   */
+
+  public String originConfigurationPath;
+
+  /**
+   * URL path to the generated configuration
+   */
+  public String generatedConfigurationPath;
+
+  /**
+   * This is where the data goes
+   */
+  public String dataPath;
+
+  /**
+   * cluster-specific options -to control both
+   * the Slider AM and the application that it deploys
+   */
+  public Map<String, String> options = new HashMap<>();
+
+  /**
+   * cluster information
+   * This is only valid when querying the cluster status.
+   */
+  public Map<String, String> info = new HashMap<>();
+
+  /**
+   * Statistics. This is only relevant when querying the cluster status
+   */
+  public Map<String, Map<String, Integer>> statistics = new HashMap<>();
+
+  /**
+   * Instances: role->count
+   */
+  public Map<String, List<String>> instances = new HashMap<>();
+
+  /**
+   * Role options, 
+   * role -> option -> value
+   */
+  public Map<String, Map<String, String>> roles = new HashMap<>();
+
+
+  /**
+   * List of key-value pairs to add to a client config to set up the client
+   */
+  public Map<String, String> clientProperties = new HashMap<>();
+
+  /**
+   * Status information
+   */
+  public Map<String, Object> status;
+
+  /**
+   * Liveness information; the same as returned
+   * on the <code>live/liveness/</code> URL
+   */
+  public ApplicationLivenessInformation liveness;
+
+  /**
+   * Creator.
+   */
+  public ClusterDescription() {
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return toJsonString();
+    } catch (Exception e) {
+      log.debug("Failed to convert CD to JSON ", e);
+      return super.toString();
+    }
+  }
+
+  /**
+   * Shallow clone
+   * @return a shallow clone
+   * @throws CloneNotSupportedException
+   */
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    return super.clone();
+  }
+
+  /**
+   * A deep clone of the spec. This is done inefficiently with a ser/derser
+   * @return the cluster description
+   */
+  public ClusterDescription deepClone() {
+    try {
+      return fromJson(toJsonString());
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+
+  /**
+   * Save a cluster description to a hadoop filesystem
+   * @param fs filesystem
+   * @param path path
+   * @param overwrite should any existing file be overwritten
+   * @throws IOException IO exception
+   */
+  public void save(FileSystem fs, Path path, boolean overwrite) throws
+                                                                IOException {
+    FSDataOutputStream dataOutputStream = fs.create(path, overwrite);
+    writeJsonAsBytes(dataOutputStream);
+  }
+  
+  /**
+   * Save a cluster description to the local filesystem
+   * @param file file
+   * @throws IOException IO excpetion
+   */
+  public void save(File file) throws IOException {
+    log.debug("Saving to {}", file.getAbsolutePath());
+    if (!file.getParentFile().mkdirs()) {
+      log.warn("Failed to mkdirs for {}", file.getParentFile());
+    }
+    DataOutputStream dataOutputStream = new DataOutputStream(new FileOutputStream(file));
+    writeJsonAsBytes(dataOutputStream);
+  }
+
+  /**
+   * Write the json as bytes -then close the file
+   * @param dataOutputStream an outout stream that will always be closed
+   * @throws IOException any failure
+   */
+  private void writeJsonAsBytes(DataOutputStream dataOutputStream)
+      throws IOException {
+    try {
+      String json = toJsonString();
+      byte[] b = json.getBytes(UTF_8);
+      dataOutputStream.write(b);
+    } finally {
+      dataOutputStream.close();
+    }
+  }
+
+  /**
+   * Load from the filesystem
+   * @param fs filesystem
+   * @param path path
+   * @return a loaded CD
+   * @throws IOException IO problems
+   */
+  public static ClusterDescription load(FileSystem fs, Path path)
+      throws IOException, JsonParseException, JsonMappingException {
+    FileStatus status = fs.getFileStatus(path);
+    byte[] b = new byte[(int) status.getLen()];
+    FSDataInputStream dataInputStream = fs.open(path);
+    int count = dataInputStream.read(b);
+    String json = new String(b, 0, count, UTF_8);
+    return fromJson(json);
+  }
+
+  /**
+   * Make a deep copy of the class
+   * @param source source
+   * @return the copy
+   */
+  public static ClusterDescription copy(ClusterDescription source) {
+    //currently the copy is done by a generate/save. Inefficient but it goes
+    //down the tree nicely
+    try {
+      return fromJson(source.toJsonString());
+    } catch (IOException e) {
+      throw new RuntimeException("ClusterDescription copy failed " + e, e);
+    }
+  }
+
+  /**
+   * Convert to a JSON string
+   * @return a JSON string description
+   * @throws IOException Problems mapping/writing the object
+   */
+  public String toJsonString() throws IOException,
+                                      JsonGenerationException,
+                                      JsonMappingException {
+    ObjectMapper mapper = new ObjectMapper();
+    mapper.configure(SerializationConfig.Feature.INDENT_OUTPUT, true);
+    return mapper.writeValueAsString(this);
+  }
+
+  /**
+   * Convert from JSON
+   * @param json input
+   * @return the parsed JSON
+   * @throws IOException IO
+   * @throws JsonMappingException failure to map from the JSON to this class
+   */
+  public static ClusterDescription fromJson(String json)
+    throws IOException, JsonParseException, JsonMappingException {
+    ObjectMapper mapper = new ObjectMapper();
+    try {
+      return mapper.readValue(json, ClusterDescription.class);
+    } catch (IOException e) {
+      log.error("Exception while parsing json : " + e + "\n" + json, e);
+      throw e;
+    }
+  }
+
+    /**
+     * Convert from input stream
+     * @param is input stream of cluster description
+     * @return the parsed JSON
+     * @throws IOException IO
+     * @throws JsonMappingException failure to map from the JSON to this class
+     */
+    public static ClusterDescription fromStream(InputStream is)
+            throws IOException, JsonParseException, JsonMappingException {
+        if (is==null) {
+          throw new FileNotFoundException("Empty Stream");
+        }
+        ObjectMapper mapper = new ObjectMapper();
+        try {
+            return mapper.readValue(is, ClusterDescription.class);
+        } catch (IOException e) {
+            log.error("Exception while parsing input stream : {}", e, e);
+      throw e;
+    }
+  }
+
+  /**
+   * Convert from a JSON file
+   * @param jsonFile input file
+   * @return the parsed JSON
+   * @throws IOException IO problems
+   * @throws JsonMappingException failure to map from the JSON to this class
+   */
+  public static ClusterDescription fromFile(File jsonFile)
+    throws IOException, JsonParseException, JsonMappingException {
+    ObjectMapper mapper = new ObjectMapper();
+    try {
+      return mapper.readValue(jsonFile, ClusterDescription.class);
+    } catch (IOException e) {
+      log.error("Exception while parsing json file {}" , jsonFile, e);
+      throw e;
+    }
+  }
+
+  /**
+   * Set a cluster option: a key val pair in the options {} section
+   * @param key key option name
+   * @param val value option value
+   */
+  public void setOption(String key, String val) {
+    options.put(key, val);
+  }
+
+  /**
+   * Set a cluster option if it is unset. If it is already set,
+   * in the Cluster Description, it is left alone
+   * @param key key key to query/set
+   * @param val value value
+   */
+
+  public void setOptionifUnset(String key, String val) {
+    if (options.get(key) == null) {
+      options.put(key, val);
+    }
+  }
+
+  /**
+   * Set an integer option -it's converted to a string before saving
+   * @param option option name
+   * @param val integer value
+   */
+  public void setOption(String option, int val) {
+    setOption(option, Integer.toString(val));
+  }
+
+  /**
+   * Set a boolean option
+   * @param option option name
+   * @param val bool value
+   */
+  public void setOption(String option, boolean val) {
+    setOption(option, Boolean.toString(val));
+  }
+
+  /**
+   * Get a cluster option or value
+   *
+   * @param key option key
+   * @param defVal option val
+   * @return resolved value or default
+   */
+  public String getOption(String key, String defVal) {
+    String val = options.get(key);
+    return val != null ? val : defVal;
+  }
+
+  /**
+   * Get a cluster option or value
+   *
+   * @param key mandatory key
+   * @return the value
+   * @throws BadConfigException if the option is missing
+   */
+  public String getMandatoryOption(String key) throws BadConfigException {
+    String val = options.get(key);
+    if (val == null) {
+      throw new BadConfigException("Missing option " + key);
+    }
+    return val;
+  }
+
+  /**
+   * Get an integer option; use {@link Integer#decode(String)} so as to take hex
+   * oct and bin values too.
+   *
+   * @param option option name
+   * @param defVal default value
+   * @return parsed value
+   * @throws NumberFormatException if the role could not be parsed.
+   */
+  public int getOptionInt(String option, int defVal) {
+    String val = getOption(option, Integer.toString(defVal));
+    return Integer.decode(val);
+  }
+
+  /**
+   * Verify that an option is set: that is defined AND non-empty
+   * @param key key to verify
+   * @throws BadConfigException
+   */
+  public void verifyOptionSet(String key) throws BadConfigException {
+    if (SliderUtils.isUnset(getOption(key, null))) {
+      throw new BadConfigException("Unset cluster option %s", key);
+    }
+  }
+
+  /**
+   * Get an option as a boolean. Note that {@link Boolean#valueOf(String)}
+   * is used for parsing -its policy of what is true vs false applies.
+   * @param option name
+   * @param defVal default
+   * @return the option.
+   */
+  public boolean getOptionBool(String option, boolean defVal) {
+    return Boolean.valueOf(getOption(option, Boolean.toString(defVal)));
+  }
+
+  /**
+   * Get a role option
+   * @param role role to get from
+   * @param option option name
+   * @param defVal default value
+   * @return resolved value
+   */
+  public String getRoleOpt(String role, String option, String defVal) {
+    Map<String, String> roleopts = getRole(role);
+    if (roleopts == null) {
+      return defVal;
+    }
+    String val = roleopts.get(option);
+    return val != null ? val : defVal;
+  }
+
+  /**
+   * Get a mandatory role option
+   * @param role role to get from
+   * @param option option name
+   * @return resolved value
+   * @throws BadConfigException if the option is not defined
+   */
+  public String getMandatoryRoleOpt(String role, String option) throws
+                                                                BadConfigException {
+    Map<String, String> roleopts = getRole(role);
+    if (roleopts == null) {
+      throw new BadConfigException("Missing role %s ", role);
+    }
+    String val = roleopts.get(option);
+    if (val == null) {
+      throw new BadConfigException("Missing option '%s' in role %s ", option, role);
+    }
+    return val;
+  }
+
+  /**
+   * Get a mandatory integer role option
+   * @param role role to get from
+   * @param option option name
+   * @return resolved value
+   * @throws BadConfigException if the option is not defined
+   */
+  public int getMandatoryRoleOptInt(String role, String option)
+      throws BadConfigException {
+    getMandatoryRoleOpt(role, option);
+    return getRoleOptInt(role, option, 0);
+  }
+  
+  /**
+   * look up a role and return its options
+   * @param role role
+   * @return role mapping or null
+   */
+  public Map<String, String> getRole(String role) {
+    return roles.get(role);
+  }
+
+  /**
+   * Get a role -adding it to the roleopts map if
+   * none with that name exists
+   * @param role role
+   * @return role mapping
+   */
+  public Map<String, String> getOrAddRole(String role) {
+    Map<String, String> map = getRole(role);
+    if (map == null) {
+      map = new HashMap<>();
+    }
+    roles.put(role, map);
+    return map;
+  }
+  
+  /*
+   * return the Set of role names
+   */
+  @JsonIgnore
+  public Set<String> getRoleNames() {
+    return new HashSet<>(roles.keySet());
+  }
+
+  /**
+   * Get a role whose presence is mandatory
+   * @param role role name
+   * @return the mapping
+   * @throws BadConfigException if the role is not there
+   */
+  public Map<String, String> getMandatoryRole(String role) throws
+                                                           BadConfigException {
+    Map<String, String> roleOptions = getRole(role);
+    if (roleOptions == null) {
+      throw new BadConfigException("Missing role " + role);
+    }
+    return roleOptions;
+  }
+
+  /**
+   * Get an integer role option; use {@link Integer#decode(String)} so as to take hex
+   * oct and bin values too.
+   *
+   * @param role role to get from
+   * @param option option name
+   * @param defVal default value
+   * @return parsed value
+   * @throws NumberFormatException if the role could not be parsed.
+   */
+  public int getRoleOptInt(String role, String option, int defVal) {
+    String val = getRoleOpt(role, option, Integer.toString(defVal));
+    return Integer.decode(val);
+  }
+
+  /**
+   * Get an integer role option; use {@link Integer#decode(String)} so as to take hex
+   * oct and bin values too.
+   *
+   * @param role role to get from
+   * @param option option name
+   * @param defVal default value
+   * @return parsed value
+   * @throws NumberFormatException if the role could not be parsed.
+   */
+  public long getRoleOptLong(String role, String option, long defVal) {
+    String val = getRoleOpt(role, option, Long.toString(defVal));
+    return Long.decode(val);
+  }
+
+  /**
+   * Set a role option, creating the role if necessary
+   * @param role role name
+   * @param option option name
+   * @param val value
+   */
+  public void setRoleOpt(String role, String option, String val) {
+    Map<String, String> roleopts = getOrAddRole(role);
+    roleopts.put(option, val);
+  }
+
+  /**
+   * Set an integer role option, creating the role if necessary
+   * @param role role name
+   * @param option option name
+   * @param val integer value
+   */
+  public void setRoleOpt(String role, String option, int val) {
+    setRoleOpt(role, option, Integer.toString(val));
+  }
+
+  /**
+   * Set a role option of any object, using its string value.
+   * This works for (Boxed) numeric values as well as other objects
+   * @param role role name
+   * @param option option name
+   * @param val non-null value
+   */
+  public void setRoleOpt(String role, String option, Object val) {
+    setRoleOpt(role, option, val.toString());
+  }
+
+  /**
+   * Get the value of a role requirement (cores, RAM, etc).
+   * These are returned as integers, but there is special handling of the 
+   * string {@link ResourceKeys#YARN_RESOURCE_MAX}, which triggers
+   * the return of the maximum value.
+   * @param role role to get from
+   * @param option option name
+   * @param defVal default value
+   * @param maxVal value to return if the max val is requested
+   * @return parsed value
+   * @throws NumberFormatException if the role could not be parsed.
+   */
+  public int getRoleResourceRequirement(String role, String option, int defVal, int maxVal) {
+    String val = getRoleOpt(role, option, Integer.toString(defVal));
+    Integer intVal;
+    if (ResourceKeys.YARN_RESOURCE_MAX.equals(val)) {
+      intVal = maxVal;
+    } else {
+      intVal = Integer.decode(val);
+    }
+    return intVal;
+  }
+
+
+  /**
+   * Set the time for an information (human, machine) timestamp pair of fields.
+   * The human time is the time in millis converted via the {@link Date} class.
+   * @param keyHumanTime name of human time key
+   * @param keyMachineTime name of machine time
+   * @param time timestamp
+   */
+  
+  public void setInfoTime(String keyHumanTime, String keyMachineTime, long time) {
+    SliderUtils.setInfoTime(info, keyHumanTime, keyMachineTime, time);
+  }
+
+  /**
+   * Set an information string. This is content that is only valid in status
+   * reports.
+   * @param key key
+   * @param value string value
+   */
+  @JsonIgnore
+  public void setInfo(String key, String value) {
+    info.put(key, value);
+  }
+
+  /**
+   * Get an information string. This is content that is only valid in status
+   * reports.
+   * @param key key
+   * @return the value or null
+   */
+  @JsonIgnore
+  public String getInfo(String key) {
+    return info.get(key);
+  }
+
+  /**
+   * Get an information string. This is content that is only valid in status
+   * reports.
+   * @param key key
+   * @return the value or null
+   */
+  @JsonIgnore
+  public boolean getInfoBool(String key) {
+    String val = info.get(key);
+    if (val != null) {
+      return Boolean.valueOf(val);
+    }
+    return false;
+  }
+
+  @JsonIgnore
+  public String getZkHosts() throws BadConfigException {
+    return getMandatoryOption(ZOOKEEPER_QUORUM);
+  }
+
+  /**
+   * Set the hosts for the ZK quorum
+   * @param zkHosts a comma separated list of hosts
+   */
+  @JsonIgnore
+  public void setZkHosts(String zkHosts) {
+    setOption(ZOOKEEPER_QUORUM, zkHosts);
+  }
+
+  @JsonIgnore
+  public String getZkPath() throws BadConfigException {
+    return getMandatoryOption(ZOOKEEPER_PATH);
+  }
+
+  @JsonIgnore
+  public void setZkPath(String zkPath) {
+    setOption(ZOOKEEPER_PATH, zkPath);
+  }
+
+  /**
+   * HBase home: if non-empty defines where a copy of HBase is preinstalled
+   */
+  @JsonIgnore
+  public String getApplicationHome() {
+    return getOption(INTERNAL_APPLICATION_HOME, "");
+  }
+
+  @JsonIgnore
+  public void setApplicationHome(String applicationHome) {
+    setOption(INTERNAL_APPLICATION_HOME, applicationHome);
+  }
+
+  /**
+   * The path in HDFS where the HBase image is
+   */
+  @JsonIgnore
+  public String getImagePath() {
+    return getOption(INTERNAL_APPLICATION_IMAGE_PATH, "");
+  }
+
+  /**
+   * Set the path in HDFS where the HBase image is
+   */
+  @JsonIgnore
+  public void setImagePath(String imagePath) {
+    setOption(INTERNAL_APPLICATION_IMAGE_PATH, imagePath);
+  }
+
+  /**
+   * Query for the image path being set (non null/non empty)
+   * @return true if there is a path in the image path option
+   */
+  @JsonIgnore
+  public boolean isImagePathSet() {
+    return SliderUtils.isSet(getImagePath());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ClusterDescriptionKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ClusterDescriptionKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ClusterDescriptionKeys.java
new file mode 100644
index 0000000..5b7a92a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ClusterDescriptionKeys.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.api;
+
+public class ClusterDescriptionKeys {
+
+  public static final String KEY_CLUSTER_LIVE = "live"; 
+  public static final String KEY_CLUSTER_FAILED = "failed"; 
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ClusterDescriptionOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ClusterDescriptionOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ClusterDescriptionOperations.java
new file mode 100644
index 0000000..5b95414
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ClusterDescriptionOperations.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.api;
+
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTree;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.providers.SliderProviderFactory;
+
+import java.util.Map;
+
+import static org.apache.slider.api.OptionKeys.ZOOKEEPER_PATH;
+import static org.apache.slider.api.OptionKeys.ZOOKEEPER_QUORUM;
+
+/**
+ * Operations on Cluster Descriptions
+ */
+public class ClusterDescriptionOperations {
+
+
+  public static ClusterDescription buildFromInstanceDefinition(AggregateConf aggregateConf) throws
+                                                                                       BadConfigException {
+
+    ClusterDescription cd = new ClusterDescription();
+    
+    aggregateConf.resolve();
+
+    //options are a merge of all globals
+    Map<String, String> options = cd.options;
+    SliderUtils.mergeMapsIgnoreDuplicateKeys(options,
+        aggregateConf.getInternal().global);
+    SliderUtils.mergeMapsIgnoreDuplicateKeys(options,
+        aggregateConf.getAppConf().global);
+    SliderUtils.mergeMapsIgnoreDuplicateKeys(options,
+        aggregateConf.getResources().global);
+
+    //roles are the role values merged in the same order
+    mergeInComponentMap(cd, aggregateConf.getInternal());
+    mergeInComponentMap(cd, aggregateConf.getAppConf());
+    mergeInComponentMap(cd, aggregateConf.getResources());
+
+    //now add the extra bits
+    cd.state = ClusterDescription.STATE_LIVE;
+    MapOperations internalOptions =
+      aggregateConf.getInternalOperations().getGlobalOptions();
+    MapOperations appOptions =
+      aggregateConf.getAppConfOperations().getGlobalOptions();
+
+    cd.type = internalOptions.getOption(InternalKeys.INTERNAL_PROVIDER_NAME,
+                                SliderProviderFactory.DEFAULT_CLUSTER_TYPE);
+
+    cd.dataPath = internalOptions.get(InternalKeys.INTERNAL_DATA_DIR_PATH);
+    cd.name = internalOptions.get(OptionKeys.APPLICATION_NAME);
+    cd.originConfigurationPath = internalOptions.get(InternalKeys.INTERNAL_SNAPSHOT_CONF_PATH);
+    cd.generatedConfigurationPath = internalOptions.get(InternalKeys.INTERNAL_GENERATED_CONF_PATH);
+    cd.setImagePath(internalOptions.get(InternalKeys.INTERNAL_APPLICATION_IMAGE_PATH));
+    cd.setApplicationHome(internalOptions.get(InternalKeys.INTERNAL_APPLICATION_HOME));
+    cd.setZkPath(appOptions.get(ZOOKEEPER_PATH));
+    cd.setZkHosts(appOptions.get(ZOOKEEPER_QUORUM));
+    
+    return cd;
+  }
+
+  private static void mergeInComponentMap(ClusterDescription cd,
+                                          ConfTree confTree) {
+
+    Map<String, Map<String, String>> components = confTree.components;
+    for (Map.Entry<String, Map<String, String>> compEntry : components.entrySet()) {
+      String name = compEntry.getKey();
+      Map<String, String> destRole = cd.getOrAddRole(name);
+      Map<String, String> sourceComponent = compEntry.getValue();
+      SliderUtils.mergeMapsIgnoreDuplicateKeys(destRole, sourceComponent);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ClusterNode.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ClusterNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ClusterNode.java
new file mode 100644
index 0000000..8b0a563
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ClusterNode.java
@@ -0,0 +1,220 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.api;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.slider.api.proto.Messages;
+import org.codehaus.jackson.JsonParseException;
+import org.codehaus.jackson.annotate.JsonIgnore;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.JsonMappingException;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/**
+ * Describe a specific node in the cluster
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL )
+public final class ClusterNode implements Cloneable {
+  protected static final Logger
+    LOG = LoggerFactory.getLogger(ClusterNode.class);
+  
+  @JsonIgnore
+  public ContainerId containerId;
+
+  /**
+   * server name
+   */
+  public String name;
+
+
+  /**
+   * UUID of container used in Slider RPC to refer to instances
+   */
+  public String id;
+  
+  public String role;
+  
+  public int roleId;
+
+  public long createTime;
+  public long startTime;
+  /**
+   * flag set when it is released, to know if it has
+   * already been targeted for termination
+   */
+  public boolean released;
+  public String host;
+  public String ip;
+  public String hostname;
+  public String hostUrl;
+
+  /**
+   * state from {@link ClusterDescription}
+   */
+  public int state;
+
+  /**
+   * Exit code: only valid if the state >= STOPPED
+   */
+  public int exitCode;
+
+  /**
+   * what was the command executed?
+   */
+  public String command;
+
+  /**
+   * Any diagnostics
+   */
+  public String diagnostics;
+
+  /**
+   * What is the tail output from the executed process (or [] if not started
+   * or the log cannot be picked up
+   */
+  public String[] output;
+
+  /**
+   * Any environment details
+   */
+  public String[] environment;
+
+  /**
+   * server-side ctor takes the container ID and builds the name from it
+   * @param containerId container ID; can be null
+   */
+  public ClusterNode(ContainerId containerId) {
+    if (containerId != null) {
+      this.containerId = containerId;
+      this.name = containerId.toString();
+    }
+  }
+
+  /**
+   * ctor for deserialization
+   */
+  public ClusterNode() {
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append(name).append(": ");
+    builder.append(state).append("\n");
+    builder.append("state: ").append(state).append("\n");
+    builder.append("role: ").append(role).append("\n");
+    append(builder, "host", host);
+    append(builder, "hostURL", hostUrl);
+    append(builder, "command", command);
+    if (output != null) {
+      for (String line : output) {
+        builder.append(line).append("\n");
+      }
+    }
+    append(builder, "diagnostics", diagnostics);
+    return builder.toString();
+  }
+
+  private void append(StringBuilder builder, String key, Object val) {
+    if (val != null) {
+      builder.append(key).append(": ").append(val.toString()).append("\n");
+    }
+  }
+  
+  /**
+   * Convert to a JSON string
+   * @return a JSON string description
+   * @throws IOException Problems mapping/writing the object
+   */
+  public String toJsonString() throws IOException {
+    ObjectMapper mapper = new ObjectMapper();
+    return mapper.writeValueAsString(this);
+  }
+
+
+  /**
+   * Convert from JSON
+   * @param json input
+   * @return the parsed JSON
+   * @throws IOException IO
+   */
+  public static ClusterNode fromJson(String json)
+    throws IOException, JsonParseException, JsonMappingException {
+    ObjectMapper mapper = new ObjectMapper();
+    try {
+      return mapper.readValue(json, ClusterNode.class);
+    } catch (IOException e) {
+      LOG.error("Exception while parsing json : {}\n{}", e , json, e);
+      throw e;
+    }
+  }
+
+  /**
+   * Build from a protobuf response
+   * @param message
+   * @return the deserialized node
+   */
+  public static ClusterNode fromProtobuf(Messages.RoleInstanceState message) {
+    ClusterNode node = new ClusterNode();
+    node.name = message.getName();
+    node.command = message.getCommand();
+    node.diagnostics = message.getDiagnostics();
+    String[] arr;
+    int environmentCount = message.getEnvironmentCount();
+    if (environmentCount > 0) {
+      arr = new String[environmentCount];
+      node.environment = message.getEnvironmentList().toArray(arr);
+    }
+    node.exitCode = message.getExitCode();
+    int outputCount = message.getOutputCount();
+    if (outputCount > 0) {
+      arr = new String[outputCount];
+      node.output = message.getOutputList().toArray(arr);
+    }
+    node.role = message.getRole();
+    node.roleId = message.getRoleId();
+    node.state = message.getState();
+    node.host = message.getHost();
+    node.hostUrl = message.getHostURL();
+    node.createTime = message.getCreateTime();
+    node.startTime = message.getStartTime();
+    node.released = message.getReleased();
+    return node;
+  }
+
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    return super.clone();
+  }
+  
+  public ClusterNode doClone() {
+    try {
+      return (ClusterNode)clone();
+    } catch (CloneNotSupportedException e) {
+      //not going to happen. This is a final class
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/InternalKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/InternalKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/InternalKeys.java
new file mode 100644
index 0000000..fcaaf0e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/InternalKeys.java
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.api;
+
+/**
+ * Keys for internal use, go into `internal.json` and not intended for normal
+ * use except when tuning Slider AM operations
+ */
+public interface InternalKeys {
+
+
+  /**
+   * Home dir of the app: {@value}
+   * If set, implies there is a home dir to use
+   */
+  String INTERNAL_APPLICATION_HOME = "internal.application.home";
+  /**
+   * Path to an image file containing the app: {@value}
+   */
+  String INTERNAL_APPLICATION_IMAGE_PATH = "internal.application.image.path";
+  /**
+   * Time in milliseconds to wait after forking any in-AM 
+   * process before attempting to start up the containers: {@value}
+   * 
+   * A shorter value brings the cluster up faster, but means that if the
+   * in AM process fails (due to a bad configuration), then time
+   * is wasted starting containers on a cluster that isn't going to come
+   * up
+   */
+  String INTERNAL_CONTAINER_STARTUP_DELAY = "internal.container.startup.delay";
+  /**
+   * internal temp directory: {@value}
+   */
+  String INTERNAL_AM_TMP_DIR = "internal.am.tmp.dir";
+  /**
+   * internal temp directory: {@value}
+   */
+  String INTERNAL_TMP_DIR = "internal.tmp.dir";
+  /**
+   * where a snapshot of the original conf dir is: {@value}
+   */
+  String INTERNAL_SNAPSHOT_CONF_PATH = "internal.snapshot.conf.path";
+  /**
+   * where a snapshot of the original conf dir is: {@value}
+   */
+  String INTERNAL_GENERATED_CONF_PATH = "internal.generated.conf.path";
+  /**
+   * where a snapshot of the original conf dir is: {@value}
+   */
+  String INTERNAL_PROVIDER_NAME = "internal.provider.name";
+  /**
+   * where a snapshot of the original conf dir is: {@value}
+   */
+  String INTERNAL_DATA_DIR_PATH = "internal.data.dir.path";
+  /**
+   * where the app def is stored
+   */
+  String INTERNAL_APPDEF_DIR_PATH = "internal.appdef.dir.path";
+  /**
+   * where addons for the app are stored
+   */
+  String INTERNAL_ADDONS_DIR_PATH = "internal.addons.dir.path";
+  /**
+   * Time in milliseconds to wait after forking any in-AM 
+   * process before attempting to start up the containers: {@value}
+   *
+   * A shorter value brings the cluster up faster, but means that if the
+   * in AM process fails (due to a bad configuration), then time
+   * is wasted starting containers on a cluster that isn't going to come
+   * up
+   */
+  int DEFAULT_INTERNAL_CONTAINER_STARTUP_DELAY = 5000;
+  /**
+   * Time in seconds before a container is considered long-lived.
+   * Shortlived containers are interpreted as a problem with the role
+   * and/or the host: {@value}
+   */
+  String INTERNAL_CONTAINER_FAILURE_SHORTLIFE =
+      "internal.container.failure.shortlife";
+  /**
+   * Default short life threshold: {@value}
+   */
+  int DEFAULT_INTERNAL_CONTAINER_FAILURE_SHORTLIFE = 60;
+  
+  /**
+   * Version of the app: {@value}
+   */
+  String KEYTAB_LOCATION = "internal.keytab.location";
+
+  /**
+   * Queue used to deploy the app: {@value}
+   */
+  String INTERNAL_QUEUE = "internal.queue";
+
+  /**
+   * Flag to indicate whether or not the chaos monkey is enabled:
+   * {@value}
+   */
+  String CHAOS_MONKEY_ENABLED = "internal.chaos.monkey.enabled";
+  boolean DEFAULT_CHAOS_MONKEY_ENABLED = false;
+
+
+  /**
+   * Rate
+   */
+
+  String CHAOS_MONKEY_INTERVAL = "internal.chaos.monkey.interval";
+  String CHAOS_MONKEY_INTERVAL_DAYS = CHAOS_MONKEY_INTERVAL + ".days";
+  String CHAOS_MONKEY_INTERVAL_HOURS = CHAOS_MONKEY_INTERVAL + ".hours";
+  String CHAOS_MONKEY_INTERVAL_MINUTES = CHAOS_MONKEY_INTERVAL + ".minutes";
+  String CHAOS_MONKEY_INTERVAL_SECONDS = CHAOS_MONKEY_INTERVAL + ".seconds";
+  
+  int DEFAULT_CHAOS_MONKEY_INTERVAL_DAYS = 0;
+  int DEFAULT_CHAOS_MONKEY_INTERVAL_HOURS = 0;
+  int DEFAULT_CHAOS_MONKEY_INTERVAL_MINUTES = 0;
+
+  String CHAOS_MONKEY_DELAY = "internal.chaos.monkey.delay";
+  String CHAOS_MONKEY_DELAY_DAYS = CHAOS_MONKEY_DELAY + ".days";
+  String CHAOS_MONKEY_DELAY_HOURS = CHAOS_MONKEY_DELAY + ".hours";
+  String CHAOS_MONKEY_DELAY_MINUTES = CHAOS_MONKEY_DELAY + ".minutes";
+  String CHAOS_MONKEY_DELAY_SECONDS = CHAOS_MONKEY_DELAY + ".seconds";
+  
+  int DEFAULT_CHAOS_MONKEY_STARTUP_DELAY = 0;
+
+  /**
+   * Prefix for all chaos monkey probabilities
+   */
+  String CHAOS_MONKEY_PROBABILITY =
+      "internal.chaos.monkey.probability";
+  /**
+   * Probabilies are out of 10000 ; 100==1%
+   */
+
+  /**
+   * Probability of a monkey check killing the AM:  {@value}
+   */
+  String CHAOS_MONKEY_PROBABILITY_AM_FAILURE =
+      CHAOS_MONKEY_PROBABILITY + ".amfailure";
+
+  /**
+   * Default probability of a monkey check killing the AM:  {@value}
+   */
+  int DEFAULT_CHAOS_MONKEY_PROBABILITY_AM_FAILURE = 0;
+
+  /**
+   * Probability of a monkey check killing the AM:  {@value}
+   */
+  String CHAOS_MONKEY_PROBABILITY_AM_LAUNCH_FAILURE =
+      CHAOS_MONKEY_PROBABILITY + ".amlaunchfailure";
+
+  /**
+   * Probability of a monkey check killing a container:  {@value}
+   */
+
+  String CHAOS_MONKEY_PROBABILITY_CONTAINER_FAILURE =
+      CHAOS_MONKEY_PROBABILITY + ".containerfailure";
+
+  /**
+   * Default probability of a monkey check killing the a container:  {@value}
+   */
+  int DEFAULT_CHAOS_MONKEY_PROBABILITY_CONTAINER_FAILURE = 0;
+
+
+  /**
+   * 1% of chaos
+   */
+  int PROBABILITY_PERCENT_1 = 100;
+  
+  /**
+   * 100% for chaos values
+   */
+  int PROBABILITY_PERCENT_100 = 100 * PROBABILITY_PERCENT_1;
+
+  /**
+   * interval between checks for escalation: {@value}
+   */
+  String ESCALATION_CHECK_INTERVAL = "escalation.check.interval.seconds";
+
+  /**
+   * default value: {@value}
+   */
+  int DEFAULT_ESCALATION_CHECK_INTERVAL = 30;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/OptionKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/OptionKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/OptionKeys.java
new file mode 100644
index 0000000..a035a99
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/OptionKeys.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.api;
+
+/**
+ *  Keys for entries in the <code>options</code> section
+ *  of a cluster description.
+ */
+public interface OptionKeys extends InternalKeys {
+
+  /**
+   * Time in milliseconds to wait after forking any in-AM 
+   * process before attempting to start up the containers: {@value}
+   * 
+   * A shorter value brings the cluster up faster, but means that if the
+   * in AM process fails (due to a bad configuration), then time
+   * is wasted starting containers on a cluster that isn't going to come
+   * up
+   */
+  String APPLICATION_TYPE = "application.type";
+  
+  String APPLICATION_NAME = "application.name";
+
+  /**
+   * Prefix for site.xml options: {@value}
+   */
+  String SITE_XML_PREFIX = "site.";
+
+
+  /**
+   * Zookeeper quorum host list: {@value}
+   */
+  String ZOOKEEPER_QUORUM = "zookeeper.quorum";
+  String ZOOKEEPER_HOSTS = "zookeeper.hosts";
+  String ZOOKEEPER_PORT = "zookeeper.port";
+
+  /**
+   * Zookeeper path value (string): {@value}
+   */
+  String ZOOKEEPER_PATH = "zookeeper.path";
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ResourceKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ResourceKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ResourceKeys.java
new file mode 100644
index 0000000..92890be
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/ResourceKeys.java
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.api;
+
+/**
+ * These are the keys valid in resource options
+ *
+ /*
+
+ Container failure window.
+
+ The window is calculated in minutes as as (days * 24 *60 + hours* 24 + minutes)
+
+ Every interval of this period after the AM is started/restarted becomes
+ the time period in which the CONTAINER_FAILURE_THRESHOLD value is calculated.
+ 
+ After the window limit is reached, the failure counts are reset. This
+ is not a sliding window/moving average policy, simply a rule such as
+ "every six hours the failure count is reset"
+
+
+ <pre>
+ ===========================================================================
+ </pre>
+
+ */
+public interface ResourceKeys {
+
+
+  /**
+   * #of instances of a component: {@value}
+   *
+  */
+  String COMPONENT_INSTANCES = "yarn.component.instances";
+
+  /**
+   * Whether to use unique names for each instance of a component: {@value}
+   */
+  String UNIQUE_NAMES = "component.unique.names";
+
+  /**
+   *  Amount of memory to ask YARN for in MB.
+   *  <i>Important:</i> this may be a hard limit on the
+   *  amount of RAM that the service can use
+   *  {@value}
+   */
+  String YARN_MEMORY = "yarn.memory";
+  
+  /** {@value} */
+  int DEF_YARN_MEMORY = 256;
+  
+  /**
+   * Number of cores/virtual cores to ask YARN for
+   *  {@value}
+   */
+  String YARN_CORES = "yarn.vcores";
+
+  /**
+   * Number of disks per instance to ask YARN for
+   *  {@value}
+   */
+  String YARN_DISKS = "yarn.disks.count-per-instance";
+
+  /**
+   * Disk size per disk to ask YARN for
+   *  {@value}
+   */
+  String YARN_DISK_SIZE = "yarn.disk.size";
+
+  /** {@value} */
+  int DEF_YARN_CORES = 1;
+
+
+  /**
+   * Label expression that this container must satisfy
+   *  {@value}
+   */
+  String YARN_LABEL_EXPRESSION = "yarn.label.expression";
+
+  /** default label expression: */
+  String DEF_YARN_LABEL_EXPRESSION = null;
+
+
+  /**
+   * Constant to indicate that the requirements of a YARN resource limit
+   * (cores, memory, ...) should be set to the maximum allowed by
+   * the queue into which the YARN container requests are placed.
+   */
+  String YARN_RESOURCE_MAX = "max";
+  
+  /**
+   * Mandatory property for all roles
+   * 1. this must be defined.
+   * 2. this must be >= 1
+   * 3. this must not match any other role priority in the cluster.
+   */
+  String COMPONENT_PRIORITY = "yarn.role.priority";
+  
+  /**
+   * placement policy
+   */
+  String COMPONENT_PLACEMENT_POLICY = "yarn.component.placement.policy";
+
+  /**
+   * Maximum number of node failures that can be tolerated by a component on a specific node
+   */
+  String NODE_FAILURE_THRESHOLD =
+      "yarn.node.failure.threshold";
+
+  /**
+   * maximum number of failed containers (in a single role)
+   * before the cluster is deemed to have failed {@value}
+   */
+  String CONTAINER_FAILURE_THRESHOLD =
+      "yarn.container.failure.threshold";
+
+  /**
+   * prefix for the time of the container failure reset window.
+   * {@value}
+   */
+
+  String CONTAINER_FAILURE_WINDOW =
+      "yarn.container.failure.window";
+
+
+
+  int DEFAULT_CONTAINER_FAILURE_WINDOW_DAYS = 0;
+  int DEFAULT_CONTAINER_FAILURE_WINDOW_HOURS = 6;
+  int DEFAULT_CONTAINER_FAILURE_WINDOW_MINUTES = 0;
+
+
+  /**
+   * Default failure threshold: {@value}
+   */
+  int DEFAULT_CONTAINER_FAILURE_THRESHOLD = 5;
+
+  /**
+   * Default node failure threshold for a component instance: {@value}
+   * Should to be lower than default component failure threshold to allow
+   * the component to start elsewhere
+   */
+  int DEFAULT_NODE_FAILURE_THRESHOLD = 3;
+
+  /**
+   * Failure threshold is unlimited: {@value}
+   */
+  int NODE_FAILURE_THRESHOLD_UNLIMITED = -1;
+
+  /**
+   * Time in seconds to escalate placement delay
+   */
+  String PLACEMENT_ESCALATE_DELAY =
+      "yarn.placement.escalate.seconds";
+
+  /**
+   * Time to have a strict placement policy outstanding before 
+   * downgrading to a lax placement (for those components which permit that).
+   * <ol>
+   *   <li>For strictly placed components, there's no relaxation.</li>
+   *   <li>For components with no locality, there's no need to relax</li>
+   * </ol>
+   * 
+   */
+  int DEFAULT_PLACEMENT_ESCALATE_DELAY_SECONDS = 30;
+
+  /**
+   * Log aggregation include, exclude patterns
+   */
+  String YARN_LOG_INCLUDE_PATTERNS = "yarn.log.include.patterns";
+  String YARN_LOG_EXCLUDE_PATTERNS = "yarn.log.exclude.patterns";
+
+  String YARN_PROFILE_NAME = "yarn.resource-profile-name";
+
+  /**
+   * Window of time where application master's failure count
+   * can be reset to 0.
+   */
+  String YARN_RESOURCEMANAGER_AM_RETRY_COUNT_WINDOW_MS  =
+      "yarn.resourcemanager.am.retry-count-window-ms";
+
+  /**
+   * The default window for Slider.
+   */
+  long DEFAULT_AM_RETRY_COUNT_WINDOW_MS = 300000;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/RoleKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/RoleKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/RoleKeys.java
new file mode 100644
index 0000000..812a6b3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/RoleKeys.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.api;
+
+/**
+ * Standard options for roles
+ */
+public interface RoleKeys {
+
+
+  /**
+   * The name of a role: {@value}
+   */
+  String ROLE_NAME = "role.name";
+
+  /**
+   * The group of a role: {@value}
+   */
+  String ROLE_GROUP = "role.group";
+
+  /**
+   * Status report: number actually granted : {@value} 
+   */
+  String ROLE_ACTUAL_INSTANCES = "role.actual.instances";
+
+  /**
+   * Status report: number currently requested: {@value} 
+   */
+  String ROLE_REQUESTED_INSTANCES = "role.requested.instances";
+
+  /**
+   * Status report: number currently being released: {@value} 
+   */
+  String ROLE_RELEASING_INSTANCES = "role.releasing.instances";
+
+  /**
+   * Status report: total number that have failed: {@value}
+   */
+  String ROLE_FAILED_INSTANCES = "role.failed.instances";
+
+  /**
+   * Status report: number that have failed recently: {@value}
+   */
+  String ROLE_FAILED_RECENTLY_INSTANCES = "role.failed.recently.instances";
+
+  /**
+   * Status report: number that have failed for node-related issues: {@value}
+   */
+  String ROLE_NODE_FAILED_INSTANCES = "role.failed.node.instances";
+
+  /**
+   * Status report: number that been pre-empted: {@value}
+   */
+  String ROLE_PREEMPTED_INSTANCES = "role.failed.preempted.instances";
+
+  /**
+   * Number of pending anti-affine instances: {@value}
+   */
+  String ROLE_PENDING_AA_INSTANCES = "role.pending.aa.instances";
+
+  /**
+   * Status report: number currently being released: {@value} 
+   */
+  String ROLE_FAILED_STARTING_INSTANCES = "role.failed.starting.instances";
+
+  /**
+   * Extra arguments (non-JVM) to use when starting this role
+   */
+  String ROLE_ADDITIONAL_ARGS = "role.additional.args";
+
+  /**
+   *  JVM heap size for Java applications in MB.  Only relevant for Java applications.
+   *  This MUST be less than or equal to the {@link ResourceKeys#YARN_MEMORY} option
+   *  {@value}
+   */
+  String JVM_HEAP = "jvm.heapsize";
+  
+  /*
+   * GC options for Java applications.
+   */
+  String GC_OPTS = "gc.opts";
+
+  /**
+   * JVM options other than heap size. Only relevant for Java applications.
+   *  {@value}
+   */
+  String JVM_OPTS = "jvm.opts";
+
+
+  /**
+   * All keys w/ env. are converted into env variables and passed down
+   */
+  String ENV_PREFIX = "env.";
+
+  /**
+   * Container service record attribute prefix.
+   */
+  String SERVICE_RECORD_ATTRIBUTE_PREFIX = "service.record.attribute";
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/SliderApplicationApi.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/SliderApplicationApi.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/SliderApplicationApi.java
new file mode 100644
index 0000000..d21785f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/SliderApplicationApi.java
@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.api;
+
+import org.apache.slider.api.types.ApplicationLivenessInformation;
+import org.apache.slider.api.types.ComponentInformation;
+import org.apache.slider.api.types.ContainerInformation;
+import org.apache.slider.api.types.NodeInformation;
+import org.apache.slider.api.types.NodeInformationList;
+import org.apache.slider.api.types.PingInformation;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTree;
+import org.apache.slider.core.conf.ConfTreeOperations;
+
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * API exported by the slider remote REST/IPC endpoints.
+ */
+public interface SliderApplicationApi {
+  /**
+   * Get the aggregate desired model
+   * @return the aggregate configuration of what was asked for
+   * -before resolution has taken place
+   * @throws IOException on any failure
+   */
+  AggregateConf getDesiredModel() throws IOException;
+
+  /**
+   * Get the desired application configuration
+   * @return the application configuration asked for
+   * -before resolution has taken place
+   * @throws IOException on any failure
+   */
+  ConfTreeOperations getDesiredAppconf() throws IOException;
+
+  /**
+   * Get the desired YARN resources
+   * @return the resources asked for
+   * -before resolution has taken place
+   * @throws IOException on any failure
+   */
+  ConfTreeOperations getDesiredResources() throws IOException;
+
+  /**
+   * Put an updated resources structure. This triggers a cluster flex
+   * operation
+   * @param updated updated resources
+   * @throws IOException on any problem.
+   */
+  void putDesiredResources(ConfTree updated) throws IOException;
+
+  /**
+   * Get the aggregate resolved model
+   * @return the aggregate configuration of what was asked for
+   * -after resolution has taken place
+   * @throws IOException on any failure
+   */
+  AggregateConf getResolvedModel() throws IOException;
+
+  /**
+   * Get the resolved application configuration
+   * @return the application configuration asked for
+   * -after resolution has taken place
+   * @throws IOException on any failure
+   */
+  ConfTreeOperations getResolvedAppconf() throws IOException;
+
+  /**
+   * Get the resolved YARN resources
+   * @return the resources asked for
+   * -after resolution has taken place
+   * @throws IOException on any failure
+   */
+  ConfTreeOperations getResolvedResources() throws IOException;
+
+  /**
+   * Get the live YARN resources
+   * @return the live set of resources in the cluster
+   * @throws IOException on any failure
+   */
+  ConfTreeOperations getLiveResources() throws IOException;
+
+  /**
+   * Get a map of live containers [containerId:info]
+   * @return a possibly empty list of serialized containers
+   * @throws IOException on any failure
+   */
+  Map<String, ContainerInformation> enumContainers() throws IOException;
+
+  /**
+   * Get a container from the container Id
+   * @param containerId YARN container ID
+   * @return the container information
+   * @throws IOException on any failure
+   */
+  ContainerInformation getContainer(String containerId) throws IOException;
+
+  /**
+   * List all components into a map of [name:info]
+   * @return a possibly empty map of components
+   * @throws IOException on any failure
+   */
+  Map<String, ComponentInformation> enumComponents() throws IOException;
+
+  /**
+   * Get information about a component
+   * @param componentName name of the component
+   * @return the component details
+   * @throws IOException on any failure
+   */
+  ComponentInformation getComponent(String componentName) throws IOException;
+
+  /**
+   * List all nodes into a map of [name:info]
+   * @return a possibly empty list of nodes
+   * @throws IOException on any failure
+   */
+  NodeInformationList getLiveNodes() throws IOException;
+
+  /**
+   * Get information about a node
+   * @param hostname name of the node
+   * @return the node details
+   * @throws IOException on any failure
+   */
+  NodeInformation getLiveNode(String hostname) throws IOException;
+
+  /**
+   * Ping as a GET
+   * @param text text to include
+   * @return the response
+   * @throws IOException on any failure
+   */
+  PingInformation ping(String text) throws IOException;
+
+  /**
+   * Stop the AM (async operation)
+   * @param text text to include
+   * @throws IOException on any failure
+   */
+  void stop(String text) throws IOException;
+
+  /**
+   * Get the application liveness
+   * @return current liveness information
+   * @throws IOException
+   */
+  ApplicationLivenessInformation getApplicationLiveness() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/SliderClusterProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/SliderClusterProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/SliderClusterProtocol.java
new file mode 100644
index 0000000..33fce22
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/SliderClusterProtocol.java
@@ -0,0 +1,179 @@
+/*
+ * 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") throws IOException, YarnException; you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.api;
+
+import org.apache.hadoop.ipc.VersionedProtocol;
+import org.apache.hadoop.security.KerberosInfo;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.slider.api.proto.Messages;
+import org.apache.slider.common.SliderXmlConfKeys;
+
+import java.io.IOException;
+
+/**
+ * Cluster protocol. This can currently act as a versioned IPC
+ * endpoint or be relayed via protobuf
+ */
+@KerberosInfo(serverPrincipal = SliderXmlConfKeys.KEY_KERBEROS_PRINCIPAL)
+public interface SliderClusterProtocol extends VersionedProtocol {
+  long versionID = 0x01;
+
+  /**
+   * Stop the cluster
+   */
+
+  Messages.StopClusterResponseProto stopCluster(Messages.StopClusterRequestProto request) throws
+                                                                                          IOException, YarnException;
+  /**
+   * Upgrade the application containers
+   * 
+   * @param request upgrade containers request object
+   * @return upgrade containers response object
+   * @throws IOException
+   * @throws YarnException
+   */
+  Messages.UpgradeContainersResponseProto upgradeContainers(
+      Messages.UpgradeContainersRequestProto request) throws IOException,
+      YarnException;
+
+  /**
+   * Flex the cluster. 
+   */
+  Messages.FlexClusterResponseProto flexCluster(Messages.FlexClusterRequestProto request)
+      throws IOException;
+
+
+  /**
+   * Get the current cluster status
+   */
+  Messages.GetJSONClusterStatusResponseProto getJSONClusterStatus(Messages.GetJSONClusterStatusRequestProto request)
+      throws IOException, YarnException;
+
+
+  /**
+   * List all running nodes in a role
+   */
+  Messages.ListNodeUUIDsByRoleResponseProto listNodeUUIDsByRole(Messages.ListNodeUUIDsByRoleRequestProto request)
+      throws IOException, YarnException;
+
+
+  /**
+   * Get the details on a node
+   */
+  Messages.GetNodeResponseProto getNode(Messages.GetNodeRequestProto request)
+      throws IOException, YarnException;
+
+  /**
+   * Get the 
+   * details on a list of nodes.
+   * Unknown nodes are not returned
+   * <i>Important: the order of the results are undefined</i>
+   */
+  Messages.GetClusterNodesResponseProto getClusterNodes(Messages.GetClusterNodesRequestProto request)
+      throws IOException, YarnException;
+
+  /**
+   * Echo back the submitted text (after logging it).
+   * Useful for adding information to the log, and for testing round trip
+   * operations of the protocol
+   * @param request request
+   * @return response
+   * @throws IOException
+   * @throws YarnException
+   */
+  Messages.EchoResponseProto echo(Messages.EchoRequestProto request) throws IOException, YarnException;
+
+  /**
+   * Kill an identified container
+   * @param request request containing the container to kill
+   * @return the response
+   * @throws IOException
+   * @throws YarnException
+   */
+  Messages.KillContainerResponseProto killContainer(Messages.KillContainerRequestProto request)
+      throws IOException, YarnException;
+
+  /**
+   * AM to commit suicide. If the Hadoop halt entry point has not been disabled,
+   * this will fail rather than return with a response.
+   * @param request request
+   * @return response (this is not the expected outcome)
+   * @throws IOException
+   * @throws YarnException
+   */
+  Messages.AMSuicideResponseProto amSuicide(Messages.AMSuicideRequestProto request)
+      throws IOException;
+
+  /**
+   * Get the instance definition
+   */
+  Messages.GetInstanceDefinitionResponseProto getInstanceDefinition(
+    Messages.GetInstanceDefinitionRequestProto request)
+    throws IOException, YarnException;
+
+  /**
+   * Get the application liveness
+   * @return current liveness information
+   * @throws IOException
+   */
+  Messages.ApplicationLivenessInformationProto getLivenessInformation(
+      Messages.GetApplicationLivenessRequestProto request
+  ) throws IOException;
+
+  Messages.GetLiveContainersResponseProto getLiveContainers(
+      Messages.GetLiveContainersRequestProto request
+  ) throws IOException;
+
+  Messages.ContainerInformationProto getLiveContainer(
+      Messages.GetLiveContainerRequestProto request
+  ) throws IOException;
+
+  Messages.GetLiveComponentsResponseProto getLiveComponents(
+      Messages.GetLiveComponentsRequestProto request
+  ) throws IOException;
+
+  Messages.ComponentInformationProto getLiveComponent(
+      Messages.GetLiveComponentRequestProto request
+  ) throws IOException;
+
+  Messages.GetLiveNodesResponseProto getLiveNodes(
+      Messages.GetLiveNodesRequestProto request
+  ) throws IOException;
+
+  Messages.NodeInformationProto getLiveNode(
+      Messages.GetLiveNodeRequestProto request
+  ) throws IOException;
+
+  Messages.WrappedJsonProto getModelDesired(Messages.EmptyPayloadProto request) throws IOException;
+
+  Messages.WrappedJsonProto getModelDesiredAppconf(Messages.EmptyPayloadProto request) throws IOException;
+
+  Messages.WrappedJsonProto getModelDesiredResources(Messages.EmptyPayloadProto request) throws IOException;
+
+  Messages.WrappedJsonProto getModelResolved(Messages.EmptyPayloadProto request) throws IOException;
+
+  Messages.WrappedJsonProto getModelResolvedAppconf(Messages.EmptyPayloadProto request) throws IOException;
+
+  Messages.WrappedJsonProto getModelResolvedResources(Messages.EmptyPayloadProto request) throws IOException;
+
+  Messages.WrappedJsonProto getLiveResources(Messages.EmptyPayloadProto request) throws IOException;
+
+  Messages.GetCertificateStoreResponseProto getClientCertificateStore(Messages.GetCertificateStoreRequestProto request)
+      throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/StateValues.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/StateValues.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/StateValues.java
new file mode 100644
index 0000000..03751e1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/StateValues.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.api;
+
+/**
+ * Enumeration of state values
+ */
+public class StateValues {
+
+  /**
+   * Specification is incomplete & cannot
+   * be used: {@value}
+   */
+  public static final int STATE_INCOMPLETE = 0;
+
+  /**
+   * Spec has been submitted: {@value}
+   */
+  public static final int STATE_SUBMITTED = 1;
+  /**
+   * Cluster created: {@value}
+   */
+  public static final int STATE_CREATED = 2;
+  /**
+   * Live: {@value}
+   */
+  public static final int STATE_LIVE = 3;
+  /**
+   * Stopped
+   */
+  public static final int STATE_STOPPED = 4;
+  /**
+   * destroyed
+   */
+  public static final int STATE_DESTROYED = 5;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/StatusKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/StatusKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/StatusKeys.java
new file mode 100644
index 0000000..8a2c4bb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/StatusKeys.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.api;
+import static org.apache.slider.api.ResourceKeys.COMPONENT_INSTANCES;
+/**
+ * Contains status and statistics keys
+ */
+public interface StatusKeys {
+
+  String STATISTICS_CONTAINERS_ACTIVE_REQUESTS = "containers.active.requests";
+  String STATISTICS_CONTAINERS_COMPLETED = "containers.completed";
+  String STATISTICS_CONTAINERS_DESIRED = "containers.desired";
+  String STATISTICS_CONTAINERS_FAILED = "containers.failed";
+  String STATISTICS_CONTAINERS_FAILED_RECENTLY = "containers.failed.recently";
+  String STATISTICS_CONTAINERS_FAILED_NODE = "containers.failed.node";
+  String STATISTICS_CONTAINERS_PREEMPTED = "containers.failed.preempted";
+  String STATISTICS_CONTAINERS_LIVE = "containers.live";
+  String STATISTICS_CONTAINERS_REQUESTED = "containers.requested";
+  String STATISTICS_CONTAINERS_ANTI_AFFINE_PENDING = "containers.anti-affine.pending";
+  String STATISTICS_CONTAINERS_STARTED = "containers.start.started";
+  String STATISTICS_CONTAINERS_START_FAILED =
+      "containers.start.failed";
+  String STATISTICS_CONTAINERS_SURPLUS =
+      "containers.surplus";
+  String STATISTICS_CONTAINERS_UNKNOWN_COMPLETED =
+      "containers.unknown.completed";
+  /**
+   * No of containers provided on AM restart
+   */
+  String INFO_CONTAINERS_AM_RESTART = "containers.at.am-restart";
+
+  String INFO_CREATE_TIME_MILLIS = "create.time.millis";
+  String INFO_CREATE_TIME_HUMAN = "create.time";
+  String INFO_LIVE_TIME_MILLIS = "live.time.millis";
+  String INFO_LIVE_TIME_HUMAN = "live.time";
+  String INFO_FLEX_TIME_MILLIS = "flex.time.millis";
+  String INFO_FLEX_TIME_HUMAN = "flex.time";
+
+  String INFO_MASTER_ADDRESS = "info.master.address";
+
+  /**
+   * System time in millis when the status report was generated
+   */
+  String INFO_STATUS_TIME_MILLIS = "status.time.millis";
+
+  /**
+   * System time in human form when the status report was generated
+   */
+  String INFO_STATUS_TIME_HUMAN = "status.time";
+
+  String INFO_AM_APP_ID = "info.am.app.id";
+  String INFO_AM_ATTEMPT_ID = "info.am.attempt.id";
+  String INFO_AM_CONTAINER_ID = "info.am.container.id";
+  String INFO_AM_HOSTNAME = "info.am.hostname";
+  String INFO_AM_RPC_PORT = "info.am.rpc.port";
+  String INFO_AM_WEB_PORT = "info.am.web.port";
+  String INFO_AM_WEB_URL = "info.am.web.url";
+  String INFO_AM_AGENT_STATUS_PORT = "info.am.agent.status.port";
+  String INFO_AM_AGENT_OPS_PORT = "info.am.agent.ops.port";
+  String INFO_AM_AGENT_OPS_URL = "info.am.agent.ops.url";
+  String INFO_AM_AGENT_STATUS_URL = "info.am.agent.status.url";
+
+      /**
+       * info: #of instances of a component requested: {@value}
+       *
+       */
+  String COMPONENT_INSTANCES_ACTUAL = COMPONENT_INSTANCES + ".actual";
+
+  /**
+   * info: #of instances of a component requested: {@value}
+   *
+   */
+  String COMPONENT_INSTANCES_REQUESTING = COMPONENT_INSTANCES + ".requesting";
+
+  /**
+   * info: #of instances of a component being released: {@value}
+   *
+   */
+  String COMPONENT_INSTANCES_RELEASING = COMPONENT_INSTANCES + ".releasing";
+
+  /**
+   * info: #of instances of a component failed: {@value}
+   *
+   */
+  String COMPONENT_INSTANCES_FAILED = COMPONENT_INSTANCES + ".failed";
+
+  /**
+   * info: #of instances of a component started: {@value}
+   *
+   */
+  String COMPONENT_INSTANCES_STARTED = COMPONENT_INSTANCES + ".started";
+
+
+  /**
+   * info: #of instances of a component completed: {@value}
+   *
+   */
+  String COMPONENT_INSTANCES_COMPLETED = COMPONENT_INSTANCES + ".completed";
+
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[46/76] [abbrv] hadoop git commit: YARN-5610. Initial code for native services REST API. Contributed by Gour Saha

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Application.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Application.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Application.java
new file mode 100644
index 0000000..cfcae95
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Application.java
@@ -0,0 +1,452 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.services.resource;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.slider.providers.PlacementPolicy;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonPropertyOrder;
+
+/**
+ * An Application resource has the following attributes.
+ **/
+
+@ApiModel(description = "An Application resource has the following attributes.")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+@XmlRootElement
+@JsonInclude(JsonInclude.Include.NON_NULL)
+@JsonPropertyOrder({ " name, state, resource, numberOfContainers, lifetime, containers " })
+public class Application extends BaseResource {
+  private static final long serialVersionUID = -4491694636566094885L;
+
+  private String id = null;
+  private String name = null;
+  private Artifact artifact = null;
+  private Resource resource = null;
+  private String launchCommand = null;
+  private Date launchTime = null;
+  private Long numberOfContainers = null;
+  private Long numberOfRunningContainers = null;
+  private Long lifetime = null;
+  private PlacementPolicy placementPolicy = null;
+  private List<Component> components = null;
+  private Configuration configuration = null;
+  private List<Container> containers = new ArrayList<>();
+  private ApplicationState state = null;
+  private Map<String, String> quicklinks = null;
+  private String queue;
+
+  /**
+   * A unique application id.
+   **/
+  public Application id(String id) {
+    this.id = id;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", required = true, value = "A unique application id.")
+  @JsonProperty("id")
+  public String getId() {
+    return id;
+  }
+
+  public void setId(String id) {
+    this.id = id;
+  }
+
+  /**
+   * A unique application name.
+   **/
+  public Application name(String name) {
+    this.name = name;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", required = true, value = "A unique application name.")
+  @JsonProperty("name")
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  /**
+   * Artifact of single-component applications. Mandatory if components
+   * attribute is not specified.
+   **/
+  public Application artifact(Artifact artifact) {
+    this.artifact = artifact;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Artifact of single-component applications. Mandatory if components attribute is not specified.")
+  @JsonProperty("artifact")
+  public Artifact getArtifact() {
+    return artifact;
+  }
+
+  public void setArtifact(Artifact artifact) {
+    this.artifact = artifact;
+  }
+
+  /**
+   * Resource of single-component applications or the global default for
+   * multi-component applications. Mandatory if it is a single-component
+   * application and if cpus and memory are not specified at the Application
+   * level.
+   **/
+  public Application resource(Resource resource) {
+    this.resource = resource;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Resource of single-component applications or the global default for multi-component applications. Mandatory if it is a single-component application and if cpus and memory are not specified at the Application level.")
+  @JsonProperty("resource")
+  public Resource getResource() {
+    return resource;
+  }
+
+  public void setResource(Resource resource) {
+    this.resource = resource;
+  }
+
+  /**
+   * The custom launch command of an application component (optional). If not
+   * specified for applications with docker images say, it will default to the
+   * default start command of the image. If there is a single component in this
+   * application, you can specify this without the need to have a 'components'
+   * section.
+   **/
+  public Application launchCommand(String launchCommand) {
+    this.launchCommand = launchCommand;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "The custom launch command of an application component (optional). If not specified for applications with docker images say, it will default to the default start command of the image. If there is a single component in this application, you can specify this without the need to have a 'components' section.")
+  @JsonProperty("launch_command")
+  public String getLaunchCommand() {
+    return launchCommand;
+  }
+
+  @XmlElement(name = "launch_command")
+  public void setLaunchCommand(String launchCommand) {
+    this.launchCommand = launchCommand;
+  }
+
+  /**
+   * The time when the application was created, e.g. 2016-03-16T01:01:49.000Z.
+   **/
+  public Application launchTime(Date launchTime) {
+    this.launchTime = launchTime;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "The time when the application was created, e.g. 2016-03-16T01:01:49.000Z.")
+  @JsonProperty("launch_time")
+  public String getLaunchTime() {
+    return launchTime.toString();
+  }
+
+  @XmlElement(name = "launch_time")
+  public void setLaunchTime(Date launchTime) {
+    this.launchTime = launchTime;
+  }
+
+  /**
+   * Number of containers for each app-component in the application. Each
+   * app-component can further override this app-level global default.
+   **/
+  public Application numberOfContainers(Long numberOfContainers) {
+    this.numberOfContainers = numberOfContainers;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Number of containers for each app-component in the application. Each app-component can further override this app-level global default.")
+  @JsonProperty("number_of_containers")
+  public Long getNumberOfContainers() {
+    return numberOfContainers;
+  }
+
+  @XmlElement(name = "number_of_containers")
+  public void setNumberOfContainers(Long numberOfContainers) {
+    this.numberOfContainers = numberOfContainers;
+  }
+
+  /**
+   * In get response this provides the total number of running containers for
+   * this application (across all components) at the time of request. Note, a
+   * subsequent request can return a different number as and when more
+   * containers get allocated until it reaches the total number of containers or
+   * if a flex request has been made between the two requests.
+   **/
+  public Application numberOfRunningContainers(Long numberOfRunningContainers) {
+    this.numberOfRunningContainers = numberOfRunningContainers;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "In get response this provides the total number of running containers for this application (across all components) at the time of request. Note, a subsequent request can return a different number as and when more containers get allocated until it reaches the total number of containers or if a flex request has been made between the two requests.")
+  @JsonProperty("number_of_running_containers")
+  public Long getNumberOfRunningContainers() {
+    return numberOfRunningContainers;
+  }
+
+  @XmlElement(name = "number_of_running_containers")
+  public void setNumberOfRunningContainers(Long numberOfRunningContainers) {
+    this.numberOfRunningContainers = numberOfRunningContainers;
+  }
+
+  /**
+   * Life time (in seconds) of the application from the time it reaches the
+   * STARTED state (after which it is automatically destroyed by YARN). For
+   * unlimited lifetime do not set a lifetime value.
+   **/
+  public Application lifetime(Long lifetime) {
+    this.lifetime = lifetime;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Life time (in seconds) of the application from the time it reaches the STARTED state (after which it is automatically destroyed by YARN). For unlimited lifetime do not set a lifetime value.")
+  @JsonProperty("lifetime")
+  public Long getLifetime() {
+    return lifetime;
+  }
+
+  public void setLifetime(Long lifetime) {
+    this.lifetime = lifetime;
+  }
+
+  /**
+   * Advanced scheduling and placement policies (optional). If not specified, it
+   * defaults to the default placement policy of the app owner. The design of
+   * placement policies are in the works. It is not very clear at this point,
+   * how policies in conjunction with labels be exposed to application owners.
+   * This is a placeholder for now. The advanced structure of this attribute
+   * will be determined by YARN-4902.
+   **/
+  public Application placementPolicy(PlacementPolicy placementPolicy) {
+    this.placementPolicy = placementPolicy;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Advanced scheduling and placement policies (optional). If not specified, it defaults to the default placement policy of the app owner. The design of placement policies are in the works. It is not very clear at this point, how policies in conjunction with labels be exposed to application owners. This is a placeholder for now. The advanced structure of this attribute will be determined by YARN-4902.")
+  @JsonProperty("placement_policy")
+  public PlacementPolicy getPlacementPolicy() {
+    return placementPolicy;
+  }
+
+  @XmlElement(name = "placement_policy")
+  public void setPlacementPolicy(PlacementPolicy placementPolicy) {
+    this.placementPolicy = placementPolicy;
+  }
+
+  /**
+   * Components of an application.
+   **/
+  public Application components(List<Component> components) {
+    this.components = components;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Components of an application.")
+  @JsonProperty("components")
+  public List<Component> getComponents() {
+    return components;
+  }
+
+  public void setComponents(List<Component> components) {
+    this.components = components;
+  }
+
+  /**
+   * Config properties of an application. Configurations provided at the
+   * application/global level are available to all the components. Specific
+   * properties can be overridden at the component level.
+   **/
+  public Application configuration(Configuration configuration) {
+    this.configuration = configuration;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Config properties of an application. Configurations provided at the application/global level are available to all the components. Specific properties can be overridden at the component level.")
+  @JsonProperty("configuration")
+  public Configuration getConfiguration() {
+    return configuration;
+  }
+
+  public void setConfiguration(Configuration configuration) {
+    this.configuration = configuration;
+  }
+
+  /**
+   * Containers of a started application. Specifying a value for this attribute
+   * for the POST payload raises a validation error. This blob is available only
+   * in the GET response of a started application.
+   **/
+  public Application containers(List<Container> containers) {
+    this.containers = containers;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Containers of a started application. Specifying a value for this attribute for the POST payload raises a validation error. This blob is available only in the GET response of a started application.")
+  @JsonProperty("containers")
+  public List<Container> getContainers() {
+    return containers;
+  }
+
+  public void setContainers(List<Container> containers) {
+    this.containers = containers;
+  }
+
+  public void addContainer(Container container) {
+    this.containers.add(container);
+  }
+
+  /**
+   * State of the application. Specifying a value for this attribute for the
+   * POST payload raises a validation error. This attribute is available only in
+   * the GET response of a started application.
+   **/
+  public Application state(ApplicationState state) {
+    this.state = state;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "State of the application. Specifying a value for this attribute for the POST payload raises a validation error. This attribute is available only in the GET response of a started application.")
+  @JsonProperty("state")
+  public ApplicationState getState() {
+    return state;
+  }
+
+  public void setState(ApplicationState state) {
+    this.state = state;
+  }
+
+  /**
+   * A blob of key-value pairs of quicklinks to be exported for an application.
+   **/
+  public Application quicklinks(Map<String, String> quicklinks) {
+    this.quicklinks = quicklinks;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "A blob of key-value pairs of quicklinks to be exported for an application.")
+  @JsonProperty("quicklinks")
+  public Map<String, String> getQuicklinks() {
+    return quicklinks;
+  }
+
+  public void setQuicklinks(Map<String, String> quicklinks) {
+    this.quicklinks = quicklinks;
+  }
+
+  /**
+   * The YARN queue that this application should be submitted to.
+   **/
+  public Application queue(String queue) {
+    this.queue = queue;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "The YARN queue that this application should be submitted to.")
+  @JsonProperty("queue")
+  public String getQueue() {
+    return queue;
+  }
+
+  public void setQueue(String queue) {
+    this.queue = queue;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    Application application = (Application) o;
+    return Objects.equals(this.name, application.name);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(name);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class Application {\n");
+
+    sb.append("    name: ").append(toIndentedString(name)).append("\n");
+    sb.append("    id: ").append(toIndentedString(id)).append("\n");
+    sb.append("    artifact: ").append(toIndentedString(artifact)).append("\n");
+    sb.append("    resource: ").append(toIndentedString(resource)).append("\n");
+    sb.append("    launchCommand: ").append(toIndentedString(launchCommand))
+        .append("\n");
+    sb.append("    launchTime: ").append(toIndentedString(launchTime))
+        .append("\n");
+    sb.append("    numberOfContainers: ")
+        .append(toIndentedString(numberOfContainers)).append("\n");
+    sb.append("    numberOfRunningContainers: ")
+        .append(toIndentedString(numberOfRunningContainers)).append("\n");
+    sb.append("    lifetime: ").append(toIndentedString(lifetime)).append("\n");
+    sb.append("    placementPolicy: ")
+        .append(toIndentedString(placementPolicy)).append("\n");
+    sb.append("    components: ").append(toIndentedString(components))
+        .append("\n");
+    sb.append("    configuration: ").append(toIndentedString(configuration))
+        .append("\n");
+    sb.append("    containers: ").append(toIndentedString(containers))
+        .append("\n");
+    sb.append("    state: ").append(toIndentedString(state)).append("\n");
+    sb.append("    quicklinks: ").append(toIndentedString(quicklinks))
+        .append("\n");
+    sb.append("    queue: ").append(toIndentedString(queue)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationState.java
new file mode 100644
index 0000000..ae96e8a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationState.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.services.resource;
+
+/**
+ * The current state of an application.
+ **/
+public enum ApplicationState {
+  ACCEPTED, STARTED, READY, STOPPED, FAILED;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationStatus.java
new file mode 100644
index 0000000..0166b48
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationStatus.java
@@ -0,0 +1,147 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.services.resource;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.Objects;
+
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * The current status of a submitted application, returned as a response to the
+ * GET API.
+ **/
+
+@ApiModel(description = "The current status of a submitted application, returned as a response to the GET api")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+@XmlRootElement
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class ApplicationStatus extends BaseResource {
+  private static final long serialVersionUID = -3469885905347851034L;
+
+  private String diagnostics = null;
+  private ApplicationState state = null;
+  private Integer code = null;
+
+  /**
+   * Diagnostic information (if any) for the reason of the current state of the
+   * application. It typically has a non-null value, if the application is in a
+   * non-running state.
+   **/
+  public ApplicationStatus diagnostics(String diagnostics) {
+    this.diagnostics = diagnostics;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Diagnostic information (if any) for the reason of the current state of the application. It typically has a non-null value, if the application is in a non-running state.")
+  @JsonProperty("diagnostics")
+  public String getDiagnostics() {
+    return diagnostics;
+  }
+
+  @XmlElement(name = "diagnostics")
+  public void setDiagnostics(String diagnostics) {
+    this.diagnostics = diagnostics;
+  }
+
+  /**
+   * Application state
+   **/
+  public ApplicationStatus state(ApplicationState state) {
+    this.state = state;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Application state")
+  @JsonProperty("state")
+  public ApplicationState getState() {
+    return state;
+  }
+
+  public void setState(ApplicationState state) {
+    this.state = state;
+  }
+
+  /**
+   * An error code specific to a scenario which app owners should be able to use
+   * to understand the failure in addition to the diagnostic information.
+   **/
+  public ApplicationStatus code(Integer code) {
+    this.code = code;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "An error code specific to a scenario which app owners should be able to use to understand the failure in addition to the diagnostic information.")
+  @JsonProperty("code")
+  public Integer getCode() {
+    return code;
+  }
+
+  public void setCode(Integer code) {
+    this.code = code;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    ApplicationStatus applicationStatus = (ApplicationStatus) o;
+    return Objects.equals(this.diagnostics, applicationStatus.diagnostics)
+        && Objects.equals(this.state, applicationStatus.state)
+        && Objects.equals(this.code, applicationStatus.code);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(diagnostics, state, code);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class ApplicationStatus {\n");
+
+    sb.append("    diagnostics: ").append(toIndentedString(diagnostics))
+        .append("\n");
+    sb.append("    state: ").append(toIndentedString(state)).append("\n");
+    sb.append("    code: ").append(toIndentedString(code)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Artifact.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Artifact.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Artifact.java
new file mode 100644
index 0000000..aee4d11
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Artifact.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.services.resource;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonValue;
+
+/**
+ * Artifact of an application component.
+ **/
+
+@ApiModel(description = "Artifact of an application component")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class Artifact {
+
+  private String id = null;
+
+  public enum TypeEnum {
+    DOCKER("docker"), TARBALL("tarball"), APPLICATION("application");
+
+    private String value;
+
+    TypeEnum(String value) {
+      this.value = value;
+    }
+
+    @Override
+    @JsonValue
+    public String toString() {
+      return value;
+    }
+  }
+
+  private TypeEnum type = TypeEnum.DOCKER;
+  private String uri = null;
+
+  /**
+   * Artifact id. Examples are package location uri for tarball based apps,
+   * image name for docker, etc.
+   **/
+  public Artifact id(String id) {
+    this.id = id;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", required = true, value = "Artifact id. Examples are package location uri for tarball based apps, image name for docker, etc.")
+  @JsonProperty("id")
+  public String getId() {
+    return id;
+  }
+
+  public void setId(String id) {
+    this.id = id;
+  }
+
+  /**
+   * Artifact type, like docker, tarball, etc. (optional)
+   **/
+  public Artifact type(TypeEnum type) {
+    this.type = type;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Artifact type, like docker, tarball, etc. (optional)")
+  @JsonProperty("type")
+  public TypeEnum getType() {
+    return type;
+  }
+
+  public void setType(TypeEnum type) {
+    this.type = type;
+  }
+
+  /**
+   * Artifact location to support multiple artifact stores (optional).
+   **/
+  public Artifact uri(String uri) {
+    this.uri = uri;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Artifact location to support multiple artifact stores (optional).")
+  @JsonProperty("uri")
+  public String getUri() {
+    return uri;
+  }
+
+  public void setUri(String uri) {
+    this.uri = uri;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    Artifact artifact = (Artifact) o;
+    return Objects.equals(this.id, artifact.id)
+        && Objects.equals(this.type, artifact.type)
+        && Objects.equals(this.uri, artifact.uri);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(id, type, uri);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class Artifact {\n");
+
+    sb.append("    id: ").append(toIndentedString(id)).append("\n");
+    sb.append("    type: ").append(toIndentedString(type)).append("\n");
+    sb.append("    uri: ").append(toIndentedString(uri)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/BaseResource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/BaseResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/BaseResource.java
new file mode 100644
index 0000000..3b2c8b1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/BaseResource.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.services.resource;
+
+import java.io.Serializable;
+
+public class BaseResource implements Serializable {
+  private static final long serialVersionUID = 1492603053176889431L;
+
+  private String uri;
+
+  /**
+   * Resource location, e.g. \
+   * "/applications/helloworld/containers/container_e3751_1458061340047_0008_01_000002\
+   * "
+   **/
+  public String getUri() {
+    return uri;
+  }
+
+  public void setUri(String uri) {
+    this.uri = uri;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append("BaseResource [uri=");
+    builder.append(uri);
+    builder.append("]");
+    return builder.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Component.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Component.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Component.java
new file mode 100644
index 0000000..3ff6945
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Component.java
@@ -0,0 +1,377 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.services.resource;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * One or more components of the application. If the application is HBase say,
+ * then the component can be a simple role like master or regionserver. If the
+ * application is a complex business webapp then a component can be other
+ * applications say Kafka or Storm. Thereby it opens up the support for complex
+ * and nested applications.
+ **/
+
+@ApiModel(description = "One or more components of the application. If the application is HBase say, then the component can be a simple role like master or regionserver. If the application is a complex business webapp then a component can be other applications say Kafka or Storm. Thereby it opens up the support for complex and nested applications.")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+@XmlRootElement
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class Component {
+
+  private String name = null;
+  private List<String> dependencies = new ArrayList<String>();
+  private ReadinessCheck readinessCheck = null;
+  private Artifact artifact = null;
+  private String launchCommand = null;
+  private Resource resource = null;
+  private Long numberOfContainers = null;
+  private Boolean uniqueComponentSupport = null;
+  private Boolean runPrivilegedContainer = null;
+  private PlacementPolicy placementPolicy = null;
+  private Configuration configuration = null;
+  private List<String> quicklinks = new ArrayList<String>();
+
+  /**
+   * Name of the application component (mandatory).
+   **/
+  public Component name(String name) {
+    this.name = name;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", required = true, value = "Name of the application component (mandatory).")
+  @JsonProperty("name")
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  /**
+   * An array of application components which should be in READY state (as
+   * defined by readiness check), before this component can be started. The
+   * dependencies across all components of an application should be represented
+   * as a DAG.
+   **/
+  public Component dependencies(List<String> dependencies) {
+    this.dependencies = dependencies;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "An array of application components which should be in READY state (as defined by readiness check), before this component can be started. The dependencies across all components of an application should be represented as a DAG.")
+  @JsonProperty("dependencies")
+  public List<String> getDependencies() {
+    return dependencies;
+  }
+
+  public void setDependencies(List<String> dependencies) {
+    this.dependencies = dependencies;
+  }
+
+  /**
+   * Readiness check for this app-component.
+   **/
+  public Component readinessCheck(ReadinessCheck readinessCheck) {
+    this.readinessCheck = readinessCheck;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Readiness check for this app-component.")
+  @JsonProperty("readiness_check")
+  public ReadinessCheck getReadinessCheck() {
+    return readinessCheck;
+  }
+
+  @XmlElement(name = "readiness_check")
+  public void setReadinessCheck(ReadinessCheck readinessCheck) {
+    this.readinessCheck = readinessCheck;
+  }
+
+  /**
+   * Artifact of the component (optional). If not specified, the application
+   * level global artifact takes effect.
+   **/
+  public Component artifact(Artifact artifact) {
+    this.artifact = artifact;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Artifact of the component (optional). If not specified, the application level global artifact takes effect.")
+  @JsonProperty("artifact")
+  public Artifact getArtifact() {
+    return artifact;
+  }
+
+  public void setArtifact(Artifact artifact) {
+    this.artifact = artifact;
+  }
+
+  /**
+   * The custom launch command of this component (optional). When specified at
+   * the component level, it overrides the value specified at the global level
+   * (if any).
+   **/
+  public Component launchCommand(String launchCommand) {
+    this.launchCommand = launchCommand;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "The custom launch command of this component (optional). When specified at the component level, it overrides the value specified at the global level (if any).")
+  @JsonProperty("launch_command")
+  public String getLaunchCommand() {
+    return launchCommand;
+  }
+
+  @XmlElement(name = "launch_command")
+  public void setLaunchCommand(String launchCommand) {
+    this.launchCommand = launchCommand;
+  }
+
+  /**
+   * Resource of this component (optional). If not specified, the application
+   * level global resource takes effect.
+   **/
+  public Component resource(Resource resource) {
+    this.resource = resource;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Resource of this component (optional). If not specified, the application level global resource takes effect.")
+  @JsonProperty("resource")
+  public Resource getResource() {
+    return resource;
+  }
+
+  public void setResource(Resource resource) {
+    this.resource = resource;
+  }
+
+  /**
+   * Number of containers for this app-component (optional). If not specified,
+   * the application level global number_of_containers takes effect.
+   **/
+  public Component numberOfContainers(Long numberOfContainers) {
+    this.numberOfContainers = numberOfContainers;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Number of containers for this app-component (optional). If not specified, the application level global number_of_containers takes effect.")
+  @JsonProperty("number_of_containers")
+  public Long getNumberOfContainers() {
+    return numberOfContainers;
+  }
+
+  @XmlElement(name = "number_of_containers")
+  public void setNumberOfContainers(Long numberOfContainers) {
+    this.numberOfContainers = numberOfContainers;
+  }
+
+  /**
+   * Certain applications need to define multiple components using the same
+   * artifact and resource profile, differing only in configurations. In such
+   * cases, this field helps app owners to avoid creating multiple component
+   * definitions with repeated information. The number_of_containers field
+   * dictates the initial number of components created. Component names
+   * typically differ with a trailing id, but assumptions should not be made on
+   * that, as the algorithm can change at any time. Configurations section will
+   * be able to use placeholders like ${APP_COMPONENT_NAME} to get its component
+   * name at runtime, and thereby differing in value at runtime. The best part
+   * of this feature is that when the component is flexed up, entirely new
+   * components (with new trailing ids) are created.
+   **/
+  public Component uniqueComponentSupport(Boolean uniqueComponentSupport) {
+    this.uniqueComponentSupport = uniqueComponentSupport;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Certain applications need to define multiple components using the same artifact and resource profile, differing only in configurations. In such cases, this field helps app owners to avoid creating multiple component definitions with repeated information. The number_of_containers field dictates the initial number of components created. Component names typically differ with a trailing id, but assumptions should not be made on that, as the algorithm can change at any time. Configurations section will be able to use placeholders like ${APP_COMPONENT_NAME} to get its component name at runtime, and thereby differing in value at runtime. The best part of this feature is that when the component is flexed up, entirely new components (with new trailing ids) are created.")
+  @JsonProperty("unique_component_support")
+  public Boolean getUniqueComponentSupport() {
+    return uniqueComponentSupport;
+  }
+
+  @XmlElement(name = "unique_component_support")
+  public void setUniqueComponentSupport(Boolean uniqueComponentSupport) {
+    this.uniqueComponentSupport = uniqueComponentSupport;
+  }
+
+  /**
+   * Run all containers of this component in privileged mode (YARN-4262).
+   **/
+  public Component runPrivilegedContainer(Boolean runPrivilegedContainer) {
+    this.runPrivilegedContainer = runPrivilegedContainer;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Run all containers of this component in privileged mode (YARN-4262).")
+  @JsonProperty("run_privileged_container")
+  public Boolean getRunPrivilegedContainer() {
+    return runPrivilegedContainer;
+  }
+
+  @XmlElement(name = "run_privileged_container")
+  public void setRunPrivilegedContainer(Boolean runPrivilegedContainer) {
+    this.runPrivilegedContainer = runPrivilegedContainer;
+  }
+
+  /**
+   * Advanced scheduling and placement policies for all containers of this
+   * component (optional). If not specified, the app level placement_policy
+   * takes effect. Refer to the description at the global level for more
+   * details.
+   **/
+  public Component placementPolicy(PlacementPolicy placementPolicy) {
+    this.placementPolicy = placementPolicy;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Advanced scheduling and placement policies for all containers of this component (optional). If not specified, the app level placement_policy takes effect. Refer to the description at the global level for more details.")
+  @JsonProperty("placement_policy")
+  public PlacementPolicy getPlacementPolicy() {
+    return placementPolicy;
+  }
+
+  @XmlElement(name = "placement_policy")
+  public void setPlacementPolicy(PlacementPolicy placementPolicy) {
+    this.placementPolicy = placementPolicy;
+  }
+
+  /**
+   * Config properties for this app-component.
+   **/
+  public Component configuration(Configuration configuration) {
+    this.configuration = configuration;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Config properties for this app-component.")
+  @JsonProperty("configuration")
+  public Configuration getConfiguration() {
+    return configuration;
+  }
+
+  public void setConfiguration(Configuration configuration) {
+    this.configuration = configuration;
+  }
+
+  /**
+   * A list of quicklink keys defined at the application level, and to be
+   * resolved by this component.
+   **/
+  public Component quicklinks(List<String> quicklinks) {
+    this.quicklinks = quicklinks;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "A list of quicklink keys defined at the application level, and to be resolved by this component.")
+  @JsonProperty("quicklinks")
+  public List<String> getQuicklinks() {
+    return quicklinks;
+  }
+
+  public void setQuicklinks(List<String> quicklinks) {
+    this.quicklinks = quicklinks;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    Component component = (Component) o;
+    return Objects.equals(this.name, component.name)
+        && Objects.equals(this.dependencies, component.dependencies)
+        && Objects.equals(this.readinessCheck, component.readinessCheck)
+        && Objects.equals(this.artifact, component.artifact)
+        && Objects.equals(this.launchCommand, component.launchCommand)
+        && Objects.equals(this.resource, component.resource)
+        && Objects
+            .equals(this.numberOfContainers, component.numberOfContainers)
+        && Objects.equals(this.uniqueComponentSupport,
+            component.uniqueComponentSupport)
+        && Objects.equals(this.runPrivilegedContainer,
+            component.runPrivilegedContainer)
+        && Objects.equals(this.placementPolicy, component.placementPolicy)
+        && Objects.equals(this.configuration, component.configuration)
+        && Objects.equals(this.quicklinks, component.quicklinks);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(name, dependencies, readinessCheck, artifact,
+        launchCommand, resource, numberOfContainers, uniqueComponentSupport,
+        runPrivilegedContainer, placementPolicy, configuration, quicklinks);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class Component {\n");
+
+    sb.append("    name: ").append(toIndentedString(name)).append("\n");
+    sb.append("    dependencies: ").append(toIndentedString(dependencies))
+        .append("\n");
+    sb.append("    readinessCheck: ").append(toIndentedString(readinessCheck))
+        .append("\n");
+    sb.append("    artifact: ").append(toIndentedString(artifact)).append("\n");
+    sb.append("    launchCommand: ").append(toIndentedString(launchCommand))
+        .append("\n");
+    sb.append("    resource: ").append(toIndentedString(resource)).append("\n");
+    sb.append("    numberOfContainers: ")
+        .append(toIndentedString(numberOfContainers)).append("\n");
+    sb.append("    uniqueComponentSupport: ")
+        .append(toIndentedString(uniqueComponentSupport)).append("\n");
+    sb.append("    runPrivilegedContainer: ")
+        .append(toIndentedString(runPrivilegedContainer)).append("\n");
+    sb.append("    placementPolicy: ")
+        .append(toIndentedString(placementPolicy)).append("\n");
+    sb.append("    configuration: ").append(toIndentedString(configuration))
+        .append("\n");
+    sb.append("    quicklinks: ").append(toIndentedString(quicklinks))
+        .append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ConfigFile.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ConfigFile.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ConfigFile.java
new file mode 100644
index 0000000..d06c1b8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ConfigFile.java
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.services.resource;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.Objects;
+
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonValue;
+
+/**
+ * A config file that needs to be created and made available as a volume in an
+ * application component container.
+ **/
+
+@ApiModel(description = "A config file that needs to be created and made available as a volume in an application component container.")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+@XmlRootElement
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class ConfigFile {
+
+  public enum TypeEnum {
+    XML("xml"), PROPERTIES("properties"), JSON("json"), YAML("yaml"), TEMPLATE(
+        "template"), ENV("env"), HADOOP_XML("hadoop_xml");
+
+    private String value;
+
+    TypeEnum(String value) {
+      this.value = value;
+    }
+
+    @Override
+    @JsonValue
+    public String toString() {
+      return value;
+    }
+  }
+
+  private TypeEnum type = null;
+  private String destFile = null;
+  private String srcFile = null;
+  private Object props = null;
+
+  /**
+   * Config file in the standard format like xml, properties, json, yaml,
+   * template.
+   **/
+  public ConfigFile type(TypeEnum type) {
+    this.type = type;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Config file in the standard format like xml, properties, json, yaml, template.")
+  @JsonProperty("type")
+  public TypeEnum getType() {
+    return type;
+  }
+
+  public void setType(TypeEnum type) {
+    this.type = type;
+  }
+
+  /**
+   * The absolute path that this configuration file should be mounted as, in the
+   * application container.
+   **/
+  public ConfigFile destFile(String destFile) {
+    this.destFile = destFile;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "The absolute path that this configuration file should be mounted as, in the application container.")
+  @JsonProperty("dest_file")
+  public String getDestFile() {
+    return destFile;
+  }
+
+  @XmlElement(name = "dest_file")
+  public void setDestFile(String destFile) {
+    this.destFile = destFile;
+  }
+
+  /**
+   * Required for type template. This provides the source location of the
+   * template which needs to be mounted as dest_file post property
+   * substitutions. Typically the src_file would point to a source controlled
+   * network accessible file maintained by tools like puppet, chef, etc.
+   **/
+  public ConfigFile srcFile(String srcFile) {
+    this.srcFile = srcFile;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Required for type template. This provides the source location of the template which needs to be mounted as dest_file post property substitutions. Typically the src_file would point to a source controlled network accessible file maintained by tools like puppet, chef, etc.")
+  @JsonProperty("src_file")
+  public String getSrcFile() {
+    return srcFile;
+  }
+
+  @XmlElement(name = "src_file")
+  public void setSrcFile(String srcFile) {
+    this.srcFile = srcFile;
+  }
+
+  /**
+   * A blob of key value pairs that will be dumped in the dest_file in the
+   * format as specified in type. If the type is template then the attribute
+   * src_file is mandatory and the src_file content is dumped to dest_file post
+   * property substitutions.
+   **/
+  public ConfigFile props(Object props) {
+    this.props = props;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "A blob of key value pairs that will be dumped in the dest_file in the format as specified in type. If the type is template then the attribute src_file is mandatory and the src_file content is dumped to dest_file post property substitutions.")
+  @JsonProperty("props")
+  public Object getProps() {
+    return props;
+  }
+
+  public void setProps(Object props) {
+    this.props = props;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    ConfigFile configFile = (ConfigFile) o;
+    return Objects.equals(this.type, configFile.type)
+        && Objects.equals(this.destFile, configFile.destFile)
+        && Objects.equals(this.srcFile, configFile.srcFile)
+        && Objects.equals(this.props, configFile.props);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(type, destFile, srcFile, props);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class ConfigFile {\n");
+
+    sb.append("    type: ").append(toIndentedString(type)).append("\n");
+    sb.append("    destFile: ").append(toIndentedString(destFile)).append("\n");
+    sb.append("    srcFile: ").append(toIndentedString(srcFile)).append("\n");
+    sb.append("    props: ").append(toIndentedString(props)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Configuration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Configuration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Configuration.java
new file mode 100644
index 0000000..05983db
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Configuration.java
@@ -0,0 +1,147 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.services.resource;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Set of configuration properties that can be injected into the application
+ * components via envs, files and custom pluggable helper docker containers.
+ * Files of several standard formats like xml, properties, json, yaml and
+ * templates will be supported.
+ **/
+
+@ApiModel(description = "Set of configuration properties that can be injected into the application components via envs, files and custom pluggable helper docker containers. Files of several standard formats like xml, properties, json, yaml and templates will be supported.")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class Configuration {
+
+  private Map<String, String> properties = new HashMap<String, String>();
+  private Map<String, String> env = new HashMap<String, String>();
+  private List<ConfigFile> files = new ArrayList<ConfigFile>();
+
+  /**
+   * A blob of key-value pairs of common application properties.
+   **/
+  public Configuration properties(Map<String, String> properties) {
+    this.properties = properties;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "A blob of key-value pairs of common application properties.")
+  @JsonProperty("properties")
+  public Map<String, String> getProperties() {
+    return properties;
+  }
+
+  public void setProperties(Map<String, String> properties) {
+    this.properties = properties;
+  }
+
+  /**
+   * A blob of key-value pairs which will be appended to the default system
+   * properties and handed off to the application at start time. All placeholder
+   * references to properties will be substituted before injection.
+   **/
+  public Configuration env(Map<String, String> env) {
+    this.env = env;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "A blob of key-value pairs which will be appended to the default system properties and handed off to the application at start time. All placeholder references to properties will be substituted before injection.")
+  @JsonProperty("env")
+  public Map<String, String> getEnv() {
+    return env;
+  }
+
+  public void setEnv(Map<String, String> env) {
+    this.env = env;
+  }
+
+  /**
+   * Array of list of files that needs to be created and made available as
+   * volumes in the application component containers.
+   **/
+  public Configuration files(List<ConfigFile> files) {
+    this.files = files;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Array of list of files that needs to be created and made available as volumes in the application component containers.")
+  @JsonProperty("files")
+  public List<ConfigFile> getFiles() {
+    return files;
+  }
+
+  public void setFiles(List<ConfigFile> files) {
+    this.files = files;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    Configuration configuration = (Configuration) o;
+    return Objects.equals(this.properties, configuration.properties)
+        && Objects.equals(this.env, configuration.env)
+        && Objects.equals(this.files, configuration.files);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(properties, env, files);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class Configuration {\n");
+
+    sb.append("    properties: ").append(toIndentedString(properties))
+        .append("\n");
+    sb.append("    env: ").append(toIndentedString(env)).append("\n");
+    sb.append("    files: ").append(toIndentedString(files)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Container.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Container.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Container.java
new file mode 100644
index 0000000..2faf6f2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Container.java
@@ -0,0 +1,256 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.services.resource;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.Date;
+import java.util.Objects;
+
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+@ApiModel(description = "An instance of a running application container")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+@XmlRootElement
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class Container extends BaseResource {
+  private static final long serialVersionUID = -8955788064529288L;
+
+  private String id = null;
+  private Date launchTime = null;
+  private String ip = null;
+  private String hostname = null;
+  private String bareHost = null;
+  private ContainerState state = null;
+  private String componentName = null;
+  private Resource resource = null;
+
+  /**
+   * Unique container id of a running application, e.g.
+   * container_e3751_1458061340047_0008_01_000002
+   **/
+  public Container id(String id) {
+    this.id = id;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Unique container id of a running application, e.g. container_e3751_1458061340047_0008_01_000002")
+  @JsonProperty("id")
+  public String getId() {
+    return id;
+  }
+
+  public void setId(String id) {
+    this.id = id;
+  }
+
+  /**
+   * The time when the container was created, e.g. 2016-03-16T01:01:49.000Z. This will most likely be different from cluster launch time.
+   **/
+  public Container launchTime(Date launchTime) {
+    this.launchTime = launchTime;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "The time when the container was created, e.g. 2016-03-16T01:01:49.000Z. This will most likely be different from cluster launch time.")
+  @JsonProperty("launch_time")
+  public String getLaunchTime() {
+    return launchTime.toString();
+  }
+
+  @XmlElement(name = "launch_time")
+  public void setLaunchTime(Date launchTime) {
+    this.launchTime = launchTime;
+  }
+
+  /**
+   * IP address of a running container, e.g. 172.31.42.141. The IP address and
+   * hostname attribute values are dependent on the cluster/docker network setup
+   * as per YARN-4007.
+   **/
+  public Container ip(String ip) {
+    this.ip = ip;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "IP address of a running container, e.g. 172.31.42.141. The IP address and hostname attribute values are dependent on the cluster/docker network setup as per YARN-4007.")
+  @JsonProperty("ip")
+  public String getIp() {
+    return ip;
+  }
+
+  public void setIp(String ip) {
+    this.ip = ip;
+  }
+
+  /**
+   * Fully qualified hostname of a running container, e.g.
+   * ctr-e3751-1458061340047-0008-01-000002.examplestg.site. The IP address and
+   * hostname attribute values are dependent on the cluster/docker network setup
+   * as per YARN-4007.
+   **/
+  public Container hostname(String hostname) {
+    this.hostname = hostname;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Fully qualified hostname of a running container, e.g. ctr-e3751-1458061340047-0008-01-000002.examplestg.site. The IP address and hostname attribute values are dependent on the cluster/docker network setup as per YARN-4007.")
+  @JsonProperty("hostname")
+  public String getHostname() {
+    return hostname;
+  }
+
+  public void setHostname(String hostname) {
+    this.hostname = hostname;
+  }
+
+  /**
+   * The bare node or host in which the container is running, e.g.
+   * cn008.example.com
+   **/
+  public Container bareHost(String bareHost) {
+    this.bareHost = bareHost;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "The bare node or host in which the container is running, e.g. cn008.example.com")
+  @JsonProperty("bare_host")
+  public String getBareHost() {
+    return bareHost;
+  }
+
+  @XmlElement(name = "bare_host")
+  public void setBareHost(String bareHost) {
+    this.bareHost = bareHost;
+  }
+
+  /**
+   * State of the container of an application.
+   **/
+  public Container state(ContainerState state) {
+    this.state = state;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "State of the container of an application.")
+  @JsonProperty("state")
+  public ContainerState getState() {
+    return state;
+  }
+
+  public void setState(ContainerState state) {
+    this.state = state;
+  }
+
+  /**
+   * Name of the component that this container instance belongs to.
+   **/
+  public Container componentName(String componentName) {
+    this.componentName = componentName;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Name of the component that this container instance belongs to.")
+  @JsonProperty("component_name")
+  public String getComponentName() {
+    return componentName;
+  }
+
+  @XmlElement(name = "component_name")
+  public void setComponentName(String componentName) {
+    this.componentName = componentName;
+  }
+
+  /**
+   * Resource used for this container.
+   **/
+  public Container resource(Resource resource) {
+    this.resource = resource;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Resource used for this container.")
+  @JsonProperty("resource")
+  public Resource getResource() {
+    return resource;
+  }
+
+  public void setResource(Resource resource) {
+    this.resource = resource;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    Container container = (Container) o;
+    return Objects.equals(this.id, container.id)
+        && Objects.equals(this.launchTime, container.launchTime)
+        && Objects.equals(this.ip, container.ip)
+        && Objects.equals(this.hostname, container.hostname)
+        && Objects.equals(this.bareHost, container.bareHost)
+        && Objects.equals(this.state, container.state)
+        && Objects.equals(this.componentName, container.componentName)
+        && Objects.equals(this.resource, container.resource);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(id, launchTime, ip, hostname, bareHost, state,
+        componentName, resource);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class Container {\n");
+
+    sb.append("    id: ").append(toIndentedString(id)).append("\n");
+    sb.append("    launchTime: ").append(toIndentedString(launchTime))
+        .append("\n");
+    sb.append("    ip: ").append(toIndentedString(ip)).append("\n");
+    sb.append("    hostname: ").append(toIndentedString(hostname)).append("\n");
+    sb.append("    bareHost: ").append(toIndentedString(bareHost)).append("\n");
+    sb.append("    state: ").append(toIndentedString(state)).append("\n");
+    sb.append("    componentName: ").append(toIndentedString(componentName))
+        .append("\n");
+    sb.append("    resource: ").append(toIndentedString(resource)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ContainerState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ContainerState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ContainerState.java
new file mode 100644
index 0000000..cb017fb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ContainerState.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.services.resource;
+
+/**
+ * The current state of the container of an application.
+ **/
+public enum ContainerState {
+  INIT, READY;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Error.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Error.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Error.java
new file mode 100644
index 0000000..91c4e3a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Error.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.services.resource;
+
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+public class Error {
+
+  private Integer code = null;
+  private String message = null;
+  private String fields = null;
+
+  /**
+   **/
+  public Error code(Integer code) {
+    this.code = code;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "")
+  @JsonProperty("code")
+  public Integer getCode() {
+    return code;
+  }
+
+  public void setCode(Integer code) {
+    this.code = code;
+  }
+
+  /**
+   **/
+  public Error message(String message) {
+    this.message = message;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "")
+  @JsonProperty("message")
+  public String getMessage() {
+    return message;
+  }
+
+  public void setMessage(String message) {
+    this.message = message;
+  }
+
+  /**
+   **/
+  public Error fields(String fields) {
+    this.fields = fields;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "")
+  @JsonProperty("fields")
+  public String getFields() {
+    return fields;
+  }
+
+  public void setFields(String fields) {
+    this.fields = fields;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    Error error = (Error) o;
+    return Objects.equals(this.code, error.code)
+        && Objects.equals(this.message, error.message)
+        && Objects.equals(this.fields, error.fields);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(code, message, fields);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class Error {\n");
+
+    sb.append("    code: ").append(toIndentedString(code)).append("\n");
+    sb.append("    message: ").append(toIndentedString(message)).append("\n");
+    sb.append("    fields: ").append(toIndentedString(fields)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/PlacementPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/PlacementPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/PlacementPolicy.java
new file mode 100644
index 0000000..7541e2f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/PlacementPolicy.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.services.resource;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Placement policy of an instance of an application. This feature is in the
+ * works in YARN-4902.
+ **/
+
+@ApiModel(description = "Placement policy of an instance of an application. This feature is in the works in YARN-4902.")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+public class PlacementPolicy {
+
+  private String label = null;
+
+  /**
+   * Assigns an app to a named partition of the cluster where the application
+   * desires to run (optional). If not specified all apps are submitted to a
+   * default label of the app owner. One or more labels can be setup for each
+   * application owner account with required constraints like no-preemption,
+   * sla-99999, preemption-ok, etc.
+   **/
+  public PlacementPolicy label(String label) {
+    this.label = label;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Assigns an app to a named partition of the cluster where the application desires to run (optional). If not specified all apps are submitted to a default label of the app owner. One or more labels can be setup for each application owner account with required constraints like no-preemption, sla-99999, preemption-ok, etc.")
+  @JsonProperty("label")
+  public String getLabel() {
+    return label;
+  }
+
+  public void setLabel(String label) {
+    this.label = label;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    PlacementPolicy placementPolicy = (PlacementPolicy) o;
+    return Objects.equals(this.label, placementPolicy.label);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(label);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class PlacementPolicy {\n");
+
+    sb.append("    label: ").append(toIndentedString(label)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/019adbc8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ReadinessCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ReadinessCheck.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ReadinessCheck.java
new file mode 100644
index 0000000..80fdf92
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ReadinessCheck.java
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.services.resource;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonValue;
+
+/**
+ * A custom command or a pluggable helper container to determine the readiness
+ * of a container of a component. Readiness for every application is different.
+ * Hence the need for a simple interface, with scope to support advanced
+ * usecases.
+ **/
+
+@ApiModel(description = "A custom command or a pluggable helper container to determine the readiness of a container of a component. Readiness for every application is different. Hence the need for a simple interface, with scope to support advanced usecases.")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+public class ReadinessCheck {
+
+  public enum TypeEnum {
+    HTTP("http");
+
+    private String value;
+
+    TypeEnum(String value) {
+      this.value = value;
+    }
+
+    @Override
+    @JsonValue
+    public String toString() {
+      return value;
+    }
+  }
+
+  private TypeEnum type = null;
+  private String uri = null;
+  private Artifact artifact = null;
+
+  /**
+   * http (YARN will perform a simple REST call at a regular interval and expect
+   * a 204 No content).
+   **/
+  public ReadinessCheck type(TypeEnum type) {
+    this.type = type;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "http (YARN will perform a simple REST call at a regular interval and expect a 204 No content).")
+  @JsonProperty("type")
+  public TypeEnum getType() {
+    return type;
+  }
+
+  public void setType(TypeEnum type) {
+    this.type = type;
+  }
+
+  /**
+   * Fully qualified REST uri endpoint.
+   **/
+  public ReadinessCheck uri(String uri) {
+    this.uri = uri;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", required = true, value = "Fully qualified REST uri endpoint.")
+  @JsonProperty("uri")
+  public String getUri() {
+    return uri;
+  }
+
+  public void setUri(String uri) {
+    this.uri = uri;
+  }
+
+  /**
+   * Artifact of the pluggable readiness check helper container (optional). If
+   * specified, this helper container typically hosts the http uri and
+   * encapsulates the complex scripts required to perform actual container
+   * readiness check. At the end it is expected to respond a 204 No content just
+   * like the simplified use case. This pluggable framework benefits application
+   * owners who can run applications without any packaging modifications. Note,
+   * artifacts of type docker only is supported for now.
+   **/
+  public ReadinessCheck artifact(Artifact artifact) {
+    this.artifact = artifact;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Artifact of the pluggable readiness check helper container (optional). If specified, this helper container typically hosts the http uri and encapsulates the complex scripts required to perform actual container readiness check. At the end it is expected to respond a 204 No content just like the simplified use case. This pluggable framework benefits application owners who can run applications without any packaging modifications. Note, artifacts of type docker only is supported for now.")
+  @JsonProperty("artifact")
+  public Artifact getArtifact() {
+    return artifact;
+  }
+
+  public void setArtifact(Artifact artifact) {
+    this.artifact = artifact;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    ReadinessCheck readinessCheck = (ReadinessCheck) o;
+    return Objects.equals(this.type, readinessCheck.type)
+        && Objects.equals(this.uri, readinessCheck.uri)
+        && Objects.equals(this.artifact, readinessCheck.artifact);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(type, uri, artifact);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class ReadinessCheck {\n");
+
+    sb.append("    type: ").append(toIndentedString(type)).append("\n");
+    sb.append("    uri: ").append(toIndentedString(uri)).append("\n");
+    sb.append("    artifact: ").append(toIndentedString(artifact)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[15/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
new file mode 100644
index 0000000..b767059
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
@@ -0,0 +1,2450 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster;
+
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.health.HealthCheckRegistry;
+import com.codahale.metrics.jvm.GarbageCollectorMetricSet;
+import com.codahale.metrics.jvm.MemoryUsageGaugeSet;
+import com.codahale.metrics.jvm.ThreadStatesGaugeSet;
+import com.google.common.base.Preconditions;
+import com.google.protobuf.BlockingService;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.http.HttpConfig;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.registry.client.binding.RegistryTypeUtils;
+import org.apache.hadoop.registry.client.binding.RegistryUtils;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.service.ServiceOperations;
+import org.apache.hadoop.service.ServiceStateChangeListener;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.client.api.AMRMClient;
+import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
+import org.apache.hadoop.yarn.client.api.async.NMClientAsync;
+import org.apache.hadoop.yarn.client.api.async.impl.NMClientAsyncImpl;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.*;
+import static org.apache.slider.common.Constants.HADOOP_JAAS_DEBUG;
+
+import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
+import org.apache.hadoop.registry.client.types.yarn.PersistencePolicies;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes;
+import org.apache.hadoop.registry.server.integration.RMRegistryOperationsService;
+import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
+import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
+import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hadoop.yarn.webapp.WebAppException;
+import org.apache.hadoop.yarn.webapp.WebApps;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
+import org.apache.slider.api.ClusterDescription;
+import org.apache.slider.api.InternalKeys;
+import org.apache.slider.api.ResourceKeys;
+import org.apache.slider.api.RoleKeys;
+import org.apache.slider.api.StatusKeys;
+import org.apache.slider.api.proto.SliderClusterAPI;
+import org.apache.slider.client.SliderYarnClientImpl;
+import org.apache.slider.common.SliderExitCodes;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.params.AbstractActionArgs;
+import org.apache.slider.common.params.SliderAMArgs;
+import org.apache.slider.common.params.SliderAMCreateAction;
+import org.apache.slider.common.params.SliderActions;
+import org.apache.slider.common.tools.ConfigHelper;
+import org.apache.slider.common.tools.PortScanner;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.common.tools.SliderVersionInfo;
+import org.apache.slider.core.build.InstanceIO;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTree;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.core.exceptions.SliderInternalStateException;
+import org.apache.slider.core.exceptions.TriggerClusterTeardownException;
+import org.apache.slider.core.launch.CredentialUtils;
+import org.apache.slider.core.main.ExitCodeProvider;
+import org.apache.slider.core.main.LauncherExitCodes;
+import org.apache.slider.core.main.RunService;
+import org.apache.slider.core.main.ServiceLauncher;
+import org.apache.slider.core.registry.info.CustomRegistryConstants;
+import org.apache.slider.providers.ProviderCompleted;
+import org.apache.slider.providers.ProviderRole;
+import org.apache.slider.providers.ProviderService;
+import org.apache.slider.providers.SliderProviderFactory;
+import org.apache.slider.providers.agent.AgentKeys;
+import org.apache.slider.providers.agent.AgentProviderService;
+import org.apache.slider.providers.slideram.SliderAMClientProvider;
+import org.apache.slider.providers.slideram.SliderAMProviderService;
+import org.apache.slider.server.appmaster.actions.ActionRegisterServiceInstance;
+import org.apache.slider.server.appmaster.actions.EscalateOutstandingRequests;
+import org.apache.slider.server.appmaster.actions.RegisterComponentInstance;
+import org.apache.slider.server.appmaster.actions.QueueExecutor;
+import org.apache.slider.server.appmaster.actions.QueueService;
+import org.apache.slider.server.appmaster.actions.ActionStopSlider;
+import org.apache.slider.server.appmaster.actions.ActionUpgradeContainers;
+import org.apache.slider.server.appmaster.actions.AsyncAction;
+import org.apache.slider.server.appmaster.actions.RenewingAction;
+import org.apache.slider.server.appmaster.actions.ResetFailureWindow;
+import org.apache.slider.server.appmaster.actions.ReviewAndFlexApplicationSize;
+import org.apache.slider.server.appmaster.actions.UnregisterComponentInstance;
+import org.apache.slider.server.appmaster.management.MetricsAndMonitoring;
+import org.apache.slider.server.appmaster.management.YarnServiceHealthCheck;
+import org.apache.slider.server.appmaster.monkey.ChaosKillAM;
+import org.apache.slider.server.appmaster.monkey.ChaosKillContainer;
+import org.apache.slider.server.appmaster.monkey.ChaosMonkeyService;
+import org.apache.slider.server.appmaster.operations.AsyncRMOperationHandler;
+import org.apache.slider.server.appmaster.operations.ProviderNotifyingOperationHandler;
+import org.apache.slider.server.appmaster.rpc.RpcBinder;
+import org.apache.slider.server.appmaster.rpc.SliderAMPolicyProvider;
+import org.apache.slider.server.appmaster.rpc.SliderClusterProtocolPBImpl;
+import org.apache.slider.server.appmaster.operations.AbstractRMOperation;
+import org.apache.slider.server.appmaster.rpc.SliderIPCService;
+import org.apache.slider.server.appmaster.security.SecurityConfiguration;
+import org.apache.slider.server.appmaster.state.AppState;
+import org.apache.slider.server.appmaster.state.AppStateBindingInfo;
+import org.apache.slider.server.appmaster.state.ContainerAssignment;
+import org.apache.slider.server.appmaster.state.ProviderAppState;
+import org.apache.slider.server.appmaster.operations.RMOperationHandler;
+import org.apache.slider.server.appmaster.state.RoleInstance;
+import org.apache.slider.server.appmaster.web.AgentService;
+import org.apache.slider.server.appmaster.web.rest.InsecureAmFilterInitializer;
+import org.apache.slider.server.appmaster.web.rest.agent.AgentWebApp;
+import org.apache.slider.server.appmaster.web.SliderAMWebApp;
+import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.apache.slider.server.appmaster.web.WebAppApiImpl;
+import org.apache.slider.server.appmaster.web.rest.RestPaths;
+import org.apache.slider.server.appmaster.web.rest.application.ApplicationResouceContentCacheFactory;
+import org.apache.slider.server.appmaster.web.rest.application.resources.ContentCache;
+import org.apache.slider.server.services.security.CertificateManager;
+import org.apache.slider.server.services.utility.AbstractSliderLaunchedService;
+import org.apache.slider.server.services.utility.WebAppService;
+import org.apache.slider.server.services.workflow.ServiceThreadFactory;
+import org.apache.slider.server.services.workflow.WorkflowExecutorService;
+import org.apache.slider.server.services.workflow.WorkflowRpcService;
+import org.apache.slider.server.services.yarnregistry.YarnRegistryViewForProviders;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.nio.ByteBuffer;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * This is the AM, which directly implements the callbacks from the AM and NM
+ */
+public class SliderAppMaster extends AbstractSliderLaunchedService 
+  implements AMRMClientAsync.CallbackHandler,
+    NMClientAsync.CallbackHandler,
+    RunService,
+    SliderExitCodes,
+    SliderKeys,
+    ServiceStateChangeListener,
+    RoleKeys,
+    ProviderCompleted,
+    AppMasterActionOperations {
+
+  protected static final Logger log =
+    LoggerFactory.getLogger(SliderAppMaster.class);
+
+  /**
+   * log for YARN events
+   */
+  protected static final Logger LOG_YARN = log;
+
+  public static final String SERVICE_CLASSNAME_SHORT = "SliderAppMaster";
+  public static final String SERVICE_CLASSNAME =
+      "org.apache.slider.server.appmaster." + SERVICE_CLASSNAME_SHORT;
+
+  public static final int HEARTBEAT_INTERVAL = 1000;
+  public static final int NUM_RPC_HANDLERS = 5;
+
+  /**
+   * Metrics and monitoring services.
+   * Deployed in {@link #serviceInit(Configuration)}
+   */
+  private final MetricsAndMonitoring metricsAndMonitoring = new MetricsAndMonitoring(); 
+
+  /**
+   * metrics registry
+   */
+  public MetricRegistry metrics;
+
+  /** Error string on chaos monkey launch failure action: {@value} */
+  public static final String E_TRIGGERED_LAUNCH_FAILURE =
+      "Chaos monkey triggered launch failure";
+
+  /** YARN RPC to communicate with the Resource Manager or Node Manager */
+  private YarnRPC yarnRPC;
+
+  /** Handle to communicate with the Resource Manager*/
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+  private AMRMClientAsync asyncRMClient;
+
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+  private RMOperationHandler rmOperationHandler;
+  
+  private RMOperationHandler providerRMOperationHandler;
+
+  /** Handle to communicate with the Node Manager*/
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+  public NMClientAsync nmClientAsync;
+
+  /**
+   * Credentials for propagating down to launched containers
+   */
+  private Credentials containerCredentials;
+
+  /**
+   * Slider IPC: Real service handler
+   */
+  private SliderIPCService sliderIPCService;
+  /**
+   * Slider IPC: binding
+   */
+  private WorkflowRpcService rpcService;
+
+  /**
+   * Secret manager
+   */
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+  private ClientToAMTokenSecretManager secretManager;
+  
+  /** Hostname of the container*/
+  private String appMasterHostname = "";
+  /* Port on which the app master listens for status updates from clients*/
+  private int appMasterRpcPort = 0;
+  /** Tracking url to which app master publishes info for clients to monitor*/
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+  private String appMasterTrackingUrl = "";
+
+  /** Proxied app master URL (as retrieved from AM report at launch time) */
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+  private String appMasterProxiedUrl = "";
+
+  /** Application Attempt Id ( combination of attemptId and fail count )*/
+  private ApplicationAttemptId appAttemptID;
+
+  /**
+   * App ACLs
+   */
+  protected Map<ApplicationAccessType, String> applicationACLs;
+
+  /**
+   * Ongoing state of the cluster: containers, nodes they
+   * live on, etc.
+   */
+  private final AppState appState =
+      new AppState(new ProtobufClusterServices(), metricsAndMonitoring);
+
+  /**
+   * App state for external objects. This is almost entirely
+   * a read-only view of the application state. To change the state,
+   * Providers (or anything else) are expected to queue async changes.
+   */
+  private final ProviderAppState stateForProviders =
+      new ProviderAppState("undefined", appState);
+
+  /**
+   * model the state using locks and conditions
+   */
+  private final ReentrantLock AMExecutionStateLock = new ReentrantLock();
+  private final Condition isAMCompleted = AMExecutionStateLock.newCondition();
+
+  /**
+   * Flag set if the AM is to be shutdown
+   */
+  private final AtomicBoolean amCompletionFlag = new AtomicBoolean(false);
+
+  /**
+   * Flag set during the init process
+   */
+  private final AtomicBoolean initCompleted = new AtomicBoolean(false);
+
+  /**
+   * Flag to set if the process exit code was set before shutdown started
+   */
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+  private boolean spawnedProcessExitedBeforeShutdownTriggered;
+
+
+  /** Arguments passed in : raw*/
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+  private SliderAMArgs serviceArgs;
+
+  /**
+   * ID of the AM container
+   */
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+  private ContainerId appMasterContainerID;
+
+  /**
+   * Monkey Service -may be null
+   */
+  private ChaosMonkeyService monkey;
+  
+  /**
+   * ProviderService of this cluster
+   */
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+  private ProviderService providerService;
+
+  /**
+   * The YARN registry service
+   */
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+  private RegistryOperations registryOperations;
+
+  /**
+   * The stop request received...the exit details are extracted
+   * from this
+   */
+  private volatile ActionStopSlider stopAction;
+
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+  private RoleLaunchService launchService;
+  
+  //username -null if it is not known/not to be set
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+  private String hadoop_user_name;
+  private String service_user_name;
+  
+  private SliderAMWebApp webApp;
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+  private InetSocketAddress rpcServiceAddress;
+  private SliderAMProviderService sliderAMProvider;
+  private CertificateManager certificateManager;
+
+  /**
+   * Executor.
+   * Assigned in {@link #serviceInit(Configuration)}
+   */
+  private WorkflowExecutorService<ExecutorService> executorService;
+
+  /**
+   * Action queues. Created at instance creation, but
+   * added as a child and inited in {@link #serviceInit(Configuration)}
+   */
+  private final QueueService actionQueues = new QueueService();
+  private String agentOpsUrl;
+  private String agentStatusUrl;
+  private YarnRegistryViewForProviders yarnRegistryOperations;
+  //private FsDelegationTokenManager fsDelegationTokenManager;
+  private RegisterApplicationMasterResponse amRegistrationData;
+  private PortScanner portScanner;
+  private SecurityConfiguration securityConfiguration;
+
+  /**
+   * Is security enabled?
+   * Set early on in the {@link #createAndRunCluster(String)} operation.
+   */
+  private boolean securityEnabled;
+  private ContentCache contentCache;
+
+  /**
+   * resource limits
+   */
+  private Resource maximumResourceCapability;
+
+  /**
+   * Service Constructor
+   */
+  public SliderAppMaster() {
+    super(SERVICE_CLASSNAME_SHORT);
+    new HdfsConfiguration();
+    new YarnConfiguration();
+  }
+
+/* =================================================================== */
+/* service lifecycle methods */
+/* =================================================================== */
+
+  @Override //AbstractService
+  public synchronized void serviceInit(Configuration conf) throws Exception {
+    // slider client if found
+    
+    Configuration customConf = SliderUtils.loadSliderClientXML();
+    // Load in the server configuration - if it is actually on the Classpath
+    URL serverXmlUrl = ConfigHelper.getResourceUrl(SLIDER_SERVER_XML);
+    if (serverXmlUrl != null) {
+      log.info("Loading {} at {}", SLIDER_SERVER_XML, serverXmlUrl);
+      Configuration serverConf = ConfigHelper.loadFromResource(SLIDER_SERVER_XML);
+      ConfigHelper.mergeConfigurations(customConf, serverConf,
+          SLIDER_SERVER_XML, true);
+    }
+    serviceArgs.applyDefinitions(customConf);
+    serviceArgs.applyFileSystemBinding(customConf);
+    // conf now contains all customizations
+
+    AbstractActionArgs action = serviceArgs.getCoreAction();
+    SliderAMCreateAction createAction = (SliderAMCreateAction) action;
+
+    // sort out the location of the AM
+    String rmAddress = createAction.getRmAddress();
+    if (rmAddress != null) {
+      log.debug("Setting RM address from the command line: {}", rmAddress);
+      SliderUtils.setRmSchedulerAddress(customConf, rmAddress);
+    }
+
+    log.info("AM configuration:\n{}",
+        ConfigHelper.dumpConfigToString(customConf));
+    for (Map.Entry<String, String> envs : System.getenv().entrySet()) {
+      log.info("System env {}={}", envs.getKey(), envs.getValue());
+    }
+
+    ConfigHelper.mergeConfigurations(conf, customConf, SLIDER_CLIENT_XML, true);
+    //init security with our conf
+    if (SliderUtils.isHadoopClusterSecure(conf)) {
+      log.info("Secure mode with kerberos realm {}",
+               SliderUtils.getKerberosRealm());
+      UserGroupInformation.setConfiguration(conf);
+      UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+      log.debug("Authenticating as {}", ugi);
+      SliderUtils.verifyPrincipalSet(conf, DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY);
+    } else {
+      log.info("Cluster is insecure");
+    }
+    log.info("Login user is {}", UserGroupInformation.getLoginUser());
+
+    //look at settings of Hadoop Auth, to pick up a problem seen once
+    checkAndWarnForAuthTokenProblems();
+    
+    // validate server env
+    boolean dependencyChecks =
+        !conf.getBoolean(KEY_SLIDER_AM_DEPENDENCY_CHECKS_DISABLED,
+            false);
+    SliderUtils.validateSliderServerEnvironment(log, dependencyChecks);
+
+    // create and register monitoring services
+    addService(metricsAndMonitoring);
+    metrics = metricsAndMonitoring.getMetrics();
+/* TODO: turn these one once the metrics testing is more under control
+    metrics.registerAll(new ThreadStatesGaugeSet());
+    metrics.registerAll(new MemoryUsageGaugeSet());
+    metrics.registerAll(new GarbageCollectorMetricSet());
+
+*/
+    contentCache = ApplicationResouceContentCacheFactory.createContentCache(stateForProviders);
+
+    executorService = new WorkflowExecutorService<>("AmExecutor",
+        Executors.newFixedThreadPool(2,
+            new ServiceThreadFactory("AmExecutor", true)));
+    addService(executorService);
+
+    addService(actionQueues);
+
+    //init all child services
+    super.serviceInit(conf);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    super.serviceStart();
+    HealthCheckRegistry health = metricsAndMonitoring.getHealth();
+    health.register("AM Health", new YarnServiceHealthCheck(this));
+  }
+
+  /**
+   * Start the queue processing
+   */
+  private void startQueueProcessing() {
+    log.info("Queue Processing started");
+    executorService.execute(actionQueues);
+    executorService.execute(new QueueExecutor(this, actionQueues));
+  }
+  
+/* =================================================================== */
+/* RunService methods called from ServiceLauncher */
+/* =================================================================== */
+
+  /**
+   * pick up the args from the service launcher
+   * @param config configuration
+   * @param args argument list
+   */
+  @Override // RunService
+  public Configuration bindArgs(Configuration config, String... args) throws Exception {
+    // let the superclass process it
+    Configuration superConf = super.bindArgs(config, args);
+    // add the slider XML config
+    ConfigHelper.injectSliderXMLResource();
+
+    //yarn-ify
+    YarnConfiguration yarnConfiguration = new YarnConfiguration(
+        superConf);
+    serviceArgs = new SliderAMArgs(args);
+    serviceArgs.parse();
+
+    return SliderUtils.patchConfiguration(yarnConfiguration);
+  }
+
+
+  /**
+   * this is called by service launcher; when it returns the application finishes
+   * @return the exit code to return by the app
+   * @throws Throwable
+   */
+  @Override
+  public int runService() throws Throwable {
+    SliderVersionInfo.loadAndPrintVersionInfo(log);
+
+    //dump the system properties if in debug mode
+    if (log.isDebugEnabled()) {
+      log.debug("System properties:\n" + SliderUtils.propertiesToString(System.getProperties()));
+    }
+
+    //choose the action
+    String action = serviceArgs.getAction();
+    List<String> actionArgs = serviceArgs.getActionArgs();
+    int exitCode;
+    switch (action) {
+      case SliderActions.ACTION_HELP:
+        log.info("{}: {}", getName(), serviceArgs.usage());
+        exitCode = SliderExitCodes.EXIT_USAGE;
+        break;
+      case SliderActions.ACTION_CREATE:
+        exitCode = createAndRunCluster(actionArgs.get(0));
+        break;
+      default:
+        throw new SliderException("Unimplemented: " + action);
+    }
+    log.info("Exiting AM; final exit code = {}", exitCode);
+    return exitCode;
+  }
+
+  /**
+   * Initialize a newly created service then add it. 
+   * Because the service is not started, this MUST be done before
+   * the AM itself starts, or it is explicitly added after
+   * @param service the service to init
+   */
+  public Service initAndAddService(Service service) {
+    service.init(getConfig());
+    addService(service);
+    return service;
+  }
+
+  /* =================================================================== */
+
+  /**
+   * Create and run the cluster.
+   * @param clustername cluster name
+   * @return exit code
+   * @throws Throwable on a failure
+   */
+  private int createAndRunCluster(String clustername) throws Throwable {
+
+    //load the cluster description from the cd argument
+    String sliderClusterDir = serviceArgs.getSliderClusterURI();
+    URI sliderClusterURI = new URI(sliderClusterDir);
+    Path clusterDirPath = new Path(sliderClusterURI);
+    log.info("Application defined at {}", sliderClusterURI);
+    SliderFileSystem fs = getClusterFS();
+
+    // build up information about the running application -this
+    // will be passed down to the cluster status
+    MapOperations appInformation = new MapOperations(); 
+
+    AggregateConf instanceDefinition =
+      InstanceIO.loadInstanceDefinitionUnresolved(fs, clusterDirPath);
+    instanceDefinition.setName(clustername);
+
+    log.info("Deploying cluster {}:", instanceDefinition);
+
+    // and resolve it
+    AggregateConf resolvedInstance = new AggregateConf( instanceDefinition);
+    resolvedInstance.resolve();
+
+    stateForProviders.setApplicationName(clustername);
+
+    Configuration serviceConf = getConfig();
+
+    // extend AM configuration with component resource
+    MapOperations amConfiguration = resolvedInstance
+      .getAppConfOperations().getComponent(COMPONENT_AM);
+    // and patch configuration with prefix
+    if (amConfiguration != null) {
+      Map<String, String> sliderAppConfKeys = amConfiguration.prefixedWith("slider.");
+      for (Map.Entry<String, String> entry : sliderAppConfKeys.entrySet()) {
+        String k = entry.getKey();
+        String v = entry.getValue();
+        boolean exists = serviceConf.get(k) != null;
+        log.info("{} {} to {}", (exists ? "Overwriting" : "Setting"), k, v);
+        serviceConf.set(k, v);
+      }
+    }
+
+    securityConfiguration = new SecurityConfiguration(serviceConf, resolvedInstance, clustername);
+    // obtain security state
+    securityEnabled = securityConfiguration.isSecurityEnabled();
+    // set the global security flag for the instance definition
+    instanceDefinition.getAppConfOperations().set(KEY_SECURITY_ENABLED, securityEnabled);
+
+    // triggers resolution and snapshotting for agent
+    appState.setInitialInstanceDefinition(instanceDefinition);
+
+    File confDir = getLocalConfDir();
+    if (!confDir.exists() || !confDir.isDirectory()) {
+      log.info("Conf dir {} does not exist.", confDir);
+      File parentFile = confDir.getParentFile();
+      log.info("Parent dir {}:\n{}", parentFile, SliderUtils.listDir(parentFile));
+    }
+    
+    //get our provider
+    MapOperations globalInternalOptions = getGlobalInternalOptions();
+    String providerType = globalInternalOptions.getMandatoryOption(
+      InternalKeys.INTERNAL_PROVIDER_NAME);
+    log.info("Cluster provider type is {}", providerType);
+    SliderProviderFactory factory =
+      SliderProviderFactory.createSliderProviderFactory(providerType);
+    providerService = factory.createServerProvider();
+    // init the provider BUT DO NOT START IT YET
+    initAndAddService(providerService);
+    providerRMOperationHandler = new ProviderNotifyingOperationHandler(providerService);
+    
+    // create a slider AM provider
+    sliderAMProvider = new SliderAMProviderService();
+    initAndAddService(sliderAMProvider);
+    
+    InetSocketAddress rmSchedulerAddress = SliderUtils.getRmSchedulerAddress(serviceConf);
+    log.info("RM is at {}", rmSchedulerAddress);
+    yarnRPC = YarnRPC.create(serviceConf);
+
+    // set up the YARN client. This may require patching in the RM client-API address if it
+    // is (somehow) unset server-side.    String clientRMaddr = serviceConf.get(YarnConfiguration.RM_ADDRESS);
+    InetSocketAddress clientRpcAddress = SliderUtils.getRmAddress(serviceConf);
+    if (!SliderUtils.isAddressDefined(clientRpcAddress)) {
+      // client addr is being unset. We can lift it from the other RM APIs
+      log.warn("Yarn RM address was unbound; attempting to fix up");
+      serviceConf.set(YarnConfiguration.RM_ADDRESS,
+          String.format("%s:%d", rmSchedulerAddress.getHostString(), clientRpcAddress.getPort() ));
+    }
+
+    /*
+     * Extract the container ID. This is then
+     * turned into an (incomplete) container
+     */
+    appMasterContainerID = ConverterUtils.toContainerId(
+      SliderUtils.mandatoryEnvVariable(ApplicationConstants.Environment.CONTAINER_ID.name()));
+    appAttemptID = appMasterContainerID.getApplicationAttemptId();
+
+    ApplicationId appid = appAttemptID.getApplicationId();
+    log.info("AM for ID {}", appid.getId());
+
+    appInformation.put(StatusKeys.INFO_AM_CONTAINER_ID, appMasterContainerID.toString());
+    appInformation.put(StatusKeys.INFO_AM_APP_ID, appid.toString());
+    appInformation.put(StatusKeys.INFO_AM_ATTEMPT_ID, appAttemptID.toString());
+
+    Map<String, String> envVars;
+    List<Container> liveContainers;
+
+    /*
+     * It is critical this section is synchronized, to stop async AM events
+     * arriving while registering a restarting AM.
+     */
+    synchronized (appState) {
+      int heartbeatInterval = HEARTBEAT_INTERVAL;
+
+      // add the RM client -this brings the callbacks in
+      asyncRMClient = AMRMClientAsync.createAMRMClientAsync(heartbeatInterval, this);
+      addService(asyncRMClient);
+      //now bring it up
+      deployChildService(asyncRMClient);
+
+
+      // nmclient relays callbacks back to this class
+      nmClientAsync = new NMClientAsyncImpl("nmclient", this);
+      deployChildService(nmClientAsync);
+
+      // set up secret manager
+      secretManager = new ClientToAMTokenSecretManager(appAttemptID, null);
+
+      if (securityEnabled) {
+        // fix up the ACLs if they are not set
+        String acls = serviceConf.get(KEY_PROTOCOL_ACL);
+        if (acls == null) {
+          getConfig().set(KEY_PROTOCOL_ACL, "*");
+        }
+      }
+
+      certificateManager = new CertificateManager();
+
+      //bring up the Slider RPC service
+      buildPortScanner(instanceDefinition);
+      startSliderRPCServer(instanceDefinition);
+
+      rpcServiceAddress = rpcService.getConnectAddress();
+      appMasterHostname = rpcServiceAddress.getAddress().getCanonicalHostName();
+      appMasterRpcPort = rpcServiceAddress.getPort();
+      appMasterTrackingUrl = null;
+      log.info("AM Server is listening at {}:{}", appMasterHostname, appMasterRpcPort);
+      appInformation.put(StatusKeys.INFO_AM_HOSTNAME, appMasterHostname);
+      appInformation.set(StatusKeys.INFO_AM_RPC_PORT, appMasterRpcPort);
+
+      log.info("Starting Yarn registry");
+      registryOperations = startRegistryOperationsService();
+      log.info(registryOperations.toString());
+
+      //build the role map
+      List<ProviderRole> providerRoles = new ArrayList<>(providerService.getRoles());
+      providerRoles.addAll(SliderAMClientProvider.ROLES);
+
+      // Start up the WebApp and track the URL for it
+      MapOperations component = instanceDefinition.getAppConfOperations()
+          .getComponent(SliderKeys.COMPONENT_AM);
+      certificateManager.initialize(component, appMasterHostname,
+                                    appMasterContainerID.toString(),
+                                    clustername);
+      certificateManager.setPassphrase(instanceDefinition.getPassphrase());
+ 
+      if (component.getOptionBool(
+          AgentKeys.KEY_AGENT_TWO_WAY_SSL_ENABLED, false)) {
+        uploadServerCertForLocalization(clustername, fs);
+      }
+
+      // Web service endpoints: initialize
+      WebAppApiImpl webAppApi =
+          new WebAppApiImpl(
+              stateForProviders,
+              providerService,
+              certificateManager,
+              registryOperations,
+              metricsAndMonitoring,
+              actionQueues,
+              this,
+              contentCache);
+      initAMFilterOptions(serviceConf);
+
+      // start the agent web app
+      startAgentWebApp(appInformation, serviceConf, webAppApi);
+      int webAppPort = deployWebApplication(webAppApi);
+
+      String scheme = WebAppUtils.HTTP_PREFIX;
+      appMasterTrackingUrl = scheme + appMasterHostname + ":" + webAppPort;
+
+      appInformation.put(StatusKeys.INFO_AM_WEB_URL, appMasterTrackingUrl + "/");
+      appInformation.set(StatusKeys.INFO_AM_WEB_PORT, webAppPort);
+
+      // *****************************************************
+      // Register self with ResourceManager
+      // This will start heartbeating to the RM
+      // address = SliderUtils.getRmSchedulerAddress(asyncRMClient.getConfig());
+      // *****************************************************
+      log.info("Connecting to RM at {}; AM tracking URL={}",
+               appMasterRpcPort, appMasterTrackingUrl);
+      amRegistrationData = asyncRMClient.registerApplicationMaster(appMasterHostname,
+                                   appMasterRpcPort,
+                                   appMasterTrackingUrl);
+      maximumResourceCapability = amRegistrationData.getMaximumResourceCapability();
+
+      int minMemory = serviceConf.getInt(RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
+          DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB);
+       // validate scheduler vcores allocation setting
+      int minCores = serviceConf.getInt(RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
+          DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+      int maxMemory = maximumResourceCapability.getMemory();
+      int maxCores = maximumResourceCapability.getVirtualCores();
+      appState.setContainerLimits(minMemory,maxMemory, minCores, maxCores );
+
+      // build the handler for RM request/release operations; this uses
+      // the max value as part of its lookup
+      rmOperationHandler = new AsyncRMOperationHandler(asyncRMClient, maximumResourceCapability);
+
+      // set the RM-defined maximum cluster values
+      appInformation.put(ResourceKeys.YARN_CORES, Integer.toString(maxCores));
+      appInformation.put(ResourceKeys.YARN_MEMORY, Integer.toString(maxMemory));
+
+      processAMCredentials(securityConfiguration);
+
+      if (securityEnabled) {
+        secretManager.setMasterKey(
+            amRegistrationData.getClientToAMTokenMasterKey().array());
+        applicationACLs = amRegistrationData.getApplicationACLs();
+
+        //tell the server what the ACLs are
+        rpcService.getServer().refreshServiceAcl(serviceConf,
+            new SliderAMPolicyProvider());
+        if (securityConfiguration.isKeytabProvided()) {
+          // perform keytab based login to establish kerberos authenticated
+          // principal.  Can do so now since AM registration with RM above required
+          // tokens associated to principal
+          String principal = securityConfiguration.getPrincipal();
+          File localKeytabFile = securityConfiguration.getKeytabFile(instanceDefinition);
+          // Now log in...
+          login(principal, localKeytabFile);
+          // obtain new FS reference that should be kerberos based and different
+          // than the previously cached reference
+          fs = new SliderFileSystem(serviceConf);
+        }
+      }
+
+      // YARN client.
+      // Important: this is only valid at startup, and must be executed within
+      // the right UGI context. Use with care.
+      SliderYarnClientImpl yarnClient = null;
+      List<NodeReport> nodeReports;
+      try {
+        yarnClient = new SliderYarnClientImpl();
+        yarnClient.init(getConfig());
+        yarnClient.start();
+        nodeReports = getNodeReports(yarnClient);
+        log.info("Yarn node report count: {}", nodeReports.size());
+        // look up the application itself -this is needed to get the proxied
+        // URL of the AM, for registering endpoints.
+        // this call must be made after the AM has registered itself, obviously
+        ApplicationAttemptReport report = getApplicationAttemptReport(yarnClient);
+        appMasterProxiedUrl = report.getTrackingUrl();
+        if (SliderUtils.isUnset(appMasterProxiedUrl)) {
+          log.warn("Proxied URL is not set in application report");
+          appMasterProxiedUrl = appMasterTrackingUrl;
+        }
+      } finally {
+        // at this point yarnClient is no longer needed.
+        // stop it immediately
+        ServiceOperations.stop(yarnClient);
+        yarnClient = null;
+      }
+
+      // extract container list
+
+      liveContainers = amRegistrationData.getContainersFromPreviousAttempts();
+
+      //now validate the installation
+      Configuration providerConf =
+        providerService.loadProviderConfigurationInformation(confDir);
+
+      providerService.initializeApplicationConfiguration(instanceDefinition, fs);
+
+      providerService.validateApplicationConfiguration(instanceDefinition,
+          confDir,
+          securityEnabled);
+
+      //determine the location for the role history data
+      Path historyDir = new Path(clusterDirPath, HISTORY_DIR_NAME);
+
+      //build the instance
+      AppStateBindingInfo binding = new AppStateBindingInfo();
+      binding.instanceDefinition = instanceDefinition;
+      binding.serviceConfig = serviceConf;
+      binding.publishedProviderConf = providerConf;
+      binding.roles = providerRoles;
+      binding.fs = fs.getFileSystem();
+      binding.historyPath = historyDir;
+      binding.liveContainers = liveContainers;
+      binding.applicationInfo = appInformation;
+      binding.releaseSelector = providerService.createContainerReleaseSelector();
+      binding.nodeReports = nodeReports;
+      appState.buildInstance(binding);
+
+      providerService.rebuildContainerDetails(liveContainers,
+          instanceDefinition.getName(), appState.getRolePriorityMap());
+
+      // add the AM to the list of nodes in the cluster
+
+      appState.buildAppMasterNode(appMasterContainerID,
+          appMasterHostname,
+          webAppPort,
+          appMasterHostname + ":" + webAppPort);
+
+      // build up environment variables that the AM wants set in every container
+      // irrespective of provider and role.
+      envVars = new HashMap<>();
+      if (hadoop_user_name != null) {
+        envVars.put(HADOOP_USER_NAME, hadoop_user_name);
+      }
+      String debug_kerberos = System.getenv(HADOOP_JAAS_DEBUG);
+      if (debug_kerberos != null) {
+        envVars.put(HADOOP_JAAS_DEBUG, debug_kerberos);
+      }
+    }
+    String rolesTmpSubdir = appMasterContainerID.toString() + "/roles";
+
+    String amTmpDir = globalInternalOptions.getMandatoryOption(InternalKeys.INTERNAL_AM_TMP_DIR);
+
+    Path tmpDirPath = new Path(amTmpDir);
+    Path launcherTmpDirPath = new Path(tmpDirPath, rolesTmpSubdir);
+    fs.getFileSystem().mkdirs(launcherTmpDirPath);
+
+    //launcher service
+    launchService = new RoleLaunchService(actionQueues,
+                                          providerService,
+                                          fs,
+                                          new Path(getGeneratedConfDir()),
+                                          envVars,
+                                          launcherTmpDirPath);
+
+    deployChildService(launchService);
+
+    appState.noteAMLaunched();
+
+
+    //Give the provider access to the state, and AM
+    providerService.bind(stateForProviders, actionQueues, liveContainers);
+    sliderAMProvider.bind(stateForProviders, actionQueues, liveContainers);
+
+    // chaos monkey
+    maybeStartMonkey();
+
+    // setup token renewal and expiry handling for long lived apps
+//    if (!securityConfiguration.isKeytabProvided() &&
+//        SliderUtils.isHadoopClusterSecure(getConfig())) {
+//      fsDelegationTokenManager = new FsDelegationTokenManager(actionQueues);
+//      fsDelegationTokenManager.acquireDelegationToken(getConfig());
+//    }
+
+    // if not a secure cluster, extract the username -it will be
+    // propagated to workers
+    if (!UserGroupInformation.isSecurityEnabled()) {
+      hadoop_user_name = System.getenv(HADOOP_USER_NAME);
+      log.info(HADOOP_USER_NAME + "='{}'", hadoop_user_name);
+    }
+    service_user_name = RegistryUtils.currentUser();
+    log.info("Registry service username ={}", service_user_name);
+
+
+    // declare the cluster initialized
+    log.info("Application Master Initialization Completed");
+    initCompleted.set(true);
+
+    scheduleFailureWindowResets(instanceDefinition.getResources());
+    scheduleEscalation(instanceDefinition.getInternal());
+
+    try {
+      // schedule YARN Registry registration
+      queue(new ActionRegisterServiceInstance(clustername, appid));
+
+      // log the YARN and web UIs
+      log.info("RM Webapp address {}",
+          serviceConf.get(YarnConfiguration.RM_WEBAPP_ADDRESS));
+      log.info("Slider webapp address {} proxied at {}",
+        appMasterTrackingUrl, appMasterProxiedUrl);
+
+      // Start the Slider AM provider
+      sliderAMProvider.start();
+
+      // launch the real provider; this is expected to trigger a callback that
+      // starts the node review process
+      launchProviderService(instanceDefinition, confDir);
+
+      // start handling any scheduled events
+
+      startQueueProcessing();
+
+      //now block waiting to be told to exit the process
+      waitForAMCompletionSignal();
+    } catch(Exception e) {
+      log.error("Exception : {}", e, e);
+      // call the AM stop command as if it had been queued (but without
+      // going via the queue, which may not have started
+      onAMStop(new ActionStopSlider(e));
+    }
+    //shutdown time
+    return finish();
+  }
+
+  /**
+   * Get the YARN application Attempt report as the logged in user
+   * @param yarnClient client to the RM
+   * @return the application report
+   * @throws YarnException
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private ApplicationAttemptReport getApplicationAttemptReport(
+    final SliderYarnClientImpl yarnClient)
+      throws YarnException, IOException, InterruptedException {
+    Preconditions.checkNotNull(yarnClient, "Null Yarn client");
+    ApplicationAttemptReport report;
+    if (securityEnabled) {
+      UserGroupInformation ugi = UserGroupInformation.getLoginUser();
+      report = ugi.doAs(new PrivilegedExceptionAction<ApplicationAttemptReport>() {
+        @Override
+        public ApplicationAttemptReport run() throws Exception {
+          return yarnClient.getApplicationAttemptReport(appAttemptID);
+        }
+      });
+    } else {
+      report = yarnClient.getApplicationAttemptReport(appAttemptID);
+    }
+    return report;
+  }
+
+  /**
+   * List the node reports: uses {@link SliderYarnClientImpl} as the login user
+   * @param yarnClient client to the RM
+   * @return the node reports
+   * @throws IOException
+   * @throws YarnException
+   * @throws InterruptedException
+   */
+  private List<NodeReport> getNodeReports(final SliderYarnClientImpl yarnClient)
+    throws IOException, YarnException, InterruptedException {
+    Preconditions.checkNotNull(yarnClient, "Null Yarn client");
+    List<NodeReport> nodeReports;
+    if (securityEnabled) {
+      nodeReports = UserGroupInformation.getLoginUser().doAs(
+        new PrivilegedExceptionAction<List<NodeReport>>() {
+          @Override
+          public List<NodeReport> run() throws Exception {
+            return yarnClient.getNodeReports(NodeState.RUNNING);
+          }
+        });
+    } else {
+      nodeReports = yarnClient.getNodeReports(NodeState.RUNNING);
+    }
+    log.info("Yarn node report count: {}", nodeReports.size());
+    return nodeReports;
+  }
+
+  /**
+   * Deploy the web application.
+   * <p>
+   *   Creates and starts the web application, and adds a
+   *   <code>WebAppService</code> service under the AM, to ensure
+   *   a managed web application shutdown.
+   * @param webAppApi web app API instance
+   * @return port the web application is deployed on
+   * @throws IOException general problems starting the webapp (network, etc)
+   * @throws WebAppException other issues
+   */
+  private int deployWebApplication(WebAppApiImpl webAppApi)
+      throws IOException, SliderException {
+
+    try {
+      webApp = new SliderAMWebApp(webAppApi);
+      HttpConfig.Policy policy = HttpConfig.Policy.HTTP_ONLY;
+      int port = getPortToRequest();
+      log.info("Launching web application at port {} with policy {}", port, policy);
+
+      WebApps.$for(SliderAMWebApp.BASE_PATH,
+          WebAppApi.class,
+          webAppApi,
+          RestPaths.WS_CONTEXT)
+             .withHttpPolicy(getConfig(), policy)
+             .at("0.0.0.0", port, true)
+             .inDevMode()
+             .start(webApp);
+
+      WebAppService<SliderAMWebApp> webAppService =
+        new WebAppService<>("slider", webApp);
+
+      deployChildService(webAppService);
+      return webApp.port();
+    } catch (WebAppException e) {
+      if (e.getCause() instanceof IOException) {
+        throw (IOException)e.getCause();
+      } else {
+        throw e;
+      }
+    }
+  }
+
+  /**
+   * Process the initial user to obtain the set of user
+   * supplied credentials (tokens were passed in by client).
+   * Removes the AM/RM token.
+   * If a keytab has been provided, also strip the HDFS delegation token.
+   * @param securityConfig slider security config
+   * @throws IOException
+   */
+  private void processAMCredentials(SecurityConfiguration securityConfig)
+      throws IOException {
+
+    List<Text> filteredTokens = new ArrayList<>(3);
+    filteredTokens.add(AMRMTokenIdentifier.KIND_NAME);
+    filteredTokens.add(TimelineDelegationTokenIdentifier.KIND_NAME);
+
+    boolean keytabProvided = securityConfig.isKeytabProvided();
+    log.info("Slider AM Security Mode: {}", keytabProvided ? "KEYTAB" : "TOKEN");
+    if (keytabProvided) {
+      filteredTokens.add(DelegationTokenIdentifier.HDFS_DELEGATION_KIND);
+    }
+    containerCredentials = CredentialUtils.filterTokens(
+        UserGroupInformation.getCurrentUser().getCredentials(),
+        filteredTokens);
+    log.info(CredentialUtils.dumpTokens(containerCredentials, "\n"));
+  }
+
+  /**
+   * Build up the port scanner. This may include setting a port range.
+   */
+  private void buildPortScanner(AggregateConf instanceDefinition)
+      throws BadConfigException {
+    portScanner = new PortScanner();
+    String portRange = instanceDefinition.
+        getAppConfOperations().getGlobalOptions().
+          getOption(SliderKeys.KEY_ALLOWED_PORT_RANGE, "0");
+    if (!"0".equals(portRange)) {
+        portScanner.setPortRange(portRange);
+    }
+  }
+  
+  /**
+   * Locate a port to request for a service such as RPC or web/REST.
+   * This uses port range definitions in the <code>instanceDefinition</code>
+   * to fix the port range \u2014if one is set.
+   * <p>
+   * The port returned is available at the time of the request; there are
+   * no guarantees as to how long that situation will last.
+   * @return the port to request.
+   * @throws SliderException
+   */
+  private int getPortToRequest() throws SliderException, IOException {
+    return portScanner.getAvailablePort();
+  }
+
+  private void uploadServerCertForLocalization(String clustername,
+                                               SliderFileSystem fs)
+      throws IOException {
+    Path certsDir = fs.buildClusterSecurityDirPath(clustername);
+    if (!fs.getFileSystem().exists(certsDir)) {
+      fs.getFileSystem().mkdirs(certsDir,
+        new FsPermission(FsAction.ALL, FsAction.NONE, FsAction.NONE));
+    }
+    Path destPath = new Path(certsDir, SliderKeys.CRT_FILE_NAME);
+    if (!fs.getFileSystem().exists(destPath)) {
+      fs.getFileSystem().copyFromLocalFile(
+          new Path(CertificateManager.getServerCertficateFilePath().getAbsolutePath()),
+          destPath);
+      log.info("Uploaded server cert to localization path {}", destPath);
+    }
+
+    fs.getFileSystem().setPermission(destPath,
+        new FsPermission(FsAction.READ, FsAction.NONE, FsAction.NONE));
+  }
+
+  protected void login(String principal, File localKeytabFile)
+      throws IOException, SliderException {
+    log.info("Logging in as {} with keytab {}", principal, localKeytabFile);
+    UserGroupInformation.loginUserFromKeytab(principal,
+                                             localKeytabFile.getAbsolutePath());
+    validateLoginUser(UserGroupInformation.getLoginUser());
+  }
+
+  /**
+   * Ensure that the user is generated from a keytab and has no HDFS delegation
+   * tokens.
+   *
+   * @param user user to validate
+   * @throws SliderException
+   */
+  protected void validateLoginUser(UserGroupInformation user)
+      throws SliderException {
+    if (!user.isFromKeytab()) {
+      log.error("User is not holding on a keytab in a secure deployment:" +
+          " slider will fail as tokens expire");
+    }
+    Credentials credentials = user.getCredentials();
+    Iterator<Token<? extends TokenIdentifier>> iter =
+        credentials.getAllTokens().iterator();
+    while (iter.hasNext()) {
+      Token<? extends TokenIdentifier> token = iter.next();
+      log.info("Token {}", token.getKind());
+      if (token.getKind().equals(
+          DelegationTokenIdentifier.HDFS_DELEGATION_KIND)) {
+        log.info("HDFS delegation token {}.  Removing...", token);
+        iter.remove();
+      }
+    }
+  }
+
+  /**
+   * Set up and start the agent web application 
+   * @param appInformation application information
+   * @param serviceConf service configuration
+   * @param webAppApi web app API instance to bind to
+   * @throws IOException
+   */
+  private void startAgentWebApp(MapOperations appInformation,
+      Configuration serviceConf, WebAppApiImpl webAppApi) throws IOException, SliderException {
+    URL[] urls = ((URLClassLoader) AgentWebApp.class.getClassLoader() ).getURLs();
+    StringBuilder sb = new StringBuilder("AM classpath:");
+    for (URL url : urls) {
+      sb.append("\n").append(url.toString());
+    }
+    LOG_YARN.debug(sb.append("\n").toString());
+    initAMFilterOptions(serviceConf);
+
+
+    // Start up the agent web app and track the URL for it
+    MapOperations appMasterConfig = getInstanceDefinition()
+        .getAppConfOperations().getComponent(SliderKeys.COMPONENT_AM);
+    AgentWebApp agentWebApp = AgentWebApp.$for(AgentWebApp.BASE_PATH,
+        webAppApi,
+        RestPaths.AGENT_WS_CONTEXT)
+        .withComponentConfig(appMasterConfig)
+        .withPort(getPortToRequest())
+        .withSecuredPort(getPortToRequest())
+            .start();
+    agentOpsUrl =
+        "https://" + appMasterHostname + ":" + agentWebApp.getSecuredPort();
+    agentStatusUrl =
+        "https://" + appMasterHostname + ":" + agentWebApp.getPort();
+    AgentService agentService =
+      new AgentService("slider-agent", agentWebApp);
+
+    agentService.init(serviceConf);
+    agentService.start();
+    addService(agentService);
+
+    appInformation.put(StatusKeys.INFO_AM_AGENT_OPS_URL, agentOpsUrl + "/");
+    appInformation.put(StatusKeys.INFO_AM_AGENT_STATUS_URL, agentStatusUrl + "/");
+    appInformation.set(StatusKeys.INFO_AM_AGENT_STATUS_PORT,
+                       agentWebApp.getPort());
+    appInformation.set(StatusKeys.INFO_AM_AGENT_OPS_PORT,
+                       agentWebApp.getSecuredPort());
+  }
+
+  /**
+   * Set up the AM filter 
+   * @param serviceConf configuration to patch
+   */
+  private void initAMFilterOptions(Configuration serviceConf) {
+    // IP filtering
+    String amFilterName = AM_FILTER_NAME;
+
+    // This is here until YARN supports proxy & redirect operations
+    // on verbs other than GET, and is only supported for testing
+    if (X_DEV_INSECURE_REQUIRED && serviceConf.getBoolean(X_DEV_INSECURE_WS, 
+        X_DEV_INSECURE_DEFAULT)) {
+      log.warn("Insecure filter enabled: REST operations are unauthenticated");
+      amFilterName = InsecureAmFilterInitializer.NAME;
+    }
+
+    serviceConf.set(HADOOP_HTTP_FILTER_INITIALIZERS, amFilterName);
+  }
+
+  /**
+   * This registers the service instance and its external values
+   * @param instanceName name of this instance
+   * @param appId application ID
+   * @throws IOException
+   */
+  public void registerServiceInstance(String instanceName,
+      ApplicationId appId) throws IOException {
+    
+    
+    // the registry is running, so register services
+    URL amWebURI = new URL(appMasterProxiedUrl);
+    URL agentOpsURI = new URL(agentOpsUrl);
+    URL agentStatusURI = new URL(agentStatusUrl);
+
+    //Give the provider restricted access to the state, registry
+    setupInitialRegistryPaths();
+    yarnRegistryOperations = new YarnRegistryViewForProviders(
+        registryOperations,
+        service_user_name,
+        SliderKeys.APP_TYPE,
+        instanceName,
+        appAttemptID);
+    providerService.bindToYarnRegistry(yarnRegistryOperations);
+    sliderAMProvider.bindToYarnRegistry(yarnRegistryOperations);
+
+    // Yarn registry
+    ServiceRecord serviceRecord = new ServiceRecord();
+    serviceRecord.set(YarnRegistryAttributes.YARN_ID, appId.toString());
+    serviceRecord.set(YarnRegistryAttributes.YARN_PERSISTENCE,
+        PersistencePolicies.APPLICATION);
+    serviceRecord.description = "Slider Application Master";
+
+    serviceRecord.addExternalEndpoint(
+        RegistryTypeUtils.ipcEndpoint(
+            CustomRegistryConstants.AM_IPC_PROTOCOL,
+            rpcServiceAddress));
+            
+    // internal services
+    sliderAMProvider.applyInitialRegistryDefinitions(amWebURI,
+        agentOpsURI,
+        agentStatusURI,
+        serviceRecord);
+
+    // provider service dynamic definitions.
+    providerService.applyInitialRegistryDefinitions(amWebURI,
+        agentOpsURI,
+        agentStatusURI,
+        serviceRecord);
+
+    // set any provided attributes
+    setProvidedServiceRecordAttributes(
+        getInstanceDefinition().getAppConfOperations().getComponent(
+            SliderKeys.COMPONENT_AM), serviceRecord);
+
+    // register the service's entry
+    log.info("Service Record \n{}", serviceRecord);
+    yarnRegistryOperations.registerSelf(serviceRecord, true);
+    log.info("Registered service under {}; absolute path {}",
+        yarnRegistryOperations.getSelfRegistrationPath(),
+        yarnRegistryOperations.getAbsoluteSelfRegistrationPath());
+    
+    boolean isFirstAttempt = 1 == appAttemptID.getAttemptId();
+    // delete the children in case there are any and this is an AM startup.
+    // just to make sure everything underneath is purged
+    if (isFirstAttempt) {
+      yarnRegistryOperations.deleteChildren(
+          yarnRegistryOperations.getSelfRegistrationPath(),
+          true);
+    }
+  }
+
+  /**
+   * TODO: purge this once RM is doing the work
+   * @throws IOException
+   */
+  protected void setupInitialRegistryPaths() throws IOException {
+    if (registryOperations instanceof RMRegistryOperationsService) {
+      RMRegistryOperationsService rmRegOperations =
+          (RMRegistryOperationsService) registryOperations;
+      rmRegOperations.initUserRegistryAsync(service_user_name);
+    }
+  }
+
+  /**
+   * Handler for {@link RegisterComponentInstance action}
+   * Register/re-register an ephemeral container that is already in the app state
+   * @param id the component
+   * @param description component description
+   * @param type component type
+   * @return true if the component is registered
+   */
+  public boolean registerComponent(ContainerId id, String description,
+      String type) throws IOException {
+    RoleInstance instance = appState.getOwnedContainer(id);
+    if (instance == null) {
+      return false;
+    }
+    // this is where component registrations  go
+    log.info("Registering component {}", id);
+    String cid = RegistryPathUtils.encodeYarnID(id.toString());
+    ServiceRecord container = new ServiceRecord();
+    container.set(YarnRegistryAttributes.YARN_ID, cid);
+    container.description = description;
+    container.set(YarnRegistryAttributes.YARN_PERSISTENCE,
+        PersistencePolicies.CONTAINER);
+    MapOperations compOps = getInstanceDefinition().getAppConfOperations().
+        getComponent(type);
+    setProvidedServiceRecordAttributes(compOps, container);
+    try {
+      yarnRegistryOperations.putComponent(cid, container);
+    } catch (IOException e) {
+      log.warn("Failed to register container {}/{}: {}",
+          id, description, e, e);
+      return false;
+    }
+    return true;
+  }
+
+  protected void setProvidedServiceRecordAttributes(MapOperations ops,
+                                                  ServiceRecord record) {
+    String prefix = RoleKeys.SERVICE_RECORD_ATTRIBUTE_PREFIX;
+    for (Map.Entry<String, String> entry : ops.entrySet()) {
+      if (entry.getKey().startsWith(
+          prefix)) {
+        String key = entry.getKey().substring(
+            prefix.length() + 1);
+        record.set(key, entry.getValue().trim());
+      }
+    }
+  }
+
+  /**
+   * Handler for {@link UnregisterComponentInstance}
+   * 
+   * unregister a component. At the time this message is received,
+   * the component may not have been registered
+   * @param id the component
+   */
+  public void unregisterComponent(ContainerId id) {
+    log.info("Unregistering component {}", id);
+    if (yarnRegistryOperations == null) {
+      log.warn("Processing unregister component event before initialization " +
+               "completed; init flag ={}", initCompleted);
+      return;
+    }
+    String cid = RegistryPathUtils.encodeYarnID(id.toString());
+    try {
+      yarnRegistryOperations.deleteComponent(cid);
+    } catch (IOException e) {
+      log.warn("Failed to delete container {} : {}", id, e, e);
+    }
+  }
+
+  /**
+   * looks for a specific case where a token file is provided as an environment
+   * variable, yet the file is not there.
+   * 
+   * This surfaced (once) in HBase, where its HDFS library was looking for this,
+   * and somehow the token was missing. This is a check in the AM so that
+   * if the problem re-occurs, the AM can fail with a more meaningful message.
+   * 
+   */
+  private void checkAndWarnForAuthTokenProblems() {
+    String fileLocation =
+      System.getenv(UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION);
+    if (fileLocation != null) {
+      File tokenFile = new File(fileLocation);
+      if (!tokenFile.exists()) {
+        log.warn("Token file {} specified in {} not found", tokenFile,
+                 UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION);
+      }
+    }
+  }
+
+  /**
+   * Build the configuration directory passed in or of the target FS
+   * @return the file
+   */
+  public File getLocalConfDir() {
+    File confdir =
+      new File(SliderKeys.PROPAGATED_CONF_DIR_NAME).getAbsoluteFile();
+    return confdir;
+  }
+
+  /**
+   * Get the path to the DFS configuration that is defined in the cluster specification 
+   * @return the generated configuration dir
+   */
+  public String getGeneratedConfDir() {
+    return getGlobalInternalOptions().get(
+        InternalKeys.INTERNAL_GENERATED_CONF_PATH);
+  }
+
+  /**
+   * Get the global internal options for the AM
+   * @return a map to access the internals
+   */
+  public MapOperations getGlobalInternalOptions() {
+    return getInstanceDefinition()
+      .getInternalOperations().
+      getGlobalOptions();
+  }
+
+  /**
+   * Get the filesystem of this cluster
+   * @return the FS of the config
+   */
+  public SliderFileSystem getClusterFS() throws IOException {
+    return new SliderFileSystem(getConfig());
+  }
+
+  /**
+   * Get the AM log
+   * @return the log of the AM
+   */
+  public static Logger getLog() {
+    return log;
+  }
+
+  /**
+   * Get the application state
+   * @return the application state
+   */
+  public AppState getAppState() {
+    return appState;
+  }
+
+  /**
+   * Block until it is signalled that the AM is done
+   */
+  private void waitForAMCompletionSignal() {
+    AMExecutionStateLock.lock();
+    try {
+      if (!amCompletionFlag.get()) {
+        log.debug("blocking until signalled to terminate");
+        isAMCompleted.awaitUninterruptibly();
+      }
+    } finally {
+      AMExecutionStateLock.unlock();
+    }
+  }
+
+  /**
+   * Signal that the AM is complete .. queues it in a separate thread
+   *
+   * @param stopActionRequest request containing shutdown details
+   */
+  public synchronized void signalAMComplete(ActionStopSlider stopActionRequest) {
+    // this is a queued action: schedule it through the queues
+    schedule(stopActionRequest);
+  }
+
+  /**
+   * Signal that the AM is complete
+   *
+   * @param stopActionRequest request containing shutdown details
+   */
+  public synchronized void onAMStop(ActionStopSlider stopActionRequest) {
+
+    AMExecutionStateLock.lock();
+    try {
+      if (amCompletionFlag.compareAndSet(false, true)) {
+        // first stop request received
+        this.stopAction = stopActionRequest;
+        isAMCompleted.signal();
+      }
+    } finally {
+      AMExecutionStateLock.unlock();
+    }
+  }
+
+  
+  /**
+   * trigger the YARN cluster termination process
+   * @return the exit code
+   * @throws Exception if the stop action contained an Exception which implements
+   * ExitCodeProvider
+   */
+  private synchronized int finish() throws Exception {
+    Preconditions.checkNotNull(stopAction, "null stop action");
+    FinalApplicationStatus appStatus;
+    log.info("Triggering shutdown of the AM: {}", stopAction);
+
+    String appMessage = stopAction.getMessage();
+    //stop the daemon & grab its exit code
+    int exitCode = stopAction.getExitCode();
+    Exception exception = stopAction.getEx();
+
+    appStatus = stopAction.getFinalApplicationStatus();
+    if (!spawnedProcessExitedBeforeShutdownTriggered) {
+      //stopped the forked process but don't worry about its exit code
+      int forkedExitCode = stopForkedProcess();
+      log.debug("Stopped forked process: exit code={}", forkedExitCode);
+    }
+
+    // make sure the AM is actually registered. If not, there's no point
+    // trying to unregister it
+    if (amRegistrationData == null) {
+      log.info("Application attempt not yet registered; skipping unregistration");
+      if (exception != null) {
+        throw exception;
+      }
+      return exitCode;
+    }
+
+    //stop any launches in progress
+    launchService.stop();
+
+    //now release all containers
+    releaseAllContainers();
+
+    // When the application completes, it should send a finish application
+    // signal to the RM
+    log.info("Application completed. Signalling finish to RM");
+
+    try {
+      log.info("Unregistering AM status={} message={}", appStatus, appMessage);
+      asyncRMClient.unregisterApplicationMaster(appStatus, appMessage, null);
+    } catch (InvalidApplicationMasterRequestException e) {
+      log.info("Application not found in YARN application list;" +
+        " it may have been terminated/YARN shutdown in progress: {}", e, e);
+    } catch (YarnException | IOException e) {
+      log.info("Failed to unregister application: " + e, e);
+    }
+    if (exception != null) {
+      throw exception;
+    }
+    return exitCode;
+  }
+
+    /**
+     * Get diagnostics info about containers
+     */
+  private String getContainerDiagnosticInfo() {
+
+    return appState.getContainerDiagnosticInfo();
+  }
+
+  public Object getProxy(Class protocol, InetSocketAddress addr) {
+    return yarnRPC.getProxy(protocol, addr, getConfig());
+  }
+
+  /**
+   * Start the slider RPC server
+   */
+  private void startSliderRPCServer(AggregateConf instanceDefinition)
+      throws IOException, SliderException {
+    verifyIPCAccess();
+
+    sliderIPCService = new SliderIPCService(
+        this,
+        certificateManager,
+        stateForProviders,
+        actionQueues,
+        metricsAndMonitoring,
+        contentCache);
+
+    deployChildService(sliderIPCService);
+    SliderClusterProtocolPBImpl protobufRelay =
+        new SliderClusterProtocolPBImpl(sliderIPCService);
+    BlockingService blockingService = SliderClusterAPI.SliderClusterProtocolPB
+        .newReflectiveBlockingService(
+            protobufRelay);
+
+    int port = getPortToRequest();
+    InetSocketAddress rpcAddress = new InetSocketAddress("0.0.0.0", port);
+    rpcService =
+        new WorkflowRpcService("SliderRPC",
+            RpcBinder.createProtobufServer(rpcAddress, getConfig(),
+                secretManager,
+            NUM_RPC_HANDLERS,
+            blockingService,
+            null));
+    deployChildService(rpcService);
+  }
+
+  /**
+   * verify that if the cluster is authed, the ACLs are set.
+   * @throws BadConfigException if Authorization is set without any ACL
+   */
+  private void verifyIPCAccess() throws BadConfigException {
+    boolean authorization = getConfig().getBoolean(
+        CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION,
+        false);
+    String acls = getConfig().get(KEY_PROTOCOL_ACL);
+    if (authorization && SliderUtils.isUnset(acls)) {
+      throw new BadConfigException("Application has IPC authorization enabled in " +
+          CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION +
+          " but no ACLs in " + KEY_PROTOCOL_ACL);
+    }
+  }
+
+
+/* =================================================================== */
+/* AMRMClientAsync callbacks */
+/* =================================================================== */
+
+  /**
+   * Callback event when a container is allocated.
+   * 
+   * The app state is updated with the allocation, and builds up a list
+   * of assignments and RM operations. The assignments are 
+   * handed off into the pool of service launchers to asynchronously schedule
+   * container launch operations.
+   * 
+   * The operations are run in sequence; they are expected to be 0 or more
+   * release operations (to handle over-allocations)
+   * 
+   * @param allocatedContainers list of containers that are now ready to be
+   * given work.
+   */
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override //AMRMClientAsync
+  public void onContainersAllocated(List<Container> allocatedContainers) {
+    LOG_YARN.info("onContainersAllocated({})", allocatedContainers.size());
+    List<ContainerAssignment> assignments = new ArrayList<>();
+    List<AbstractRMOperation> operations = new ArrayList<>();
+    
+    //app state makes all the decisions
+    appState.onContainersAllocated(allocatedContainers, assignments, operations);
+
+    //for each assignment: instantiate that role
+    for (ContainerAssignment assignment : assignments) {
+      try {
+        launchService.launchRole(assignment, getInstanceDefinition(),
+            buildContainerCredentials());
+      } catch (IOException e) {
+        // Can be caused by failure to renew credentials with the remote
+        // service. If so, don't launch the application. Container is retained,
+        // though YARN will take it away after a timeout.
+        log.error("Failed to build credentials to launch container: {}", e, e);
+
+      }
+    }
+
+    //for all the operations, exec them
+    execute(operations);
+    log.info("Diagnostics: {}", getContainerDiagnosticInfo());
+  }
+
+  @Override //AMRMClientAsync
+  public synchronized void onContainersCompleted(List<ContainerStatus> completedContainers) {
+    LOG_YARN.info("onContainersCompleted([{}]", completedContainers.size());
+    for (ContainerStatus status : completedContainers) {
+      ContainerId containerId = status.getContainerId();
+      LOG_YARN.info("Container Completion for" +
+                    " containerID={}," +
+                    " state={}," +
+                    " exitStatus={}," +
+                    " diagnostics={}",
+                    containerId, status.getState(),
+                    status.getExitStatus(),
+                    status.getDiagnostics());
+
+      // non complete containers should not be here
+      assert (status.getState() == ContainerState.COMPLETE);
+      AppState.NodeCompletionResult result = appState.onCompletedNode(status);
+      if (result.containerFailed) {
+        RoleInstance ri = result.roleInstance;
+        log.error("Role instance {} failed ", ri);
+      }
+
+      //  known nodes trigger notifications
+      if(!result.unknownNode) {
+        getProviderService().notifyContainerCompleted(containerId);
+        queue(new UnregisterComponentInstance(containerId, 0,
+            TimeUnit.MILLISECONDS));
+      }
+    }
+
+    reviewRequestAndReleaseNodes("onContainersCompleted");
+  }
+
+  /**
+   * Signal that containers are being upgraded. Containers specified with
+   * --containers option and all containers of all roles specified with
+   * --components option are merged and upgraded.
+   * 
+   * @param upgradeContainersRequest
+   *          request containing upgrade details
+   */
+  public synchronized void onUpgradeContainers(
+      ActionUpgradeContainers upgradeContainersRequest) throws IOException,
+      SliderException {
+    LOG_YARN.info("onUpgradeContainers({})",
+        upgradeContainersRequest.getMessage());
+    Set<String> containers = upgradeContainersRequest.getContainers() == null ? new HashSet<String>()
+        : upgradeContainersRequest.getContainers();
+    LOG_YARN.info("  Container list provided (total {}) : {}",
+        containers.size(), containers);
+    Set<String> components = upgradeContainersRequest.getComponents() == null ? new HashSet<String>()
+        : upgradeContainersRequest.getComponents();
+    LOG_YARN.info("  Component list provided (total {}) : {}",
+        components.size(), components);
+    // If components are specified as well, then grab all the containers of
+    // each of the components (roles)
+    if (CollectionUtils.isNotEmpty(components)) {
+      Map<ContainerId, RoleInstance> liveContainers = appState.getLiveContainers();
+      if (CollectionUtils.isNotEmpty(liveContainers.keySet())) {
+        Map<String, Set<String>> roleContainerMap = prepareRoleContainerMap(liveContainers);
+        for (String component : components) {
+          Set<String> roleContainers = roleContainerMap.get(component);
+          if (roleContainers != null) {
+            containers.addAll(roleContainers);
+          }
+        }
+      }
+    }
+    LOG_YARN.info("Final list of containers to be upgraded (total {}) : {}",
+        containers.size(), containers);
+    if (providerService instanceof AgentProviderService) {
+      AgentProviderService agentProviderService = (AgentProviderService) providerService;
+      agentProviderService.setInUpgradeMode(true);
+      agentProviderService.addUpgradeContainers(containers);
+    }
+  }
+
+  // create a reverse map of roles -> set of all live containers
+  private Map<String, Set<String>> prepareRoleContainerMap(
+      Map<ContainerId, RoleInstance> liveContainers) {
+    // liveContainers is ensured to be not empty
+    Map<String, Set<String>> roleContainerMap = new HashMap<>();
+    for (Map.Entry<ContainerId, RoleInstance> liveContainer : liveContainers
+        .entrySet()) {
+      RoleInstance role = liveContainer.getValue();
+      if (roleContainerMap.containsKey(role.role)) {
+        roleContainerMap.get(role.role).add(liveContainer.getKey().toString());
+      } else {
+        Set<String> containers = new HashSet<String>();
+        containers.add(liveContainer.getKey().toString());
+        roleContainerMap.put(role.role, containers);
+      }
+    }
+    return roleContainerMap;
+  }
+
+  /**
+   * Implementation of cluster flexing.
+   * It should be the only way that anything -even the AM itself on startup-
+   * asks for nodes. 
+   * @param resources the resource tree
+   * @throws SliderException slider problems, including invalid configs
+   * @throws IOException IO problems
+   */
+  public void flexCluster(ConfTree resources)
+      throws IOException, SliderException {
+
+    AggregateConf newConf =
+        new AggregateConf(appState.getInstanceDefinitionSnapshot());
+    newConf.setResources(resources);
+    // verify the new definition is valid
+    sliderAMProvider.validateInstanceDefinition(newConf);
+    providerService.validateInstanceDefinition(newConf);
+
+    appState.updateResourceDefinitions(resources);
+
+    // reset the scheduled windows...the values
+    // may have changed
+    appState.resetFailureCounts();
+
+    // ask for more containers if needed
+    reviewRequestAndReleaseNodes("flexCluster");
+  }
+
+  /**
+   * Schedule the failure window
+   * @param resources the resource tree
+   * @throws BadConfigException if the window is out of range
+   */
+  private void scheduleFailureWindowResets(ConfTree resources) throws
+      BadConfigException {
+    ResetFailureWindow reset = new ResetFailureWindow();
+    ConfTreeOperations ops = new ConfTreeOperations(resources);
+    MapOperations globals = ops.getGlobalOptions();
+    long seconds = globals.getTimeRange(ResourceKeys.CONTAINER_FAILURE_WINDOW,
+        ResourceKeys.DEFAULT_CONTAINER_FAILURE_WINDOW_DAYS,
+        ResourceKeys.DEFAULT_CONTAINER_FAILURE_WINDOW_HOURS,
+        ResourceKeys.DEFAULT_CONTAINER_FAILURE_WINDOW_MINUTES, 0);
+    if (seconds > 0) {
+      log.info(
+          "Scheduling the failure window reset interval to every {} seconds",
+          seconds);
+      RenewingAction<ResetFailureWindow> renew = new RenewingAction<>(
+          reset, seconds, seconds, TimeUnit.SECONDS, 0);
+      actionQueues.renewing("failures", renew);
+    } else {
+      log.info("Failure window reset interval is not set");
+    }
+  }
+
+  /**
+   * Schedule the escalation action
+   * @param internal
+   * @throws BadConfigException
+   */
+  private void scheduleEscalation(ConfTree internal) throws BadConfigException {
+    EscalateOutstandingRequests escalate = new EscalateOutstandingRequests();
+    ConfTreeOperations ops = new ConfTreeOperations(internal);
+    int seconds = ops.getGlobalOptions().getOptionInt(InternalKeys.ESCALATION_CHECK_INTERVAL,
+        InternalKeys.DEFAULT_ESCALATION_CHECK_INTERVAL);
+    RenewingAction<EscalateOutstandingRequests> renew = new RenewingAction<>(
+        escalate, seconds, seconds, TimeUnit.SECONDS, 0);
+    actionQueues.renewing("escalation", renew);
+  }
+  
+  /**
+   * Look at where the current node state is -and whether it should be changed
+   * @param reason reason for operation
+   */
+  private synchronized void reviewRequestAndReleaseNodes(String reason) {
+    log.debug("reviewRequestAndReleaseNodes({})", reason);
+    queue(new ReviewAndFlexApplicationSize(reason, 0, TimeUnit.SECONDS));
+  }
+
+  /**
+   * Handle the event requesting a review ... look at the queue and decide
+   * whether to act or not
+   * @param action action triggering the event. It may be put
+   * back into the queue
+   * @throws SliderInternalStateException
+   */
+  public void handleReviewAndFlexApplicationSize(ReviewAndFlexApplicationSize action)
+      throws SliderInternalStateException {
+
+    if ( actionQueues.hasQueuedActionWithAttribute(
+        AsyncAction.ATTR_REVIEWS_APP_SIZE | AsyncAction.ATTR_HALTS_APP)) {
+      // this operation isn't needed at all -existing duplicate or shutdown due
+      return;
+    }
+    // if there is an action which changes cluster size, wait
+    if (actionQueues.hasQueuedActionWithAttribute(
+        AsyncAction.ATTR_CHANGES_APP_SIZE)) {
+      // place the action at the back of the queue
+      actionQueues.put(action);
+    }
+    
+    executeNodeReview(action.name);
+  }
+  
+  /**
+   * Look at where the current node state is -and whether it should be changed
+   */
+  public synchronized void executeNodeReview(String reason)
+      throws SliderInternalStateException {
+    
+    log.debug("in executeNodeReview({})", reason);
+    if (amCompletionFlag.get()) {
+      log.info("Ignoring node review operation: shutdown in progress");
+    }
+    try {
+      List<AbstractRMOperation> allOperations = appState.reviewRequestAndReleaseNodes();
+      // tell the provider
+      providerRMOperationHandler.execute(allOperations);
+      //now apply the operations
+      execute(allOperations);
+    } catch (TriggerClusterTeardownException e) {
+      //App state has decided that it is time to exit
+      log.error("Cluster teardown triggered {}", e, e);
+      queue(new ActionStopSlider(e));
+    }
+  }
+
+  /**
+   * Escalate operation as triggered by external timer.
+   * <p>
+   * Get the list of new operations off the AM, then executest them.
+   */
+  public void escalateOutstandingRequests() {
+    List<AbstractRMOperation> operations = appState.escalateOutstandingRequests();
+    providerRMOperationHandler.execute(operations);
+    execute(operations);
+  }
+
+
+  /**
+   * Shutdown operation: release all containers
+   */
+  private void releaseAllContainers() {
+    if (providerService instanceof AgentProviderService) {
+      log.info("Setting stopInitiated flag to true");
+      AgentProviderService agentProviderService = (AgentProviderService) providerService;
+      agentProviderService.setAppStopInitiated(true);
+    }
+    // Add the sleep here (before releasing containers) so that applications get
+    // time to perform graceful shutdown
+    try {
+      long timeout = getContainerReleaseTimeout();
+      if (timeout > 0) {
+        Thread.sleep(timeout);
+      }
+    } catch (InterruptedException e) {
+      log.info("Sleep for container release interrupted");
+    } finally {
+      List<AbstractRMOperation> operations = appState.releaseAllContainers();
+      providerRMOperationHandler.execute(operations);
+      // now apply the operations
+      execute(operations);
+    }
+  }
+
+  private long getContainerReleaseTimeout() {
+    // Get container release timeout in millis or 0 if the property is not set.
+    // If non-zero then add the agent heartbeat delay time, since it can take up
+    // to that much time for agents to receive the stop command.
+    int timeout = getInstanceDefinition().getAppConfOperations()
+        .getGlobalOptions()
+        .getOptionInt(SliderKeys.APP_CONTAINER_RELEASE_TIMEOUT, 0);
+    if (timeout > 0) {
+      timeout += SliderKeys.APP_CONTAINER_HEARTBEAT_INTERVAL_SEC;
+    }
+    // convert to millis
+    long timeoutInMillis = timeout * 1000l;
+    log.info("Container release timeout in millis = {}", timeoutInMillis);
+    return timeoutInMillis;
+  }
+
+  /**
+   * RM wants to shut down the AM
+   */
+  @Override //AMRMClientAsync
+  public void onShutdownRequest() {
+    LOG_YARN.info("Shutdown Request received");
+    signalAMComplete(new ActionStopSlider("stop",
+        EXIT_SUCCESS,
+        FinalApplicationStatus.SUCCEEDED,
+        "Shutdown requested from RM"));
+  }
+
+  /**
+   * Monitored nodes have been changed
+   * @param updatedNodes list of updated nodes
+   */
+  @Override //AMRMClientAsync
+  public void onNodesUpdated(List<NodeReport> updatedNodes) {
+    LOG_YARN.info("onNodesUpdated({})", updatedNodes.size());
+    log.info("Updated nodes {}", updatedNodes);
+    // Check if any nodes are lost or revived and update state accordingly
+
+    AppState.NodeUpdatedOutcome outcome = appState.onNodesUpdated(updatedNodes);
+    if (!outcome.operations.isEmpty()) {
+      execute(outcome.operations);
+    }
+    // trigger a review if the cluster changed
+    if (outcome.clusterChanged) {
+      reviewRequestAndReleaseNodes("nodes updated");
+    }
+  }
+
+  /**
+   * heartbeat operation; return the ratio of requested
+   * to actual
+   * @return progress
+   */
+  @Override //AMRMClientAsync
+  public float getProgress() {
+    return appState.getApplicationProgressPercentage();
+  }
+
+  @Override //AMRMClientAsync
+  public void onError(Throwable e) {
+    //callback says it's time to finish
+    LOG_YARN.error("AMRMClientAsync.onError() received {}", e, e);
+    signalAMComplete(new ActionStopSlider("stop",
+        EXIT_EXCEPTION_THROWN,
+        FinalApplicationStatus.FAILED,
+        "AMRMClientAsync.onError() received " + e));
+  }
+  
+/* =================================================================== */
+/* RMOperationHandlerActions */
+/* =================================================================== */
+
+ 
+  @Override
+  public void execute(List<AbstractRMOperation> operations) {
+    rmOperationHandler.execute(operations);
+  }
+
+  @Override
+  public void releaseAssignedContainer(ContainerId containerId) {
+    rmOperationHandler.releaseAssignedContainer(containerId);
+  }
+
+  @Override
+  public void addContainerRequest(AMRMClient.ContainerRequest req) {
+    rmOperationHandler.addContainerRequest(req);
+  }
+
+  @Override
+  public int cancelContainerRequests(Priority priority1,
+      Priority priority2,
+      int count) {
+    return rmOperationHandler.cancelContainerRequests(priority1, priority2, count);
+  }
+
+  @Override
+  public void cancelSingleRequest(AMRMClient.ContainerRequest request) {
+    rmOperationHandler.cancelSingleRequest(request);
+  }
+
+/* =================================================================== */
+/* END */
+/* =================================================================== */
+
+  /**
+   * Launch the provider service
+   *
+   * @param instanceDefinition definition of the service
+   * @param confDir directory of config data
+   * @throws IOException
+   * @throws SliderException
+   */
+  protected synchronized void launchProviderService(AggregateConf instanceDefinition,
+                                                    File confDir)
+    throws IOException, SliderException {
+    Map<String, String> env = new HashMap<>();
+    boolean execStarted = providerService.exec(instanceDefinition, confDir, env,
+        this);
+    if (execStarted) {
+      providerService.registerServiceListener(this);
+      providerService.start();
+    } else {
+      // didn't start, so don't register
+      providerService.start();
+      // and send the started event ourselves
+      eventCallbackEvent(null);
+    }
+  }
+
+  /* =================================================================== */
+  /* EventCallback  from the child or ourselves directly */
+  /* =================================================================== */
+
+  @Override // ProviderCompleted
+  public void eventCallbackEvent(Object parameter) {
+    // signalled that the child process is up.
+    appState.noteAMLive();
+    // now ask for the cluster nodes
+    try {
+      flexCluster(getInstanceDefinition().getResources());
+    } catch (Exception e) {
+      // cluster flex failure: log
+      log.error("Failed to flex cluster nodes: {}", e, e);
+      // then what? exit
+      queue(new ActionStopSlider(e));
+    }
+  }
+
+  /**
+   * report container loss. If this isn't already known about, react
+   *
+   * @param containerId       id of the container which has failed
+   * @throws SliderException
+   */
+  public synchronized void providerLostContainer(
+      ContainerId containerId)
+      throws SliderException {
+    log.info("containerLostContactWithProvider: container {} lost",
+        containerId);
+    RoleInstance activeContainer = appState.getOwnedContainer(containerId);
+    if (activeContainer != null) {
+      execute(appState.releaseContainer(containerId));
+      // ask for more containers if needed
+      log.info("Container released; triggering review");
+      reviewRequestAndReleaseNodes("Loss of container");
+    } else {
+      log.info("Container not in active set - ignoring");
+    }
+  }
+
+  /* =================================================================== */
+  /* ServiceStateChangeListener */
+  /* =================================================================== */
+
+  /**
+   * Received on listening service termination.
+   * @param service the service that has changed.
+   */
+  @Override //ServiceStateChangeListener
+  public void stateChanged(Service service) {
+    if (service == providerService && service.isInState(STATE.STOPPED)) {
+      //its the current master process in play
+      int exitCode = providerService.getExitCode();
+      int mappedProcessExitCode = exitCode;
+
+      boolean shouldTriggerFailure = !amCompletionFlag.get()
+         && (mappedProcessExitCode != 0);
+
+      if (shouldTriggerFailure) {
+        String reason =
+            "Spawned process failed with raw " + exitCode + " mapped to " +
+            mappedProcessExitCode;
+        ActionStopSlider stop = new ActionStopSlider("stop",
+            mappedProcessExitCode,
+            FinalApplicationStatus.FAILED,
+            reason);
+        //this wasn't expected: the process finished early
+        spawnedProcessExitedBeforeShutdownTriggered = true;
+        log.info(
+          "Process has exited with exit code {} mapped to {} -triggering termination",
+          exitCode,
+          mappedProcessExitCode);
+
+        //tell the AM the cluster is complete 
+        signalAMComplete(stop);
+      } else {
+        //we don't care
+        log.info(
+          "Process has exited with exit code {} mapped to {} -ignoring",
+          exitCode,
+          mappedProcessExitCode);
+      }
+    } else {
+      super.stateChanged(service);
+    }
+  }
+
+  /**
+   * stop forked process if it the running process var is not null
+   * @return the process exit code
+   */
+  protected synchronized Integer stopForkedProcess() {
+    providerService.stop();
+    return providerService.getExitCode();
+  }
+
+  /**
+   *  Async start container request
+   * @param container container
+   * @param ctx context
+   * @param instance node details
+   */
+  public void startContainer(Container container,
+                             ContainerLaunchContext ctx,
+                             RoleInstance instance) throws IOException {
+    appState.containerStartSubmitted(container, instance);
+        
+    nmClientAsync.startContainerAsync(container, ctx);
+  }
+
+  /**
+   * Build the credentials needed for containers. This will include
+   * getting new delegation tokens for HDFS if the AM is running
+   * with a keytab.
+   * @return a buffer of credentials
+   * @throws IOException
+   */
+
+  private Credentials buildContainerCredentials() throws IOException {
+    Credentials credentials = new Credentials(containerCredentials);
+    if (securityConfiguration.isKeytabProvided()) {
+      CredentialUtils.addSelfRenewableFSDelegationTokens(
+          getClusterFS().getFileSystem(),
+          credentials);
+    }
+    return credentials;
+  }
+
+  @Override //  NMClientAsync.CallbackHandler 
+  public void onContainerStopped(ContainerId containerId) {
+    // do nothing but log: container events from the AM
+    // are the source of container halt details to react to
+    log.info("onContainerStopped {} ", containerId);
+  }
+
+  @Override //  NMClientAsync.CallbackHandler 
+  public void onContainerStarted(ContainerId containerId,
+      Map<String, ByteBuffer> allServiceResponse) {
+    LOG_YARN.info("Started Container {} ", containerId);
+    RoleInstance cinfo = appState.onNodeManagerContainerStarted(containerId);
+    if (cinfo != null) {
+      LOG_YARN.info("Deployed instance of role {} onto {}",
+          cinfo.role, containerId);
+      //trigger an async container status
+      nmClientAsync.getContainerStatusAsync(containerId,
+                                            cinfo.container.getNodeId());
+      // push out a registration
+      queue(new RegisterComponentInstance(containerId, cinfo.role, cinfo.group,
+          0, TimeUnit.MILLISECONDS));
+      
+    } else {
+      //this is a hypothetical path not seen. We react by warning
+      log.error("Noti

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[07/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentEnv.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentEnv.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentEnv.java
new file mode 100644
index 0000000..781ae00
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentEnv.java
@@ -0,0 +1,376 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.agent;
+
+import com.google.gson.annotations.SerializedName;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class AgentEnv {
+
+  /**
+   * Various directories, configurable in <code>ambari-agent.ini</code>
+   */
+  private Directory[] stackFoldersAndFiles = new Directory[0];
+
+  /**
+   * Directories that match name <code>/etc/alternatives/*conf</code>
+   */
+  private Alternative[] alternatives = new Alternative[0];
+
+  /**
+   * List of existing users
+   */
+  private ExistingUser[] existingUsers = new ExistingUser[0];
+
+  /**
+   * List of repos
+   */
+  private String[] existingRepos = new String[0];
+
+  /**
+   * List of packages
+   */
+  private PackageDetail[] installedPackages = new PackageDetail[0];
+
+  /**
+   * The host health report
+   */
+  private HostHealth hostHealth = new HostHealth();
+
+  private Integer umask;
+
+  private Boolean iptablesIsRunning;
+
+  public Integer getUmask() {
+    return umask;
+  }
+
+  public void setUmask(Integer umask) {
+    this.umask = umask;
+  }
+
+  public Directory[] getStackFoldersAndFiles() {
+    return stackFoldersAndFiles;
+  }
+
+  public void setStackFoldersAndFiles(Directory[] dirs) {
+    stackFoldersAndFiles = dirs;
+  }
+
+  public void setExistingUsers(ExistingUser[] users) {
+    existingUsers = users;
+  }
+
+  public ExistingUser[] getExistingUsers() {
+    return existingUsers;
+  }
+
+  public void setAlternatives(Alternative[] dirs) {
+    alternatives = dirs;
+  }
+
+  public Alternative[] getAlternatives() {
+    return alternatives;
+  }
+
+  public void setExistingRepos(String[] repos) {
+    existingRepos = repos;
+  }
+
+  public String[] getExistingRepos() {
+    return existingRepos;
+  }
+
+  public void setInstalledPackages(PackageDetail[] packages) {
+    installedPackages = packages;
+  }
+
+  public PackageDetail[] getInstalledPackages() {
+    return installedPackages;
+  }
+
+  public void setHostHealth(HostHealth healthReport) {
+    hostHealth = healthReport;
+  }
+
+  public HostHealth getHostHealth() {
+    return hostHealth;
+  }
+
+  public Boolean getIptablesIsRunning() {
+    return iptablesIsRunning;
+  }
+
+  public void setIptablesIsRunning(Boolean iptablesIsRunning) {
+    this.iptablesIsRunning = iptablesIsRunning;
+  }
+
+  public static class HostHealth {
+    /**
+     * Java processes running on the system.  Default empty array.
+     */
+    @SerializedName("activeJavaProcs")
+    private JavaProc[] activeJavaProcs = new JavaProc[0];
+
+    /**
+     * The current time when agent send the host check report
+     */
+    @SerializedName("agentTimeStampAtReporting")
+    private long agentTimeStampAtReporting = 0;
+
+    /**
+     * The current time when host check report was received
+     */
+    @SerializedName("serverTimeStampAtReporting")
+    private long serverTimeStampAtReporting = 0;
+
+    /**
+     * Live services running on the agent
+     */
+    @SerializedName("liveServices")
+    private LiveService[] liveServices = new LiveService[0];
+
+    public void setAgentTimeStampAtReporting(long currentTime) {
+      agentTimeStampAtReporting = currentTime;
+    }
+
+    public long getAgentTimeStampAtReporting() {
+      return agentTimeStampAtReporting;
+    }
+
+    public void setServerTimeStampAtReporting(long currentTime) {
+      serverTimeStampAtReporting = currentTime;
+    }
+
+    public long getServerTimeStampAtReporting() {
+      return serverTimeStampAtReporting;
+    }
+
+    public void setActiveJavaProcs(JavaProc[] procs) {
+      activeJavaProcs = procs;
+    }
+
+    public JavaProc[] getActiveJavaProcs() {
+      return activeJavaProcs;
+    }
+
+    public void setLiveServices(LiveService[] services) {
+      liveServices = services;
+    }
+
+    public LiveService[] getLiveServices() {
+      return liveServices;
+    }
+  }
+
+  public static class PackageDetail {
+    @SerializedName("name")
+    private String pkgName;
+    @SerializedName("version")
+    private String pkgVersion;
+    @SerializedName("repoName")
+    private String pkgRepoName;
+
+    public void setName(String name) {
+      pkgName = name;
+    }
+
+    public String getName() {
+      return pkgName;
+    }
+
+    public void setVersion(String version) {
+      pkgVersion = version;
+    }
+
+    public String getVersion() {
+      return pkgVersion;
+    }
+
+    public void setRepoName(String repoName) {
+      pkgRepoName = repoName;
+    }
+
+    public String getRepoName() {
+      return pkgRepoName;
+    }
+  }
+
+  /**
+   * Represents information about a directory of interest.
+   */
+  public static class Directory {
+    @SerializedName("name")
+    private String dirName;
+    @SerializedName("type")
+    private String dirType;
+
+    public void setName(String name) {
+      dirName = name;
+    }
+
+    public String getName() {
+      return dirName;
+    }
+
+    public void setType(String type) {
+      dirType = type;
+    }
+
+    public String getType() {
+      return dirType;
+    }
+  }
+
+  /**
+   * Represents information about running java processes.
+   */
+  public static class JavaProc {
+    @SerializedName("user")
+    private String user;
+    @SerializedName("pid")
+    private int pid = 0;
+    @SerializedName("hadoop")
+    private boolean is_hadoop = false;
+    @SerializedName("command")
+    private String command;
+
+    public void setUser(String user) {
+      this.user = user;
+    }
+
+    public String getUser() {
+      return user;
+    }
+
+    public void setPid(int pid) {
+      this.pid = pid;
+    }
+
+    public int getPid() {
+      return pid;
+    }
+
+    public void setHadoop(boolean hadoop) {
+      is_hadoop = hadoop;
+    }
+
+    public boolean isHadoop() {
+      return is_hadoop;
+    }
+
+    public void setCommand(String cmd) {
+      command = cmd;
+    }
+
+    public String getCommand() {
+      return command;
+    }
+  }
+
+  public static class Alternative {
+    @SerializedName("name")
+    private String altName;
+    @SerializedName("target")
+    private String altTarget;
+
+    public void setName(String name) {
+      altName = name;
+    }
+
+    public String getName() {
+      return altName;
+    }
+
+    public void setTarget(String target) {
+      altTarget = target;
+    }
+
+    public String getTarget() {
+      return altTarget;
+    }
+  }
+
+  public static class LiveService {
+    @SerializedName("name")
+    private String svcName;
+    @SerializedName("status")
+    private String svcStatus;
+    @SerializedName("desc")
+    private String svcDesc;
+
+    public void setName(String name) {
+      svcName = name;
+    }
+
+    public String getName() {
+      return svcName;
+    }
+
+    public void setStatus(String status) {
+      svcStatus = status;
+    }
+
+    public String getStatus() {
+      return svcStatus;
+    }
+
+    public void setDesc(String desc) {
+      svcDesc = desc;
+    }
+
+    public String getDesc() {
+      return svcDesc;
+    }
+  }
+
+  public static class ExistingUser {
+    @SerializedName("name")
+    private String name;
+    @SerializedName("homeDir")
+    private String homeDir;
+    @SerializedName("status")
+    private String status;
+
+    public void setUserName(String userName) {
+      name = userName;
+    }
+
+    public String getUserName() {
+      return name;
+    }
+
+    public void setUserHomeDir(String userHomeDir) {
+      homeDir = userHomeDir;
+    }
+
+    public String getUserHomeDir() {
+      return homeDir;
+    }
+
+    public void setUserStatus(String userStatus) {
+      status = userStatus;
+    }
+
+    public String getUserStatus() {
+      return status;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentResource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentResource.java
new file mode 100644
index 0000000..f1e105a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentResource.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.agent;
+
+import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.apache.slider.server.services.security.SignCertResponse;
+import org.apache.slider.server.services.security.SignMessage;
+import org.apache.slider.server.appmaster.web.rest.AbstractSliderResource;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+
+/**
+ *
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class AgentResource extends AbstractSliderResource {
+
+  private String agent_name;
+
+  public AgentResource(WebAppApi slider) {
+    super(slider);
+  }
+
+  private void init(HttpServletResponse res) {
+    res.setContentType(null);
+  }
+
+  @GET
+  @Path("/agent/register")
+  public Response endpointAgentRegister() {
+    Response response = Response.status(200).entity("/agent/register").build();
+    return response;
+  }
+
+  @GET
+  @Path("/agent")
+  public Response endpointAgent() {
+    Response response = Response.status(200).entity("/agent").build();
+    return response;
+  }
+  @GET
+  @Path("/")
+  public Response endpointRoot() {
+    Response response = Response.status(200).entity("/").build();
+    return response;
+  }
+
+  @POST
+  @Path("/{agent_name: [a-zA-Z][a-zA-Z_0-9]*}/register")
+  @Consumes({MediaType.APPLICATION_JSON})
+  @Produces({MediaType.APPLICATION_JSON})
+  public RegistrationResponse register(Register registration,
+                                       @Context HttpServletResponse res,
+                                       @PathParam("agent_name") String agent_name) {
+    init(res);
+    this.agent_name = agent_name;
+    AgentRestOperations ops = slider.getAgentRestOperations();
+    return ops.handleRegistration(registration);
+
+  }
+
+  @POST
+  @Path("/{agent_name: [a-zA-Z][a-zA-Z_0-9]*}/heartbeat")
+  @Consumes(MediaType.APPLICATION_JSON)
+  @Produces({MediaType.APPLICATION_JSON})
+  public HeartBeatResponse heartbeat(HeartBeat message,
+                                     @Context HttpServletResponse res,
+                                     @PathParam("agent_name") String agent_name) {
+    init(res);
+    AgentRestOperations ops = slider.getAgentRestOperations();
+    return ops.handleHeartBeat(message);
+  }
+
+  @GET
+  @Path("/cert/ca")
+  @Produces({MediaType.TEXT_PLAIN})
+  public String downloadSrvrCrt() {
+    return slider.getCertificateManager().getServerCert();
+  }
+
+  @Path("/certs/{hostName}")
+  @POST
+  @Consumes(MediaType.APPLICATION_JSON)
+  @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+  public SignCertResponse signAgentCrt(@PathParam("hostName") String hostname,
+                                       SignMessage message, @Context HttpServletRequest req) {
+    return slider.getCertificateManager().signAgentCrt(hostname,
+                                                       message.getCsr(),
+                                                       message.getPassphrase());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentRestOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentRestOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentRestOperations.java
new file mode 100644
index 0000000..2891be8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentRestOperations.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.agent;
+
+/**
+ *
+ */
+public interface AgentRestOperations {
+
+  RegistrationResponse handleRegistration(Register registration);
+
+  HeartBeatResponse handleHeartBeat(HeartBeat heartBeat);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebApp.java
new file mode 100644
index 0000000..3a3b0c0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebApp.java
@@ -0,0 +1,258 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.agent;
+
+import com.google.common.base.Preconditions;
+import com.sun.jersey.api.core.ResourceConfig;
+import com.sun.jersey.spi.container.WebApplication;
+import com.sun.jersey.spi.container.servlet.ServletContainer;
+import com.sun.jersey.spi.container.servlet.WebConfig;
+import com.sun.jersey.spi.inject.SingletonTypeInjectableProvider;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.providers.agent.AgentKeys;
+import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.apache.slider.server.services.security.SecurityUtils;
+import org.mortbay.jetty.Connector;
+import org.mortbay.jetty.Server;
+import org.mortbay.jetty.security.SslSelectChannelConnector;
+import org.mortbay.jetty.servlet.Context;
+import org.mortbay.jetty.servlet.ServletHolder;
+import org.mortbay.thread.QueuedThreadPool;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.ws.rs.ext.Provider;
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.net.BindException;
+import java.util.Set;
+
+/**
+ *
+ */
+public class AgentWebApp implements Closeable {
+  protected static final Logger LOG = LoggerFactory.getLogger(AgentWebApp.class);
+  private int port;
+  private int securedPort;
+  private static Server agentServer;
+  public static final String BASE_PATH = "slideragent";
+
+  public static class Builder {
+    final String name;
+    final String wsName;
+    final WebAppApi application;
+    int port;
+    int securedPort;
+    MapOperations configsMap;
+
+    public Builder(String name, String wsName, WebAppApi application) {
+      this.name = name;
+      this.wsName = wsName;
+      this.application = application;
+    }
+
+    public Builder withComponentConfig(MapOperations appMasterConfig) {
+      this.configsMap = appMasterConfig;
+      return this;
+    }
+
+    public Builder withPort (int port) {
+      this.port = port;
+      return this;
+    }
+
+    public Builder withSecuredPort (int securedPort) {
+      this.securedPort = securedPort;
+      return this;
+    }
+
+    public AgentWebApp start() throws IOException {
+      if (configsMap == null) {
+        throw new IllegalStateException("No SSL Configuration Available");
+      }
+
+      agentServer = new Server();
+      agentServer.setThreadPool(
+          new QueuedThreadPool(
+              configsMap.getOptionInt("agent.threadpool.size.max", 25)));
+      agentServer.setStopAtShutdown(true);
+      agentServer.setGracefulShutdown(1000);
+
+      SslSelectChannelConnector ssl1WayConnector = createSSLConnector(false, port);
+      SslSelectChannelConnector ssl2WayConnector =
+          createSSLConnector(Boolean.valueOf(
+              configsMap.getOption(AgentKeys.KEY_AGENT_TWO_WAY_SSL_ENABLED,
+                                   "false")), securedPort);
+      agentServer.setConnectors(new Connector[]{ssl1WayConnector,
+          ssl2WayConnector});
+
+      ServletHolder agent = new ServletHolder(new AgentServletContainer());
+      Context agentRoot = new Context(agentServer, "/", Context.SESSIONS);
+
+      agent.setInitParameter("com.sun.jersey.config.property.resourceConfigClass",
+                             "com.sun.jersey.api.core.PackagesResourceConfig");
+      agent.setInitParameter("com.sun.jersey.config.property.packages",
+                             "org.apache.slider.server.appmaster.web.rest.agent");
+      agent.setInitParameter("com.sun.jersey.api.json.POJOMappingFeature",
+                             "true");
+//      agent.setInitParameter("com.sun.jersey.spi.container.ContainerRequestFilters", "com.sun.jersey.api.container.filter.LoggingFilter");
+//      agent.setInitParameter("com.sun.jersey.spi.container.ContainerResponseFilters", "com.sun.jersey.api.container.filter.LoggingFilter");
+//      agent.setInitParameter("com.sun.jersey.config.feature.Trace", "true");
+      agentRoot.addServlet(agent, "/*");
+
+      try {
+        openListeners();
+        agentServer.start();
+      } catch (IOException e) {
+        LOG.error("Unable to start agent server", e);
+        throw e;
+      } catch (Exception e) {
+        LOG.error("Unable to start agent server", e);
+        throw new IOException("Unable to start agent server: " + e, e);
+      }
+
+      AgentWebApp webApp = new AgentWebApp();
+      webApp.setPort(getConnectorPort(agentServer, 0));
+      webApp.setSecuredPort(getConnectorPort(agentServer, 1));
+      return webApp;
+
+    }
+
+    private void openListeners() throws Exception {
+      // from HttpServer2.openListeners()
+      for (Connector listener : agentServer.getConnectors()) {
+        if (listener.getLocalPort() != -1) {
+          // This listener is either started externally or has been bound
+          continue;
+        }
+        int port = listener.getPort();
+        while (true) {
+          // jetty has a bug where you can't reopen a listener that previously
+          // failed to open w/o issuing a close first, even if the port is changed
+          try {
+            listener.close();
+            listener.open();
+            LOG.info("Jetty bound to port " + listener.getLocalPort());
+            break;
+          } catch (BindException ex) {
+            if (port == 0) {
+              BindException be = new BindException("Port in use: "
+                  + listener.getHost() + ":" + listener.getPort());
+              be.initCause(ex);
+              throw be;
+            }
+          }
+          // try the next port number
+          listener.setPort(++port);
+          Thread.sleep(100);
+        }
+      }
+    }
+
+    private SslSelectChannelConnector createSSLConnector(boolean needClientAuth, int port) {
+      SslSelectChannelConnector sslConnector = new
+          SslSelectChannelConnector();
+
+      String keystore = SecurityUtils.getSecurityDir() +
+                        File.separator + "keystore.p12";
+      String srvrCrtPass = SecurityUtils.getKeystorePass();
+      sslConnector.setKeystore(keystore);
+      sslConnector.setTruststore(keystore);
+      sslConnector.setPassword(srvrCrtPass);
+      sslConnector.setKeyPassword(srvrCrtPass);
+      sslConnector.setTrustPassword(srvrCrtPass);
+      sslConnector.setKeystoreType("PKCS12");
+      sslConnector.setTruststoreType("PKCS12");
+      sslConnector.setNeedClientAuth(needClientAuth);
+
+      sslConnector.setPort(port);
+      sslConnector.setAcceptors(2);
+      return sslConnector;
+    }
+
+    @Provider
+    public class WebAppApiProvider extends
+        SingletonTypeInjectableProvider<javax.ws.rs.core.Context, WebAppApi> {
+
+      public WebAppApiProvider () {
+        super(WebAppApi.class, application);
+      }
+    }
+
+    public class AgentServletContainer extends ServletContainer {
+      public AgentServletContainer() {
+        super();
+      }
+
+      @Override
+      protected void configure(WebConfig wc,
+                               ResourceConfig rc,
+                               WebApplication wa) {
+        super.configure(wc, rc, wa);
+        Set<Object> singletons = rc.getSingletons();
+        singletons.add(new WebAppApiProvider());
+      }
+    }
+
+    private int getConnectorPort(Server webServer, int index) {
+      Preconditions.checkArgument(index >= 0);
+      if (index > webServer.getConnectors().length)
+        throw new IllegalStateException("Illegal connect index requested");
+
+      Connector c = webServer.getConnectors()[index];
+      if (c.getLocalPort() == -1) {
+        // The connector is not bounded
+        throw new IllegalStateException("The connector is not bound to a port");
+      }
+
+      return c.getLocalPort();
+    }
+  }
+
+  public static Builder $for(String name, WebAppApi app, String wsPrefix) {
+    return new Builder(name, wsPrefix, app);
+  }
+
+  public int getPort() {
+    return port;
+  }
+
+  public void setPort(int port) {
+    this.port = port;
+  }
+
+  public void setSecuredPort(int securedPort) {
+    this.securedPort = securedPort;
+  }
+
+  public int getSecuredPort() {
+    return securedPort;
+  }
+
+  public void close() throws IOException{
+    //need to stop server and reset injector
+    try {
+      agentServer.stop();
+    } catch (IOException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new IOException(e.toString(), e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebServices.java
new file mode 100644
index 0000000..684ce6f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebServices.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.agent;
+
+import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.apache.slider.server.appmaster.web.rest.RestPaths;
+
+import javax.ws.rs.Path;
+import javax.ws.rs.core.Context;
+
+/** The available agent REST services exposed by a slider AM. */
+@Path(RestPaths.SLIDER_AGENT_CONTEXT_ROOT)
+public class AgentWebServices {
+  /** AM/WebApp info object */
+  @Context
+  private WebAppApi slider;
+
+  public AgentWebServices() {
+  }
+
+  @Path(RestPaths.SLIDER_SUBPATH_AGENTS)
+  public AgentResource getAgentResource () {
+    return new AgentResource(slider);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/CommandReport.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/CommandReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/CommandReport.java
new file mode 100644
index 0000000..a37e490
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/CommandReport.java
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.agent;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.annotate.JsonProperty;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.Map;
+
+/**
+ *
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class CommandReport {
+
+  int exitCode;
+  private String role;
+  private String actionId;
+  private String stdout;
+  private String stderr;
+  private String structuredOut;
+  private String status;
+  private String clusterName;
+  private String serviceName;
+  private long taskId;
+  private String roleCommand;
+  private Map<String, String> folders;
+  private Map<String, String> allocatedPorts;
+  private Map<String, Map<String, String>> configurationTags;
+
+  @JsonProperty("taskId")
+  public long getTaskId() {
+    return taskId;
+  }
+
+  @JsonProperty("taskId")
+  public void setTaskId(long taskId) {
+    this.taskId = taskId;
+  }
+
+  @JsonProperty("clusterName")
+  public String getClusterName() {
+    return this.clusterName;
+  }
+
+  @JsonProperty("clusterName")
+  public void setClusterName(String clusterName) {
+    this.clusterName = clusterName;
+  }
+
+  @JsonProperty("actionId")
+  public String getActionId() {
+    return this.actionId;
+  }
+
+  @JsonProperty("actionId")
+  public void setActionId(String actionId) {
+    this.actionId = actionId;
+  }
+
+  @JsonProperty("stderr")
+  public String getStdErr() {
+    return this.stderr;
+  }
+
+  @JsonProperty("stderr")
+  public void setStdErr(String stderr) {
+    this.stderr = stderr;
+  }
+
+  @JsonProperty("exitcode")
+  public int getExitCode() {
+    return this.exitCode;
+  }
+
+  @JsonProperty("exitcode")
+  public void setExitCode(int exitCode) {
+    this.exitCode = exitCode;
+  }
+
+  @JsonProperty("stdout")
+  public String getStdOut() {
+    return this.stdout;
+  }
+
+  @JsonProperty("stdout")
+  public void setStdOut(String stdout) {
+    this.stdout = stdout;
+  }
+
+  @JsonProperty("structuredOut")
+  public String getStructuredOut() {
+    return this.structuredOut;
+  }
+
+  @JsonProperty("structuredOut")
+  public void setStructuredOut(String structuredOut) {
+    this.structuredOut = structuredOut;
+  }
+
+  @JsonProperty("roleCommand")
+  public String getRoleCommand() {
+    return this.roleCommand;
+  }
+
+  @JsonProperty("roleCommand")
+  public void setRoleCommand(String roleCommand) {
+    this.roleCommand = roleCommand;
+  }
+
+  @JsonProperty("role")
+  public String getRole() {
+    return role;
+  }
+
+  @JsonProperty("role")
+  public void setRole(String role) {
+    this.role = role;
+  }
+
+  @JsonProperty("status")
+  public String getStatus() {
+    return status;
+  }
+
+  @JsonProperty("status")
+  public void setStatus(String status) {
+    this.status = status;
+  }
+
+  @JsonProperty("serviceName")
+  public String getServiceName() {
+    return serviceName;
+  }
+
+  @JsonProperty("serviceName")
+  public void setServiceName(String serviceName) {
+    this.serviceName = serviceName;
+  }
+
+  /** @return the config tags that match this command, or <code>null</code> if none are present */
+  @JsonProperty("configurationTags")
+  public Map<String, Map<String, String>> getConfigurationTags() {
+    return configurationTags;
+  }
+
+  /** @param tags the config tags that match this command */
+  @JsonProperty("configurationTags")
+  public void setConfigurationTags(Map<String, Map<String, String>> tags) {
+    configurationTags = tags;
+  }
+
+  /** @return the allocated ports, or <code>null</code> if none are present */
+  @JsonProperty("allocatedPorts")
+  public Map<String, String> getAllocatedPorts() {
+    return allocatedPorts;
+  }
+
+  /** @param ports allocated ports */
+  @JsonProperty("allocatedPorts")
+  public void setAllocatedPorts(Map<String, String> ports) {
+    this.allocatedPorts = ports;
+  }
+
+  /** @return the folders, or <code>null</code> if none are present */
+  @JsonProperty("folders")
+  public Map<String, String> getFolders() {
+    return folders;
+  }
+
+  /** @param folders allocated ports */
+  @JsonProperty("folders")
+  public void setFolders(Map<String, String> folders) {
+    this.folders = folders;
+  }
+
+  @Override
+  public String toString() {
+    return "CommandReport{" +
+           "role='" + role + '\'' +
+           ", actionId='" + actionId + '\'' +
+           ", status='" + status + '\'' +
+           ", exitCode=" + exitCode +
+           ", clusterName='" + clusterName + '\'' +
+           ", serviceName='" + serviceName + '\'' +
+           ", taskId=" + taskId +
+           ", roleCommand=" + roleCommand +
+           ", configurationTags=" + configurationTags +
+           '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/ComponentStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/ComponentStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/ComponentStatus.java
new file mode 100644
index 0000000..acdc234
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/ComponentStatus.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.agent;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.annotate.JsonProperty;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.Map;
+
+/**
+ *
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class ComponentStatus {
+  String componentName;
+  String msg;
+  String status;
+  String serviceName;
+  String clusterName;
+  String roleCommand;
+  String ip;
+  String hostname;
+  @JsonProperty("configurations")
+  private Map<String, Map<String, String>> configurations;
+
+  public String getRoleCommand() {
+    return roleCommand;
+  }
+
+  public void setRoleCommand(String roleCommand) {
+    this.roleCommand = roleCommand;
+  }
+
+  public String getComponentName() {
+    return this.componentName;
+  }
+
+  public void setComponentName(String componentName) {
+    this.componentName = componentName;
+  }
+
+  public String getMessage() {
+    return this.msg;
+  }
+
+  public void setMessage(String msg) {
+    this.msg = msg;
+  }
+
+  public String getStatus() {
+    return this.status;
+  }
+
+  public void setStatus(String status) {
+    this.status = status;
+  }
+
+  public String getServiceName() {
+    return serviceName;
+  }
+
+  public void setServiceName(String serviceName) {
+    this.serviceName = serviceName;
+  }
+
+  public String getClusterName() {
+    return clusterName;
+  }
+
+  public void setClusterName(String clusterName) {
+    this.clusterName = clusterName;
+  }
+
+  /** @return the config tags that match this command, or <code>null</code> if none are present */
+  public Map<String, Map<String, String>> getConfigs() {
+    return configurations;
+  }
+
+  /** @param configs the config tags that match this status */
+  public void setConfigs(Map<String, Map<String, String>> configs) {
+    this.configurations = configs;
+  }
+
+  @Override
+  public String toString() {
+    return "ComponentStatus{" +
+           "componentName='" + componentName + '\'' +
+           ", msg='" + msg + '\'' +
+           ", status='" + status + '\'' +
+           ", serviceName='" + serviceName + '\'' +
+           ", clusterName='" + clusterName + '\'' +
+           ", roleCommand='" + roleCommand + '\'' +
+           ", ip='" + ip + '\'' +
+           ", hostname='" + hostname + '\'' +
+           '}';
+  }
+
+  public String getIp() {
+    return ip;
+  }
+
+  public void setIp(String ip) {
+    this.ip = ip;
+  }
+
+  public String getHostname() {
+    return hostname;
+  }
+
+  public void setHostname(String hostname) {
+    this.hostname = hostname;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/DiskInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/DiskInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/DiskInfo.java
new file mode 100644
index 0000000..27c4d54
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/DiskInfo.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.agent;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.annotate.JsonProperty;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+/**
+ *
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class DiskInfo {
+  String available;
+  String mountpoint;
+  String device;
+  String used;
+  String percent;
+  String size;
+  String type;
+
+  /**
+   * DiskInfo object that tracks information about a disk.
+   * @param mountpoint
+   * @param available
+   * @param used
+   * @param percent
+   * @param size
+   */
+  public DiskInfo(String device, String mountpoint, String available,
+                  String used, String percent, String size, String type) {
+    this.device = device;
+    this.mountpoint = mountpoint;
+    this.available = available;
+    this.used = used;
+    this.percent = percent;
+    this.size = size;
+    this.type = type;
+  }
+
+  /**
+   * Needed for Serialization
+   */
+  public DiskInfo() {}
+
+  @JsonProperty("available")
+  public void setAvailable(String available) {
+    this.available = available;
+  }
+
+  @JsonProperty("available")
+  public String getAvailable() {
+    return this.available;
+  }
+
+  @JsonProperty("mountpoint")
+  public String getMountPoint() {
+    return this.mountpoint;
+  }
+
+  @JsonProperty("mountpoint")
+  public void setMountPoint(String mountpoint) {
+    this.mountpoint = mountpoint;
+  }
+
+  @JsonProperty("type")
+  public String getType() {
+    return this.type;
+  }
+
+  @JsonProperty("type")
+  public void setType(String type) {
+    this.type = type;
+  }
+
+  @JsonProperty("used")
+  public String getUsed() {
+    return this.used;
+  }
+
+  @JsonProperty("used")
+  public void setUsed(String used) {
+    this.used = used;
+  }
+
+  @JsonProperty("percent")
+  public String getPercent() {
+    return this.percent;
+  }
+
+  @JsonProperty("percent")
+  public void setPercent(String percent) {
+    this.percent = percent;
+  }
+
+  @JsonProperty("size")
+  public String getSize() {
+    return this.size;
+  }
+
+  @JsonProperty("size")
+  public void setSize(String size) {
+    this.size = size;
+  }
+
+  @Override
+  public String toString() {
+    return "available=" + this.available + ",mountpoint=" + this.mountpoint
+           + ",used=" + this.used + ",percent=" + this.percent + ",size=" +
+           this.size + ",device=" + this.device +
+           ",type=" + this.type;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/ExecutionCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/ExecutionCommand.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/ExecutionCommand.java
new file mode 100644
index 0000000..d3864b8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/ExecutionCommand.java
@@ -0,0 +1,310 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.agent;
+
+import org.apache.slider.providers.agent.application.metadata.Component;
+import org.apache.slider.providers.agent.application.metadata.DockerContainer;
+import org.apache.slider.providers.agent.application.metadata.DockerContainerInputFile;
+import org.apache.slider.providers.agent.application.metadata.DockerContainerMount;
+import org.apache.slider.providers.agent.application.metadata.DockerContainerPort;
+import org.apache.slider.providers.agent.application.metadata.Metainfo;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.annotate.JsonProperty;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ *
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class ExecutionCommand {
+  protected static final Logger log =
+      LoggerFactory.getLogger(ExecutionCommand.class);
+  private AgentCommandType commandType = AgentCommandType.EXECUTION_COMMAND;
+  private String clusterName;
+  private long taskId;
+  private String commandId;
+  //TODO Remove hostname from being set in the command
+  private String hostname;
+  private String role;
+  private Map<String, String> hostLevelParams = new HashMap<String, String>();
+  private Map<String, String> roleParams = null;
+  private String roleCommand;
+  private Map<String, Map<String, String>> configurations;
+  private Map<String, Map<String, String>> componentConfigurations;
+  private Map<String, String> commandParams;
+  private String serviceName;
+  private String componentName;
+  private String componentType;
+  private List<DockerContainer> containers = new ArrayList<>();
+  private String pkg;
+  private boolean yarnDockerMode = false;
+
+  public ExecutionCommand(AgentCommandType commandType) {
+    this.commandType = commandType;
+  }
+
+  @JsonProperty("commandType")
+  public AgentCommandType getCommandType() {
+    return commandType;
+  }
+
+  @JsonProperty("commandType")
+  public void setCommandType(AgentCommandType commandType) {
+    this.commandType = commandType;
+  }
+
+  @JsonProperty("commandId")
+  public String getCommandId() {
+    return this.commandId;
+  }
+
+  @JsonProperty("commandId")
+  public void setCommandId(String commandId) {
+    this.commandId = commandId;
+  }
+
+  @JsonProperty("taskId")
+  public long getTaskId() {
+    return taskId;
+  }
+
+  @JsonProperty("taskId")
+  public void setTaskId(long taskId) {
+    this.taskId = taskId;
+  }
+
+  @JsonProperty("role")
+  public String getRole() {
+    return role;
+  }
+
+  @JsonProperty("role")
+  public void setRole(String role) {
+    this.role = role;
+  }
+
+  @JsonProperty("roleParams")
+  public Map<String, String> getRoleParams() {
+    return roleParams;
+  }
+
+  @JsonProperty("roleParams")
+  public void setRoleParams(Map<String, String> roleParams) {
+    this.roleParams = roleParams;
+  }
+
+  @JsonProperty("roleCommand")
+  public String getRoleCommand() {
+    return roleCommand;
+  }
+
+  @JsonProperty("roleCommand")
+  public void setRoleCommand(String cmd) {
+    this.roleCommand = cmd;
+  }
+
+  @JsonProperty("clusterName")
+  public String getClusterName() {
+    return clusterName;
+  }
+
+  @JsonProperty("clusterName")
+  public void setClusterName(String clusterName) {
+    this.clusterName = clusterName;
+  }
+
+  @JsonProperty("componentType")
+  public String getComponentType() {
+    return componentType;
+  }
+
+  @JsonProperty("componentType")
+  public void setComponentType(String componentType) {
+    this.componentType = componentType;
+  }
+
+  @JsonProperty("hostname")
+  public String getHostname() {
+    return hostname;
+  }
+
+  @JsonProperty("hostname")
+  public void setHostname(String hostname) {
+    this.hostname = hostname;
+  }
+
+  @JsonProperty("hostLevelParams")
+  public Map<String, String> getHostLevelParams() {
+    return hostLevelParams;
+  }
+
+  @JsonProperty("hostLevelParams")
+  public void setHostLevelParams(Map<String, String> params) {
+    this.hostLevelParams = params;
+  }
+
+  @JsonProperty("configurations")
+  public Map<String, Map<String, String>> getConfigurations() {
+    return configurations;
+  }
+
+  @JsonProperty("configurations")
+  public void setConfigurations(Map<String, Map<String, String>> configurations) {
+    this.configurations = configurations;
+  }
+
+  @JsonProperty("commandParams")
+  public Map<String, String> getCommandParams() {
+    return commandParams;
+  }
+
+  @JsonProperty("commandParams")
+  public void setCommandParams(Map<String, String> commandParams) {
+    this.commandParams = commandParams;
+  }
+
+  @JsonProperty("serviceName")
+  public String getServiceName() {
+    return serviceName;
+  }
+
+  @JsonProperty("serviceName")
+  public void setServiceName(String serviceName) {
+    this.serviceName = serviceName;
+  }
+
+  @JsonProperty("componentName")
+  public String getComponentName() {
+    return componentName;
+  }
+
+  @JsonProperty("componentName")
+  public void setComponentName(String componentName) {
+    this.componentName = componentName;
+  }
+
+  @JsonProperty("package")
+  public String getPkg() {
+    return pkg;
+  }
+
+  @JsonProperty("package")
+  public void setPkg(String pkg) {
+    this.pkg = pkg;
+  }
+
+  @JsonProperty("componentConfig")
+  public Map<String, Map<String, String>> getComponentConfigurations() {
+    return componentConfigurations;
+  }
+
+  @JsonProperty("componentConfig")
+  public void setComponentConfigurations(
+      Map<String, Map<String, String>> componentConfigurations) {
+    this.componentConfigurations = componentConfigurations;
+  }
+
+  @JsonProperty("containers")
+  public List<DockerContainer> getContainers() {
+    return containers;
+  }
+
+  @JsonProperty("yarnDockerMode")
+  public boolean isYarnDockerMode() {
+    return yarnDockerMode ;
+  }
+
+  @JsonProperty("yarnDockerMode")
+  public void setYarnDockerMode(boolean yarnDockerMode) {
+    this.yarnDockerMode = yarnDockerMode;
+  }
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append("ExecutionCommand [commandType=").append(commandType)
+        .append(", clusterName=").append(clusterName).append(", taskId=")
+        .append(taskId).append(", commandId=").append(commandId)
+        .append(", hostname=").append(hostname).append(", role=").append(role)
+        .append(", hostLevelParams=").append(hostLevelParams)
+        .append(", roleParams=").append(roleParams).append(", roleCommand=")
+        .append(roleCommand).append(", configurations=").append(configurations)
+        .append(", commandParams=").append(commandParams)
+        .append(", serviceName=").append(serviceName)
+        .append(", componentName=").append(componentName)
+        .append(", componentType=").append(componentType)
+        .append(", yarnDockerMode=").append(yarnDockerMode).append(", pkg=")
+        .append(pkg).append("]");
+    return builder.toString();
+  }
+  
+  public void addContainerDetails(String componentGroup, Metainfo metaInfo) {
+    Component component = metaInfo.getApplicationComponent(componentGroup);
+    this.setComponentType(component.getType());
+    log.info("Adding container details for {}", componentGroup, " from ",
+        metaInfo.toString());
+    for (DockerContainer metaContainer : component.getDockerContainers()) {
+      DockerContainer container = new DockerContainer();
+      container.setImage(metaContainer.getImage());
+      container.setNetwork(metaContainer.getNetwork());
+      container.setUseNetworkScript(metaContainer.getUseNetworkScript());
+      container.setName(metaContainer.getName());
+      container.setOptions(metaContainer.getOptions());
+      container.setAdditionalParam(metaContainer.getAdditionalParam());
+      container.setCommandPath(metaContainer.getAdditionalParam());
+      container.setStatusCommand(metaContainer.getStatusCommand());
+      container.setStartCommand(metaContainer.getStartCommand());
+      if (metaContainer.getMounts().size() > 0) {
+        for (DockerContainerMount metaContMount : metaContainer.getMounts()) {
+          DockerContainerMount contMnt = new DockerContainerMount();
+          contMnt.setContainerMount(metaContMount.getContainerMount());
+          contMnt.setHostMount(metaContMount.getHostMount());
+          container.getMounts().add(contMnt);
+        }
+      }
+      if (metaContainer.getPorts().size() > 0) {
+        for (DockerContainerPort metaCntPort : metaContainer.getPorts()) {
+          DockerContainerPort cntPort = new DockerContainerPort();
+          cntPort.setContainerPort(metaCntPort.getContainerPort());
+          cntPort.setHostPort(metaCntPort.getHostPort());
+          container.getPorts().add(cntPort);
+        }
+      }
+      if (metaContainer.getInputFiles().size() > 0) {
+        for (DockerContainerInputFile metaInpFile : metaContainer
+            .getInputFiles()) {
+          DockerContainerInputFile inpFile = new DockerContainerInputFile();
+          inpFile.setContainerMount(metaInpFile.getContainerMount());
+          inpFile.setFileLocalPath(metaInpFile.getFileLocalPath());
+          container.getInputFiles().add(inpFile);
+        }
+      }
+      if (metaContainer.getConfigFiles() != null) {
+        container.setConfigFiles(metaContainer.getConfigFiles());
+      }
+      log.info("Docker container meta info ready: " + container.toString());
+      this.getContainers().add(container);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HeartBeat.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HeartBeat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HeartBeat.java
new file mode 100644
index 0000000..d17c465
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HeartBeat.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.web.rest.agent;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.annotate.JsonProperty;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ *
+ *
+ * Data model for agent heartbeat for server (ambari or app master).
+ *
+ */
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class HeartBeat {
+  private long responseId = -1;
+  private long timestamp;
+  private String hostname;
+  List<CommandReport> reports = new ArrayList<CommandReport>();
+  List<ComponentStatus> componentStatus = new ArrayList<ComponentStatus>();
+  private List<DiskInfo> mounts = new ArrayList<DiskInfo>();
+  HostStatus nodeStatus;
+  private AgentEnv agentEnv = null;
+  private String fqdn;
+  private String pkg;
+
+  public long getResponseId() {
+    return responseId;
+  }
+
+  public void setResponseId(long responseId) {
+    this.responseId=responseId;
+  }
+
+  public long getTimestamp() {
+    return timestamp;
+  }
+
+  public void setTimestamp(long timestamp) {
+    this.timestamp = timestamp;
+  }
+
+  public String getHostname() {
+    return hostname;
+  }
+
+  public void setHostname(String hostname) {
+    this.hostname = hostname;
+  }
+
+  public String getFqdn() {
+    return fqdn;
+  }
+
+  public void setFqdn(String fqdn) {
+    this.fqdn = fqdn;
+  }
+
+  @JsonProperty("reports")
+  public List<CommandReport> getReports() {
+    return this.reports;
+  }
+
+  @JsonProperty("reports")
+  public void setReports(List<CommandReport> reports) {
+    this.reports = reports;
+  }
+
+  public HostStatus getNodeStatus() {
+    return nodeStatus;
+  }
+
+  public void setNodeStatus(HostStatus nodeStatus) {
+    this.nodeStatus = nodeStatus;
+  }
+
+  public AgentEnv getAgentEnv() {
+    return agentEnv;
+  }
+
+  public void setAgentEnv(AgentEnv env) {
+    agentEnv = env;
+  }
+
+  @JsonProperty("componentStatus")
+  public List<ComponentStatus> getComponentStatus() {
+    return componentStatus;
+  }
+
+  @JsonProperty("componentStatus")
+  public void setComponentStatus(List<ComponentStatus> componentStatus) {
+    this.componentStatus = componentStatus;
+  }
+
+  @JsonProperty("mounts")
+  public List<DiskInfo> getMounts() {
+    return this.mounts;
+  }
+
+  @JsonProperty("mounts")
+  public void setMounts(List<DiskInfo> mounts) {
+    this.mounts = mounts;
+  }
+
+  @JsonProperty("package")
+  public String getPackage() {
+    return pkg;
+  }
+
+  @JsonProperty("package")
+  public void setPackage(String pkg) {
+    this.pkg = pkg;
+  }
+
+  @Override
+  public String toString() {
+    return "HeartBeat{" +
+           "responseId=" + responseId +
+           ", timestamp=" + timestamp +
+           ", hostname='" + hostname + '\'' +
+           ", reports=" + reports +
+           ", componentStatus=" + componentStatus +
+           ", package=" + pkg +
+           ", nodeStatus=" + nodeStatus +
+           '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HeartBeatResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HeartBeatResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HeartBeatResponse.java
new file mode 100644
index 0000000..b500d67
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HeartBeatResponse.java
@@ -0,0 +1,147 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.agent;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.annotate.JsonProperty;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ *
+ * Controller to Agent response data model.
+ *
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class HeartBeatResponse {
+
+  private long responseId;
+
+  List<ExecutionCommand> executionCommands = new ArrayList<ExecutionCommand>();
+  List<StatusCommand> statusCommands = new ArrayList<StatusCommand>();
+
+  RegistrationCommand registrationCommand;
+
+  boolean yarnDockerMode = false;
+  boolean restartAgent = false;
+  boolean restartEnabled = true;
+  boolean hasMappedComponents = false;
+  boolean terminateAgent = false;
+
+  @JsonProperty("responseId")
+  public long getResponseId() {
+    return responseId;
+  }
+
+  @JsonProperty("responseId")
+  public void setResponseId(long responseId) {
+    this.responseId=responseId;
+  }
+
+  @JsonProperty("executionCommands")
+  public List<ExecutionCommand> getExecutionCommands() {
+    return executionCommands;
+  }
+
+  @JsonProperty("executionCommands")
+  public void setExecutionCommands(List<ExecutionCommand> executionCommands) {
+    this.executionCommands = executionCommands;
+  }
+
+  @JsonProperty("statusCommands")
+  public List<StatusCommand> getStatusCommands() {
+    return statusCommands;
+  }
+
+  @JsonProperty("statusCommands")
+  public void setStatusCommands(List<StatusCommand> statusCommands) {
+    this.statusCommands = statusCommands;
+  }
+
+  @JsonProperty("registrationCommand")
+  public RegistrationCommand getRegistrationCommand() {
+    return registrationCommand;
+  }
+
+  @JsonProperty("registrationCommand")
+  public void setRegistrationCommand(RegistrationCommand registrationCommand) {
+    this.registrationCommand = registrationCommand;
+  }
+
+  @JsonProperty("restartAgent")
+  public boolean isRestartAgent() {
+    return restartAgent;
+  }
+
+  @JsonProperty("restartAgent")
+  public void setRestartAgent(boolean restartAgent) {
+    this.restartAgent = restartAgent;
+  }
+
+  @JsonProperty("restartEnabled")
+  public boolean getRstartEnabled() {
+    return restartEnabled;
+  }
+
+  @JsonProperty("restartEnabled")
+  public void setRestartEnabled(boolean restartEnabled) {
+    this.restartEnabled = restartEnabled;
+  }
+
+  @JsonProperty("hasMappedComponents")
+  public boolean hasMappedComponents() {
+    return hasMappedComponents;
+  }
+
+  @JsonProperty("hasMappedComponents")
+  public void setHasMappedComponents(boolean hasMappedComponents) {
+    this.hasMappedComponents = hasMappedComponents;
+  }
+
+  @JsonProperty("terminateAgent")
+  public boolean isTerminateAgent() {
+    return terminateAgent;
+  }
+
+  @JsonProperty("terminateAgent")
+  public void setTerminateAgent(boolean terminateAgent) {
+    this.terminateAgent = terminateAgent;
+  }
+
+  public void addExecutionCommand(ExecutionCommand execCmd) {
+    executionCommands.add(execCmd);
+  }
+
+  public void addStatusCommand(StatusCommand statCmd) {
+    statusCommands.add(statCmd);
+  }
+
+  @Override
+  public String toString() {
+    return "HeartBeatResponse{" +
+           "responseId=" + responseId +
+           ", executionCommands=" + executionCommands +
+           ", statusCommands=" + statusCommands +
+           ", registrationCommand=" + registrationCommand +
+           ", restartAgent=" + restartAgent +
+           ", terminateAgent=" + terminateAgent +
+           '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HostInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HostInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HostInfo.java
new file mode 100644
index 0000000..bef7b07
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HostInfo.java
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.agent;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.annotate.JsonProperty;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.ArrayList;
+import java.util.List;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class HostInfo {
+  private String architecture;
+  private String domain;
+  private String fqdn;
+  private String hardwareisa;
+  private String hardwaremodel;
+  private String hostname;
+  private String id;
+  private String interfaces;
+  private String ipaddress;
+  private String kernel;
+  private String kernelmajversion;
+  private String kernelrelease;
+  private String kernelversion;
+  private String macaddress;
+  private long memoryfree;
+  private long memorysize;
+  private List<DiskInfo> mounts = new ArrayList<DiskInfo>();
+  private long memorytotal;
+  private String netmask;
+  private String operatingsystem;
+  private String operatingsystemrelease;
+  private String osfamily;
+  private int physicalprocessorcount;
+  private int processorcount;
+  private boolean selinux;
+  private String swapfree;
+  private String swapsize;
+  private String timezone;
+  private String uptime;
+  private long uptime_days;
+  private long uptime_hours;
+
+
+  @JsonProperty("architecture")
+  public String getArchitecture() {
+    return this.architecture;
+  }
+
+  @JsonProperty("architecture")
+  public void setArchitecture(String architecture) {
+    this.architecture = architecture;
+  }
+
+  @JsonProperty("domain")
+  public String getDomain() {
+    return this.domain;
+  }
+
+  @JsonProperty("domain")
+  public void setDomain(String domain) {
+    this.domain = domain;
+  }
+
+  @JsonProperty("fqdn")
+  public String getFQDN() {
+    return this.fqdn;
+  }
+
+  @JsonProperty("fqdn")
+  public void setFQDN(String fqdn) {
+    this.fqdn = fqdn;
+  }
+
+  @JsonProperty("hardwareisa")
+  public String getHardwareIsa() {
+    return hardwareisa;
+  }
+
+  @JsonProperty("hardwareisa")
+  public void setHardwareIsa(String hardwareisa) {
+    this.hardwareisa = hardwareisa;
+  }
+
+  @JsonProperty("hardwaremodel")
+  public String getHardwareModel() {
+    return this.hardwaremodel;
+  }
+
+  @JsonProperty("hardwaremodel")
+  public void setHardwareModel(String hardwaremodel) {
+    this.hardwaremodel = hardwaremodel;
+  }
+
+  @JsonProperty("hostname")
+  public String getHostName() {
+    return this.hostname;
+  }
+
+  @JsonProperty("hostname")
+  public void setHostName(String hostname) {
+    this.hostname = hostname;
+  }
+
+  @JsonProperty("id")
+  public String getAgentUserId() {
+    return id;
+  }
+
+  @JsonProperty("id")
+  public void setAgentUserId(String id) {
+    this.id = id;
+  }
+
+  @JsonProperty("interfaces")
+  public String getInterfaces() {
+    return this.interfaces;
+  }
+
+  @JsonProperty("interfaces")
+  public void setInterfaces(String interfaces) {
+    this.interfaces = interfaces;
+  }
+
+  @JsonProperty("ipaddress")
+  public String getIPAddress() {
+    return this.ipaddress;
+  }
+
+  @JsonProperty("ipaddress")
+  public void setIPAddress(String ipaddress) {
+    this.ipaddress = ipaddress;
+  }
+
+  @JsonProperty("kernel")
+  public String getKernel() {
+    return this.kernel;
+  }
+
+  @JsonProperty("kernel")
+  public void setKernel(String kernel) {
+    this.kernel = kernel;
+  }
+
+  @JsonProperty("kernelmajversion")
+  public String getKernelMajVersion() {
+    return this.kernelmajversion;
+  }
+
+  @JsonProperty("kernelmajversion")
+  public void setKernelMajVersion(String kernelmajversion) {
+    this.kernelmajversion = kernelmajversion;
+  }
+
+  @JsonProperty("kernelrelease")
+  public String getKernelRelease() {
+    return this.kernelrelease;
+  }
+
+  @JsonProperty("kernelrelease")
+  public void setKernelRelease(String kernelrelease) {
+    this.kernelrelease = kernelrelease;
+  }
+
+  @JsonProperty("kernelversion")
+  public String getKernelVersion() {
+    return this.kernelversion;
+  }
+
+  @JsonProperty("kernelversion")
+  public void setKernelVersion(String kernelversion) {
+    this.kernelversion = kernelversion;
+  }
+
+  @JsonProperty("macaddress")
+  public String getMacAddress() {
+    return this.macaddress;
+  }
+
+  @JsonProperty("macaddress")
+  public void setMacAddress(String macaddress) {
+    this.macaddress = macaddress;
+  }
+
+  @JsonProperty("memoryfree")
+  public long getFreeMemory() {
+    return this.memoryfree;
+  }
+
+  @JsonProperty("memoryfree")
+  public void setFreeMemory(long memoryfree) {
+    this.memoryfree = memoryfree;
+  }
+
+  @JsonProperty("memorysize")
+  public long getMemorySize() {
+    return this.memorysize;
+  }
+
+  @JsonProperty("memorysize")
+  public void setMemorySize(long memorysize) {
+    this.memorysize = memorysize;
+  }
+
+  @JsonProperty("mounts")
+  public List<DiskInfo> getMounts() {
+    return this.mounts;
+  }
+
+  @JsonProperty("mounts")
+  public void setMounts(List<DiskInfo> mounts) {
+    this.mounts = mounts;
+  }
+
+  @JsonProperty("memorytotal")
+  public long getMemoryTotal() {
+    return this.memorytotal;
+  }
+
+  @JsonProperty("memorytotal")
+  public void setMemoryTotal(long memorytotal) {
+    this.memorytotal = memorytotal;
+  }
+
+  @JsonProperty("netmask")
+  public String getNetMask() {
+    return this.netmask;
+  }
+
+  @JsonProperty("netmask")
+  public void setNetMask(String netmask) {
+    this.netmask = netmask;
+  }
+
+  @JsonProperty("operatingsystem")
+  public String getOS() {
+    return this.operatingsystem;
+  }
+
+  @JsonProperty("operatingsystem")
+  public void setOS(String operatingsystem) {
+    this.operatingsystem = operatingsystem;
+  }
+
+  @JsonProperty("operatingsystemrelease")
+  public String getOSRelease() {
+    return this.operatingsystemrelease;
+  }
+
+  @JsonProperty("operatingsystemrelease")
+  public void setOSRelease(String operatingsystemrelease) {
+    this.operatingsystemrelease = operatingsystemrelease;
+  }
+
+  @JsonProperty("osfamily")
+  public String getOSFamily() {
+    return this.osfamily;
+  }
+
+  @JsonProperty("osfamily")
+  public void setOSFamily(String osfamily) {
+    this.osfamily = osfamily;
+  }
+
+  @JsonProperty("physicalprocessorcount")
+  public int getPhysicalProcessorCount() {
+    return this.physicalprocessorcount;
+  }
+
+  @JsonProperty("physicalprocessorcount")
+  public void setPhysicalProcessorCount(int physicalprocessorcount) {
+    this.physicalprocessorcount = physicalprocessorcount;
+  }
+
+  @JsonProperty("processorcount")
+  public int getProcessorCount() {
+    return this.processorcount;
+  }
+
+  @JsonProperty("processorcount")
+  public void setProcessorCount(int processorcount) {
+    this.processorcount = processorcount;
+  }
+
+  @JsonProperty("selinux")
+  public boolean getSeLinux() {
+    return selinux;
+  }
+
+  @JsonProperty("selinux")
+  public void setSeLinux(boolean selinux) {
+    this.selinux = selinux;
+  }
+
+  @JsonProperty("swapfree")
+  public String getSwapFree() {
+    return this.swapfree;
+  }
+
+  @JsonProperty("swapfree")
+  public void setSwapFree(String swapfree) {
+    this.swapfree = swapfree;
+  }
+
+  @JsonProperty("swapsize")
+  public String getSwapSize() {
+    return swapsize;
+  }
+
+  @JsonProperty("swapsize")
+  public void setSwapSize(String swapsize) {
+    this.swapsize = swapsize;
+  }
+
+  @JsonProperty("timezone")
+  public String getTimeZone() {
+    return this.timezone;
+  }
+
+  @JsonProperty("timezone")
+  public void setTimeZone(String timezone) {
+    this.timezone = timezone;
+  }
+
+  @JsonProperty("uptime")
+  public String getUptime() {
+    return this.uptime;
+  }
+
+  @JsonProperty("uptime")
+  public void setUpTime(String uptime) {
+    this.uptime = uptime;
+  }
+
+  @JsonProperty("uptime_hours")
+  public long getUptimeHours() {
+    return this.uptime_hours;
+  }
+
+  @JsonProperty("uptime_hours")
+  public void setUpTimeHours(long uptime_hours) {
+    this.uptime_hours = uptime_hours;
+  }
+
+  @JsonProperty("uptime_days")
+  public long getUpTimeDays() {
+    return this.uptime_days;
+  }
+
+  @JsonProperty("uptime_days")
+  public void setUpTimeDays(long uptime_days) {
+    this.uptime_days = uptime_days;
+  }
+
+  private String getDiskString() {
+    if (mounts == null) {
+      return null;
+    }
+    StringBuilder ret = new StringBuilder();
+    for (DiskInfo diskInfo : mounts) {
+      ret.append("(").append(diskInfo.toString()).append(")");
+    }
+    return ret.toString();
+  }
+
+  public String toString() {
+    return "[" +
+           "hostname=" + this.hostname + "," +
+           "fqdn=" + this.fqdn + "," +
+           "domain=" + this.domain + "," +
+           "architecture=" + this.architecture + "," +
+           "processorcount=" + this.processorcount + "," +
+           "physicalprocessorcount=" + this.physicalprocessorcount + "," +
+           "osname=" + this.operatingsystem + "," +
+           "osversion=" + this.operatingsystemrelease + "," +
+           "osfamily=" + this.osfamily + "," +
+           "memory=" + this.memorytotal + "," +
+           "uptime_hours=" + this.uptime_hours + "," +
+           "mounts=" + getDiskString() + "]\n";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HostStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HostStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HostStatus.java
new file mode 100644
index 0000000..c584149
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HostStatus.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.agent;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+/**
+ *
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class HostStatus {
+  public HostStatus(Status status, String cause) {
+    super();
+    this.status = status;
+    this.cause = cause;
+  }
+  public HostStatus() {
+    super();
+  }
+
+  public enum Status {
+    HEALTHY,
+    UNHEALTHY
+  }
+  Status status;
+  String cause;
+  public Status getStatus() {
+    return status;
+  }
+  public void setStatus(Status status) {
+    this.status = status;
+  }
+  public String getCause() {
+    return cause;
+  }
+  public void setCause(String cause) {
+    this.cause = cause;
+  }
+
+  @Override
+  public String toString() {
+    return "HostStatus{" +
+           "status=" + status +
+           ", cause='" + cause + '\'' +
+           '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/Register.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/Register.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/Register.java
new file mode 100644
index 0000000..0150079
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/Register.java
@@ -0,0 +1,193 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.agent;
+
+import org.apache.slider.providers.agent.State;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.annotate.JsonProperty;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.util.Map;
+
+/** Data model for agent to send heartbeat to ambari and/or app master. */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class Register {
+  private int responseId = -1;
+  private long timestamp;
+  private String label;
+  private int currentPingPort;
+  private HostInfo hardwareProfile;
+  private String publicHostname;
+  private String tags;
+  private AgentEnv agentEnv;
+  private String agentVersion;
+  private State actualState;
+  private State expectedState;
+  private Map<String, String> allocatedPorts;
+  private Map<String, String> logFolders;
+  private String pkg;
+  private String appVersion;
+
+  @JsonProperty("responseId")
+  public int getResponseId() {
+    return responseId;
+  }
+
+  @JsonProperty("responseId")
+  public void setResponseId(int responseId) {
+    this.responseId = responseId;
+  }
+
+  public long getTimestamp() {
+    return timestamp;
+  }
+
+  public void setTimestamp(long timestamp) {
+    this.timestamp = timestamp;
+  }
+
+  public String getLabel() {
+    return label;
+  }
+
+  public void setLabel(String label) {
+    this.label = label;
+  }
+
+  public String getTags() {
+    return tags;
+  }
+
+  public void setTags(String tags) {
+    this.tags = tags;
+  }
+
+  public HostInfo getHardwareProfile() {
+    return hardwareProfile;
+  }
+
+  public void setHardwareProfile(HostInfo hardwareProfile) {
+    this.hardwareProfile = hardwareProfile;
+  }
+
+  public String getPublicHostname() {
+    return publicHostname;
+  }
+
+  public void setPublicHostname(String name) {
+    this.publicHostname = name;
+  }
+
+  public AgentEnv getAgentEnv() {
+    return agentEnv;
+  }
+
+  public void setAgentEnv(AgentEnv env) {
+    this.agentEnv = env;
+  }
+
+  public String getAgentVersion() {
+    return agentVersion;
+  }
+
+  public void setAgentVersion(String agentVersion) {
+    this.agentVersion = agentVersion;
+  }
+
+  public int getCurrentPingPort() {
+    return currentPingPort;
+  }
+
+  public void setCurrentPingPort(int currentPingPort) {
+    this.currentPingPort = currentPingPort;
+  }
+
+  public State getActualState() {
+    return actualState;
+  }
+
+  public void setActualState(State actualState) {
+    this.actualState = actualState;
+  }
+
+  public State getExpectedState() {
+    return expectedState;
+  }
+
+  public void setExpectedState(State expectedState) {
+    this.expectedState = expectedState;
+  }
+
+  /** @return the allocated ports, or <code>null</code> if none are present */
+  @JsonProperty("allocatedPorts")
+  public Map<String, String> getAllocatedPorts() {
+    return allocatedPorts;
+  }
+
+  /** @param ports allocated ports */
+  @JsonProperty("allocatedPorts")
+  public void setAllocatedPorts(Map<String, String> ports) {
+    this.allocatedPorts = ports;
+  }
+
+  /** @return the log folders, or <code>null</code> if none are present */
+  @JsonProperty("logFolders")
+  public Map<String, String> getLogFolders() {
+    return logFolders;
+  }
+
+  /** @param logFolders assigned log folders */
+  @JsonProperty("logFolders")
+  public void setLogFolders(Map<String, String> logFolders) {
+    this.logFolders = logFolders;
+  }
+
+  public String getPkg() {
+    return pkg;
+  }
+
+  public void setPkg(String pkg) {
+    this.pkg = pkg;
+  }
+
+  @JsonProperty("appVersion")
+  public String getAppVersion() {
+    return appVersion;
+  }
+
+  @JsonProperty("appVersion")
+  public void setAppVersion(String appVersion) {
+    this.appVersion = appVersion;
+  }
+
+  @Override
+  public String toString() {
+    String ret = "responseId=" + responseId + "\n" +
+                 "timestamp=" + timestamp + "\n" +
+                 "label=" + label + "\n" +
+                 "hostname=" + publicHostname + "\n" +
+                 "expectedState=" + expectedState + "\n" +
+                 "actualState=" + actualState + "\n" +
+                 "appVersion=" + appVersion + "\n";
+
+    if (hardwareProfile != null) {
+      ret = ret + "hardwareprofile=" + this.hardwareProfile.toString();
+    }
+    return ret;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationCommand.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationCommand.java
new file mode 100644
index 0000000..4b87dd2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/RegistrationCommand.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.server.appmaster.web.rest.agent;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+/**
+ *
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class RegistrationCommand {
+
+  private String command;
+
+  public String getCommand() {
+    return command;
+  }
+
+  public void setCommand(String command) {
+    this.command = command;
+  }
+
+  public RegistrationCommand(String command) {
+
+    this.command = command;
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[62/76] [abbrv] hadoop git commit: YARN-5909. Remove agent related code in slider AM. Contributed by Jian He

Posted by ji...@apache.org.
YARN-5909. Remove agent related code in slider AM. Contributed by Jian He


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

Branch: refs/heads/yarn-native-services
Commit: 9fdab600e38b270587d6fc7afef727a7725104c0
Parents: 5877e0c
Author: Jian He <ji...@apache.org>
Authored: Tue Nov 29 13:06:45 2016 -0800
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 hadoop-project/pom.xml                          |    5 -
 .../services/webapp/ApplicationApiWebApp.java   |    2 +-
 .../hadoop-yarn-slider-core/pom.xml             |   23 +-
 .../apache/slider/common/tools/SliderUtils.java |    1 -
 .../providers/AbstractProviderService.java      |   21 +-
 .../slider/providers/ProviderService.java       |    9 +-
 .../providers/agent/AgentClientProvider.java    |  713 -----
 .../providers/agent/AgentLaunchParameter.java   |  130 -
 .../providers/agent/AgentProviderFactory.java   |   47 -
 .../providers/agent/AgentProviderService.java   | 2850 ------------------
 .../slider/providers/agent/AgentRoles.java      |   38 -
 .../slider/providers/agent/AgentUtils.java      |  150 -
 .../apache/slider/providers/agent/Command.java  |   59 -
 .../slider/providers/agent/CommandResult.java   |   40 -
 .../providers/agent/ComponentCommandOrder.java  |  225 --
 .../providers/agent/ComponentInstanceState.java |  340 ---
 .../providers/agent/ComponentTagProvider.java   |  127 -
 .../slider/providers/agent/ContainerState.java  |   41 -
 .../providers/agent/HeartbeatMonitor.java       |  130 -
 .../apache/slider/providers/agent/State.java    |  199 --
 .../application/metadata/AbstractComponent.java |   80 -
 .../metadata/AbstractMetainfoParser.java        |  130 -
 .../metadata/AbstractMetainfoSchema.java        |   69 -
 .../metadata/AddonPackageMetainfoParser.java    |   53 -
 .../agent/application/metadata/Application.java |  193 --
 .../metadata/ApplicationPackage.java            |   69 -
 .../application/metadata/CommandOrder.java      |   61 -
 .../application/metadata/CommandScript.java     |   72 -
 .../agent/application/metadata/Component.java   |  217 --
 .../application/metadata/ComponentCommand.java  |   85 -
 .../application/metadata/ComponentExport.java   |   54 -
 .../metadata/ComponentsInAddonPackage.java      |   26 -
 .../agent/application/metadata/ConfigFile.java  |   59 -
 .../application/metadata/DefaultConfig.java     |   39 -
 .../metadata/DefaultConfigParser.java           |   54 -
 .../application/metadata/DockerContainer.java   |  187 --
 .../metadata/DockerContainerInputFile.java      |   50 -
 .../metadata/DockerContainerMount.java          |   60 -
 .../metadata/DockerContainerPort.java           |   66 -
 .../agent/application/metadata/Export.java      |   61 -
 .../agent/application/metadata/ExportGroup.java |   71 -
 .../agent/application/metadata/Metainfo.java    |  118 -
 .../application/metadata/MetainfoParser.java    |   97 -
 .../agent/application/metadata/OSPackage.java   |   51 -
 .../agent/application/metadata/OSSpecific.java  |   57 -
 .../agent/application/metadata/Package.java     |   60 -
 .../application/metadata/PropertyInfo.java      |   54 -
 .../agent/application/metadata/Validate.java    |   27 -
 .../org/apache/slider/providers/agent/todo.md   |   22 -
 .../server/appmaster/SliderAppMaster.java       |  140 +-
 .../server/appmaster/web/AgentService.java      |   37 -
 .../slider/server/appmaster/web/WebAppApi.java  |    7 -
 .../server/appmaster/web/WebAppApiImpl.java     |    6 -
 .../appmaster/web/rest/AMWebServices.java       |    1 -
 .../web/rest/agent/AgentCommandType.java        |   23 -
 .../appmaster/web/rest/agent/AgentEnv.java      |  376 ---
 .../appmaster/web/rest/agent/AgentResource.java |  118 -
 .../web/rest/agent/AgentRestOperations.java     |   28 -
 .../appmaster/web/rest/agent/AgentWebApp.java   |  258 --
 .../web/rest/agent/AgentWebServices.java        |   40 -
 .../appmaster/web/rest/agent/CommandReport.java |  207 --
 .../web/rest/agent/ComponentStatus.java         |  129 -
 .../appmaster/web/rest/agent/DiskInfo.java      |  128 -
 .../web/rest/agent/ExecutionCommand.java        |  310 --
 .../appmaster/web/rest/agent/HeartBeat.java     |  149 -
 .../web/rest/agent/HeartBeatResponse.java       |  147 -
 .../appmaster/web/rest/agent/HostInfo.java      |  398 ---
 .../appmaster/web/rest/agent/HostStatus.java    |   63 -
 .../appmaster/web/rest/agent/Register.java      |  193 --
 .../web/rest/agent/RegistrationCommand.java     |   43 -
 .../web/rest/agent/RegistrationResponse.java    |  133 -
 .../web/rest/agent/RegistrationStatus.java      |   22 -
 .../appmaster/web/rest/agent/StatusCommand.java |  152 -
 .../agent/TestAgentClientProvider.java          |   77 -
 .../agent/TestAgentLaunchParameter.java         |   76 -
 .../slider/providers/agent/TestAgentUtils.java  |   94 -
 .../agent/TestAppDefinitionPersister.java       |  264 --
 .../agent/TestComponentTagProvider.java         |  115 -
 .../slider/providers/agent/TestState.java       |   33 -
 .../application/metadata/TestConfigParser.java  |  107 -
 .../metadata/TestMetainfoParser.java            |  177 --
 .../publisher/TestAgentProviderService.java     |   60 -
 .../publisher/TestSliderProviderFactory.java    |   40 -
 83 files changed, 26 insertions(+), 11517 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 96b3d86..2e61bd4 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -879,11 +879,6 @@
         <version>${jackson2.version}</version>
       </dependency>
       <dependency>
-        <groupId>com.fasterxml.jackson.jaxrs</groupId>
-        <artifactId>jackson-jaxrs-json-provider</artifactId>
-        <version>${jackson2.version}</version>
-      </dependency>
-      <dependency>
         <groupId>com.fasterxml.jackson.dataformat</groupId>
         <artifactId>jackson-dataformat-cbor</artifactId>
         <version>${jackson2.version}</version>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/webapp/ApplicationApiWebApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/webapp/ApplicationApiWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/webapp/ApplicationApiWebApp.java
index 52a9de6..e1bddb5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/webapp/ApplicationApiWebApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/webapp/ApplicationApiWebApp.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.services.api.impl.ApplicationApiService;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
-import org.mortbay.jetty.webapp.Configuration;
+import org.eclipse.jetty.webapp.Configuration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
index a2c67c0..66e9ee9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/pom.xml
@@ -225,7 +225,7 @@
 
     <dependency>
       <groupId>javax.servlet</groupId>
-      <artifactId>servlet-api</artifactId>
+      <artifactId>javax.servlet-api</artifactId>
     </dependency>
 
     <dependency>
@@ -289,21 +289,6 @@
     </dependency>
 
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty-util</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty-sslengine</artifactId>
-    </dependency>
-
-    <dependency>
       <groupId>javax.servlet.jsp</groupId>
       <artifactId>jsp-api</artifactId>
       <scope>runtime</scope>
@@ -315,12 +300,6 @@
     </dependency>
 
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty-sslengine</artifactId>
-      <version>6.1.26</version>
-    </dependency>
-
-    <dependency>
       <groupId>org.yaml</groupId>
       <artifactId>snakeyaml</artifactId>
       <version>1.16</version>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
index 713cd02..b101d34 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
@@ -70,7 +70,6 @@ import org.apache.slider.core.exceptions.SliderException;
 import org.apache.slider.core.launch.ClasspathConstructor;
 import org.apache.slider.core.main.LauncherExitCodes;
 import org.apache.slider.providers.agent.AgentKeys;
-import org.apache.slider.providers.agent.application.metadata.Component;
 import org.apache.slider.server.services.utility.PatternValidator;
 import org.apache.slider.server.services.workflow.ForkedProcessService;
 import org.apache.zookeeper.server.util.KerberosUtil;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
index 19fa07b..00fc606 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
@@ -19,17 +19,17 @@
 package org.apache.slider.providers;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.registry.client.binding.RegistryTypeUtils;
+import org.apache.hadoop.registry.client.exceptions.InvalidRecordException;
+import org.apache.hadoop.registry.client.types.AddressTypes;
+import org.apache.hadoop.registry.client.types.Endpoint;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
 import org.apache.hadoop.service.Service;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
-import org.apache.hadoop.registry.client.binding.RegistryTypeUtils;
-import org.apache.hadoop.registry.client.exceptions.InvalidRecordException;
-import org.apache.hadoop.registry.client.types.AddressTypes;
-import org.apache.hadoop.registry.client.types.Endpoint;
-import org.apache.hadoop.registry.client.types.ServiceRecord;
 import org.apache.slider.api.ClusterDescription;
 import org.apache.slider.common.SliderKeys;
 import org.apache.slider.common.tools.ConfigHelper;
@@ -44,7 +44,6 @@ import org.apache.slider.server.appmaster.operations.AbstractRMOperation;
 import org.apache.slider.server.appmaster.state.ContainerReleaseSelector;
 import org.apache.slider.server.appmaster.state.MostRecentContainerReleaseSelector;
 import org.apache.slider.server.appmaster.state.StateAccessForProviders;
-import org.apache.slider.server.appmaster.web.rest.agent.AgentRestOperations;
 import org.apache.slider.server.services.workflow.ForkedProcessService;
 import org.apache.slider.server.services.workflow.ServiceParent;
 import org.apache.slider.server.services.workflow.WorkflowSequenceService;
@@ -77,7 +76,6 @@ public abstract class AbstractProviderService
   private static final Logger log =
     LoggerFactory.getLogger(AbstractProviderService.class);
   protected StateAccessForProviders amState;
-  protected AgentRestOperations restOps;
   protected URL amWebAPI;
   protected YarnRegistryViewForProviders yarnRegistry;
   protected QueueAccess queueAccess;
@@ -127,18 +125,9 @@ public abstract class AbstractProviderService
   }
 
   @Override
-  public AgentRestOperations getAgentRestOperations() {
-    return restOps;
-  }
-
-  @Override
   public void notifyContainerCompleted(ContainerId containerId) {
   }
 
-  public void setAgentRestOperations(AgentRestOperations agentRestOperations) {
-    this.restOps = agentRestOperations;
-  }
-
   /**
    * Load default Configuration
    * @param confDir configuration directory

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
index b62510a..4ca9326 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
@@ -20,10 +20,10 @@ package org.apache.slider.providers;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
 import org.apache.hadoop.service.Service;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.registry.client.types.ServiceRecord;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.slider.api.ClusterDescription;
 import org.apache.slider.common.tools.SliderFileSystem;
@@ -37,7 +37,6 @@ import org.apache.slider.server.appmaster.actions.QueueAccess;
 import org.apache.slider.server.appmaster.operations.RMOperationHandlerActions;
 import org.apache.slider.server.appmaster.state.ContainerReleaseSelector;
 import org.apache.slider.server.appmaster.state.StateAccessForProviders;
-import org.apache.slider.server.appmaster.web.rest.agent.AgentRestOperations;
 import org.apache.slider.server.services.yarnregistry.YarnRegistryViewForProviders;
 
 import java.io.File;
@@ -176,12 +175,6 @@ public interface ProviderService extends ProviderCore,
   void bindToYarnRegistry(YarnRegistryViewForProviders yarnRegistry);
 
   /**
-   * Returns the agent rest operations interface.
-   * @return  the interface if available, null otherwise.
-   */
-  AgentRestOperations getAgentRestOperations();
-
-  /**
    * Build up the endpoint details for this service
    * @param details
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
deleted file mode 100644
index 7ca469f..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
+++ /dev/null
@@ -1,713 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.slider.providers.agent;
-
-import com.google.common.io.Files;
-import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
-import org.apache.commons.compress.archivers.tar.TarArchiveInputStream;
-import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang.StringUtils;
-import org.apache.commons.lang.exception.ExceptionUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.registry.client.api.RegistryOperations;
-import org.apache.hadoop.registry.client.binding.RegistryUtils;
-import org.apache.slider.api.InternalKeys;
-import org.apache.slider.api.ResourceKeys;
-import org.apache.slider.client.ClientUtils;
-import org.apache.slider.common.SliderKeys;
-import org.apache.slider.common.tools.SliderFileSystem;
-import org.apache.slider.common.tools.SliderUtils;
-import org.apache.slider.core.conf.AggregateConf;
-import org.apache.slider.core.conf.ConfTreeOperations;
-import org.apache.slider.core.conf.MapOperations;
-import org.apache.slider.core.exceptions.BadConfigException;
-import org.apache.slider.core.exceptions.SliderException;
-import org.apache.slider.core.launch.AbstractLauncher;
-import org.apache.slider.core.registry.docstore.PublishedConfiguration;
-import org.apache.slider.providers.AbstractClientProvider;
-import org.apache.slider.providers.ProviderRole;
-import org.apache.slider.providers.ProviderUtils;
-import org.apache.slider.providers.agent.application.metadata.Application;
-import org.apache.slider.providers.agent.application.metadata.Component;
-import org.apache.slider.providers.agent.application.metadata.ConfigFile;
-import org.apache.slider.providers.agent.application.metadata.Metainfo;
-import org.apache.slider.providers.agent.application.metadata.MetainfoParser;
-import org.apache.slider.providers.agent.application.metadata.OSPackage;
-import org.apache.slider.providers.agent.application.metadata.OSSpecific;
-import org.apache.slider.providers.agent.application.metadata.Package;
-import org.codehaus.jettison.json.JSONException;
-import org.codehaus.jettison.json.JSONObject;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.nio.charset.Charset;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Set;
-import java.util.zip.ZipEntry;
-import java.util.zip.ZipInputStream;
-
-import static org.apache.slider.common.tools.SliderUtils.getApplicationDefinitionPath;
-
-/** This class implements  the client-side aspects of the agent deployer */
-public class AgentClientProvider extends AbstractClientProvider
-    implements AgentKeys, SliderKeys {
-
-
-  protected static final Logger log =
-      LoggerFactory.getLogger(AgentClientProvider.class);
-  protected static final String NAME = "agent";
-  private static final ProviderUtils providerUtils = new ProviderUtils(log);
-  public static final String E_COULD_NOT_READ_METAINFO
-      = "Not a valid app package. Could not read metainfo.";
-
-  protected Map<String, Metainfo> metaInfoMap = new ConcurrentHashMap<String, Metainfo>();
-
-  protected AgentClientProvider(Configuration conf) {
-    super(conf);
-  }
-
-  @Override
-  public String getName() {
-    return NAME;
-  }
-
-  @Override
-  public List<ProviderRole> getRoles() {
-    return AgentRoles.getRoles();
-  }
-
-  @Override //Client
-  public void preflightValidateClusterConfiguration(SliderFileSystem sliderFileSystem,
-                                                    String clustername,
-                                                    Configuration configuration,
-                                                    AggregateConf instanceDefinition,
-                                                    Path clusterDirPath,
-                                                    Path generatedConfDirPath,
-                                                    boolean secure) throws
-      SliderException,
-      IOException {
-    super.preflightValidateClusterConfiguration(sliderFileSystem, clustername,
-                                                configuration,
-                                                instanceDefinition,
-                                                clusterDirPath,
-                                                generatedConfDirPath, secure);
-
-    String appDef = SliderUtils.getApplicationDefinitionPath(instanceDefinition
-        .getAppConfOperations());
-    Path appDefPath = new Path(appDef);
-    sliderFileSystem.verifyFileExists(appDefPath);
-
-    String agentConf = instanceDefinition.getAppConfOperations().
-        getGlobalOptions().getOption(AGENT_CONF, "");
-    if (StringUtils.isNotEmpty(agentConf)) {
-      sliderFileSystem.verifyFileExists(new Path(agentConf));
-    }
-
-    String appHome = instanceDefinition.getAppConfOperations().
-        getGlobalOptions().get(PACKAGE_PATH);
-    if (SliderUtils.isUnset(appHome)) {
-      String agentImage = instanceDefinition.getInternalOperations().
-          get(InternalKeys.INTERNAL_APPLICATION_IMAGE_PATH);
-      sliderFileSystem.verifyFileExists(new Path(agentImage));
-    }
-  }
-
-  @Override
-  public void validateInstanceDefinition(AggregateConf instanceDefinition, SliderFileSystem fs) throws
-      SliderException {
-    super.validateInstanceDefinition(instanceDefinition, fs);
-    log.debug("Validating conf {}", instanceDefinition);
-    ConfTreeOperations resources =
-        instanceDefinition.getResourceOperations();
-
-    providerUtils.validateNodeCount(instanceDefinition, ROLE_NODE,
-                                    0, -1);
-
-    String appDef = null;
-    try {
-      // Validate the app definition
-      appDef = SliderUtils.getApplicationDefinitionPath(instanceDefinition
-          .getAppConfOperations());
-    } catch (BadConfigException bce) {
-      throw new BadConfigException("Application definition must be provided. " + bce.getMessage());
-    }
-
-    log.info("Validating app definition {}", appDef);
-    String extension = appDef.substring(appDef.lastIndexOf(".") + 1, appDef.length());
-    if (!"zip".equals(extension.toLowerCase(Locale.ENGLISH))) {
-      throw new BadConfigException("App definition must be packaged as a .zip file. File provided is " + appDef);
-    }
-
-    Set<String> names = resources.getComponentNames();
-    names.remove(COMPONENT_AM);
-    Map<Integer, String> priorityMap = new HashMap<Integer, String>();
-
-    for (String name : names) {
-      try {
-        // Validate the app definition
-        appDef = SliderUtils.getApplicationDefinitionPath(instanceDefinition
-            .getAppConfOperations(), name);
-      } catch (BadConfigException bce) {
-        throw new BadConfigException("Application definition must be provided. " + bce.getMessage());
-      }
-      Metainfo metaInfo = getMetainfo(fs, appDef);
-
-      MapOperations component = resources.getMandatoryComponent(name);
-
-      if (metaInfo != null) {
-        Component componentDef = metaInfo.getApplicationComponent(
-            AgentUtils.getMetainfoComponentName(name,
-                instanceDefinition.getAppConfOperations()));
-        if (componentDef == null) {
-          throw new BadConfigException(
-              "Component %s is not a member of application.", name);
-        }
-      }
-
-      int priority =
-          component.getMandatoryOptionInt(ResourceKeys.COMPONENT_PRIORITY);
-      if (priority <= 0) {
-        throw new BadConfigException("Component %s %s value out of range %d",
-                                     name,
-                                     ResourceKeys.COMPONENT_PRIORITY,
-                                     priority);
-      }
-
-      String existing = priorityMap.get(priority);
-      if (existing != null) {
-        throw new BadConfigException(
-            "Component %s has a %s value %d which duplicates that of %s",
-            name,
-            ResourceKeys.COMPONENT_PRIORITY,
-            priority,
-            existing);
-      }
-      priorityMap.put(priority, name);
-
-      // fileSystem may be null for tests
-      if (metaInfo != null) {
-        Component componentDef = metaInfo.getApplicationComponent(
-            AgentUtils.getMetainfoComponentName(name,
-                instanceDefinition.getAppConfOperations()));
-
-        // ensure that intance count is 0 for client components
-        if ("CLIENT".equals(componentDef.getCategory())) {
-          MapOperations componentConfig = resources.getMandatoryComponent(name);
-          int count =
-              componentConfig.getMandatoryOptionInt(ResourceKeys.COMPONENT_INSTANCES);
-          if (count > 0) {
-            throw new BadConfigException("Component %s is of type CLIENT and cannot be instantiated."
-                                         + " Use \"slider client install ...\" command instead.",
-                                         name);
-          }
-        } else {
-          MapOperations componentConfig = resources.getMandatoryComponent(name);
-          int count =
-              componentConfig.getMandatoryOptionInt(ResourceKeys.COMPONENT_INSTANCES);
-          int definedMinCount = componentDef.getMinInstanceCountInt();
-          int definedMaxCount = componentDef.getMaxInstanceCountInt();
-          if (count < definedMinCount || count > definedMaxCount) {
-            throw new BadConfigException("Component %s, %s value %d out of range. "
-                                         + "Expected minimum is %d and maximum is %d",
-                                         name,
-                                         ResourceKeys.COMPONENT_INSTANCES,
-                                         count,
-                                         definedMinCount,
-                                         definedMaxCount);
-          }
-        }
-      }
-    }
-  }
-
-
-  @Override
-  public void prepareAMAndConfigForLaunch(SliderFileSystem fileSystem,
-                                          Configuration serviceConf,
-                                          AbstractLauncher launcher,
-                                          AggregateConf instanceDefinition,
-                                          Path snapshotConfDirPath,
-                                          Path generatedConfDirPath,
-                                          Configuration clientConfExtras,
-                                          String libdir,
-                                          Path tempPath,
-                                          boolean miniClusterTestRun) throws
-      IOException,
-      SliderException {
-    String agentImage = instanceDefinition.getInternalOperations().
-        get(InternalKeys.INTERNAL_APPLICATION_IMAGE_PATH);
-    if (SliderUtils.isUnset(agentImage)) {
-      Path agentPath = new Path(tempPath.getParent(), PROVIDER_AGENT);
-      log.info("Automatically uploading the agent tarball at {}", agentPath);
-      fileSystem.getFileSystem().mkdirs(agentPath);
-      if (ProviderUtils.addAgentTar(this, AGENT_TAR, fileSystem, agentPath)) {
-        instanceDefinition.getInternalOperations().set(
-            InternalKeys.INTERNAL_APPLICATION_IMAGE_PATH,
-            new Path(agentPath, AGENT_TAR).toUri());
-      }
-    }
-  }
-
-  @Override
-  public Set<String> getApplicationTags(SliderFileSystem fileSystem,
-      ConfTreeOperations appConf) throws SliderException {
-    return getApplicationTags(fileSystem,
-        getApplicationDefinitionPath(appConf));
-  }
-
-  public Set<String> getApplicationTags(SliderFileSystem fileSystem,
-                                        String appDef) throws SliderException {
-    Set<String> tags;
-    Metainfo metaInfo = getMetainfo(fileSystem, appDef);
-
-    if (metaInfo == null) {
-      log.error("Error retrieving metainfo from {}", appDef);
-      throw new SliderException("Error parsing metainfo file, possibly bad structure.");
-    }
-
-    Application application = metaInfo.getApplication();
-    tags = new HashSet<String>();
-    tags.add("Name: " + application.getName());
-    tags.add("Version: " + application.getVersion());
-    tags.add("Description: " + SliderUtils.truncate(application.getComment(), 80));
-
-    return tags;
-  }
-
-  @Override
-  public void processClientOperation(SliderFileSystem fileSystem,
-                                     RegistryOperations rops,
-                                     Configuration configuration,
-                                     String operation,
-                                     File clientInstallPath,
-                                     File appPackage,
-                                     JSONObject config,
-                                     String name) throws SliderException {
-    // create temp folder
-    // create sub-folders app_pkg, agent_pkg, command
-    File tmpDir = Files.createTempDir();
-    log.info("Command is being executed at {}", tmpDir.getAbsolutePath());
-    File appPkgDir = new File(tmpDir, "app_pkg");
-    appPkgDir.mkdir();
-
-    File agentPkgDir = new File(tmpDir, "agent_pkg");
-    agentPkgDir.mkdir();
-
-    File cmdDir = new File(tmpDir, "command");
-    cmdDir.mkdir();
-
-    Metainfo metaInfo = null;
-    JSONObject defaultConfig = null;
-    try {
-      // expand app package into /app_pkg
-      ZipInputStream zipInputStream = null;
-      try {
-        zipInputStream = new ZipInputStream(new FileInputStream(appPackage));
-        {
-          ZipEntry zipEntry = zipInputStream.getNextEntry();
-          while (zipEntry != null) {
-            log.info("Processing {}", zipEntry.getName());
-            String filePath = appPkgDir + File.separator + zipEntry.getName();
-            if (!zipEntry.isDirectory()) {
-              log.info("Extracting file {}", filePath);
-              extractFile(zipInputStream, filePath);
-
-              if ("metainfo.xml".equals(zipEntry.getName())) {
-                FileInputStream input = null;
-                try {
-                  input = new FileInputStream(filePath);
-                  metaInfo = new MetainfoParser().fromXmlStream(input);
-                } finally {
-                  IOUtils.closeStream(input);
-                }
-              } else if ("metainfo.json".equals(zipEntry.getName())) {
-                FileInputStream input = null;
-                try {
-                  input = new FileInputStream(filePath);
-                  metaInfo = new MetainfoParser().fromJsonStream(input);
-                } finally {
-                  IOUtils.closeStream(input);
-                }
-              } else if ("clientInstallConfig-default.json".equals(zipEntry.getName())) {
-                try {
-                  defaultConfig = new JSONObject(FileUtils.readFileToString(new File(filePath), Charset.defaultCharset()));
-                } catch (JSONException jex) {
-                  throw new SliderException("Unable to read default client config.", jex);
-                }
-              }
-            } else {
-              log.info("Creating dir {}", filePath);
-              File dir = new File(filePath);
-              dir.mkdir();
-            }
-            zipInputStream.closeEntry();
-            zipEntry = zipInputStream.getNextEntry();
-          }
-        }
-      } finally {
-        zipInputStream.close();
-      }
-
-      if (metaInfo == null) {
-        throw new BadConfigException(E_COULD_NOT_READ_METAINFO);
-      }
-
-      String clientScript = null;
-      String clientComponent = null;
-      for (Component component : metaInfo.getApplication().getComponents()) {
-        if (component.getCategory().equals("CLIENT")) {
-          clientComponent = component.getName();
-          if (component.getCommandScript() != null) {
-            clientScript = component.getCommandScript().getScript();
-          }
-          break;
-        }
-      }
-
-      if (SliderUtils.isUnset(clientScript)) {
-        log.info("Installing CLIENT without script");
-        List<Package> packages = metaInfo.getApplication().getPackages();
-        if (packages.size() > 0) {
-          // retrieve package resources from HDFS and extract
-          for (Package pkg : packages) {
-            Path pkgPath = fileSystem.buildResourcePath(pkg.getName());
-            if (!fileSystem.isFile(pkgPath) && name != null) {
-              pkgPath = fileSystem.buildResourcePath(name, pkg.getName());
-            }
-            if (!fileSystem.isFile(pkgPath)) {
-              throw new IOException("Package doesn't exist as a resource: " +
-                  pkg.getName());
-            }
-            if ("archive".equals(pkg.getType())) {
-              File pkgFile = new File(tmpDir, pkg.getName());
-              fileSystem.copyHdfsFileToLocal(pkgPath, pkgFile);
-              expandTar(pkgFile, clientInstallPath);
-            } else {
-              File pkgFile = new File(clientInstallPath, pkg.getName());
-              fileSystem.copyHdfsFileToLocal(pkgPath, pkgFile);
-            }
-          }
-        } else {
-          // extract tarball from app def
-          for (OSSpecific osSpecific : metaInfo.getApplication()
-              .getOSSpecifics()) {
-            for (OSPackage pkg : osSpecific.getPackages()) {
-              if ("tarball".equals(pkg.getType())) {
-                File pkgFile = new File(appPkgDir, pkg.getName());
-                expandTar(pkgFile, clientInstallPath);
-              }
-            }
-          }
-        }
-        if (name == null) {
-          log.warn("Conf files not being generated because no app name was " +
-              "provided");
-          return;
-        }
-        File confInstallDir;
-        String clientRoot = null;
-        if (config != null) {
-          try {
-            clientRoot = config.getJSONObject("global")
-                .getString(APP_CLIENT_ROOT);
-          } catch (JSONException e) {
-            log.info("Couldn't read {} from provided client config, falling " +
-                "back on default", APP_CLIENT_ROOT);
-          }
-        }
-        if (clientRoot == null && defaultConfig != null) {
-          try {
-            clientRoot = defaultConfig.getJSONObject("global")
-                .getString(APP_CLIENT_ROOT);
-          } catch (JSONException e) {
-            log.info("Couldn't read {} from default client config, using {}",
-                APP_CLIENT_ROOT, clientInstallPath);
-          }
-        }
-        if (clientRoot == null) {
-          confInstallDir = clientInstallPath;
-        } else {
-          confInstallDir = new File(new File(clientInstallPath, clientRoot), "conf");
-          if (!confInstallDir.exists()) {
-            confInstallDir.mkdirs();
-          }
-        }
-        String user = RegistryUtils.currentUser();
-        for (ConfigFile configFile : metaInfo.getComponentConfigFiles(clientComponent)) {
-          retrieveConfigFile(rops, configuration, configFile, name, user,
-              confInstallDir);
-        }
-      } else {
-        log.info("Installing CLIENT using script {}", clientScript);
-        expandAgentTar(agentPkgDir);
-
-        JSONObject commandJson = getCommandJson(defaultConfig, config, metaInfo, clientInstallPath, name);
-        FileWriter file = new FileWriter(new File(cmdDir, "command.json"));
-        try {
-          file.write(commandJson.toString());
-
-        } catch (IOException e) {
-          log.error("Couldn't write command.json to file");
-        } finally {
-          file.flush();
-          file.close();
-        }
-
-        runCommand(appPkgDir, agentPkgDir, cmdDir, clientScript);
-      }
-
-    } catch (IOException ioex) {
-      log.warn("Error while executing INSTALL command {}", ioex.getMessage());
-      throw new SliderException("INSTALL client failed.");
-    }
-  }
-
-  protected void runCommand(
-      File appPkgDir,
-      File agentPkgDir,
-      File cmdDir,
-      String clientScript) throws SliderException {
-    int exitCode = 0;
-    Exception exp = null;
-    try {
-      String clientScriptPath = appPkgDir.getAbsolutePath() + File.separator + "package" +
-                                File.separator + clientScript;
-      List<String> command = Arrays.asList(PYTHON_EXE,
-               "-S",
-               clientScriptPath,
-               "INSTALL",
-               cmdDir.getAbsolutePath() + File.separator + "command.json",
-               appPkgDir.getAbsolutePath() + File.separator + "package",
-               cmdDir.getAbsolutePath() + File.separator + "command-out.json",
-               "DEBUG");
-      ProcessBuilder pb = new ProcessBuilder(command);
-      log.info("Command: " + StringUtils.join(pb.command(), " "));
-      pb.environment().put(PYTHONPATH,
-                           agentPkgDir.getAbsolutePath()
-                           + File.separator + "slider-agent" + File.pathSeparator
-                           + agentPkgDir.getAbsolutePath()
-                           + File.separator + "slider-agent/jinja2");
-      log.info("{}={}", PYTHONPATH, pb.environment().get(PYTHONPATH));
-
-      Process proc = pb.start();
-      InputStream stderr = proc.getErrorStream();
-      InputStream stdout = proc.getInputStream();
-      BufferedReader stdOutReader = new BufferedReader(new InputStreamReader(stdout));
-      BufferedReader stdErrReader = new BufferedReader(new InputStreamReader(stderr));
-
-      proc.waitFor();
-
-      String line;
-      while ((line = stdOutReader.readLine()) != null) {
-        log.info("Stdout: " + line);
-      }
-      while ((line = stdErrReader.readLine()) != null) {
-        log.info("Stderr: " + line);
-      }
-
-      exitCode = proc.exitValue();
-      log.info("Exit value is {}", exitCode);
-    } catch (IOException e) {
-      exp = e;
-    } catch (InterruptedException e) {
-      exp = e;
-    }
-
-    if (exitCode != 0) {
-      throw new SliderException("INSTALL client failed with exit code " + exitCode);
-    }
-
-    if (exp != null) {
-      log.error("Error while executing INSTALL command {}. Stack trace {}",
-                exp.getMessage(),
-                ExceptionUtils.getStackTrace(exp));
-      throw new SliderException("INSTALL client failed.", exp);
-    }
-  }
-
-  private void expandAgentTar(File agentPkgDir) throws IOException {
-    String libDirProp = SliderUtils.getLibDir();
-    File tarFile = new File(libDirProp, AGENT_TAR);
-    expandTar(tarFile, agentPkgDir);
-  }
-
-  private void expandTar(File tarFile, File destDir) throws IOException {
-    log.info("Expanding tar {} to {}", tarFile, destDir);
-    TarArchiveInputStream tarIn = new TarArchiveInputStream(
-        new GzipCompressorInputStream(
-            new BufferedInputStream(
-                new FileInputStream(tarFile)
-            )
-        )
-    );
-    try {
-      TarArchiveEntry tarEntry = tarIn.getNextTarEntry();
-      while (tarEntry != null) {
-        File destPath = new File(destDir, tarEntry.getName());
-        File parent = destPath.getParentFile();
-        if (!parent.exists()) {
-          parent.mkdirs();
-        }
-        if (tarEntry.isDirectory()) {
-          destPath.mkdirs();
-        } else {
-          byte[] byteToRead = new byte[1024];
-          BufferedOutputStream buffOut =
-              new BufferedOutputStream(new FileOutputStream(destPath));
-          try {
-            int len;
-            while ((len = tarIn.read(byteToRead)) != -1) {
-              buffOut.write(byteToRead, 0, len);
-            }
-          } finally {
-            buffOut.close();
-          }
-        }
-        if ((tarEntry.getMode() & 0100) != 0) {
-          destPath.setExecutable(true);
-        }
-        tarEntry = tarIn.getNextTarEntry();
-      }
-    } finally {
-      tarIn.close();
-    }
-  }
-
-  private void retrieveConfigFile(RegistryOperations rops,
-      Configuration configuration, ConfigFile configFile, String name,
-      String user, File destDir) throws IOException, SliderException {
-    log.info("Retrieving config {} to {}", configFile.getDictionaryName(),
-        destDir);
-    PublishedConfiguration published = ClientUtils.getConfigFromRegistry(rops,
-        configuration, configFile.getDictionaryName(), name, user, true);
-    ClientUtils.saveOrReturnConfig(published, configFile.getType(),
-        destDir, configFile.getFileName());
-  }
-
-  protected JSONObject getCommandJson(JSONObject defaultConfig,
-                                      JSONObject inputConfig,
-                                      Metainfo metainfo,
-                                      File clientInstallPath,
-                                      String name) throws SliderException {
-    try {
-      JSONObject pkgList = new JSONObject();
-      pkgList.put(PACKAGE_LIST,
-                  AgentProviderService.getPackageListFromApplication(metainfo.getApplication()));
-      JSONObject obj = new JSONObject();
-      obj.put("hostLevelParams", pkgList);
-
-      String user = RegistryUtils.currentUser();
-      JSONObject configuration = new JSONObject();
-      JSONObject global = new JSONObject();
-      global.put("app_install_dir", clientInstallPath.getAbsolutePath());
-      global.put("app_user", user);
-      if (name != null) {
-        global.put("app_name", name);
-      }
-
-      if (defaultConfig != null) {
-        readConfigEntries(defaultConfig, clientInstallPath, global, name, user);
-      }
-      if (inputConfig != null) {
-        readConfigEntries(inputConfig, clientInstallPath, global, name, user);
-      }
-
-      configuration.put("global", global);
-      obj.put("configurations", configuration);
-      return obj;
-    } catch (JSONException jex) {
-      log.warn("Error while executing INSTALL command {}", jex.getMessage());
-      throw new SliderException("INSTALL client failed.");
-    }
-  }
-
-  private void readConfigEntries(JSONObject inpConfig,
-                                 File clientInstallPath,
-                                 JSONObject globalConfig,
-                                 String name, String user)
-      throws JSONException {
-    JSONObject globalSection = inpConfig.getJSONObject("global");
-    Iterator it = globalSection.keys();
-    while (it.hasNext()) {
-      String key = (String) it.next();
-      String value = globalSection.getString(key);
-      if (SliderUtils.isSet(value)) {
-        value = value.replace("{app_install_dir}", clientInstallPath.getAbsolutePath());
-        value = value.replace("{app_user}", user);
-        if (name != null) {
-          value = value.replace("{app_name}", name);
-        }
-      }
-      if (globalConfig.has(key)) {
-        // last one wins
-        globalConfig.remove(key);
-      }
-      globalConfig.put(key, value);
-    }
-  }
-
-  private void extractFile(ZipInputStream zipInputStream, String filePath) throws IOException {
-    BufferedOutputStream output = new BufferedOutputStream(new FileOutputStream(filePath));
-    try {
-      byte[] bytesRead = new byte[4096];
-      int read = 0;
-      while ((read = zipInputStream.read(bytesRead)) != -1) {
-        output.write(bytesRead, 0, read);
-      }
-    } finally {
-      output.close();
-    }
-  }
-
-  private Metainfo getMetainfo(SliderFileSystem fs, String appDef) {
-    Metainfo metaInfo = metaInfoMap.get(appDef);
-    if (fs != null && metaInfo == null) {
-      try {
-        metaInfo = AgentUtils.getApplicationMetainfo(fs, appDef, false);
-        metaInfoMap.put(appDef, metaInfo);
-      } catch (IOException ioe) {
-        // Ignore missing metainfo file for now
-        log.info("Missing metainfo {}", ioe.getMessage());
-      } catch (BadConfigException bce) {
-        log.info("Bad Configuration {}", bce.getMessage());
-      }
-    }
-    return metaInfo;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentLaunchParameter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentLaunchParameter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentLaunchParameter.java
deleted file mode 100644
index 18c6374..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentLaunchParameter.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.slider.providers.agent;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.regex.Pattern;
-
-class AgentLaunchParameter {
-  public static final Logger log =
-      LoggerFactory.getLogger(AgentLaunchParameter.class);
-  private static final String DEFAULT_PARAMETER = "";
-  private static final String ANY_COMPONENT = "ANY";
-  private static final String NONE_VALUE = "NONE";
-  private final Map<String, CommandTracker> launchParameterTracker;
-
-  public AgentLaunchParameter(String parameters) {
-    launchParameterTracker = parseExpectedLaunchParameters(parameters);
-  }
-
-  /**
-   * Get command for the component type
-   *
-   * @param componentGroup
-   *
-   * @return
-   */
-  public String getNextLaunchParameter(String componentGroup) {
-    if (launchParameterTracker != null) {
-      if (launchParameterTracker.containsKey(componentGroup)
-          || launchParameterTracker.containsKey(ANY_COMPONENT)) {
-        synchronized (this) {
-          CommandTracker indexTracker = null;
-          if (launchParameterTracker.containsKey(componentGroup)) {
-            indexTracker = launchParameterTracker.get(componentGroup);
-          } else {
-            indexTracker = launchParameterTracker.get(ANY_COMPONENT);
-          }
-
-          return indexTracker.getNextCommand();
-        }
-      }
-    }
-
-    return DEFAULT_PARAMETER;
-  }
-
-  /**
-   * Parse launch parameters of the form ANY:PARAM_FOR_FIRST:PARAM_FOR_SECOND:...:PARAM_FOR_REST|HBASE_MASTER:...
-   *
-   * E.g. ANY:DO_NOT_REGISTER:DO_NOT_HEARTBEAT:NONE For any container, first one gets DO_NOT_REGISTER second one gets
-   * DO_NOT_HEARTBEAT, then all of the rest get nothing
-   *
-   * E.g. HBASE_MASTER:FAIL_AFTER_START:NONE For HBASE_MASTER, first one gets FAIL_AFTER_START then "" for all
-   *
-   * @param launchParameters
-   *
-   * @return
-   */
-  Map<String, CommandTracker> parseExpectedLaunchParameters(String launchParameters) {
-    Map<String, CommandTracker> trackers = null;
-    if (launchParameters != null && launchParameters.length() > 0) {
-      String[] componentSpecificParameters = launchParameters.split(Pattern.quote("|"));
-      for (String componentSpecificParameter : componentSpecificParameters) {
-        if (componentSpecificParameter.length() != 0) {
-          String[] parameters = componentSpecificParameter.split(Pattern.quote(":"));
-
-          if (parameters.length > 1 && parameters[0].length() > 0) {
-
-            for (int index = 1; index < parameters.length; index++) {
-              if (parameters[index].equals(NONE_VALUE)) {
-                parameters[index] = DEFAULT_PARAMETER;
-              }
-            }
-
-            if (trackers == null) {
-              trackers = new HashMap<String, CommandTracker>(10);
-            }
-            String componentName = parameters[0];
-            CommandTracker tracker = new CommandTracker(Arrays.copyOfRange(parameters, 1, parameters.length));
-            trackers.put(componentName, tracker);
-          }
-        }
-      }
-    }
-
-    return trackers;
-  }
-
-  class CommandTracker {
-    private final int maxIndex;
-    private final String[] launchCommands;
-    private int currentIndex;
-
-    CommandTracker(String[] launchCommands) {
-      this.currentIndex = 0;
-      this.maxIndex = launchCommands.length - 1;
-      this.launchCommands = launchCommands;
-    }
-
-    String getNextCommand() {
-      String retVal = launchCommands[currentIndex];
-      if (currentIndex != maxIndex) {
-        currentIndex++;
-      }
-
-      return retVal;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderFactory.java
deleted file mode 100644
index d5ca749..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderFactory.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.slider.providers.agent;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.slider.providers.AbstractClientProvider;
-import org.apache.slider.providers.ProviderService;
-import org.apache.slider.providers.SliderProviderFactory;
-
-public class AgentProviderFactory extends SliderProviderFactory {
-
-  public static final String CLASSNAME =
-      "org.apache.slider.providers.agent.AgentProviderFactory";
-
-  public AgentProviderFactory() {
-  }
-
-  public AgentProviderFactory(Configuration conf) {
-    super(conf);
-  }
-
-  @Override
-  public AbstractClientProvider createClientProvider() {
-    return new AgentClientProvider(getConf());
-  }
-
-  @Override
-  public ProviderService createServerProvider() {
-    return new AgentProviderService();
-  }
-}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[39/76] [abbrv] hadoop git commit: YARN-5513. Move Java only tests from slider develop to yarn-native-services. Contributed by Gour Saha

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/test/ContractTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/test/ContractTestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/test/ContractTestUtils.java
new file mode 100644
index 0000000..7eaaefe
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/test/ContractTestUtils.java
@@ -0,0 +1,901 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.test;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Assert;
+import org.junit.internal.AssumptionViolatedException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.EOFException;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.Properties;
+import java.util.UUID;
+
+/**
+ * Utilities used across test cases to make assertions about filesystems
+ * -assertions which fail with useful information.
+ * This is lifted from Hadoop common Test; that JAR isn't published, so
+ * we have to make do.
+ */
+public class ContractTestUtils extends Assert {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContractTestUtils.class);
+
+  public static final String IO_FILE_BUFFER_SIZE = "io.file.buffer.size";
+
+  // For scale testing, we can repeatedly write small chunk data to generate
+  // a large file.
+  public static final String IO_CHUNK_BUFFER_SIZE = "io.chunk.buffer.size";
+  public static final int DEFAULT_IO_CHUNK_BUFFER_SIZE = 128;
+  public static final String IO_CHUNK_MODULUS_SIZE = "io.chunk.modulus.size";
+  public static final int DEFAULT_IO_CHUNK_MODULUS_SIZE = 128;
+
+  /**
+   * Assert that a property in the property set matches the expected value
+   * @param props property set
+   * @param key property name
+   * @param expected expected value. If null, the property must not be in the set
+   */
+  public static void assertPropertyEquals(Properties props,
+                                          String key,
+                                          String expected) {
+    String val = props.getProperty(key);
+    if (expected == null) {
+      assertNull("Non null property " + key + " = " + val, val);
+    } else {
+      assertEquals("property " + key + " = " + val,
+                          expected,
+                          val);
+    }
+  }
+
+  /**
+   *
+   * Write a file and read it in, validating the result. Optional flags control
+   * whether file overwrite operations should be enabled, and whether the
+   * file should be deleted afterwards.
+   *
+   * If there is a mismatch between what was written and what was expected,
+   * a small range of bytes either side of the first error are logged to aid
+   * diagnosing what problem occurred -whether it was a previous file
+   * or a corrupting of the current file. This assumes that two
+   * sequential runs to the same path use datasets with different character
+   * moduli.
+   *
+   * @param fs filesystem
+   * @param path path to write to
+   * @param len length of data
+   * @param overwrite should the create option allow overwrites?
+   * @param delete should the file be deleted afterwards? -with a verification
+   * that it worked. Deletion is not attempted if an assertion has failed
+   * earlier -it is not in a <code>finally{}</code> block.
+   * @throws IOException IO problems
+   */
+  public static void writeAndRead(FileSystem fs,
+                                  Path path,
+                                  byte[] src,
+                                  int len,
+                                  int blocksize,
+                                  boolean overwrite,
+                                  boolean delete) throws IOException {
+    fs.mkdirs(path.getParent());
+
+    writeDataset(fs, path, src, len, blocksize, overwrite);
+
+    byte[] dest = readDataset(fs, path, len);
+
+    compareByteArrays(src, dest, len);
+
+    if (delete) {
+      rejectRootOperation(path);
+      boolean deleted = fs.delete(path, false);
+      assertTrue("Deleted", deleted);
+      assertPathDoesNotExist(fs, "Cleanup failed", path);
+    }
+  }
+
+  /**
+   * Write a file.
+   * Optional flags control
+   * whether file overwrite operations should be enabled
+   * @param fs filesystem
+   * @param path path to write to
+   * @param len length of data
+   * @param overwrite should the create option allow overwrites?
+   * @throws IOException IO problems
+   */
+  public static void writeDataset(FileSystem fs,
+                                   Path path,
+                                   byte[] src,
+                                   int len,
+                                   int buffersize,
+                                   boolean overwrite) throws IOException {
+    assertTrue(
+      "Not enough data in source array to write " + len + " bytes",
+      src.length >= len);
+    FSDataOutputStream out = fs.create(path,
+                                       overwrite,
+                                       fs.getConf()
+                                         .getInt(IO_FILE_BUFFER_SIZE,
+                                                 4096),
+                                       (short) 1,
+                                       buffersize);
+    out.write(src, 0, len);
+    out.close();
+    assertFileHasLength(fs, path, len);
+  }
+
+  /**
+   * Read the file and convert to a byte dataset.
+   * This implements readfully internally, so that it will read
+   * in the file without ever having to seek()
+   * @param fs filesystem
+   * @param path path to read from
+   * @param len length of data to read
+   * @return the bytes
+   * @throws IOException IO problems
+   */
+  public static byte[] readDataset(FileSystem fs, Path path, int len)
+      throws IOException {
+    FSDataInputStream in = fs.open(path);
+    byte[] dest = new byte[len];
+    int offset =0;
+    int nread = 0;
+    try {
+      while (nread < len) {
+        int nbytes = in.read(dest, offset + nread, len - nread);
+        if (nbytes < 0) {
+          throw new EOFException("End of file reached before reading fully.");
+        }
+        nread += nbytes;
+      }
+    } finally {
+      in.close();
+    }
+    return dest;
+  }
+
+  /**
+   * Read a file, verify its length and contents match the expected array
+   * @param fs filesystem
+   * @param path path to file
+   * @param original original dataset
+   * @throws IOException IO Problems
+   */
+  public static void verifyFileContents(FileSystem fs,
+                                        Path path,
+                                        byte[] original) throws IOException {
+    FileStatus stat = fs.getFileStatus(path);
+    String statText = stat.toString();
+    assertTrue("not a file " + statText, stat.isFile());
+    assertEquals("wrong length " + statText, original.length, stat.getLen());
+    byte[] bytes = readDataset(fs, path, original.length);
+    compareByteArrays(original,bytes,original.length);
+  }
+
+  /**
+   * Verify that the read at a specific offset in a stream
+   * matches that expected
+   * @param stm stream
+   * @param fileContents original file contents
+   * @param seekOff seek offset
+   * @param toRead number of bytes to read
+   * @throws IOException IO problems
+   */
+  public static void verifyRead(FSDataInputStream stm, byte[] fileContents,
+                                int seekOff, int toRead) throws IOException {
+    byte[] out = new byte[toRead];
+    stm.seek(seekOff);
+    stm.readFully(out);
+    byte[] expected = Arrays.copyOfRange(fileContents, seekOff,
+                                         seekOff + toRead);
+    compareByteArrays(expected, out,toRead);
+  }
+
+  /**
+   * Assert that tthe array original[0..len] and received[] are equal.
+   * A failure triggers the logging of the bytes near where the first
+   * difference surfaces.
+   * @param original source data
+   * @param received actual
+   * @param len length of bytes to compare
+   */
+  public static void compareByteArrays(byte[] original,
+                                       byte[] received,
+                                       int len) {
+    assertEquals("Number of bytes read != number written",
+                        len, received.length);
+    int errors = 0;
+    int first_error_byte = -1;
+    for (int i = 0; i < len; i++) {
+      if (original[i] != received[i]) {
+        if (errors == 0) {
+          first_error_byte = i;
+        }
+        errors++;
+      }
+    }
+
+    if (errors > 0) {
+      String message = String.format(" %d errors in file of length %d",
+                                     errors, len);
+      LOG.warn(message);
+      // the range either side of the first error to print
+      // this is a purely arbitrary number, to aid user debugging
+      final int overlap = 10;
+      for (int i = Math.max(0, first_error_byte - overlap);
+           i < Math.min(first_error_byte + overlap, len);
+           i++) {
+        byte actual = received[i];
+        byte expected = original[i];
+        String letter = toChar(actual);
+        String line = String.format("[%04d] %2x %s\n", i, actual, letter);
+        if (expected != actual) {
+          line = String.format("[%04d] %2x %s -expected %2x %s\n",
+                               i,
+                               actual,
+                               letter,
+                               expected,
+                               toChar(expected));
+        }
+        LOG.warn(line);
+      }
+      fail(message);
+    }
+  }
+
+  /**
+   * Convert a byte to a character for printing. If the
+   * byte value is < 32 -and hence unprintable- the byte is
+   * returned as a two digit hex value
+   * @param b byte
+   * @return the printable character string
+   */
+  public static String toChar(byte b) {
+    if (b >= 0x20) {
+      return Character.toString((char) b);
+    } else {
+      return String.format("%02x", b);
+    }
+  }
+
+  /**
+   * Convert a buffer to a string, character by character
+   * @param buffer input bytes
+   * @return a string conversion
+   */
+  public static String toChar(byte[] buffer) {
+    StringBuilder builder = new StringBuilder(buffer.length);
+    for (byte b : buffer) {
+      builder.append(toChar(b));
+    }
+    return builder.toString();
+  }
+
+  public static byte[] toAsciiByteArray(String s) {
+    char[] chars = s.toCharArray();
+    int len = chars.length;
+    byte[] buffer = new byte[len];
+    for (int i = 0; i < len; i++) {
+      buffer[i] = (byte) (chars[i] & 0xff);
+    }
+    return buffer;
+  }
+
+  /**
+   * Cleanup at the end of a test run
+   * @param action action triggering the operation (for use in logging)
+   * @param fileSystem filesystem to work with. May be null
+   * @param cleanupPath path to delete as a string
+   */
+  public static void cleanup(String action,
+                             FileSystem fileSystem,
+                             String cleanupPath) {
+    if (fileSystem == null) {
+      return;
+    }
+    Path path = new Path(cleanupPath).makeQualified(fileSystem.getUri(),
+        fileSystem.getWorkingDirectory());
+    cleanup(action, fileSystem, path);
+  }
+
+  /**
+   * Cleanup at the end of a test run
+   * @param action action triggering the operation (for use in logging)
+   * @param fileSystem filesystem to work with. May be null
+   * @param path path to delete
+   */
+  public static void cleanup(String action, FileSystem fileSystem, Path path) {
+    noteAction(action);
+    try {
+      rm(fileSystem, path, true, false);
+    } catch (Exception e) {
+      LOG.error("Error deleting in "+ action + " - "  + path + ": " + e, e);
+    }
+  }
+
+  /**
+   * Delete a directory. There's a safety check for operations against the
+   * root directory -these are intercepted and rejected with an IOException
+   * unless the allowRootDelete flag is true
+   * @param fileSystem filesystem to work with. May be null
+   * @param path path to delete
+   * @param recursive flag to enable recursive delete
+   * @param allowRootDelete can the root directory be deleted?
+   * @throws IOException on any problem.
+   */
+  public static boolean rm(FileSystem fileSystem,
+      Path path,
+      boolean recursive,
+      boolean allowRootDelete) throws
+      IOException {
+    if (fileSystem != null) {
+      rejectRootOperation(path, allowRootDelete);
+      if (fileSystem.exists(path)) {
+        return fileSystem.delete(path, recursive);
+      }
+    }
+    return false;
+
+  }
+
+  /**
+   * Block any operation on the root path. This is a safety check
+   * @param path path in the filesystem
+   * @param allowRootOperation can the root directory be manipulated?
+   * @throws IOException if the operation was rejected
+   */
+  public static void rejectRootOperation(Path path,
+      boolean allowRootOperation) throws IOException {
+    if (path.isRoot() && !allowRootOperation) {
+      throw new IOException("Root directory operation rejected: " + path);
+    }
+  }
+
+  /**
+   * Block any operation on the root path. This is a safety check
+   * @param path path in the filesystem
+   * @throws IOException if the operation was rejected
+   */
+  public static void rejectRootOperation(Path path) throws IOException {
+    rejectRootOperation(path, false);
+  }
+
+
+  public static void noteAction(String action) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("==============  "+ action +" =============");
+    }
+  }
+
+  /**
+   * downgrade a failure to a message and a warning, then an
+   * exception for the Junit test runner to mark as failed
+   * @param message text message
+   * @param failure what failed
+   * @throws AssumptionViolatedException always
+   */
+  public static void downgrade(String message, Throwable failure) {
+    LOG.warn("Downgrading test " + message, failure);
+    AssumptionViolatedException ave =
+      new AssumptionViolatedException(failure, null);
+    throw ave;
+  }
+
+  /**
+   * report an overridden test as unsupported
+   * @param message message to use in the text
+   * @throws AssumptionViolatedException always
+   */
+  public static void unsupported(String message) {
+    skip(message);
+  }
+
+  /**
+   * report a test has been skipped for some reason
+   * @param message message to use in the text
+   * @throws AssumptionViolatedException always
+   */
+  public static void skip(String message) {
+    LOG.info("Skipping: {}", message);
+    throw new AssumptionViolatedException(message);
+  }
+
+  /**
+   * Fail with an exception that was received
+   * @param text text to use in the exception
+   * @param thrown a (possibly null) throwable to init the cause with
+   * @throws AssertionError with the text and throwable -always
+   */
+  public static void fail(String text, Throwable thrown) {
+    AssertionError e = new AssertionError(text);
+    e.initCause(thrown);
+    throw e;
+  }
+
+  /**
+   * Make an assertion about the length of a file
+   * @param fs filesystem
+   * @param path path of the file
+   * @param expected expected length
+   * @throws IOException on File IO problems
+   */
+  public static void assertFileHasLength(FileSystem fs, Path path,
+                                         int expected) throws IOException {
+    FileStatus status = fs.getFileStatus(path);
+    assertEquals(
+      "Wrong file length of file " + path + " status: " + status,
+      expected,
+      status.getLen());
+  }
+
+  /**
+   * Assert that a path refers to a directory
+   * @param fs filesystem
+   * @param path path of the directory
+   * @throws IOException on File IO problems
+   */
+  public static void assertIsDirectory(FileSystem fs,
+                                       Path path) throws IOException {
+    FileStatus fileStatus = fs.getFileStatus(path);
+    assertIsDirectory(fileStatus);
+  }
+
+  /**
+   * Assert that a path refers to a directory
+   * @param fileStatus stats to check
+   */
+  public static void assertIsDirectory(FileStatus fileStatus) {
+    assertTrue("Should be a directory -but isn't: " + fileStatus,
+               fileStatus.isDirectory());
+  }
+
+  /**
+   * Write the text to a file, returning the converted byte array
+   * for use in validating the round trip
+   * @param fs filesystem
+   * @param path path of file
+   * @param text text to write
+   * @param overwrite should the operation overwrite any existing file?
+   * @return the read bytes
+   * @throws IOException on IO problems
+   */
+  public static byte[] writeTextFile(FileSystem fs,
+                                   Path path,
+                                   String text,
+                                   boolean overwrite) throws IOException {
+    byte[] bytes = new byte[0];
+    if (text != null) {
+      bytes = toAsciiByteArray(text);
+    }
+    createFile(fs, path, overwrite, bytes);
+    return bytes;
+  }
+
+  /**
+   * Create a file
+   * @param fs filesystem
+   * @param path       path to write
+   * @param overwrite overwrite flag
+   * @param data source dataset. Can be null
+   * @throws IOException on any problem
+   */
+  public static void createFile(FileSystem fs,
+                                 Path path,
+                                 boolean overwrite,
+                                 byte[] data) throws IOException {
+    FSDataOutputStream stream = fs.create(path, overwrite);
+    if (data != null && data.length > 0) {
+      stream.write(data);
+    }
+    stream.close();
+  }
+
+  /**
+   * Touch a file
+   * @param fs filesystem
+   * @param path path
+   * @throws IOException IO problems
+   */
+  public static void touch(FileSystem fs,
+                           Path path) throws IOException {
+    createFile(fs, path, true, null);
+  }
+
+  /**
+   * Delete a file/dir and assert that delete() returned true
+   * <i>and</i> that the path no longer exists. This variant rejects
+   * all operations on root directories
+   * @param fs filesystem
+   * @param file path to delete
+   * @param recursive flag to enable recursive delete
+   * @throws IOException IO problems
+   */
+  public static void assertDeleted(FileSystem fs,
+                                   Path file,
+                                   boolean recursive) throws IOException {
+    assertDeleted(fs, file, recursive, false);
+  }
+
+  /**
+   * Delete a file/dir and assert that delete() returned true
+   * <i>and</i> that the path no longer exists. This variant rejects
+   * all operations on root directories
+   * @param fs filesystem
+   * @param file path to delete
+   * @param recursive flag to enable recursive delete
+   * @param allowRootOperations can the root dir be deleted?
+   * @throws IOException IO problems
+   */
+  public static void assertDeleted(FileSystem fs,
+      Path file,
+      boolean recursive,
+      boolean allowRootOperations) throws IOException {
+    rejectRootOperation(file, allowRootOperations);
+    assertPathExists(fs, "about to be deleted file", file);
+    boolean deleted = fs.delete(file, recursive);
+    String dir = ls(fs, file.getParent());
+    assertTrue("Delete failed on " + file + ": " + dir, deleted);
+    assertPathDoesNotExist(fs, "Deleted file", file);
+  }
+
+  /**
+   * Read in "length" bytes, convert to an ascii string
+   * @param fs filesystem
+   * @param path path to read
+   * @param length #of bytes to read.
+   * @return the bytes read and converted to a string
+   * @throws IOException IO problems
+   */
+  public static String readBytesToString(FileSystem fs,
+                                  Path path,
+                                  int length) throws IOException {
+    FSDataInputStream in = fs.open(path);
+    try {
+      byte[] buf = new byte[length];
+      in.readFully(0, buf);
+      return toChar(buf);
+    } finally {
+      in.close();
+    }
+  }
+
+  /**
+   * Take an array of filestats and convert to a string (prefixed w/ a [01] counter
+   * @param stats array of stats
+   * @param separator separator after every entry
+   * @return a stringified set
+   */
+  public static String fileStatsToString(FileStatus[] stats, String separator) {
+    StringBuilder buf = new StringBuilder(stats.length * 128);
+    for (int i = 0; i < stats.length; i++) {
+      buf.append(String.format("[%02d] %s", i, stats[i])).append(separator);
+    }
+    return buf.toString();
+  }
+
+  /**
+   * List a directory
+   * @param fileSystem FS
+   * @param path path
+   * @return a directory listing or failure message
+   * @throws IOException
+   */
+  public static String ls(FileSystem fileSystem, Path path) throws IOException {
+    if (path == null) {
+      //surfaces when someone calls getParent() on something at the top of the path
+      return "/";
+    }
+    FileStatus[] stats;
+    String pathtext = "ls " + path;
+    try {
+      stats = fileSystem.listStatus(path);
+    } catch (FileNotFoundException e) {
+      return pathtext + " -file not found";
+    } catch (IOException e) {
+      return pathtext + " -failed: " + e;
+    }
+    return dumpStats(pathtext, stats);
+  }
+
+  public static String dumpStats(String pathname, FileStatus[] stats) {
+    return pathname + fileStatsToString(stats, "\n");
+  }
+
+   /**
+   * Assert that a file exists and whose {@link FileStatus} entry
+   * declares that this is a file and not a symlink or directory.
+   * @param fileSystem filesystem to resolve path against
+   * @param filename name of the file
+   * @throws IOException IO problems during file operations
+   */
+  public static void assertIsFile(FileSystem fileSystem, Path filename) throws
+                                                                 IOException {
+    assertPathExists(fileSystem, "Expected file", filename);
+    FileStatus status = fileSystem.getFileStatus(filename);
+    assertIsFile(filename, status);
+  }
+
+  /**
+   * Assert that a file exists and whose {@link FileStatus} entry
+   * declares that this is a file and not a symlink or directory.
+   * @param filename name of the file
+   * @param status file status
+   */
+  public static void assertIsFile(Path filename, FileStatus status) {
+    String fileInfo = filename + "  " + status;
+    assertFalse("File claims to be a directory " + fileInfo,
+                status.isDirectory());
+    assertFalse("File claims to be a symlink " + fileInfo,
+                       status.isSymlink());
+  }
+
+  /**
+   * Create a dataset for use in the tests; all data is in the range
+   * base to (base+modulo-1) inclusive
+   * @param len length of data
+   * @param base base of the data
+   * @param modulo the modulo
+   * @return the newly generated dataset
+   */
+  public static byte[] dataset(int len, int base, int modulo) {
+    byte[] dataset = new byte[len];
+    for (int i = 0; i < len; i++) {
+      dataset[i] = (byte) (base + (i % modulo));
+    }
+    return dataset;
+  }
+
+  /**
+   * Assert that a path exists -but make no assertions as to the
+   * type of that entry
+   *
+   * @param fileSystem filesystem to examine
+   * @param message message to include in the assertion failure message
+   * @param path path in the filesystem
+   * @throws FileNotFoundException raised if the path is missing
+   * @throws IOException IO problems
+   */
+  public static void assertPathExists(FileSystem fileSystem, String message,
+                               Path path) throws IOException {
+    if (!fileSystem.exists(path)) {
+      //failure, report it
+      String listing = ls(fileSystem, path.getParent());
+      throw new FileNotFoundException(message + ": not found " + path
+        + " in \"" + path.getParent() + "\" :\n" + listing);
+    }
+  }
+
+  /**
+   * Assert that a path does not exist
+   *
+   * @param fileSystem filesystem to examine
+   * @param message message to include in the assertion failure message
+   * @param path path in the filesystem
+   * @throws IOException IO problems
+   */
+  public static void assertPathDoesNotExist(FileSystem fileSystem,
+                                            String message,
+                                            Path path) throws IOException {
+    try {
+      FileStatus status = fileSystem.getFileStatus(path);
+      fail(message + ": unexpectedly found " + path + " as  " + status);
+    } catch (FileNotFoundException expected) {
+      //this is expected
+
+    }
+  }
+
+  /**
+   * Assert that a FileSystem.listStatus on a dir finds the subdir/child entry
+   * @param fs filesystem
+   * @param dir directory to scan
+   * @param subdir full path to look for
+   * @throws IOException IO probles
+   */
+  public static void assertListStatusFinds(FileSystem fs,
+                                           Path dir,
+                                           Path subdir) throws IOException {
+    FileStatus[] stats = fs.listStatus(dir);
+    boolean found = false;
+    StringBuilder builder = new StringBuilder();
+    for (FileStatus stat : stats) {
+      builder.append(stat.toString()).append('\n');
+      if (stat.getPath().equals(subdir)) {
+        found = true;
+      }
+    }
+    assertTrue("Path " + subdir
+                      + " not found in directory " + dir + ":" + builder,
+                      found);
+  }
+
+  /**
+   * Test for the host being an OSX machine
+   * @return true if the JVM thinks that is running on OSX
+   */
+  public static boolean isOSX() {
+    return System.getProperty("os.name").contains("OS X");
+  }
+
+  /**
+   * compare content of file operations using a double byte array
+   * @param concat concatenated files
+   * @param bytes bytes
+   */
+  public static void validateFileContent(byte[] concat, byte[][] bytes) {
+    int idx = 0;
+    boolean mismatch = false;
+
+    for (byte[] bb : bytes) {
+      for (byte b : bb) {
+        if (b != concat[idx++]) {
+          mismatch = true;
+          break;
+        }
+      }
+      if (mismatch)
+        break;
+    }
+    assertFalse("File content of file is not as expected at offset " + idx,
+                mismatch);
+  }
+
+  /**
+   * Receives test data from the given input file and checks the size of the
+   * data as well as the pattern inside the received data.
+   *
+   * @param fs FileSystem
+   * @param path Input file to be checked
+   * @param expectedSize the expected size of the data to be read from the
+   *        input file in bytes
+   * @param bufferLen Pattern length
+   * @param modulus   Pattern modulus
+   * @throws IOException
+   *         thrown if an error occurs while reading the data
+   */
+  public static void verifyReceivedData(FileSystem fs, Path path,
+                                      final long expectedSize,
+                                      final int bufferLen,
+                                      final int modulus) throws IOException {
+    final byte[] testBuffer = new byte[bufferLen];
+
+    long totalBytesRead = 0;
+    int nextExpectedNumber = 0;
+    final InputStream inputStream = fs.open(path);
+    try {
+      while (true) {
+        final int bytesRead = inputStream.read(testBuffer);
+        if (bytesRead < 0) {
+          break;
+        }
+
+        totalBytesRead += bytesRead;
+
+        for (int i = 0; i < bytesRead; ++i) {
+          if (testBuffer[i] != nextExpectedNumber) {
+            throw new IOException("Read number " + testBuffer[i]
+                + " but expected " + nextExpectedNumber);
+          }
+
+          ++nextExpectedNumber;
+
+          if (nextExpectedNumber == modulus) {
+            nextExpectedNumber = 0;
+          }
+        }
+      }
+
+      if (totalBytesRead != expectedSize) {
+        throw new IOException("Expected to read " + expectedSize +
+            " bytes but only received " + totalBytesRead);
+      }
+    } finally {
+      inputStream.close();
+    }
+  }
+
+  /**
+   * Generates test data of the given size according to some specific pattern
+   * and writes it to the provided output file.
+   *
+   * @param fs FileSystem
+   * @param path Test file to be generated
+   * @param size The size of the test data to be generated in bytes
+   * @param bufferLen Pattern length
+   * @param modulus   Pattern modulus
+   * @throws IOException
+   *         thrown if an error occurs while writing the data
+   */
+  public static long generateTestFile(FileSystem fs, Path path,
+                                      final long size,
+                                      final int bufferLen,
+                                      final int modulus) throws IOException {
+    final byte[] testBuffer = new byte[bufferLen];
+    for (int i = 0; i < testBuffer.length; ++i) {
+      testBuffer[i] = (byte) (i % modulus);
+    }
+
+    final OutputStream outputStream = fs.create(path, false);
+    long bytesWritten = 0;
+    try {
+      while (bytesWritten < size) {
+        final long diff = size - bytesWritten;
+        if (diff < testBuffer.length) {
+          outputStream.write(testBuffer, 0, (int) diff);
+          bytesWritten += diff;
+        } else {
+          outputStream.write(testBuffer);
+          bytesWritten += testBuffer.length;
+        }
+      }
+
+      return bytesWritten;
+    } finally {
+      outputStream.close();
+    }
+  }
+
+  /**
+   * Creates and reads a file with the given size. The test file is generated
+   * according to a specific pattern so it can be easily verified even if it's
+   * a multi-GB one.
+   * During the read phase the incoming data stream is also checked against
+   * this pattern.
+   *
+   * @param fs FileSystem
+   * @param parent Test file parent dir path
+   * @throws IOException
+   *    thrown if an I/O error occurs while writing or reading the test file
+   */
+  public static void createAndVerifyFile(FileSystem fs, Path parent, final long fileSize)
+      throws IOException {
+    int testBufferSize = fs.getConf()
+        .getInt(IO_CHUNK_BUFFER_SIZE, DEFAULT_IO_CHUNK_BUFFER_SIZE);
+    int modulus = fs.getConf()
+        .getInt(IO_CHUNK_MODULUS_SIZE, DEFAULT_IO_CHUNK_MODULUS_SIZE);
+
+    final String objectName = UUID.randomUUID().toString();
+    final Path objectPath = new Path(parent, objectName);
+
+    // Write test file in a specific pattern
+    assertEquals(fileSize,
+        generateTestFile(fs, objectPath, fileSize, testBufferSize, modulus));
+    assertPathExists(fs, "not created successful", objectPath);
+
+    // Now read the same file back and verify its content
+    try {
+      verifyReceivedData(fs, objectPath, fileSize, testBufferSize, modulus);
+    } finally {
+      // Delete test file
+      fs.delete(objectPath, false);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/test/MiniZooKeeperCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/test/MiniZooKeeperCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/test/MiniZooKeeperCluster.java
new file mode 100644
index 0000000..d739324
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/test/MiniZooKeeperCluster.java
@@ -0,0 +1,395 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.zookeeper.server.NIOServerCnxnFactory;
+import org.apache.zookeeper.server.ZooKeeperServer;
+import org.apache.zookeeper.server.persistence.FileTxnLog;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.Reader;
+import java.net.BindException;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+
+/**
+ * This is a version of the HBase ZK cluster cut out to be standalone
+ */
+public class MiniZooKeeperCluster {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      MiniZooKeeperCluster.class);
+
+  private static final int TICK_TIME = 2000;
+  private static final int CONNECTION_TIMEOUT = 30000;
+  public static final int MAX_CLIENT_CONNECTIONS = 1000;
+
+  private boolean started;
+
+  /** The default port. If zero, we use a random port. */
+  private int defaultClientPort = 0;
+
+  private int clientPort;
+
+  private List<NIOServerCnxnFactory> standaloneServerFactoryList;
+  private List<ZooKeeperServer> zooKeeperServers;
+  private List<Integer> clientPortList;
+
+  private int activeZKServerIndex;
+  private int tickTime = 0;
+
+  private Configuration configuration;
+
+  public MiniZooKeeperCluster() {
+    this(new Configuration());
+  }
+
+  public MiniZooKeeperCluster(Configuration configuration) {
+    this.started = false;
+    this.configuration = configuration;
+    activeZKServerIndex = -1;
+    zooKeeperServers = new ArrayList<ZooKeeperServer>();
+    clientPortList = new ArrayList<Integer>();
+    standaloneServerFactoryList = new ArrayList<NIOServerCnxnFactory>();
+  }
+
+  public void setDefaultClientPort(int clientPort) {
+    if (clientPort <= 0) {
+      throw new IllegalArgumentException("Invalid default ZK client port: "
+                                         + clientPort);
+    }
+    this.defaultClientPort = clientPort;
+  }
+
+  /**
+   * Selects a ZK client port. Returns the default port if specified.
+   * Otherwise, returns a random port. The random port is selected from the
+   * range between 49152 to 65535. These ports cannot be registered with IANA
+   * and are intended for dynamic allocation (see http://bit.ly/dynports).
+   */
+  private int selectClientPort() {
+    if (defaultClientPort > 0) {
+      return defaultClientPort;
+    }
+    return 0xc000 + new Random().nextInt(0x3f00);
+  }
+
+  public void setTickTime(int tickTime) {
+    this.tickTime = tickTime;
+  }
+
+  public int getBackupZooKeeperServerNum() {
+    return zooKeeperServers.size() - 1;
+  }
+
+  public int getZooKeeperServerNum() {
+    return zooKeeperServers.size();
+  }
+
+  // / XXX: From o.a.zk.t.ClientBase
+  private static void setupTestEnv() {
+    // during the tests we run with 100K prealloc in the logs.
+    // on windows systems prealloc of 64M was seen to take ~15seconds
+    // resulting in test failure (client timeout on first session).
+    // set env and directly in order to handle static init/gc issues
+    System.setProperty("zookeeper.preAllocSize", "100");
+    FileTxnLog.setPreallocSize(100 * 1024);
+  }
+
+  public int startup(File baseDir) throws IOException, InterruptedException {
+    return startup(baseDir, 1);
+  }
+
+  /**
+   * @param baseDir
+   * @param numZooKeeperServers
+   * @return ClientPort server bound to, -1 if there was a
+   *         binding problem and we couldn't pick another port.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public int startup(File baseDir, int numZooKeeperServers) throws IOException,
+      InterruptedException {
+    if (numZooKeeperServers <= 0)
+      return -1;
+
+    setupTestEnv();
+    shutdown();
+
+    int tentativePort = selectClientPort();
+
+    // running all the ZK servers
+    for (int i = 0; i < numZooKeeperServers; i++) {
+      File dir = new File(baseDir, "zookeeper_" + i).getAbsoluteFile();
+      recreateDir(dir);
+      int tickTimeToUse;
+      if (this.tickTime > 0) {
+        tickTimeToUse = this.tickTime;
+      } else {
+        tickTimeToUse = TICK_TIME;
+      }
+      ZooKeeperServer server = new ZooKeeperServer(dir, dir, tickTimeToUse);
+      NIOServerCnxnFactory standaloneServerFactory;
+      while (true) {
+        try {
+          standaloneServerFactory = new NIOServerCnxnFactory();
+          standaloneServerFactory.configure(
+              new InetSocketAddress(tentativePort),
+              MAX_CLIENT_CONNECTIONS
+          );
+        } catch (BindException e) {
+          LOG.debug("Failed binding ZK Server to client port: " +
+                    tentativePort, e);
+          // We're told to use some port but it's occupied, fail
+          if (defaultClientPort > 0) return -1;
+          // This port is already in use, try to use another.
+          tentativePort = selectClientPort();
+          continue;
+        }
+        break;
+      }
+
+      // Start up this ZK server
+      standaloneServerFactory.startup(server);
+      if (!waitForServerUp(tentativePort, CONNECTION_TIMEOUT)) {
+        throw new IOException("Waiting for startup of standalone server");
+      }
+
+      // We have selected this port as a client port.
+      clientPortList.add(tentativePort);
+      standaloneServerFactoryList.add(standaloneServerFactory);
+      zooKeeperServers.add(server);
+      tentativePort++; //for the next server
+    }
+
+    // set the first one to be active ZK; Others are backups
+    activeZKServerIndex = 0;
+    started = true;
+    clientPort = clientPortList.get(activeZKServerIndex);
+    LOG.info("Started MiniZK Cluster and connect 1 ZK server " +
+             "on client port: " + clientPort);
+    return clientPort;
+  }
+
+  private void recreateDir(File dir) throws IOException {
+    if (dir.exists()) {
+      if (!FileUtil.fullyDelete(dir)) {
+        throw new IOException("Could not delete zk base directory: " + dir);
+      }
+    }
+    try {
+      dir.mkdirs();
+    } catch (SecurityException e) {
+      throw new IOException("creating dir: " + dir, e);
+    }
+  }
+
+  /**
+   * @throws IOException
+   */
+  public void shutdown() throws IOException {
+    if (!started) {
+      return;
+    }
+
+    // shut down all the zk servers
+    for (int i = 0; i < standaloneServerFactoryList.size(); i++) {
+      NIOServerCnxnFactory standaloneServerFactory =
+          standaloneServerFactoryList.get(i);
+      int clientPort = clientPortList.get(i);
+
+      standaloneServerFactory.shutdown();
+      if (!waitForServerDown(clientPort, CONNECTION_TIMEOUT)) {
+        throw new IOException("Waiting for shutdown of standalone server");
+      }
+    }
+    for (ZooKeeperServer zkServer : zooKeeperServers) {
+      //explicitly close ZKDatabase since ZookeeperServer does not close them
+      zkServer.getZKDatabase().close();
+    }
+
+    // clear everything
+    started = false;
+    activeZKServerIndex = 0;
+    standaloneServerFactoryList.clear();
+    clientPortList.clear();
+    zooKeeperServers.clear();
+
+    LOG.info("Shutdown MiniZK cluster with all ZK servers");
+  }
+
+  /**@return clientPort return clientPort if there is another ZK backup can run
+   *         when killing the current active; return -1, if there is no backups.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public int killCurrentActiveZooKeeperServer() throws IOException,
+      InterruptedException {
+    if (!started || activeZKServerIndex < 0) {
+      return -1;
+    }
+
+    // Shutdown the current active one
+    NIOServerCnxnFactory standaloneServerFactory =
+        standaloneServerFactoryList.get(activeZKServerIndex);
+    int clientPort = clientPortList.get(activeZKServerIndex);
+
+    standaloneServerFactory.shutdown();
+    if (!waitForServerDown(clientPort, CONNECTION_TIMEOUT)) {
+      throw new IOException("Waiting for shutdown of standalone server");
+    }
+
+    zooKeeperServers.get(activeZKServerIndex).getZKDatabase().close();
+
+    // remove the current active zk server
+    standaloneServerFactoryList.remove(activeZKServerIndex);
+    clientPortList.remove(activeZKServerIndex);
+    zooKeeperServers.remove(activeZKServerIndex);
+    LOG.info("Kill the current active ZK servers in the cluster " +
+             "on client port: " + clientPort);
+
+    if (standaloneServerFactoryList.size() == 0) {
+      // there is no backup servers;
+      return -1;
+    }
+    clientPort = clientPortList.get(activeZKServerIndex);
+    LOG.info("Activate a backup zk server in the cluster " +
+             "on client port: " + clientPort);
+    // return the next back zk server's port
+    return clientPort;
+  }
+
+  /**
+   * Kill one back up ZK servers
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public void killOneBackupZooKeeperServer() throws IOException,
+      InterruptedException {
+    if (!started || activeZKServerIndex < 0 ||
+        standaloneServerFactoryList.size() <= 1) {
+      return;
+    }
+
+    int backupZKServerIndex = activeZKServerIndex + 1;
+    // Shutdown the current active one
+    NIOServerCnxnFactory standaloneServerFactory =
+        standaloneServerFactoryList.get(backupZKServerIndex);
+    int clientPort = clientPortList.get(backupZKServerIndex);
+
+    standaloneServerFactory.shutdown();
+    if (!waitForServerDown(clientPort, CONNECTION_TIMEOUT)) {
+      throw new IOException("Waiting for shutdown of standalone server");
+    }
+
+    zooKeeperServers.get(backupZKServerIndex).getZKDatabase().close();
+
+    // remove this backup zk server
+    standaloneServerFactoryList.remove(backupZKServerIndex);
+    clientPortList.remove(backupZKServerIndex);
+    zooKeeperServers.remove(backupZKServerIndex);
+    LOG.info("Kill one backup ZK servers in the cluster " +
+             "on client port: " + clientPort);
+  }
+
+  // XXX: From o.a.zk.t.ClientBase
+  private static boolean waitForServerDown(int port, long timeout) {
+    long start = System.currentTimeMillis();
+    while (true) {
+      try {
+        Socket sock = new Socket("localhost", port);
+        try {
+          OutputStream outstream = sock.getOutputStream();
+          outstream.write("stat".getBytes());
+          outstream.flush();
+        } finally {
+          sock.close();
+        }
+      } catch (IOException e) {
+        return true;
+      }
+
+      if (System.currentTimeMillis() > start + timeout) {
+        break;
+      }
+      try {
+        Thread.sleep(250);
+      } catch (InterruptedException e) {
+        // ignore
+      }
+    }
+    return false;
+  }
+
+  // XXX: From o.a.zk.t.ClientBase
+  private static boolean waitForServerUp(int port, long timeout) {
+    long start = System.currentTimeMillis();
+    while (true) {
+      try {
+        Socket sock = new Socket("localhost", port);
+        BufferedReader reader = null;
+        try {
+          OutputStream outstream = sock.getOutputStream();
+          outstream.write("stat".getBytes());
+          outstream.flush();
+
+          Reader isr = new InputStreamReader(sock.getInputStream());
+          reader = new BufferedReader(isr);
+          String line = reader.readLine();
+          if (line != null && line.startsWith("Zookeeper version:")) {
+            return true;
+          }
+        } finally {
+          sock.close();
+          if (reader != null) {
+            reader.close();
+          }
+        }
+      } catch (IOException e) {
+        // ignore as this is expected
+        LOG.info("server localhost:" + port + " not up " + e);
+      }
+
+      if (System.currentTimeMillis() > start + timeout) {
+        break;
+      }
+      try {
+        Thread.sleep(250);
+      } catch (InterruptedException e) {
+        // ignore
+      }
+    }
+    return false;
+  }
+
+  public int getClientPort() {
+    return clientPort;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/tools/TestUtility.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/tools/TestUtility.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/tools/TestUtility.java
new file mode 100644
index 0000000..78ce3e7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/java/org/apache/slider/tools/TestUtility.java
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.tools;
+
+import org.apache.commons.compress.archivers.zip.ZipArchiveEntry;
+import org.apache.commons.compress.archivers.zip.ZipArchiveOutputStream;
+import org.apache.commons.compress.utils.IOUtils;
+import org.junit.Assert;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+/**
+ *  Various utility methods
+ *  Byte comparison methods are from
+ *  <code>org.apache.hadoop.fs.contract.ContractTestUtils</code>
+ */
+public class TestUtility {
+  protected static final Logger log =
+      LoggerFactory.getLogger(TestUtility.class);
+
+  public static void addDir(File dirObj, ZipArchiveOutputStream zipFile, String prefix) throws IOException {
+    for (File file : dirObj.listFiles()) {
+      if (file.isDirectory()) {
+        addDir(file, zipFile, prefix + file.getName() + File.separator);
+      } else {
+        log.info("Adding to zip - " + prefix + file.getName());
+        zipFile.putArchiveEntry(new ZipArchiveEntry(prefix + file.getName()));
+        IOUtils.copy(new FileInputStream(file), zipFile);
+        zipFile.closeArchiveEntry();
+      }
+    }
+  }
+
+  public static void zipDir(String zipFile, String dir) throws IOException {
+    File dirObj = new File(dir);
+    ZipArchiveOutputStream out = new ZipArchiveOutputStream(new FileOutputStream(zipFile));
+    log.info("Creating : {}", zipFile);
+    try {
+      addDir(dirObj, out, "");
+    } finally {
+      out.close();
+    }
+  }
+
+  public static String createAppPackage(
+      TemporaryFolder folder, String subDir, String pkgName, String srcPath) throws IOException {
+    String zipFileName;
+    File pkgPath = folder.newFolder(subDir);
+    File zipFile = new File(pkgPath, pkgName).getAbsoluteFile();
+    zipFileName = zipFile.getAbsolutePath();
+    TestUtility.zipDir(zipFileName, srcPath);
+    log.info("Created temporary zip file at {}", zipFileName);
+    return zipFileName;
+  }
+
+
+  /**
+   * Assert that tthe array original[0..len] and received[] are equal.
+   * A failure triggers the logging of the bytes near where the first
+   * difference surfaces.
+   * @param original source data
+   * @param received actual
+   * @param len length of bytes to compare
+   */
+  public static void compareByteArrays(byte[] original,
+      byte[] received,
+      int len) {
+    Assert.assertEquals("Number of bytes read != number written",
+        len, received.length);
+    int errors = 0;
+    int first_error_byte = -1;
+    for (int i = 0; i < len; i++) {
+      if (original[i] != received[i]) {
+        if (errors == 0) {
+          first_error_byte = i;
+        }
+        errors++;
+      }
+    }
+
+    if (errors > 0) {
+      String message = String.format(" %d errors in file of length %d",
+          errors, len);
+      log.warn(message);
+      // the range either side of the first error to print
+      // this is a purely arbitrary number, to aid user debugging
+      final int overlap = 10;
+      for (int i = Math.max(0, first_error_byte - overlap);
+           i < Math.min(first_error_byte + overlap, len);
+           i++) {
+        byte actual = received[i];
+        byte expected = original[i];
+        String letter = toChar(actual);
+        String line = String.format("[%04d] %2x %s\n", i, actual, letter);
+        if (expected != actual) {
+          line = String.format("[%04d] %2x %s -expected %2x %s\n",
+              i,
+              actual,
+              letter,
+              expected,
+              toChar(expected));
+        }
+        log.warn(line);
+      }
+      Assert.fail(message);
+    }
+  }
+  /**
+   * Convert a byte to a character for printing. If the
+   * byte value is < 32 -and hence unprintable- the byte is
+   * returned as a two digit hex value
+   * @param b byte
+   * @return the printable character string
+   */
+  public static String toChar(byte b) {
+    if (b >= 0x20) {
+      return Character.toString((char) b);
+    } else {
+      return String.format("%02x", b);
+    }
+  }
+
+  /**
+   * Convert a buffer to a string, character by character
+   * @param buffer input bytes
+   * @return a string conversion
+   */
+  public static String toChar(byte[] buffer) {
+    StringBuilder builder = new StringBuilder(buffer.length);
+    for (byte b : buffer) {
+      builder.append(toChar(b));
+    }
+    return builder.toString();
+  }
+
+  public static byte[] toAsciiByteArray(String s) {
+    char[] chars = s.toCharArray();
+    int len = chars.length;
+    byte[] buffer = new byte[len];
+    for (int i = 0; i < len; i++) {
+      buffer[i] = (byte) (chars[i] & 0xff);
+    }
+    return buffer;
+  }
+
+  /**
+   * Create a dataset for use in the tests; all data is in the range
+   * base to (base+modulo-1) inclusive
+   * @param len length of data
+   * @param base base of the data
+   * @param modulo the modulo
+   * @return the newly generated dataset
+   */
+  public static byte[] dataset(int len, int base, int modulo) {
+    byte[] dataset = new byte[len];
+    for (int i = 0; i < len; i++) {
+      dataset[i] = (byte) (base + (i % modulo));
+    }
+    return dataset;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/metainfo.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/metainfo.txt b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/metainfo.txt
new file mode 100644
index 0000000..a1d7780
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/metainfo.txt
@@ -0,0 +1,16 @@
+<!--
+   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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/metainfo.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/metainfo.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/metainfo.xml
new file mode 100644
index 0000000..cb8eab2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/metainfo.xml
@@ -0,0 +1,98 @@
+<?xml version="1.0"?>
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<metainfo>
+  <schemaVersion>2.0</schemaVersion>
+  <application>
+    <name>STORM</name>
+    <comment>Apache Hadoop Stream processing framework</comment>
+    <version>0.9.1.2.1</version>
+    <components>
+
+      <component>
+        <name>NIMBUS</name>
+        <category>MASTER</category>
+        <commandScript>
+          <script>scripts/nimbus.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+
+      <component>
+        <name>STORM_REST_API</name>
+        <category>MASTER</category>
+        <commandScript>
+          <script>scripts/rest_api.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+
+      <component>
+        <name>SUPERVISOR</name>
+        <category>SLAVE</category>
+        <commandScript>
+          <script>scripts/supervisor.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+
+      <component>
+        <name>STORM_UI_SERVER</name>
+        <category>MASTER</category>
+        <commandScript>
+          <script>scripts/ui_server.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+
+      <component>
+        <name>DRPC_SERVER</name>
+        <category>MASTER</category>
+        <commandScript>
+          <script>scripts/drpc_server.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+    </components>
+
+    <osSpecifics>
+      <osSpecific>
+        <osType>any</osType>
+        <packages>
+          <package>
+            <type>tarball</type>
+            <name>files/apache-storm-0.9.1.2.1.1.0-237.tar.gz</name>
+          </package>
+        </packages>
+      </osSpecific>
+    </osSpecifics>
+
+    <configFiles>
+      <configFile>
+        <type>xml</type>
+        <fileName>storm-site.xml</fileName>
+        <dictionaryName>storm-site</dictionaryName>
+      </configFile>
+    </configFiles>
+  </application>
+</metainfo>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/someOtherFile.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/someOtherFile.txt b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/someOtherFile.txt
new file mode 100644
index 0000000..a1d7780
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/someOtherFile.txt
@@ -0,0 +1,16 @@
+<!--
+   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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/someOtherFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/someOtherFile.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/someOtherFile.xml
new file mode 100644
index 0000000..f86e687
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/common/tools/test/someOtherFile.xml
@@ -0,0 +1,17 @@
+<!--
+   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.
+-->
+<metainfo></metainfo>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4beaeb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/providers/agent/application/metadata/metainfo.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/providers/agent/application/metadata/metainfo.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/providers/agent/application/metadata/metainfo.xml
new file mode 100644
index 0000000..fbe9299
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/test/resources/org/apache/slider/providers/agent/application/metadata/metainfo.xml
@@ -0,0 +1,180 @@
+<?xml version="1.0"?>
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<metainfo>
+  <schemaVersion>2.0</schemaVersion>
+  <application>
+    <name>STORM</name>
+    <comment>Apache Hadoop Stream processing framework</comment>
+    <version>0.9.1.2.1</version>
+    <exportedConfigs>storm-site</exportedConfigs>
+
+    <exportGroups>
+      <exportGroup>
+        <name>QuickLinks</name>
+        <exports>
+          <export>
+            <name>app.jmx</name>
+            <value>http://${STORM_REST_API_HOST}:${site.global.rest_api_port}/api/cluster/summary</value>
+          </export>
+          <export>
+            <name>app.monitor</name>
+            <value>http://${STORM_UI_SERVER_HOST}:${site.storm-site.ui.port}</value>
+          </export>
+          <export>
+            <name>app.metrics</name>
+            <value>http://${site.global.ganglia_server_host}/cgi-bin/rrd.py?c=${site.global.ganglia_server_id}</value>
+          </export>
+          <export>
+            <name>ganglia.ui</name>
+            <value>http://${site.global.ganglia_server_host}/ganglia?c=${site.global.ganglia_server_id}</value>
+          </export>
+          <export>
+            <name>nimbus.url</name>
+            <value>http://${NIMBUS_HOST}:${site.storm-site.nimbus.thrift.port}</value>
+          </export>
+        </exports>
+      </exportGroup>
+    </exportGroups>
+
+    <commandOrders>
+      <commandOrder>
+        <command>NIMBUS-START</command>
+        <requires>SUPERVISOR-INSTALLED,STORM_UI_SERVER-INSTALLED,DRPC_SERVER-INSTALLED,STORM_REST_API-INSTALLED
+        </requires>
+      </commandOrder>
+      <commandOrder>
+        <command>SUPERVISOR-START</command>
+        <requires>NIMBUS-STARTED</requires>
+      </commandOrder>
+      <commandOrder>
+        <command>DRPC_SERVER-START</command>
+        <requires>NIMBUS-STARTED</requires>
+      </commandOrder>
+      <commandOrder>
+        <command>STORM_REST_API-START</command>
+        <requires>NIMBUS-STARTED,DRPC_SERVER-STARTED,STORM_UI_SERVER-STARTED</requires>
+      </commandOrder>
+      <commandOrder>
+        <command>STORM_UI_SERVER-START</command>
+        <requires>NIMBUS-STARTED</requires>
+      </commandOrder>
+    </commandOrders>
+
+    <components>
+
+      <component>
+        <name>NIMBUS</name>
+        <category>MASTER</category>
+        <autoStartOnFailure>true</autoStartOnFailure>
+        <appExports>QuickLinks-nimbus.url,QuickLinks-ganglia.ui,QuickLinks-app.metrics</appExports>
+        <commandScript>
+          <script>scripts/nimbus.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+
+      <component>
+        <name>STORM_REST_API</name>
+        <category>MASTER</category>
+        <autoStartOnFailure>true</autoStartOnFailure>
+        <appExports>QuickLinks-app.jmx</appExports>
+        <commandScript>
+          <script>scripts/rest_api.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+
+      <component>
+        <name>SUPERVISOR</name>
+        <category>SLAVE</category>
+        <autoStartOnFailure>true</autoStartOnFailure>
+        <componentExports>
+          <componentExport>
+            <name>log_viewer_port</name>
+            <value>${THIS_HOST}:${site.storm-site.logviewer.port}</value>
+          </componentExport>
+        </componentExports>
+        <commandScript>
+          <script>scripts/supervisor.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+
+      <component>
+        <name>STORM_UI_SERVER</name>
+        <category>MASTER</category>
+        <publishConfig>true</publishConfig>
+        <appExports>QuickLinks-app.monitor</appExports>
+        <autoStartOnFailure>true</autoStartOnFailure>
+        <commandScript>
+          <script>scripts/ui_server.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+
+      <component>
+        <name>DRPC_SERVER</name>
+        <category>MASTER</category>
+        <autoStartOnFailure>true</autoStartOnFailure>
+        <commandScript>
+          <script>scripts/drpc_server.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+
+      <component>
+        <name>ANOTHER_COMPONENT</name>
+        <category>MASTER</category>
+        <commands>
+          <command>
+            <exec>start command</exec>
+          </command>
+          <command>
+            <exec>stop command</exec>
+            <name>STOP</name>
+          </command>
+        </commands>
+      </component>
+    </components>
+
+    <osSpecifics>
+      <osSpecific>
+        <osType>any</osType>
+        <packages>
+          <package>
+            <type>tarball</type>
+            <name>files/apache-storm-0.9.1.2.1.1.0-237.tar.gz</name>
+          </package>
+        </packages>
+      </osSpecific>
+    </osSpecifics>
+
+    <packages>
+      <package>
+        <type>tarball</type>
+        <name>test-tarball-name.tgz</name>
+      </package>
+    </packages>
+  </application>
+</metainfo>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[76/76] [abbrv] hadoop git commit: YARN-5701. Fix issues in yarn native services apps-of-apps. Contributed by Billie Rinaldi

Posted by ji...@apache.org.
YARN-5701. Fix issues in yarn native services apps-of-apps. Contributed by Billie Rinaldi


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

Branch: refs/heads/yarn-native-services
Commit: 7ec43e697366cef4517f652a7248591d07166d2b
Parents: be69595
Author: Jian He <ji...@apache.org>
Authored: Sun Oct 16 17:01:09 2016 -0700
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 .../org/apache/slider/client/SliderClient.java  | 72 ++++++++--------
 .../slider/core/buildutils/InstanceBuilder.java |  4 +
 .../apache/slider/providers/ProviderUtils.java  | 62 ++++++++++++--
 .../providers/docker/DockerClientProvider.java  |  4 +-
 .../providers/docker/DockerProviderService.java | 87 ++++++++++++++++----
 5 files changed, 164 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ec43e69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
index 2840c4b..94e51e5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
@@ -178,6 +178,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.BufferedReader;
+import java.io.Console;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
@@ -918,57 +919,56 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
       return;
     }
 
-    BufferedReader br = null;
-    try {
-      for (Entry<String, List<String>> cred : tree.credentials.entrySet()) {
-        String provider = cred.getKey()
-            .replaceAll(Pattern.quote("${CLUSTER_NAME}"), clusterName)
-            .replaceAll(Pattern.quote("${CLUSTER}"), clusterName);
-        List<String> aliases = cred.getValue();
-        if (aliases == null || aliases.isEmpty()) {
-          continue;
-        }
-        Configuration c = new Configuration(conf);
-        c.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, provider);
-        CredentialProvider credentialProvider = CredentialProviderFactory.getProviders(c).get(0);
-        Set<String> existingAliases = new HashSet<>(credentialProvider.getAliases());
-        for (String alias : aliases) {
-          if (existingAliases.contains(alias.toLowerCase(Locale.ENGLISH))) {
-            log.info("Credentials for " + alias + " found in " + provider);
-          } else {
-            if (br == null) {
-              br = new BufferedReader(new InputStreamReader(System.in));
-            }
-            char[] pass = readPassword(alias, br);
-            credentialProvider.createCredentialEntry(alias, pass);
-            credentialProvider.flush();
-            Arrays.fill(pass, ' ');
+    Console console = System.console();
+    for (Entry<String, List<String>> cred : tree.credentials.entrySet()) {
+      String provider = cred.getKey()
+          .replaceAll(Pattern.quote("${CLUSTER_NAME}"), clusterName)
+          .replaceAll(Pattern.quote("${CLUSTER}"), clusterName);
+      List<String> aliases = cred.getValue();
+      if (aliases == null || aliases.isEmpty()) {
+        continue;
+      }
+      Configuration c = new Configuration(conf);
+      c.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, provider);
+      CredentialProvider credentialProvider = CredentialProviderFactory.getProviders(c).get(0);
+      Set<String> existingAliases = new HashSet<>(credentialProvider.getAliases());
+      for (String alias : aliases) {
+        if (existingAliases.contains(alias.toLowerCase(Locale.ENGLISH))) {
+          log.info("Credentials for " + alias + " found in " + provider);
+        } else {
+          if (console == null) {
+            throw new IOException("Unable to input password for " + alias +
+                " because System.console() is null; provider " + provider +
+                " must be populated manually");
           }
+          char[] pass = readPassword(alias, console);
+          credentialProvider.createCredentialEntry(alias, pass);
+          credentialProvider.flush();
+          Arrays.fill(pass, ' ');
         }
       }
-    } finally {
-      org.apache.hadoop.io.IOUtils.closeStream(br);
     }
   }
 
   private static char[] readOnePassword(String alias) throws IOException {
-    try(BufferedReader br = new BufferedReader(new InputStreamReader(System.in))) {
-      return readPassword(alias, br);
+    Console console = System.console();
+    if (console == null) {
+      throw new IOException("Unable to input password for " + alias +
+          " because System.console() is null");
     }
+    return readPassword(alias, console);
   }
 
-  // using a normal reader instead of a secure one,
-  // because stdin is not hooked up to the command line
-  private static char[] readPassword(String alias, BufferedReader br)
+  private static char[] readPassword(String alias, Console console)
       throws IOException {
     char[] cred = null;
 
     boolean noMatch;
     do {
-      log.info(String.format("%s %s: ", PASSWORD_PROMPT, alias));
-      char[] newPassword1 = br.readLine().toCharArray();
-      log.info(String.format("%s %s again: ", PASSWORD_PROMPT, alias));
-      char[] newPassword2 = br.readLine().toCharArray();
+      console.printf("%s %s: \n", PASSWORD_PROMPT, alias);
+      char[] newPassword1 = console.readPassword();
+      console.printf("%s %s again: \n", PASSWORD_PROMPT, alias);
+      char[] newPassword2 = console.readPassword();
       noMatch = !Arrays.equals(newPassword1, newPassword2);
       if (noMatch) {
         if (newPassword1 != null) Arrays.fill(newPassword1, ' ');

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ec43e69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/buildutils/InstanceBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/buildutils/InstanceBuilder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/buildutils/InstanceBuilder.java
index 25c65fc..f0686af 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/buildutils/InstanceBuilder.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/buildutils/InstanceBuilder.java
@@ -439,6 +439,10 @@ public class InstanceBuilder {
       }
       log.info("External appdefs after {}: {}", component, externalAppDefs);
 
+      SliderUtils.mergeMapsIgnoreDuplicateKeys(
+          appConf.getConfTree().credentials,
+          componentAppConf.getConfTree().credentials);
+
       mergeExternalComponent(appConf, componentAppConf, component, null);
       mergeExternalComponent(resources, componentConf.getResourceOperations(),
           component, getNextPriority());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ec43e69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java
index 47556f0..c5e6782 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderUtils.java
@@ -308,6 +308,7 @@ public class ProviderUtils implements RoleKeys, SliderKeys {
   public Map<String, String> filterSiteOptions(Map<String, String> options,
       Map<String, String> tokenMap) {
     String prefix = OptionKeys.SITE_XML_PREFIX;
+    String format = "${%s}";
     Map<String, String> filteredOptions = new HashMap<>();
     for (Map.Entry<String, String> entry : options.entrySet()) {
       String key = entry.getKey();
@@ -319,7 +320,7 @@ public class ProviderUtils implements RoleKeys, SliderKeys {
                 token.getValue());
           }
         }
-        filteredOptions.put(key, value);
+        filteredOptions.put(String.format(format, key), value);
       }
     }
     return filteredOptions;
@@ -545,9 +546,14 @@ public class ProviderUtils implements RoleKeys, SliderKeys {
    * @param clusterName app name
    * @throws IOException file cannot be created
    */
-  private void createConfigFile(SliderFileSystem fileSystem, File file,
-      ConfigFormat configFormat, String configFileDN,
+  private synchronized void createConfigFile(SliderFileSystem fileSystem,
+      File file, ConfigFormat configFormat, String configFileDN,
       Map<String, String> config, String clusterName) throws IOException {
+    if (file.exists()) {
+      log.info("Skipping writing {} file {} because it already exists",
+          configFormat, file);
+      return;
+    }
     log.info("Writing {} file {}", configFormat, file);
 
     ConfigUtils.prepConfigForTemplateOutputter(configFormat, config,
@@ -643,11 +649,10 @@ public class ProviderUtils implements RoleKeys, SliderKeys {
     String fileName = ConfigUtils.replaceProps(config, configFileName);
     File localFile = new File(RESOURCE_DIR);
     if (!localFile.exists()) {
-      if (!localFile.mkdir()) {
+      if (!localFile.mkdir() && !localFile.exists()) {
         throw new IOException(RESOURCE_DIR + " could not be created!");
       }
     }
-    localFile = new File(localFile, new File(fileName).getName());
 
     String folder = null;
     if ("true".equals(config.get(PER_COMPONENT))) {
@@ -655,12 +660,25 @@ public class ProviderUtils implements RoleKeys, SliderKeys {
     } else if ("true".equals(config.get(PER_GROUP))) {
       folder = roleGroup;
     }
+    if (folder != null) {
+      localFile = new File(localFile, folder);
+      if (!localFile.exists()) {
+        if (!localFile.mkdir() && !localFile.exists()) {
+          throw new IOException(localFile + " could not be created!");
+        }
+      }
+    }
+    localFile = new File(localFile, new File(fileName).getName());
 
     log.info("Localizing {} configs to config file {} (destination {}) " +
             "based on {} configs", config.size(), localFile, fileName,
         configFileDN);
-    createConfigFile(fileSystem, localFile, configFormat, configFileDN, config,
-        clusterName);
+    if (!localFile.exists()) {
+      createConfigFile(fileSystem, localFile, configFormat, configFileDN,
+          config, clusterName);
+    } else {
+      log.info("Local {} file {} already exists", configFormat, localFile);
+    }
     Path destPath = uploadResource(localFile, fileSystem, folder, clusterName);
     LocalResource configResource = fileSystem.createAmResource(destPath,
         LocalResourceType.FILE);
@@ -807,12 +825,12 @@ public class ProviderUtils implements RoleKeys, SliderKeys {
    */
   public Map<String, Map<String, String>> buildConfigurations(
       ConfTreeOperations appConf, ConfTreeOperations internalsConf,
-      String containerId, String roleName, String roleGroup,
+      String containerId, String clusterName, String roleName, String roleGroup,
       StateAccessForProviders amState) {
 
     Map<String, Map<String, String>> configurations = new TreeMap<>();
     Map<String, String> tokens = getStandardTokenMap(appConf,
-        internalsConf, roleName, roleGroup, containerId);
+        internalsConf, roleName, roleGroup, containerId, clusterName);
 
     Set<String> configs = new HashSet<>();
     configs.addAll(getApplicationConfigurationTypes(roleGroup, appConf));
@@ -1164,6 +1182,32 @@ public class ProviderUtils implements RoleKeys, SliderKeys {
   }
 
   /**
+   * Return a list of hostnames based on current ClusterNodes.
+   * @param values cluster nodes
+   * @return list of hosts
+   */
+  public Iterable<String> getHostNamesList(Collection<ClusterNode> values) {
+    List<String> hosts = new ArrayList<>();
+    for (ClusterNode cn : values) {
+      hosts.add(cn.hostname);
+    }
+    return hosts;
+  }
+
+  /**
+   * Return a list of IPs based on current ClusterNodes.
+   * @param values cluster nodes
+   * @return list of hosts
+   */
+  public Iterable<String> getIPsList(Collection<ClusterNode> values) {
+    List<String> hosts = new ArrayList<>();
+    for (ClusterNode cn : values) {
+      hosts.add(cn.ip);
+    }
+    return hosts;
+  }
+
+  /**
    * Update ServiceRecord in Registry with IP and hostname.
    * @param amState access to AM state
    * @param yarnRegistry acces to YARN registry

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ec43e69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerClientProvider.java
index 13473e5..d554427 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerClientProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerClientProvider.java
@@ -82,8 +82,8 @@ public class DockerClientProvider extends AbstractClientProvider
       if (appConf.getComponentOptBool(roleGroup, AM_CONFIG_GENERATION, false)) {
         // build and localize configuration files
         Map<String, Map<String, String>> configurations =
-            providerUtils.buildConfigurations(appConf, appConf, null, roleGroup,
-                roleGroup, null);
+            providerUtils.buildConfigurations(appConf, appConf, null,
+                null, roleGroup, roleGroup, null);
         try {
           providerUtils.localizeConfigFiles(null, roleGroup, roleGroup, appConf,
               configurations, null, fs, null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ec43e69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerProviderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerProviderService.java
index bebb5f0..af36620 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerProviderService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerProviderService.java
@@ -41,6 +41,7 @@ import org.apache.slider.core.registry.docstore.ConfigFormat;
 import org.apache.slider.core.registry.docstore.ConfigUtils;
 import org.apache.slider.core.registry.docstore.ExportEntry;
 import org.apache.slider.providers.AbstractProviderService;
+import org.apache.slider.providers.MonitorDetail;
 import org.apache.slider.providers.ProviderCore;
 import org.apache.slider.providers.ProviderRole;
 import org.apache.slider.providers.ProviderUtils;
@@ -62,6 +63,9 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Scanner;
+import java.util.regex.Pattern;
+
+import static org.apache.slider.api.RoleKeys.ROLE_PREFIX;
 
 public class DockerProviderService extends AbstractProviderService implements
     ProviderCore,
@@ -130,10 +134,13 @@ public class DockerProviderService extends AbstractProviderService implements
         DOCKER_USE_PRIVILEGED, false));
 
     // Set the environment
-    launcher.putEnv(SliderUtils.buildEnvMap(appComponent,
-        providerUtils.getStandardTokenMap(getAmState().getAppConfSnapshot(),
-            getAmState().getInternalsSnapshot(), roleName, roleGroup,
-            getClusterName())));
+    Map<String, String> standardTokens = providerUtils.getStandardTokenMap(
+        getAmState().getAppConfSnapshot(), getAmState().getInternalsSnapshot(),
+        roleName, roleGroup, container.getId().toString(), getClusterName());
+    Map<String, String> replaceTokens = providerUtils.filterSiteOptions(
+            appConf.getComponent(roleGroup).options, standardTokens);
+    replaceTokens.putAll(standardTokens);
+    launcher.putEnv(SliderUtils.buildEnvMap(appComponent, replaceTokens));
 
     String workDir = ApplicationConstants.Environment.PWD.$();
     launcher.setEnv("WORK_DIR", workDir);
@@ -169,8 +176,8 @@ public class DockerProviderService extends AbstractProviderService implements
           providerUtils.buildConfigurations(
               instanceDefinition.getAppConfOperations(),
               instanceDefinition.getInternalOperations(),
-              container.getId().toString(), roleName, roleGroup,
-              getAmState());
+              container.getId().toString(), getClusterName(),
+              roleName, roleGroup, getAmState());
       providerUtils.localizeConfigFiles(launcher, roleName, roleGroup,
           appConf, configurations, launcher.getEnv(), fileSystem,
           getClusterName());
@@ -251,8 +258,8 @@ public class DockerProviderService extends AbstractProviderService implements
     // build and localize configuration files
     Map<String, Map<String, String>> configurations =
         providerUtils.buildConfigurations(appConf, getAmState()
-            .getInternalsSnapshot(), null, clientName, clientName,
-            getAmState());
+            .getInternalsSnapshot(), null, getClusterName(), clientName,
+            clientName, getAmState());
 
     for (String configFileDN : configurations.keySet()) {
       String configFileName = appConf.getComponentOpt(clientName,
@@ -316,19 +323,45 @@ public class DockerProviderService extends AbstractProviderService implements
         getAmState().getAppConfSnapshot(), roleGroup);
 
     String hostKeyFormat = "${%s_HOST}";
+    String hostNameKeyFormat = "${%s_HOSTNAME}";
+    String ipKeyFormat = "${%s_IP}";
 
     // publish export groups if any
-    Map<String, String> replaceTokens =
-        providerUtils.filterSiteOptions(
-            appConf.getComponent(roleGroup).options,
-            providerUtils.getStandardTokenMap(appConf, internalsConf, roleName,
-                roleGroup, containerId, getClusterName()));
+    Map<String, String> standardTokens = providerUtils.getStandardTokenMap(
+        appConf, internalsConf, roleName, roleGroup, containerId,
+        getClusterName());
+    Map<String, String> replaceTokens = providerUtils.filterSiteOptions(
+            appConf.getComponent(roleGroup).options, standardTokens);
+    replaceTokens.putAll(standardTokens);
+
+    String rolePrefix = appConf.getComponentOpt(roleGroup, ROLE_PREFIX, "");
     for (Map.Entry<String, Map<String, ClusterNode>> entry :
         getAmState().getRoleClusterNodeMapping().entrySet()) {
-      String hostName = providerUtils.getHostsList(
+      String otherRolePrefix = appConf.getComponentOpt(entry.getKey(),
+          ROLE_PREFIX, "");
+      if (!otherRolePrefix.equals(rolePrefix)) {
+        // hostname replacements are only made within role prefix groups
+        continue;
+      }
+      String key = entry.getKey();
+      if (!rolePrefix.isEmpty()) {
+        if (!key.startsWith(rolePrefix)) {
+          log.warn("Something went wrong, {} doesn't start with {}", key,
+              rolePrefix);
+          continue;
+        }
+        key = key.substring(rolePrefix.length());
+      }
+      key = key.toUpperCase(Locale.ENGLISH);
+      String host = providerUtils.getHostsList(
           entry.getValue().values(), true).iterator().next();
-      replaceTokens.put(String.format(hostKeyFormat,
-          entry.getKey().toUpperCase(Locale.ENGLISH)), hostName);
+      replaceTokens.put(String.format(hostKeyFormat, key), host);
+      String hostName = providerUtils.getHostNamesList(
+          entry.getValue().values()).iterator().next();
+      replaceTokens.put(String.format(hostNameKeyFormat, key), hostName);
+      String ip = providerUtils.getIPsList(
+          entry.getValue().values()).iterator().next();
+      replaceTokens.put(String.format(ipKeyFormat, key), ip);
     }
     replaceTokens.put("${THIS_HOST}", thisHost);
 
@@ -338,7 +371,7 @@ public class DockerProviderService extends AbstractProviderService implements
       // replace host names and site properties
       for (String token : replaceTokens.keySet()) {
         if (value.contains(token)) {
-          value = value.replace(token, replaceTokens.get(token));
+          value = value.replaceAll(Pattern.quote(token), replaceTokens.get(token));
         }
       }
       ExportEntry entry = new ExportEntry();
@@ -350,6 +383,24 @@ public class DockerProviderService extends AbstractProviderService implements
       log.info("Preparing to publish. Key {} and Value {}",
           export.getKey(), value);
     }
-    providerUtils.publishExportGroup(entries, getAmState(), EXPORT_GROUP);
+    if (!entries.isEmpty()) {
+      providerUtils.publishExportGroup(entries, getAmState(), EXPORT_GROUP);
+    }
+  }
+
+  @Override
+  public Map<String, MonitorDetail> buildMonitorDetails(ClusterDescription clusterDesc) {
+    Map<String, MonitorDetail> details = super.buildMonitorDetails(clusterDesc);
+    buildRoleHostDetails(details);
+    return details;
+  }
+
+  private void buildRoleHostDetails(Map<String, MonitorDetail> details) {
+    for (Map.Entry<String, Map<String, ClusterNode>> entry :
+        getAmState().getRoleClusterNodeMapping().entrySet()) {
+      details.put(entry.getKey() + " Host(s)/Container(s)",
+          new MonitorDetail(providerUtils.getHostsList(
+              entry.getValue().values(), false).toString(), false));
+    }
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[60/76] [abbrv] hadoop git commit: YARN-5909. Remove agent related code in slider AM. Contributed by Jian He

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentRoles.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentRoles.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentRoles.java
deleted file mode 100644
index 281895a..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentRoles.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.slider.providers.agent;
-
-import org.apache.slider.providers.ProviderRole;
-
-import java.util.ArrayList;
-import java.util.List;
-
-public class AgentRoles {
-
-  /**
-   * List of roles Agent provider does not have any roles by default. All roles are read from the application
-   * specification.
-   */
-  protected static final List<ProviderRole> ROLES =
-      new ArrayList<ProviderRole>();
-
-  public static List<ProviderRole> getRoles() {
-    return ROLES;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentUtils.java
deleted file mode 100644
index 23e05a3..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/AgentUtils.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent;
-
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.slider.common.tools.SliderFileSystem;
-import org.apache.slider.common.tools.SliderUtils;
-import org.apache.slider.core.conf.ConfTreeOperations;
-import org.apache.slider.core.exceptions.BadConfigException;
-import org.apache.slider.providers.agent.application.metadata.AbstractMetainfoParser;
-import org.apache.slider.providers.agent.application.metadata.AddonPackageMetainfoParser;
-import org.apache.slider.providers.agent.application.metadata.DefaultConfig;
-import org.apache.slider.providers.agent.application.metadata.DefaultConfigParser;
-import org.apache.slider.providers.agent.application.metadata.Metainfo;
-import org.apache.slider.providers.agent.application.metadata.MetainfoParser;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
-
-import static org.apache.slider.api.RoleKeys.ROLE_PREFIX;
-
-/**
- *
- */
-public class AgentUtils {
-  private static final Logger log = LoggerFactory.getLogger(AgentUtils.class);
-
-  public static Metainfo getApplicationMetainfoFromSummaryFile(
-      SliderFileSystem fileSystem, String metainfoPath, boolean metainfoForAddon) {
-    FileSystem fs = fileSystem.getFileSystem();
-    Path appPathXML = new Path(metainfoPath + ".metainfo.xml");
-    Path appPathJson = new Path(metainfoPath + ".metainfo.json");
-    Path appPathUsed = null;
-    try {
-      FSDataInputStream appStream = null;
-      if (fs.exists(appPathXML)) {
-        appPathUsed = appPathXML;
-        appStream = fs.open(appPathXML);
-        return parseMetainfo(appStream, metainfoForAddon, "xml");
-      } else if (fs.exists(appPathJson)) {
-        appPathUsed = appPathJson;
-        appStream = fs.open(appPathJson);
-        return parseMetainfo(appStream, metainfoForAddon, "json");
-      }
-    } catch (IOException e) {
-      log.info("Failed to get metainfo from summary file {} - {}", appPathUsed,
-          e.getMessage());
-      log.debug("Failed to get metainfo", e);
-    }
-    return null;
-  }
-
-  public static Metainfo getApplicationMetainfo(SliderFileSystem fileSystem,
-      String metainfoPath, boolean metainfoForAddon) throws IOException,
-      BadConfigException {
-    log.info("Reading metainfo at {}", metainfoPath);
-    Metainfo metainfo = getApplicationMetainfoFromSummaryFile(fileSystem,
-        metainfoPath, metainfoForAddon);
-    if (metainfo != null) {
-      log.info("Got metainfo from summary file");
-      return metainfo;
-    }
-
-    FileSystem fs = fileSystem.getFileSystem();
-    Path appPath = new Path(metainfoPath);
-
-    InputStream metainfoJsonStream = SliderUtils.getApplicationResourceInputStream(
-        fs, appPath, "metainfo.json");
-    if (metainfoJsonStream == null) {
-      InputStream metainfoXMLStream = SliderUtils.getApplicationResourceInputStream(
-          fs, appPath, "metainfo.xml");
-      if (metainfoXMLStream != null) {
-        metainfo = parseMetainfo(metainfoXMLStream, metainfoForAddon, "xml");
-      }
-    } else {
-      metainfo = parseMetainfo(metainfoJsonStream, metainfoForAddon, "json");
-    }
-
-    if (metainfo == null) {
-      log.error("metainfo is unavailable at {}.", metainfoPath);
-      throw new FileNotFoundException("metainfo.xml/json is required in app package. " +
-                                      appPath);
-    }
-    return metainfo;
-  }
-
-  private static Metainfo parseMetainfo(InputStream stream,
-      boolean metainfoForAddon, String type) throws IOException {
-    AbstractMetainfoParser metainfoParser = null;
-    if (metainfoForAddon) {
-      metainfoParser = new AddonPackageMetainfoParser();
-    } else {
-      metainfoParser = new MetainfoParser();
-    }
-    if (type.equals("xml")) {
-      return metainfoParser.fromXmlStream(stream);
-    } else if (type.equals("json")) {
-      return metainfoParser.fromJsonStream(stream);
-    }
-    return null;
-  }
-
-  static DefaultConfig getDefaultConfig(SliderFileSystem fileSystem,
-                                        String appDef, String configFileName)
-      throws IOException {
-    // this is the path inside the zip file
-    String fileToRead = "configuration/" + configFileName;
-    log.info("Reading default config file {} at {}", fileToRead, appDef);
-    InputStream configStream = SliderUtils.getApplicationResourceInputStream(
-        fileSystem.getFileSystem(), new Path(appDef), fileToRead);
-    if (configStream == null) {
-      log.error("{} is unavailable at {}.", fileToRead, appDef);
-      throw new IOException("Expected config file " + fileToRead + " is not available.");
-    }
-
-    return new DefaultConfigParser().parse(configStream);
-  }
-
-  static String getMetainfoComponentName(String roleGroup,
-      ConfTreeOperations appConf) throws BadConfigException {
-    String prefix = appConf.getComponentOpt(roleGroup, ROLE_PREFIX, null);
-    if (prefix == null) {
-      return roleGroup;
-    }
-    if (!roleGroup.startsWith(prefix)) {
-      throw new BadConfigException("Component " + roleGroup + " doesn't start" +
-          " with prefix " + prefix);
-    }
-    return roleGroup.substring(prefix.length());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/Command.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/Command.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/Command.java
deleted file mode 100644
index 647cb86..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/Command.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.slider.providers.agent;
-
-/** The states a component instance can be. */
-public enum Command {
-  NOP,           // do nothing
-  INSTALL,       // Install the component
-  INSTALL_ADDON, // Install add on packages if any
-  START,         // Start the component
-  STOP,          // Stop the component
-  UPGRADE,       // The component will undergo upgrade
-  TERMINATE;     // Send terminate signal to agent
-
-  public static Command getCommand(String commandVal) {
-    if (commandVal.equals(Command.START.toString())) {
-      return Command.START;
-    }
-    if (commandVal.equals(Command.INSTALL.toString())) {
-      return Command.INSTALL;
-    }
-    if (commandVal.equals(Command.STOP.toString())) {
-      return Command.STOP;
-    }
-    if (commandVal.equals(Command.UPGRADE.toString())) {
-      return Command.UPGRADE;
-    }
-    if (commandVal.equals(Command.TERMINATE.toString())) {
-      return Command.TERMINATE;
-    }
-
-    return Command.NOP;
-  }
-
-  public static String transform(Command command, boolean isUpgrade) {
-    switch (command) {
-    case STOP:
-      return isUpgrade ? "UPGRADE_STOP" : command.name();
-    default:
-      return command.name();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/CommandResult.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/CommandResult.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/CommandResult.java
deleted file mode 100644
index 35d9116..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/CommandResult.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.slider.providers.agent;
-
-/** Command results. */
-public enum CommandResult {
-  IN_PROGRESS,  // Command is in progress
-  COMPLETED,    // Command has successfully completed
-  FAILED;        // Command has failed
-
-  public static CommandResult getCommandResult(String commandResVal) {
-    if (commandResVal.equals(CommandResult.COMPLETED.toString())) {
-      return CommandResult.COMPLETED;
-    }
-    if (commandResVal.equals(CommandResult.FAILED.toString())) {
-      return CommandResult.FAILED;
-    }
-    if (commandResVal.equals(CommandResult.IN_PROGRESS.toString())) {
-      return CommandResult.IN_PROGRESS;
-    }
-
-    throw new IllegalArgumentException("Unrecognized value " + commandResVal);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentCommandOrder.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentCommandOrder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentCommandOrder.java
deleted file mode 100644
index 4abac7a..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentCommandOrder.java
+++ /dev/null
@@ -1,225 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.slider.providers.agent;
-
-import org.apache.slider.common.tools.SliderUtils;
-import org.apache.slider.core.conf.ConfTreeOperations;
-import org.apache.slider.providers.agent.application.metadata.CommandOrder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-
-import static org.apache.slider.api.RoleKeys.ROLE_PREFIX;
-
-/**
- * Stores the command dependency order for all components in a service. <commandOrder>
- * <command>SUPERVISOR-START</command> <requires>NIMBUS-STARTED</requires> </commandOrder> Means, SUPERVISOR START
- * requires NIMBUS to be STARTED
- */
-public class ComponentCommandOrder {
-  public static final Logger log =
-      LoggerFactory.getLogger(ComponentCommandOrder.class);
-  private static char SPLIT_CHAR = '-';
-  Map<Command, Map<String, List<ComponentState>>> dependencies =
-      new HashMap<Command, Map<String, List<ComponentState>>>();
-  Map<String, Collection<String>> prefixRoleMap = new HashMap<>();
-  Map<String, String> rolePrefixMap = new HashMap<>();
-
-  public ComponentCommandOrder() {}
-
-  public ComponentCommandOrder(List<CommandOrder> commandOrders,
-      ConfTreeOperations resources) {
-    mergeCommandOrders(commandOrders, resources);
-  }
-
-  void mergeCommandOrders(List<CommandOrder> commandOrders,
-      ConfTreeOperations resources) {
-    for (String component : resources.getComponentNames()) {
-      String prefix = SliderUtils.trimPrefix(
-          resources.getComponentOpt(component, ROLE_PREFIX, null));
-      if (prefix != null) {
-        rolePrefixMap.put(component, prefix);
-        if (!prefixRoleMap.containsKey(prefix)) {
-          prefixRoleMap.put(prefix, new HashSet<String>());
-        }
-        prefixRoleMap.get(prefix).add(component);
-      }
-    }
-    if (commandOrders != null && commandOrders.size() > 0) {
-      for (CommandOrder commandOrder : commandOrders) {
-        ComponentCommand componentCmd = getComponentCommand(
-            commandOrder.getCommand(), resources);
-        String requires = commandOrder.getRequires();
-        List<ComponentState> requiredStates = parseRequiredStates(requires,
-            resources);
-        if (requiredStates.size() > 0) {
-          Map<String, List<ComponentState>> compDep = dependencies.get(componentCmd.command);
-          if (compDep == null) {
-            compDep = new HashMap<>();
-            dependencies.put(componentCmd.command, compDep);
-          }
-
-          List<ComponentState> requirements = compDep.get(componentCmd.componentName);
-          if (requirements == null) {
-            requirements = new ArrayList<>();
-            compDep.put(componentCmd.componentName, requirements);
-          }
-
-          requirements.addAll(requiredStates);
-        }
-      }
-    }
-  }
-
-  private List<ComponentState> parseRequiredStates(String requires,
-      ConfTreeOperations resources) {
-    if (requires == null || requires.length() < 2) {
-      throw new IllegalArgumentException("Input cannot be null and must contain component and state.");
-    }
-
-    String[] componentStates = requires.split(",");
-    List<ComponentState> retList = new ArrayList<ComponentState>();
-    for (String componentStateStr : componentStates) {
-      retList.add(getComponentState(componentStateStr, resources));
-    }
-
-    return retList;
-  }
-
-  private ComponentCommand getComponentCommand(String compCmdStr,
-      ConfTreeOperations resources) {
-    if (compCmdStr == null || compCmdStr.trim().length() < 2) {
-      throw new IllegalArgumentException("Input cannot be null and must contain component and command.");
-    }
-
-    compCmdStr = compCmdStr.trim();
-    int splitIndex = compCmdStr.lastIndexOf(SPLIT_CHAR);
-    if (splitIndex == -1 || splitIndex == 0 || splitIndex == compCmdStr.length() - 1) {
-      throw new IllegalArgumentException("Input does not appear to be well-formed.");
-    }
-    String compStr = compCmdStr.substring(0, splitIndex);
-    String cmdStr = compCmdStr.substring(splitIndex + 1);
-
-    if (resources.getComponent(compStr) == null && !prefixRoleMap.containsKey(compStr)) {
-      throw new IllegalArgumentException("Component " + compStr + " specified" +
-          " in command order does not exist");
-    }
-
-    Command cmd = Command.valueOf(cmdStr);
-
-    if (cmd != Command.START) {
-      throw new IllegalArgumentException("Dependency order can only be specified for START.");
-    }
-    return new ComponentCommand(compStr, cmd);
-  }
-
-  private ComponentState getComponentState(String compStStr,
-      ConfTreeOperations resources) {
-    if (compStStr == null || compStStr.trim().length() < 2) {
-      throw new IllegalArgumentException("Input cannot be null.");
-    }
-
-    compStStr = compStStr.trim();
-    int splitIndex = compStStr.lastIndexOf(SPLIT_CHAR);
-    if (splitIndex == -1 || splitIndex == 0 || splitIndex == compStStr.length() - 1) {
-      throw new IllegalArgumentException("Input does not appear to be well-formed.");
-    }
-    String compStr = compStStr.substring(0, splitIndex);
-    String stateStr = compStStr.substring(splitIndex + 1);
-
-    if (resources.getComponent(compStr) == null && !prefixRoleMap.containsKey(compStr)) {
-      throw new IllegalArgumentException("Component " + compStr + " specified" +
-          " in command order does not exist");
-    }
-
-    State state = State.valueOf(stateStr);
-    if (state != State.STARTED && state != State.INSTALLED) {
-      throw new IllegalArgumentException("Dependency order can only be specified against STARTED/INSTALLED.");
-    }
-    return new ComponentState(compStr, state);
-  }
-
-  // dependency is still on component level, but not package level
-  // so use component name to check dependency, not component-package
-  public boolean canExecute(String component, Command command, Collection<ComponentInstanceState> currentStates) {
-    if (!dependencies.containsKey(command)) {
-      return true;
-    }
-    List<ComponentState> required = new ArrayList<>();
-    if (dependencies.get(command).containsKey(component)) {
-      required.addAll(dependencies.get(command).get(component));
-    }
-    String prefix = rolePrefixMap.get(component);
-    if (prefix != null && dependencies.get(command).containsKey(prefix)) {
-      required.addAll(dependencies.get(command).get(prefix));
-    }
-
-    for (ComponentState stateToMatch : required) {
-      for (ComponentInstanceState currState : currentStates) {
-        log.debug("Checking schedule {} {} against dependency {} is {}",
-            component, command, currState.getComponentName(), currState.getState());
-        if (currState.getComponentName().equals(stateToMatch.componentName) ||
-            (prefixRoleMap.containsKey(stateToMatch.componentName) &&
-                prefixRoleMap.get(stateToMatch.componentName).contains(currState.getComponentName()))) {
-          if (currState.getState() != stateToMatch.state) {
-            if (stateToMatch.state == State.STARTED) {
-              log.info("Cannot schedule {} {} as dependency {} is {}",
-                  component, command, currState.getComponentName(), currState.getState());
-              return false;
-            } else {
-              //state is INSTALLED
-              if (currState.getState() != State.STARTING && currState.getState() != State.STARTED) {
-                log.info("Cannot schedule {} {} as dependency {} is {}",
-                    component, command, currState.getComponentName(), currState.getState());
-                return false;
-              }
-            }
-          }
-        }
-      }
-    }
-    return true;
-  }
-
-  static class ComponentState {
-    public String componentName;
-    public State state;
-
-    public ComponentState(String componentName, State state) {
-      this.componentName = componentName;
-      this.state = state;
-    }
-  }
-
-  static class ComponentCommand {
-    public String componentName;
-    public Command command;
-
-    public ComponentCommand(String componentName, Command command) {
-      this.componentName = componentName;
-      this.command = command;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentInstanceState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentInstanceState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentInstanceState.java
deleted file mode 100644
index 6ee0ebb..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentInstanceState.java
+++ /dev/null
@@ -1,340 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.slider.providers.agent;
-
-import java.util.Map;
-import java.util.TreeMap;
-
-import com.google.common.annotations.VisibleForTesting;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.slider.providers.agent.application.metadata.Component;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/** This class implements a simple state machine for component instances. */
-public class ComponentInstanceState {
-  public static final Logger log =
-      LoggerFactory.getLogger(ComponentInstanceState.class);
-  private static int MAX_FAILURE_TOLERATED = 3;
-  private static String INVALID_TRANSITION_ERROR =
-      "Result %s for command %s is not expected for component %s in state %s.";
-
-  private final String componentName;
-  private final ContainerId containerId;
-  private final String containerIdAsString;
-  private final String applicationId;
-  private State state = State.INIT;
-  private State targetState = State.STARTED;
-  private int failuresSeen = 0;
-  private Boolean configReported = false;
-  private long lastHeartbeat = 0;
-  private String ip;
-  private String hostname;
-  private ContainerState containerState;
-
-  private Map<String, State> pkgStatuses;
-  private String nextPkgToInstall;
-
-  private boolean stopInitiated;
-
-  public ComponentInstanceState(String componentName,
-      ContainerId containerId,
-      String applicationId) {
-    this(componentName, containerId, applicationId,
-        new TreeMap<String, State>());
-  }
-
-  public ComponentInstanceState(String componentName,
-      ContainerId containerId,
-      String applicationId, Map<String, State> pkgStatuses) {
-    this.componentName = componentName;
-    this.containerId = containerId;
-    this.containerIdAsString = containerId.toString();
-    this.applicationId = applicationId;
-    this.containerState = ContainerState.INIT;
-    this.lastHeartbeat = System.currentTimeMillis();
-    this.pkgStatuses = pkgStatuses;
-  }
-  
-  public String getComponentName() {
-    return componentName;
-  }
-
-  public Boolean getConfigReported() {
-    return configReported;
-  }
-
-  public void setConfigReported(Boolean configReported) {
-    this.configReported = configReported;
-  }
-
-  public ContainerState getContainerState() {
-    return containerState;
-  }
-
-  public void setContainerState(ContainerState containerState) {
-    this.containerState = containerState;
-  }
-
-  public long getLastHeartbeat() {
-    return lastHeartbeat;
-  }
-
-  /**
-   * Update the heartbeat, and change container state
-   * to mark as healthy if appropriate
-   * @param heartbeatTime last time the heartbeat was seen
-   * @return the current container state
-   */
-  public ContainerState heartbeat(long heartbeatTime) {
-    this.lastHeartbeat = heartbeatTime;
-    if(containerState == ContainerState.UNHEALTHY ||
-       containerState == ContainerState.INIT) {
-      containerState = ContainerState.HEALTHY;
-    }
-    return containerState;
-  }
-  
-
-  public ContainerId getContainerId() {
-    return containerId;
-  }
-
-  public void commandIssued(Command command) {
-    commandIssued(command, false);
-  }
-
-  public void commandIssued(Command command, boolean isInUpgradeMode) {
-    Command expected = getNextCommand(isInUpgradeMode);
-    if (expected != command) {
-      throw new IllegalArgumentException("Command " + command + " is not allowed in state " + state);
-    }
-    if (expected == Command.INSTALL_ADDON) {
-      // for add on packages, the pkg must be nextPkgToInstall
-      State currentState = pkgStatuses.get(nextPkgToInstall);
-      log.debug("Command issued: component: {} is in {}", componentName,
-          currentState);
-      State nextState = currentState.getNextState(command);
-      pkgStatuses.put(nextPkgToInstall, nextState);
-      log.debug("Command issued: component: {} is now in {}", componentName,
-          nextState);
-    } else {
-      // for master package
-      state = state.getNextState(command);
-    }
-  }
-
-  public void applyCommandResult(CommandResult result, Command command,
-      String pkg) {
-    // if the heartbeat is for a package
-    // update that package's state in the component status
-    // and don't bother with the master pkg
-    if (StringUtils.isNotEmpty(pkg)
-        && !Component.MASTER_PACKAGE_NAME.equals(pkg)) {
-      log.debug("This result is for component: {} pkg: {}", componentName, pkg);
-      State previousPkgState = pkgStatuses.get(pkg);
-      log.debug("Currently component: {} pkg: {} is in state: {}",
-          componentName, pkg, previousPkgState.toString());
-      State nextPkgState = previousPkgState.getNextState(result);
-      pkgStatuses.put(pkg, nextPkgState);
-      log.debug("Component: {} pkg: {} next state: {}", componentName, pkg,
-          nextPkgState);
-    } else {
-      log.debug("This result is for component: {} master package",
-          componentName);
-      applyCommandResult(result, command);
-    }
-  }
-
-  public void applyCommandResult(CommandResult result, Command command) {
-    if (!this.state.couldHaveIssued(command)) {
-      throw new IllegalStateException("Invalid command " + command + " for state " + this.state);
-    }
-
-    try {
-      if (result == CommandResult.FAILED) {
-        failuresSeen++;
-      } else if (result == CommandResult.COMPLETED) {
-        failuresSeen = 0;
-      }
-      state = state.getNextState(result);
-    } catch (IllegalArgumentException e) {
-      String message = String.format(INVALID_TRANSITION_ERROR,
-                                     result.toString(),
-                                     command.toString(),
-                                     componentName,
-                                     state.toString());
-      log.warn(message);
-      throw new IllegalStateException(message);
-    }
-  }
-
-  public boolean hasPendingCommand() {
-    if (state.canIssueCommands() &&
-        state != targetState &&
-        failuresSeen < MAX_FAILURE_TOLERATED) {
-      return true;
-    }
-
-    return false;
-  }
-
-  public Command getNextCommand() {
-    return getNextCommand(false);
-  }
-
-  public Command getNextCommand(boolean isInUpgradeMode) {
-    if (!hasPendingCommand()) {
-      nextPkgToInstall = null;
-      return Command.NOP;
-    }
-
-    log.debug("In getNextCommand, checking for component: {} ", componentName);
-    // if the master pkg is just installed, check if any add on pkg need to be
-    // installed
-    nextPkgToInstall = null;
-    if (state == State.INSTALLED) {
-      for (Map.Entry<String, State> pkgStatus : pkgStatuses.entrySet()) {
-        String pkg = pkgStatus.getKey();
-        State pkgState = pkgStatus.getValue();
-        log.debug("In getNextCommand, pkg: {} is in {}", pkg, pkgState);
-        if (pkgState == State.INSTALLING) {
-          // first check if any pkg is install in progress, if so, wait
-          // so we don't need to do anything, just return NOP
-          log.debug("In getNextCommand, pkg: {} we are issuing NOP", pkg);
-          nextPkgToInstall = pkg;
-          return Command.NOP;
-        } else if (pkgState == State.INIT) {
-          // temporarily storing pkg here
-          // in case no pkg in 'installing' state
-          // will return the package to install
-          nextPkgToInstall = pkg;
-        }
-      }
-      // when we reach here, no pkg is in 'installing' state
-      if (nextPkgToInstall != null) {
-        // nextPkgToInstall != null means some pkg is in INIT state 
-        // issue 'install' to the pkg we have stored in nextPkgToInstall
-        log.debug("In getNextCommand, pkg: {} we are issuing install addon",
-            nextPkgToInstall);
-        return Command.INSTALL_ADDON;
-      }
-    }
-    return this.state.getSupportedCommand(isInUpgradeMode, stopInitiated);
-  }
-
-  public State getState() {
-    return state;
-  }
-
-  @VisibleForTesting
-  protected void setState(State state) {
-    this.state = state;
-  }
-
-  public State getTargetState() {
-    return targetState;
-  }
-
-  public void setTargetState(State targetState) {
-    this.targetState = targetState;
-  }
-
-  public String getNextPkgToInstall() {
-    return nextPkgToInstall;
-  }
-
-  public boolean isStopInitiated() {
-    return stopInitiated;
-  }
-
-  public void setStopInitiated(boolean stopInitiated) {
-    this.stopInitiated = stopInitiated;
-  }
-
-  @Override
-  public int hashCode() {
-    int hashCode = 1;
-
-    hashCode = hashCode ^ (componentName != null ? componentName.hashCode() : 0);
-    hashCode = hashCode ^ (containerIdAsString != null ? containerIdAsString.hashCode() : 0);
-    hashCode = hashCode ^ (applicationId != null ? applicationId.hashCode() : 0);
-    return hashCode;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-
-    if (o == null || getClass() != o.getClass()) return false;
-
-    ComponentInstanceState that = (ComponentInstanceState) o;
-
-    if (this.componentName != null ?
-        !this.componentName.equals(that.componentName) : this.componentName != null) {
-      return false;
-    }
-
-    if (this.containerIdAsString != null ?
-        !this.containerIdAsString.equals(that.containerIdAsString) : this.containerIdAsString != null) {
-      return false;
-    }
-
-    if (this.applicationId != null ?
-        !this.applicationId.equals(that.applicationId) : this.applicationId != null) {
-      return false;
-    }
-
-    return true;
-  }
-
-  @Override
-  public String toString() {
-    final StringBuilder sb =
-        new StringBuilder("ComponentInstanceState{");
-    sb.append("containerIdAsString='").append(containerIdAsString).append('\'');
-    sb.append(", state=").append(state);
-    sb.append(", failuresSeen=").append(failuresSeen);
-    sb.append(", lastHeartbeat=").append(lastHeartbeat);
-    sb.append(", containerState=").append(containerState);
-    sb.append(", componentName='").append(componentName).append('\'');
-    sb.append(", ip=").append(ip);
-    sb.append(", hostname='").append(hostname).append('\'');
-    sb.append('}');
-    return sb.toString();
-  }
-
-  public String getIp() {
-    return ip;
-  }
-
-  public void setIp(String ip) {
-    this.ip = ip;
-  }
-
-  public String getHostname() {
-    return hostname;
-  }
-
-  public void setHostname(String hostname) {
-    this.hostname = hostname;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentTagProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentTagProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentTagProvider.java
deleted file mode 100644
index 68f63fa..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ComponentTagProvider.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent;
-
-import org.apache.slider.common.tools.SliderUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.ConcurrentHashMap;
-
-/** A simple tag provider that attempts to associate tags from 1-N to all container of a component */
-public class ComponentTagProvider {
-  private static final Logger log = LoggerFactory.getLogger(ComponentTagProvider.class);
-  private static String FREE = "free";
-  private final ConcurrentHashMap<String, ConcurrentHashMap<String, String>> allTags;
-
-  public ComponentTagProvider() {
-    allTags = new ConcurrentHashMap<String, ConcurrentHashMap<String, String>>();
-  }
-
-  /**
-   * Record an assigned tag to a container
-   *
-   * @param component
-   * @param containerId
-   * @param tag
-   */
-  public void recordAssignedTag(String component, String containerId, String tag) {
-    if (SliderUtils.isSet(component) && SliderUtils.isSet(containerId)) {
-      Integer key = null;
-      try {
-        key = Integer.valueOf(tag);
-      } catch (NumberFormatException nfe) {
-        //ignore
-      }
-      if (key != null && key > 0) {
-        ConcurrentHashMap<String, String> compTags = getComponentSpecificTags(component);
-        synchronized (compTags) {
-          for (int index = 1; index <= key.intValue(); index++) {
-            String tempKey = new Integer(index).toString();
-            if (!compTags.containsKey(tempKey)) {
-              compTags.put(tempKey, FREE);
-            }
-          }
-          compTags.put(key.toString(), containerId);
-        }
-      }
-    }
-  }
-
-  /**
-   * Get a tag for container
-   *
-   * @param component
-   * @param containerId
-   *
-   * @return
-   */
-  public String getTag(String component, String containerId) {
-    if (SliderUtils.isSet(component) && SliderUtils.isSet(containerId)) {
-      ConcurrentHashMap<String, String> compTags = getComponentSpecificTags(component);
-      synchronized (compTags) {
-        for (String key : compTags.keySet()) {
-          if (compTags.get(key).equals(containerId)) {
-            return key;
-          }
-        }
-        for (String key : compTags.keySet()) {
-          if (compTags.get(key).equals(FREE)) {
-            compTags.put(key, containerId);
-            return key;
-          }
-        }
-        String newKey = new Integer(compTags.size() + 1).toString();
-        compTags.put(newKey, containerId);
-        return newKey;
-      }
-    }
-    return "";
-  }
-
-  /**
-   * Release a tag associated with a container
-   *
-   * @param component
-   * @param containerId
-   */
-  public void releaseTag(String component, String containerId) {
-    if (SliderUtils.isSet(component) && SliderUtils.isSet(containerId)) {
-      ConcurrentHashMap<String, String> compTags = allTags.get(component);
-      if (compTags != null) {
-        synchronized (compTags) {
-          for (String key : compTags.keySet()) {
-            if (compTags.get(key).equals(containerId)) {
-              compTags.put(key, FREE);
-            }
-          }
-        }
-      }
-    }
-  }
-
-  private ConcurrentHashMap<String, String> getComponentSpecificTags(String component) {
-    if (!allTags.containsKey(component)) {
-      synchronized (allTags) {
-        if (!allTags.containsKey(component)) {
-          allTags.put(component, new ConcurrentHashMap<String, String>());
-        }
-      }
-    }
-    return allTags.get(component);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ContainerState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ContainerState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ContainerState.java
deleted file mode 100644
index 0394ba2..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/ContainerState.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.slider.providers.agent;
-
-/** The states a component instance can be. */
-public enum ContainerState {
-  INIT,           // Container is not net activated
-  HEALTHY,     // Agent is heartbeating
-  UNHEALTHY,      // Container is unhealthy - no heartbeat for some interval
-  HEARTBEAT_LOST;  // Container is lost - request a new instance
-
-  /**
-   * Indicates whether or not it is a valid state to produce a command.
-   *
-   * @return true if command can be issued for this state.
-   */
-  public boolean canIssueCommands() {
-    switch (this) {
-      case HEALTHY:
-        return true;
-      default:
-        return false;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/HeartbeatMonitor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/HeartbeatMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/HeartbeatMonitor.java
deleted file mode 100644
index 4293916..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/HeartbeatMonitor.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent;
-
-import com.google.common.annotations.VisibleForTesting;
-
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Map;
-
-/** Monitors the container state and heartbeats. */
-public class HeartbeatMonitor implements Runnable {
-  protected static final Logger log =
-      LoggerFactory.getLogger(HeartbeatMonitor.class);
-  private final int threadWakeupInterval; //1 minute
-  private final AgentProviderService provider;
-  private volatile boolean shouldRun = true;
-  private Thread monitorThread = null;
-
-  public HeartbeatMonitor(AgentProviderService provider, int threadWakeupInterval) {
-    this.provider = provider;
-    this.threadWakeupInterval = threadWakeupInterval;
-  }
-
-  public void shutdown() {
-    shouldRun = false;
-  }
-
-  public void start() {
-    log.info("Starting heartbeat monitor with interval {}", threadWakeupInterval);
-    monitorThread = new Thread(this);
-    monitorThread.start();
-  }
-
-  void join(long millis) throws InterruptedException {
-    if (isAlive()) {
-      monitorThread.join(millis);
-    }
-  }
-
-  public boolean isAlive() {
-    return monitorThread != null && monitorThread.isAlive();
-  }
-
-  @Override
-  public void run() {
-    while (shouldRun) {
-      try {
-        log.debug("Putting monitor to sleep for " + threadWakeupInterval + " " +
-                  "milliseconds");
-        Thread.sleep(threadWakeupInterval);
-        doWork(System.currentTimeMillis());
-      } catch (InterruptedException ex) {
-        log.warn("Scheduler thread is interrupted going to stop", ex);
-        shouldRun = false;
-      } catch (Exception ex) {
-        log.warn("Exception received", ex);
-      } catch (Throwable t) {
-        log.warn("ERROR", t);
-      }
-    }
-  }
-
-  /**
-   * Every interval the current state of the container are checked. If the state is INIT or HEALTHY and no HB are
-   * received in last check interval they are marked as UNHEALTHY. INIT is when the agent is started but it did not
-   * communicate at all. HEALTHY being the AM has received heartbeats. After an interval as UNHEALTHY the container is
-   * declared unavailable
-   * @param now current time in milliseconds ... tests can set this explicitly
-   */
-  @VisibleForTesting
-  public void doWork(long now) {
-    Map<String, ComponentInstanceState> componentStatuses = provider.getComponentStatuses();
-    if (componentStatuses != null) {
-      for (String containerLabel : componentStatuses.keySet()) {
-        ComponentInstanceState componentInstanceState = componentStatuses.get(containerLabel);
-        long timeSinceLastHeartbeat = now - componentInstanceState.getLastHeartbeat();
-
-        if (timeSinceLastHeartbeat > threadWakeupInterval) {
-          switch (componentInstanceState.getContainerState()) {
-            case INIT:
-            case HEALTHY:
-              componentInstanceState.setContainerState(ContainerState.UNHEALTHY);
-              log.warn(
-                  "Component {} marked UNHEALTHY. Last heartbeat received at {} approx. {} ms. back.",
-                  componentInstanceState,
-                  componentInstanceState.getLastHeartbeat(),
-                  timeSinceLastHeartbeat);
-              break;
-            case UNHEALTHY:
-              if (timeSinceLastHeartbeat > threadWakeupInterval * 2) {
-                componentInstanceState.setContainerState(
-                    ContainerState.HEARTBEAT_LOST);
-                log.warn(
-                    "Component {} marked HEARTBEAT_LOST. Last heartbeat received at {} approx. {} ms. back.",
-                    componentInstanceState, componentInstanceState.getLastHeartbeat(),
-                    timeSinceLastHeartbeat);
-                ContainerId containerId =
-                    componentInstanceState.getContainerId();
-                provider.lostContainer(containerLabel, containerId);
-              }
-              break;
-            case HEARTBEAT_LOST:
-              // unexpected case
-              log.warn("Heartbeat from lost component: {}", componentInstanceState);
-              break;
-          }
-            
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/State.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/State.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/State.java
deleted file mode 100644
index 5603f8d..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/State.java
+++ /dev/null
@@ -1,199 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.slider.providers.agent;
-
-/** The states a component instance can be. */
-public enum State {
-  INIT,           // Not installed
-  INSTALLING,     // Being installed
-  INSTALLED,      // Installed (or stopped)
-  STARTING,       // Starting
-  STARTED,        // Started
-  INSTALL_FAILED, // Install failed, start failure in INSTALLED
-  UPGRADING,      // Undergoing upgrade, perform necessary pre-upgrade steps
-  UPGRADED,       // Pre-upgrade steps completed
-  STOPPING,       // Stop has been issued
-  STOPPED,        // Agent has stopped
-  TERMINATING;    // Terminate signal to ask the agent to kill itself
-                  // No need for state TERMINATED (as the agent is dead by then)
-
-  /**
-   * Indicates whether or not it is a valid state to produce a command.
-   *
-   * @return true if command can be issued for this state.
-   */
-  public boolean canIssueCommands() {
-    switch (this) {
-      case INSTALLING:
-      case STARTING:
-      case UPGRADING:
-      case STOPPING:
-      case TERMINATING:
-        return false;
-      default:
-        return true;
-    }
-  }
-
-  /**
-   * Returns valid command in this state.
-   *
-   * @return command allowed in this state.
-   */
-  public Command getSupportedCommand() {
-    return getSupportedCommand(false);
-  }
-
-  public Command getSupportedCommand(boolean isInUpgradeMode) {
-    return getSupportedCommand(isInUpgradeMode, false);
-  }
-
-  public Command getSupportedCommand(boolean isInUpgradeMode,
-      boolean stopInitiated) {
-    switch (this) {
-      case INIT:
-      case INSTALL_FAILED:
-        return Command.INSTALL;
-      case INSTALLED:
-        return Command.START;
-      case STARTED:
-      return isInUpgradeMode ? Command.UPGRADE : (stopInitiated) ? Command.STOP
-          : Command.NOP;
-      case UPGRADED:
-        return Command.STOP;
-      case STOPPED:
-        return Command.TERMINATE;
-      default:
-        return Command.NOP;
-    }
-  }
-
-  /**
-   * Returns next state based on the command result.
-   *
-   * @return next state.
-   */
-  public State getNextState(CommandResult result) throws IllegalArgumentException {
-    switch (result) {
-      case IN_PROGRESS:
-        if (this == State.INSTALLING || this == State.STARTING
-            || this == State.UPGRADING || this == State.STOPPING
-            || this == State.TERMINATING) {
-          return this;
-        } else {
-          throw new IllegalArgumentException(result + " is not valid for " + this);
-        }
-      case COMPLETED:
-        if (this == State.INSTALLING) {
-          return State.INSTALLED;
-        } else if (this == State.STARTING) {
-          return State.STARTED;
-        } else if (this == State.UPGRADING) {
-          return State.UPGRADED;
-        } else if (this == State.STOPPING) {
-          return State.STOPPED;
-        } else {
-          throw new IllegalArgumentException(result + " is not valid for " + this);
-        }
-      case FAILED:
-        if (this == State.INSTALLING) {
-          return State.INSTALL_FAILED;
-        } else if (this == State.STARTING) {
-          return State.INSTALLED;
-        } else if (this == State.UPGRADING) {
-          // if pre-upgrade failed, force stop now, so mark it upgraded
-          // what other options can be exposed to app owner?
-          return State.UPGRADED;
-        } else if (this == State.STOPPING) {
-          // if stop fails, force mark it stopped (and let container terminate)
-          return State.STOPPED;
-        } else if (this == State.STOPPED) {
-          // if in stopped state, force mark it as terminating
-          return State.TERMINATING;
-        } else {
-          throw new IllegalArgumentException(result + " is not valid for " + this);
-        }
-      default:
-        throw new IllegalArgumentException("Bad command result " + result);
-    }
-  }
-
-  /**
-   * Returns next state based on the command.
-   *
-   * @return next state.
-   */
-  public State getNextState(Command command) throws IllegalArgumentException {
-    switch (command) {
-      case INSTALL:
-        if (this == State.INIT || this == State.INSTALL_FAILED) {
-          return State.INSTALLING;
-        } else {
-          throw new IllegalArgumentException(command + " is not valid for " + this);
-        }
-      case INSTALL_ADDON:
-          if (this == State.INIT || this == State.INSTALL_FAILED) {
-            return State.INSTALLING;
-          } else {
-            throw new IllegalArgumentException(command + " is not valid for " + this);
-          }
-      case START:
-        if (this == State.INSTALLED) {
-          return State.STARTING;
-        } else {
-          throw new IllegalArgumentException(command + " is not valid for " + this);
-        }
-      case UPGRADE:
-        if (this == State.STARTED) {
-          return State.UPGRADING;
-        } else {
-          throw new IllegalArgumentException(command + " is not valid for " + this);
-        }
-      case STOP:
-        if (this == State.STARTED || this == State.UPGRADED) {
-          return State.STOPPING;
-        } else {
-          throw new IllegalArgumentException(command + " is not valid for " + this);
-        }
-      case TERMINATE:
-        if (this == State.STOPPED) {
-          return State.TERMINATING;
-        } else {
-          throw new IllegalArgumentException(command + " is not valid for " + this);
-        }
-      case NOP:
-        return this;
-      default:
-        throw new IllegalArgumentException("Bad command " + command);
-    }
-  }
-
-  public boolean couldHaveIssued(Command command) {
-    if ((this == State.INSTALLING && command == Command.INSTALL)
-        || (this == State.STARTING && command == Command.START)
-        || (this == State.UPGRADING && command == Command.UPGRADE)
-        || (this == State.STOPPING 
-           && (command == Command.STOP || command == Command.NOP))
-        || (this == State.TERMINATING && command == Command.TERMINATE)
-       ) {
-      return true;
-    }
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractComponent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractComponent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractComponent.java
deleted file mode 100644
index b6ae4de..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractComponent.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/**
- *  Component defined in master package metainfo.json
- */
-public abstract class AbstractComponent implements Validate {
-  public static final String TYPE_STANDARD = "STANDARD";
-  public static final String TYPE_DOCKER = "DOCKER";
-  public static final String TYPE_PYTHON = "PYTHON";
-  public static final String CATEGORY_MASTER = "MASTER";
-  public static final String CATEGORY_SLAVE = "SLAVE";
-  public static final String CATEGORY_CLIENT = "CLIENT";
-  public static final String MASTER_PACKAGE_NAME = "MASTER";
-
-  protected String name;
-  protected CommandScript commandScript;
-  protected List<ComponentCommand> commands = new ArrayList<>();
-
-  public AbstractComponent() {
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  public CommandScript getCommandScript() {
-    return commandScript;
-  }
-
-  public void addCommandScript(CommandScript commandScript) {
-    this.commandScript = commandScript;
-  }
-
-  @JsonProperty("commands")
-  public List<ComponentCommand> getCommands() {
-    return commands;
-  }
-
-  public void setCommands(List<ComponentCommand> commands) {
-    this.commands = commands;
-  }
-
-  public void addCommand(ComponentCommand command) {
-    commands.add(command);
-  }
-
-  @Override
-  public String toString() {
-    final StringBuilder sb = new StringBuilder("{");
-    sb.append("\n\"name\": ").append(name);
-    sb.append(",\n\"commandScript\" :").append(commandScript);
-    sb.append('}');
-    return sb.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractMetainfoParser.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractMetainfoParser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractMetainfoParser.java
deleted file mode 100644
index 67d1f15..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractMetainfoParser.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-import com.google.gson.Gson;
-import com.google.gson.GsonBuilder;
-
-import org.apache.commons.digester.Digester;
-import org.apache.commons.io.IOUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.xml.sax.SAXException;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.StringWriter;
-
-/**
- * This abstract class provide common functionality to parse metainfo.json for
- * either master package or add on packages.
- */
-public abstract class AbstractMetainfoParser {
-  protected final GsonBuilder gsonBuilder = new GsonBuilder();
-  protected final Gson gson;
-  private static final Logger log = LoggerFactory
-      .getLogger(AbstractMetainfoParser.class);
-
-  public AbstractMetainfoParser() {
-    gson = gsonBuilder.create();
-  }
-
-  /**
-   * Convert to a JSON string
-   *
-   * @return a JSON string description
-   *
-   * @throws IOException Problems mapping/writing the object
-   */
-  public String toJsonString(Metainfo metaInfo) throws IOException {
-    return gson.toJson(metaInfo);
-  }
-
-  /**
-   * Convert from JSON
-   *
-   * @param json input
-   *
-   * @return the parsed JSON
-   *
-   * @throws IOException IO
-   */
-  public Metainfo fromJsonString(String json)
-      throws IOException {
-    return gson.fromJson(json, Metainfo.class);
-  }
-
-  /**
-   * Parse metainfo from an IOStream
-   *
-   * @param is
-   *
-   * @return
-   *
-   * @throws IOException
-   */
-  public Metainfo fromJsonStream(InputStream is) throws IOException {
-    log.debug("loading from xml stream");
-    StringWriter writer = new StringWriter();
-    IOUtils.copy(is, writer);
-    return fromJsonString(writer.toString());
-  }
-
-  /**
-   * Parse metainfo from an XML formatted IOStream
-   *
-   * @param metainfoStream
-   *
-   * @return
-   *
-   * @throws IOException
-   */
-  public Metainfo fromXmlStream(InputStream metainfoStream) throws IOException {
-    log.debug("loading from xml stream");
-    Digester digester = new Digester();
-    digester.setValidating(false);
-
-    composeSchema(digester);
-
-    try {
-      return (Metainfo) digester.parse(metainfoStream);
-    } catch (IOException e) {
-      log.debug("IOException in metainfoparser during fromXmlStream: "
-          + e.getMessage());
-    } catch (SAXException e) {
-      log.debug("SAXException in metainfoparser during fromXmlStream: "
-          + e.getMessage());
-    } finally {
-      if (metainfoStream != null) {
-        metainfoStream.close();
-      }
-    }
-
-    return null;
-  }
-
-  /**
-   * Compose the schema for the metainfo
-   *
-   * @param Digester - The Digester object we passed in to compose the schema
-   *
-   * @return
-   *
-   * @throws IOException
-   */
-  abstract protected void composeSchema(Digester digester);
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractMetainfoSchema.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractMetainfoSchema.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractMetainfoSchema.java
deleted file mode 100644
index cfa2895..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractMetainfoSchema.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.codehaus.jackson.annotate.JsonProperty;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Application type defined in the metainfo
- */
-public abstract class AbstractMetainfoSchema implements Validate {
-  protected String name;
-  protected String comment;
-  protected String version;
-  protected List<ConfigFile> configFiles = new ArrayList<>();
-
-  public AbstractMetainfoSchema() {
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  public String getComment() {
-    return comment;
-  }
-
-  public void setComment(String comment) {
-    this.comment = comment;
-  }
-
-  public String getVersion() {
-    return version;
-  }
-
-  public void setVersion(String version) {
-    this.version = version;
-  }
-
-  public void addConfigFile(ConfigFile configFile) {
-    this.configFiles.add(configFile);
-  }
-
-  @JsonProperty("configFiles")
-  public List<ConfigFile> getConfigFiles() {
-    return configFiles;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AddonPackageMetainfoParser.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AddonPackageMetainfoParser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AddonPackageMetainfoParser.java
deleted file mode 100644
index c75837f..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AddonPackageMetainfoParser.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.apache.commons.digester.Digester;
-
-/**
- *
- */
-public class AddonPackageMetainfoParser extends AbstractMetainfoParser {
-
-  protected void composeSchema(Digester digester) {
-    digester.addObjectCreate("metainfo", Metainfo.class);
-    digester.addBeanPropertySetter("metainfo/schemaVersion");
-
-    digester.addObjectCreate("*/applicationPackage", ApplicationPackage.class);
-    digester.addBeanPropertySetter("*/applicationPackage/name");
-    digester.addBeanPropertySetter("*/applicationPackage/comment");
-    digester.addBeanPropertySetter("*/applicationPackage/version");
-
-    digester.addObjectCreate("*/component", ComponentsInAddonPackage.class);
-    digester.addBeanPropertySetter("*/component/name");
-    digester.addSetNext("*/component", "addComponent");
-
-    digester.addObjectCreate("*/commandScript", CommandScript.class);
-    digester.addBeanPropertySetter("*/commandScript/script");
-    digester.addBeanPropertySetter("*/commandScript/scriptType");
-    digester.addBeanPropertySetter("*/commandScript/timeout");
-    digester.addSetNext("*/commandScript", "addCommandScript");
-
-    digester.addObjectCreate("*/configFile", ConfigFile.class);
-    digester.addBeanPropertySetter("*/configFile/type");
-    digester.addBeanPropertySetter("*/configFile/fileName");
-    digester.addBeanPropertySetter("*/configFile/dictionaryName");
-    digester.addSetNext("*/configFile", "addConfigFile");
-
-    digester.addSetRoot("*/applicationPackage", "setApplicationPackage");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Application.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Application.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Application.java
deleted file mode 100644
index 5556c7f..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Application.java
+++ /dev/null
@@ -1,193 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.apache.slider.common.tools.SliderUtils;
-import org.apache.slider.core.exceptions.BadCommandArgumentsException;
-import org.apache.slider.core.exceptions.SliderException;
-import org.codehaus.jackson.annotate.JsonIgnore;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Application type defined in the metainfo
- */
-public class Application extends AbstractMetainfoSchema {
-  String exportedConfigs;
-  List<ExportGroup> exportGroups = new ArrayList<>();
-  List<OSSpecific> osSpecifics = new ArrayList<>();
-  List<CommandOrder> commandOrders = new ArrayList<>();
-  List<Package> packages = new ArrayList<>();
-  private List<Component> components = new ArrayList<>();
-
-  public Application() {
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  public String getComment() {
-    return comment;
-  }
-
-  public void setComment(String comment) {
-    this.comment = comment;
-  }
-
-  public String getVersion() {
-    return version;
-  }
-
-  public void setVersion(String version) {
-    this.version = version;
-  }
-
-  public String getExportedConfigs() {
-    return exportedConfigs;
-  }
-
-  public void setExportedConfigs(String exportedConfigs) {
-    this.exportedConfigs = exportedConfigs;
-  }
-
-  public void addConfigFile(ConfigFile configFile) {
-    this.configFiles.add(configFile);
-  }
-
-  @JsonProperty("configFiles")
-  public List<ConfigFile> getConfigFiles() {
-    return configFiles;
-  }
-
-  public void addComponent(Component component) {
-    components.add(component);
-  }
-
-  @JsonProperty("components")
-  public List<Component> getComponents() {
-    return components;
-  }
-
-  public void addExportGroup(ExportGroup exportGroup) {
-    exportGroups.add(exportGroup);
-  }
-
-  @JsonProperty("exportGroups")
-  public List<ExportGroup> getExportGroups() {
-    return exportGroups;
-  }
-
-  public void addOSSpecific(OSSpecific osSpecific) {
-    osSpecifics.add(osSpecific);
-  }
-
-  @JsonIgnore
-  public List<OSSpecific> getOSSpecifics() {
-    return osSpecifics;
-  }
-
-  public void addCommandOrder(CommandOrder commandOrder) {
-    commandOrders.add(commandOrder);
-  }
-
-  @JsonProperty("commandOrders")
-  public List<CommandOrder> getCommandOrders() {
-    return commandOrders;
-  }
-
-  public void addPackage(Package pkg) {
-    packages.add(pkg);
-  }
-
-  @JsonProperty("packages")
-  public List<Package> getPackages() {
-    return packages;
-  }
-
-  @Override
-  public String toString() {
-    final StringBuilder sb =
-        new StringBuilder("{");
-    sb.append(",\n\"name\": ").append(name);
-    sb.append(",\n\"comment\": ").append(comment);
-    sb.append(",\n\"version\" :").append(version);
-    sb.append(",\n\"components\" : {");
-    for (Component component : components) {
-      sb.append("\n").append(component.toString());
-    }
-    sb.append("\n},");
-    sb.append('}');
-    return sb.toString();
-  }
-
-  public void validate(String version) throws SliderException {
-    if(SliderUtils.isUnset(version)) {
-      throw new BadCommandArgumentsException("schema version cannot be null");
-    }
-
-    Metainfo.checkNonNull(getName(), "name", "application");
-
-    Metainfo.checkNonNull(getVersion(), "version", "application");
-
-    if(getComponents().size() == 0) {
-      throw new SliderException("application must contain at least one component");
-    }
-
-    if(version.equals(Metainfo.VERSION_TWO_ZERO)) {
-      if(getPackages().size() > 0) {
-        throw new SliderException("packages is not supported in version " + version);
-      }
-    }
-
-    if(version.equals(Metainfo.VERSION_TWO_ONE)) {
-      if(getOSSpecifics().size() > 0) {
-        throw new SliderException("osSpecifics is not supported in version " + version);
-      }
-    }
-
-    for(CommandOrder co : getCommandOrders()) {
-      co.validate(version);
-    }
-
-    for(Component comp : getComponents()) {
-      comp.validate(version);
-    }
-
-    for(ConfigFile cf : getConfigFiles()) {
-      cf.validate(version);
-    }
-
-    for(ExportGroup eg : getExportGroups()) {
-      eg.validate(version);
-    }
-
-    for(Package pkg : getPackages()) {
-      pkg.validate(version);
-    }
-
-    for(OSSpecific os : getOSSpecifics()) {
-      os.validate(version);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ApplicationPackage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ApplicationPackage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ApplicationPackage.java
deleted file mode 100644
index a94a213..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/ApplicationPackage.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.slider.core.exceptions.SliderException;
-
-public class ApplicationPackage extends AbstractMetainfoSchema{
-  private List<ComponentsInAddonPackage> components = new ArrayList<ComponentsInAddonPackage>();
-
-  public void addComponent(ComponentsInAddonPackage component) {
-    components.add(component);
-  }
-
-  // we must override getcomponent() as well. otherwise it is pointing to the
-  // overriden components of type List<Component>
-  public List<ComponentsInAddonPackage> getComponents(){
-    return this.components;
-  }
-
-  @Override
-  public String toString() {
-    final StringBuilder sb = new StringBuilder("{");
-    sb.append("\n\"name\": ").append(name);
-    sb.append(",\n\"comment\": ").append(comment);
-    sb.append(",\n\"version\" :").append(version);
-    sb.append(",\n\"components\" : {");
-    for (ComponentsInAddonPackage component : components) {
-      sb.append("\n").append(component);
-    }
-    sb.append("\n},");
-    sb.append('}');
-    return sb.toString();
-  }
-
-  @Override
-  public void validate(String version) throws SliderException {
-    if (name == null || name.isEmpty()) {
-      throw new SliderException(
-          "Missing name in metainfo.json for add on packages");
-    }
-    if (components.isEmpty()) {
-      throw new SliderException(
-          "Missing components in metainfo.json for add on packages");
-    }
-    for (ComponentsInAddonPackage component : components) {
-      if (component.name == null || component.name.isEmpty()) {
-        throw new SliderException(
-            "Missing name of components in metainfo.json for add on packages");
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/CommandOrder.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/CommandOrder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/CommandOrder.java
deleted file mode 100644
index 40d8cc6..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/CommandOrder.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.apache.slider.core.exceptions.SliderException;
-
-/**
- *
- */
-public class CommandOrder implements Validate {
-  String command;
-  String requires;
-
-  public CommandOrder() {
-  }
-
-  public String getCommand() {
-    return command;
-  }
-
-  public void setCommand(String command) {
-    this.command = command;
-  }
-
-  public String getRequires() {
-    return requires;
-  }
-
-  public void setRequires(String requires) {
-    this.requires = requires;
-  }
-
-  @Override
-  public String toString() {
-    final StringBuilder sb =
-        new StringBuilder("{");
-    sb.append(",\n\"command\": ").append(command);
-    sb.append(",\n\"requires\": ").append(requires);
-    sb.append('}');
-    return sb.toString();
-  }
-
-  public void validate(String version) throws SliderException {
-    Metainfo.checkNonNull(getCommand(), "command", "package");
-    Metainfo.checkNonNull(getRequires(), "requires", "package");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/CommandScript.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/CommandScript.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/CommandScript.java
deleted file mode 100644
index 9915ba1..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/CommandScript.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.providers.agent.application.metadata;
-
-import org.apache.slider.core.exceptions.SliderException;
-
-/**
- * CommandScript that implements all component commands
- */
-public class CommandScript implements Validate {
-  String script;
-  String scriptType;
-  long timeout;
-
-  public CommandScript() {
-
-  }
-
-  public String getScript() {
-    return script;
-  }
-
-  public void setScript(String script) {
-    this.script = script;
-  }
-
-  public String getScriptType() {
-    return scriptType;
-  }
-
-  public void setScriptType(String scriptType) {
-    this.scriptType = scriptType;
-  }
-
-  public long getTimeout() {
-    return timeout;
-  }
-
-  public void setTimeout(long timeout) {
-    this.timeout = timeout;
-  }
-
-  @Override
-  public String toString() {
-    final StringBuilder sb =
-        new StringBuilder("{");
-    sb.append(",\n\"script\": ").append(script);
-    sb.append(",\n\"scriptType\": ").append(scriptType);
-    sb.append(",\n\"timeout\" :").append(timeout);
-    sb.append('}');
-    return sb.toString();
-  }
-
-  public void validate(String version) throws SliderException {
-    Metainfo.checkNonNull(getScript(), "script", "commandScript");
-    Metainfo.checkNonNull(getScriptType(), "scriptType", "commandScript");
-  }
-}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[13/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsAndMonitoring.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsAndMonitoring.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsAndMonitoring.java
new file mode 100644
index 0000000..37a8935
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsAndMonitoring.java
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.management;
+
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.MetricSet;
+import com.codahale.metrics.health.HealthCheckRegistry;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.CompositeService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Class for all metrics and monitoring
+ */
+public class MetricsAndMonitoring extends CompositeService {
+  protected static final Logger log =
+    LoggerFactory.getLogger(MetricsAndMonitoring.class);
+  public MetricsAndMonitoring(String name) {
+    super(name);
+  }
+  
+  public MetricsAndMonitoring() {
+    super("MetricsAndMonitoring");
+  }
+  
+  /**
+   * Singleton of metrics registry
+   */
+  final MetricRegistry metrics = new MetricRegistry();
+
+  final HealthCheckRegistry health = new HealthCheckRegistry();
+
+  private final Map<String, MeterAndCounter> meterAndCounterMap
+      = new ConcurrentHashMap<>();
+
+  private final List<MetricSet> metricSets = new ArrayList<>();
+
+  /**
+   * List of recorded events
+   */
+  private final List<RecordedEvent> eventHistory = new ArrayList<>(100);
+
+  public static final int EVENT_LIMIT = 1000;
+
+  public MetricRegistry getMetrics() {
+    return metrics;
+  }
+
+  public HealthCheckRegistry getHealth() {
+    return health;
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    addService(new MetricsBindingService("MetricsBindingService",
+        metrics));
+    super.serviceInit(conf);
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    super.serviceStop();
+    for (MetricSet set : metricSets) {
+      unregister(set);
+    }
+  }
+
+  public MeterAndCounter getMeterAndCounter(String name) {
+    return meterAndCounterMap.get(name);
+  }
+
+  /**
+   * Get or create the meter/counter pair
+   * @param name name of instance
+   * @return an instance
+   */
+  public MeterAndCounter getOrCreateMeterAndCounter(String name) {
+    MeterAndCounter instance = meterAndCounterMap.get(name);
+    if (instance == null) {
+      synchronized (this) {
+        // check in a sync block
+        instance = meterAndCounterMap.get(name);
+        if (instance == null) {
+          instance = new MeterAndCounter(metrics, name);
+          meterAndCounterMap.put(name, instance);
+        }
+      }
+    }
+    return instance;
+  }
+
+  /**
+   * Get a specific meter and mark it. This will create and register it on demand.
+   * @param name name of meter/counter
+   */
+  public void markMeterAndCounter(String name) {
+    MeterAndCounter meter = getOrCreateMeterAndCounter(name);
+    meter.mark();
+  }
+
+  /**
+   * Given a {@link Metric}, registers it under the given name.
+   *
+   * @param name   the name of the metric
+   * @param metric the metric
+   * @param <T>    the type of the metric
+   * @return {@code metric}
+   * @throws IllegalArgumentException if the name is already registered
+   */
+  public <T extends Metric> T register(String name, T metric) throws IllegalArgumentException {
+    return metrics.register(name, metric);
+  }
+
+  public <T extends Metric> T register(Class<?> klass, T metric, String... names)
+      throws IllegalArgumentException {
+    return register(MetricRegistry.name(klass, names), metric);
+  }
+
+
+  /**
+   * Add an event (synchronized)
+   * @param event event
+   */
+  public synchronized void noteEvent(RecordedEvent event) {
+    if (eventHistory.size() > EVENT_LIMIT) {
+      eventHistory.remove(0);
+    }
+    eventHistory.add(event);
+  }
+
+  /**
+   * Clone the event history; blocks for the duration of the copy operation.
+   * @return a new list
+   */
+  public synchronized List<RecordedEvent> cloneEventHistory() {
+    return new ArrayList<>(eventHistory);
+  }
+
+  /**
+   * Add a metric set for registering and deregistration on service stop
+   * @param metricSet metric set
+   */
+  public void addMetricSet(MetricSet metricSet) {
+    metricSets.add(metricSet);
+    metrics.registerAll(metricSet);
+  }
+
+  /**
+   * add a metric set, giving each entry a prefix
+   * @param prefix prefix (a trailing "." is automatically added)
+   * @param metricSet the metric set to register
+   */
+  public void addMetricSet(String prefix, MetricSet metricSet) {
+    addMetricSet(new PrefixedMetricsSet(prefix, metricSet));
+  }
+
+  /**
+   * Unregister a metric set; robust
+   * @param metricSet metric set to unregister
+   */
+  public void unregister(MetricSet metricSet) {
+    for (String s : metricSet.getMetrics().keySet()) {
+      try {
+        metrics.remove(s);
+      } catch (IllegalArgumentException e) {
+        // log but continue
+        log.info("Exception when trying to unregister {}", s, e);
+      }
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsBindingService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsBindingService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsBindingService.java
new file mode 100644
index 0000000..864a1cf
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsBindingService.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.management;
+
+import com.codahale.metrics.JmxReporter;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.MetricSet;
+import com.codahale.metrics.ScheduledReporter;
+import com.codahale.metrics.Slf4jReporter;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.slider.server.services.workflow.ClosingService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * YARN service which hooks up Codahale metrics to 
+ * JMX, and, if enabled Ganglia and/or an SLF4J log.
+ */
+public class MetricsBindingService extends CompositeService
+    implements MetricsKeys {
+  protected static final Logger log =
+      LoggerFactory.getLogger(MetricsBindingService.class);
+  private final MetricRegistry metrics;
+
+  private String reportingDetails = "not started";
+
+
+  public MetricsBindingService(String name,
+      MetricRegistry metrics) {
+    super(name);
+    Preconditions.checkArgument(metrics != null, "Null metrics");
+    this.metrics = metrics;
+  }
+
+  /**
+   * Instantiate...create a metric registry in the process
+   * @param name service name
+   */
+  public MetricsBindingService(String name) {
+    this(name, new MetricRegistry());
+  }
+
+  /**
+   * Accessor for the metrics instance
+   * @return the metrics
+   */
+  public MetricRegistry getMetrics() {
+    return metrics;
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    super.serviceStart();
+
+    StringBuilder summary = new StringBuilder();
+    Configuration conf = getConfig();
+
+    summary.append("Reporting to JMX");
+    // always start the JMX binding
+    JmxReporter jmxReporter;
+    jmxReporter = JmxReporter.forRegistry(metrics).build();
+    jmxReporter.start();
+    addService(new ClosingService<>(jmxReporter));
+
+
+    // Ganglia
+    if (conf.getBoolean(METRICS_GANGLIA_ENABLED, false)) {
+      log.warn("Ganglia integration is not implemented");
+/*
+      // This is all disabled due to transitive dependencies on an LGPL library
+      com.codahale.metrics.ganglia.GangliaReporter gangliaReporter;
+      String host = conf.getTrimmed(METRICS_GANGLIA_HOST, "");
+      int port = conf.getInt(METRICS_GANGLIA_PORT, DEFAULT_GANGLIA_PORT);
+      int interval = conf.getInt(METRICS_GANGLIA_REPORT_INTERVAL, 60);
+      int ttl = 1;
+      info.ganglia.gmetric4j.gmetric.GMetric.UDPAddressingMode
+          mcast = info.ganglia.gmetric4j.gmetric.GMetric.UDPAddressingMode.getModeForAddress(host);
+      boolean ganglia31 = conf.getBoolean(METRICS_GANGLIA_VERSION_31, true);
+
+      final info.ganglia.gmetric4j.gmetric.GMetric ganglia =
+          new info.ganglia.gmetric4j.gmetric.GMetric(
+              host,
+              port,
+              mcast,
+              ttl,
+              ganglia31);
+      gangliaReporter = com.codahale.metrics.ganglia.GangliaReporter.forRegistry(metrics)
+                                       .convertRatesTo(TimeUnit.SECONDS)
+                                       .convertDurationsTo(
+                                           TimeUnit.MILLISECONDS)
+                                       .build(ganglia);
+      gangliaReporter.start(interval, TimeUnit.SECONDS);
+      addService(new ClosingService<ScheduledReporter>(gangliaReporter));
+      summary.append(String.format(", Ganglia at %s:%d interval=%d",
+          host, port, interval));
+      */
+    }
+
+    // Logging
+    if (conf.getBoolean(METRICS_LOGGING_ENABLED, false)) {
+      ScheduledReporter reporter;
+      String logName =
+          conf.getTrimmed(METRICS_LOGGING_LOG, METRICS_DEFAULT_LOG);
+      int interval = conf.getInt(METRICS_LOGGING_LOG_INTERVAL,
+          METRICS_DEFAULT_LOG_INTERVAL);
+      reporter = Slf4jReporter.forRegistry(metrics)
+                              .convertRatesTo(TimeUnit.SECONDS)
+                              .outputTo(LoggerFactory.getLogger(logName))
+                              .convertDurationsTo(TimeUnit.MILLISECONDS)
+                              .build();
+      reporter.start(interval, TimeUnit.MINUTES);
+      addService(new ClosingService<>(reporter));
+      summary.append(String.format(", SLF4J to log %s interval=%d",
+          logName, interval));
+    }
+    reportingDetails = summary.toString();
+    log.info(reportingDetails);
+  }
+
+
+  @Override
+  public String toString() {
+    return super.toString() + " " + reportingDetails;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsConstants.java
new file mode 100644
index 0000000..fa6bfc0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsConstants.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.management;
+
+/**
+ * Constants used in slider for metrics registration and lookup
+ */
+public class MetricsConstants {
+
+  /**
+   * {@value}
+   */
+  public static final String CONTAINERS_OUTSTANDING_REQUESTS = "containers.outstanding-requests";
+
+  /**
+   * {@value}
+   */
+  public static final String CONTAINERS_STARTED = "containers.started";
+
+  /**
+   * {@value}
+   */
+  public static final String CONTAINERS_SURPLUS = "containers.surplus";
+
+  /**
+   * {@value}
+   */
+  public static final String CONTAINERS_COMPLETED = "containers.completed";
+
+  /**
+   * {@value}
+   */
+  public static final String CONTAINERS_FAILED = "containers.failed";
+
+  /**
+   * {@value}
+   */
+  public static final String CONTAINERS_START_FAILED = "containers.start-failed";
+
+  public static final String PREFIX_SLIDER_ROLES = "slider.roles.";
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsKeys.java
new file mode 100644
index 0000000..13b3b6b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/MetricsKeys.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.management;
+
+public interface MetricsKeys {
+
+  /**
+   * Prefix for metrics configuration options: {@value}
+   */
+  String METRICS_PREFIX = "slider.metrics.";
+  
+  /**
+   * Boolean to enable Ganglia metrics reporting
+   * {@value}
+   */
+  String METRICS_GANGLIA_ENABLED =
+      METRICS_PREFIX + "ganglia.enabled";
+  /**
+   * {@value}
+   */
+  String METRICS_GANGLIA_HOST = METRICS_PREFIX + "ganglia.host";
+  /**
+   * {@value}
+   */
+  String METRICS_GANGLIA_PORT = METRICS_PREFIX + "ganglia.port";
+  /**
+   * {@value}
+   */
+  String METRICS_GANGLIA_VERSION_31 = METRICS_PREFIX + "ganglia.version-31";
+  /**
+   * {@value}
+   */
+  String METRICS_GANGLIA_REPORT_INTERVAL = METRICS_PREFIX + "ganglia.report.interval";
+  /**
+   * {@value}
+   */
+  int DEFAULT_GANGLIA_PORT = 8649;
+
+
+  /**
+   * Boolean to enable Logging metrics reporting
+   * {@value}
+   */
+  String METRICS_LOGGING_ENABLED =
+      METRICS_PREFIX + "logging.enabled";
+  
+  /**
+   * String name of log to log to
+   * {@value}
+   */
+  String METRICS_LOGGING_LOG =
+      METRICS_PREFIX + "logging.log.name";
+
+  /**
+   * Default log name: {@value}
+   */
+  String METRICS_DEFAULT_LOG = 
+      "org.apache.slider.metrics.log";
+
+
+  /**
+   * Int log interval in seconds
+   * {@value}
+   */
+  String METRICS_LOGGING_LOG_INTERVAL =
+      METRICS_PREFIX + "logging.interval.minutes";
+
+
+  /**
+   * Default log interval: {@value}.
+   * This is a big interval as in a long lived service, log overflows are easy
+   * to create. 
+   */
+  int METRICS_DEFAULT_LOG_INTERVAL = 60;
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/PrefixedMetricsSet.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/PrefixedMetricsSet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/PrefixedMetricsSet.java
new file mode 100644
index 0000000..e9ad46a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/PrefixedMetricsSet.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.management;
+
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricSet;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * From an existing metrics set, generate a new metrics set with the
+ * prefix in front of every key.
+ *
+ * The prefix is added directly: if you want a '.' between prefix and metric
+ * keys, include it in the prefix.
+ */
+public class PrefixedMetricsSet implements MetricSet {
+
+  private final String prefix;
+  private final MetricSet source;
+
+  public PrefixedMetricsSet(String prefix, MetricSet source) {
+    this.prefix = prefix;
+    this.source = source;
+  }
+
+  @Override
+  public Map<String, Metric> getMetrics() {
+    Map<String, Metric> sourceMetrics = source.getMetrics();
+    Map<String, Metric> metrics = new HashMap<>(sourceMetrics.size());
+    for (Map.Entry<String, Metric> entry : sourceMetrics.entrySet()) {
+      metrics.put(prefix + "." + entry.getKey(), entry.getValue());
+    }
+    return metrics;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/RangeLimitedCounter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/RangeLimitedCounter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/RangeLimitedCounter.java
new file mode 100644
index 0000000..80e88fc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/RangeLimitedCounter.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.management;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Counting;
+import com.codahale.metrics.Metric;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This is a counter whose range can be given a min and a max
+ */
+public class RangeLimitedCounter implements Metric, Counting {
+
+  private final AtomicLong value;
+  private final long min, max;
+
+  /**
+   * Instantiate
+   * @param val current value
+   * @param min minimum value
+   * @param max max value (or 0 for no max)
+   */
+  public RangeLimitedCounter(long val, long min, long max) {
+    this.value = new AtomicLong(val);
+    this.min = min;
+    this.max = max;
+  }
+
+  /**
+   * Set to a new value. If below the min, set to the minimum. If the max is non
+   * zero and the value is above that maximum, set it to the maximum instead.
+   * @param val value
+   */
+  public synchronized void set(long val) {
+    if (val < min) {
+      val = min;
+    } else if (max > 0  && val > max) {
+      val = max;
+    }
+    value.set(val);
+  }
+
+  public void inc() {
+    inc(1);
+  }
+
+  public void dec() {
+    dec(1);
+  }
+
+  public synchronized void inc(int delta) {
+    set(value.get() + delta);
+  }
+
+  public synchronized void dec(int delta) {
+    set(value.get() - delta);
+  }
+
+  public long get() {
+    return value.get();
+  }
+
+  @Override
+  public long getCount() {
+    return value.get();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/RecordedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/RecordedEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/RecordedEvent.java
new file mode 100644
index 0000000..d48d337
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/RecordedEvent.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.management;
+
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import org.codehaus.jackson.map.annotate.JsonSerialize;
+
+import java.text.DateFormat;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+public class RecordedEvent {
+  private static final DateFormat dateFormat = DateFormat.getDateInstance();
+  public long id;
+  public String name;
+  public long timestamp;
+  public String time;
+  public String category;
+  public String host;
+  public int role;
+  public String text;
+
+  public RecordedEvent() {
+  }
+
+  /**
+   * Create an event. The timestamp is also converted to a time string
+   * @param id id counter
+   * @param name event name
+   * @param timestamp timestamp. If non-zero, is used to build the {@code time} text field.
+   * @param category even category
+   * @param text arbitrary text
+   */
+  public RecordedEvent(long id, String name, long timestamp, String category, String text) {
+    this.id = id;
+    this.name = name;
+    this.timestamp = timestamp;
+    this.time = timestamp > 0 ? dateFormat.format(timestamp) : "";
+    this.category = category;
+    this.text = text;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/Timestamp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/Timestamp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/Timestamp.java
new file mode 100644
index 0000000..c30e749
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/Timestamp.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.management;
+
+/**
+ * A timestamp metric
+ */
+public class Timestamp extends LongGauge {
+
+  public Timestamp(long val) {
+    super(val);
+  }
+
+  public Timestamp() {
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/YarnServiceHealthCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/YarnServiceHealthCheck.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/YarnServiceHealthCheck.java
new file mode 100644
index 0000000..936563c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/management/YarnServiceHealthCheck.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.management;
+
+import com.codahale.metrics.health.HealthCheck;
+import org.apache.hadoop.service.Service;
+
+public class YarnServiceHealthCheck extends HealthCheck {
+  
+  private final Service service;
+
+  public YarnServiceHealthCheck(Service service) {
+    this.service = service;
+  }
+
+  @Override
+  protected Result check() throws Exception {
+    return service.isInState(Service.STATE.STARTED)
+           ? Result.healthy()
+           : Result.unhealthy("Service is not running: %s", service);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosEntry.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosEntry.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosEntry.java
new file mode 100644
index 0000000..a397e19
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosEntry.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.monkey;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.MetricRegistry;
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang.StringUtils;
+import org.apache.slider.api.InternalKeys;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Entry in the chaos list
+ */
+public class ChaosEntry {
+
+  protected static final Logger log =
+      LoggerFactory.getLogger(ChaosEntry.class);
+  public final String name;
+  public final ChaosTarget target;
+  public final long probability;
+
+  private final Counter invocationCounter;
+
+
+  /**
+   * Constructor -includes validation of all arguments
+   * @param name entry name
+   * @param target target
+   * @param probability probability of occurring
+   */
+  public ChaosEntry(String name, ChaosTarget target, long probability,
+      MetricRegistry metrics) {
+    Preconditions.checkArgument(!StringUtils.isEmpty(name), "missing name");
+    Preconditions.checkArgument(target != null, "null target");
+    Preconditions.checkArgument(probability > 0, "negative probability");
+    Preconditions.checkArgument(probability <= InternalKeys.PROBABILITY_PERCENT_100,
+        "probability over 100%: "+ probability);
+    this.name = name;
+    this.target = target;
+    this.probability = probability;
+    invocationCounter =
+        metrics.counter(MetricRegistry.name(ChaosEntry.class, name));
+  }
+
+  /**
+   * Trigger the chaos action
+   */
+  public void invokeChaos() {
+    log.info("Invoking {}", name);
+    invocationCounter.inc();
+    target.chaosAction();
+  }
+
+  /**
+   * Invoke Chaos if the trigger value is in range of the probability
+   * @param value trigger value, 0-10K
+   * @return true if the chaos method was invoked
+   */
+  public boolean maybeInvokeChaos(long value) {
+    log.debug("Probability {} trigger={}", probability, value);
+    if (value < probability) {
+      invokeChaos();
+      return true;
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosKillAM.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosKillAM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosKillAM.java
new file mode 100644
index 0000000..3c1a914
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosKillAM.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.monkey;
+
+import org.apache.slider.server.appmaster.actions.ActionHalt;
+import org.apache.slider.server.appmaster.actions.QueueAccess;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Kill the AM
+ */
+public class ChaosKillAM implements ChaosTarget {
+
+  public static final int DELAY = 1000;
+  private final QueueAccess queues;
+  private final int exitCode;
+
+  public ChaosKillAM(QueueAccess queues, int exitCode) {
+    this.queues = queues;
+    this.exitCode = exitCode;
+  }
+
+  /**
+   * Trigger a delayed halt
+   */
+  @Override
+  public void chaosAction() {
+    queues.schedule(new ActionHalt(exitCode, "Chaos invoked halt", DELAY,
+        TimeUnit.MILLISECONDS));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosKillContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosKillContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosKillContainer.java
new file mode 100644
index 0000000..ae38e4c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosKillContainer.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.monkey;
+
+import com.google.common.base.Preconditions;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.server.appmaster.actions.ActionKillContainer;
+import org.apache.slider.server.appmaster.actions.QueueAccess;
+import org.apache.slider.server.appmaster.operations.RMOperationHandler;
+import org.apache.slider.server.appmaster.state.AppState;
+import org.apache.slider.server.appmaster.state.RoleInstance;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Kill a container
+ */
+public class ChaosKillContainer implements ChaosTarget {
+  protected static final Logger log =
+      LoggerFactory.getLogger(ChaosKillContainer.class);
+  public static final int DELAY = 100;
+  private final AppState appState;
+  private final QueueAccess queues;
+  private final Random random = new Random();
+  private final RMOperationHandler operationHandler;
+
+  public ChaosKillContainer(AppState appState,
+      QueueAccess queues,
+      RMOperationHandler operationHandler) {
+    Preconditions.checkNotNull(appState);
+    Preconditions.checkNotNull(queues);
+    this.appState = appState;
+    this.queues = queues;
+    this.operationHandler = operationHandler;
+  }
+
+  /**
+   * Trigger a container kill 
+   */
+  @Override
+  public void chaosAction() {
+    List<RoleInstance> liveContainers =
+        appState.cloneLiveContainerInfoList();
+    // purge any and all components which declare that they are an AM
+    ListIterator<RoleInstance> containers =
+        liveContainers.listIterator();
+    while (containers.hasNext()) {
+      RoleInstance instance = containers.next();
+      if (SliderKeys.COMPONENT_AM.equals(instance.role)) {
+        containers.remove();
+      }
+    }
+    int size = liveContainers.size();
+    if (size > 0) {
+      int target = random.nextInt(size);
+      RoleInstance roleInstance = liveContainers.get(target);
+      log.info("Killing {}", roleInstance);
+
+      queues.schedule(new ActionKillContainer(roleInstance.getId(),
+          DELAY, TimeUnit.MILLISECONDS, operationHandler));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosMonkeyService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosMonkeyService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosMonkeyService.java
new file mode 100644
index 0000000..8948f0d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosMonkeyService.java
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.monkey;
+
+import com.codahale.metrics.MetricRegistry;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.slider.api.InternalKeys;
+import org.apache.slider.server.appmaster.actions.QueueAccess;
+import org.apache.slider.server.appmaster.actions.RenewingAction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A chaos monkey service which will invoke ChaosTarget events 
+ */
+public class ChaosMonkeyService extends AbstractService {
+  protected static final Logger log =
+      LoggerFactory.getLogger(ChaosMonkeyService.class);
+
+  private final MetricRegistry metrics;
+  private final QueueAccess queues;
+  private final Random random = new Random();
+
+  private final List<ChaosEntry> chaosEntries =
+      new ArrayList<ChaosEntry>();
+
+  public ChaosMonkeyService(MetricRegistry metrics, QueueAccess queues) {
+    super("ChaosMonkeyService");
+    this.metrics = metrics;
+    this.queues = queues;
+  }
+
+  /**
+   * Add a target ... it is only added if <code>probability &gt; 0</code>
+   * @param name name
+   * @param target chaos target
+   * @param probability probability
+   */
+  public synchronized void addTarget(String name,
+      ChaosTarget target, long probability) {
+    if (probability > 0) {
+      log.info("Adding {} with probability {}", name,
+          ((double)probability) / InternalKeys.PROBABILITY_PERCENT_1);
+      chaosEntries.add(new ChaosEntry(name, target, probability, metrics));
+    } else {
+      log.debug("Action {} not enabled", name);
+    }
+  }
+
+  /**
+   * Get the number of targets in the list
+   * @return the count of added targets
+   */
+  public int getTargetCount() {
+    return chaosEntries.size();
+  }
+  
+  /**
+   * Iterate through all the entries and invoke chaos on those wanted
+   */
+  public void play() {
+    for (ChaosEntry chaosEntry : chaosEntries) {
+      long p = randomPercentage();
+      chaosEntry.maybeInvokeChaos(p);
+    }
+  }
+
+  public int randomPercentage() {
+    return random.nextInt(InternalKeys.PROBABILITY_PERCENT_100);
+  }
+
+  /**
+   * Check for callers to see if chaos should be triggered; shares the
+   * same random number source as the rest of the monkey entries
+   * @param probability probability 
+   * @return true if the action should happen
+   */
+  public boolean chaosCheck(long probability) {
+    return randomPercentage() < probability; 
+  }
+  
+  /**
+   * Schedule the monkey
+   *
+   * @param delay initial delay
+   * @param timeUnit time unit
+   * @return true if it was scheduled (i.e. 1+ action) and interval > 0
+   */
+  public boolean schedule(long delay, long interval, TimeUnit timeUnit) {
+    if (interval > 0 && !chaosEntries.isEmpty()) {
+      queues.schedule(getChaosAction(delay, interval, timeUnit));
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Get the chaos action
+   *
+   * @param delay
+   * @param timeUnit time unit
+   * @return the action to schedule
+   */
+  public RenewingAction<MonkeyPlayAction> getChaosAction(long delay,
+      long interval,
+      TimeUnit timeUnit) {
+    RenewingAction<MonkeyPlayAction> action = new RenewingAction<MonkeyPlayAction>(
+        new MonkeyPlayAction(this, 0, TimeUnit.MILLISECONDS),
+        delay,
+        interval,
+        timeUnit,
+        0
+    );
+    return action;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosTarget.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosTarget.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosTarget.java
new file mode 100644
index 0000000..1c3a9ac
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/ChaosTarget.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.monkey;
+
+public interface ChaosTarget {
+
+  public void chaosAction();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/MonkeyPlayAction.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/MonkeyPlayAction.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/MonkeyPlayAction.java
new file mode 100644
index 0000000..20e4466
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/monkey/MonkeyPlayAction.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.monkey;
+
+import org.apache.slider.server.appmaster.SliderAppMaster;
+import org.apache.slider.server.appmaster.actions.AsyncAction;
+import org.apache.slider.server.appmaster.actions.QueueAccess;
+import org.apache.slider.server.appmaster.state.AppState;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Queueable action which calls {@link ChaosMonkeyService#play()} when
+ * executed.
+ */
+public class MonkeyPlayAction extends AsyncAction {
+
+  private final ChaosMonkeyService monkey;
+
+  public MonkeyPlayAction(ChaosMonkeyService monkey, long delay,
+      TimeUnit timeUnit) {
+    super("chaos monkey", delay, timeUnit);
+    this.monkey = monkey;
+  }
+
+  @Override
+  public void execute(SliderAppMaster appMaster,
+      QueueAccess queueService,
+      AppState appState) throws Exception {
+    monkey.play();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/AbstractRMOperation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/AbstractRMOperation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/AbstractRMOperation.java
new file mode 100644
index 0000000..ed3f197
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/AbstractRMOperation.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.operations;
+
+public abstract class AbstractRMOperation {
+
+  /**
+   * Execute the operation
+   * @param asyncRMClient client
+   * @param handler handler to perform the execution
+   */
+  public abstract void execute(RMOperationHandlerActions handler);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/AsyncRMOperationHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/AsyncRMOperationHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/AsyncRMOperationHandler.java
new file mode 100644
index 0000000..03231ef
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/AsyncRMOperationHandler.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.operations;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.client.api.AMRMClient;
+import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Hands off RM operations to the Resource Manager.
+ */
+public class AsyncRMOperationHandler extends RMOperationHandler {
+  protected static final Logger log =
+    LoggerFactory.getLogger(AsyncRMOperationHandler.class);
+  private final AMRMClientAsync client;
+  private final Resource maxResources;
+
+  public AsyncRMOperationHandler(AMRMClientAsync client, Resource maxResources) {
+    this.client = client;
+    this.maxResources = maxResources;
+  }
+
+  @Override
+  public int cancelContainerRequests(Priority priority1,
+      Priority priority2,
+      int count) {
+    // need to revoke a previously issued container request
+    // so enum the sets and pick some
+    int remaining = cancelSinglePriorityRequests(priority1, count);
+    if (priority2 != null) {
+      remaining = cancelSinglePriorityRequests(priority2, remaining);
+    }
+
+    return remaining;
+  }
+
+  /**
+   * Cancel just one of the priority levels
+   * @param priority priority to cancel
+   * @param count count to cancel
+   * @return number of requests cancelled
+   */
+  @SuppressWarnings("unchecked")
+  protected int cancelSinglePriorityRequests(Priority priority,
+      int count) {
+    List<Collection<AMRMClient.ContainerRequest>> requestSets =
+        client.getMatchingRequests(priority, "", maxResources);
+    if (count <= 0) {
+      return 0;
+    }
+    int remaining = count;
+    for (Collection<AMRMClient.ContainerRequest> requestSet : requestSets) {
+      if (remaining == 0) {
+        break;
+      }
+      for (AMRMClient.ContainerRequest request : requestSet) {
+        if (remaining == 0) {
+          break;
+        }
+        // a single release
+        cancelSingleRequest(request);
+        remaining --;
+      }
+    }
+    return remaining;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public void cancelSingleRequest(AMRMClient.ContainerRequest request) {
+    // a single release
+    client.removeContainerRequest(request);
+  }
+
+  @Override
+  public void releaseAssignedContainer(ContainerId containerId) {
+    log.debug("Releasing container {}", containerId);
+
+    client.releaseAssignedContainer(containerId);
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public void addContainerRequest(AMRMClient.ContainerRequest req) {
+    client.addContainerRequest(req);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/CancelSingleRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/CancelSingleRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/CancelSingleRequest.java
new file mode 100644
index 0000000..d7673d3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/CancelSingleRequest.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.operations;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.yarn.client.api.AMRMClient;
+import org.apache.slider.server.appmaster.state.ContainerPriority;
+
+/**
+ * Cancel a container request
+ */
+public class CancelSingleRequest extends AbstractRMOperation {
+
+  private final AMRMClient.ContainerRequest request;
+
+  public CancelSingleRequest(AMRMClient.ContainerRequest request) {
+    Preconditions.checkArgument(request != null, "Null container request");
+    this.request = request;
+  }
+
+  @Override
+  public void execute(RMOperationHandlerActions handler) {
+    handler.cancelSingleRequest(request);
+  }
+
+  public AMRMClient.ContainerRequest getRequest() {
+    return request;
+  }
+
+  @Override
+  public String toString() {
+    return "Cancel container request"
+        + " for :" + ContainerPriority.toString(request.getPriority())
+        + " request " + request;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/ContainerReleaseOperation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/ContainerReleaseOperation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/ContainerReleaseOperation.java
new file mode 100644
index 0000000..4271d50
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/ContainerReleaseOperation.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.operations;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.slider.server.appmaster.state.ContainerPriority;
+
+public class ContainerReleaseOperation extends AbstractRMOperation {
+
+  private final ContainerId containerId;
+
+  public ContainerReleaseOperation(ContainerId containerId) {
+    Preconditions.checkArgument(containerId != null, "Null containerId");
+    this.containerId = containerId;
+  }
+
+  public ContainerId getContainerId() {
+    return containerId;
+  }
+
+  @Override
+  public void execute(RMOperationHandlerActions handler) {
+    handler.releaseAssignedContainer(containerId);
+  }
+
+  @Override
+  public String toString() {
+    return "release container " + containerId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/ContainerRequestOperation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/ContainerRequestOperation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/ContainerRequestOperation.java
new file mode 100644
index 0000000..e29ddd0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/ContainerRequestOperation.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.operations;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.client.api.AMRMClient;
+import org.apache.slider.server.appmaster.state.ContainerPriority;
+
+/**
+ * A container request operation
+ */
+public class ContainerRequestOperation extends AbstractRMOperation {
+
+  private final AMRMClient.ContainerRequest request;
+
+  public ContainerRequestOperation(AMRMClient.ContainerRequest request) {
+    Preconditions.checkArgument(request != null, "Null container request");
+    this.request = request;
+  }
+
+  public AMRMClient.ContainerRequest getRequest() {
+    return request;
+  }
+
+  public Priority getPriority() {
+    return request.getPriority();
+  }
+
+  public  boolean getRelaxLocality() {
+    return request.getRelaxLocality();
+  }
+
+  @Override
+  public void execute(RMOperationHandlerActions handler) {
+    handler.addContainerRequest(request);
+  }
+
+  @Override
+  public String toString() {
+    return "request container for role "
+        + ContainerPriority.toString(getPriority())
+        + " request " + request
+        + " relaxLocality=" + getRelaxLocality();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/ProviderNotifyingOperationHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/ProviderNotifyingOperationHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/ProviderNotifyingOperationHandler.java
new file mode 100644
index 0000000..184a36a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/ProviderNotifyingOperationHandler.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.operations;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.client.api.AMRMClient;
+import org.apache.slider.providers.ProviderService;
+
+public class ProviderNotifyingOperationHandler extends RMOperationHandler {
+  
+  private final ProviderService providerService;
+
+  public ProviderNotifyingOperationHandler(ProviderService providerService) {
+    this.providerService = providerService;
+  }
+
+  @Override
+  public void releaseAssignedContainer(ContainerId containerId) {
+    providerService.releaseAssignedContainer(containerId);
+  }
+
+  @Override
+  public void addContainerRequest(AMRMClient.ContainerRequest req) {
+    providerService.addContainerRequest(req);
+  }
+
+  @Override
+  public int cancelContainerRequests(Priority priority1,
+      Priority priority2,
+      int count) {
+    return providerService.cancelContainerRequests(priority1, priority2, count);
+  }
+
+  @Override
+  public void cancelSingleRequest(AMRMClient.ContainerRequest request) {
+    providerService.cancelSingleRequest(request);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/RMOperationHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/RMOperationHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/RMOperationHandler.java
new file mode 100644
index 0000000..d0d038a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/RMOperationHandler.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.operations;
+
+import java.util.List;
+
+public abstract class RMOperationHandler implements RMOperationHandlerActions {
+
+  @Override
+  public void execute(List<AbstractRMOperation> operations) {
+    for (AbstractRMOperation operation : operations) {
+      operation.execute(this);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/RMOperationHandlerActions.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/RMOperationHandlerActions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/RMOperationHandlerActions.java
new file mode 100644
index 0000000..b7794ed
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/operations/RMOperationHandlerActions.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.operations;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.client.api.AMRMClient;
+
+import java.util.List;
+
+public interface RMOperationHandlerActions {
+
+  /**
+   * Release an assigned container
+   * @param containerId container
+   */
+  void releaseAssignedContainer(ContainerId containerId);
+
+  /**
+   * Issue a container request
+   * @param request
+   */
+  void addContainerRequest(AMRMClient.ContainerRequest request);
+
+  /**
+   * Cancel a specific request
+   * @param request request to cancel
+   */
+  void cancelSingleRequest(AMRMClient.ContainerRequest request);
+
+  /**
+   * Remove a container request
+   * @param priority1 priority to remove at
+   * @param priority2 second priority to target
+   * @param count number to remove
+   */
+  int cancelContainerRequests(Priority priority1, Priority priority2, int count);
+
+  /**
+   * Execute an entire list of operations
+   * @param operations ops
+   */
+  void execute(List<AbstractRMOperation> operations);
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[58/76] [abbrv] hadoop git commit: YARN-5909. Remove agent related code in slider AM. Contributed by Jian He

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
index a0fe310..bd4d2bf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
@@ -23,7 +23,6 @@ import org.apache.slider.server.appmaster.actions.QueueAccess;
 import org.apache.slider.server.appmaster.management.MetricsAndMonitoring;
 import org.apache.slider.server.appmaster.state.RoleStatus;
 import org.apache.slider.server.appmaster.state.StateAccessForProviders;
-import org.apache.slider.server.appmaster.web.rest.agent.AgentRestOperations;
 import org.apache.slider.server.appmaster.web.rest.application.resources.ContentCache;
 import org.apache.slider.server.services.security.CertificateManager;
 import org.slf4j.Logger;
@@ -97,11 +96,6 @@ public class WebAppApiImpl implements WebAppApi {
   }
 
   @Override
-  public AgentRestOperations getAgentRestOperations() {
-    return provider.getAgentRestOperations();
-  }
-
-  @Override
   public RegistryOperations getRegistryOperations() {
     return registryOperations;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java
index 03bf703..aed87d8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java
@@ -19,7 +19,6 @@ package org.apache.slider.server.appmaster.web.rest;
 import com.google.inject.Inject;
 import com.google.inject.Singleton;
 import org.apache.slider.server.appmaster.web.WebAppApi;
-import org.apache.slider.server.appmaster.web.rest.agent.AgentResource;
 import org.apache.slider.server.appmaster.web.rest.application.ApplicationResource;
 import org.apache.slider.server.appmaster.web.rest.management.ManagementResource;
 import org.apache.slider.server.appmaster.web.rest.publisher.PublisherResource;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentCommandType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentCommandType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentCommandType.java
deleted file mode 100644
index 17cd8f2..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentCommandType.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.server.appmaster.web.rest.agent;
-
-public enum AgentCommandType {
-  EXECUTION_COMMAND,
-  STATUS_COMMAND,
-  REGISTRATION_COMMAND
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentEnv.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentEnv.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentEnv.java
deleted file mode 100644
index 781ae00..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentEnv.java
+++ /dev/null
@@ -1,376 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.server.appmaster.web.rest.agent;
-
-import com.google.gson.annotations.SerializedName;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-@JsonIgnoreProperties(ignoreUnknown = true)
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-public class AgentEnv {
-
-  /**
-   * Various directories, configurable in <code>ambari-agent.ini</code>
-   */
-  private Directory[] stackFoldersAndFiles = new Directory[0];
-
-  /**
-   * Directories that match name <code>/etc/alternatives/*conf</code>
-   */
-  private Alternative[] alternatives = new Alternative[0];
-
-  /**
-   * List of existing users
-   */
-  private ExistingUser[] existingUsers = new ExistingUser[0];
-
-  /**
-   * List of repos
-   */
-  private String[] existingRepos = new String[0];
-
-  /**
-   * List of packages
-   */
-  private PackageDetail[] installedPackages = new PackageDetail[0];
-
-  /**
-   * The host health report
-   */
-  private HostHealth hostHealth = new HostHealth();
-
-  private Integer umask;
-
-  private Boolean iptablesIsRunning;
-
-  public Integer getUmask() {
-    return umask;
-  }
-
-  public void setUmask(Integer umask) {
-    this.umask = umask;
-  }
-
-  public Directory[] getStackFoldersAndFiles() {
-    return stackFoldersAndFiles;
-  }
-
-  public void setStackFoldersAndFiles(Directory[] dirs) {
-    stackFoldersAndFiles = dirs;
-  }
-
-  public void setExistingUsers(ExistingUser[] users) {
-    existingUsers = users;
-  }
-
-  public ExistingUser[] getExistingUsers() {
-    return existingUsers;
-  }
-
-  public void setAlternatives(Alternative[] dirs) {
-    alternatives = dirs;
-  }
-
-  public Alternative[] getAlternatives() {
-    return alternatives;
-  }
-
-  public void setExistingRepos(String[] repos) {
-    existingRepos = repos;
-  }
-
-  public String[] getExistingRepos() {
-    return existingRepos;
-  }
-
-  public void setInstalledPackages(PackageDetail[] packages) {
-    installedPackages = packages;
-  }
-
-  public PackageDetail[] getInstalledPackages() {
-    return installedPackages;
-  }
-
-  public void setHostHealth(HostHealth healthReport) {
-    hostHealth = healthReport;
-  }
-
-  public HostHealth getHostHealth() {
-    return hostHealth;
-  }
-
-  public Boolean getIptablesIsRunning() {
-    return iptablesIsRunning;
-  }
-
-  public void setIptablesIsRunning(Boolean iptablesIsRunning) {
-    this.iptablesIsRunning = iptablesIsRunning;
-  }
-
-  public static class HostHealth {
-    /**
-     * Java processes running on the system.  Default empty array.
-     */
-    @SerializedName("activeJavaProcs")
-    private JavaProc[] activeJavaProcs = new JavaProc[0];
-
-    /**
-     * The current time when agent send the host check report
-     */
-    @SerializedName("agentTimeStampAtReporting")
-    private long agentTimeStampAtReporting = 0;
-
-    /**
-     * The current time when host check report was received
-     */
-    @SerializedName("serverTimeStampAtReporting")
-    private long serverTimeStampAtReporting = 0;
-
-    /**
-     * Live services running on the agent
-     */
-    @SerializedName("liveServices")
-    private LiveService[] liveServices = new LiveService[0];
-
-    public void setAgentTimeStampAtReporting(long currentTime) {
-      agentTimeStampAtReporting = currentTime;
-    }
-
-    public long getAgentTimeStampAtReporting() {
-      return agentTimeStampAtReporting;
-    }
-
-    public void setServerTimeStampAtReporting(long currentTime) {
-      serverTimeStampAtReporting = currentTime;
-    }
-
-    public long getServerTimeStampAtReporting() {
-      return serverTimeStampAtReporting;
-    }
-
-    public void setActiveJavaProcs(JavaProc[] procs) {
-      activeJavaProcs = procs;
-    }
-
-    public JavaProc[] getActiveJavaProcs() {
-      return activeJavaProcs;
-    }
-
-    public void setLiveServices(LiveService[] services) {
-      liveServices = services;
-    }
-
-    public LiveService[] getLiveServices() {
-      return liveServices;
-    }
-  }
-
-  public static class PackageDetail {
-    @SerializedName("name")
-    private String pkgName;
-    @SerializedName("version")
-    private String pkgVersion;
-    @SerializedName("repoName")
-    private String pkgRepoName;
-
-    public void setName(String name) {
-      pkgName = name;
-    }
-
-    public String getName() {
-      return pkgName;
-    }
-
-    public void setVersion(String version) {
-      pkgVersion = version;
-    }
-
-    public String getVersion() {
-      return pkgVersion;
-    }
-
-    public void setRepoName(String repoName) {
-      pkgRepoName = repoName;
-    }
-
-    public String getRepoName() {
-      return pkgRepoName;
-    }
-  }
-
-  /**
-   * Represents information about a directory of interest.
-   */
-  public static class Directory {
-    @SerializedName("name")
-    private String dirName;
-    @SerializedName("type")
-    private String dirType;
-
-    public void setName(String name) {
-      dirName = name;
-    }
-
-    public String getName() {
-      return dirName;
-    }
-
-    public void setType(String type) {
-      dirType = type;
-    }
-
-    public String getType() {
-      return dirType;
-    }
-  }
-
-  /**
-   * Represents information about running java processes.
-   */
-  public static class JavaProc {
-    @SerializedName("user")
-    private String user;
-    @SerializedName("pid")
-    private int pid = 0;
-    @SerializedName("hadoop")
-    private boolean is_hadoop = false;
-    @SerializedName("command")
-    private String command;
-
-    public void setUser(String user) {
-      this.user = user;
-    }
-
-    public String getUser() {
-      return user;
-    }
-
-    public void setPid(int pid) {
-      this.pid = pid;
-    }
-
-    public int getPid() {
-      return pid;
-    }
-
-    public void setHadoop(boolean hadoop) {
-      is_hadoop = hadoop;
-    }
-
-    public boolean isHadoop() {
-      return is_hadoop;
-    }
-
-    public void setCommand(String cmd) {
-      command = cmd;
-    }
-
-    public String getCommand() {
-      return command;
-    }
-  }
-
-  public static class Alternative {
-    @SerializedName("name")
-    private String altName;
-    @SerializedName("target")
-    private String altTarget;
-
-    public void setName(String name) {
-      altName = name;
-    }
-
-    public String getName() {
-      return altName;
-    }
-
-    public void setTarget(String target) {
-      altTarget = target;
-    }
-
-    public String getTarget() {
-      return altTarget;
-    }
-  }
-
-  public static class LiveService {
-    @SerializedName("name")
-    private String svcName;
-    @SerializedName("status")
-    private String svcStatus;
-    @SerializedName("desc")
-    private String svcDesc;
-
-    public void setName(String name) {
-      svcName = name;
-    }
-
-    public String getName() {
-      return svcName;
-    }
-
-    public void setStatus(String status) {
-      svcStatus = status;
-    }
-
-    public String getStatus() {
-      return svcStatus;
-    }
-
-    public void setDesc(String desc) {
-      svcDesc = desc;
-    }
-
-    public String getDesc() {
-      return svcDesc;
-    }
-  }
-
-  public static class ExistingUser {
-    @SerializedName("name")
-    private String name;
-    @SerializedName("homeDir")
-    private String homeDir;
-    @SerializedName("status")
-    private String status;
-
-    public void setUserName(String userName) {
-      name = userName;
-    }
-
-    public String getUserName() {
-      return name;
-    }
-
-    public void setUserHomeDir(String userHomeDir) {
-      homeDir = userHomeDir;
-    }
-
-    public String getUserHomeDir() {
-      return homeDir;
-    }
-
-    public void setUserStatus(String userStatus) {
-      status = userStatus;
-    }
-
-    public String getUserStatus() {
-      return status;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentResource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentResource.java
deleted file mode 100644
index 20ef068..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentResource.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.server.appmaster.web.rest.agent;
-
-import org.apache.slider.server.appmaster.web.WebAppApi;
-import org.apache.slider.server.services.security.SignCertResponse;
-import org.apache.slider.server.services.security.SignMessage;
-import org.apache.slider.server.appmaster.web.rest.AbstractSliderResource;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-import javax.ws.rs.Consumes;
-import javax.ws.rs.GET;
-import javax.ws.rs.POST;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.Produces;
-import javax.ws.rs.core.Context;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.Response;
-
-/**
- *
- */
-@JsonIgnoreProperties(ignoreUnknown = true)
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-public class AgentResource extends AbstractSliderResource {
-
-  private String agent_name;
-
-  public AgentResource(WebAppApi slider) {
-    super(slider);
-  }
-
-  private void init(HttpServletResponse res) {
-    res.setContentType(null);
-  }
-
-  @GET
-  @Path("/agent/register")
-  public Response endpointAgentRegister() {
-    Response response = Response.status(200).entity("/agent/register").build();
-    return response;
-  }
-
-  @GET
-  @Path("/agent")
-  public Response endpointAgent() {
-    Response response = Response.status(200).entity("/agent").build();
-    return response;
-  }
-  @GET
-  @Path("/")
-  public Response endpointRoot() {
-    Response response = Response.status(200).entity("/").build();
-    return response;
-  }
-
-  @POST
-  @Path("/{agent_name: [a-zA-Z][a-zA-Z0-9_-]*}/register")
-  @Consumes({MediaType.APPLICATION_JSON})
-  @Produces({MediaType.APPLICATION_JSON})
-  public RegistrationResponse register(Register registration,
-                                       @Context HttpServletResponse res,
-                                       @PathParam("agent_name") String agent_name) {
-    init(res);
-    this.agent_name = agent_name;
-    AgentRestOperations ops = slider.getAgentRestOperations();
-    return ops.handleRegistration(registration);
-
-  }
-
-  @POST
-  @Path("/{agent_name: [a-zA-Z][a-zA-Z0-9_-]*}/heartbeat")
-  @Consumes(MediaType.APPLICATION_JSON)
-  @Produces({MediaType.APPLICATION_JSON})
-  public HeartBeatResponse heartbeat(HeartBeat message,
-                                     @Context HttpServletResponse res,
-                                     @PathParam("agent_name") String agent_name) {
-    init(res);
-    AgentRestOperations ops = slider.getAgentRestOperations();
-    return ops.handleHeartBeat(message);
-  }
-
-  @GET
-  @Path("/cert/ca")
-  @Produces({MediaType.TEXT_PLAIN})
-  public String downloadSrvrCrt() {
-    return slider.getCertificateManager().getServerCert();
-  }
-
-  @Path("/certs/{hostName}")
-  @POST
-  @Consumes(MediaType.APPLICATION_JSON)
-  @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-  public SignCertResponse signAgentCrt(@PathParam("hostName") String hostname,
-                                       SignMessage message, @Context HttpServletRequest req) {
-    return slider.getCertificateManager().signAgentCrt(hostname,
-                                                       message.getCsr(),
-                                                       message.getPassphrase());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentRestOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentRestOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentRestOperations.java
deleted file mode 100644
index 2891be8..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentRestOperations.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.server.appmaster.web.rest.agent;
-
-/**
- *
- */
-public interface AgentRestOperations {
-
-  RegistrationResponse handleRegistration(Register registration);
-
-  HeartBeatResponse handleHeartBeat(HeartBeat heartBeat);
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebApp.java
deleted file mode 100644
index 3a3b0c0..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebApp.java
+++ /dev/null
@@ -1,258 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.server.appmaster.web.rest.agent;
-
-import com.google.common.base.Preconditions;
-import com.sun.jersey.api.core.ResourceConfig;
-import com.sun.jersey.spi.container.WebApplication;
-import com.sun.jersey.spi.container.servlet.ServletContainer;
-import com.sun.jersey.spi.container.servlet.WebConfig;
-import com.sun.jersey.spi.inject.SingletonTypeInjectableProvider;
-import org.apache.slider.core.conf.MapOperations;
-import org.apache.slider.providers.agent.AgentKeys;
-import org.apache.slider.server.appmaster.web.WebAppApi;
-import org.apache.slider.server.services.security.SecurityUtils;
-import org.mortbay.jetty.Connector;
-import org.mortbay.jetty.Server;
-import org.mortbay.jetty.security.SslSelectChannelConnector;
-import org.mortbay.jetty.servlet.Context;
-import org.mortbay.jetty.servlet.ServletHolder;
-import org.mortbay.thread.QueuedThreadPool;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.ws.rs.ext.Provider;
-import java.io.Closeable;
-import java.io.File;
-import java.io.IOException;
-import java.net.BindException;
-import java.util.Set;
-
-/**
- *
- */
-public class AgentWebApp implements Closeable {
-  protected static final Logger LOG = LoggerFactory.getLogger(AgentWebApp.class);
-  private int port;
-  private int securedPort;
-  private static Server agentServer;
-  public static final String BASE_PATH = "slideragent";
-
-  public static class Builder {
-    final String name;
-    final String wsName;
-    final WebAppApi application;
-    int port;
-    int securedPort;
-    MapOperations configsMap;
-
-    public Builder(String name, String wsName, WebAppApi application) {
-      this.name = name;
-      this.wsName = wsName;
-      this.application = application;
-    }
-
-    public Builder withComponentConfig(MapOperations appMasterConfig) {
-      this.configsMap = appMasterConfig;
-      return this;
-    }
-
-    public Builder withPort (int port) {
-      this.port = port;
-      return this;
-    }
-
-    public Builder withSecuredPort (int securedPort) {
-      this.securedPort = securedPort;
-      return this;
-    }
-
-    public AgentWebApp start() throws IOException {
-      if (configsMap == null) {
-        throw new IllegalStateException("No SSL Configuration Available");
-      }
-
-      agentServer = new Server();
-      agentServer.setThreadPool(
-          new QueuedThreadPool(
-              configsMap.getOptionInt("agent.threadpool.size.max", 25)));
-      agentServer.setStopAtShutdown(true);
-      agentServer.setGracefulShutdown(1000);
-
-      SslSelectChannelConnector ssl1WayConnector = createSSLConnector(false, port);
-      SslSelectChannelConnector ssl2WayConnector =
-          createSSLConnector(Boolean.valueOf(
-              configsMap.getOption(AgentKeys.KEY_AGENT_TWO_WAY_SSL_ENABLED,
-                                   "false")), securedPort);
-      agentServer.setConnectors(new Connector[]{ssl1WayConnector,
-          ssl2WayConnector});
-
-      ServletHolder agent = new ServletHolder(new AgentServletContainer());
-      Context agentRoot = new Context(agentServer, "/", Context.SESSIONS);
-
-      agent.setInitParameter("com.sun.jersey.config.property.resourceConfigClass",
-                             "com.sun.jersey.api.core.PackagesResourceConfig");
-      agent.setInitParameter("com.sun.jersey.config.property.packages",
-                             "org.apache.slider.server.appmaster.web.rest.agent");
-      agent.setInitParameter("com.sun.jersey.api.json.POJOMappingFeature",
-                             "true");
-//      agent.setInitParameter("com.sun.jersey.spi.container.ContainerRequestFilters", "com.sun.jersey.api.container.filter.LoggingFilter");
-//      agent.setInitParameter("com.sun.jersey.spi.container.ContainerResponseFilters", "com.sun.jersey.api.container.filter.LoggingFilter");
-//      agent.setInitParameter("com.sun.jersey.config.feature.Trace", "true");
-      agentRoot.addServlet(agent, "/*");
-
-      try {
-        openListeners();
-        agentServer.start();
-      } catch (IOException e) {
-        LOG.error("Unable to start agent server", e);
-        throw e;
-      } catch (Exception e) {
-        LOG.error("Unable to start agent server", e);
-        throw new IOException("Unable to start agent server: " + e, e);
-      }
-
-      AgentWebApp webApp = new AgentWebApp();
-      webApp.setPort(getConnectorPort(agentServer, 0));
-      webApp.setSecuredPort(getConnectorPort(agentServer, 1));
-      return webApp;
-
-    }
-
-    private void openListeners() throws Exception {
-      // from HttpServer2.openListeners()
-      for (Connector listener : agentServer.getConnectors()) {
-        if (listener.getLocalPort() != -1) {
-          // This listener is either started externally or has been bound
-          continue;
-        }
-        int port = listener.getPort();
-        while (true) {
-          // jetty has a bug where you can't reopen a listener that previously
-          // failed to open w/o issuing a close first, even if the port is changed
-          try {
-            listener.close();
-            listener.open();
-            LOG.info("Jetty bound to port " + listener.getLocalPort());
-            break;
-          } catch (BindException ex) {
-            if (port == 0) {
-              BindException be = new BindException("Port in use: "
-                  + listener.getHost() + ":" + listener.getPort());
-              be.initCause(ex);
-              throw be;
-            }
-          }
-          // try the next port number
-          listener.setPort(++port);
-          Thread.sleep(100);
-        }
-      }
-    }
-
-    private SslSelectChannelConnector createSSLConnector(boolean needClientAuth, int port) {
-      SslSelectChannelConnector sslConnector = new
-          SslSelectChannelConnector();
-
-      String keystore = SecurityUtils.getSecurityDir() +
-                        File.separator + "keystore.p12";
-      String srvrCrtPass = SecurityUtils.getKeystorePass();
-      sslConnector.setKeystore(keystore);
-      sslConnector.setTruststore(keystore);
-      sslConnector.setPassword(srvrCrtPass);
-      sslConnector.setKeyPassword(srvrCrtPass);
-      sslConnector.setTrustPassword(srvrCrtPass);
-      sslConnector.setKeystoreType("PKCS12");
-      sslConnector.setTruststoreType("PKCS12");
-      sslConnector.setNeedClientAuth(needClientAuth);
-
-      sslConnector.setPort(port);
-      sslConnector.setAcceptors(2);
-      return sslConnector;
-    }
-
-    @Provider
-    public class WebAppApiProvider extends
-        SingletonTypeInjectableProvider<javax.ws.rs.core.Context, WebAppApi> {
-
-      public WebAppApiProvider () {
-        super(WebAppApi.class, application);
-      }
-    }
-
-    public class AgentServletContainer extends ServletContainer {
-      public AgentServletContainer() {
-        super();
-      }
-
-      @Override
-      protected void configure(WebConfig wc,
-                               ResourceConfig rc,
-                               WebApplication wa) {
-        super.configure(wc, rc, wa);
-        Set<Object> singletons = rc.getSingletons();
-        singletons.add(new WebAppApiProvider());
-      }
-    }
-
-    private int getConnectorPort(Server webServer, int index) {
-      Preconditions.checkArgument(index >= 0);
-      if (index > webServer.getConnectors().length)
-        throw new IllegalStateException("Illegal connect index requested");
-
-      Connector c = webServer.getConnectors()[index];
-      if (c.getLocalPort() == -1) {
-        // The connector is not bounded
-        throw new IllegalStateException("The connector is not bound to a port");
-      }
-
-      return c.getLocalPort();
-    }
-  }
-
-  public static Builder $for(String name, WebAppApi app, String wsPrefix) {
-    return new Builder(name, wsPrefix, app);
-  }
-
-  public int getPort() {
-    return port;
-  }
-
-  public void setPort(int port) {
-    this.port = port;
-  }
-
-  public void setSecuredPort(int securedPort) {
-    this.securedPort = securedPort;
-  }
-
-  public int getSecuredPort() {
-    return securedPort;
-  }
-
-  public void close() throws IOException{
-    //need to stop server and reset injector
-    try {
-      agentServer.stop();
-    } catch (IOException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new IOException(e.toString(), e);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebServices.java
deleted file mode 100644
index 684ce6f..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebServices.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.server.appmaster.web.rest.agent;
-
-import org.apache.slider.server.appmaster.web.WebAppApi;
-import org.apache.slider.server.appmaster.web.rest.RestPaths;
-
-import javax.ws.rs.Path;
-import javax.ws.rs.core.Context;
-
-/** The available agent REST services exposed by a slider AM. */
-@Path(RestPaths.SLIDER_AGENT_CONTEXT_ROOT)
-public class AgentWebServices {
-  /** AM/WebApp info object */
-  @Context
-  private WebAppApi slider;
-
-  public AgentWebServices() {
-  }
-
-  @Path(RestPaths.SLIDER_SUBPATH_AGENTS)
-  public AgentResource getAgentResource () {
-    return new AgentResource(slider);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/CommandReport.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/CommandReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/CommandReport.java
deleted file mode 100644
index a37e490..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/CommandReport.java
+++ /dev/null
@@ -1,207 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.server.appmaster.web.rest.agent;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.annotate.JsonProperty;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-import java.util.Map;
-
-/**
- *
- */
-@JsonIgnoreProperties(ignoreUnknown = true)
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-public class CommandReport {
-
-  int exitCode;
-  private String role;
-  private String actionId;
-  private String stdout;
-  private String stderr;
-  private String structuredOut;
-  private String status;
-  private String clusterName;
-  private String serviceName;
-  private long taskId;
-  private String roleCommand;
-  private Map<String, String> folders;
-  private Map<String, String> allocatedPorts;
-  private Map<String, Map<String, String>> configurationTags;
-
-  @JsonProperty("taskId")
-  public long getTaskId() {
-    return taskId;
-  }
-
-  @JsonProperty("taskId")
-  public void setTaskId(long taskId) {
-    this.taskId = taskId;
-  }
-
-  @JsonProperty("clusterName")
-  public String getClusterName() {
-    return this.clusterName;
-  }
-
-  @JsonProperty("clusterName")
-  public void setClusterName(String clusterName) {
-    this.clusterName = clusterName;
-  }
-
-  @JsonProperty("actionId")
-  public String getActionId() {
-    return this.actionId;
-  }
-
-  @JsonProperty("actionId")
-  public void setActionId(String actionId) {
-    this.actionId = actionId;
-  }
-
-  @JsonProperty("stderr")
-  public String getStdErr() {
-    return this.stderr;
-  }
-
-  @JsonProperty("stderr")
-  public void setStdErr(String stderr) {
-    this.stderr = stderr;
-  }
-
-  @JsonProperty("exitcode")
-  public int getExitCode() {
-    return this.exitCode;
-  }
-
-  @JsonProperty("exitcode")
-  public void setExitCode(int exitCode) {
-    this.exitCode = exitCode;
-  }
-
-  @JsonProperty("stdout")
-  public String getStdOut() {
-    return this.stdout;
-  }
-
-  @JsonProperty("stdout")
-  public void setStdOut(String stdout) {
-    this.stdout = stdout;
-  }
-
-  @JsonProperty("structuredOut")
-  public String getStructuredOut() {
-    return this.structuredOut;
-  }
-
-  @JsonProperty("structuredOut")
-  public void setStructuredOut(String structuredOut) {
-    this.structuredOut = structuredOut;
-  }
-
-  @JsonProperty("roleCommand")
-  public String getRoleCommand() {
-    return this.roleCommand;
-  }
-
-  @JsonProperty("roleCommand")
-  public void setRoleCommand(String roleCommand) {
-    this.roleCommand = roleCommand;
-  }
-
-  @JsonProperty("role")
-  public String getRole() {
-    return role;
-  }
-
-  @JsonProperty("role")
-  public void setRole(String role) {
-    this.role = role;
-  }
-
-  @JsonProperty("status")
-  public String getStatus() {
-    return status;
-  }
-
-  @JsonProperty("status")
-  public void setStatus(String status) {
-    this.status = status;
-  }
-
-  @JsonProperty("serviceName")
-  public String getServiceName() {
-    return serviceName;
-  }
-
-  @JsonProperty("serviceName")
-  public void setServiceName(String serviceName) {
-    this.serviceName = serviceName;
-  }
-
-  /** @return the config tags that match this command, or <code>null</code> if none are present */
-  @JsonProperty("configurationTags")
-  public Map<String, Map<String, String>> getConfigurationTags() {
-    return configurationTags;
-  }
-
-  /** @param tags the config tags that match this command */
-  @JsonProperty("configurationTags")
-  public void setConfigurationTags(Map<String, Map<String, String>> tags) {
-    configurationTags = tags;
-  }
-
-  /** @return the allocated ports, or <code>null</code> if none are present */
-  @JsonProperty("allocatedPorts")
-  public Map<String, String> getAllocatedPorts() {
-    return allocatedPorts;
-  }
-
-  /** @param ports allocated ports */
-  @JsonProperty("allocatedPorts")
-  public void setAllocatedPorts(Map<String, String> ports) {
-    this.allocatedPorts = ports;
-  }
-
-  /** @return the folders, or <code>null</code> if none are present */
-  @JsonProperty("folders")
-  public Map<String, String> getFolders() {
-    return folders;
-  }
-
-  /** @param folders allocated ports */
-  @JsonProperty("folders")
-  public void setFolders(Map<String, String> folders) {
-    this.folders = folders;
-  }
-
-  @Override
-  public String toString() {
-    return "CommandReport{" +
-           "role='" + role + '\'' +
-           ", actionId='" + actionId + '\'' +
-           ", status='" + status + '\'' +
-           ", exitCode=" + exitCode +
-           ", clusterName='" + clusterName + '\'' +
-           ", serviceName='" + serviceName + '\'' +
-           ", taskId=" + taskId +
-           ", roleCommand=" + roleCommand +
-           ", configurationTags=" + configurationTags +
-           '}';
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/ComponentStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/ComponentStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/ComponentStatus.java
deleted file mode 100644
index acdc234..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/ComponentStatus.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.server.appmaster.web.rest.agent;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.annotate.JsonProperty;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-import java.util.Map;
-
-/**
- *
- */
-@JsonIgnoreProperties(ignoreUnknown = true)
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-public class ComponentStatus {
-  String componentName;
-  String msg;
-  String status;
-  String serviceName;
-  String clusterName;
-  String roleCommand;
-  String ip;
-  String hostname;
-  @JsonProperty("configurations")
-  private Map<String, Map<String, String>> configurations;
-
-  public String getRoleCommand() {
-    return roleCommand;
-  }
-
-  public void setRoleCommand(String roleCommand) {
-    this.roleCommand = roleCommand;
-  }
-
-  public String getComponentName() {
-    return this.componentName;
-  }
-
-  public void setComponentName(String componentName) {
-    this.componentName = componentName;
-  }
-
-  public String getMessage() {
-    return this.msg;
-  }
-
-  public void setMessage(String msg) {
-    this.msg = msg;
-  }
-
-  public String getStatus() {
-    return this.status;
-  }
-
-  public void setStatus(String status) {
-    this.status = status;
-  }
-
-  public String getServiceName() {
-    return serviceName;
-  }
-
-  public void setServiceName(String serviceName) {
-    this.serviceName = serviceName;
-  }
-
-  public String getClusterName() {
-    return clusterName;
-  }
-
-  public void setClusterName(String clusterName) {
-    this.clusterName = clusterName;
-  }
-
-  /** @return the config tags that match this command, or <code>null</code> if none are present */
-  public Map<String, Map<String, String>> getConfigs() {
-    return configurations;
-  }
-
-  /** @param configs the config tags that match this status */
-  public void setConfigs(Map<String, Map<String, String>> configs) {
-    this.configurations = configs;
-  }
-
-  @Override
-  public String toString() {
-    return "ComponentStatus{" +
-           "componentName='" + componentName + '\'' +
-           ", msg='" + msg + '\'' +
-           ", status='" + status + '\'' +
-           ", serviceName='" + serviceName + '\'' +
-           ", clusterName='" + clusterName + '\'' +
-           ", roleCommand='" + roleCommand + '\'' +
-           ", ip='" + ip + '\'' +
-           ", hostname='" + hostname + '\'' +
-           '}';
-  }
-
-  public String getIp() {
-    return ip;
-  }
-
-  public void setIp(String ip) {
-    this.ip = ip;
-  }
-
-  public String getHostname() {
-    return hostname;
-  }
-
-  public void setHostname(String hostname) {
-    this.hostname = hostname;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/DiskInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/DiskInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/DiskInfo.java
deleted file mode 100644
index 27c4d54..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/DiskInfo.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.server.appmaster.web.rest.agent;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.annotate.JsonProperty;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-/**
- *
- */
-@JsonIgnoreProperties(ignoreUnknown = true)
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-public class DiskInfo {
-  String available;
-  String mountpoint;
-  String device;
-  String used;
-  String percent;
-  String size;
-  String type;
-
-  /**
-   * DiskInfo object that tracks information about a disk.
-   * @param mountpoint
-   * @param available
-   * @param used
-   * @param percent
-   * @param size
-   */
-  public DiskInfo(String device, String mountpoint, String available,
-                  String used, String percent, String size, String type) {
-    this.device = device;
-    this.mountpoint = mountpoint;
-    this.available = available;
-    this.used = used;
-    this.percent = percent;
-    this.size = size;
-    this.type = type;
-  }
-
-  /**
-   * Needed for Serialization
-   */
-  public DiskInfo() {}
-
-  @JsonProperty("available")
-  public void setAvailable(String available) {
-    this.available = available;
-  }
-
-  @JsonProperty("available")
-  public String getAvailable() {
-    return this.available;
-  }
-
-  @JsonProperty("mountpoint")
-  public String getMountPoint() {
-    return this.mountpoint;
-  }
-
-  @JsonProperty("mountpoint")
-  public void setMountPoint(String mountpoint) {
-    this.mountpoint = mountpoint;
-  }
-
-  @JsonProperty("type")
-  public String getType() {
-    return this.type;
-  }
-
-  @JsonProperty("type")
-  public void setType(String type) {
-    this.type = type;
-  }
-
-  @JsonProperty("used")
-  public String getUsed() {
-    return this.used;
-  }
-
-  @JsonProperty("used")
-  public void setUsed(String used) {
-    this.used = used;
-  }
-
-  @JsonProperty("percent")
-  public String getPercent() {
-    return this.percent;
-  }
-
-  @JsonProperty("percent")
-  public void setPercent(String percent) {
-    this.percent = percent;
-  }
-
-  @JsonProperty("size")
-  public String getSize() {
-    return this.size;
-  }
-
-  @JsonProperty("size")
-  public void setSize(String size) {
-    this.size = size;
-  }
-
-  @Override
-  public String toString() {
-    return "available=" + this.available + ",mountpoint=" + this.mountpoint
-           + ",used=" + this.used + ",percent=" + this.percent + ",size=" +
-           this.size + ",device=" + this.device +
-           ",type=" + this.type;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/ExecutionCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/ExecutionCommand.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/ExecutionCommand.java
deleted file mode 100644
index d3864b8..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/ExecutionCommand.java
+++ /dev/null
@@ -1,310 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.server.appmaster.web.rest.agent;
-
-import org.apache.slider.providers.agent.application.metadata.Component;
-import org.apache.slider.providers.agent.application.metadata.DockerContainer;
-import org.apache.slider.providers.agent.application.metadata.DockerContainerInputFile;
-import org.apache.slider.providers.agent.application.metadata.DockerContainerMount;
-import org.apache.slider.providers.agent.application.metadata.DockerContainerPort;
-import org.apache.slider.providers.agent.application.metadata.Metainfo;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.annotate.JsonProperty;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- *
- */
-@JsonIgnoreProperties(ignoreUnknown = true)
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-public class ExecutionCommand {
-  protected static final Logger log =
-      LoggerFactory.getLogger(ExecutionCommand.class);
-  private AgentCommandType commandType = AgentCommandType.EXECUTION_COMMAND;
-  private String clusterName;
-  private long taskId;
-  private String commandId;
-  //TODO Remove hostname from being set in the command
-  private String hostname;
-  private String role;
-  private Map<String, String> hostLevelParams = new HashMap<String, String>();
-  private Map<String, String> roleParams = null;
-  private String roleCommand;
-  private Map<String, Map<String, String>> configurations;
-  private Map<String, Map<String, String>> componentConfigurations;
-  private Map<String, String> commandParams;
-  private String serviceName;
-  private String componentName;
-  private String componentType;
-  private List<DockerContainer> containers = new ArrayList<>();
-  private String pkg;
-  private boolean yarnDockerMode = false;
-
-  public ExecutionCommand(AgentCommandType commandType) {
-    this.commandType = commandType;
-  }
-
-  @JsonProperty("commandType")
-  public AgentCommandType getCommandType() {
-    return commandType;
-  }
-
-  @JsonProperty("commandType")
-  public void setCommandType(AgentCommandType commandType) {
-    this.commandType = commandType;
-  }
-
-  @JsonProperty("commandId")
-  public String getCommandId() {
-    return this.commandId;
-  }
-
-  @JsonProperty("commandId")
-  public void setCommandId(String commandId) {
-    this.commandId = commandId;
-  }
-
-  @JsonProperty("taskId")
-  public long getTaskId() {
-    return taskId;
-  }
-
-  @JsonProperty("taskId")
-  public void setTaskId(long taskId) {
-    this.taskId = taskId;
-  }
-
-  @JsonProperty("role")
-  public String getRole() {
-    return role;
-  }
-
-  @JsonProperty("role")
-  public void setRole(String role) {
-    this.role = role;
-  }
-
-  @JsonProperty("roleParams")
-  public Map<String, String> getRoleParams() {
-    return roleParams;
-  }
-
-  @JsonProperty("roleParams")
-  public void setRoleParams(Map<String, String> roleParams) {
-    this.roleParams = roleParams;
-  }
-
-  @JsonProperty("roleCommand")
-  public String getRoleCommand() {
-    return roleCommand;
-  }
-
-  @JsonProperty("roleCommand")
-  public void setRoleCommand(String cmd) {
-    this.roleCommand = cmd;
-  }
-
-  @JsonProperty("clusterName")
-  public String getClusterName() {
-    return clusterName;
-  }
-
-  @JsonProperty("clusterName")
-  public void setClusterName(String clusterName) {
-    this.clusterName = clusterName;
-  }
-
-  @JsonProperty("componentType")
-  public String getComponentType() {
-    return componentType;
-  }
-
-  @JsonProperty("componentType")
-  public void setComponentType(String componentType) {
-    this.componentType = componentType;
-  }
-
-  @JsonProperty("hostname")
-  public String getHostname() {
-    return hostname;
-  }
-
-  @JsonProperty("hostname")
-  public void setHostname(String hostname) {
-    this.hostname = hostname;
-  }
-
-  @JsonProperty("hostLevelParams")
-  public Map<String, String> getHostLevelParams() {
-    return hostLevelParams;
-  }
-
-  @JsonProperty("hostLevelParams")
-  public void setHostLevelParams(Map<String, String> params) {
-    this.hostLevelParams = params;
-  }
-
-  @JsonProperty("configurations")
-  public Map<String, Map<String, String>> getConfigurations() {
-    return configurations;
-  }
-
-  @JsonProperty("configurations")
-  public void setConfigurations(Map<String, Map<String, String>> configurations) {
-    this.configurations = configurations;
-  }
-
-  @JsonProperty("commandParams")
-  public Map<String, String> getCommandParams() {
-    return commandParams;
-  }
-
-  @JsonProperty("commandParams")
-  public void setCommandParams(Map<String, String> commandParams) {
-    this.commandParams = commandParams;
-  }
-
-  @JsonProperty("serviceName")
-  public String getServiceName() {
-    return serviceName;
-  }
-
-  @JsonProperty("serviceName")
-  public void setServiceName(String serviceName) {
-    this.serviceName = serviceName;
-  }
-
-  @JsonProperty("componentName")
-  public String getComponentName() {
-    return componentName;
-  }
-
-  @JsonProperty("componentName")
-  public void setComponentName(String componentName) {
-    this.componentName = componentName;
-  }
-
-  @JsonProperty("package")
-  public String getPkg() {
-    return pkg;
-  }
-
-  @JsonProperty("package")
-  public void setPkg(String pkg) {
-    this.pkg = pkg;
-  }
-
-  @JsonProperty("componentConfig")
-  public Map<String, Map<String, String>> getComponentConfigurations() {
-    return componentConfigurations;
-  }
-
-  @JsonProperty("componentConfig")
-  public void setComponentConfigurations(
-      Map<String, Map<String, String>> componentConfigurations) {
-    this.componentConfigurations = componentConfigurations;
-  }
-
-  @JsonProperty("containers")
-  public List<DockerContainer> getContainers() {
-    return containers;
-  }
-
-  @JsonProperty("yarnDockerMode")
-  public boolean isYarnDockerMode() {
-    return yarnDockerMode ;
-  }
-
-  @JsonProperty("yarnDockerMode")
-  public void setYarnDockerMode(boolean yarnDockerMode) {
-    this.yarnDockerMode = yarnDockerMode;
-  }
-  @Override
-  public String toString() {
-    StringBuilder builder = new StringBuilder();
-    builder.append("ExecutionCommand [commandType=").append(commandType)
-        .append(", clusterName=").append(clusterName).append(", taskId=")
-        .append(taskId).append(", commandId=").append(commandId)
-        .append(", hostname=").append(hostname).append(", role=").append(role)
-        .append(", hostLevelParams=").append(hostLevelParams)
-        .append(", roleParams=").append(roleParams).append(", roleCommand=")
-        .append(roleCommand).append(", configurations=").append(configurations)
-        .append(", commandParams=").append(commandParams)
-        .append(", serviceName=").append(serviceName)
-        .append(", componentName=").append(componentName)
-        .append(", componentType=").append(componentType)
-        .append(", yarnDockerMode=").append(yarnDockerMode).append(", pkg=")
-        .append(pkg).append("]");
-    return builder.toString();
-  }
-  
-  public void addContainerDetails(String componentGroup, Metainfo metaInfo) {
-    Component component = metaInfo.getApplicationComponent(componentGroup);
-    this.setComponentType(component.getType());
-    log.info("Adding container details for {}", componentGroup, " from ",
-        metaInfo.toString());
-    for (DockerContainer metaContainer : component.getDockerContainers()) {
-      DockerContainer container = new DockerContainer();
-      container.setImage(metaContainer.getImage());
-      container.setNetwork(metaContainer.getNetwork());
-      container.setUseNetworkScript(metaContainer.getUseNetworkScript());
-      container.setName(metaContainer.getName());
-      container.setOptions(metaContainer.getOptions());
-      container.setAdditionalParam(metaContainer.getAdditionalParam());
-      container.setCommandPath(metaContainer.getAdditionalParam());
-      container.setStatusCommand(metaContainer.getStatusCommand());
-      container.setStartCommand(metaContainer.getStartCommand());
-      if (metaContainer.getMounts().size() > 0) {
-        for (DockerContainerMount metaContMount : metaContainer.getMounts()) {
-          DockerContainerMount contMnt = new DockerContainerMount();
-          contMnt.setContainerMount(metaContMount.getContainerMount());
-          contMnt.setHostMount(metaContMount.getHostMount());
-          container.getMounts().add(contMnt);
-        }
-      }
-      if (metaContainer.getPorts().size() > 0) {
-        for (DockerContainerPort metaCntPort : metaContainer.getPorts()) {
-          DockerContainerPort cntPort = new DockerContainerPort();
-          cntPort.setContainerPort(metaCntPort.getContainerPort());
-          cntPort.setHostPort(metaCntPort.getHostPort());
-          container.getPorts().add(cntPort);
-        }
-      }
-      if (metaContainer.getInputFiles().size() > 0) {
-        for (DockerContainerInputFile metaInpFile : metaContainer
-            .getInputFiles()) {
-          DockerContainerInputFile inpFile = new DockerContainerInputFile();
-          inpFile.setContainerMount(metaInpFile.getContainerMount());
-          inpFile.setFileLocalPath(metaInpFile.getFileLocalPath());
-          container.getInputFiles().add(inpFile);
-        }
-      }
-      if (metaContainer.getConfigFiles() != null) {
-        container.setConfigFiles(metaContainer.getConfigFiles());
-      }
-      log.info("Docker container meta info ready: " + container.toString());
-      this.getContainers().add(container);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HeartBeat.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HeartBeat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HeartBeat.java
deleted file mode 100644
index d17c465..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HeartBeat.java
+++ /dev/null
@@ -1,149 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.slider.server.appmaster.web.rest.agent;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.annotate.JsonProperty;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- *
- *
- * Data model for agent heartbeat for server (ambari or app master).
- *
- */
-
-@JsonIgnoreProperties(ignoreUnknown = true)
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-public class HeartBeat {
-  private long responseId = -1;
-  private long timestamp;
-  private String hostname;
-  List<CommandReport> reports = new ArrayList<CommandReport>();
-  List<ComponentStatus> componentStatus = new ArrayList<ComponentStatus>();
-  private List<DiskInfo> mounts = new ArrayList<DiskInfo>();
-  HostStatus nodeStatus;
-  private AgentEnv agentEnv = null;
-  private String fqdn;
-  private String pkg;
-
-  public long getResponseId() {
-    return responseId;
-  }
-
-  public void setResponseId(long responseId) {
-    this.responseId=responseId;
-  }
-
-  public long getTimestamp() {
-    return timestamp;
-  }
-
-  public void setTimestamp(long timestamp) {
-    this.timestamp = timestamp;
-  }
-
-  public String getHostname() {
-    return hostname;
-  }
-
-  public void setHostname(String hostname) {
-    this.hostname = hostname;
-  }
-
-  public String getFqdn() {
-    return fqdn;
-  }
-
-  public void setFqdn(String fqdn) {
-    this.fqdn = fqdn;
-  }
-
-  @JsonProperty("reports")
-  public List<CommandReport> getReports() {
-    return this.reports;
-  }
-
-  @JsonProperty("reports")
-  public void setReports(List<CommandReport> reports) {
-    this.reports = reports;
-  }
-
-  public HostStatus getNodeStatus() {
-    return nodeStatus;
-  }
-
-  public void setNodeStatus(HostStatus nodeStatus) {
-    this.nodeStatus = nodeStatus;
-  }
-
-  public AgentEnv getAgentEnv() {
-    return agentEnv;
-  }
-
-  public void setAgentEnv(AgentEnv env) {
-    agentEnv = env;
-  }
-
-  @JsonProperty("componentStatus")
-  public List<ComponentStatus> getComponentStatus() {
-    return componentStatus;
-  }
-
-  @JsonProperty("componentStatus")
-  public void setComponentStatus(List<ComponentStatus> componentStatus) {
-    this.componentStatus = componentStatus;
-  }
-
-  @JsonProperty("mounts")
-  public List<DiskInfo> getMounts() {
-    return this.mounts;
-  }
-
-  @JsonProperty("mounts")
-  public void setMounts(List<DiskInfo> mounts) {
-    this.mounts = mounts;
-  }
-
-  @JsonProperty("package")
-  public String getPackage() {
-    return pkg;
-  }
-
-  @JsonProperty("package")
-  public void setPackage(String pkg) {
-    this.pkg = pkg;
-  }
-
-  @Override
-  public String toString() {
-    return "HeartBeat{" +
-           "responseId=" + responseId +
-           ", timestamp=" + timestamp +
-           ", hostname='" + hostname + '\'' +
-           ", reports=" + reports +
-           ", componentStatus=" + componentStatus +
-           ", package=" + pkg +
-           ", nodeStatus=" + nodeStatus +
-           '}';
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HeartBeatResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HeartBeatResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HeartBeatResponse.java
deleted file mode 100644
index b500d67..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HeartBeatResponse.java
+++ /dev/null
@@ -1,147 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.server.appmaster.web.rest.agent;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.annotate.JsonProperty;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- *
- * Controller to Agent response data model.
- *
- */
-@JsonIgnoreProperties(ignoreUnknown = true)
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-public class HeartBeatResponse {
-
-  private long responseId;
-
-  List<ExecutionCommand> executionCommands = new ArrayList<ExecutionCommand>();
-  List<StatusCommand> statusCommands = new ArrayList<StatusCommand>();
-
-  RegistrationCommand registrationCommand;
-
-  boolean yarnDockerMode = false;
-  boolean restartAgent = false;
-  boolean restartEnabled = true;
-  boolean hasMappedComponents = false;
-  boolean terminateAgent = false;
-
-  @JsonProperty("responseId")
-  public long getResponseId() {
-    return responseId;
-  }
-
-  @JsonProperty("responseId")
-  public void setResponseId(long responseId) {
-    this.responseId=responseId;
-  }
-
-  @JsonProperty("executionCommands")
-  public List<ExecutionCommand> getExecutionCommands() {
-    return executionCommands;
-  }
-
-  @JsonProperty("executionCommands")
-  public void setExecutionCommands(List<ExecutionCommand> executionCommands) {
-    this.executionCommands = executionCommands;
-  }
-
-  @JsonProperty("statusCommands")
-  public List<StatusCommand> getStatusCommands() {
-    return statusCommands;
-  }
-
-  @JsonProperty("statusCommands")
-  public void setStatusCommands(List<StatusCommand> statusCommands) {
-    this.statusCommands = statusCommands;
-  }
-
-  @JsonProperty("registrationCommand")
-  public RegistrationCommand getRegistrationCommand() {
-    return registrationCommand;
-  }
-
-  @JsonProperty("registrationCommand")
-  public void setRegistrationCommand(RegistrationCommand registrationCommand) {
-    this.registrationCommand = registrationCommand;
-  }
-
-  @JsonProperty("restartAgent")
-  public boolean isRestartAgent() {
-    return restartAgent;
-  }
-
-  @JsonProperty("restartAgent")
-  public void setRestartAgent(boolean restartAgent) {
-    this.restartAgent = restartAgent;
-  }
-
-  @JsonProperty("restartEnabled")
-  public boolean getRstartEnabled() {
-    return restartEnabled;
-  }
-
-  @JsonProperty("restartEnabled")
-  public void setRestartEnabled(boolean restartEnabled) {
-    this.restartEnabled = restartEnabled;
-  }
-
-  @JsonProperty("hasMappedComponents")
-  public boolean hasMappedComponents() {
-    return hasMappedComponents;
-  }
-
-  @JsonProperty("hasMappedComponents")
-  public void setHasMappedComponents(boolean hasMappedComponents) {
-    this.hasMappedComponents = hasMappedComponents;
-  }
-
-  @JsonProperty("terminateAgent")
-  public boolean isTerminateAgent() {
-    return terminateAgent;
-  }
-
-  @JsonProperty("terminateAgent")
-  public void setTerminateAgent(boolean terminateAgent) {
-    this.terminateAgent = terminateAgent;
-  }
-
-  public void addExecutionCommand(ExecutionCommand execCmd) {
-    executionCommands.add(execCmd);
-  }
-
-  public void addStatusCommand(StatusCommand statCmd) {
-    statusCommands.add(statCmd);
-  }
-
-  @Override
-  public String toString() {
-    return "HeartBeatResponse{" +
-           "responseId=" + responseId +
-           ", executionCommands=" + executionCommands +
-           ", statusCommands=" + statusCommands +
-           ", registrationCommand=" + registrationCommand +
-           ", restartAgent=" + restartAgent +
-           ", terminateAgent=" + terminateAgent +
-           '}';
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HostInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HostInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HostInfo.java
deleted file mode 100644
index bef7b07..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HostInfo.java
+++ /dev/null
@@ -1,398 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.server.appmaster.web.rest.agent;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.annotate.JsonProperty;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-import java.util.ArrayList;
-import java.util.List;
-
-@JsonIgnoreProperties(ignoreUnknown = true)
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-public class HostInfo {
-  private String architecture;
-  private String domain;
-  private String fqdn;
-  private String hardwareisa;
-  private String hardwaremodel;
-  private String hostname;
-  private String id;
-  private String interfaces;
-  private String ipaddress;
-  private String kernel;
-  private String kernelmajversion;
-  private String kernelrelease;
-  private String kernelversion;
-  private String macaddress;
-  private long memoryfree;
-  private long memorysize;
-  private List<DiskInfo> mounts = new ArrayList<DiskInfo>();
-  private long memorytotal;
-  private String netmask;
-  private String operatingsystem;
-  private String operatingsystemrelease;
-  private String osfamily;
-  private int physicalprocessorcount;
-  private int processorcount;
-  private boolean selinux;
-  private String swapfree;
-  private String swapsize;
-  private String timezone;
-  private String uptime;
-  private long uptime_days;
-  private long uptime_hours;
-
-
-  @JsonProperty("architecture")
-  public String getArchitecture() {
-    return this.architecture;
-  }
-
-  @JsonProperty("architecture")
-  public void setArchitecture(String architecture) {
-    this.architecture = architecture;
-  }
-
-  @JsonProperty("domain")
-  public String getDomain() {
-    return this.domain;
-  }
-
-  @JsonProperty("domain")
-  public void setDomain(String domain) {
-    this.domain = domain;
-  }
-
-  @JsonProperty("fqdn")
-  public String getFQDN() {
-    return this.fqdn;
-  }
-
-  @JsonProperty("fqdn")
-  public void setFQDN(String fqdn) {
-    this.fqdn = fqdn;
-  }
-
-  @JsonProperty("hardwareisa")
-  public String getHardwareIsa() {
-    return hardwareisa;
-  }
-
-  @JsonProperty("hardwareisa")
-  public void setHardwareIsa(String hardwareisa) {
-    this.hardwareisa = hardwareisa;
-  }
-
-  @JsonProperty("hardwaremodel")
-  public String getHardwareModel() {
-    return this.hardwaremodel;
-  }
-
-  @JsonProperty("hardwaremodel")
-  public void setHardwareModel(String hardwaremodel) {
-    this.hardwaremodel = hardwaremodel;
-  }
-
-  @JsonProperty("hostname")
-  public String getHostName() {
-    return this.hostname;
-  }
-
-  @JsonProperty("hostname")
-  public void setHostName(String hostname) {
-    this.hostname = hostname;
-  }
-
-  @JsonProperty("id")
-  public String getAgentUserId() {
-    return id;
-  }
-
-  @JsonProperty("id")
-  public void setAgentUserId(String id) {
-    this.id = id;
-  }
-
-  @JsonProperty("interfaces")
-  public String getInterfaces() {
-    return this.interfaces;
-  }
-
-  @JsonProperty("interfaces")
-  public void setInterfaces(String interfaces) {
-    this.interfaces = interfaces;
-  }
-
-  @JsonProperty("ipaddress")
-  public String getIPAddress() {
-    return this.ipaddress;
-  }
-
-  @JsonProperty("ipaddress")
-  public void setIPAddress(String ipaddress) {
-    this.ipaddress = ipaddress;
-  }
-
-  @JsonProperty("kernel")
-  public String getKernel() {
-    return this.kernel;
-  }
-
-  @JsonProperty("kernel")
-  public void setKernel(String kernel) {
-    this.kernel = kernel;
-  }
-
-  @JsonProperty("kernelmajversion")
-  public String getKernelMajVersion() {
-    return this.kernelmajversion;
-  }
-
-  @JsonProperty("kernelmajversion")
-  public void setKernelMajVersion(String kernelmajversion) {
-    this.kernelmajversion = kernelmajversion;
-  }
-
-  @JsonProperty("kernelrelease")
-  public String getKernelRelease() {
-    return this.kernelrelease;
-  }
-
-  @JsonProperty("kernelrelease")
-  public void setKernelRelease(String kernelrelease) {
-    this.kernelrelease = kernelrelease;
-  }
-
-  @JsonProperty("kernelversion")
-  public String getKernelVersion() {
-    return this.kernelversion;
-  }
-
-  @JsonProperty("kernelversion")
-  public void setKernelVersion(String kernelversion) {
-    this.kernelversion = kernelversion;
-  }
-
-  @JsonProperty("macaddress")
-  public String getMacAddress() {
-    return this.macaddress;
-  }
-
-  @JsonProperty("macaddress")
-  public void setMacAddress(String macaddress) {
-    this.macaddress = macaddress;
-  }
-
-  @JsonProperty("memoryfree")
-  public long getFreeMemory() {
-    return this.memoryfree;
-  }
-
-  @JsonProperty("memoryfree")
-  public void setFreeMemory(long memoryfree) {
-    this.memoryfree = memoryfree;
-  }
-
-  @JsonProperty("memorysize")
-  public long getMemorySize() {
-    return this.memorysize;
-  }
-
-  @JsonProperty("memorysize")
-  public void setMemorySize(long memorysize) {
-    this.memorysize = memorysize;
-  }
-
-  @JsonProperty("mounts")
-  public List<DiskInfo> getMounts() {
-    return this.mounts;
-  }
-
-  @JsonProperty("mounts")
-  public void setMounts(List<DiskInfo> mounts) {
-    this.mounts = mounts;
-  }
-
-  @JsonProperty("memorytotal")
-  public long getMemoryTotal() {
-    return this.memorytotal;
-  }
-
-  @JsonProperty("memorytotal")
-  public void setMemoryTotal(long memorytotal) {
-    this.memorytotal = memorytotal;
-  }
-
-  @JsonProperty("netmask")
-  public String getNetMask() {
-    return this.netmask;
-  }
-
-  @JsonProperty("netmask")
-  public void setNetMask(String netmask) {
-    this.netmask = netmask;
-  }
-
-  @JsonProperty("operatingsystem")
-  public String getOS() {
-    return this.operatingsystem;
-  }
-
-  @JsonProperty("operatingsystem")
-  public void setOS(String operatingsystem) {
-    this.operatingsystem = operatingsystem;
-  }
-
-  @JsonProperty("operatingsystemrelease")
-  public String getOSRelease() {
-    return this.operatingsystemrelease;
-  }
-
-  @JsonProperty("operatingsystemrelease")
-  public void setOSRelease(String operatingsystemrelease) {
-    this.operatingsystemrelease = operatingsystemrelease;
-  }
-
-  @JsonProperty("osfamily")
-  public String getOSFamily() {
-    return this.osfamily;
-  }
-
-  @JsonProperty("osfamily")
-  public void setOSFamily(String osfamily) {
-    this.osfamily = osfamily;
-  }
-
-  @JsonProperty("physicalprocessorcount")
-  public int getPhysicalProcessorCount() {
-    return this.physicalprocessorcount;
-  }
-
-  @JsonProperty("physicalprocessorcount")
-  public void setPhysicalProcessorCount(int physicalprocessorcount) {
-    this.physicalprocessorcount = physicalprocessorcount;
-  }
-
-  @JsonProperty("processorcount")
-  public int getProcessorCount() {
-    return this.processorcount;
-  }
-
-  @JsonProperty("processorcount")
-  public void setProcessorCount(int processorcount) {
-    this.processorcount = processorcount;
-  }
-
-  @JsonProperty("selinux")
-  public boolean getSeLinux() {
-    return selinux;
-  }
-
-  @JsonProperty("selinux")
-  public void setSeLinux(boolean selinux) {
-    this.selinux = selinux;
-  }
-
-  @JsonProperty("swapfree")
-  public String getSwapFree() {
-    return this.swapfree;
-  }
-
-  @JsonProperty("swapfree")
-  public void setSwapFree(String swapfree) {
-    this.swapfree = swapfree;
-  }
-
-  @JsonProperty("swapsize")
-  public String getSwapSize() {
-    return swapsize;
-  }
-
-  @JsonProperty("swapsize")
-  public void setSwapSize(String swapsize) {
-    this.swapsize = swapsize;
-  }
-
-  @JsonProperty("timezone")
-  public String getTimeZone() {
-    return this.timezone;
-  }
-
-  @JsonProperty("timezone")
-  public void setTimeZone(String timezone) {
-    this.timezone = timezone;
-  }
-
-  @JsonProperty("uptime")
-  public String getUptime() {
-    return this.uptime;
-  }
-
-  @JsonProperty("uptime")
-  public void setUpTime(String uptime) {
-    this.uptime = uptime;
-  }
-
-  @JsonProperty("uptime_hours")
-  public long getUptimeHours() {
-    return this.uptime_hours;
-  }
-
-  @JsonProperty("uptime_hours")
-  public void setUpTimeHours(long uptime_hours) {
-    this.uptime_hours = uptime_hours;
-  }
-
-  @JsonProperty("uptime_days")
-  public long getUpTimeDays() {
-    return this.uptime_days;
-  }
-
-  @JsonProperty("uptime_days")
-  public void setUpTimeDays(long uptime_days) {
-    this.uptime_days = uptime_days;
-  }
-
-  private String getDiskString() {
-    if (mounts == null) {
-      return null;
-    }
-    StringBuilder ret = new StringBuilder();
-    for (DiskInfo diskInfo : mounts) {
-      ret.append("(").append(diskInfo.toString()).append(")");
-    }
-    return ret.toString();
-  }
-
-  public String toString() {
-    return "[" +
-           "hostname=" + this.hostname + "," +
-           "fqdn=" + this.fqdn + "," +
-           "domain=" + this.domain + "," +
-           "architecture=" + this.architecture + "," +
-           "processorcount=" + this.processorcount + "," +
-           "physicalprocessorcount=" + this.physicalprocessorcount + "," +
-           "osname=" + this.operatingsystem + "," +
-           "osversion=" + this.operatingsystemrelease + "," +
-           "osfamily=" + this.osfamily + "," +
-           "memory=" + this.memorytotal + "," +
-           "uptime_hours=" + this.uptime_hours + "," +
-           "mounts=" + getDiskString() + "]\n";
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fdab600/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HostStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HostStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HostStatus.java
deleted file mode 100644
index c584149..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/HostStatus.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.slider.server.appmaster.web.rest.agent;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-
-/**
- *
- */
-@JsonIgnoreProperties(ignoreUnknown = true)
-@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
-public class HostStatus {
-  public HostStatus(Status status, String cause) {
-    super();
-    this.status = status;
-    this.cause = cause;
-  }
-  public HostStatus() {
-    super();
-  }
-
-  public enum Status {
-    HEALTHY,
-    UNHEALTHY
-  }
-  Status status;
-  String cause;
-  public Status getStatus() {
-    return status;
-  }
-  public void setStatus(Status status) {
-    this.status = status;
-  }
-  public String getCause() {
-    return cause;
-  }
-  public void setCause(String cause) {
-    this.cause = cause;
-  }
-
-  @Override
-  public String toString() {
-    return "HostStatus{" +
-           "status=" + status +
-           ", cause='" + cause + '\'' +
-           '}';
-  }
-}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[21/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/UgiJerseyBinding.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/UgiJerseyBinding.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/UgiJerseyBinding.java
new file mode 100644
index 0000000..bf71861
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/UgiJerseyBinding.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.restclient;
+
+import com.google.common.base.Preconditions;
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.UniformInterfaceException;
+import com.sun.jersey.api.client.config.ClientConfig;
+import com.sun.jersey.api.client.config.DefaultClientConfig;
+import com.sun.jersey.api.json.JSONConfiguration;
+import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
+import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.slider.core.exceptions.ExceptionConverter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URL;
+
+/**
+ * Class to bond to a Jersey client, for UGI integration and SPNEGO.
+ * <p>
+ *   Usage: create an instance, then when creating a Jersey <code>Client</code>
+ *   pass in to the constructor the handler provided by {@link #getHandler()}
+ *
+ * see <a href="https://jersey.java.net/apidocs/1.17/jersey/com/sun/jersey/client/urlconnection/HttpURLConnectionFactory.html">Jersey docs</a>
+ */
+public class UgiJerseyBinding implements
+    HttpURLConnectionFactory {
+  private static final Logger log =
+      LoggerFactory.getLogger(UgiJerseyBinding.class);
+
+  private final UrlConnectionOperations operations;
+  private final URLConnectionClientHandler handler;
+
+  /**
+   * Construct an instance
+   * @param operations operations instance
+   */
+  @SuppressWarnings("ThisEscapedInObjectConstruction")
+  public UgiJerseyBinding(UrlConnectionOperations operations) {
+    Preconditions.checkArgument(operations != null, "Null operations");
+    this.operations = operations;
+    handler = new URLConnectionClientHandler(this);
+  }
+
+  /**
+   * Create an instance off the configuration. The SPNEGO policy
+   * is derived from the current UGI settings.
+   * @param conf config
+   */
+  public UgiJerseyBinding(Configuration conf) {
+    this(new UrlConnectionOperations(conf));
+  }
+
+  /**
+   * Get a URL connection. 
+   * @param url URL to connect to
+   * @return the connection
+   * @throws IOException any problem. {@link AuthenticationException} 
+   * errors are wrapped
+   */
+  @Override
+  public HttpURLConnection getHttpURLConnection(URL url) throws IOException {
+    try {
+      // open a connection handling status codes and so redirections
+      // but as it opens a connection, it's less useful than you think.
+
+      return operations.openConnection(url);
+    } catch (AuthenticationException e) {
+      throw new IOException(e);
+    }
+  }
+
+  public UrlConnectionOperations getOperations() {
+    return operations;
+  }
+
+  public URLConnectionClientHandler getHandler() {
+    return handler;
+  }
+  
+  /**
+   * Get the SPNEGO flag (as found in the operations instance
+   * @return the spnego policy
+   */
+  public boolean isUseSpnego() {
+    return operations.isUseSpnego();
+  }
+
+
+  /**
+   * Uprate error codes 400 and up into faults; 
+   * <p>
+   * see {@link ExceptionConverter#convertJerseyException(String, String, UniformInterfaceException)}
+   */
+  public static IOException uprateFaults(HttpVerb verb, String url,
+      UniformInterfaceException ex)
+      throws IOException {
+    return ExceptionConverter.convertJerseyException(verb.getVerb(),
+        url, ex);
+  }
+
+  /**
+   * Create the standard Jersey client Config
+   * @return the recommended Jersey Client config
+   */
+  public ClientConfig createJerseyClientConfig() {
+    ClientConfig clientConfig = new DefaultClientConfig();
+    clientConfig.getFeatures().put(JSONConfiguration.FEATURE_POJO_MAPPING, true);
+    return clientConfig;
+  }
+
+  /**
+   * Create a jersey client bonded to this handler, using the
+   * supplied client config
+   * @param clientConfig client configuratin
+   * @return a new client instance to use
+   */
+  public Client createJerseyClient(ClientConfig clientConfig) {
+    return new Client(getHandler(), clientConfig);
+  }
+
+  /**
+   * Create a jersey client bonded to this handler, using the
+   * client config created with {@link #createJerseyClientConfig()}
+   * @return a new client instance to use
+   */
+  public Client createJerseyClient() {
+    return createJerseyClient(createJerseyClientConfig());
+  }
+
+}
+
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/UrlConnectionOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/UrlConnectionOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/UrlConnectionOperations.java
new file mode 100644
index 0000000..20ef198
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/restclient/UrlConnectionOperations.java
@@ -0,0 +1,210 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.restclient;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.yarn.webapp.ForbiddenException;
+import org.apache.hadoop.yarn.webapp.NotFoundException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.net.ssl.SSLException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.HttpURLConnection;
+import java.net.URL;
+
+/**
+ * Operations on the JDK UrlConnection class.
+ *
+ */
+public class UrlConnectionOperations extends Configured  {
+  private static final Logger log =
+      LoggerFactory.getLogger(UrlConnectionOperations.class);
+
+  private SliderURLConnectionFactory connectionFactory;
+
+  private boolean useSpnego = false;
+
+  /**
+   * Create an instance off the configuration. The SPNEGO policy
+   * is derived from the current UGI settings.
+   * @param conf config
+   */
+  public UrlConnectionOperations(Configuration conf) {
+    super(conf);
+    connectionFactory = SliderURLConnectionFactory.newInstance(conf);
+    if (UserGroupInformation.isSecurityEnabled()) {
+      log.debug("SPNEGO is enabled");
+      setUseSpnego(true);
+    }
+  }
+
+
+  public boolean isUseSpnego() {
+    return useSpnego;
+  }
+
+  public void setUseSpnego(boolean useSpnego) {
+    this.useSpnego = useSpnego;
+  }
+
+  /**
+   * Opens a url with cache disabled, redirect handled in 
+   * (JDK) implementation.
+   *
+   * @param url to open
+   * @return URLConnection
+   * @throws IOException
+   * @throws AuthenticationException authentication failure
+   */
+  public HttpURLConnection openConnection(URL url) throws
+      IOException,
+      AuthenticationException {
+    Preconditions.checkArgument(url.getPort() != 0, "no port");
+    return (HttpURLConnection) connectionFactory.openConnection(url, useSpnego);
+  }
+
+  public HttpOperationResponse execGet(URL url) throws
+      IOException,
+      AuthenticationException {
+    return execHttpOperation(HttpVerb.GET, url, null, "");
+  }
+
+  public HttpOperationResponse execHttpOperation(HttpVerb verb,
+      URL url,
+      byte[] payload,
+      String contentType)
+      throws IOException, AuthenticationException {
+    HttpURLConnection conn = null;
+    HttpOperationResponse outcome = new HttpOperationResponse();
+    int resultCode;
+    byte[] body = null;
+    log.debug("{} {} spnego={}", verb, url, useSpnego);
+
+    boolean doOutput = verb.hasUploadBody();
+    if (doOutput) {
+      Preconditions.checkArgument(payload !=null,
+          "Null payload on a verb which expects one");
+    }
+    try {
+      conn = openConnection(url);
+      conn.setRequestMethod(verb.getVerb());
+      conn.setDoOutput(doOutput);
+      if (doOutput) {
+        conn.setRequestProperty("Content-Type", contentType);
+      }
+
+      // now do the connection
+      conn.connect();
+      
+      if (doOutput) {
+        OutputStream output = conn.getOutputStream();
+        IOUtils.write(payload, output);
+        output.close();
+      }
+      
+      resultCode = conn.getResponseCode();
+      outcome.lastModified = conn.getLastModified();
+      outcome.contentType = conn.getContentType();
+      outcome.headers = conn.getHeaderFields();
+      InputStream stream = conn.getErrorStream();
+      if (stream == null) {
+        stream = conn.getInputStream();
+      }
+      if (stream != null) {
+        // read into a buffer.
+        body = IOUtils.toByteArray(stream);
+      } else {
+        // no body: 
+        log.debug("No body in response");
+
+      }
+    } catch (SSLException e) {
+      throw e;
+    } catch (IOException e) {
+      throw NetUtils.wrapException(url.toString(),
+          url.getPort(), "localhost", 0, e);
+
+    } catch (AuthenticationException e) {
+      throw new AuthenticationException("From " + url + ": " + e, e);
+
+    } finally {
+      if (conn != null) {
+        conn.disconnect();
+      }
+    }
+    uprateFaults(HttpVerb.GET, url.toString(), resultCode, "", body);
+    outcome.responseCode = resultCode;
+    outcome.data = body;
+    return outcome;
+  }
+
+  /**
+   * Uprate error codes 400 and up into faults; 
+   * 404 is converted to a {@link NotFoundException},
+   * 401 to {@link ForbiddenException}
+   *
+   * @param verb HTTP Verb used
+   * @param url URL as string
+   * @param resultCode response from the request
+   * @param bodyAsString
+   *@param body optional body of the request  @throws IOException if the result was considered a failure
+   */
+  public static void uprateFaults(HttpVerb verb, String url,
+      int resultCode, String bodyAsString, byte[] body)
+      throws IOException {
+
+    if (resultCode < 400) {
+      //success
+      return;
+    }
+    String msg = verb.toString() +" "+ url;
+    if (resultCode == 404) {
+      throw new NotFoundException(msg);
+    }
+    if (resultCode == 401) {
+      throw new ForbiddenException(msg);
+    }
+    // all other error codes
+    
+    // get a string respnse
+    if (bodyAsString == null) {
+      if (body != null && body.length > 0) {
+        bodyAsString = new String(body);
+      } else {
+        bodyAsString = "";
+      }
+    }
+    String message =  msg +
+                     " failed with exit code " + resultCode
+                     + ", body length " + bodyAsString.length()
+                     + ":\n" + bodyAsString;
+    log.error(message);
+    throw new IOException(message);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/BlockingZKWatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/BlockingZKWatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/BlockingZKWatcher.java
new file mode 100644
index 0000000..ca49888
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/BlockingZKWatcher.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.zk;
+
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.ConnectException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class BlockingZKWatcher implements Watcher {
+  
+  protected static final Logger log =
+    LoggerFactory.getLogger(BlockingZKWatcher.class);
+  private final AtomicBoolean connectedFlag = new AtomicBoolean(false);
+
+  @Override
+  public void process(WatchedEvent event) {
+    log.info("ZK binding callback received");
+    connectedFlag.set(true);
+    synchronized (connectedFlag) {
+      try {
+        connectedFlag.notify();
+      } catch (Exception e) {
+        log.warn("failed while waiting for notification", e);
+      }
+    }
+  }
+
+  /**
+   * Wait for a flag to go true
+   * @param timeout timeout in millis
+   */
+
+  public void waitForZKConnection(int timeout)
+      throws InterruptedException, ConnectException {
+    synchronized (connectedFlag) {
+      if (!connectedFlag.get()) {
+        log.info("waiting for ZK event");
+        //wait a bit
+        connectedFlag.wait(timeout);
+      }
+    }
+    if (!connectedFlag.get()) {
+      throw new ConnectException("Unable to connect to ZK quorum");
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/MiniZooKeeperCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/MiniZooKeeperCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/MiniZooKeeperCluster.java
new file mode 100644
index 0000000..c8b3adb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/MiniZooKeeperCluster.java
@@ -0,0 +1,423 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.zk;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.zookeeper.server.NIOServerCnxnFactory;
+import org.apache.zookeeper.server.ZooKeeperServer;
+import org.apache.zookeeper.server.persistence.FileTxnLog;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.Reader;
+import java.net.BindException;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+
+/**
+ * This is a version of the HBase ZK cluster cut out to be standalone.
+ * 
+ * <i>Important: keep this Java6 language level for now</i>
+ */
+public class MiniZooKeeperCluster extends AbstractService {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      MiniZooKeeperCluster.class);
+
+  private static final int TICK_TIME = 2000;
+  private static final int CONNECTION_TIMEOUT = 30000;
+  public static final int MAX_CLIENT_CONNECTIONS = 1000;
+
+  private boolean started;
+
+  /** The default port. If zero, we use a random port. */
+  private int defaultClientPort = 0;
+
+  private int clientPort;
+
+  private final List<NIOServerCnxnFactory> standaloneServerFactoryList;
+  private final List<ZooKeeperServer> zooKeeperServers;
+  private final List<Integer> clientPortList;
+
+  private int activeZKServerIndex;
+  private int tickTime = 0;
+  private File baseDir;
+  private final int numZooKeeperServers;
+  private String zkQuorum = "";
+
+  public MiniZooKeeperCluster(int numZooKeeperServers) {
+    super("MiniZooKeeperCluster");
+    this.numZooKeeperServers = numZooKeeperServers;
+    this.started = false;
+    activeZKServerIndex = -1;
+    zooKeeperServers = new ArrayList<ZooKeeperServer>();
+    clientPortList = new ArrayList<Integer>();
+    standaloneServerFactoryList = new ArrayList<NIOServerCnxnFactory>();
+  }
+
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    super.serviceInit(conf);
+  }
+
+  public void setDefaultClientPort(int clientPort) {
+    if (clientPort <= 0) {
+      throw new IllegalArgumentException("Invalid default ZK client port: "
+                                         + clientPort);
+    }
+    this.defaultClientPort = clientPort;
+  }
+
+  /**
+   * Selects a ZK client port. Returns the default port if specified.
+   * Otherwise, returns a random port. The random port is selected from the
+   * range between 49152 to 65535. These ports cannot be registered with IANA
+   * and are intended for dynamic allocation (see http://bit.ly/dynports).
+   */
+  private int selectClientPort(Random r) {
+    if (defaultClientPort > 0) {
+      return defaultClientPort;
+    }
+    return 0xc000 + r.nextInt(0x3f00);
+  }
+
+  public void setTickTime(int tickTime) {
+    this.tickTime = tickTime;
+  }
+
+  public int getBackupZooKeeperServerNum() {
+    return zooKeeperServers.size() - 1;
+  }
+
+  public int getZooKeeperServerNum() {
+    return zooKeeperServers.size();
+  }
+
+  // / XXX: From o.a.zk.t.ClientBase
+  private static void setupTestEnv() {
+    // during the tests we run with 100K prealloc in the logs.
+    // on windows systems prealloc of 64M was seen to take ~15seconds
+    // resulting in test failure (client timeout on first session).
+    // set env and directly in order to handle static init/gc issues
+    System.setProperty("zookeeper.preAllocSize", "100");
+    FileTxnLog.setPreallocSize(100 * 1024);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    startup();
+  }
+
+  /**
+   * @param baseDir
+   * @param numZooKeeperServers
+   * @return ClientPort server bound to, -1 if there was a
+   *         binding problem and we couldn't pick another port.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private int startup() throws IOException,
+      InterruptedException {
+    if (numZooKeeperServers <= 0)
+      return -1;
+
+    setupTestEnv();
+    started = true;
+    baseDir = File.createTempFile("zookeeper", ".dir");
+    recreateDir(baseDir);
+
+    StringBuilder quorumList = new StringBuilder();
+    Random rnd = new Random();
+    int tentativePort = selectClientPort(rnd);
+
+    // running all the ZK servers
+    for (int i = 0; i < numZooKeeperServers; i++) {
+      File dir = new File(baseDir, "zookeeper_" + i).getAbsoluteFile();
+      recreateDir(dir);
+      int tickTimeToUse;
+      if (this.tickTime > 0) {
+        tickTimeToUse = this.tickTime;
+      } else {
+        tickTimeToUse = TICK_TIME;
+      }
+      ZooKeeperServer server = new ZooKeeperServer(dir, dir, tickTimeToUse);
+      NIOServerCnxnFactory standaloneServerFactory;
+      while (true) {
+        try {
+          standaloneServerFactory = new NIOServerCnxnFactory();
+          standaloneServerFactory.configure(
+              new InetSocketAddress(tentativePort),
+              MAX_CLIENT_CONNECTIONS
+          );
+        } catch (BindException e) {
+          LOG.debug("Failed binding ZK Server to client port: " +
+                    tentativePort, e);
+          // We're told to use some port but it's occupied, fail
+          if (defaultClientPort > 0) return -1;
+          // This port is already in use, try to use another.
+          tentativePort = selectClientPort(rnd);
+          continue;
+        }
+        break;
+      }
+
+      // Start up this ZK server
+      standaloneServerFactory.startup(server);
+      if (!waitForServerUp(tentativePort, CONNECTION_TIMEOUT)) {
+        throw new IOException("Waiting for startup of standalone server");
+      }
+
+      // We have selected this port as a client port.
+      clientPortList.add(tentativePort);
+      standaloneServerFactoryList.add(standaloneServerFactory);
+      zooKeeperServers.add(server);
+      if (quorumList.length() > 0) {
+        quorumList.append(",");
+      }
+      quorumList.append("localhost:").append(tentativePort);
+      tentativePort++; //for the next server
+    }
+
+    // set the first one to be active ZK; Others are backups
+    activeZKServerIndex = 0;
+
+    clientPort = clientPortList.get(activeZKServerIndex);
+    zkQuorum = quorumList.toString();
+    LOG.info("Started MiniZK Cluster and connect 1 ZK server " +
+             "on client port: " + clientPort);
+    return clientPort;
+  }
+
+  private void recreateDir(File dir) throws IOException {
+    if (dir.exists()) {
+      if (!FileUtil.fullyDelete(dir)) {
+        throw new IOException("Could not delete zk base directory: " + dir);
+      }
+    }
+    try {
+      dir.mkdirs();
+    } catch (SecurityException e) {
+      throw new IOException("creating dir: " + dir, e);
+    }
+  }
+
+  /**
+   * Delete the basedir
+   */
+  private void deleteBaseDir() {
+    if (baseDir != null) {
+      baseDir.delete();
+      baseDir = null;
+    }
+
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+
+    if (!started) {
+      return;
+    }
+    started = false;
+
+    try {
+      // shut down all the zk servers
+      for (int i = 0; i < standaloneServerFactoryList.size(); i++) {
+        NIOServerCnxnFactory standaloneServerFactory =
+            standaloneServerFactoryList.get(i);
+        int clientPort = clientPortList.get(i);
+  
+        standaloneServerFactory.shutdown();
+        if (!waitForServerDown(clientPort, CONNECTION_TIMEOUT)) {
+          throw new IOException("Waiting for shutdown of standalone server");
+        }
+      }
+      for (ZooKeeperServer zkServer : zooKeeperServers) {
+        //explicitly close ZKDatabase since ZookeeperServer does not close them
+        zkServer.getZKDatabase().close();
+      }
+    } finally {
+      // clear everything
+      activeZKServerIndex = 0;
+      standaloneServerFactoryList.clear();
+      clientPortList.clear();
+      zooKeeperServers.clear();
+    }
+
+    LOG.info("Shutdown MiniZK cluster with all ZK servers");
+  }
+
+  /**@return clientPort return clientPort if there is another ZK backup can run
+   *         when killing the current active; return -1, if there is no backups.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public int killCurrentActiveZooKeeperServer() throws IOException,
+      InterruptedException {
+    if (!started || activeZKServerIndex < 0) {
+      return -1;
+    }
+
+    // Shutdown the current active one
+    NIOServerCnxnFactory standaloneServerFactory =
+        standaloneServerFactoryList.get(activeZKServerIndex);
+    int clientPort = clientPortList.get(activeZKServerIndex);
+
+    standaloneServerFactory.shutdown();
+    if (!waitForServerDown(clientPort, CONNECTION_TIMEOUT)) {
+      throw new IOException("Waiting for shutdown of standalone server");
+    }
+
+    zooKeeperServers.get(activeZKServerIndex).getZKDatabase().close();
+
+    // remove the current active zk server
+    standaloneServerFactoryList.remove(activeZKServerIndex);
+    clientPortList.remove(activeZKServerIndex);
+    zooKeeperServers.remove(activeZKServerIndex);
+    LOG.info("Kill the current active ZK servers in the cluster " +
+             "on client port: " + clientPort);
+
+    if (standaloneServerFactoryList.size() == 0) {
+      // there is no backup servers;
+      return -1;
+    }
+    clientPort = clientPortList.get(activeZKServerIndex);
+    LOG.info("Activate a backup zk server in the cluster " +
+             "on client port: " + clientPort);
+    // return the next back zk server's port
+    return clientPort;
+  }
+
+  /**
+   * Kill one back up ZK servers
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public void killOneBackupZooKeeperServer() throws IOException,
+      InterruptedException {
+    if (!started || activeZKServerIndex < 0 ||
+        standaloneServerFactoryList.size() <= 1) {
+      return;
+    }
+
+    int backupZKServerIndex = activeZKServerIndex + 1;
+    // Shutdown the current active one
+    NIOServerCnxnFactory standaloneServerFactory =
+        standaloneServerFactoryList.get(backupZKServerIndex);
+    int clientPort = clientPortList.get(backupZKServerIndex);
+
+    standaloneServerFactory.shutdown();
+    if (!waitForServerDown(clientPort, CONNECTION_TIMEOUT)) {
+      throw new IOException("Waiting for shutdown of standalone server");
+    }
+
+    zooKeeperServers.get(backupZKServerIndex).getZKDatabase().close();
+
+    // remove this backup zk server
+    standaloneServerFactoryList.remove(backupZKServerIndex);
+    clientPortList.remove(backupZKServerIndex);
+    zooKeeperServers.remove(backupZKServerIndex);
+    LOG.info("Kill one backup ZK servers in the cluster " +
+             "on client port: " + clientPort);
+  }
+
+  // XXX: From o.a.zk.t.ClientBase
+  private static boolean waitForServerDown(int port, long timeout) throws
+      InterruptedException {
+    long start = System.currentTimeMillis();
+    while (true) {
+      try {
+        Socket sock = null;
+        try {
+          sock = new Socket("localhost", port);
+          OutputStream outstream = sock.getOutputStream();
+          outstream.write("stat".getBytes());
+          outstream.flush();
+        } finally {
+          IOUtils.closeSocket(sock);
+        }
+      } catch (IOException e) {
+        return true;
+      }
+
+      if (System.currentTimeMillis() > start + timeout) {
+        break;
+      }
+      Thread.sleep(250);
+    }
+    return false;
+  }
+
+  // XXX: From o.a.zk.t.ClientBase
+  private static boolean waitForServerUp(int port, long timeout) throws
+      InterruptedException {
+    long start = System.currentTimeMillis();
+    while (true) {
+      try {
+        Socket sock = null;
+        sock = new Socket("localhost", port);
+        BufferedReader reader = null;
+        try {
+          OutputStream outstream = sock.getOutputStream();
+          outstream.write("stat".getBytes());
+          outstream.flush();
+
+          Reader isr = new InputStreamReader(sock.getInputStream());
+          reader = new BufferedReader(isr);
+          String line = reader.readLine();
+          if (line != null && line.startsWith("Zookeeper version:")) {
+            return true;
+          }
+        } finally {
+          IOUtils.closeSocket(sock);
+          IOUtils.closeStream(reader);
+        }
+      } catch (IOException e) {
+        // ignore as this is expected
+        LOG.debug("server localhost:" + port + " not up " + e);
+      }
+
+      if (System.currentTimeMillis() > start + timeout) {
+        break;
+      }
+      Thread.sleep(250);
+    }
+    return false;
+  }
+
+  public int getClientPort() {
+    return clientPort;
+  }
+
+  public String getZkQuorum() {
+    return zkQuorum;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKCallback.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKCallback.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKCallback.java
new file mode 100644
index 0000000..045b72c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKCallback.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.core.zk;
+
+import org.apache.zookeeper.Watcher;
+
+/**
+ * Relays ZK watcher events to a closure
+ */
+public abstract class ZKCallback implements Watcher {
+
+  public ZKCallback() {
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKIntegration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKIntegration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKIntegration.java
new file mode 100644
index 0000000..ca41e4b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKIntegration.java
@@ -0,0 +1,323 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.core.zk;
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+
+public class ZKIntegration implements Watcher, Closeable {
+
+/**
+ * Base path for services
+ */
+  public static String ZK_SERVICES = "services";
+  /**
+   * Base path for all Slider references
+   */
+  public static String ZK_SLIDER = "slider";
+  public static String ZK_USERS = "users";
+  public static String SVC_SLIDER = "/" + ZK_SERVICES + "/" + ZK_SLIDER;
+  public static String SVC_SLIDER_USERS = SVC_SLIDER + "/" + ZK_USERS;
+
+  public static final List<String> ZK_USERS_PATH_LIST = new ArrayList<String>();
+  static {
+    ZK_USERS_PATH_LIST.add(ZK_SERVICES);
+    ZK_USERS_PATH_LIST.add(ZK_SLIDER);
+    ZK_USERS_PATH_LIST.add(ZK_USERS);
+  }
+
+  public static int SESSION_TIMEOUT = 30000;
+  protected static final Logger log =
+    LoggerFactory.getLogger(ZKIntegration.class);
+  private ZooKeeper zookeeper;
+  private final String username;
+  private final String clustername;
+  private final String userPath;
+  private int sessionTimeout = SESSION_TIMEOUT;
+/**
+ flag to set to indicate that the user path should be created if
+ it is not already there
+ */
+  private final AtomicBoolean toInit = new AtomicBoolean(false);
+  private final boolean createClusterPath;
+  private final Watcher watchEventHandler;
+  private final String zkConnection;
+  private final boolean canBeReadOnly;
+
+  protected ZKIntegration(String zkConnection,
+                          String username,
+                          String clustername,
+                          boolean canBeReadOnly,
+                          boolean createClusterPath,
+                          Watcher watchEventHandler,
+                          int sessionTimeout
+  ) throws IOException {
+    this.username = username;
+    this.clustername = clustername;
+    this.watchEventHandler = watchEventHandler;
+    this.zkConnection = zkConnection;
+    this.canBeReadOnly = canBeReadOnly;
+    this.createClusterPath = createClusterPath;
+    this.sessionTimeout = sessionTimeout;
+    this.userPath = mkSliderUserPath(username);
+  }
+
+  public void init() throws IOException {
+    assert zookeeper == null;
+    log.debug("Binding ZK client to {}", zkConnection);
+    zookeeper = new ZooKeeper(zkConnection, sessionTimeout, this, canBeReadOnly);
+  }
+
+  /**
+   * Create an instance bonded to the specific closure
+   * @param zkConnection
+   * @param username
+   * @param clustername
+   * @param canBeReadOnly
+   * @param watchEventHandler
+   * @return the new instance
+   * @throws IOException
+   */
+  public static ZKIntegration newInstance(String zkConnection,
+      String username,
+      String clustername,
+      boolean createClusterPath,
+      boolean canBeReadOnly,
+      Watcher watchEventHandler,
+      int sessionTimeout) throws IOException {
+
+    return new ZKIntegration(zkConnection,
+                             username,
+                             clustername,
+                             canBeReadOnly,
+                             createClusterPath,
+                             watchEventHandler,
+                             sessionTimeout);
+  }
+
+
+  @Override
+  public synchronized void close() throws IOException {
+    if (zookeeper != null) {
+      try {
+        zookeeper.close();
+      } catch (InterruptedException ignored) {
+
+      }
+      zookeeper = null;
+    }
+  }
+
+  public String getConnectionString() {
+    return zkConnection;
+  }
+
+  public String getClusterPath() {
+    return mkClusterPath(username, clustername);
+  }
+
+  public boolean getConnected() {
+    return zookeeper.getState().isConnected();
+  }
+
+  public boolean getAlive() {
+    return zookeeper.getState().isAlive();
+  }
+
+  public ZooKeeper.States getState() {
+    return zookeeper.getState();
+  }
+
+  public Stat getClusterStat() throws KeeperException, InterruptedException {
+    return stat(getClusterPath());
+  }
+
+  public boolean exists(String path) throws
+                                     KeeperException,
+                                     InterruptedException {
+    return stat(path) != null;
+  }
+
+  public Stat stat(String path) throws KeeperException, InterruptedException {
+    return zookeeper.exists(path, false);
+  }
+
+  @Override
+  public String toString() {
+    return "ZK integration bound @  " + zkConnection + ": " + zookeeper;
+  }
+  
+/**
+ * Event handler to notify of state events
+ * @param event
+ */
+  @Override
+  public void process(WatchedEvent event) {
+    log.debug("{}", event);
+    try {
+      maybeInit();
+    } catch (Exception e) {
+      log.error("Failed to init", e);
+    }
+    if (watchEventHandler != null) {
+      watchEventHandler.process(event);
+    }
+  }
+
+  private void maybeInit() throws KeeperException, InterruptedException {
+    if (!toInit.getAndSet(true) && createClusterPath) {
+      log.debug("initing");
+      //create the user path
+      mkPath(ZK_USERS_PATH_LIST, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+      //create the specific user
+      createPath(userPath, null, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+    }
+  }
+
+  /**
+   * Create a path under a parent, don't care if it already exists
+   * As the path isn't returned, this isn't the way to create sequentially
+   * numbered nodes.
+   * @param parent parent dir. Must have a trailing / if entry!=null||empty 
+   * @param entry entry -can be null or "", in which case it is not appended
+   * @param acl
+   * @param createMode
+   * @return the path if created; null if not
+   */
+  public String createPath(String parent,
+                           String entry,
+                           List<ACL> acl,
+                           CreateMode createMode) throws KeeperException, InterruptedException {
+    //initial create of full path
+    assert acl != null;
+    assert !acl.isEmpty();
+    assert parent != null;
+    String path = parent;
+    if (entry != null) {
+      path = path + entry;
+    }
+    try {
+      log.debug("Creating ZK path {}", path);
+      return zookeeper.create(path, null, acl, createMode);
+    } catch (KeeperException.NodeExistsException ignored) {
+      //node already there
+      log.debug("node already present:{}",path);
+      return null;
+    }
+  }
+
+  /**
+   * Recursive path create
+   * @param paths path list
+   * @param acl acl list
+   * @param createMode create modes
+   */
+  public void mkPath(List<String> paths,
+                     List<ACL> acl,
+                     CreateMode createMode) throws KeeperException, InterruptedException {
+    String history = "/";
+    for (String entry : paths) {
+      createPath(history, entry, acl, createMode);
+      history = history + entry + "/";
+    }
+  }
+
+/**
+ * Blocking enum of users
+ * @return an unordered list of clusters under a user
+ */
+  public List<String> getClusters() throws KeeperException, InterruptedException {
+    return zookeeper.getChildren(userPath, null);
+  }
+
+  /**
+   * Delete a node, does not throw an exception if the path is not fond
+   * @param path path to delete
+   * @return true if the path could be deleted, false if there was no node to delete 
+   *
+   */
+  public boolean delete(String path) throws
+                                     InterruptedException,
+                                     KeeperException {
+    try {
+      zookeeper.delete(path, -1);
+      log.debug("Deleting {}", path);
+      return true;
+    } catch (KeeperException.NoNodeException ignored) {
+      return false;
+    }
+  }
+
+  /**
+   * Recursively delete a node, does not throw exception if any node does not exist.
+   * @param path
+   * @return true if delete was successful
+   */
+  public boolean deleteRecursive(String path) throws KeeperException, InterruptedException {
+
+    try {
+      List<String> children = zookeeper.getChildren(path, false);
+      for (String child : children) {
+        deleteRecursive(path + "/" + child);
+      }
+      delete(path);
+    } catch (KeeperException.NoNodeException ignored) {
+      return false;
+    }
+
+    return true;
+  }
+
+  /**
+ * Build the path to a cluster; exists once the cluster has come up.
+ * Even before that, a ZK watcher could wait for it.
+ * @param username user
+ * @param clustername name of the cluster
+ * @return a strin
+ */
+  public static String mkClusterPath(String username, String clustername) {
+    return mkSliderUserPath(username) + "/" + clustername;
+  }
+/**
+ * Build the path to a cluster; exists once the cluster has come up.
+ * Even before that, a ZK watcher could wait for it.
+ * @param username user
+ * @return a string
+ */
+  public static String mkSliderUserPath(String username) {
+    return SVC_SLIDER_USERS + "/" + username;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKPathBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKPathBuilder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKPathBuilder.java
new file mode 100644
index 0000000..b088568
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZKPathBuilder.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.zk;
+
+import java.util.Locale;
+
+public final class ZKPathBuilder {
+
+  private final String username, appname, clustername;
+  private final String quorum;
+
+  private String appPath;
+  private String registryPath;
+  private final String appQuorum;
+  
+  public ZKPathBuilder(String username,
+    String appname,
+    String clustername,
+    String quorum,
+      String appQuorum) {
+    this.username = username;
+    this.appname = appname;
+    this.clustername = clustername;
+    this.quorum = quorum;
+    appPath = buildAppPath();
+    registryPath = buildRegistryPath();
+    this.appQuorum = appQuorum;
+  }
+
+  public String buildAppPath() {
+    return String.format(Locale.ENGLISH, "/yarnapps_%s_%s_%s", appname,
+                         username, clustername);
+
+  }
+
+  public String buildRegistryPath() {
+    return String.format(Locale.ENGLISH, "/services_%s_%s_%s", appname,
+                         username, clustername);
+
+  }
+
+  public String getQuorum() {
+    return quorum;
+  }
+
+  public String getAppQuorum() {
+    return appQuorum;
+  }
+
+  public String getAppPath() {
+    return appPath;
+  }
+
+  public void setAppPath(String appPath) {
+    this.appPath = appPath;
+  }
+
+  public String getRegistryPath() {
+    return registryPath;
+  }
+
+  public void setRegistryPath(String registryPath) {
+    this.registryPath = registryPath;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZookeeperUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZookeeperUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZookeeperUtils.java
new file mode 100644
index 0000000..cc1b2c9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/core/zk/ZookeeperUtils.java
@@ -0,0 +1,147 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.core.zk;
+
+import com.google.common.net.HostAndPort;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.exceptions.BadConfigException;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class ZookeeperUtils {
+  public static final int DEFAULT_PORT = 2181;
+
+  public static String buildConnectionString(String zkHosts, int port) {
+    String zkPort = Integer.toString(port);
+    //parse the hosts
+    String[] hostlist = zkHosts.split(",", 0);
+    String quorum = SliderUtils.join(hostlist, ":" + zkPort + ",", false);
+    return quorum;
+  }
+
+  /**
+   * Take a quorum list and split it to (trimmed) pairs
+   * @param hostPortQuorumList list of form h1:port, h2:port2,...
+   * @return a possibly empty list of values between commas. They may not be
+   * valid hostname:port pairs
+   */
+  public static List<String> splitToPairs(String hostPortQuorumList) {
+    // split an address hot
+    String[] strings = StringUtils.getStrings(hostPortQuorumList);
+    int len = 0;
+    if (strings != null) {
+      len = strings.length;
+    }
+    List<String> tuples = new ArrayList<String>(len);
+    if (strings != null) {
+      for (String s : strings) {
+        tuples.add(s.trim());
+      }
+    }
+    return tuples;
+  }
+
+  /**
+   * Split a quorum list into a list of hostnames and ports
+   * @param hostPortQuorumList split to a list of hosts and ports
+   * @return a list of values
+   */
+  public static List<HostAndPort> splitToHostsAndPorts(String hostPortQuorumList) {
+    // split an address hot
+    String[] strings = StringUtils.getStrings(hostPortQuorumList);
+    int len = 0;
+    if (strings != null) {
+      len = strings.length;
+    }
+    List<HostAndPort> list = new ArrayList<HostAndPort>(len);
+    if (strings != null) {
+      for (String s : strings) {
+        list.add(HostAndPort.fromString(s.trim()).withDefaultPort(DEFAULT_PORT));
+      }
+    }
+    return list;
+  }
+
+  /**
+   * Build up to a hosts only list
+   * @param hostAndPorts
+   * @return a list of the hosts only
+   */
+  public static String buildHostsOnlyList(List<HostAndPort> hostAndPorts) {
+    StringBuilder sb = new StringBuilder();
+    for (HostAndPort hostAndPort : hostAndPorts) {
+      sb.append(hostAndPort.getHostText()).append(",");
+    }
+    if (sb.length() > 0) {
+      sb.delete(sb.length() - 1, sb.length());
+    }
+    return sb.toString();
+  }
+
+  public static String buildQuorumEntry(HostAndPort hostAndPort,
+    int defaultPort) {
+    String s = hostAndPort.toString();
+    if (hostAndPort.hasPort()) {
+      return s;
+    } else {
+      return s + ":" + defaultPort;
+    }
+  }
+
+  /**
+   * Build a quorum list, injecting a ":defaultPort" ref if needed on
+   * any entry without one
+   * @param hostAndPorts
+   * @param defaultPort
+   * @return
+   */
+  public static String buildQuorum(List<HostAndPort> hostAndPorts, int defaultPort) {
+    List<String> entries = new ArrayList<String>(hostAndPorts.size());
+    for (HostAndPort hostAndPort : hostAndPorts) {
+      entries.add(buildQuorumEntry(hostAndPort, defaultPort));
+    }
+    return SliderUtils.join(entries, ",", false);
+  }
+  
+  public static String convertToHostsOnlyList(String quorum) throws
+      BadConfigException {
+    List<HostAndPort> hostAndPorts = splitToHostsAndPortsStrictly(quorum);
+    return ZookeeperUtils.buildHostsOnlyList(hostAndPorts);
+  }
+
+  public static List<HostAndPort> splitToHostsAndPortsStrictly(String quorum) throws
+      BadConfigException {
+    List<HostAndPort> hostAndPorts =
+        ZookeeperUtils.splitToHostsAndPorts(quorum);
+    if (hostAndPorts.isEmpty()) {
+      throw new BadConfigException("empty zookeeper quorum");
+    }
+    return hostAndPorts;
+  }
+  
+  public static int getFirstPort(String quorum, int defVal) throws
+      BadConfigException {
+    List<HostAndPort> hostAndPorts = splitToHostsAndPortsStrictly(quorum);
+    int port = hostAndPorts.get(0).getPortOrDefault(defVal);
+    return port;
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractClientProvider.java
new file mode 100644
index 0000000..510de5d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractClientProvider.java
@@ -0,0 +1,248 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.exceptions.BadClusterStateException;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.core.launch.AbstractLauncher;
+import org.codehaus.jettison.json.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.slider.api.ResourceKeys.COMPONENT_INSTANCES;
+import static org.apache.slider.api.ResourceKeys.DEF_YARN_CORES;
+import static org.apache.slider.api.ResourceKeys.DEF_YARN_MEMORY;
+import static org.apache.slider.api.ResourceKeys.YARN_CORES;
+import static org.apache.slider.api.ResourceKeys.YARN_MEMORY;
+
+public abstract class AbstractClientProvider extends Configured {
+  private static final Logger log =
+    LoggerFactory.getLogger(AbstractClientProvider.class);
+  protected static final ProviderUtils providerUtils =
+    new ProviderUtils(log);
+
+  public static final String PROVIDER_RESOURCE_BASE =
+    "org/apache/slider/providers/";
+  public static final String PROVIDER_RESOURCE_BASE_ROOT =
+    "/" + PROVIDER_RESOURCE_BASE;
+
+  public AbstractClientProvider(Configuration conf) {
+    super(conf);
+  }
+
+  public abstract String getName();
+
+  public abstract List<ProviderRole> getRoles();
+
+  /**
+   * Verify that an instance definition is considered valid by the provider
+   * @param instanceDefinition instance definition
+   * @throws SliderException if the configuration is not valid
+   */
+  public void validateInstanceDefinition(AggregateConf instanceDefinition, SliderFileSystem fs) throws
+      SliderException {
+
+    List<ProviderRole> roles = getRoles();
+    ConfTreeOperations resources =
+      instanceDefinition.getResourceOperations();
+    for (ProviderRole role : roles) {
+      String name = role.name;
+      MapOperations component = resources.getComponent(role.group);
+      if (component != null) {
+        String instances = component.get(COMPONENT_INSTANCES);
+        if (instances == null) {
+          String message = "No instance count provided for " + name;
+          log.error("{} with \n{}", message, resources.toString());
+          throw new BadClusterStateException(message);
+        }
+        String ram = component.get(YARN_MEMORY);
+        String cores = component.get(YARN_CORES);
+
+
+        providerUtils.getRoleResourceRequirement(ram,
+                                                 DEF_YARN_MEMORY,
+                                                 Integer.MAX_VALUE);
+        providerUtils.getRoleResourceRequirement(cores,
+                                                 DEF_YARN_CORES,
+                                                 Integer.MAX_VALUE);
+      }
+    }
+  }
+
+
+  /**
+   * Any provider-side alteration of a configuration can take place here.
+   * @param aggregateConf config to patch
+   * @throws IOException IO problems
+   * @throws SliderException Slider-specific issues
+   */
+  public void prepareInstanceConfiguration(AggregateConf aggregateConf) throws
+      SliderException,
+                                                                    IOException {
+    //default: do nothing
+  }
+
+
+  /**
+   * Prepare the AM settings for launch
+   * @param fileSystem filesystem
+   * @param serviceConf configuration of the client
+   * @param launcher launcher to set up
+   * @param instanceDescription instance description being launched
+   * @param snapshotConfDirPath
+   * @param generatedConfDirPath
+   * @param clientConfExtras
+   * @param libdir
+   * @param tempPath
+   * @param miniClusterTestRun flag set to true on a mini cluster run
+   * @throws IOException
+   * @throws SliderException
+   */
+  public void prepareAMAndConfigForLaunch(SliderFileSystem fileSystem,
+      Configuration serviceConf,
+      AbstractLauncher launcher,
+      AggregateConf instanceDescription,
+      Path snapshotConfDirPath,
+      Path generatedConfDirPath,
+      Configuration clientConfExtras,
+      String libdir,
+      Path tempPath,
+      boolean miniClusterTestRun)
+    throws IOException, SliderException {
+    
+  }
+  
+  /**
+   * Load in and merge in templates. Null arguments means "no such template"
+   * @param instanceConf instance to patch 
+   * @param internalTemplate patch to internal.json
+   * @param resourceTemplate path to resources.json
+   * @param appConfTemplate path to app_conf.json
+   * @throws IOException any IO problems
+   */
+  protected void mergeTemplates(AggregateConf instanceConf,
+                                String internalTemplate,
+                                String resourceTemplate,
+                                String appConfTemplate) throws IOException {
+    if (internalTemplate != null) {
+      ConfTreeOperations template =
+        ConfTreeOperations.fromResource(internalTemplate);
+      instanceConf.getInternalOperations()
+                  .mergeWithoutOverwrite(template.confTree);
+    }
+
+    if (resourceTemplate != null) {
+      ConfTreeOperations resTemplate =
+        ConfTreeOperations.fromResource(resourceTemplate);
+      instanceConf.getResourceOperations()
+                   .mergeWithoutOverwrite(resTemplate.confTree);
+    }
+   
+    if (appConfTemplate != null) {
+      ConfTreeOperations template =
+        ConfTreeOperations.fromResource(appConfTemplate);
+      instanceConf.getAppConfOperations()
+                   .mergeWithoutOverwrite(template.confTree);
+    }
+    
+  }
+
+  /**
+   * This is called pre-launch to validate that the cluster specification
+   * is valid. This can include checking that the security options
+   * are in the site files prior to launch, that there are no conflicting operations
+   * etc.
+   *
+   * This check is made prior to every launch of the cluster -so can 
+   * pick up problems which manually edited cluster files have added,
+   * or from specification files from previous versions.
+   *
+   * The provider MUST NOT change the remote specification. This is
+   * purely a pre-launch validation of options.
+   *
+   *
+   * @param sliderFileSystem filesystem
+   * @param clustername name of the cluster
+   * @param configuration cluster configuration
+   * @param instanceDefinition cluster specification
+   * @param clusterDirPath directory of the cluster
+   * @param generatedConfDirPath path to place generated artifacts
+   * @param secure flag to indicate that the cluster is secure
+   * @throws SliderException on any validation issue
+   * @throws IOException on any IO problem
+   */
+  public void preflightValidateClusterConfiguration(SliderFileSystem sliderFileSystem,
+                                                      String clustername,
+                                                      Configuration configuration,
+                                                      AggregateConf instanceDefinition,
+                                                      Path clusterDirPath,
+                                                      Path generatedConfDirPath,
+                                                      boolean secure)
+      throws SliderException, IOException {
+    validateInstanceDefinition(instanceDefinition, sliderFileSystem);
+  }
+
+  /**
+   * Return a set of application specific string tags.
+   * @return the set of tags.
+   */
+  public Set<String> getApplicationTags (SliderFileSystem fileSystem,
+                                         String appDef) throws SliderException {
+    return Collections.emptySet();
+  }
+
+  /**
+   * Process client operations for applications such as install, configure
+   * @param fileSystem
+   * @param registryOperations
+   * @param configuration
+   * @param operation
+   * @param clientInstallPath
+   * @param clientPackage
+   * @param clientConfig
+   * @param name
+   * @throws SliderException
+   */
+  public void processClientOperation(SliderFileSystem fileSystem,
+                                     RegistryOperations registryOperations,
+                                     Configuration configuration,
+                                     String operation,
+                                     File clientInstallPath,
+                                     File clientPackage,
+                                     JSONObject clientConfig,
+                                     String name)
+      throws SliderException {
+    throw new SliderException("Provider does not support client operations.");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
new file mode 100644
index 0000000..61b2655
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
@@ -0,0 +1,424 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.client.api.AMRMClient;
+import org.apache.hadoop.registry.client.binding.RegistryTypeUtils;
+import org.apache.hadoop.registry.client.exceptions.InvalidRecordException;
+import org.apache.hadoop.registry.client.types.AddressTypes;
+import org.apache.hadoop.registry.client.types.Endpoint;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.slider.api.ClusterDescription;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.tools.ConfigHelper;
+import org.apache.slider.common.tools.SliderFileSystem;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.core.main.ExitCodeProvider;
+import org.apache.slider.server.appmaster.actions.QueueAccess;
+import org.apache.slider.server.appmaster.operations.AbstractRMOperation;
+import org.apache.slider.server.appmaster.state.ContainerReleaseSelector;
+import org.apache.slider.server.appmaster.state.MostRecentContainerReleaseSelector;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+import org.apache.slider.server.appmaster.web.rest.agent.AgentRestOperations;
+import org.apache.slider.server.services.workflow.ForkedProcessService;
+import org.apache.slider.server.services.workflow.ServiceParent;
+import org.apache.slider.server.services.workflow.WorkflowSequenceService;
+import org.apache.slider.server.services.yarnregistry.YarnRegistryViewForProviders;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+/**
+ * The base class for provider services. It lets the implementations
+ * add sequences of operations, and propagates service failures
+ * upstream
+ */
+public abstract class AbstractProviderService
+    extends WorkflowSequenceService
+    implements
+    ProviderCore,
+    SliderKeys,
+    ProviderService {
+  private static final Logger log =
+    LoggerFactory.getLogger(AbstractProviderService.class);
+  protected StateAccessForProviders amState;
+  protected AgentRestOperations restOps;
+  protected URL amWebAPI;
+  protected YarnRegistryViewForProviders yarnRegistry;
+  protected QueueAccess queueAccess;
+
+  protected AbstractProviderService(String name) {
+    super(name);
+    setStopIfNoChildServicesAtStartup(false);
+  }
+
+  @Override
+  public Configuration getConf() {
+    return getConfig();
+  }
+
+  public StateAccessForProviders getAmState() {
+    return amState;
+  }
+
+  public QueueAccess getQueueAccess() {
+    return queueAccess;
+  }
+
+  public void setAmState(StateAccessForProviders amState) {
+    this.amState = amState;
+  }
+
+  @Override
+  public String getHumanName() {
+    return getName().toLowerCase(Locale.ENGLISH);
+  }
+  
+  @Override
+  public void bind(StateAccessForProviders stateAccessor,
+      QueueAccess queueAccess,
+      List<Container> liveContainers) {
+    this.amState = stateAccessor;
+    this.queueAccess = queueAccess;
+  }
+
+  @Override
+  public void bindToYarnRegistry(YarnRegistryViewForProviders yarnRegistry) {
+    this.yarnRegistry = yarnRegistry;
+  }
+
+  public YarnRegistryViewForProviders getYarnRegistry() {
+    return yarnRegistry;
+  }
+
+  @Override
+  public AgentRestOperations getAgentRestOperations() {
+    return restOps;
+  }
+
+  @Override
+  public void notifyContainerCompleted(ContainerId containerId) {
+  }
+
+  public void setAgentRestOperations(AgentRestOperations agentRestOperations) {
+    this.restOps = agentRestOperations;
+  }
+
+  /**
+   * Load a specific XML configuration file for the provider config
+   * @param confDir configuration directory
+   * @param siteXMLFilename provider-specific filename
+   * @return a configuration to be included in status
+   * @throws BadCommandArgumentsException argument problems
+   * @throws IOException IO problems
+   */
+  protected Configuration loadProviderConfigurationInformation(File confDir,
+                                                               String siteXMLFilename)
+    throws BadCommandArgumentsException, IOException {
+    Configuration siteConf;
+    File siteXML = new File(confDir, siteXMLFilename);
+    if (!siteXML.exists()) {
+      throw new BadCommandArgumentsException(
+        "Configuration directory %s doesn't contain %s - listing is %s",
+        confDir, siteXMLFilename, SliderUtils.listDir(confDir));
+    }
+
+    //now read it in
+    siteConf = ConfigHelper.loadConfFromFile(siteXML);
+    log.info("{} file is at {}", siteXMLFilename, siteXML);
+    log.info(ConfigHelper.dumpConfigToString(siteConf));
+    return siteConf;
+  }
+
+  /**
+   * No-op implementation of this method.
+   */
+  @Override
+  public void initializeApplicationConfiguration(
+      AggregateConf instanceDefinition, SliderFileSystem fileSystem)
+      throws IOException, SliderException {
+  }
+
+  /**
+   * No-op implementation of this method.
+   *
+   * {@inheritDoc}
+   */
+  @Override
+  public void validateApplicationConfiguration(AggregateConf instance,
+                                               File confDir,
+                                               boolean secure)
+      throws IOException, SliderException {
+
+  }
+
+  /**
+   * Scan through the roles and see if it is supported.
+   * @param role role to look for
+   * @return true if the role is known about -and therefore
+   * that a launcher thread can be deployed to launch it
+   */
+  @Override
+  public boolean isSupportedRole(String role) {
+    Collection<ProviderRole> roles = getRoles();
+    for (ProviderRole providedRole : roles) {
+      if (providedRole.name.equals(role)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * override point to allow a process to start executing in this container
+   * @param instanceDefinition cluster description
+   * @param confDir configuration directory
+   * @param env environment
+   * @param execInProgress the callback for the exec events
+   * @return false
+   * @throws IOException
+   * @throws SliderException
+   */
+  @Override
+  public boolean exec(AggregateConf instanceDefinition,
+      File confDir,
+      Map<String, String> env,
+      ProviderCompleted execInProgress) throws IOException, SliderException {
+    return false;
+  }
+
+  @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+  @Override // ExitCodeProvider
+  public int getExitCode() {
+    Throwable cause = getFailureCause();
+    if (cause != null) {
+      //failed for some reason
+      if (cause instanceof ExitCodeProvider) {
+        return ((ExitCodeProvider) cause).getExitCode();
+      }
+    }
+    ForkedProcessService lastProc = latestProcess();
+    if (lastProc == null || !lastProc.isProcessTerminated()) {
+      return 0;
+    } else {
+      return lastProc.getExitCode();
+    }
+  }
+
+  /**
+   * Return the latest forked process service that ran
+   * @return the forkes service
+   */
+  protected ForkedProcessService latestProcess() {
+    Service current = getActiveService();
+    Service prev = getPreviousService();
+
+    Service latest = current != null ? current : prev;
+    if (latest instanceof ForkedProcessService) {
+      return (ForkedProcessService) latest;
+    } else {
+      //its a composite object, so look inside it for a process
+      if (latest instanceof ServiceParent) {
+        return getFPSFromParentService((ServiceParent) latest);
+      } else {
+        //no match
+        return null;
+      }
+    }
+  }
+
+
+  /**
+   * Given a parent service, find the one that is a forked process
+   * @param serviceParent parent
+   * @return the forked process service or null if there is none
+   */
+  protected ForkedProcessService getFPSFromParentService(ServiceParent serviceParent) {
+    List<Service> services = serviceParent.getServices();
+    for (Service s : services) {
+      if (s instanceof ForkedProcessService) {
+        return (ForkedProcessService) s;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * if we are already running, start this service
+   */
+  protected void maybeStartCommandSequence() {
+    if (isInState(STATE.STARTED)) {
+      startNextService();
+    }
+  }
+
+  /**
+   * Create a new forked process service with the given
+   * name, environment and command list -then add it as a child
+   * for execution in the sequence.
+   *
+   * @param name command name
+   * @param env environment
+   * @param commands command line
+   * @throws IOException
+   * @throws SliderException
+   */
+  protected ForkedProcessService queueCommand(String name,
+                              Map<String, String> env,
+                              List<String> commands) throws
+                                                     IOException,
+      SliderException {
+    ForkedProcessService process = buildProcess(name, env, commands);
+    //register the service for lifecycle management; when this service
+    //is terminated, so is the master process
+    addService(process);
+    return process;
+  }
+
+  public ForkedProcessService buildProcess(String name,
+                                           Map<String, String> env,
+                                           List<String> commands) throws
+                                                                  IOException,
+      SliderException {
+    ForkedProcessService process;
+    process = new ForkedProcessService(name);
+    process.init(getConfig());
+    process.build(env, commands);
+    return process;
+  }
+
+  /*
+   * Build the provider status, can be empty
+   * @return the provider status - map of entries to add to the info section
+   */
+  @Override
+  public Map<String, String> buildProviderStatus() {
+    return new HashMap<String, String>();
+  }
+
+  /*
+  Build the monitor details. The base implementation includes all the external URL endpoints
+  in the external view
+   */
+  @Override
+  public Map<String, MonitorDetail> buildMonitorDetails(ClusterDescription clusterDesc) {
+    Map<String, MonitorDetail> details = new LinkedHashMap<String, MonitorDetail>();
+
+    // add in all the endpoints
+    buildEndpointDetails(details);
+
+    return details;
+  }
+
+  @Override
+  public void buildEndpointDetails(Map<String, MonitorDetail> details) {
+    ServiceRecord self = yarnRegistry.getSelfRegistration();
+
+    List<Endpoint> externals = self.external;
+    for (Endpoint endpoint : externals) {
+      String addressType = endpoint.addressType;
+      if (AddressTypes.ADDRESS_URI.equals(addressType)) {
+        try {
+          List<URL> urls = RegistryTypeUtils.retrieveAddressURLs(endpoint);
+          if (!urls.isEmpty()) {
+            details.put(endpoint.api, new MonitorDetail(urls.get(0).toString(), true));
+          }
+        } catch (InvalidRecordException  | MalformedURLException ignored) {
+          // Ignored
+        }
+
+      }
+
+    }
+  }
+
+  @Override
+  public void applyInitialRegistryDefinitions(URL amWebURI,
+      URL agentOpsURI,
+      URL agentStatusURI,
+      ServiceRecord serviceRecord)
+    throws IOException {
+      this.amWebAPI = amWebURI;
+  }
+
+  /**
+   * {@inheritDoc}
+   * 
+   * 
+   * @return The base implementation returns the most recent containers first.
+   */
+  @Override
+  public ContainerReleaseSelector createContainerReleaseSelector() {
+    return new MostRecentContainerReleaseSelector();
+  }
+
+  @Override
+  public void releaseAssignedContainer(ContainerId containerId) {
+    // no-op
+  }
+
+  @Override
+  public void addContainerRequest(AMRMClient.ContainerRequest req) {
+    // no-op
+  }
+
+  @Override
+  public void cancelSingleRequest(AMRMClient.ContainerRequest request) {
+    // no-op
+  }
+
+  @Override
+  public int cancelContainerRequests(Priority priority1,
+      Priority priority2,
+      int count) {
+    return 0;
+  }
+
+  @Override
+  public void execute(List<AbstractRMOperation> operations) {
+    for (AbstractRMOperation operation : operations) {
+      operation.execute(this);
+    }
+  }
+  /**
+   * No-op implementation of this method.
+   */
+  @Override
+  public void rebuildContainerDetails(List<Container> liveContainers,
+      String applicationId, Map<Integer, ProviderRole> providerRoles) {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/MonitorDetail.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/MonitorDetail.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/MonitorDetail.java
new file mode 100644
index 0000000..27d3415
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/MonitorDetail.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.providers;
+
+/**
+ * Details about some exported information from a provider to the AM web UI.
+ */
+public class MonitorDetail {
+
+  private final String value;
+  private final boolean isUrl;
+
+  public MonitorDetail(String value, boolean isUrl) {
+    this.value = value;
+    this.isUrl = isUrl;
+  }
+
+  public String getValue() {
+    return value;
+  }
+
+  public boolean isUrl() {
+    return isUrl;
+  }
+
+  public String toString() {
+    return "MonitorDetail[" + value + " isUrl=" + isUrl + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/PlacementPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/PlacementPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/PlacementPolicy.java
new file mode 100644
index 0000000..128dd5d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/PlacementPolicy.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers;
+
+/**
+ * Placement values.
+ * This is nominally a bitmask, though not all values make sense
+ */
+public class PlacementPolicy {
+
+  /**
+   * Default value: history used, anti-affinity hinted at on rebuild/flex up
+   */
+  public static final int NONE = 0;
+
+  /**
+   * Default value: history used, anti-affinity hinted at on rebuild/flex up
+   */
+  public static final int DEFAULT = NONE;
+
+  /**
+   * Strict placement: when asking for an instance for which there is
+   * history, mandate that it is strict
+   */
+  public static final int STRICT = 1;
+
+  /**
+   * No data locality; do not use placement history
+   */
+  public static final int ANYWHERE = 2;
+
+  /**
+   * @Deprecated: use {@link #ANYWHERE}
+   */
+  @Deprecated
+  public static final int NO_DATA_LOCALITY = ANYWHERE;
+
+  /**
+   * Anti-affinity is mandatory.
+   */
+  public static final int ANTI_AFFINITY_REQUIRED = 4;
+  
+  /**
+   * Exclude from flexing; used internally to mark AMs.
+   */
+  public static final int EXCLUDE_FROM_FLEXING = 16;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/PlacementPolicyOptions.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/PlacementPolicyOptions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/PlacementPolicyOptions.java
new file mode 100644
index 0000000..e61f944
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/PlacementPolicyOptions.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers;
+
+public enum PlacementPolicyOptions {
+
+  EXCLUDE_FROM_FLEXING,
+  NO_DATA_LOCALITY,
+  ANTI_AFFINITY_REQUIRED,
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderCompleted.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderCompleted.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderCompleted.java
new file mode 100644
index 0000000..f6ff4fd
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderCompleted.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers;
+
+/**
+ * This is the callback triggered by the {@link ProviderCompletedCallable}
+ * when it generates a notification
+ */
+public interface ProviderCompleted {
+  
+  public void eventCallbackEvent(Object parameter);
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderCompletedCallable.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderCompletedCallable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderCompletedCallable.java
new file mode 100644
index 0000000..47939c9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/ProviderCompletedCallable.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.providers;
+
+import java.util.concurrent.Callable;
+
+public class ProviderCompletedCallable implements Callable<Object> {
+
+  private final ProviderCompleted callback;
+  private final Object parameter;
+
+  public ProviderCompletedCallable(ProviderCompleted callback, Object parameter) {
+    this.callback = callback;
+    this.parameter = parameter;
+  }
+
+  @Override
+  public Object call() throws Exception {
+    callback.eventCallbackEvent(parameter);
+    return parameter;
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[67/76] [abbrv] hadoop git commit: YARN-5735. Make the service REST API use the app timeout feature YARN-4205. Contributed by Jian He

Posted by ji...@apache.org.
YARN-5735. Make the service REST API use the app timeout feature YARN-4205. Contributed by Jian He


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

Branch: refs/heads/yarn-native-services
Commit: be695958b8216557f9aa46ed410ce9320cc094bf
Parents: b279e66
Author: Gour Saha <go...@apache.org>
Authored: Fri Oct 14 17:40:51 2016 -0700
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 .../api/impl/ApplicationApiService.java         | 10 ++++--
 .../org/apache/slider/client/SliderClient.java  | 33 ++++++++++----------
 .../AbstractClusterBuildingActionArgs.java      |  5 +++
 .../slider/common/params/ActionThawArgs.java    |  6 ++++
 .../apache/slider/common/params/Arguments.java  |  1 +
 5 files changed, 36 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/be695958/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
index 21cf113..73df4a1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
@@ -347,7 +347,7 @@ public class ApplicationApiService implements ApplicationApi {
     if (queueName != null && queueName.trim().length() > 0) {
       createArgs.queue = queueName.trim();
     }
-
+    createArgs.lifetime = application.getLifetime();
     return invokeSliderClientRunnable(new SliderClientContextRunnable<String>() {
       @Override
       public String run(SliderClient sliderClient) throws YarnException,
@@ -1246,13 +1246,17 @@ public class ApplicationApiService implements ApplicationApi {
     });
   }
 
-  private Response startSliderApplication(final String appName)
+  private Response startSliderApplication(final String appName, Application app)
       throws IOException, YarnException, InterruptedException {
     return invokeSliderClientRunnable(new SliderClientContextRunnable<Response>() {
       @Override
       public Response run(SliderClient sliderClient) throws YarnException,
           IOException, InterruptedException {
         ActionThawArgs thawArgs = new ActionThawArgs();
+        if (app.getLifetime() == null) {
+          app.setLifetime(DEFAULT_UNLIMITED_LIFETIME);
+        }
+        thawArgs.lifetime = app.getLifetime();
         int returnCode = sliderClient.actionThaw(appName, thawArgs);
         if (returnCode == 0) {
           logger.info("Successfully started application {}", appName);
@@ -1344,7 +1348,7 @@ public class ApplicationApiService implements ApplicationApi {
       try {
         int livenessCheck = getSliderList(appName);
         if (livenessCheck != 0) {
-          return startSliderApplication(appName);
+          return startSliderApplication(appName, updateAppData);
         } else {
           logger.info("Application {} is already running", appName);
           ApplicationStatus applicationStatus = new ApplicationStatus();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/be695958/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
index fe4f1d2..2840c4b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/client/SliderClient.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.NodeReport;
@@ -734,7 +735,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
       sliderFileSystem.getFileSystem().delete(clusterDirectory, true);
       throw e;
     }
-    return startCluster(clustername, createArgs);
+    return startCluster(clustername, createArgs, createArgs.lifetime);
   }
 
   @Override
@@ -1960,14 +1961,13 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
    *
    * @param clustername name of the cluster.
    * @param launchArgs launch arguments
+   * @param lifetime
    * @return the exit code
    * @throws YarnException
    * @throws IOException
    */
-  protected int startCluster(String clustername,
-                           LaunchArgsAccessor launchArgs) throws
-                                                          YarnException,
-                                                          IOException {
+  protected int startCluster(String clustername, LaunchArgsAccessor launchArgs,
+      long lifetime) throws YarnException, IOException {
     Path clusterDirectory = sliderFileSystem.buildClusterDirPath(clustername);
     AggregateConf instanceDefinition = loadInstanceDefinitionUnresolved(
       clustername,
@@ -1975,7 +1975,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
 
     LaunchedApplication launchedApplication =
       launchApplication(clustername, clusterDirectory, instanceDefinition,
-                        serviceArgs.isDebug());
+                        serviceArgs.isDebug(), lifetime);
 
     if (launchArgs.getOutputFile() != null) {
       // output file has been requested. Get the app report and serialize it
@@ -2044,9 +2044,8 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   }
 
   protected AppMasterLauncher setupAppMasterLauncher(String clustername,
-      Path clusterDirectory,
-      AggregateConf instanceDefinition,
-      boolean debugAM)
+      Path clusterDirectory, AggregateConf instanceDefinition, boolean debugAM,
+      long lifetime)
     throws YarnException, IOException{
     deployedClusterName = clustername;
     validateClusterName(clustername);
@@ -2119,7 +2118,10 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     ApplicationId appId = amLauncher.getApplicationId();
     // set the application name;
     amLauncher.setKeepContainersOverRestarts(true);
-
+    // set lifetime in submission context;
+    Map<ApplicationTimeoutType, Long> appTimeout = new HashMap<>();
+    appTimeout.put(ApplicationTimeoutType.LIFETIME, lifetime);
+    amLauncher.submissionContext.setApplicationTimeouts(appTimeout);
     int maxAppAttempts = config.getInt(KEY_AM_RESTART_LIMIT, 0);
     amLauncher.setMaxAppAttempts(maxAppAttempts);
 
@@ -2383,20 +2385,19 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
    * @param clusterDirectory cluster dir
    * @param instanceDefinition the instance definition
    * @param debugAM enable debug AM options
+   * @param lifetime
    * @return the launched application
    * @throws YarnException
    * @throws IOException
    */
-  public LaunchedApplication launchApplication(String clustername,
-                                               Path clusterDirectory,
-                                               AggregateConf instanceDefinition,
-                                               boolean debugAM)
+  public LaunchedApplication launchApplication(String clustername, Path clusterDirectory,
+      AggregateConf instanceDefinition, boolean debugAM, long lifetime)
     throws YarnException, IOException {
 
     AppMasterLauncher amLauncher = setupAppMasterLauncher(clustername,
         clusterDirectory,
         instanceDefinition,
-        debugAM);
+        debugAM, lifetime);
 
     applicationId = amLauncher.getApplicationId();
     log.info("Submitting application {}", applicationId);
@@ -3254,7 +3255,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     verifyNoLiveClusters(clustername, "Start");
 
     //start the cluster
-    return startCluster(clustername, thaw);
+    return startCluster(clustername, thaw, thaw.lifetime);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/be695958/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractClusterBuildingActionArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractClusterBuildingActionArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractClusterBuildingActionArgs.java
index 2a5eedc..3cb75e1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractClusterBuildingActionArgs.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/AbstractClusterBuildingActionArgs.java
@@ -102,6 +102,11 @@ public abstract class AbstractClusterBuildingActionArgs extends
              description = "Queue to submit the application")
   public String queue;
 
+  @Parameter(names = {ARG_LIFETIME},
+      description = "Life time of the application since application started at"
+          + " running state")
+  public long lifetime;
+
   @ParametersDelegate
   public ComponentArgsDelegate componentDelegate = new ComponentArgsDelegate();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/be695958/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionThawArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionThawArgs.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionThawArgs.java
index b43a14e..2bd856f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionThawArgs.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/ActionThawArgs.java
@@ -18,6 +18,7 @@
 
 package org.apache.slider.common.params;
 
+import com.beust.jcommander.Parameter;
 import com.beust.jcommander.Parameters;
 import com.beust.jcommander.ParametersDelegate;
 
@@ -43,6 +44,11 @@ public class ActionThawArgs extends AbstractActionArgs implements
   @ParametersDelegate
   LaunchArgsDelegate launchArgs = new LaunchArgsDelegate();
 
+  @Parameter(names = {ARG_LIFETIME},
+      description = "Life time of the application since application started at"
+          + " running state")
+  public long lifetime;
+
   @Override
   public String getRmAddress() {
     return launchArgs.getRmAddress();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/be695958/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/Arguments.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
index aec4e26..cbf7e59 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
@@ -103,6 +103,7 @@ public interface Arguments {
   String ARG_PRINCIPAL = "--principal";
   String ARG_PROVIDER = "--provider";
   String ARG_QUEUE = "--queue";
+  String ARG_LIFETIME = "--lifetime";
   String ARG_REPLACE_PKG = "--replacepkg";
   String ARG_RESOURCE = "--resource";
   String ARG_RESOURCES = "--resources";


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[02/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcess.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcess.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcess.java
new file mode 100644
index 0000000..9e9e7ac
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcess.java
@@ -0,0 +1,598 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * Execute a long-lived process.
+ *
+ * <p>
+ * Hadoop's {@link org.apache.hadoop.util.Shell} class assumes it is executing
+ * a short lived application; this class allows for the process to run for the
+ * life of the Java process that forked it.
+ * It is designed to be embedded inside a YARN service, though this is not
+ * the sole way that it can be used
+ * <p>
+ * Key Features:
+ * <ol>
+ *   <li>Output is streamed to the output logger provided</li>.
+ *   <li>the input stream is closed as soon as the process starts.</li>
+ *   <li>The most recent lines of output are saved to a linked list</li>.
+ *   <li>A synchronous callback, {@link LongLivedProcessLifecycleEvent},
+ *   is raised on the start and finish of a process.</li>
+ * </ol>
+ * 
+ */
+public class LongLivedProcess implements Runnable {
+  /**
+   * Limit on number of lines to retain in the "recent" line list:{@value}
+   */
+  public static final int RECENT_LINE_LOG_LIMIT = 64;
+
+  /**
+   * Const defining the time in millis between polling for new text.
+   */
+  private static final int STREAM_READER_SLEEP_TIME = 200;
+  
+  /**
+   * limit on the length of a stream before it triggers an automatic newline.
+   */
+  private static final int LINE_LENGTH = 256;
+  private final ProcessBuilder processBuilder;
+  private Process process;
+  private Integer exitCode = null;
+  private final String name;
+  private final ExecutorService processExecutor;
+  private final ExecutorService logExecutor;
+  
+  private ProcessStreamReader processStreamReader;
+  //list of recent lines, recorded for extraction into reports
+  private final List<String> recentLines = new LinkedList<>();
+  private int recentLineLimit = RECENT_LINE_LOG_LIMIT;
+  private LongLivedProcessLifecycleEvent lifecycleCallback;
+  private final AtomicBoolean finalOutputProcessed = new AtomicBoolean(false);
+
+  /**
+   * Log supplied in the constructor for the spawned process -accessible
+   * to inner classes
+   */
+  private Logger processLog;
+  
+  /**
+   * Class log -accessible to inner classes
+   */
+  private static final Logger LOG = LoggerFactory.getLogger(LongLivedProcess.class);
+
+  /**
+   *  flag to indicate that the process is done
+   */
+  private final AtomicBoolean finished = new AtomicBoolean(false);
+
+  /**
+   * Create an instance
+   * @param name process name
+   * @param processLog log for output (or null)
+   * @param commands command list
+   */
+  public LongLivedProcess(String name,
+      Logger processLog,
+      List<String> commands) {
+    Preconditions.checkArgument(commands != null, "commands");
+
+    this.name = name;
+    this.processLog = processLog;
+    ServiceThreadFactory factory = new ServiceThreadFactory(name, true);
+    processExecutor = Executors.newSingleThreadExecutor(factory);
+    logExecutor = Executors.newSingleThreadExecutor(factory);
+    processBuilder = new ProcessBuilder(commands);
+    processBuilder.redirectErrorStream(false);
+  }
+
+  /**
+   * Set the limit on recent lines to retain
+   * @param recentLineLimit size of rolling list of recent lines.
+   */
+  public void setRecentLineLimit(int recentLineLimit) {
+    this.recentLineLimit = recentLineLimit;
+  }
+
+  /**
+   * Set an optional application exit callback
+   * @param lifecycleCallback callback to notify on application exit
+   */
+  public void setLifecycleCallback(LongLivedProcessLifecycleEvent lifecycleCallback) {
+    this.lifecycleCallback = lifecycleCallback;
+  }
+
+  /**
+   * Add an entry to the environment
+   * @param envVar envVar -must not be null
+   * @param val value 
+   */
+  public void setEnv(String envVar, String val) {
+    Preconditions.checkArgument(envVar != null, "envVar");
+    Preconditions.checkArgument(val != null, "val");
+    processBuilder.environment().put(envVar, val);
+  }
+
+  /**
+   * Bulk set the environment from a map. This does
+   * not replace the existing environment, just extend it/overwrite single
+   * entries.
+   * @param map map to add
+   */
+  public void putEnvMap(Map<String, String> map) {
+    for (Map.Entry<String, String> entry : map.entrySet()) {
+      String val = entry.getValue();
+      String key = entry.getKey();
+      setEnv(key, val);
+    }
+  }
+
+  /**
+   * Get the process environment
+   * @param variable environment variable
+   * @return the value or null if there is no match
+   */
+  public String getEnv(String variable) {
+    return processBuilder.environment().get(variable);
+  }
+
+  /**
+   * Set the process log. Ignored once the process starts
+   * @param processLog new log ... may be null
+   */
+  public void setProcessLog(Logger processLog) {
+    this.processLog = processLog;
+  }
+
+  /**
+   * Get the process reference
+   * @return the process -null if the process is  not started
+   */
+  public Process getProcess() {
+    return process;
+  }
+
+  /**
+   * Get the process builder -this can be manipulated
+   * up to the start() operation. As there is no synchronization
+   * around it, it must only be used in the same thread setting up the commmand.
+   * @return the process builder
+   */
+  public ProcessBuilder getProcessBuilder() {
+    return processBuilder;
+  }
+
+  /**
+   * Get the command list
+   * @return the comands
+   */
+  public List<String> getCommands() {
+    return processBuilder.command();
+  }
+
+  public String getCommand() {
+    return getCommands().get(0);
+  }
+
+  /**
+   * probe to see if the process is running
+   * @return true iff the process has been started and is not yet finished
+   */
+  public boolean isRunning() {
+    return process != null && !finished.get();
+  }
+
+  /**
+   * Get the exit code: null until the process has finished
+   * @return the exit code or null
+   */
+  public Integer getExitCode() {
+    return exitCode;
+  }
+  
+    /**
+   * Get the exit code sign corrected: null until the process has finished
+   * @return the exit code or null
+   */
+  public Integer getExitCodeSignCorrected() {
+    Integer result;
+    if (exitCode != null) {
+      result = (exitCode << 24) >> 24;
+    } else {
+      result = null;
+    }
+    return result;
+  }
+
+  /**
+   * Stop the process if it is running.
+   * This will trigger an application completion event with the given exit code
+   */
+  public void stop() {
+    if (!isRunning()) {
+      return;
+    }
+    process.destroy();
+  }
+
+  /**
+   * Get a text description of the builder suitable for log output
+   * @return a multiline string 
+   */
+  protected String describeBuilder() {
+    StringBuilder buffer = new StringBuilder();
+    for (String arg : processBuilder.command()) {
+      buffer.append('"').append(arg).append("\" ");
+    }
+    return buffer.toString();
+  }
+
+  /**
+   * Dump the environment to a string builder
+   * @param buffer the buffer to append to
+   */
+  public void dumpEnv(StringBuilder buffer) {
+    buffer.append("\nEnvironment\n-----------");
+    Map<String, String> env = processBuilder.environment();
+    Set<String> keys = env.keySet();
+    List<String> sortedKeys = new ArrayList<String>(keys);
+    Collections.sort(sortedKeys);
+    for (String key : sortedKeys) {
+      buffer.append(key).append("=").append(env.get(key)).append('\n');
+    }
+  }
+
+  /**
+   * Exec the process
+   * @return the process
+   * @throws IOException on aany failure to start the process
+   * @throws FileNotFoundException if the process could not be found
+   */
+  private Process spawnChildProcess() throws IOException {
+    if (process != null) {
+      throw new IOException("Process already started");
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Spawning process:\n " + describeBuilder());
+    }
+    try {
+      process = processBuilder.start();
+    } catch (IOException e) {
+      // on windows, upconvert DOS error 2 from ::CreateProcess()
+      // to its real meaning: FileNotFound
+      if (e.toString().contains("CreateProcess error=2")) {
+        FileNotFoundException fnfe =
+            new FileNotFoundException(e.toString());
+        fnfe.initCause(e);
+        throw fnfe;
+      } else {
+        throw e;
+      }
+    }
+    return process;
+  }
+
+  /**
+   * Entry point for waiting for the program to finish
+   */
+  @Override // Runnable
+  public void run() {
+    Preconditions.checkNotNull(process, "null process");
+    LOG.debug("Lifecycle callback thread running");
+    //notify the callback that the process has started
+    if (lifecycleCallback != null) {
+      lifecycleCallback.onProcessStarted(this);
+    }
+    try {
+      //close stdin for the process
+      IOUtils.closeStream(process.getOutputStream());
+      exitCode = process.waitFor();
+    } catch (InterruptedException e) {
+      LOG.debug("Process wait interrupted -exiting thread", e);
+    } finally {
+      //here the process has finished
+      LOG.debug("process {} has finished", name);
+      //tell the logger it has to finish too
+      finished.set(true);
+
+      // shut down the threads
+      logExecutor.shutdown();
+      try {
+        logExecutor.awaitTermination(60, TimeUnit.SECONDS);
+      } catch (InterruptedException ignored) {
+        //ignored
+      }
+
+      //now call the callback if it is set
+      if (lifecycleCallback != null) {
+        lifecycleCallback.onProcessExited(this, exitCode,
+            getExitCodeSignCorrected());
+      }
+    }
+  }
+
+  /**
+   * Spawn the application
+   * @throws IOException IO problems
+   */
+  public void start() throws IOException {
+
+    spawnChildProcess();
+    processStreamReader =
+        new ProcessStreamReader(processLog, STREAM_READER_SLEEP_TIME);
+    logExecutor.submit(processStreamReader);
+    processExecutor.submit(this);
+  }
+
+  /**
+   * Get the lines of recent output
+   * @return the last few lines of output; an empty list if there are none
+   * or the process is not actually running
+   */
+  public synchronized List<String> getRecentOutput() {
+    return new ArrayList<String>(recentLines);
+  }
+
+  /**
+   * @return whether lines of recent output are empty
+   */
+  public synchronized boolean isRecentOutputEmpty() {
+    return recentLines.isEmpty();
+  }
+
+  /**
+   * Query to see if the final output has been processed
+   * @return
+   */
+  public boolean isFinalOutputProcessed() {
+    return finalOutputProcessed.get();
+  }
+
+  /**
+   * Get the recent output from the process, or [] if not defined
+   *
+   * @param finalOutput flag to indicate "wait for the final output of the process"
+   * @param duration the duration, in ms, 
+   * ro wait for recent output to become non-empty
+   * @return a possibly empty list
+   */
+  public List<String> getRecentOutput(boolean finalOutput, int duration) {
+    long start = System.currentTimeMillis();
+    while (System.currentTimeMillis() - start <= duration) {
+      boolean finishedOutput;
+      if (finalOutput) {
+        // final flag means block until all data is done
+        finishedOutput = isFinalOutputProcessed();
+      } else {
+        // there is some output
+        finishedOutput = !isRecentOutputEmpty();
+      }
+      if (finishedOutput) {
+        break;
+      }
+      try {
+        Thread.sleep(100);
+      } catch (InterruptedException ie) {
+        Thread.currentThread().interrupt();
+        break;
+      }
+    }
+    return getRecentOutput();
+  }
+
+  /**
+   * add the recent line to the list of recent lines; deleting
+   * an earlier on if the limit is reached.
+   *
+   * Implementation note: yes, a circular array would be more
+   * efficient, especially with some power of two as the modulo,
+   * but is it worth the complexity and risk of errors for
+   * something that is only called once per line of IO?
+   * @param line line to record
+   * @param isErrorStream is the line from the error stream
+   * @param logger logger to log to - null for no logging
+   */
+  private synchronized void recordRecentLine(String line,
+      boolean isErrorStream,
+      Logger logger) {
+    if (line == null) {
+      return;
+    }
+    String entry = (isErrorStream ? "[ERR] " : "[OUT] ") + line;
+    recentLines.add(entry);
+    if (recentLines.size() > recentLineLimit) {
+      recentLines.remove(0);
+    }
+    if (logger != null) {
+      if (isErrorStream) {
+        logger.warn(line);
+      } else {
+        logger.info(line);
+      }
+    }
+  }
+
+  /**
+   * Class to read data from the two process streams, and, when run in a thread
+   * to keep running until the <code>done</code> flag is set. 
+   * Lines are fetched from stdout and stderr and logged at info and error
+   * respectively.
+   */
+
+  private class ProcessStreamReader implements Runnable {
+    private final Logger streamLog;
+    private final int sleepTime;
+
+    /**
+     * Create an instance
+     * @param streamLog log -or null to disable logging (recent entries
+     * will still be retained)
+     * @param sleepTime time to sleep when stopping
+     */
+    private ProcessStreamReader(Logger streamLog, int sleepTime) {
+      this.streamLog = streamLog;
+      this.sleepTime = sleepTime;
+    }
+
+    /**
+     * Return a character if there is one, -1 if nothing is ready yet
+     * @param reader reader
+     * @return the value from the reader, or -1 if it is not ready
+     * @throws IOException IO problems
+     */
+    private int readCharNonBlocking(BufferedReader reader) throws IOException {
+      if (reader.ready()) {
+        return reader.read();
+      } else {
+        return -1;
+      }
+    }
+
+    /**
+     * Read in a line, or, if the limit has been reached, the buffer
+     * so far
+     * @param reader source of data
+     * @param line line to build
+     * @param limit limit of line length
+     * @return true if the line can be printed
+     * @throws IOException IO trouble
+     */
+    @SuppressWarnings("NestedAssignment")
+    private boolean readAnyLine(BufferedReader reader,
+                                StringBuilder line,
+                                int limit)
+      throws IOException {
+      int next;
+      while ((-1 != (next = readCharNonBlocking(reader)))) {
+        if (next != '\n') {
+          line.append((char) next);
+          limit--;
+          if (line.length() > limit) {
+            //enough has been read in to print it any
+            return true;
+          }
+        } else {
+          //line end return flag to say so
+          return true;
+        }
+      }
+      //here the end of the stream is hit, or the limit
+      return false;
+    }
+
+
+    @Override //Runnable
+    @SuppressWarnings("IOResourceOpenedButNotSafelyClosed")
+    public void run() {
+      BufferedReader errReader = null;
+      BufferedReader outReader = null;
+      StringBuilder outLine = new StringBuilder(LINE_LENGTH);
+      StringBuilder errorLine = new StringBuilder(LINE_LENGTH);
+      try {
+        errReader = new BufferedReader(
+            new InputStreamReader(process.getErrorStream()));
+        outReader = new BufferedReader(
+            new InputStreamReader(process.getInputStream()));
+        while (!finished.get()) {
+          boolean processed = false;
+          if (readAnyLine(errReader, errorLine, LINE_LENGTH)) {
+            recordRecentLine(errorLine.toString(), true, streamLog);
+            errorLine.setLength(0);
+            processed = true;
+          }
+          if (readAnyLine(outReader, outLine, LINE_LENGTH)) {
+            recordRecentLine(outLine.toString(), false, streamLog);
+            outLine.setLength(0);
+            processed |= true;
+          }
+          if (!processed && !finished.get()) {
+            //nothing processed: wait a bit for data.
+            try {
+              Thread.sleep(sleepTime);
+            } catch (InterruptedException e) {
+              //ignore this, rely on the done flag
+              LOG.debug("Ignoring ", e);
+            }
+          }
+        }
+        // finished: cleanup
+
+        //print the current error line then stream through the rest
+        recordFinalOutput(errReader, errorLine, true, streamLog);
+        //now do the info line
+        recordFinalOutput(outReader, outLine, false, streamLog);
+
+      } catch (Exception ignored) {
+        LOG.warn("encountered {}", ignored, ignored);
+        //process connection has been torn down
+      } finally {
+        // close streams
+        IOUtils.closeStream(errReader);
+        IOUtils.closeStream(outReader);
+        //mark output as done
+        finalOutputProcessed.set(true);
+      }
+    }
+
+    /**
+     * Record the final output of a process stream
+     * @param reader reader of output
+     * @param lineBuilder string builder into which line is built
+     * @param isErrorStream flag to indicate whether or not this is the
+     * is the line from the error stream
+     * @param logger logger to log to
+     * @throws IOException
+     */
+    protected void recordFinalOutput(BufferedReader reader,
+        StringBuilder lineBuilder, boolean isErrorStream, Logger logger) throws
+        IOException {
+      String line = lineBuilder.toString();
+      recordRecentLine(line, isErrorStream, logger);
+      line = reader.readLine();
+      while (line != null) {
+        recordRecentLine(line, isErrorStream, logger);
+        line = reader.readLine();
+        if (Thread.interrupted()) {
+          break;
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcessLifecycleEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcessLifecycleEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcessLifecycleEvent.java
new file mode 100644
index 0000000..a13b508
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcessLifecycleEvent.java
@@ -0,0 +1,41 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+/**
+ * Callback when a long-lived application exits
+ */
+public interface LongLivedProcessLifecycleEvent {
+
+  /**
+   * Callback when a process is started
+   * @param process the process invoking the callback
+   */
+  void onProcessStarted(LongLivedProcess process);
+
+  /**
+   * Callback when a process has finished
+   * @param process the process invoking the callback
+   * @param exitCode exit code from the process
+   * @param signCorrectedCode the code- as sign corrected
+   */
+  void onProcessExited(LongLivedProcess process,
+      int exitCode,
+      int signCorrectedCode);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceParent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceParent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceParent.java
new file mode 100644
index 0000000..a123584
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceParent.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+import org.apache.hadoop.service.Service;
+
+import java.util.List;
+
+/**
+ * Interface for accessing services that contain one or more child
+ * services. 
+ */
+public interface ServiceParent extends Service {
+
+  /**
+   * Add a child service. It must be in a consistent state with the
+   * service to which it is being added.
+   * @param service the service to add.
+   */
+  void addService(Service service);
+
+  /**
+   * Get an unmodifiable list of services
+   * @return a list of child services at the time of invocation -
+   * added services will not be picked up.
+   */
+  List<Service> getServices();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceTerminatingCallable.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceTerminatingCallable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceTerminatingCallable.java
new file mode 100644
index 0000000..5ebf77c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceTerminatingCallable.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.service.Service;
+
+import java.util.concurrent.Callable;
+
+/**
+ * A runnable which terminates its owner; it also catches any
+ * exception raised and can serve it back.  
+ * 
+ */
+public class ServiceTerminatingCallable<V> implements Callable<V> {
+
+  private final Service owner;
+  private Exception exception;
+  /**
+   * This is the callback
+   */
+  private final Callable<V> callable;
+
+
+  /**
+   * Create an instance. If the owner is null, the owning service
+   * is not terminated.
+   * @param owner owning service -can be null
+   * @param callable callback.
+   */
+  public ServiceTerminatingCallable(Service owner,
+      Callable<V> callable) {
+    Preconditions.checkArgument(callable != null, "null callable");
+    this.owner = owner;
+    this.callable = callable;
+  }
+
+
+  /**
+   * Get the owning service
+   * @return the service to receive notification when
+   * the runnable completes.
+   */
+  public Service getOwner() {
+    return owner;
+  }
+
+  /**
+   * Any exception raised by inner <code>action's</code> run.
+   * @return an exception or null.
+   */
+  public Exception getException() {
+    return exception;
+  }
+
+  /**
+   * Delegates the call to the callable supplied in the constructor,
+   * then calls the stop() operation on its owner. Any exception
+   * is caught, noted and rethrown
+   * @return the outcome of the delegated call operation
+   * @throws Exception if one was raised.
+   */
+  @Override
+  public V call() throws Exception {
+    try {
+      return callable.call();
+    } catch (Exception e) {
+      exception = e;
+      throw e;
+    } finally {
+      if (owner != null) {
+        owner.stop();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceTerminatingRunnable.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceTerminatingRunnable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceTerminatingRunnable.java
new file mode 100644
index 0000000..dc591df
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceTerminatingRunnable.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.service.Service;
+
+/**
+ * A runnable which terminates its after running; it also catches any
+ * exception raised and can serve it back. 
+ */
+public class ServiceTerminatingRunnable implements Runnable {
+
+  private final Service owner;
+  private final Runnable action;
+  private Exception exception;
+
+  /**
+   * Create an instance.
+   * @param owner owning service
+   * @param action action to execute before terminating the service
+   */
+  public ServiceTerminatingRunnable(Service owner, Runnable action) {
+    Preconditions.checkArgument(owner != null, "null owner");
+    Preconditions.checkArgument(action != null, "null action");
+    this.owner = owner;
+    this.action = action;
+  }
+
+  /**
+   * Get the owning service.
+   * @return the service to receive notification when
+   * the runnable completes.
+   */
+  public Service getOwner() {
+    return owner;
+  }
+
+  /**
+   * Any exception raised by inner <code>action's</code> run.
+   * @return an exception or null.
+   */
+  public Exception getException() {
+    return exception;
+  }
+
+  @Override
+  public void run() {
+    try {
+      action.run();
+    } catch (Exception e) {
+      exception = e;
+    }
+    owner.stop();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceThreadFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceThreadFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceThreadFactory.java
new file mode 100644
index 0000000..737197b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/ServiceThreadFactory.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+import com.google.common.base.Preconditions;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * A thread factory that creates threads (possibly daemon threads)
+ * using the name and naming policy supplied.
+ * The thread counter starts at 1, increments atomically, 
+ * and is supplied as the second argument in the format string.
+ * 
+ * A static method, {@link #singleThreadExecutor(String, boolean)},
+ * exists to simplify the construction of an executor with a single well-named
+ * threads. 
+ * 
+ * Example
+ * <pre>
+ *  ExecutorService exec = ServiceThreadFactory.newSingleThreadExecutor("live", true)
+ * </pre>
+ */
+public class ServiceThreadFactory implements ThreadFactory {
+
+  private static final AtomicInteger counter = new AtomicInteger(1);
+
+  /**
+   * Default format for thread names: {@value}.
+   */
+  public static final String DEFAULT_NAMING_FORMAT = "%s-%03d";
+  private final String name;
+  private final boolean daemons;
+  private final String namingFormat;
+
+  /**
+   * Create an instance
+   * @param name base thread name
+   * @param daemons flag to indicate the threads should be marked as daemons
+   * @param namingFormat format string to generate thread names from
+   */
+  public ServiceThreadFactory(String name,
+      boolean daemons,
+      String namingFormat) {
+    Preconditions.checkArgument(name != null, "null name");
+    Preconditions.checkArgument(namingFormat != null, "null naming format");
+    this.name = name;
+    this.daemons = daemons;
+    this.namingFormat = namingFormat;
+  }
+
+  /**
+   * Create an instance with the default naming format.
+   * @param name base thread name
+   * @param daemons flag to indicate the threads should be marked as daemons
+   */
+  public ServiceThreadFactory(String name,
+      boolean daemons) {
+    this(name, daemons, DEFAULT_NAMING_FORMAT);
+  }
+
+  @Override
+  public Thread newThread(Runnable r) {
+    Preconditions.checkArgument(r != null, "null runnable");
+    String threadName =
+        String.format(namingFormat, name, counter.getAndIncrement());
+    Thread thread = new Thread(r, threadName);
+    thread.setDaemon(daemons);
+    return thread;
+  }
+
+  /**
+   * Create a single thread executor using this naming policy.
+   * @param name base thread name
+   * @param daemons flag to indicate the threads should be marked as daemons
+   * @return an executor
+   */
+  public static ExecutorService singleThreadExecutor(String name,
+      boolean daemons) {
+    return Executors.newSingleThreadExecutor(
+        new ServiceThreadFactory(name, daemons));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowCallbackService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowCallbackService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowCallbackService.java
new file mode 100644
index 0000000..65d14b7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowCallbackService.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A service that calls the supplied callback when it is started -after the 
+ * given delay.
+ *
+ * It can be configured to stop itself after the callback has
+ * completed, marking any exception raised as the exception of this service.
+ * The notifications come in on a callback thread -a thread that is only
+ * started in this service's <code>start()</code> operation.
+ */
+public class WorkflowCallbackService<V> extends
+    WorkflowScheduledExecutorService<ScheduledExecutorService> {
+  protected static final Logger LOG =
+      LoggerFactory.getLogger(WorkflowCallbackService.class);
+
+  /**
+   * This is the callback.
+   */
+  private final Callable<V> callback;
+  private final int delay;
+  private final ServiceTerminatingCallable<V> command;
+
+  private ScheduledFuture<V> scheduledFuture;
+
+  /**
+   * Create an instance of the service
+   * @param name service name
+   * @param callback callback to invoke
+   * @param delay delay -or 0 for no delay
+   * @param terminate terminate this service after the callback?
+   */
+  public WorkflowCallbackService(String name,
+      Callable<V> callback,
+      int delay,
+      boolean terminate) {
+    super(name);
+    Preconditions.checkNotNull(callback, "Null callback argument");
+    this.callback = callback;
+    this.delay = delay;
+    command = new ServiceTerminatingCallable<V>(
+        terminate ? this : null,
+        callback);
+  }
+
+  public ScheduledFuture<V> getScheduledFuture() {
+    return scheduledFuture;
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    LOG.debug("Notifying {} after a delay of {} millis", callback, delay);
+    ScheduledExecutorService executorService =
+        Executors.newSingleThreadScheduledExecutor(
+            new ServiceThreadFactory(getName(), true));
+    setExecutor(executorService);
+    scheduledFuture =
+        executorService.schedule(command, delay, TimeUnit.MILLISECONDS);
+  }
+
+  /**
+   * Stop the service.
+   * If there is any exception noted from any executed notification,
+   * note the exception in this class
+   * @throws Exception exception.
+   */
+  @Override
+  protected void serviceStop() throws Exception {
+    super.serviceStop();
+    // propagate any failure
+    if (getCallbackException() != null) {
+      throw getCallbackException();
+    }
+  }
+
+  /**
+   * Get the exception raised by a callback. Will always be null if the 
+   * callback has not been executed; will only be non-null after any success.
+   * @return a callback
+   */
+  public Exception getCallbackException() {
+    return command.getException();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowCompositeService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowCompositeService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowCompositeService.java
new file mode 100644
index 0000000..9c653f3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowCompositeService.java
@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.service.ServiceStateChangeListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+/**
+ * An extended composite service which stops itself if any child service
+ * fails, or when all its children have successfully stopped without failure.
+ *
+ * Lifecycle
+ * <ol>
+ *   <li>If any child exits with a failure: this service stops, propagating
+ *   the exception.</li>
+ *   <li>When all child services has stopped, this service stops itself</li>
+ * </ol>
+ *
+ */
+public class WorkflowCompositeService extends CompositeService
+    implements ServiceParent, ServiceStateChangeListener {
+
+  private static final Logger LOG =
+    LoggerFactory.getLogger(WorkflowCompositeService.class);
+
+  /**
+   * Deadlock-avoiding overridden config for slider services; see SLIDER-1052
+   */
+  private volatile Configuration configuration;
+
+  /**
+   * Construct an instance
+   * @param name name of this service instance
+   */
+  public WorkflowCompositeService(String name) {
+    super(name);
+  }
+
+  @Override
+  public Configuration getConfig() {
+    return configuration;
+  }
+
+  @Override
+  protected void setConfig(Configuration conf) {
+    super.setConfig(conf);
+    configuration = conf;
+  }
+
+  /**
+   * Construct an instance with the default name.
+   */
+  public WorkflowCompositeService() {
+    this("WorkflowCompositeService");
+  }
+
+  /**
+   * Varargs constructor
+   * @param name name of this service instance
+   * @param children children
+   */
+  public WorkflowCompositeService(String name, Service... children) {
+    this(name);
+    for (Service child : children) {
+      addService(child);
+    }
+  }
+
+  /**
+   * Construct with a list of children
+   * @param name name of this service instance
+   * @param children children to add
+   */
+  public WorkflowCompositeService(String name, List<Service> children) {
+    this(name);
+    for (Service child : children) {
+      addService(child);
+    }
+  }
+
+  /**
+   * Add a service, and register it
+   * @param service the {@link Service} to be added.
+   * Important: do not add a service to a parent during your own serviceInit/start,
+   * in Hadoop 2.2; you will trigger a ConcurrentModificationException.
+   */
+  @Override
+  public synchronized void addService(Service service) {
+    Preconditions.checkArgument(service != null, "null service argument");
+    service.registerServiceListener(this);
+    super.addService(service);
+  }
+
+  /**
+   * When this service is started, any service stopping with a failure
+   * exception is converted immediately into a failure of this service, 
+   * storing the failure and stopping ourselves.
+   * @param child the service that has changed.
+   */
+  @Override
+  public void stateChanged(Service child) {
+    //if that child stopped while we are running:
+    if (isInState(STATE.STARTED) && child.isInState(STATE.STOPPED)) {
+      // a child service has stopped
+      //did the child fail? if so: propagate
+      Throwable failureCause = child.getFailureCause();
+      if (failureCause != null) {
+        LOG.info("Child service " + child + " failed", failureCause);
+        //failure. Convert to an exception
+        Exception e = (failureCause instanceof Exception) ?
+            (Exception) failureCause : new Exception(failureCause);
+        //flip ourselves into the failed state
+        noteFailure(e);
+        stop();
+      } else {
+        LOG.info("Child service completed {}", child);
+        if (areAllChildrenStopped()) {
+          LOG.info("All children are halted: stopping");
+          stop();
+        }
+      }
+    }
+  }
+
+  /**
+   * Probe to query if all children are stopped -simply
+   * by taking a snapshot of the child service list and enumerating
+   * their state. 
+   * The state of the children may change during this operation -that will
+   * not get picked up.
+   * @return true if all the children are stopped.
+   */
+  private boolean areAllChildrenStopped() {
+    List<Service> children = getServices();
+    boolean stopped = true;
+    for (Service child : children) {
+      if (!child.isInState(STATE.STOPPED)) {
+        stopped = false;
+        break;
+      }
+    }
+    return stopped;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowExecutorService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowExecutorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowExecutorService.java
new file mode 100644
index 0000000..7409d32
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowExecutorService.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.service.AbstractService;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+/**
+ * A service that hosts an executor -when the service is stopped,
+ * {@link ExecutorService#shutdownNow()} is invoked.
+ */
+public class WorkflowExecutorService<E extends ExecutorService> extends AbstractService {
+
+  private E executor;
+
+  /**
+   * Construct an instance with the given name -but
+   * no executor
+   * @param name service name
+   */
+  public WorkflowExecutorService(String name) {
+    this(name, null);
+  }
+
+  /**
+   * Construct an instance with the given name and executor
+   * @param name service name
+   * @param executor exectuor
+   */
+  public WorkflowExecutorService(String name,
+      E executor) {
+    super(name);
+    this.executor = executor;
+  }
+
+  /**
+   * Get the executor
+   * @return the executor
+   */
+  public synchronized E getExecutor() {
+    return executor;
+  }
+
+  /**
+   * Set the executor. Only valid if the current one is null
+   * @param executor executor
+   */
+  public synchronized void setExecutor(E executor) {
+    Preconditions.checkState(this.executor == null,
+        "Executor already set");
+    this.executor = executor;
+  }
+
+  /**
+   * Execute the runnable with the executor (which 
+   * must have been created already)
+   * @param runnable runnable to execute
+   */
+  public void execute(Runnable runnable) {
+    getExecutor().execute(runnable);
+  }
+
+  /**
+   * Submit a callable
+   * @param callable callable
+   * @param <V> type of the final get
+   * @return a future to wait on
+   */
+  public <V> Future<V> submit(Callable<V> callable) {
+    return getExecutor().submit(callable);
+  }
+
+  /**
+   * Stop the service: halt the executor. 
+   * @throws Exception exception.
+   */
+  @Override
+  protected void serviceStop() throws Exception {
+    stopExecutor();
+    super.serviceStop();
+  }
+
+  /**
+   * Stop the executor if it is not null.
+   * This uses {@link ExecutorService#shutdownNow()}
+   * and so does not block until they have completed.
+   */
+  protected synchronized void stopExecutor() {
+    if (executor != null) {
+      executor.shutdownNow();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowRpcService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowRpcService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowRpcService.java
new file mode 100644
index 0000000..b71530f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowRpcService.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.service.AbstractService;
+
+import java.net.InetSocketAddress;
+
+/**
+ * A YARN service that maps the start/stop lifecycle of an RPC server
+ * to the YARN service lifecycle. 
+ */
+public class WorkflowRpcService extends AbstractService {
+
+  /** RPC server*/
+  private final Server server;
+
+  /**
+   * Construct an instance
+   * @param name service name
+   * @param server service to stop
+   */
+  public WorkflowRpcService(String name, Server server) {
+    super(name);
+    Preconditions.checkArgument(server != null, "Null server");
+    this.server = server;
+  }
+
+  /**
+   * Get the server
+   * @return the server
+   */
+  public Server getServer() {
+    return server;
+  }
+
+  /**
+   * Get the socket address of this server
+   * @return the address this server is listening on
+   */
+  public InetSocketAddress getConnectAddress() {
+    return NetUtils.getConnectAddress(server);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    super.serviceStart();
+    server.start();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    if (server != null) {
+      server.stop();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowScheduledExecutorService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowScheduledExecutorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowScheduledExecutorService.java
new file mode 100644
index 0000000..e9f53ed
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowScheduledExecutorService.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+import java.util.concurrent.ScheduledExecutorService;
+
+/**
+ * Scheduled executor or subclass thereof
+ * @param <E> scheduled executor service type
+ */
+public class WorkflowScheduledExecutorService<E extends ScheduledExecutorService>
+    extends WorkflowExecutorService<E> {
+
+  public WorkflowScheduledExecutorService(String name) {
+    super(name);
+  }
+
+  public WorkflowScheduledExecutorService(String name,
+      E executor) {
+    super(name, executor);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowSequenceService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowSequenceService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowSequenceService.java
new file mode 100644
index 0000000..97f97e8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/WorkflowSequenceService.java
@@ -0,0 +1,306 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.service.ServiceStateChangeListener;
+import org.apache.hadoop.service.ServiceStateException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * This resembles the YARN CompositeService, except that it
+ * starts one service after another
+ * 
+ * Workflow
+ * <ol>
+ *   <li>When the <code>WorkflowSequenceService</code> instance is
+ *   initialized, it only initializes itself.</li>
+ *   
+ *   <li>When the <code>WorkflowSequenceService</code> instance is
+ *   started, it initializes then starts the first of its children.
+ *   If there are no children, it immediately stops.</li>
+ *   
+ *   <li>When the active child stops, it did not fail, and the parent has not
+ *   stopped -then the next service is initialized and started. If there is no
+ *   remaining child the parent service stops.</li>
+ *   
+ *   <li>If the active child did fail, the parent service notes the exception
+ *   and stops -effectively propagating up the failure.
+ *   </li>
+ * </ol>
+ * 
+ * New service instances MAY be added to a running instance -but no guarantees
+ * can be made as to whether or not they will be run.
+ */
+
+public class WorkflowSequenceService extends AbstractService implements
+    ServiceParent, ServiceStateChangeListener {
+
+  private static final Logger LOG =
+    LoggerFactory.getLogger(WorkflowSequenceService.class);
+
+  /**
+   * list of services
+   */
+  private final List<Service> serviceList = new ArrayList<>();
+
+  /**
+   * The currently active service.
+   * Volatile -may change & so should be read into a 
+   * local variable before working with
+   */
+  private volatile Service activeService;
+
+  /**
+  the previous service -the last one that finished. 
+  null if one did not finish yet
+   */
+  private volatile Service previousService;
+  
+  private boolean stopIfNoChildServicesAtStartup = true;
+
+  /**
+   * Construct an instance
+   * @param name service name
+   */
+  public WorkflowSequenceService(String name) {
+    super(name);
+  }
+
+  /**
+   * Construct an instance with the default name
+   */
+  public WorkflowSequenceService() {
+    this("WorkflowSequenceService");
+  }
+
+  /**
+   * Create a service sequence with the given list of services
+   * @param name service name
+   * @param children initial sequence
+   */
+  public WorkflowSequenceService(String name, Service... children) {
+    super(name);
+    for (Service service : children) {
+      addService(service);
+    }
+  }  /**
+   * Create a service sequence with the given list of services
+   * @param name service name
+   * @param children initial sequence
+   */
+  public WorkflowSequenceService(String name, List<Service> children) {
+    super(name);
+    for (Service service : children) {
+      addService(service);
+    }
+  }
+
+  /**
+   * Get the current service -which may be null
+   * @return service running
+   */
+  public Service getActiveService() {
+    return activeService;
+  }
+
+  /**
+   * Get the previously active service
+   * @return the service last run, or null if there is none.
+   */
+  public Service getPreviousService() {
+    return previousService;
+  }
+
+  protected void setStopIfNoChildServicesAtStartup(boolean stopIfNoChildServicesAtStartup) {
+    this.stopIfNoChildServicesAtStartup = stopIfNoChildServicesAtStartup;
+  }
+
+  /**
+   * When started
+   * @throws Exception
+   */
+  @Override
+  protected void serviceStart() throws Exception {
+    if (!startNextService() && stopIfNoChildServicesAtStartup) {
+        //nothing to start -so stop
+        stop();
+    }
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    //stop current service.
+    //this triggers a callback that is caught and ignored
+    Service current = activeService;
+    previousService = current;
+    activeService = null;
+    if (current != null) {
+      current.stop();
+    }
+  }
+
+  /**
+   * Start the next service in the list.
+   * Return false if there are no more services to run, or this
+   * service has stopped
+   * @return true if a service was started
+   * @throws RuntimeException from any init or start failure
+   * @throws ServiceStateException if this call is made before
+   * the service is started
+   */
+  public synchronized boolean startNextService() {
+    if (isInState(STATE.STOPPED)) {
+      //downgrade to a failed
+      LOG.debug("Not starting next service -{} is stopped", this);
+      return false;
+    }
+    if (!isInState(STATE.STARTED)) {
+      //reject attempts to start a service too early
+      throw new ServiceStateException(
+        "Cannot start a child service when not started");
+    }
+    if (serviceList.isEmpty()) {
+      //nothing left to run
+      return false;
+    }
+    if (activeService != null && activeService.getFailureCause() != null) {
+      //did the last service fail? Is this caused by some premature callback?
+      LOG.debug("Not starting next service due to a failure of {}",
+          activeService);
+      return false;
+    }
+    //bear in mind that init & start can fail, which
+    //can trigger re-entrant calls into the state change listener.
+    //by setting the current service to null
+    //the start-next-service logic is skipped.
+    //now, what does that mean w.r.t exit states?
+
+    activeService = null;
+    Service head = serviceList.remove(0);
+
+    try {
+      head.init(getConfig());
+      head.registerServiceListener(this);
+      head.start();
+    } catch (RuntimeException e) {
+      noteFailure(e);
+      throw e;
+    }
+    //at this point the service must have explicitly started & not failed,
+    //else an exception would have been raised
+    activeService = head;
+    return true;
+  }
+
+  /**
+   * State change event relays service stop events to
+   * {@link #onServiceCompleted(Service)}. Subclasses can
+   * extend that with extra logic
+   * @param service the service that has changed.
+   */
+  @Override
+  public void stateChanged(Service service) {
+    // only react to the state change when it is the current service
+    // and it has entered the STOPPED state
+    if (service == activeService && service.isInState(STATE.STOPPED)) {
+      onServiceCompleted(service);
+    }
+  }
+
+  /**
+   * handler for service completion: base class starts the next service
+   * @param service service that has completed
+   */
+  protected synchronized void onServiceCompleted(Service service) {
+    LOG.info("Running service stopped: {}", service);
+    previousService = activeService;
+    
+
+    //start the next service if we are not stopped ourselves
+    if (isInState(STATE.STARTED)) {
+
+      //did the service fail? if so: propagate
+      Throwable failureCause = service.getFailureCause();
+      if (failureCause != null) {
+        Exception e = (failureCause instanceof Exception) ?
+                      (Exception) failureCause : new Exception(failureCause);
+        noteFailure(e);
+        stop();
+      }
+      
+      //start the next service
+      boolean started;
+      try {
+        started = startNextService();
+      } catch (Exception e) {
+        //something went wrong here
+        noteFailure(e);
+        started = false;
+      }
+      if (!started) {
+        //no start because list is empty
+        //stop and expect the notification to go upstream
+        stop();
+      }
+    } else {
+      //not started, so just note that the current service
+      //has gone away
+      activeService = null;
+    }
+  }
+
+  /**
+   * Add the passed {@link Service} to the list of services managed by this
+   * {@link WorkflowSequenceService}
+   * @param service the {@link Service} to be added
+   */
+  @Override
+  public synchronized void addService(Service service) {
+    Preconditions.checkArgument(service != null, "null service argument");
+    LOG.debug("Adding service {} ", service.getName());
+    synchronized (serviceList) {
+      serviceList.add(service);
+    }
+  }
+
+  /**
+   * Get an unmodifiable list of services
+   * @return a list of child services at the time of invocation -
+   * added services will not be picked up.
+   */
+  @Override //Parent
+  public synchronized List<Service> getServices() {
+    return Collections.unmodifiableList(serviceList);
+  }
+
+  @Override // Object
+  public synchronized String toString() {
+    return super.toString() + "; current service " + activeService
+           + "; queued service count=" + serviceList.size();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/package-info.java
new file mode 100644
index 0000000..36d059a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/workflow/package-info.java
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.workflow;
+
+/**
+
+<p>
+ This package contains classes which can be aggregated to build up
+ complex workflows of services: sequences of operations, callbacks
+ and composite services with a shared lifespan.
+ </p>
+
+<h2>
+ Core concepts:
+</h2>
+
+
+<p>
+The Workflow Services are set of Hadoop YARN services, all implementing
+the {@link org.apache.hadoop.service.Service} API.
+They are designed to be aggregated, to be composed to produce larger
+composite services which than perform ordered operations, notify other services
+when work has completed, and to propagate failure up the service hierarchy.
+</p>
+<p>
+Service instances may a limited lifespan, and may self-terminate when
+they consider it appropriate.</p>
+<p>
+Workflow Services that have children implement the
+{@link org.apache.slider.server.services.workflow.ServiceParent}
+class, which provides (thread-safe) access to the children -allowing new children
+to be added, and existing children to be ennumerated. The implement policies
+on how to react to the termination of children -so can sequence operations
+which terminate themselves when complete.
+</p>
+
+<p>
+Workflow Services may be subclassed to extend their behavior, or to use them
+in specific applications. Just as the standard
+{@link org.apache.hadoop.service.CompositeService}
+is often subclassed to aggregate child services, the
+{@link org.apache.slider.server.services.workflow.WorkflowCompositeService}
+can be used instead -adding the feature that failing services trigger automatic
+parent shutdown. If that is the desired operational mode of a class,
+swapping the composite service implementation may be sufficient to adopt it.
+</p>
+
+
+<h2> How do the workflow services differ from the standard YARN services? </h2>
+
+ <p>
+ 
+ There is exactly one standard YARN service for managing children, the
+ {@link org.apache.hadoop.service.CompositeService}.
+ </p><p>
+ The {@link org.apache.slider.server.services.workflow.WorkflowCompositeService}
+ shares the same model of "child services, all inited and started together".
+ Where it differs is that if any child service stops -either due to a failure
+ or to an action which invokes that service's
+ {@link org.apache.hadoop.service.Service#stop()} method.
+ </p>
+ <p>
+
+In contrast, the original <code>CompositeService</code> class starts its children
+in its{@link org.apache.hadoop.service.Service#start()}  method, but does not
+listen or react to any child service halting. As a result, changes in child 
+state are not automatically detected or propagated, other than failures in
+the actual init() and start() methods.
+</p>
+
+<p>
+If a child service runs until completed -that is it will not be stopped until
+instructed to do so, and if it is only the parent service that attempts to
+stop the child, then this difference is unimportant. 
+</p>
+<p>
+
+However, if any service that depends upon all it child services running -
+and if those child services are written so as to stop when they fail, using
+the <code>WorkflowCompositeService</code> as a base class will enable the 
+parent service to be automatically notified of a child stopping.
+
+</p>
+<p>
+The {@link org.apache.slider.server.services.workflow.WorkflowSequenceService}
+resembles the composite service in API, but its workflow is different. It
+initializes and starts its children one-by-one, only starting the second after
+the first one succeeds, the third after the second, etc. If any service in
+the sequence fails, the parent <code>WorkflowSequenceService</code> stops, 
+reporting the same exception. 
+</p>
+
+<p>
+The {@link org.apache.slider.server.services.workflow.ForkedProcessService}:
+Executes a process when started, and binds to the life of that process. When the
+process terminates, so does the service -and vice versa. This service enables
+external processes to be executed as part of a sequence of operations -or,
+using the {@link org.apache.slider.server.services.workflow.WorkflowCompositeService}
+in parallel with other services, terminating the process when the other services
+stop -and vice versa.
+</p>
+
+<p>
+The {@link org.apache.slider.server.services.workflow.WorkflowCallbackService}
+executes a {@link java.util.concurrent.Callable} callback a specified delay
+after the service is started, then potentially terminates itself.
+This is useful for callbacks when a workflow  reaches a specific point
+-or simply for executing arbitrary code in the workflow.
+
+ </p>
+
+
+<h2>
+Other Workflow Services
+</h2>
+
+There are some minor services that have proven useful within aggregate workflows,
+and simply in applications which are built from composite YARN services.
+
+ <ul>
+ <li>{@link org.apache.slider.server.services.workflow.WorkflowRpcService }:
+ Maintains a reference to an RPC {@link org.apache.hadoop.ipc.Server} instance.
+ When the service is started, so is the RPC server. Similarly, when the service
+ is stopped, so is the RPC server instance. 
+ </li>
+
+ <li>{@link org.apache.slider.server.services.workflow.ClosingService}: Closes
+ an instance of {@link java.io.Closeable} when the service is stopped. This
+ is purely a housekeeping class.
+ </li>
+
+ </ul>
+
+ Lower-level classes 
+ <ul>
+ <li>{@link org.apache.slider.server.services.workflow.ServiceTerminatingRunnable }:
+ A {@link java.lang.Runnable} which runs the runnable supplied in its constructor
+ then signals its owning service to stop once that runnable is completed. 
+ Any exception raised in the run is stored.
+ </li>
+ <li>{@link org.apache.slider.server.services.workflow.WorkflowExecutorService}:
+ A base class for services that wish to have a {@link java.util.concurrent.ExecutorService}
+ with a lifespan mapped to that of a service. When the service is stopped, the
+ {@link java.util.concurrent.ExecutorService#shutdownNow()} method is called to
+ attempt to shut down all running tasks.
+ </li>
+ <li>{@link org.apache.slider.server.services.workflow.ServiceThreadFactory}:
+ This is a simple {@link java.util.concurrent.ThreadFactory} which generates
+ meaningful thread names. It can be used as a parameter to constructors of 
+ {@link java.util.concurrent.ExecutorService} instances, to ensure that
+ log information can tie back text to the related services</li>
+ </ul>
+
+
+
+ */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/yarnregistry/YarnRegistryViewForProviders.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/yarnregistry/YarnRegistryViewForProviders.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/yarnregistry/YarnRegistryViewForProviders.java
new file mode 100644
index 0000000..254bf27
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/services/yarnregistry/YarnRegistryViewForProviders.java
@@ -0,0 +1,270 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.yarnregistry;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.registry.client.api.RegistryConstants;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.registry.client.api.BindFlags;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.registry.client.binding.RegistryUtils;
+import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
+
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.slider.common.tools.SliderUtils;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.registry.client.binding.RegistryPathUtils.join;
+
+/**
+ * Registry view for providers. This tracks where the service
+ * is registered, offers access to the record and other things.
+ */
+public class YarnRegistryViewForProviders {
+
+  private final RegistryOperations registryOperations;
+
+  private final String user;
+
+  private final String sliderServiceClass;
+  private final String instanceName;
+  private final ApplicationAttemptId applicationAttemptId;
+  /**
+   * Record used where the service registered itself.
+   * Null until the service is registered
+   */
+  private ServiceRecord selfRegistration;
+
+  /**
+   * Path where record was registered
+   * Null until the service is registered
+   */
+  private String selfRegistrationPath;
+
+  public YarnRegistryViewForProviders(RegistryOperations registryOperations,
+      String user,
+      String sliderServiceClass,
+      String instanceName,
+      ApplicationAttemptId applicationAttemptId) {
+    Preconditions.checkArgument(registryOperations != null,
+        "null registry operations");
+    Preconditions.checkArgument(user != null, "null user");
+    Preconditions.checkArgument(SliderUtils.isSet(sliderServiceClass),
+        "unset service class");
+    Preconditions.checkArgument(SliderUtils.isSet(instanceName),
+        "instanceName");
+    Preconditions.checkArgument(applicationAttemptId != null,
+        "null applicationAttemptId");
+    this.registryOperations = registryOperations;
+    this.user = user;
+    this.sliderServiceClass = sliderServiceClass;
+    this.instanceName = instanceName;
+    this.applicationAttemptId = applicationAttemptId;
+  }
+
+  public ApplicationAttemptId getApplicationAttemptId() {
+    return applicationAttemptId;
+  }
+
+  public String getUser() {
+    return user;
+  }
+
+  public String getSliderServiceClass() {
+    return sliderServiceClass;
+  }
+
+  public String getInstanceName() {
+    return instanceName;
+  }
+
+  public RegistryOperations getRegistryOperations() {
+    return registryOperations;
+  }
+
+  public ServiceRecord getSelfRegistration() {
+    return selfRegistration;
+  }
+
+  private void setSelfRegistration(ServiceRecord selfRegistration) {
+    this.selfRegistration = selfRegistration;
+  }
+
+  /**
+   * Get the path to where the service has registered itself.
+   * Null until the service is registered
+   * @return the service registration path.
+   */
+  public String getSelfRegistrationPath() {
+    return selfRegistrationPath;
+  }
+
+  /**
+   * Get the absolute path to where the service has registered itself.
+   * This includes the base registry path
+   * Null until the service is registered
+   * @return the service registration path.
+   */
+  public String getAbsoluteSelfRegistrationPath() {
+    if (selfRegistrationPath == null) {
+      return null;
+    }
+    String root = registryOperations.getConfig().getTrimmed(
+        RegistryConstants.KEY_REGISTRY_ZK_ROOT,
+        RegistryConstants.DEFAULT_ZK_REGISTRY_ROOT);
+    return RegistryPathUtils.join(root, selfRegistrationPath);
+  }
+
+  /**
+   * Add a component under the slider name/entry
+   * @param componentName component name
+   * @param record record to put
+   * @throws IOException
+   */
+  public void putComponent(String componentName,
+      ServiceRecord record) throws
+      IOException {
+    putComponent(sliderServiceClass, instanceName,
+        componentName,
+        record);
+  }
+
+  /**
+   * Add a component 
+   * @param serviceClass service class to use under ~user
+   * @param componentName component name
+   * @param record record to put
+   * @throws IOException
+   */
+  public void putComponent(String serviceClass,
+      String serviceName,
+      String componentName,
+      ServiceRecord record) throws IOException {
+    String path = RegistryUtils.componentPath(
+        user, serviceClass, serviceName, componentName);
+    registryOperations.mknode(RegistryPathUtils.parentOf(path), true);
+    registryOperations.bind(path, record, BindFlags.OVERWRITE);
+  }
+    
+  /**
+   * Add a service under a path, optionally purging any history
+   * @param username user
+   * @param serviceClass service class to use under ~user
+   * @param serviceName name of the service
+   * @param record service record
+   * @param deleteTreeFirst perform recursive delete of the path first.
+   * @return the path the service was created at
+   * @throws IOException
+   */
+  public String putService(String username,
+      String serviceClass,
+      String serviceName,
+      ServiceRecord record,
+      boolean deleteTreeFirst) throws IOException {
+    String path = RegistryUtils.servicePath(
+        username, serviceClass, serviceName);
+    if (deleteTreeFirst) {
+      registryOperations.delete(path, true);
+    }
+    registryOperations.mknode(RegistryPathUtils.parentOf(path), true);
+    registryOperations.bind(path, record, BindFlags.OVERWRITE);
+    return path;
+  }
+
+  /**
+   * Add a service under a path for the current user
+   * @param serviceClass service class to use under ~user
+   * @param serviceName name of the service
+   * @param record service record
+   * @param deleteTreeFirst perform recursive delete of the path first
+   * @return the path the service was created at
+   * @throws IOException
+   */
+  public String putService(
+      String serviceClass,
+      String serviceName,
+      ServiceRecord record,
+      boolean deleteTreeFirst) throws IOException {
+    return putService(user, serviceClass, serviceName, record, deleteTreeFirst);
+  }
+
+
+  /**
+   * Add a service under a path for the current user
+   * @param serviceClass service class to use under ~user
+   * @param serviceName name of the service
+   * @param record service record
+   * @param deleteTreeFirst perform recursive delete of the path first
+   * @return the path the service was created at
+   * @throws IOException
+   */
+  public String registerSelf(
+      ServiceRecord record,
+      boolean deleteTreeFirst) throws IOException {
+    selfRegistrationPath =
+        putService(user, sliderServiceClass, instanceName, record, deleteTreeFirst);
+    setSelfRegistration(record);
+    return selfRegistrationPath;
+  }
+
+  /**
+   * Update the self record by pushing out the latest version of the service
+   * registration record. 
+   * @throws IOException any failure.
+   */
+  public void updateSelf() throws IOException {
+    putService(user, sliderServiceClass, instanceName, selfRegistration, false);
+  }
+    
+  /**
+   * Delete a component
+   * @param componentName component name
+   * @throws IOException
+   */
+  public void deleteComponent(String componentName) throws IOException {
+    String path = RegistryUtils.componentPath(
+        user, sliderServiceClass, instanceName,
+        componentName);
+    registryOperations.delete(path, false);
+  }
+
+  /**
+   * Delete the children of a path -but not the path itself.
+   * It is not an error if the path does not exist
+   * @param path path to delete
+   * @param recursive flag to request recursive deletes
+   * @throws IOException IO problems
+   */
+  public void deleteChildren(String path, boolean recursive) throws IOException {
+    List<String> childNames = null;
+    try {
+      childNames = registryOperations.list(path);
+    } catch (PathNotFoundException e) {
+      return;
+    }
+    for (String childName : childNames) {
+      String child = join(path, childName);
+      registryOperations.delete(child, recursive);
+    }
+  }
+  
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[10/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppStateBindingInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppStateBindingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppStateBindingInfo.java
new file mode 100644
index 0000000..a8aa1a2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppStateBindingInfo.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.slider.core.conf.AggregateConf;
+import org.apache.slider.providers.ProviderRole;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Binding information for application states; designed to be extensible
+ * so that tests don't have to be massivley reworked when new arguments
+ * are added.
+ */
+public class AppStateBindingInfo {
+  public AggregateConf instanceDefinition;
+  public Configuration serviceConfig = new Configuration();
+  public Configuration publishedProviderConf = new Configuration(false);
+  public List<ProviderRole> roles = new ArrayList<>();
+  public FileSystem fs;
+  public Path historyPath;
+  public List<Container> liveContainers = new ArrayList<>(0);
+  public Map<String, String> applicationInfo = new HashMap<>();
+  public ContainerReleaseSelector releaseSelector = new SimpleReleaseSelector();
+  /** node reports off the RM. */
+  public List<NodeReport> nodeReports = new ArrayList<>(0);
+
+  public void validate() throws IllegalArgumentException {
+    Preconditions.checkArgument(instanceDefinition != null, "null instanceDefinition");
+    Preconditions.checkArgument(serviceConfig != null, "null appmasterConfig");
+    Preconditions.checkArgument(publishedProviderConf != null, "null publishedProviderConf");
+    Preconditions.checkArgument(releaseSelector != null, "null releaseSelector");
+    Preconditions.checkArgument(roles != null, "null providerRoles");
+    Preconditions.checkArgument(fs != null, "null fs");
+    Preconditions.checkArgument(historyPath != null, "null historyDir");
+    Preconditions.checkArgument(nodeReports != null, "null nodeReports");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocationOutcome.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocationOutcome.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocationOutcome.java
new file mode 100644
index 0000000..5b3a93c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocationOutcome.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+/**
+ * Outcome of the assignment
+ */
+public enum ContainerAllocationOutcome {
+  /**
+   * There wasn't a request for this
+   */
+  Unallocated,
+
+  /**
+   * Open placement
+   */
+  Open,
+
+  /**
+   * Allocated explicitly  where requested
+   */
+  Placed,
+
+  /**
+   * This was an escalated placement
+   */
+  Escalated
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocationResults.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocationResults.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocationResults.java
new file mode 100644
index 0000000..e80639e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAllocationResults.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+import org.apache.slider.server.appmaster.operations.AbstractRMOperation;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This is just a tuple of the outcome of a container allocation
+ */
+public class ContainerAllocationResults {
+
+  /**
+   * What was the outcome of this allocation: placed, escalated, ...
+   */
+  public ContainerAllocationOutcome outcome;
+
+  /**
+   * The outstanding request which originated this.
+   * This will be null if the outcome is {@link ContainerAllocationOutcome#Unallocated}
+   * as it wasn't expected.
+   */
+  public OutstandingRequest origin;
+
+  /**
+   * A possibly empty list of requests to add to the follow-up actions
+   */
+  public List<AbstractRMOperation> operations = new ArrayList<>(0);
+
+  public ContainerAllocationResults() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAssignment.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAssignment.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAssignment.java
new file mode 100644
index 0000000..3e8a3c3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerAssignment.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+import org.apache.hadoop.yarn.api.records.Container;
+
+/**
+ * Static assignment structure
+ */
+public class ContainerAssignment {
+
+  /**
+   * Container that has been allocated
+   */
+  public final Container container;
+
+  /**
+   * Role to assign to it
+   */
+  public final RoleStatus role;
+
+  /**
+   * Placement outcome: was this from history or not
+   */
+  public final ContainerAllocationOutcome placement;
+
+  public ContainerAssignment(Container container,
+      RoleStatus role,
+      ContainerAllocationOutcome placement) {
+    this.container = container;
+    this.role = role;
+    this.placement = placement;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder("ContainerAssignment{");
+    sb.append("container=").append(container);
+    sb.append(", role=").append(role);
+    sb.append(", placement=").append(placement);
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerOutcome.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerOutcome.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerOutcome.java
new file mode 100644
index 0000000..59ab30b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerOutcome.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
+
+/**
+ * Container outcomes we care about; slightly simplified from
+ * {@link ContainerExitStatus} -and hopefully able to handle
+ * any new exit codes.
+ */
+public enum ContainerOutcome {
+  Completed,
+  Failed,
+  Failed_limits_exceeded,
+  Node_failure,
+  Preempted;
+
+  /**
+   * Build a container outcome from an exit status.
+   * The values in {@link ContainerExitStatus} are used
+   * here.
+   * @param exitStatus exit status
+   * @return an enumeration of the outcome.
+   */
+  public static ContainerOutcome fromExitStatus(int exitStatus) {
+    switch (exitStatus) {
+      case ContainerExitStatus.ABORTED:
+      case ContainerExitStatus.KILLED_BY_APPMASTER:
+      case ContainerExitStatus.KILLED_BY_RESOURCEMANAGER:
+      case ContainerExitStatus.KILLED_AFTER_APP_COMPLETION:
+        // could either be a release or node failure. Treat as completion
+        return Completed;
+      case ContainerExitStatus.DISKS_FAILED:
+        return Node_failure;
+      case ContainerExitStatus.PREEMPTED:
+        return Preempted;
+      case ContainerExitStatus.KILLED_EXCEEDED_PMEM:
+      case ContainerExitStatus.KILLED_EXCEEDED_VMEM:
+        return Failed_limits_exceeded;
+      default:
+        return exitStatus == 0 ? Completed : Failed;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerPriority.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerPriority.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerPriority.java
new file mode 100644
index 0000000..df222fa
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerPriority.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.util.Records;
+
+import java.util.Locale;
+
+/**
+ * Class containing the logic to build/split container priorities into the
+ * different fields used by Slider
+ *
+ * The original design here had a requestID merged with the role, to
+ * track outstanding requests. However, this isn't possible, so
+ * the request ID has been dropped. A "location specified" flag was
+ * added to indicate whether or not the request was for a specific location
+ * -though this is currently unused.
+ * 
+ * The methods are effectively surplus -but retained to preserve the
+ * option of changing behavior in future
+ */
+public final class ContainerPriority {
+
+  // bit that represents whether location is specified
+  static final int NOLOCATION = 1 << 30;
+  
+  public static int buildPriority(int role,
+                                  boolean locationSpecified) {
+    int location = locationSpecified ? 0 : NOLOCATION;
+    return role | location;
+  }
+
+
+  public static Priority createPriority(int role,
+                                        boolean locationSpecified) {
+    Priority pri = Records.newRecord(Priority.class);
+    pri.setPriority(ContainerPriority.buildPriority(role,
+                                                    locationSpecified));
+    return pri;
+  }
+
+  public static int extractRole(int priority) {
+    return priority >= NOLOCATION ? priority ^ NOLOCATION : priority;
+  }
+
+  /**
+   * Does the priority have location
+   * @param priority priority index
+   * @return true if the priority has the location marker
+   */
+  public static boolean hasLocation(int priority) {
+    return (priority ^ NOLOCATION ) == 0;
+  }
+  
+  /**
+   * Map from a container to a role key by way of its priority
+   * @param container container
+   * @return role key
+   */
+  public static int extractRole(Container container) {
+    Priority priority = container.getPriority();
+    return extractRole(priority);
+  }
+  
+  /**
+   * Priority record to role mapper
+   * @param priorityRecord priority record
+   * @return the role #
+   */
+  public static int extractRole(Priority priorityRecord) {
+    Preconditions.checkNotNull(priorityRecord);
+    return extractRole(priorityRecord.getPriority());
+  }
+
+  /**
+   * Convert a priority record to a string, extracting role and locality
+   * @param priorityRecord priority record. May be null
+   * @return a string value
+   */
+  public static String toString(Priority priorityRecord) {
+    if (priorityRecord==null) {
+      return "(null)";
+    } else {
+      return String.format(Locale.ENGLISH,
+          "role %d (locality=%b)",
+          extractRole(priorityRecord),
+          hasLocation(priorityRecord.getPriority()));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerReleaseSelector.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerReleaseSelector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerReleaseSelector.java
new file mode 100644
index 0000000..fafbada
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/ContainerReleaseSelector.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+import java.util.List;
+
+/**
+ * Interface implemented by anything that must choose containers to release
+ * 
+ */
+public interface ContainerReleaseSelector {
+
+  /**
+   * Given a list of candidate containers, return a sorted version of the priority
+   * in which they should be released. 
+   * @param candidates candidate list ... everything considered suitable
+   * @return the list of candidates
+   */
+  List<RoleInstance> sortCandidates(int roleId,
+      List<RoleInstance> candidates);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/MostRecentContainerReleaseSelector.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/MostRecentContainerReleaseSelector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/MostRecentContainerReleaseSelector.java
new file mode 100644
index 0000000..38c5b8e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/MostRecentContainerReleaseSelector.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+import org.apache.slider.common.tools.Comparators;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+/**
+ * Sort the candidate list by the most recent container first.
+ */
+public class MostRecentContainerReleaseSelector implements ContainerReleaseSelector {
+
+  @Override
+  public List<RoleInstance> sortCandidates(int roleId,
+      List<RoleInstance> candidates) {
+    Collections.sort(candidates, new newerThan());
+    return candidates;
+  }
+
+  private static class newerThan implements Comparator<RoleInstance>, Serializable {
+    private final Comparator<Long> innerComparator =
+        new Comparators.ComparatorReverser<>(new Comparators.LongComparator());
+    public int compare(RoleInstance o1, RoleInstance o2) {
+      return innerComparator.compare(o1.createTime, o2.createTime);
+
+    }
+    
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeEntry.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeEntry.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeEntry.java
new file mode 100644
index 0000000..eb8ff03
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeEntry.java
@@ -0,0 +1,325 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.slider.api.types.NodeEntryInformation;
+
+/**
+ * Information about the state of a role on a specific node instance.
+ * No fields are synchronized; sync on the instance to work with it
+ * <p>
+ * The two fields `releasing` and `requested` are used to track the ongoing
+ * state of YARN requests; they do not need to be persisted across stop/start
+ * cycles. They may be relevant across AM restart, but without other data
+ * structures in the AM, not enough to track what the AM was up to before
+ * it was restarted. The strategy will be to ignore unexpected allocation
+ * responses (which may come from pre-restart) requests, while treating
+ * unexpected container release responses as failures.
+ * <p>
+ * The `active` counter is only decremented after a container release response
+ * has been received.
+ * <p>
+ *
+ */
+public class NodeEntry implements Cloneable {
+  
+  public final int rolePriority;
+
+  public NodeEntry(int rolePriority) {
+    this.rolePriority = rolePriority;
+  }
+
+  /**
+   * instance explicitly requested on this node: it's OK if an allocation
+   * comes in that has not been (and when that happens, this count should 
+   * not drop).
+   */
+  private int requested;
+
+  /** number of starting instances */
+  private int starting;
+
+  /** incrementing counter of instances that failed to start */
+  private int startFailed;
+
+  /** incrementing counter of instances that failed */
+  private int failed;
+
+  /**
+   * Counter of "failed recently" events. These are all failures
+   * which have happened since it was last reset.
+   */
+  private int failedRecently;
+
+  /** incrementing counter of instances that have been pre-empted. */
+  private int preempted;
+
+  /**
+   * Number of live nodes. 
+   */
+  private int live;
+
+  /** number of containers being released off this node */
+  private int releasing;
+
+  /** timestamp of last use */
+  private long lastUsed;
+
+  /**
+   * Is the node available for assignments? That is, it is
+   * not running any instances of this type, nor are there
+   * any requests oustanding for it.
+   * @return true if a new request could be issued without taking
+   * the number of instances &gt; 1.
+   */
+  public synchronized boolean isAvailable() {
+    return live + requested + starting - releasing <= 0;
+  }
+
+  /**
+   * Are the anti-affinity constraints held. That is, zero or one
+   * node running or starting
+   * @return true if the constraint holds.
+   */
+  public synchronized boolean isAntiAffinityConstraintHeld() {
+    return (live - releasing + starting) <= 1;
+  }
+
+  /**
+   * return no of active instances -those that could be released as they
+   * are live and not already being released
+   * @return a number, possibly 0
+   */
+  public synchronized int getActive() {
+    return (live - releasing);
+  }
+
+  /**
+   * Return true if the node is not busy, and it
+   * has not been used since the absolute time
+   * @param absoluteTime time
+   * @return true if the node could be cleaned up
+   */
+  public synchronized boolean notUsedSince(long absoluteTime) {
+    return isAvailable() && lastUsed < absoluteTime;
+  }
+
+  public synchronized int getLive() {
+    return live;
+  }
+
+  public int getStarting() {
+    return starting;
+  }
+
+  /**
+   * Set the live value directly -used on AM restart
+   * @param v value
+   */
+  public synchronized void setLive(int v) {
+    live = v;
+  }
+  
+  private synchronized void incLive() {
+    ++live;
+  }
+
+  private synchronized void decLive() {
+    live = RoleHistoryUtils.decToFloor(live);
+  }
+  
+  public synchronized void onStarting() {
+    ++starting;
+  }
+
+  private void decStarting() {
+    starting = RoleHistoryUtils.decToFloor(starting);
+  }
+
+  public synchronized void onStartCompleted() {
+    decStarting();
+    incLive();
+  }
+  
+    /**
+   * start failed -decrement the starting flag.
+   * @return true if the node is now available
+   */
+  public synchronized boolean onStartFailed() {
+    decStarting();
+    ++startFailed;
+    return containerCompleted(false, ContainerOutcome.Failed);
+  }
+  
+  /**
+   * no of requests made of this role of this node. If it goes above
+   * 1 there's a problem
+   */
+  public synchronized  int getRequested() {
+    return requested;
+  }
+
+  /**
+   * request a node: 
+   */
+  public synchronized void request() {
+    ++requested;
+  }
+
+  /**
+   * A request made explicitly to this node has completed
+   */
+  public synchronized void requestCompleted() {
+    requested = RoleHistoryUtils.decToFloor(requested);
+  }
+
+  /**
+   * No of instances in release state
+   */
+  public synchronized int getReleasing() {
+    return releasing;
+  }
+
+  /**
+   * Release an instance -which is no longer marked as active
+   */
+  public synchronized void release() {
+    releasing++;
+  }
+
+  /**
+   * completion event, which can be a planned or unplanned
+   * planned: dec our release count
+   * unplanned: dec our live count
+   * @param wasReleased true if this was planned
+   * @param outcome
+   * @return true if this node is now available
+   */
+  public synchronized boolean containerCompleted(boolean wasReleased, ContainerOutcome outcome) {
+    if (wasReleased) {
+      releasing = RoleHistoryUtils.decToFloor(releasing);
+    } else {
+      // for the node, we use the outcome of the faiure to decide
+      // whether this is potentially "node-related"
+      switch(outcome) {
+        // general "any reason" app failure
+        case Failed:
+        // specific node failure
+        case Node_failure:
+
+          ++failed;
+          ++failedRecently;
+          break;
+
+        case Preempted:
+          preempted++;
+          break;
+
+          // failures which are node-independent
+        case Failed_limits_exceeded:
+        case Completed:
+        default:
+          break;
+      }
+    }
+    decLive();
+    return isAvailable();
+  }
+
+  /**
+   * Time last used.
+   */
+  public synchronized long getLastUsed() {
+    return lastUsed;
+  }
+
+  public synchronized void setLastUsed(long lastUsed) {
+    this.lastUsed = lastUsed;
+  }
+
+  public synchronized int getStartFailed() {
+    return startFailed;
+  }
+
+  public synchronized int getFailed() {
+    return failed;
+  }
+
+  public synchronized int getFailedRecently() {
+    return failedRecently;
+  }
+
+  @VisibleForTesting
+  public synchronized void setFailedRecently(int failedRecently) {
+    this.failedRecently = failedRecently;
+  }
+
+  public synchronized int getPreempted() {
+    return preempted;
+  }
+
+
+  /**
+   * Reset the failed recently count.
+   */
+  public synchronized void resetFailedRecently() {
+    failedRecently = 0;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder("NodeEntry{");
+    sb.append("priority=").append(rolePriority);
+    sb.append(", requested=").append(requested);
+    sb.append(", starting=").append(starting);
+    sb.append(", live=").append(live);
+    sb.append(", releasing=").append(releasing);
+    sb.append(", lastUsed=").append(lastUsed);
+    sb.append(", failedRecently=").append(failedRecently);
+    sb.append(", preempted=").append(preempted);
+    sb.append(", startFailed=").append(startFailed);
+    sb.append('}');
+    return sb.toString();
+  }
+
+  /**
+   * Produced a serialized form which can be served up as JSON
+   * @return a summary of the current role status.
+   */
+  public synchronized NodeEntryInformation serialize() {
+    NodeEntryInformation info = new NodeEntryInformation();
+    info.priority = rolePriority;
+    info.requested = requested;
+    info.releasing = releasing;
+    info.starting = starting;
+    info.startFailed = startFailed;
+    info.failed = failed;
+    info.failedRecently = failedRecently;
+    info.preempted = preempted;
+    info.live = live;
+    info.lastUsed = lastUsed;
+    return info;
+  }
+
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    return super.clone();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeInstance.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeInstance.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeInstance.java
new file mode 100644
index 0000000..cc17cf0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeInstance.java
@@ -0,0 +1,409 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.slider.api.types.NodeInformation;
+import org.apache.slider.common.tools.Comparators;
+import org.apache.slider.common.tools.SliderUtils;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+
+/**
+ * A node instance -stores information about a node in the cluster.
+ * <p>
+ * Operations on the array/set of roles are synchronized.
+ */
+public class NodeInstance {
+
+  public final String hostname;
+
+  /**
+   * last state of node. Starts off as {@link NodeState#RUNNING},
+   * on the assumption that it is live.
+   */
+  private NodeState nodeState = NodeState.RUNNING;
+
+  /**
+   * Last node report. If null: none
+   */
+  private NodeReport nodeReport = null;
+
+  /**
+   * time of state update
+   */
+  private long nodeStateUpdateTime = 0;
+
+  /**
+   * Node labels.
+   *
+   * IMPORTANT: we assume that there is one label/node, which is the policy
+   * for Hadoop as of November 2015
+   */
+  private String nodeLabels = "";
+
+  /**
+   * An unordered list of node entries of specific roles. There's nothing
+   * indexed so as to support sparser datastructures.
+   */
+  private final List<NodeEntry> nodeEntries;
+
+  /**
+   * Create an instance and the (empty) array of nodes
+   * @param roles role count -the no. of roles
+   */
+  public NodeInstance(String hostname, int roles) {
+    this.hostname = hostname;
+    nodeEntries = new ArrayList<>(roles);
+  }
+
+  /**
+   * Update the node status.
+   * The return code is true if the node state changed enough to
+   * trigger a re-evaluation of pending requests. That is, either a node
+   * became available when it was previously not, or the label changed
+   * on an available node.
+   *
+   * Transitions of a node from live to dead aren't treated as significant,
+   * nor label changes on a dead node.
+   *
+   * @param report latest node report
+   * @return true if the node state changed enough for a request evaluation.
+   */
+  public synchronized boolean updateNode(NodeReport report) {
+    nodeStateUpdateTime = report.getLastHealthReportTime();
+    nodeReport = report;
+    NodeState oldState = nodeState;
+    boolean oldStateUnusable = oldState.isUnusable();
+    nodeState = report.getNodeState();
+    boolean newUsable = !nodeState.isUnusable();
+    boolean nodeNowAvailable = oldStateUnusable && newUsable;
+    String labels = this.nodeLabels;
+    nodeLabels = SliderUtils.extractNodeLabel(report);
+    return nodeNowAvailable
+        || newUsable && !this.nodeLabels.equals(labels);
+  }
+
+  public String getNodeLabels() {
+    return nodeLabels;
+  }
+
+  /**
+   * Get the entry for a role -if present
+   * @param role role index
+   * @return the entry
+   * null if the role is out of range
+   */
+  public synchronized NodeEntry get(int role) {
+    for (NodeEntry nodeEntry : nodeEntries) {
+      if (nodeEntry.rolePriority == role) {
+        return nodeEntry;
+      }
+    }
+    return null;
+  }
+  
+  /**
+   * Get the entry for a role -if present
+   * @param role role index
+   * @return the entry
+   * @throws ArrayIndexOutOfBoundsException if the role is out of range
+   */
+  public synchronized NodeEntry getOrCreate(int role) {
+    NodeEntry entry = get(role);
+    if (entry == null) {
+      entry = new NodeEntry(role);
+      nodeEntries.add(entry);
+    }
+    return entry;
+  }
+
+  /**
+   * Get the node entry matching a container on this node
+   * @param container container
+   * @return matching node instance for the role
+   */
+  public NodeEntry getOrCreate(Container container) {
+    return getOrCreate(ContainerPriority.extractRole(container));
+  }
+
+  /**
+   * Count the number of active role instances on this node
+   * @param role role index
+   * @return 0 if there are none, otherwise the #of nodes that are running and
+   * not being released already.
+   */
+  public int getActiveRoleInstances(int role) {
+    NodeEntry nodeEntry = get(role);
+    return (nodeEntry != null ) ? nodeEntry.getActive() : 0;
+  }
+  
+  /**
+   * Count the number of live role instances on this node
+   * @param role role index
+   * @return 0 if there are none, otherwise the #of nodes that are running 
+   */
+  public int getLiveRoleInstances(int role) {
+    NodeEntry nodeEntry = get(role);
+    return (nodeEntry != null ) ? nodeEntry.getLive() : 0;
+  }
+
+  /**
+   * Is the node considered online
+   * @return the node
+   */
+  public boolean isOnline() {
+    return !nodeState.isUnusable();
+  }
+
+  /**
+   * Query for a node being considered unreliable
+   * @param role role key
+   * @param threshold threshold above which a node is considered unreliable
+   * @return true if the node is considered unreliable
+   */
+  public boolean isConsideredUnreliable(int role, int threshold) {
+    NodeEntry entry = get(role);
+    return entry != null && entry.getFailedRecently() > threshold;
+  }
+
+  /**
+   * Get the entry for a role -and remove it if present
+   * @param role the role index
+   * @return the entry that WAS there
+   */
+  public synchronized NodeEntry remove(int role) {
+    NodeEntry nodeEntry = get(role);
+    if (nodeEntry != null) {
+      nodeEntries.remove(nodeEntry);
+    }
+    return nodeEntry;
+  }
+
+  public synchronized void set(int role, NodeEntry nodeEntry) {
+    remove(role);
+    nodeEntries.add(nodeEntry);
+  }
+
+  /**
+   * run through each entry; gc'ing & removing old ones that don't have
+   * a recent failure count (we care about those)
+   * @param absoluteTime age in millis
+   * @return true if there are still entries left
+   */
+  public synchronized boolean purgeUnusedEntries(long absoluteTime) {
+    boolean active = false;
+    ListIterator<NodeEntry> entries = nodeEntries.listIterator();
+    while (entries.hasNext()) {
+      NodeEntry entry = entries.next();
+      if (entry.notUsedSince(absoluteTime) && entry.getFailedRecently() == 0) {
+        entries.remove();
+      } else {
+        active = true;
+      }
+    }
+    return active;
+  }
+
+
+  /**
+   * run through each entry resetting the failure count
+   */
+  public synchronized void resetFailedRecently() {
+    for (NodeEntry entry : nodeEntries) {
+      entry.resetFailedRecently();
+    }
+  }
+  
+  @Override
+  public String toString() {
+    return hostname;
+  }
+
+  /**
+   * Full dump of entry including children
+   * @return a multi-line description fo the node
+   */
+  public String toFullString() {
+    final StringBuilder sb =
+      new StringBuilder(toString());
+    sb.append("{ ");
+    for (NodeEntry entry : nodeEntries) {
+      sb.append(String.format("\n  [%02d]  ", entry.rolePriority));
+        sb.append(entry.toString());
+    }
+    sb.append("} ");
+    return sb.toString();
+  }
+
+  /**
+   * Equality test is purely on the hostname of the node address
+   * @param o other
+   * @return true if the hostnames are equal
+   */
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    NodeInstance that = (NodeInstance) o;
+    return hostname.equals(that.hostname);
+  }
+
+  @Override
+  public int hashCode() {
+    return hostname.hashCode();
+  }
+
+
+  /**
+   * Predicate to query if the number of recent failures of a role
+   * on this node exceeds that role's failure threshold.
+   * If there is no record of a deployment of that role on this
+   * node, the failure count is taken as "0".
+   * @param role role to look up
+   * @return true if the failure rate is above the threshold.
+   */
+  public boolean exceedsFailureThreshold(RoleStatus role) {
+    NodeEntry entry = get(role.getKey());
+    int numFailuresOnLastHost = entry != null ? entry.getFailedRecently() : 0;
+    int failureThreshold = role.getNodeFailureThreshold();
+    return failureThreshold < 0 || numFailuresOnLastHost > failureThreshold;
+  }
+
+  /**
+   * Produced a serialized form which can be served up as JSON
+   * @param naming map of priority -> value for naming entries
+   * @return a summary of the current role status.
+   */
+  public synchronized NodeInformation serialize(Map<Integer, String> naming) {
+    NodeInformation info = new NodeInformation();
+    info.hostname = hostname;
+    // null-handling state constructor
+    info.state = "" + nodeState;
+    info.lastUpdated = nodeStateUpdateTime;
+    info.labels = nodeLabels;
+    if (nodeReport != null) {
+      info.httpAddress = nodeReport.getHttpAddress();
+      info.rackName = nodeReport.getRackName();
+      info.healthReport = nodeReport.getHealthReport();
+    }
+    info.entries = new HashMap<>(nodeEntries.size());
+    for (NodeEntry nodeEntry : nodeEntries) {
+      String name = naming.get(nodeEntry.rolePriority);
+      if (name == null) {
+        name = Integer.toString(nodeEntry.rolePriority);
+      }
+      info.entries.put(name, nodeEntry.serialize());
+    }
+    return info;
+  }
+
+  /**
+   * Is this node instance a suitable candidate for the specific role?
+   * @param role role ID
+   * @param label label which must match, or "" for no label checks
+   * @return true if the node has space for this role, is running and the labels
+   * match.
+   */
+  public boolean canHost(int role, String label) {
+    return isOnline()
+        && (SliderUtils.isUnset(label) || label.equals(nodeLabels))   // label match
+        && getOrCreate(role).isAvailable();                          // no live role
+  }
+
+  /**
+   * A comparator for sorting entries where the node is preferred over another.
+   *
+   * The exact algorithm may change: current policy is "most recent first", so sorted
+   * on the lastUsed
+   *
+   * the comparision is a positive int if left is preferred to right;
+   * negative if right over left, 0 for equal
+   */
+  public static class Preferred implements Comparator<NodeInstance>, Serializable {
+
+    private static final Comparators.InvertedLongComparator comparator =
+        new Comparators.InvertedLongComparator();
+    private final int role;
+
+    public Preferred(int role) {
+      this.role = role;
+    }
+
+    @Override
+    public int compare(NodeInstance o1, NodeInstance o2) {
+      NodeEntry left = o1.get(role);
+      NodeEntry right = o2.get(role);
+      long ageL = left != null ? left.getLastUsed() : -1;
+      long ageR = right != null ? right.getLastUsed() : -1;
+      return comparator.compare(ageL, ageR);
+    }
+  }
+
+  /**
+   * A comparator for sorting entries where the role is newer than
+   * the other. 
+   * This sort only compares the lastUsed field, not whether the
+   * node is in use or not
+   */
+  public static class MoreActiveThan implements Comparator<NodeInstance>,
+                                           Serializable {
+
+    private final int role;
+
+    public MoreActiveThan(int role) {
+      this.role = role;
+    }
+
+    @Override
+    public int compare(NodeInstance left, NodeInstance right) {
+      int activeLeft = left.getActiveRoleInstances(role);
+      int activeRight = right.getActiveRoleInstances(role);
+      return activeRight - activeLeft;
+    }
+  }
+  /**
+   * A comparator for sorting entries alphabetically
+   */
+  public static class CompareNames implements Comparator<NodeInstance>,
+                                           Serializable {
+
+    public CompareNames() {
+    }
+
+    @Override
+    public int compare(NodeInstance left, NodeInstance right) {
+      return left.hostname.compareTo(right.hostname);
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeMap.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeMap.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeMap.java
new file mode 100644
index 0000000..3858b68
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeMap.java
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Node map map -and methods to work with it. 
+ * Not Synchronized: caller is expected to lock access.
+ */
+public class NodeMap extends HashMap<String, NodeInstance> {
+  protected static final Logger log =
+    LoggerFactory.getLogger(NodeMap.class);
+
+  /**
+   * number of roles
+   */
+  private final int roleSize;
+
+  /**
+   * Construct
+   * @param roleSize number of roles
+   */
+  public NodeMap(int roleSize) {
+    this.roleSize = roleSize;
+  }
+
+  /**
+   * Get the node instance for the specific node -creating it if needed
+   * @param hostname node
+   * @return the instance
+   */
+  public NodeInstance getOrCreate(String hostname) {
+    NodeInstance node = get(hostname);
+    if (node == null) {
+      node = new NodeInstance(hostname, roleSize);
+      put(hostname, node);
+    }
+    return node;
+  }
+
+  /**
+   * List the active nodes
+   * @param role role
+   * @return a possibly empty sorted list of all nodes that are active
+   * in that role
+   */
+  public List<NodeInstance> listActiveNodes(int role) {
+    List<NodeInstance> nodes = new ArrayList<>();
+    for (NodeInstance instance : values()) {
+      if (instance.getActiveRoleInstances(role) > 0) {
+        nodes.add(instance);
+      }
+    }
+    Collections.sort(nodes, new NodeInstance.MoreActiveThan(role));
+    return nodes;
+  }
+
+  /**
+   * reset the failed recently counters
+   */
+  public void resetFailedRecently() {
+    for (Map.Entry<String, NodeInstance> entry : entrySet()) {
+      NodeInstance ni = entry.getValue();
+      ni.resetFailedRecently();
+    }
+  }
+
+  /**
+   * Update the node state. Return true if the node state changed: either by
+   * being created, or by changing its internal state as defined
+   * by {@link NodeInstance#updateNode(NodeReport)}.
+   *
+   * @param hostname host name
+   * @param report latest node report
+   * @return true if the node state changed enough for a request evaluation.
+   */
+  public boolean updateNode(String hostname, NodeReport report) {
+    boolean nodeExisted = get(hostname) != null;
+    boolean updated = getOrCreate(hostname).updateNode(report);
+    return updated || !nodeExisted;
+  }
+
+  /**
+   * Clone point
+   * @return a shallow clone
+   */
+  @Override
+  public Object clone() {
+    return super.clone();
+  }
+
+  /**
+   * Insert a list of nodes into the map; overwrite any with that name
+   * This is a bulk operation for testing.
+   * @param nodes collection of nodes.
+   */
+  @VisibleForTesting
+  public void insert(Collection<NodeInstance> nodes) {
+    for (NodeInstance node : nodes) {
+      put(node.hostname, node);
+    }
+  }
+
+  /**
+   * Test helper: build or update a cluster from a list of node reports
+   * @param reports the list of reports
+   * @return true if this has been considered to have changed the cluster
+   */
+  @VisibleForTesting
+  public boolean buildOrUpdate(List<NodeReport> reports) {
+    boolean updated = false;
+    for (NodeReport report : reports) {
+      updated |= getOrCreate(report.getNodeId().getHost()).updateNode(report);
+    }
+    return updated;
+  }
+
+  /**
+   * Scan the current node map for all nodes capable of hosting an instance
+   * @param role role ID
+   * @param label label which must match, or "" for no label checks
+   * @return a possibly empty list of node instances matching the criteria.
+   */
+  public List<NodeInstance> findAllNodesForRole(int role, String label) {
+    List<NodeInstance> nodes = new ArrayList<>(size());
+    for (NodeInstance instance : values()) {
+      if (instance.canHost(role, label)) {
+        nodes.add(instance);
+      }
+    }
+    Collections.sort(nodes, new NodeInstance.CompareNames());
+    return nodes;
+  }
+
+  @Override
+  public synchronized String toString() {
+    final StringBuilder sb = new StringBuilder("NodeMap{");
+    List<String> keys = new ArrayList<>(keySet());
+    Collections.sort(keys);
+    for (String key : keys) {
+      sb.append(key).append(": ");
+      sb.append(get(key).toFullString()).append("\n");
+    }
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequest.java
new file mode 100644
index 0000000..4357ef8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequest.java
@@ -0,0 +1,428 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.client.api.AMRMClient;
+import org.apache.hadoop.yarn.client.api.InvalidContainerRequestException;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.server.appmaster.operations.CancelSingleRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Tracks an outstanding request. This is used to correlate an allocation response
+ * with the node and role used in the request.
+ * <p>
+ * The node identifier may be null -which indicates that a request was made without
+ * a specific target node
+ * <p>
+ * Equality and the hash code are based <i>only</i> on the role and hostname,
+ * which are fixed in the constructor. This means that a simple 
+ * instance constructed with (role, hostname) can be used to look up
+ * a complete request instance in the {@link OutstandingRequestTracker} map
+ */
+public final class OutstandingRequest extends RoleHostnamePair {
+  protected static final Logger log =
+    LoggerFactory.getLogger(OutstandingRequest.class);
+
+  /**
+   * Node the request is for -may be null
+   */
+  public final NodeInstance node;
+
+  /**
+   * A list of all possible nodes to list in an AA request. For a non-AA
+   * request where {@link #node} is set, element 0 of the list is the same
+   * value.
+   */
+  public final List<NodeInstance> nodes = new ArrayList<>(1);
+
+  /**
+   * Optional label. This is cached as the request option (explicit-location + label) is forbidden,
+   * yet the label needs to be retained for escalation.
+   */
+  public String label;
+
+  /**
+   * Requested time in millis.
+   * <p>
+   * Only valid after {@link #buildContainerRequest(Resource, RoleStatus, long)}
+   */
+  private AMRMClient.ContainerRequest issuedRequest;
+  
+  /**
+   * Requested time in millis.
+   * <p>
+   * Only valid after {@link #buildContainerRequest(Resource, RoleStatus, long)}
+   */
+  private long requestedTimeMillis;
+
+  /**
+   * Time in millis after which escalation should be triggered..
+   * <p>
+   * Only valid after {@link #buildContainerRequest(Resource, RoleStatus, long)}
+   */
+  private long escalationTimeoutMillis;
+
+  /**
+   * Has the placement request been escalated?
+   */
+  private boolean escalated;
+
+  /**
+   * Flag to indicate that escalation is allowed
+   */
+  private boolean mayEscalate;
+
+  /**
+   * Priority of request; only valid after the request is built up
+   */
+  private int priority = -1;
+
+  /**
+   * Is this an Anti-affine request which should be cancelled on
+   * a cluster resize?
+   */
+  private boolean antiAffine = false;
+
+  /**
+   * Create a request
+   * @param roleId role
+   * @param node node -can be null
+   */
+  public OutstandingRequest(int roleId,
+                            NodeInstance node) {
+    super(roleId, node != null ? node.hostname : null);
+    this.node = node;
+    nodes.add(node);
+  }
+
+  /**
+   * Create an outstanding request with the given role and hostname
+   * Important: this is useful only for map lookups -the other constructor
+   * with the NodeInstance parameter is needed to generate node-specific
+   * container requests
+   * @param roleId role
+   * @param hostname hostname
+   */
+  public OutstandingRequest(int roleId, String hostname) {
+    super(roleId, hostname);
+    this.node = null;
+  }
+
+  /**
+   * Create an Anti-affine reques, including all listed nodes (there must be one)
+   * as targets.
+   * @param roleId role
+   * @param nodes list of nodes
+   */
+  public OutstandingRequest(int roleId, List<NodeInstance> nodes) {
+    super(roleId, nodes.get(0).hostname);
+    this.node = null;
+    this.antiAffine = true;
+    this.nodes.addAll(nodes);
+  }
+
+  /**
+   * Is the request located in the cluster, that is: does it have a node.
+   * @return true if a node instance was supplied in the constructor
+   */
+  public boolean isLocated() {
+    return node != null;
+  }
+
+  public long getRequestedTimeMillis() {
+    return requestedTimeMillis;
+  }
+
+  public long getEscalationTimeoutMillis() {
+    return escalationTimeoutMillis;
+  }
+
+  public synchronized boolean isEscalated() {
+    return escalated;
+  }
+
+  public boolean mayEscalate() {
+    return mayEscalate;
+  }
+
+  public AMRMClient.ContainerRequest getIssuedRequest() {
+    return issuedRequest;
+  }
+
+  public int getPriority() {
+    return priority;
+  }
+
+  public boolean isAntiAffine() {
+    return antiAffine;
+  }
+
+  public void setAntiAffine(boolean antiAffine) {
+    this.antiAffine = antiAffine;
+  }
+
+  /**
+   * Build a container request.
+   * <p>
+   *  The value of {@link #node} is used to direct a lot of policy. If null,
+   *  placement is relaxed.
+   *  If not null, the choice of whether to use the suggested node
+   *  is based on the placement policy and failure history.
+   * <p>
+   * If the request has an address, it is set in the container request
+   * (with a flag to enable relaxed priorities).
+   * <p>
+   * This operation sets the requested time flag, used for tracking timeouts
+   * on outstanding requests
+   * @param resource resource
+   * @param role role
+   * @param time time in millis to record as request time
+   * @return the request to raise
+   */
+  public synchronized AMRMClient.ContainerRequest buildContainerRequest(
+      Resource resource, RoleStatus role, long time) {
+    Preconditions.checkArgument(resource != null, "null `resource` arg");
+    Preconditions.checkArgument(role != null, "null `role` arg");
+
+    // cache label for escalation
+    label = role.getLabelExpression();
+    requestedTimeMillis = time;
+    escalationTimeoutMillis = time + role.getPlacementTimeoutSeconds() * 1000;
+    String[] hosts;
+    boolean relaxLocality;
+    boolean strictPlacement = role.isStrictPlacement();
+    NodeInstance target = this.node;
+    String nodeLabels;
+
+    if (isAntiAffine()) {
+      int size = nodes.size();
+      log.info("Creating anti-affine request across {} nodes; first node = {}",
+          size, hostname);
+      hosts = new String[size];
+      StringBuilder builder = new StringBuilder(size * 16);
+      int c = 0;
+      for (NodeInstance nodeInstance : nodes) {
+        hosts[c++] = nodeInstance.hostname;
+        builder.append(nodeInstance.hostname).append(" ");
+      }
+      log.debug("Full host list: [ {}]", builder);
+      escalated = false;
+      mayEscalate = false;
+      relaxLocality = false;
+      nodeLabels = null;
+    } else if (target != null) {
+      // placed request. Hostname is used in request
+      hosts = new String[1];
+      hosts[0] = target.hostname;
+      // and locality flag is set to false; Slider will decide when
+      // to relax things
+      relaxLocality = false;
+
+      log.info("Submitting request for container on {}", hosts[0]);
+      // enable escalation for all but strict placements.
+      escalated = false;
+      mayEscalate = !strictPlacement;
+      nodeLabels = null;
+    } else {
+      // no hosts
+      hosts = null;
+      // relax locality is mandatory on an unconstrained placement
+      relaxLocality = true;
+      // declare that the the placement is implicitly escalated.
+      escalated = true;
+      // and forbid it happening
+      mayEscalate = false;
+      nodeLabels = label;
+    }
+    Priority pri = ContainerPriority.createPriority(roleId, !relaxLocality);
+    priority = pri.getPriority();
+    issuedRequest = new AMRMClient.ContainerRequest(resource,
+                                      hosts,
+                                      null,
+                                      pri,
+                                      relaxLocality,
+                                      nodeLabels);
+    validate();
+    return issuedRequest;
+  }
+
+
+  /**
+   * Build an escalated container request, updating {@link #issuedRequest} with
+   * the new value.
+   * @return the new container request, which has the same resource and label requirements
+   * as the original one, and the same host, but: relaxed placement, and a changed priority
+   * so as to place it into the relaxed list.
+   */
+  public synchronized AMRMClient.ContainerRequest escalate() {
+    Preconditions.checkNotNull(issuedRequest, "cannot escalate if request not issued " + this);
+    log.debug("Escalating {}", this.toString());
+    escalated = true;
+
+    // this is now the priority
+    // it is tagged as unlocated because it needs to go into a different
+    // set of outstanding requests from the strict placements
+    Priority pri = ContainerPriority.createPriority(roleId, false);
+    // update the field
+    priority = pri.getPriority();
+
+    String[] nodes;
+    List<String> issuedRequestNodes = issuedRequest.getNodes();
+    if (SliderUtils.isUnset(label) && issuedRequestNodes != null) {
+      nodes = issuedRequestNodes.toArray(new String[issuedRequestNodes.size()]);
+    } else {
+      nodes = null;
+    }
+
+    issuedRequest = new AMRMClient.ContainerRequest(issuedRequest.getCapability(),
+        nodes,
+        null,
+        pri,
+        true,
+        label);
+    validate();
+    return issuedRequest;
+  }
+      
+  /**
+   * Mark the request as completed (or canceled).
+   * <p>
+   *   Current action: if a node is defined, its request count is decremented
+   */
+  public void completed() {
+    if (node != null) {
+      node.getOrCreate(roleId).requestCompleted();
+    }
+  }
+
+  /**
+   * Query to see if the request is available and ready to be escalated
+   * @param time time to check against
+   * @return true if escalation should begin
+   */
+  public synchronized boolean shouldEscalate(long time) {
+    return mayEscalate
+           && !escalated
+           && issuedRequest != null
+           && escalationTimeoutMillis < time;
+  }
+
+  /**
+   * Query for the resource requirements matching; always false before a request is issued
+   * @param resource
+   * @return
+   */
+  public synchronized boolean resourceRequirementsMatch(Resource resource) {
+    return issuedRequest != null && Resources.fitsIn(issuedRequest.getCapability(), resource);
+  }
+
+  @Override
+  public String toString() {
+    boolean requestHasLocation = ContainerPriority.hasLocation(getPriority());
+    final StringBuilder sb = new StringBuilder("OutstandingRequest{");
+    sb.append("roleId=").append(roleId);
+    if (hostname != null) {
+      sb.append(", hostname='").append(hostname).append('\'');
+    }
+    sb.append(", node=").append(node);
+    sb.append(", hasLocation=").append(requestHasLocation);
+    sb.append(", label=").append(label);
+    sb.append(", requestedTimeMillis=").append(requestedTimeMillis);
+    sb.append(", mayEscalate=").append(mayEscalate);
+    sb.append(", escalated=").append(escalated);
+    sb.append(", escalationTimeoutMillis=").append(escalationTimeoutMillis);
+    sb.append(", issuedRequest=").append(
+        issuedRequest != null ? SliderUtils.requestToString(issuedRequest) : "(null)");
+    sb.append('}');
+    return sb.toString();
+  }
+
+  /**
+   * Create a cancel operation
+   * @return an operation that can be used to cancel the request
+   */
+  public CancelSingleRequest createCancelOperation() {
+    Preconditions.checkState(issuedRequest != null, "No issued request to cancel");
+    return new CancelSingleRequest(issuedRequest);
+  }
+
+  /**
+   * Valid if a node label expression specified on container request is valid or
+   * not. Mimics the logic in AMRMClientImpl, so can be used for preflight checking
+   * and in mock tests
+   *
+   */
+  public void validate() throws InvalidContainerRequestException {
+    Preconditions.checkNotNull(issuedRequest, "request has not yet been built up");
+    AMRMClient.ContainerRequest containerRequest = issuedRequest;
+    String requestDetails = this.toString();
+    validateContainerRequest(containerRequest, priority, requestDetails);
+  }
+
+  /**
+   * Inner Validation logic for container request
+   * @param containerRequest request
+   * @param priority raw priority of role
+   * @param requestDetails details for error messages
+   */
+  @VisibleForTesting
+  public static void validateContainerRequest(AMRMClient.ContainerRequest containerRequest,
+    int priority, String requestDetails) {
+    String exp = containerRequest.getNodeLabelExpression();
+    boolean hasRacks = containerRequest.getRacks() != null &&
+      (!containerRequest.getRacks().isEmpty());
+    boolean hasNodes = containerRequest.getNodes() != null &&
+      (!containerRequest.getNodes().isEmpty());
+
+    boolean hasLabel = SliderUtils.isSet(exp);
+
+    // Don't support specifying >= 2 node labels in a node label expression now
+    if (hasLabel && (exp.contains("&&") || exp.contains("||"))) {
+      throw new InvalidContainerRequestException(
+          "Cannot specify more than two node labels"
+              + " in a single node label expression: " + requestDetails);
+    }
+
+    // Don't allow specify node label against ANY request listing hosts or racks
+    if (hasLabel && ( hasRacks || hasNodes)) {
+      throw new InvalidContainerRequestException(
+          "Cannot specify node label with rack or node: " + requestDetails);
+    }
+  }
+
+  /**
+   * Create a new role/hostname pair for indexing.
+   * @return a new index.
+   */
+  public RoleHostnamePair getIndex() {
+    return new RoleHostnamePair(roleId, hostname);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequestTracker.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequestTracker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequestTracker.java
new file mode 100644
index 0000000..c16aa3c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/state/OutstandingRequestTracker.java
@@ -0,0 +1,482 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.appmaster.state;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.client.api.AMRMClient;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.server.appmaster.operations.AbstractRMOperation;
+import org.apache.slider.server.appmaster.operations.CancelSingleRequest;
+import org.apache.slider.server.appmaster.operations.ContainerRequestOperation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Tracks outstanding requests made with a specific placement option.
+ * <p>
+ *   <ol>
+ *     <li>Used to decide when to return a node to 'can request containers here' list</li>
+ *     <li>Used to identify requests where placement has timed out, and so issue relaxed requests</li>
+ *   </ol>
+ * <p>
+ * If an allocation comes in that is not in the map: either the allocation
+ * was unplaced, or the placed allocation could not be met on the specified
+ * host, and the RM/scheduler fell back to another location. 
+ */
+
+public class OutstandingRequestTracker {
+  protected static final Logger log =
+    LoggerFactory.getLogger(OutstandingRequestTracker.class);
+
+  /**
+   * no requests; saves creating a new list if not needed
+   */
+  private final List<AbstractRMOperation> NO_REQUESTS = new ArrayList<>(0);
+ 
+  private Map<RoleHostnamePair, OutstandingRequest> placedRequests = new HashMap<>();
+
+  /**
+   * List of open requests; no specific details on them.
+   */
+  private List<OutstandingRequest> openRequests = new ArrayList<>();
+
+  /**
+   * Create a new request for the specific role.
+   * <p>
+   * If a location is set, the request is added to {@link #placedRequests}.
+   * If not, it is added to {@link #openRequests}
+   * <p>
+   * This does not update the node instance's role's request count
+   * @param instance node instance to manager
+   * @param role role index
+   * @return a new request
+   */
+  public synchronized OutstandingRequest newRequest(NodeInstance instance, int role) {
+    OutstandingRequest request = new OutstandingRequest(role, instance);
+    if (request.isLocated()) {
+      placedRequests.put(request.getIndex(), request);
+    } else {
+      openRequests.add(request);
+    }
+    return request;
+  }
+
+  /**
+   * Create a new Anti-affine request for the specific role
+   * <p>
+   * It is added to {@link #openRequests}
+   * <p>
+   * This does not update the node instance's role's request count
+   * @param role role index
+   * @param nodes list of suitable nodes
+   * @param label label to use
+   * @return a new request
+   */
+  public synchronized OutstandingRequest newAARequest(int role,
+      List<NodeInstance> nodes,
+      String label) {
+    Preconditions.checkArgument(!nodes.isEmpty());
+    // safety check to verify the allocation will hold
+    for (NodeInstance node : nodes) {
+      Preconditions.checkState(node.canHost(role, label),
+        "Cannot allocate role ID %d to node %s", role, node);
+    }
+    OutstandingRequest request = new OutstandingRequest(role, nodes);
+    openRequests.add(request);
+    return request;
+  }
+
+  /**
+   * Look up any oustanding request to a (role, hostname). 
+   * @param role role index
+   * @param hostname hostname
+   * @return the request or null if there was no outstanding one in the {@link #placedRequests}
+   */
+  @VisibleForTesting
+  public synchronized OutstandingRequest lookupPlacedRequest(int role, String hostname) {
+    Preconditions.checkArgument(hostname != null, "null hostname");
+    return placedRequests.get(new RoleHostnamePair(role, hostname));
+  }
+
+  /**
+   * Remove a request
+   * @param request matching request to find
+   * @return the request or null for no match in the {@link #placedRequests}
+   */
+  @VisibleForTesting
+  public synchronized OutstandingRequest removePlacedRequest(OutstandingRequest request) {
+    return placedRequests.remove(request);
+  }
+
+  /**
+   * Notification that a container has been allocated
+   *
+   * <ol>
+   *   <li>drop it from the {@link #placedRequests} structure.</li>
+   *   <li>generate the cancellation request</li>
+   *   <li>for AA placement, any actions needed</li>
+   * </ol>
+   *
+   * @param role role index
+   * @param hostname hostname
+   * @return the allocation outcome
+   */
+  public synchronized ContainerAllocationResults onContainerAllocated(int role,
+      String hostname,
+      Container container) {
+    final String containerDetails = SliderUtils.containerToString(container);
+    log.debug("Processing allocation for role {}  on {}", role,
+        containerDetails);
+    ContainerAllocationResults allocation = new ContainerAllocationResults();
+    ContainerAllocationOutcome outcome;
+    OutstandingRequest request = placedRequests.remove(new OutstandingRequest(role, hostname));
+    if (request != null) {
+      //satisfied request
+      log.debug("Found oustanding placed request for container: {}", request);
+      request.completed();
+      // derive outcome from status of tracked request
+      outcome = request.isEscalated()
+          ? ContainerAllocationOutcome.Escalated
+          : ContainerAllocationOutcome.Placed;
+    } else {
+      // not in the list; this is an open placement
+      // scan through all containers in the open request list
+      request = removeOpenRequest(container);
+      if (request != null) {
+        log.debug("Found open outstanding request for container: {}", request);
+        request.completed();
+        outcome = ContainerAllocationOutcome.Open;
+      } else {
+        log.warn("No oustanding request found for container {}, outstanding queue has {} entries ",
+            containerDetails,
+            openRequests.size());
+        outcome = ContainerAllocationOutcome.Unallocated;
+      }
+    }
+    if (request != null && request.getIssuedRequest() != null) {
+      allocation.operations.add(request.createCancelOperation());
+    } else {
+      // there's a request, but no idea what to cancel.
+      // rather than try to recover from it inelegantly, (and cause more confusion),
+      // log the event, but otherwise continue
+      log.warn("Unexpected allocation of container " + SliderUtils.containerToString(container));
+    }
+
+    allocation.origin = request;
+    allocation.outcome = outcome;
+    return allocation;
+  }
+
+  /**
+   * Find and remove an open request. Determine it by scanning open requests
+   * for one whose priority & resource requirements match that of the container
+   * allocated.
+   * @param container container allocated
+   * @return a request which matches the allocation, or null for "no match"
+   */
+  private OutstandingRequest removeOpenRequest(Container container) {
+    int pri = container.getPriority().getPriority();
+    Resource resource = container.getResource();
+    OutstandingRequest request = null;
+    ListIterator<OutstandingRequest> openlist = openRequests.listIterator();
+    while (openlist.hasNext() && request == null) {
+      OutstandingRequest r = openlist.next();
+      if (r.getPriority() == pri) {
+        // matching resource
+        if (r.resourceRequirementsMatch(resource)) {
+          // match of priority and resources
+          request = r;
+          openlist.remove();
+        } else {
+          log.debug("Matched priorities but resources different");
+        }
+      }
+    }
+    return request;
+  }
+  
+  /**
+   * Determine which host was a role type most recently used on, so that
+   * if a choice is made of which (potentially surplus) containers to use,
+   * the most recent one is picked first. This operation <i>does not</i>
+   * change the role history, though it queries it.
+   */
+  static class newerThan implements Comparator<Container>, Serializable {
+    private RoleHistory rh;
+    
+    public newerThan(RoleHistory rh) {
+      this.rh = rh;
+    }
+
+    /**
+     * Get the age of a node hosting container. If it is not known in the history, 
+     * return 0.
+     * @param c container
+     * @return age, null if there's no entry for it. 
+     */
+    private long getAgeOf(Container c) {
+      long age = 0;
+      NodeInstance node = rh.getExistingNodeInstance(c);
+      int role = ContainerPriority.extractRole(c);
+      if (node != null) {
+        NodeEntry nodeEntry = node.get(role);
+        if (nodeEntry != null) {
+          age = nodeEntry.getLastUsed();
+        }
+      }
+      return age;
+    }
+
+    /**
+     * Comparator: which host is more recent?
+     * @param c1 container 1
+     * @param c2 container 2
+     * @return 1 if c2 older-than c1, 0 if equal; -1 if c1 older-than c2
+     */
+    @Override
+    public int compare(Container c1, Container c2) {
+      int role1 = ContainerPriority.extractRole(c1);
+      int role2 = ContainerPriority.extractRole(c2);
+      if (role1 < role2) return -1;
+      if (role1 > role2) return 1;
+
+      long age = getAgeOf(c1);
+      long age2 = getAgeOf(c2);
+
+      if (age > age2) {
+        return -1;
+      } else if (age < age2) {
+        return 1;
+      }
+      // equal
+      return 0;
+    }
+  }
+
+  /**
+   * Take a list of requests and split them into specific host requests and
+   * generic assignments. This is to give requested hosts priority
+   * in container assignments if more come back than expected
+   * @param rh RoleHistory instance
+   * @param inAllocated the list of allocated containers
+   * @param outPlaceRequested initially empty list of requested locations 
+   * @param outUnplaced initially empty list of unrequested hosts
+   */
+  public synchronized void partitionRequests(RoleHistory rh,
+      List<Container> inAllocated,
+      List<Container> outPlaceRequested,
+      List<Container> outUnplaced) {
+    Collections.sort(inAllocated, new newerThan(rh));
+    for (Container container : inAllocated) {
+      int role = ContainerPriority.extractRole(container);
+      String hostname = RoleHistoryUtils.hostnameOf(container);
+      if (placedRequests.containsKey(new OutstandingRequest(role, hostname))) {
+        outPlaceRequested.add(container);
+      } else {
+        outUnplaced.add(container);
+      }
+    }
+  }
+  
+
+  /**
+   * Reset list all outstanding requests for a role: return the hostnames
+   * of any canceled requests
+   *
+   * @param role role to cancel
+   * @return possibly empty list of hostnames
+   */
+  public synchronized List<NodeInstance> resetOutstandingRequests(int role) {
+    List<NodeInstance> hosts = new ArrayList<>();
+    Iterator<Map.Entry<RoleHostnamePair, OutstandingRequest>> iterator =
+      placedRequests.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<RoleHostnamePair, OutstandingRequest> next =
+        iterator.next();
+      OutstandingRequest request = next.getValue();
+      if (request.roleId == role) {
+        iterator.remove();
+        request.completed();
+        hosts.add(request.node);
+      }
+    }
+    ListIterator<OutstandingRequest> openlist = openRequests.listIterator();
+    while (openlist.hasNext()) {
+      OutstandingRequest next = openlist.next();
+      if (next.roleId == role) {
+        openlist.remove();
+      }
+    }
+    return hosts;
+  }
+
+  /**
+   * Get a list of outstanding requests. The list is cloned, but the contents
+   * are shared
+   * @return a list of the current outstanding requests
+   */
+  public synchronized List<OutstandingRequest> listPlacedRequests() {
+    return new ArrayList<>(placedRequests.values());
+  }
+
+  /**
+   * Get a list of outstanding requests. The list is cloned, but the contents
+   * are shared
+   * @return a list of the current outstanding requests
+   */
+  public synchronized List<OutstandingRequest> listOpenRequests() {
+    return new ArrayList<>(openRequests);
+  }
+
+  /**
+   * Escalate operation as triggered by external timer.
+   * @return a (usually empty) list of cancel/request operations.
+   */
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  public synchronized List<AbstractRMOperation> escalateOutstandingRequests(long now) {
+    if (placedRequests.isEmpty()) {
+      return NO_REQUESTS;
+    }
+
+    List<AbstractRMOperation> operations = new ArrayList<>();
+    for (OutstandingRequest outstandingRequest : placedRequests.values()) {
+      synchronized (outstandingRequest) {
+        // sync escalation check with operation so that nothing can happen to state
+        // of the request during the escalation
+        if (outstandingRequest.shouldEscalate(now)) {
+
+          // time to escalate
+          CancelSingleRequest cancel = outstandingRequest.createCancelOperation();
+          operations.add(cancel);
+          AMRMClient.ContainerRequest escalated = outstandingRequest.escalate();
+          operations.add(new ContainerRequestOperation(escalated));
+        }
+      }
+      
+    }
+    return operations;
+  }
+
+  /**
+   * Cancel all outstanding AA requests from the lists of requests.
+   *
+   * This does not remove them from the role status; they must be reset
+   * by the caller.
+   *
+   */
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  public synchronized List<AbstractRMOperation> cancelOutstandingAARequests() {
+
+    log.debug("Looking for AA request to cancel");
+    List<AbstractRMOperation> operations = new ArrayList<>();
+
+    // first, all placed requests
+    List<RoleHostnamePair> requestsToRemove = new ArrayList<>(placedRequests.size());
+    for (Map.Entry<RoleHostnamePair, OutstandingRequest> entry : placedRequests.entrySet()) {
+      OutstandingRequest outstandingRequest = entry.getValue();
+      synchronized (outstandingRequest) {
+        if (outstandingRequest.isAntiAffine()) {
+          // time to escalate
+          operations.add(outstandingRequest.createCancelOperation());
+          requestsToRemove.add(entry.getKey());
+        }
+      }
+    }
+    for (RoleHostnamePair keys : requestsToRemove) {
+      placedRequests.remove(keys);
+    }
+
+    // second, all open requests
+    ListIterator<OutstandingRequest> orit = openRequests.listIterator();
+    while (orit.hasNext()) {
+      OutstandingRequest outstandingRequest =  orit.next();
+      synchronized (outstandingRequest) {
+        if (outstandingRequest.isAntiAffine()) {
+          // time to escalate
+          operations.add(outstandingRequest.createCancelOperation());
+          orit.remove();
+        }
+      }
+    }
+    log.info("Cancelling {} outstanding AA requests", operations.size());
+
+    return operations;
+  }
+
+  /**
+   * Extract a specific number of open requests for a role
+   * @param roleId role Id
+   * @param count count to extract
+   * @return a list of requests which are no longer in the open request list
+   */
+  public synchronized List<OutstandingRequest> extractOpenRequestsForRole(int roleId, int count) {
+    List<OutstandingRequest> results = new ArrayList<>();
+    ListIterator<OutstandingRequest> openlist = openRequests.listIterator();
+    while (openlist.hasNext() && count > 0) {
+      OutstandingRequest openRequest = openlist.next();
+      if (openRequest.roleId == roleId) {
+        results.add(openRequest);
+        openlist.remove();
+        count--;
+      }
+    }
+    return results;
+  }
+
+  /**
+   * Extract a specific number of placed requests for a role
+   * @param roleId role Id
+   * @param count count to extract
+   * @return a list of requests which are no longer in the placed request data structure
+   */
+  public synchronized List<OutstandingRequest> extractPlacedRequestsForRole(int roleId, int count) {
+    List<OutstandingRequest> results = new ArrayList<>();
+    Iterator<Map.Entry<RoleHostnamePair, OutstandingRequest>>
+        iterator = placedRequests.entrySet().iterator();
+    while (iterator.hasNext() && count > 0) {
+      OutstandingRequest request = iterator.next().getValue();
+      if (request.roleId == roleId) {
+        results.add(request);
+        count--;
+      }
+    }
+    // now cull them from the map
+    for (OutstandingRequest result : results) {
+      placedRequests.remove(result);
+    }
+
+    return results;
+  }
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[74/76] [abbrv] hadoop git commit: YARN-5812. Exception during GET call - "Failed to retrieve application: null". Contributed by Gour Saha

Posted by ji...@apache.org.
YARN-5812. Exception during GET call - "Failed to retrieve application: null". Contributed by Gour Saha


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

Branch: refs/heads/yarn-native-services
Commit: fc6901a7ae6dcee08faf149cc4026a15a4cbc908
Parents: 92f6fe5
Author: Jian He <ji...@apache.org>
Authored: Wed Nov 2 15:55:48 2016 -0700
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 .../api/impl/ApplicationApiService.java         | 36 ++++++++++++++------
 1 file changed, 26 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc6901a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
index 37bd134..6db69ac 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
@@ -50,7 +50,6 @@ import javax.ws.rs.core.Response.Status;
 
 import org.apache.commons.lang.SerializationUtils;
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.fs.PathNotFoundException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -905,6 +904,10 @@ public class ApplicationApiService implements ApplicationApi {
 
     // state
     String appState = jsonGetAsString(appStatus, "state");
+    if (appState == null) {
+      // consider that app is still in ACCEPTED state
+      appState = String.valueOf(StateValues.STATE_INCOMPLETE);
+    }
     switch (Integer.parseInt(appState)) {
       case StateValues.STATE_LIVE:
         app.setState(ApplicationState.STARTED);
@@ -1069,6 +1072,9 @@ public class ApplicationApiService implements ApplicationApi {
             String status = null;
             try {
               status = sliderClient.actionStatus(appName);
+            } catch (BadClusterStateException e) {
+              logger.warn("Application not running yet", e);
+              return EMPTY_JSON_OBJECT;
             } catch (Exception e) {
               logger.error("Exception calling slider.actionStatus", e);
               return EMPTY_JSON_OBJECT;
@@ -1097,7 +1103,7 @@ public class ApplicationApiService implements ApplicationApi {
             try {
               registry = sliderClient.actionRegistryGetConfig(registryArgs)
                 .asJson();
-            } catch (FileNotFoundException | PathNotFoundException e) {
+            } catch (FileNotFoundException | NotFoundException e) {
               // ignore and return empty object
               return EMPTY_JSON_OBJECT;
             } catch (Exception e) {
@@ -1192,23 +1198,33 @@ public class ApplicationApiService implements ApplicationApi {
     // little longer for it to stop from YARN point of view. Slider destroy
     // fails if the application is not completely stopped. Hence the need to
     // call destroy in a controlled loop few times (only if exit code is
-    // EXIT_APPLICATION_IN_USE), before giving up.
+    // EXIT_APPLICATION_IN_USE or EXIT_INSTANCE_EXISTS), before giving up.
     boolean keepTrying = true;
-    int maxDeleteAttempt = 5;
-    int deleteAttempt = 0;
-    while (keepTrying && deleteAttempt < maxDeleteAttempt) {
+    int maxDeleteAttempts = 5;
+    int deleteAttempts = 0;
+    int sleepIntervalInMillis = 500;
+    while (keepTrying && deleteAttempts < maxDeleteAttempts) {
       try {
         destroySliderApplication(appName);
         keepTrying = false;
       } catch (SliderException e) {
-        logger.error("Delete application threw exception", e);
-        if (e.getExitCode() == SliderExitCodes.EXIT_APPLICATION_IN_USE) {
-          deleteAttempt++;
+        if (e.getExitCode() == SliderExitCodes.EXIT_APPLICATION_IN_USE
+            || e.getExitCode() == SliderExitCodes.EXIT_INSTANCE_EXISTS) {
+          deleteAttempts++;
+          // If we used up all the allowed delete attempts, let's log it as
+          // error before giving up. Otherwise log as warn.
+          if (deleteAttempts < maxDeleteAttempts) {
+            logger.warn("Application not in stopped state, waiting for {}ms"
+                + " before trying delete again", sleepIntervalInMillis);
+          } else {
+            logger.error("Delete application failed", e);
+          }
           try {
-            Thread.sleep(500);
+            Thread.sleep(sleepIntervalInMillis);
           } catch (InterruptedException e1) {
           }
         } else {
+          logger.error("Delete application threw exception", e);
           return Response.status(Status.INTERNAL_SERVER_ERROR).build();
         }
       } catch (Exception e) {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[26/76] [abbrv] hadoop git commit: YARN-5461. Initial code ported from slider-core module. (jianhe)

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8cab88d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
new file mode 100644
index 0000000..73e0879
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
@@ -0,0 +1,2548 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.common.tools;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
+import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream;
+import org.apache.commons.compress.archivers.zip.ZipArchiveEntry;
+import org.apache.commons.compress.archivers.zip.ZipArchiveInputStream;
+import org.apache.commons.io.output.ByteArrayOutputStream;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.GlobFilter;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.nativeio.NativeIO;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.ExitUtil;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.util.VersionInfo;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.client.api.AMRMClient;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.slider.Slider;
+import org.apache.slider.api.InternalKeys;
+import org.apache.slider.api.RoleKeys;
+import org.apache.slider.api.types.ContainerInformation;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.common.SliderXmlConfKeys;
+import org.apache.slider.common.params.Arguments;
+import org.apache.slider.common.params.SliderActions;
+import org.apache.slider.core.conf.ConfTreeOperations;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.core.exceptions.BadClusterStateException;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.BadConfigException;
+import org.apache.slider.core.exceptions.ErrorStrings;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.core.launch.ClasspathConstructor;
+import org.apache.slider.core.main.LauncherExitCodes;
+import org.apache.slider.providers.agent.AgentKeys;
+import org.apache.slider.server.services.utility.PatternValidator;
+import org.apache.slider.server.services.workflow.ForkedProcessService;
+import org.apache.zookeeper.server.util.KerberosUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedOutputStream;
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.PrintWriter;
+import java.io.Serializable;
+import java.io.StringWriter;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.net.URLDecoder;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.TimeZone;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Pattern;
+import java.util.zip.GZIPOutputStream;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipOutputStream;
+
+/**
+ * These are slider-specific Util methods
+ */
+public final class SliderUtils {
+
+  private static final Logger log = LoggerFactory.getLogger(SliderUtils.class);
+
+  /**
+   * Atomic bool to track whether or not process security has already been
+   * turned on (prevents re-entrancy)
+   */
+  private static final AtomicBoolean processSecurityAlreadyInitialized =
+      new AtomicBoolean(false);
+  public static final String JAVA_SECURITY_KRB5_REALM =
+      "java.security.krb5.realm";
+  public static final String JAVA_SECURITY_KRB5_KDC = "java.security.krb5.kdc";
+
+  /**
+   * Winutils
+   */
+  public static final String WINUTILS = "WINUTILS.EXE";
+  /**
+   * name of openssl program
+   */
+  public static final String OPENSSL = "openssl";
+
+  /**
+   * name of python program
+   */
+  public static final String PYTHON = "python";
+
+  /**
+   * type of docker standalone application
+   */
+  public static final String DOCKER = "docker";
+  /**
+   * type of docker on yarn application
+   */
+  public static final String DOCKER_YARN = "yarn_docker";
+
+  public static final int NODE_LIST_LIMIT = 10;
+
+  private SliderUtils() {
+  }
+
+  /**
+   * Implementation of set-ness, groovy definition of true/false for a string
+   * @param s string
+   * @return true iff the string is neither null nor empty
+   */
+  public static boolean isUnset(String s) {
+    return s == null || s.isEmpty();
+  }
+
+  public static boolean isSet(String s) {
+    return !isUnset(s);
+  }
+
+  /**
+   * Probe for a list existing and not being empty
+   * @param l list
+   * @return true if the reference is valid and it contains entries
+   */
+
+  public static boolean isNotEmpty(List l) {
+    return l != null && !l.isEmpty();
+  }
+
+  /**
+   * Probe for a map existing and not being empty
+   * @param m map
+   * @return true if the reference is valid and it contains map entries
+   */
+  public static boolean isNotEmpty(Map m) {
+    return m != null && !m.isEmpty();
+  }
+  
+  /*
+   * Validates whether num is an integer
+   * @param num
+   * @param msg the message to be shown in exception
+   */
+  @SuppressWarnings("ResultOfMethodCallIgnored")
+  private static void validateNumber(String num, String msg) throws
+      BadConfigException {
+    try {
+      Integer.parseInt(num);
+    } catch (NumberFormatException nfe) {
+      throw new BadConfigException(msg + num);
+    }
+  }
+
+  /*
+   * Translates the trailing JVM heapsize unit: g, G, m, M
+   * This assumes designated unit of 'm'
+   * @param heapsize
+   * @return heapsize in MB
+   */
+  public static String translateTrailingHeapUnit(String heapsize) throws
+      BadConfigException {
+    String errMsg = "Bad heapsize: ";
+    if (heapsize.endsWith("m") || heapsize.endsWith("M")) {
+      String num = heapsize.substring(0, heapsize.length() - 1);
+      validateNumber(num, errMsg);
+      return num;
+    }
+    if (heapsize.endsWith("g") || heapsize.endsWith("G")) {
+      String num = heapsize.substring(0, heapsize.length() - 1) + "000";
+      validateNumber(num, errMsg);
+      return num;
+    }
+    // check if specified heap size is a number
+    validateNumber(heapsize, errMsg);
+    return heapsize;
+  }
+
+  /**
+   * recursive directory delete
+   * @param dir dir to delete
+   * @throws IOException on any problem
+   */
+  public static void deleteDirectoryTree(File dir) throws IOException {
+    if (dir.exists()) {
+      if (dir.isDirectory()) {
+        log.info("Cleaning up {}", dir);
+        //delete the children
+        File[] files = dir.listFiles();
+        if (files == null) {
+          throw new IOException("listfiles() failed for " + dir);
+        }
+        for (File file : files) {
+          log.info("deleting {}", file);
+          if (!file.delete()) {
+            log.warn("Unable to delete " + file);
+          }
+        }
+        if (!dir.delete()) {
+          log.warn("Unable to delete " + dir);
+        }
+      } else {
+        throw new IOException("Not a directory " + dir);
+      }
+    } else {
+      //not found, do nothing
+      log.debug("No output dir yet");
+    }
+  }
+
+  /**
+   * Find a containing JAR
+   * @param clazz class to find
+   * @return the file
+   * @throws IOException any IO problem, including the class not having a
+   * classloader
+   * @throws FileNotFoundException if the class did not resolve to a file
+   */
+  public static File findContainingJarOrFail(Class clazz) throws IOException {
+    File localFile = SliderUtils.findContainingJar(clazz);
+    if (null == localFile) {
+      throw new FileNotFoundException("Could not find JAR containing " + clazz);
+    }
+    return localFile;
+  }
+
+
+  /**
+   * Find a containing JAR
+   * @param my_class class to find
+   * @return the file or null if it is not found
+   * @throws IOException any IO problem, including the class not having a
+   * classloader
+   */
+  public static File findContainingJar(Class my_class) throws IOException {
+    ClassLoader loader = my_class.getClassLoader();
+    if (loader == null) {
+      throw new IOException(
+          "Class " + my_class + " does not have a classloader!");
+    }
+    String class_file = my_class.getName().replaceAll("\\.", "/") + ".class";
+    Enumeration<URL> urlEnumeration = loader.getResources(class_file);
+    if (urlEnumeration == null) {
+      throw new IOException("Unable to find resources for class " + my_class);
+    }
+
+    for (; urlEnumeration.hasMoreElements(); ) {
+      URL url = urlEnumeration.nextElement();
+      if ("jar".equals(url.getProtocol())) {
+        String toReturn = url.getPath();
+        if (toReturn.startsWith("file:")) {
+          toReturn = toReturn.substring("file:".length());
+        }
+        // URLDecoder is a misnamed class, since it actually decodes
+        // x-www-form-urlencoded MIME type rather than actual
+        // URL encoding (which the file path has). Therefore it would
+        // decode +s to ' 's which is incorrect (spaces are actually
+        // either unencoded or encoded as "%20"). Replace +s first, so
+        // that they are kept sacred during the decoding process.
+        toReturn = toReturn.replaceAll("\\+", "%2B");
+        toReturn = URLDecoder.decode(toReturn, "UTF-8");
+        String jarFilePath = toReturn.replaceAll("!.*$", "");
+        return new File(jarFilePath);
+      } else {
+        log.info("could not locate JAR containing {} URL={}", my_class, url);
+      }
+    }
+    return null;
+  }
+
+  public static void checkPort(String hostname, int port, int connectTimeout)
+      throws IOException {
+    InetSocketAddress addr = new InetSocketAddress(hostname, port);
+    checkPort(hostname, addr, connectTimeout);
+  }
+
+  @SuppressWarnings("SocketOpenedButNotSafelyClosed")
+  public static void checkPort(String name,
+      InetSocketAddress address,
+      int connectTimeout)
+      throws IOException {
+    try(Socket socket = new Socket()) {
+      socket.connect(address, connectTimeout);
+    } catch (Exception e) {
+      throw new IOException("Failed to connect to " + name
+                            + " at " + address
+                            + " after " + connectTimeout + "milliseconds"
+                            + ": " + e,
+          e);
+    }
+  }
+
+  public static void checkURL(String name, String url, int timeout) throws
+      IOException {
+    InetSocketAddress address = NetUtils.createSocketAddr(url);
+    checkPort(name, address, timeout);
+  }
+
+  /**
+   * A required file
+   * @param role role of the file (for errors)
+   * @param filename the filename
+   * @throws ExitUtil.ExitException if the file is missing
+   * @return the file
+   */
+  public static File requiredFile(String filename, String role) throws
+      IOException {
+    if (filename.isEmpty()) {
+      throw new ExitUtil.ExitException(-1, role + " file not defined");
+    }
+    File file = new File(filename);
+    if (!file.exists()) {
+      throw new ExitUtil.ExitException(-1,
+          role + " file not found: " +
+          file.getCanonicalPath());
+    }
+    return file;
+  }
+
+  private static final PatternValidator clusternamePattern
+      = new PatternValidator("[a-z][a-z0-9_-]*");
+
+  /**
+   * Normalize a cluster name then verify that it is valid
+   * @param name proposed cluster name
+   * @return true iff it is valid
+   */
+  public static boolean isClusternameValid(String name) {
+    return name != null && clusternamePattern.matches(name);
+  }
+
+  public static boolean oldIsClusternameValid(String name) {
+    if (name == null || name.isEmpty()) {
+      return false;
+    }
+    int first = name.charAt(0);
+    if (0 == (Character.getType(first) & Character.LOWERCASE_LETTER)) {
+      return false;
+    }
+
+    for (int i = 0; i < name.length(); i++) {
+      int elt = (int) name.charAt(i);
+      int t = Character.getType(elt);
+      if (0 == (t & Character.LOWERCASE_LETTER)
+          && 0 == (t & Character.DECIMAL_DIGIT_NUMBER)
+          && elt != '-'
+          && elt != '_') {
+        return false;
+      }
+      if (!Character.isLetterOrDigit(elt) && elt != '-' && elt != '_') {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Copy a directory to a new FS -both paths must be qualified. If
+   * a directory needs to be created, supplied permissions can override
+   * the default values. Existing directories are not touched
+   * @param conf conf file
+   * @param srcDirPath src dir
+   * @param destDirPath dest dir
+   * @param permission permission for the dest directory; null means "default"
+   * @return # of files copies
+   */
+  public static int copyDirectory(Configuration conf,
+      Path srcDirPath,
+      Path destDirPath,
+      FsPermission permission) throws
+      IOException,
+      BadClusterStateException {
+    FileSystem srcFS = FileSystem.get(srcDirPath.toUri(), conf);
+    FileSystem destFS = FileSystem.get(destDirPath.toUri(), conf);
+    //list all paths in the src.
+    if (!srcFS.exists(srcDirPath)) {
+      throw new FileNotFoundException("Source dir not found " + srcDirPath);
+    }
+    if (!srcFS.isDirectory(srcDirPath)) {
+      throw new FileNotFoundException(
+          "Source dir not a directory " + srcDirPath);
+    }
+    GlobFilter dotFilter = new GlobFilter("[!.]*");
+    FileStatus[] entries = srcFS.listStatus(srcDirPath, dotFilter);
+    int srcFileCount = entries.length;
+    if (srcFileCount == 0) {
+      return 0;
+    }
+    if (permission == null) {
+      permission = FsPermission.getDirDefault();
+    }
+    if (!destFS.exists(destDirPath)) {
+      new SliderFileSystem(destFS, conf).createWithPermissions(destDirPath,
+          permission);
+    }
+    Path[] sourcePaths = new Path[srcFileCount];
+    for (int i = 0; i < srcFileCount; i++) {
+      FileStatus e = entries[i];
+      Path srcFile = e.getPath();
+      if (srcFS.isDirectory(srcFile)) {
+        String msg = "Configuration dir " + srcDirPath
+                     + " contains a directory " + srcFile;
+        log.warn(msg);
+        throw new IOException(msg);
+      }
+      log.debug("copying src conf file {}", srcFile);
+      sourcePaths[i] = srcFile;
+    }
+    log.debug("Copying {} files from {} to dest {}", srcFileCount,
+        srcDirPath,
+        destDirPath);
+    FileUtil.copy(srcFS, sourcePaths, destFS, destDirPath, false, true, conf);
+    return srcFileCount;
+  }
+
+
+  public static String stringify(Throwable t) {
+    StringWriter sw = new StringWriter();
+    sw.append(t.toString()).append('\n');
+    t.printStackTrace(new PrintWriter(sw));
+    return sw.toString();
+  }
+
+  /**
+   * Create a configuration with Slider-specific tuning.
+   * This is done rather than doing custom configs.
+   * @return the config
+   */
+  public static YarnConfiguration createConfiguration() {
+    YarnConfiguration conf = new YarnConfiguration();
+    patchConfiguration(conf);
+    return conf;
+  }
+
+  /**
+   * Take an existing conf and patch it for Slider's needs. Useful
+   * in Service.init & RunService methods where a shared config is being
+   * passed in
+   * @param conf configuration
+   * @return the patched configuration
+   */
+  public static Configuration patchConfiguration(Configuration conf) {
+
+    //if the fallback option is NOT set, enable it.
+    //if it is explicitly set to anything -leave alone
+    if (conf.get(SliderXmlConfKeys.IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH) == null) {
+      conf.set(SliderXmlConfKeys.IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH, "true");
+    }
+    return conf;
+  }
+
+  /**
+   * Take a collection, return a list containing the string value of every
+   * element in the collection.
+   * @param c collection
+   * @return a stringified list
+   */
+  public static List<String> collectionToStringList(Collection c) {
+    List<String> l = new ArrayList<>(c.size());
+    for (Object o : c) {
+      l.add(o.toString());
+    }
+    return l;
+  }
+
+  /**
+   * Join an collection of objects with a separator that appears after every
+   * instance in the list -including at the end
+   * @param collection collection to call toString() on each element
+   * @param separator separator string
+   * @return the joined entries
+   */
+  public static String join(Collection collection, String separator) {
+    return join(collection, separator, true);
+  }
+
+  /**
+   * Join an collection of objects with a separator that appears after every
+   * instance in the list -optionally at the end
+   * @param collection collection to call toString() on each element
+   * @param separator separator string
+   * @param trailing add a trailing entry or not
+   * @return the joined entries
+   */
+  public static String join(Collection collection,
+      String separator,
+      boolean trailing) {
+    StringBuilder b = new StringBuilder();
+    // fast return on empty collection
+    if (collection.isEmpty()) {
+      return trailing ? separator : "";
+    }
+    for (Object o : collection) {
+      b.append(o);
+      b.append(separator);
+    }
+    int length = separator.length();
+    String s = b.toString();
+    return (trailing || s.isEmpty()) ?
+           s : (b.substring(0, b.length() - length));
+  }
+
+  /**
+   * Join an array of strings with a separator that appears after every
+   * instance in the list -including at the end
+   * @param collection strings
+   * @param separator separator string
+   * @return the joined entries
+   */
+  public static String join(String[] collection, String separator) {
+    return join(collection, separator, true);
+
+
+  }
+
+  /**
+   * Join an array of strings with a separator that appears after every
+   * instance in the list -optionally at the end
+   * @param collection strings
+   * @param separator separator string
+   * @param trailing add a trailing entry or not
+   * @return the joined entries
+   */
+  public static String join(String[] collection, String separator,
+      boolean trailing) {
+    return join(Arrays.asList(collection), separator, trailing);
+  }
+
+  /**
+   * Join an array of strings with a separator that appears after every
+   * instance in the list -except at the end
+   * @param collection strings
+   * @param separator separator string
+   * @return the list
+   */
+  public static String joinWithInnerSeparator(String separator,
+      Object... collection) {
+    StringBuilder b = new StringBuilder();
+    boolean first = true;
+
+    for (Object o : collection) {
+      if (first) {
+        first = false;
+      } else {
+        b.append(separator);
+      }
+      b.append(o.toString());
+      b.append(separator);
+    }
+    return b.toString();
+  }
+
+  /**
+   * Resolve a mandatory environment variable
+   * @param key env var
+   * @return the resolved value
+   * @throws BadClusterStateException
+   */
+  public static String mandatoryEnvVariable(String key) throws
+      BadClusterStateException {
+    String v = System.getenv(key);
+    if (v == null) {
+      throw new BadClusterStateException("Missing Environment variable " + key);
+    }
+    return v;
+  }
+
+  public static String appReportToString(ApplicationReport r,
+      String separator) {
+    StringBuilder builder = new StringBuilder(512);
+    builder.append("application ")
+           .append(
+               r.getName())
+           .append("/")
+           .append(r.getApplicationType())
+           .append(separator);
+    Set<String> tags = r.getApplicationTags();
+    if (!tags.isEmpty()) {
+      for (String tag : tags) {
+        builder.append(tag).append(separator);
+      }
+    }
+    DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm");
+    dateFormat.setTimeZone(TimeZone.getDefault());
+    builder.append("state: ").append(r.getYarnApplicationState());
+    String trackingUrl = r.getTrackingUrl();
+    if (isSet(trackingUrl)) {
+      builder.append(separator).append("URL: ").append(trackingUrl);
+    }
+    builder.append(separator)
+           .append("Started: ")
+           .append(dateFormat.format(new Date(r.getStartTime())));
+    long finishTime = r.getFinishTime();
+    if (finishTime > 0) {
+      builder.append(separator)
+             .append("Finished: ")
+             .append(dateFormat.format(new Date(finishTime)));
+    }
+    String rpcHost = r.getHost();
+    if (!isSet(rpcHost)) {
+      builder.append(separator)
+             .append("RPC :")
+             .append(rpcHost)
+             .append(':')
+             .append(r.getRpcPort());
+    }
+    String diagnostics = r.getDiagnostics();
+    if (!isSet(diagnostics)) {
+      builder.append(separator).append("Diagnostics :").append(diagnostics);
+    }
+    return builder.toString();
+  }
+
+  /**
+   * Convert the instance details of an application to a string
+   * @param name instance name
+   * @param report the application report
+   * @param verbose verbose output
+   * @return a string
+   */
+  public static String instanceDetailsToString(String name,
+      ApplicationReport report,
+      List<ContainerInformation> containers,
+      String version,
+      Set<String> components,
+      boolean verbose) {
+    // format strings
+    String staticf = "%-30s";
+    String reportedf = staticf + "  %10s  %-42s";
+    String livef = reportedf + "  %s";
+    StringBuilder builder = new StringBuilder(200);
+    if (report == null) {
+      builder.append(String.format(staticf, name));
+    } else {
+      // there's a report to look at
+      String appId = report.getApplicationId().toString();
+      String state = report.getYarnApplicationState().toString();
+      if (report.getYarnApplicationState() == YarnApplicationState.RUNNING) {
+        // running: there's a URL
+        builder.append(
+            String.format(livef, name, state, appId, report.getTrackingUrl()));
+      } else {
+        builder.append(String.format(reportedf, name, state, appId));
+      }
+      if (verbose) {
+        builder.append('\n');
+        builder.append(SliderUtils.appReportToString(report, "\n  "));
+      }
+      if (containers != null) {
+        builder.append('\n');
+        builder.append(SliderUtils.containersToString(containers, version,
+            components));
+      }
+    }
+
+    builder.append('\n');
+    return builder.toString();
+  }
+
+  public static String containersToString(
+      List<ContainerInformation> containers, String version,
+      Set<String> components) {
+    String containerf = "  %-28s  %30s  %45s  %s\n";
+    StringBuilder builder = new StringBuilder(512);
+    builder.append("Containers:\n");
+    builder.append(String.format("  %-28s  %30s  %45s  %s\n", "Component Name",
+        "App Version", "Container Id", "Container Info/Logs"));
+    for (ContainerInformation container : containers) {
+      if (filter(container.appVersion, version)
+          || filter(container.component, components)) {
+        continue;
+      }
+      builder.append(String.format(containerf, container.component,
+          container.appVersion, container.containerId, container.host
+              + SliderKeys.YARN_CONTAINER_PATH + container.containerId));
+    }
+    return builder.toString();
+  }
+
+  /**
+   * Filter a string value given a single filter
+   * 
+   * @param value
+   *          the string value to check
+   * @param filter
+   *          a single string filter
+   * @return return true if value should be trapped, false if it should be let
+   *         through
+   */
+  public static boolean filter(String value, String filter) {
+    return !(StringUtils.isEmpty(filter) || filter.equals(value));
+  }
+
+  /**
+   * Filter a string value given a set of filters
+   * 
+   * @param value
+   *          the string value to check
+   * @param filters
+   *          a set of string filters
+   * @return return true if value should be trapped, false if it should be let
+   *         through
+   */
+  public static boolean filter(String value, Set<String> filters) {
+    return !(filters.isEmpty() || filters.contains(value));
+  }
+
+  /**
+   * Sorts the given list of application reports, most recently started 
+   * or finished instance first.
+   *
+   * @param instances list of instances
+   */
+  public static void sortApplicationsByMostRecent(List<ApplicationReport> instances) {
+    Collections.sort(instances, new MostRecentlyStartedOrFinishedFirst());
+  }
+
+  /**
+   * Sorts the given list of application reports
+   * Finished instances are ordered by finished time and running/accepted instances are
+   * ordered by start time
+   * Finally Instance are order by finished instances coming after running instances
+   *
+   * @param instances list of instances
+   */
+  public static void sortApplicationReport(List<ApplicationReport> instances) {
+    if (instances.size() <= 1) {
+      return;
+    }
+    List<ApplicationReport> nonLiveInstance =
+        new ArrayList<>(instances.size());
+    List<ApplicationReport> liveInstance =
+        new ArrayList<>(instances.size());
+
+    for (ApplicationReport report : instances) {
+      if (report.getYarnApplicationState() == YarnApplicationState.RUNNING
+          ||
+          report.getYarnApplicationState() == YarnApplicationState.ACCEPTED) {
+        liveInstance.add(report);
+      } else {
+        nonLiveInstance.add(report);
+      }
+    }
+
+    if (liveInstance.size() > 1) {
+      Collections.sort(liveInstance, new MostRecentlyStartedAppFirst());
+    }
+    if (nonLiveInstance.size() > 1) {
+      Collections.sort(nonLiveInstance, new MostRecentAppFinishFirst());
+    }
+    instances.clear();
+    instances.addAll(liveInstance);
+    instances.addAll(nonLiveInstance);
+  }
+
+  /**
+   * Built a (sorted) map of application reports, mapped to the instance name
+   * The list is sorted, and the addition process does not add a report
+   * if there is already one that exists. If the list handed in is sorted,
+   * those that are listed first form the entries returned
+   * @param instances list of intances
+   * @param minState minimum YARN state to be included
+   * @param maxState maximum YARN state to be included
+   * @return all reports in the list whose state &gt;= minimum and &lt;= maximum
+   */
+  public static Map<String, ApplicationReport> buildApplicationReportMap(
+      List<ApplicationReport> instances,
+      YarnApplicationState minState, YarnApplicationState maxState) {
+    TreeMap<String, ApplicationReport> map = new TreeMap<>();
+    for (ApplicationReport report : instances) {
+      YarnApplicationState state = report.getYarnApplicationState();
+      if (state.ordinal() >= minState.ordinal() &&
+          state.ordinal() <= maxState.ordinal() &&
+          map.get(report.getName()) == null) {
+        map.put(report.getName(), report);
+      }
+    }
+    return map;
+  }
+
+  /**
+   * Take a map and produce a sorted equivalent
+   * @param source source map
+   * @return a map whose iterator returns the string-sorted ordering of entries
+   */
+  public static Map<String, String> sortedMap(Map<String, String> source) {
+    Map<String, String> out = new TreeMap<>(source);
+    return out;
+  }
+
+  /**
+   * Convert a properties instance to a string map.
+   * @param properties source property object
+   * @return a string map
+   */
+  public static Map<String, String> toMap(Properties properties) {
+    Map<String, String> out = new HashMap<>(properties.size());
+    for (Map.Entry<Object, Object> entry : properties.entrySet()) {
+      out.put(entry.getKey().toString(), entry.getValue().toString());
+    }
+    return out;
+  }
+
+  /**
+   * Merge in one map to another -all entries in the second map are
+   * merged into the first -overwriting any duplicate keys.
+   * @param first first map -the updated one.
+   * @param second the map that is merged in
+   * @return the first map
+   */
+  public static Map<String, String> mergeMap(Map<String, String> first,
+      Map<String, String> second) {
+    first.putAll(second);
+    return first;
+  }
+
+  /**
+   * Merge a set of entries into a map. This will take the entryset of
+   * a map, or a Hadoop collection itself
+   * @param dest destination
+   * @param entries entries
+   * @return dest -with the entries merged in
+   */
+  public static Map<String, String> mergeEntries(Map<String, String> dest,
+      Iterable<Map.Entry<String, String>> entries) {
+    for (Map.Entry<String, String> entry : entries) {
+      dest.put(entry.getKey(), entry.getValue());
+    }
+    return dest;
+  }
+
+  /**
+   * Generic map merge logic
+   * @param first first map
+   * @param second second map
+   * @param <T1> key type
+   * @param <T2> value type
+   * @return 'first' merged with the second
+   */
+  public static <T1, T2> Map<T1, T2> mergeMaps(Map<T1, T2> first,
+      Map<T1, T2> second) {
+    first.putAll(second);
+    return first;
+  }
+
+  /**
+   * Generic map merge logic
+   * @param first first map
+   * @param second second map
+   * @param <T1> key type
+   * @param <T2> value type
+   * @return 'first' merged with the second
+   */
+  public static <T1, T2> Map<T1, T2> mergeMapsIgnoreDuplicateKeys(Map<T1, T2> first,
+      Map<T1, T2> second) {
+    Preconditions.checkArgument(first != null, "Null 'first' value");
+    Preconditions.checkArgument(second != null, "Null 'second' value");
+    for (Map.Entry<T1, T2> entry : second.entrySet()) {
+      T1 key = entry.getKey();
+      if (!first.containsKey(key)) {
+        first.put(key, entry.getValue());
+      }
+    }
+    return first;
+  }
+
+  /**
+   * Convert a map to a multi-line string for printing
+   * @param map map to stringify
+   * @return a string representation of the map
+   */
+  public static String stringifyMap(Map<String, String> map) {
+    StringBuilder builder = new StringBuilder();
+    for (Map.Entry<String, String> entry : map.entrySet()) {
+      builder.append(entry.getKey())
+             .append("=\"")
+             .append(entry.getValue())
+             .append("\"\n");
+
+    }
+    return builder.toString();
+  }
+
+  /**
+   * Get the int value of a role
+   * @param roleMap map of role key->val entries
+   * @param key key the key to look for
+   * @param defVal default value to use if the key is not in the map
+   * @param min min value or -1 for do not check
+   * @param max max value or -1 for do not check
+   * @return the int value the integer value
+   * @throws BadConfigException if the value could not be parsed
+   */
+  public static int getIntValue(Map<String, String> roleMap,
+      String key,
+      int defVal,
+      int min,
+      int max
+  ) throws BadConfigException {
+    String valS = roleMap.get(key);
+    return parseAndValidate(key, valS, defVal, min, max);
+
+  }
+
+  /**
+   * Parse an int value, replacing it with defval if undefined;
+   * @param errorKey key to use in exceptions
+   * @param defVal default value to use if the key is not in the map
+   * @param min min value or -1 for do not check
+   * @param max max value or -1 for do not check
+   * @return the int value the integer value
+   * @throws BadConfigException if the value could not be parsed
+   */
+  public static int parseAndValidate(String errorKey,
+      String valS,
+      int defVal,
+      int min, int max) throws
+      BadConfigException {
+    if (valS == null) {
+      valS = Integer.toString(defVal);
+    }
+    String trim = valS.trim();
+    int val;
+    try {
+      val = Integer.decode(trim);
+    } catch (NumberFormatException e) {
+      throw new BadConfigException("Failed to parse value of "
+                                   + errorKey + ": \"" + trim + "\"");
+    }
+    if (min >= 0 && val < min) {
+      throw new BadConfigException("Value of "
+                                   + errorKey + ": " + val + ""
+                                   + "is less than the minimum of " + min);
+    }
+    if (max >= 0 && val > max) {
+      throw new BadConfigException("Value of "
+                                   + errorKey + ": " + val + ""
+                                   + "is more than the maximum of " + max);
+    }
+    return val;
+  }
+
+  public static InetSocketAddress getRmAddress(Configuration conf) {
+    return conf.getSocketAddr(YarnConfiguration.RM_ADDRESS,
+        YarnConfiguration.DEFAULT_RM_ADDRESS,
+        YarnConfiguration.DEFAULT_RM_PORT);
+  }
+
+  public static InetSocketAddress getRmSchedulerAddress(Configuration conf) {
+    return conf.getSocketAddr(YarnConfiguration.RM_SCHEDULER_ADDRESS,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_PORT);
+  }
+
+  /**
+   * probe to see if the RM scheduler is defined
+   * @param conf config
+   * @return true if the RM scheduler address is set to
+   * something other than 0.0.0.0
+   */
+  public static boolean isRmSchedulerAddressDefined(Configuration conf) {
+    InetSocketAddress address = getRmSchedulerAddress(conf);
+    return isAddressDefined(address);
+  }
+
+  /**
+   * probe to see if the address
+   * @param address network address
+   * @return true if the scheduler address is set to
+   * something other than 0.0.0.0
+   */
+  public static boolean isAddressDefined(InetSocketAddress address) {
+    if (address == null || address.getHostString() == null) {
+      return false;
+    }
+    return !(address.getHostString().equals("0.0.0.0"));
+  }
+
+  public static void setRmAddress(Configuration conf, String rmAddr) {
+    conf.set(YarnConfiguration.RM_ADDRESS, rmAddr);
+  }
+
+  public static void setRmSchedulerAddress(Configuration conf, String rmAddr) {
+    conf.set(YarnConfiguration.RM_SCHEDULER_ADDRESS, rmAddr);
+  }
+
+  public static boolean hasAppFinished(ApplicationReport report) {
+    return report == null ||
+           report.getYarnApplicationState().ordinal() >=
+           YarnApplicationState.FINISHED.ordinal();
+  }
+
+  public static String containerToString(Container container) {
+    if (container == null) {
+      return "null container";
+    }
+    return String.format(Locale.ENGLISH,
+        "ContainerID=%s nodeID=%s http=%s priority=%s resource=%s",
+        container.getId(),
+        container.getNodeId(),
+        container.getNodeHttpAddress(),
+        container.getPriority(),
+        container.getResource());
+  }
+
+  /**
+   * convert an AM report to a string for diagnostics
+   * @param report the report
+   * @return the string value
+   */
+  public static String reportToString(ApplicationReport report) {
+    if (report == null) {
+      return "Null application report";
+    }
+
+    return "App " + report.getName() + "/" + report.getApplicationType() +
+           "# " +
+           report.getApplicationId() + " user " + report.getUser() +
+           " is in state " + report.getYarnApplicationState() +
+           " RPC: " + report.getHost() + ":" + report.getRpcPort() +
+           " URL: " + report.getOriginalTrackingUrl();
+  }
+
+  /**
+   * Convert a YARN URL into a string value of a normal URL
+   * @param url URL
+   * @return string representatin
+   */
+  public static String stringify(org.apache.hadoop.yarn.api.records.URL url) {
+    StringBuilder builder = new StringBuilder();
+    builder.append(url.getScheme()).append("://");
+    if (url.getHost() != null) {
+      builder.append(url.getHost()).append(":").append(url.getPort());
+    }
+    builder.append(url.getFile());
+    return builder.toString();
+  }
+
+  public static int findFreePort(int start, int limit) {
+    if (start == 0) {
+      //bail out if the default is "dont care"
+      return 0;
+    }
+    int found = 0;
+    int port = start;
+    int finish = start + limit;
+    while (found == 0 && port < finish) {
+      if (isPortAvailable(port)) {
+        found = port;
+      } else {
+        port++;
+      }
+    }
+    return found;
+  }
+
+  /**
+   * Get a random open port
+   * @return true if the port was available for listening on
+   */
+  public static int getOpenPort() throws IOException {
+    ServerSocket socket = null;
+    try {
+      socket = new ServerSocket(0);
+      return socket.getLocalPort();
+    } finally {
+      if (socket != null) {
+        socket.close();
+      }
+    }
+  }
+
+  /**
+   * See if a port is available for listening on by trying to listen
+   * on it and seeing if that works or fails.
+   * @param port port to listen to
+   * @return true if the port was available for listening on
+   */
+  public static boolean isPortAvailable(int port) {
+    try {
+      ServerSocket socket = new ServerSocket(port);
+      socket.close();
+      return true;
+    } catch (IOException e) {
+      return false;
+    }
+  }
+
+  /**
+   * Build the environment map from a role option map, finding all entries
+   * beginning with "env.", adding them to a map of (prefix-removed)
+   * env vars
+   * @param roleOpts role options. This can be null, meaning the
+   * role is undefined
+   * @return a possibly empty map of environment variables.
+   */
+  public static Map<String, String> buildEnvMap(Map<String, String> roleOpts) {
+    return buildEnvMap(roleOpts, null);
+  }
+
+  public static Map<String, String> buildEnvMap(Map<String, String> roleOpts,
+      Map<String,String> tokenMap) {
+    Map<String, String> env = new HashMap<>();
+    if (roleOpts != null) {
+      for (Map.Entry<String, String> entry : roleOpts.entrySet()) {
+        String key = entry.getKey();
+        if (key.startsWith(RoleKeys.ENV_PREFIX)) {
+          String envName = key.substring(RoleKeys.ENV_PREFIX.length());
+          if (!envName.isEmpty()) {
+            String value = entry.getValue();
+            if (tokenMap != null) {
+              for (Map.Entry<String,String> token : tokenMap.entrySet()) {
+                value = value.replaceAll(Pattern.quote(token.getKey()),
+                    token.getValue());
+              }
+            }
+            env.put(envName, value);
+          }
+        }
+      }
+    }
+    return env;
+  }
+
+  /**
+   * Apply a set of command line options to a cluster role map
+   * @param clusterRoleMap cluster role map to merge onto
+   * @param commandOptions command opts
+   */
+  public static void applyCommandLineRoleOptsToRoleMap(
+      Map<String, Map<String, String>> clusterRoleMap,
+      Map<String, Map<String, String>> commandOptions) {
+    for (Map.Entry<String, Map<String, String>> entry : commandOptions.entrySet()) {
+      String key = entry.getKey();
+      Map<String, String> optionMap = entry.getValue();
+      Map<String, String> existingMap = clusterRoleMap.get(key);
+      if (existingMap == null) {
+        existingMap = new HashMap<String, String>();
+      }
+      log.debug("Overwriting role options with command line values {}",
+          stringifyMap(optionMap));
+      mergeMap(existingMap, optionMap);
+      //set or overwrite the role
+      clusterRoleMap.put(key, existingMap);
+    }
+  }
+
+  /**
+   * verify that the supplied cluster name is valid
+   * @param clustername cluster name
+   * @throws BadCommandArgumentsException if it is invalid
+   */
+  public static void validateClusterName(String clustername) throws
+      BadCommandArgumentsException {
+    if (!isClusternameValid(clustername)) {
+      throw new BadCommandArgumentsException(
+          "Illegal cluster name: " + clustername);
+    }
+  }
+
+  /**
+   * Verify that a Kerberos principal has been set -if not fail
+   * with an error message that actually tells you what is missing
+   * @param conf configuration to look at
+   * @param principal key of principal
+   * @throws BadConfigException if the key is not set
+   */
+  public static void verifyPrincipalSet(Configuration conf,
+      String principal) throws
+      BadConfigException {
+    String principalName = conf.get(principal);
+    if (principalName == null) {
+      throw new BadConfigException("Unset Kerberos principal : %s",
+          principal);
+    }
+    log.debug("Kerberos princial {}={}", principal, principalName);
+  }
+
+  /**
+   * Flag to indicate whether the cluster is in secure mode
+   * @param conf configuration to look at
+   * @return true if the slider client/service should be in secure mode
+   */
+  public static boolean isHadoopClusterSecure(Configuration conf) {
+    return SecurityUtil.getAuthenticationMethod(conf) !=
+           UserGroupInformation.AuthenticationMethod.SIMPLE;
+  }
+
+  /**
+   * Init security if the cluster configuration declares the cluster is secure
+   * @param conf configuration to look at
+   * @return true if the cluster is secure
+   * @throws IOException cluster is secure
+   * @throws SliderException the configuration/process is invalid
+   */
+  public static boolean maybeInitSecurity(Configuration conf) throws
+      IOException,
+      SliderException {
+    boolean clusterSecure = isHadoopClusterSecure(conf);
+    if (clusterSecure) {
+      log.debug("Enabling security");
+      initProcessSecurity(conf);
+    }
+    return clusterSecure;
+  }
+
+  /**
+   * Turn on security. This is setup to only run once.
+   * @param conf configuration to build up security
+   * @return true if security was initialized in this call
+   * @throws IOException IO/Net problems
+   * @throws BadConfigException the configuration and system state are inconsistent
+   */
+  public static boolean initProcessSecurity(Configuration conf) throws
+      IOException,
+      SliderException {
+
+    if (processSecurityAlreadyInitialized.compareAndSet(true, true)) {
+      //security is already inited
+      return false;
+    }
+
+    log.info("JVM initialized into secure mode with kerberos realm {}",
+        SliderUtils.getKerberosRealm());
+    //this gets UGI to reset its previous world view (i.e simple auth)
+    //security
+    log.debug("java.security.krb5.realm={}",
+        System.getProperty(JAVA_SECURITY_KRB5_REALM, ""));
+    log.debug("java.security.krb5.kdc={}",
+        System.getProperty(JAVA_SECURITY_KRB5_KDC, ""));
+    log.debug("hadoop.security.authentication={}",
+        conf.get(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION));
+    log.debug("hadoop.security.authorization={}",
+        conf.get(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION));
+    UserGroupInformation.setConfiguration(conf);
+    UserGroupInformation authUser = UserGroupInformation.getCurrentUser();
+    log.debug("Authenticating as {}", authUser);
+    log.debug("Login user is {}", UserGroupInformation.getLoginUser());
+    if (!UserGroupInformation.isSecurityEnabled()) {
+      throw new SliderException(LauncherExitCodes.EXIT_UNAUTHORIZED,
+          "Although secure mode is enabled," +
+         "the application has already set up its user as an insecure entity %s",
+          authUser);
+    }
+    if (authUser.getAuthenticationMethod() ==
+        UserGroupInformation.AuthenticationMethod.SIMPLE) {
+      throw new BadConfigException("Auth User is not Kerberized %s" +
+         " -security has already been set up with the wrong authentication method. "
+         + "This can occur if a file system has already been created prior to the loading of "
+         + "the security configuration.",
+          authUser);
+
+    }
+
+    SliderUtils.verifyPrincipalSet(conf, YarnConfiguration.RM_PRINCIPAL);
+    SliderUtils.verifyPrincipalSet(conf, SliderXmlConfKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY);
+    return true;
+  }
+
+  /**
+   * Force an early login: This catches any auth problems early rather than
+   * in RPC operations
+   * @throws IOException if the login fails
+   */
+  public static void forceLogin() throws IOException {
+    if (UserGroupInformation.isSecurityEnabled()) {
+      if (UserGroupInformation.isLoginKeytabBased()) {
+        UserGroupInformation.getLoginUser().reloginFromKeytab();
+      } else {
+        UserGroupInformation.getLoginUser().reloginFromTicketCache();
+      }
+    }
+  }
+
+  /**
+   * Submit a JAR containing a specific class and map it
+   * @param providerResources provider map to build up
+   * @param sliderFileSystem remote fs
+   * @param clazz class to look for
+   * @param libdir lib directory
+   * @param jarName <i>At the destination</i>
+   * @return the local resource ref
+   * @throws IOException trouble copying to HDFS
+   */
+  public static LocalResource putJar(Map<String, LocalResource> providerResources,
+      SliderFileSystem sliderFileSystem,
+      Class clazz,
+      Path tempPath,
+      String libdir,
+      String jarName
+  )
+      throws IOException, SliderException {
+    LocalResource res = sliderFileSystem.submitJarWithClass(
+        clazz,
+        tempPath,
+        libdir,
+        jarName);
+    providerResources.put(libdir + "/" + jarName, res);
+    return res;
+  }
+
+  /**
+   * Submit a JAR containing and map it
+   * @param providerResources provider map to build up
+   * @param sliderFileSystem remote fs
+   * @param libDir lib directory
+   * @param srcPath copy jars from
+   * @throws IOException, SliderException trouble copying to HDFS
+   */
+  public static void putAllJars(Map<String, LocalResource> providerResources,
+                                SliderFileSystem sliderFileSystem,
+                                Path tempPath,
+                                String libDir,
+                                String srcPath) throws IOException, SliderException {
+    log.info("Loading all dependencies from {}", srcPath);
+    if (SliderUtils.isSet(srcPath)) {
+      File srcFolder = new File(srcPath);
+      FilenameFilter jarFilter = createJarFilter();
+      File[] listOfJars = srcFolder.listFiles(jarFilter);
+      if (listOfJars == null || listOfJars.length == 0) {
+        return;
+      }
+      for (File jarFile : listOfJars) {
+        LocalResource res = sliderFileSystem.submitFile(jarFile, tempPath, libDir, jarFile.getName());
+        providerResources.put(libDir + "/" + jarFile.getName(), res);
+      }
+    }
+  }
+
+  /**
+   * Accept all filenames ending with {@code .jar}
+   * @return a filename filter
+   */
+  public static FilenameFilter createJarFilter() {
+    return new FilenameFilter() {
+      public boolean accept(File dir, String name) {
+        return name.toLowerCase(Locale.ENGLISH).endsWith(".jar");
+      }
+    };
+  }
+
+  /**
+   * Submit the AM tar.gz containing all dependencies and map it
+   * @param providerResources provider map to build up
+   * @param sliderFileSystem remote fs
+   * @throws IOException, SliderException trouble copying to HDFS
+   */
+  public static void putAmTarGzipAndUpdate(
+      Map<String, LocalResource> providerResources,
+      SliderFileSystem sliderFileSystem
+  ) throws IOException, SliderException {
+    log.info("Loading all dependencies from {}{}",
+        SliderKeys.SLIDER_DEPENDENCY_TAR_GZ_FILE_NAME,
+        SliderKeys.SLIDER_DEPENDENCY_TAR_GZ_FILE_EXT);
+    sliderFileSystem.submitTarGzipAndUpdate(providerResources);
+  }
+
+  public static Map<String, Map<String, String>> deepClone(Map<String, Map<String, String>> src) {
+    Map<String, Map<String, String>> dest = new HashMap<>();
+    for (Map.Entry<String, Map<String, String>> entry : src.entrySet()) {
+      dest.put(entry.getKey(), stringMapClone(entry.getValue()));
+    }
+    return dest;
+  }
+
+  public static Map<String, String> stringMapClone(Map<String, String> src) {
+    Map<String, String> dest = new HashMap<>();
+    return mergeEntries(dest, src.entrySet());
+  }
+
+  /**
+   * List a directory in the local filesystem
+   * @param dir directory
+   * @return a listing, one to a line
+   */
+  public static String listDir(File dir) {
+    if (dir == null) {
+      return "";
+    }
+    String[] confDirEntries = dir.list();
+    if (confDirEntries == null) {
+      return "";
+    }
+    StringBuilder builder = new StringBuilder();
+    for (String entry : confDirEntries) {
+      builder.append(entry).append("\n");
+    }
+    return builder.toString();
+  }
+
+  /**
+   * Create a file:// path from a local file
+   * @param file file to point the path
+   * @return a new Path
+   */
+  public static Path createLocalPath(File file) {
+    return new Path(file.toURI());
+  }
+
+  /**
+   * Get the current user -relays to
+   * {@link UserGroupInformation#getCurrentUser()}
+   * with any Slider-specific post processing and exception handling
+   * @return user info
+   * @throws IOException on a failure to get the credentials
+   */
+  public static UserGroupInformation getCurrentUser() throws IOException {
+
+    try {
+      UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
+      return currentUser;
+    } catch (IOException e) {
+      log.info("Failed to get user info", e);
+      throw e;
+    }
+  }
+
+  public static String getKerberosRealm() {
+    try {
+      return KerberosUtil.getDefaultRealm();
+    } catch (Exception e) {
+      log.debug("introspection into JVM internals failed", e);
+      return "(unknown)";
+
+    }
+  }
+
+  /**
+   * Register the client resource in
+   * {@link SliderKeys#SLIDER_CLIENT_XML}
+   * for Configuration instances.
+   *
+   * @return true if the resource could be loaded
+   */
+  public static URL registerClientResource() {
+    return ConfigHelper.registerDefaultResource(SliderKeys.SLIDER_CLIENT_XML);
+  }
+  
+  /**
+   * Attempt to load the slider client resource. If the
+   * resource is not on the CP an empty config is returned.
+   * @return a config
+   */
+  public static Configuration loadSliderClientXML() {
+    return ConfigHelper.loadFromResource(SliderKeys.SLIDER_CLIENT_XML);
+  }
+
+  /**
+   * Convert a char sequence to a string.
+   * This ensures that comparisons work
+   * @param charSequence source
+   * @return the string equivalent
+   */
+  public static String sequenceToString(CharSequence charSequence) {
+    StringBuilder stringBuilder = new StringBuilder(charSequence);
+    return stringBuilder.toString();
+  }
+
+  /**
+   * Build up the classpath for execution
+   * -behaves very differently on a mini test cluster vs a production
+   * production one.
+   *
+   * @param sliderConfDir relative path to the dir containing slider config
+   *                      options to put on the classpath -or null
+   * @param libdir directory containing the JAR files
+   * @param config the configuration
+   * @param usingMiniMRCluster flag to indicate the MiniMR cluster is in use
+   * (and hence the current classpath should be used, not anything built up)
+   * @return a classpath
+   */
+  public static ClasspathConstructor buildClasspath(String sliderConfDir,
+      String libdir,
+      Configuration config,
+      SliderFileSystem sliderFileSystem,
+      boolean usingMiniMRCluster) {
+
+    ClasspathConstructor classpath = new ClasspathConstructor();
+
+    // add the runtime classpath needed for tests to work
+    if (usingMiniMRCluster) {
+      // for mini cluster we pass down the java CP properties
+      // and nothing else
+      classpath.appendAll(classpath.localJVMClasspath());
+    } else {
+      if (sliderConfDir != null) {
+        classpath.addClassDirectory(sliderConfDir);
+      }
+      classpath.addLibDir(libdir);
+      if (sliderFileSystem.isFile(sliderFileSystem.getDependencyTarGzip())) {
+        classpath.addLibDir(SliderKeys.SLIDER_DEPENDENCY_LOCALIZED_DIR_LINK);
+      } else {
+        log.info(
+            "For faster submission of apps, upload dependencies using cmd {} {}",
+            SliderActions.ACTION_DEPENDENCY, Arguments.ARG_UPLOAD);
+      }
+      classpath.addRemoteClasspathEnvVar();
+      classpath.append(ApplicationConstants.Environment.HADOOP_CONF_DIR.$$());
+    }
+    return classpath;
+  }
+
+  /**
+   * Verify that a path refers to a directory. If not
+   * logs the parent dir then throws an exception
+   * @param dir the directory
+   * @param errorlog log for output on an error
+   * @throws FileNotFoundException if it is not a directory
+   */
+  public static void verifyIsDir(File dir, Logger errorlog) throws
+      FileNotFoundException {
+    if (!dir.exists()) {
+      errorlog.warn("contents of {}: {}", dir,
+          listDir(dir.getParentFile()));
+      throw new FileNotFoundException(dir.toString());
+    }
+    if (!dir.isDirectory()) {
+      errorlog.info("contents of {}: {}", dir,
+          listDir(dir.getParentFile()));
+      throw new FileNotFoundException(
+          "Not a directory: " + dir);
+    }
+  }
+
+  /**
+   * Verify that a file exists
+   * @param file file
+   * @param errorlog log for output on an error
+   * @throws FileNotFoundException
+   */
+  public static void verifyFileExists(File file, Logger errorlog) throws
+      FileNotFoundException {
+    if (!file.exists()) {
+      errorlog.warn("contents of {}: {}", file,
+          listDir(file.getParentFile()));
+      throw new FileNotFoundException(file.toString());
+    }
+    if (!file.isFile()) {
+      throw new FileNotFoundException("Not a file: " + file.toString());
+    }
+  }
+
+  /**
+   * verify that a config option is set
+   * @param configuration config
+   * @param key key
+   * @return the value, in case it needs to be verified too
+   * @throws BadConfigException if the key is missing
+   */
+  public static String verifyOptionSet(Configuration configuration, String key,
+      boolean allowEmpty) throws BadConfigException {
+    String val = configuration.get(key);
+    if (val == null) {
+      throw new BadConfigException(
+          "Required configuration option \"%s\" not defined ", key);
+    }
+    if (!allowEmpty && val.isEmpty()) {
+      throw new BadConfigException(
+          "Configuration option \"%s\" must not be empty", key);
+    }
+    return val;
+  }
+
+  /**
+   * Verify that a keytab property is defined and refers to a non-empty file
+   *
+   * @param siteConf configuration
+   * @param prop property to look for
+   * @return the file referenced
+   * @throws BadConfigException on a failure
+   */
+  public static File verifyKeytabExists(Configuration siteConf,
+      String prop) throws
+      BadConfigException {
+    String keytab = siteConf.get(prop);
+    if (keytab == null) {
+      throw new BadConfigException("Missing keytab property %s",
+          prop);
+
+    }
+    File keytabFile = new File(keytab);
+    if (!keytabFile.exists()) {
+      throw new BadConfigException("Missing keytab file %s defined in %s",
+          keytabFile,
+          prop);
+    }
+    if (keytabFile.length() == 0 || !keytabFile.isFile()) {
+      throw new BadConfigException("Invalid keytab file %s defined in %s",
+          keytabFile,
+          prop);
+    }
+    return keytabFile;
+  }
+
+  /**
+   * Convert an epoch time to a GMT time. This
+   * uses the deprecated Date.toString() operation,
+   * so is in one place to reduce the number of deprecation warnings.
+   * @param time timestamp
+   * @return string value as ISO-9601
+   */
+  @SuppressWarnings({"CallToDateToString", "deprecation"})
+  public static String toGMTString(long time) {
+    return new Date(time).toGMTString();
+  }
+
+  /**
+   * Add the cluster build information; this will include Hadoop details too
+   * @param info cluster info
+   * @param prefix prefix for the build info
+   */
+  public static void addBuildInfo(Map<String, String> info, String prefix) {
+
+    Properties props = SliderVersionInfo.loadVersionProperties();
+    info.put(prefix + "." + SliderVersionInfo.APP_BUILD_INFO, props.getProperty(
+        SliderVersionInfo.APP_BUILD_INFO));
+    info.put(prefix + "." + SliderVersionInfo.HADOOP_BUILD_INFO,
+        props.getProperty(SliderVersionInfo.HADOOP_BUILD_INFO));
+
+    info.put(prefix + "." + SliderVersionInfo.HADOOP_DEPLOYED_INFO,
+        VersionInfo.getBranch() + " @" + VersionInfo.getSrcChecksum());
+  }
+
+  /**
+   * Set the time for an information (human, machine) timestamp pair of fields.
+   * The human time is the time in millis converted via the {@link Date} class.
+   * @param info info fields
+   * @param keyHumanTime name of human time key
+   * @param keyMachineTime name of machine time
+   * @param time timestamp
+   */
+  public static void setInfoTime(Map info,
+      String keyHumanTime,
+      String keyMachineTime,
+      long time) {
+    info.put(keyHumanTime, SliderUtils.toGMTString(time));
+    info.put(keyMachineTime, Long.toString(time));
+  }
+
+  public static Path extractImagePath(CoreFileSystem fs,
+      MapOperations internalOptions) throws
+      SliderException, IOException {
+    Path imagePath;
+    String imagePathOption =
+        internalOptions.get(InternalKeys.INTERNAL_APPLICATION_IMAGE_PATH);
+    String appHomeOption =
+        internalOptions.get(InternalKeys.INTERNAL_APPLICATION_HOME);
+    if (!isUnset(imagePathOption)) {
+      if (!isUnset(appHomeOption)) {
+        throw new BadClusterStateException(
+            ErrorStrings.E_BOTH_IMAGE_AND_HOME_DIR_SPECIFIED);
+      }
+      imagePath = fs.createPathThatMustExist(imagePathOption);
+    } else {
+      imagePath = null;
+      if (isUnset(appHomeOption)) {
+        throw new BadClusterStateException(
+            ErrorStrings.E_NO_IMAGE_OR_HOME_DIR_SPECIFIED);
+      }
+    }
+    return imagePath;
+  }
+
+  /**
+   * trigger a  JVM halt with no clean shutdown at all
+   * @param status status code for exit
+   * @param text text message
+   * @param delay delay in millis
+   * @return the timer (assuming the JVM hasn't halted yet)
+   *
+   */
+  public static Timer haltAM(int status, String text, int delay) {
+
+    Timer timer = new Timer("halt timer", false);
+    timer.schedule(new DelayedHalt(status, text), delay);
+    return timer;
+  }
+
+  public static String propertiesToString(Properties props) {
+    TreeSet<String> keys = new TreeSet<>(props.stringPropertyNames());
+    StringBuilder builder = new StringBuilder();
+    for (String key : keys) {
+      builder.append(key)
+             .append("=")
+             .append(props.getProperty(key))
+             .append("\n");
+    }
+    return builder.toString();
+  }
+
+  /**
+   * Add a subpath to an existing URL. This extends
+   * the path, inserting a / between all entries
+   * if needed.
+   * @param base base path/URL
+   * @param path subpath
+   * @return base+"/"+subpath
+   */
+  public static String appendToURL(String base, String path) {
+    StringBuilder fullpath = new StringBuilder(base);
+    if (!base.endsWith("/")) {
+      fullpath.append("/");
+    }
+    if (path.startsWith("/")) {
+      fullpath.append(path.substring(1));
+    } else {
+      fullpath.append(path);
+    }
+    return fullpath.toString();
+  }
+
+  /**
+   * Append a list of paths, inserting "/" signs as appropriate
+   * @param base base path/URL
+   * @param paths subpaths
+   * @return base+"/"+paths[0]+"/"+paths[1]...
+   */
+  public static String appendToURL(String base, String... paths) {
+    String result = base;
+    for (String path : paths) {
+      result = appendToURL(result, path);
+    }
+    return result;
+  }
+
+
+  /**
+   * Truncate the given string to a maximum length provided
+   * with a pad (...) added to the end if expected size if more than 10.
+   * @param toTruncate string to truncate; may be null
+   * @param maxSize maximum size
+   * @return the truncated/padded string. 
+   */
+  public static String truncate(String toTruncate, int maxSize) {
+    if (toTruncate == null || maxSize < 1
+        || toTruncate.length() <= maxSize) {
+      return toTruncate;
+    }
+
+    String pad = "...";
+    if (maxSize < 10) {
+      pad = "";
+    }
+    return toTruncate.substring(0, maxSize - pad.length()).concat(pad);
+  }
+
+  /**
+   * Get a string node label value from a node report
+   * @param report node report
+   * @return a single trimmed label or ""
+   */
+  public static String extractNodeLabel(NodeReport report) {
+    Set<String> newlabels = report.getNodeLabels();
+    if (newlabels != null && !newlabels.isEmpty()) {
+      return newlabels.iterator().next().trim();
+    } else {
+      return "";
+    }
+  }
+
+  /**
+   * Callable for async/scheduled halt
+   */
+  public static class DelayedHalt extends TimerTask {
+    private final int status;
+    private final String text;
+
+    public DelayedHalt(int status, String text) {
+      this.status = status;
+      this.text = text;
+    }
+
+    @Override
+    public void run() {
+      try {
+        ExitUtil.halt(status, text);
+        //this should never be reached
+      } catch (ExitUtil.HaltException e) {
+        log.info("Halt failed");
+      }
+    }
+  }
+
+  /**
+   * A compareTo function that converts the result of a long
+   * comparision into the integer that <code>Comparable</code>
+   * expects.
+   * @param left left side
+   * @param right right side
+   * @return -1, 0, 1 depending on the diff
+   */
+  public static int compareTo(long left, long right) {
+    long diff = left - right;
+    if (diff < 0) {
+      return -1;
+    }
+    if (diff > 0) {
+      return 1;
+    }
+    return 0;
+  }
+
+  /**
+   * Given a source folder create zipped file
+   *
+   * @param srcFolder
+   * @param zipFile
+   *
+   * @throws IOException
+   */
+  public static void zipFolder(File srcFolder, File zipFile) throws IOException {
+    log.info("Zipping folder {} to {}", srcFolder.getAbsolutePath(), zipFile.getAbsolutePath());
+    List<String> files = new ArrayList<>();
+    generateFileList(files, srcFolder, srcFolder, true);
+
+    byte[] buffer = new byte[1024];
+
+    try (FileOutputStream fos = new FileOutputStream(zipFile)) {
+      try (ZipOutputStream zos = new ZipOutputStream(fos)) {
+
+        for (String file : files) {
+          ZipEntry ze = new ZipEntry(file);
+          zos.putNextEntry(ze);
+          try (FileInputStream in = new FileInputStream(srcFolder + File.separator + file)) {
+            int len;
+            while ((len = in.read(buffer)) > 0) {
+              zos.write(buffer, 0, len);
+            }
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Given a source folder create a tar.gz file
+   * 
+   * @param srcFolder
+   * @param tarGzipFile
+   * 
+   * @throws IOException
+   */
+  public static void tarGzipFolder(File srcFolder, File tarGzipFile,
+      FilenameFilter filter) throws IOException {
+    log.info("Tar-gzipping folder {} to {}", srcFolder.getAbsolutePath(),
+        tarGzipFile.getAbsolutePath());
+    List<String> files = new ArrayList<>();
+    generateFileList(files, srcFolder, srcFolder, true, filter);
+
+    try(TarArchiveOutputStream taos =
+            new TarArchiveOutputStream(new GZIPOutputStream(
+        new BufferedOutputStream(new FileOutputStream(tarGzipFile))))) {
+      for (String file : files) {
+        File srcFile = new File(srcFolder, file);
+        TarArchiveEntry tarEntry = new TarArchiveEntry(
+            srcFile, file);
+        taos.putArchiveEntry(tarEntry);
+        try(FileInputStream in = new FileInputStream(srcFile)) {
+          org.apache.commons.io.IOUtils.copy(in, taos);
+        }
+        taos.flush();
+        taos.closeArchiveEntry();
+      }
+    }
+  }
+
+  /**
+   * Retrieve the HDP version if it is an HDP cluster, or null otherwise. It
+   * first checks if system property HDP_VERSION is defined. If not it checks if
+   * system env HDP_VERSION is defined.
+   * 
+   * @return HDP version (if defined) or null otherwise
+   */
+  public static String getHdpVersion() {
+    String hdpVersion = System
+        .getProperty(SliderKeys.HDP_VERSION_PROP_NAME);
+    if (StringUtils.isEmpty(hdpVersion)) {
+      hdpVersion = System.getenv(SliderKeys.HDP_VERSION_PROP_NAME);
+    }
+    return hdpVersion;
+  }
+
+  /**
+   * Query to find if it is an HDP cluster
+   * 
+   * @return true if this is invoked in an HDP cluster or false otherwise
+   */
+  public static boolean isHdp() {
+    return StringUtils.isNotEmpty(getHdpVersion());
+  }
+
+  /**
+   * Retrieve the version of the current Slider install
+   * 
+   * @return the version string of the Slider release
+   */
+  public static String getSliderVersion() {
+    if (isHdp()) {
+      return getHdpVersion();
+    } else {
+      Properties props = SliderVersionInfo.loadVersionProperties();
+      return props.getProperty(SliderVersionInfo.APP_VERSION);
+    }
+  }
+
+  private static void generateFileList(List<String> fileList, File node,
+      File rootFolder, Boolean relative) {
+    generateFileList(fileList, node, rootFolder, relative, null);
+  }
+
+  private static void generateFileList(List<String> fileList, File node,
+      File rootFolder, Boolean relative, FilenameFilter filter) {
+    if (node.isFile()) {
+      String fileFullPath = node.toString();
+      if (relative) {
+        fileList.add(fileFullPath.substring(rootFolder.toString().length() + 1,
+            fileFullPath.length()));
+      } else {
+        fileList.add(fileFullPath);
+      }
+    }
+
+    if (node.isDirectory()) {
+      String[] subNode = node.list(filter);
+      if (subNode == null || subNode.length == 0) {
+          return;
+      }
+      for (String filename : subNode) {
+        generateFileList(fileList, new File(node, filename), rootFolder,
+            relative, filter);
+      }
+    }
+  }
+
+  /**
+   * This wraps ApplicationReports and generates a string version
+   * iff the toString() operator is invoked
+   */
+  public static class OnDemandReportStringifier {
+    private final ApplicationReport report;
+
+    public OnDemandReportStringifier(ApplicationReport report) {
+      this.report = report;
+    }
+
+    @Override
+    public String toString() {
+      return appReportToString(report, "\n");
+    }
+  }
+
+  public static InputStream getApplicationResourceInputStream(FileSystem fs,
+      Path appPath,
+      String entry)
+      throws IOException {
+    InputStream is = null;
+    try(FSDataInputStream appStream = fs.open(appPath)) {
+      ZipArchiveInputStream zis = new ZipArchiveInputStream(appStream);
+      ZipArchiveEntry zipEntry;
+      boolean done = false;
+      while (!done && (zipEntry = zis.getNextZipEntry()) != null) {
+        if (entry.equals(zipEntry.getName())) {
+          int size = (int) zipEntry.getSize();
+          if (size != -1) {
+            log.info("Reading {} of size {}", zipEntry.getName(),
+                zipEntry.getSize());
+            byte[] content = new byte[size];
+            int offset = 0;
+            while (offset < size) {
+              offset += zis.read(content, offset, size - offset);
+            }
+            is = new ByteArrayInputStream(content);
+          } else {
+            log.debug("Size unknown. Reading {}", zipEntry.getName());
+            ByteArrayOutputStream baos = new ByteArrayOutputStream();
+            while (true) {
+              int byteRead = zis.read();
+              if (byteRead == -1) {
+                break;
+              }
+              baos.write(byteRead);
+            }
+            is = new ByteArrayInputStream(baos.toByteArray());
+          }
+          done = true;
+        }
+      }
+    }
+
+    return is;
+  }
+
+  /**
+   * Check for any needed libraries being present. On Unix none are needed;
+   * on windows they must be present
+   * @return true if all is well
+   */
+  public static String checkForRequiredNativeLibraries() {
+
+    if (!Shell.WINDOWS) {
+      return "";
+    }
+    StringBuilder errorText = new StringBuilder("");
+    if (!NativeIO.isAvailable()) {
+      errorText.append("No native IO library. ");
+    }
+    try {
+      String path = Shell.getQualifiedBinPath(WINUTILS);
+      log.debug("winutils is at {}", path);
+    } catch (IOException e) {
+      errorText.append("No " + WINUTILS);
+      log.warn("No winutils: {}", e, e);
+    }
+    try {
+      File target = new File("target");
+      FileUtil.canRead(target);
+    } catch (UnsatisfiedLinkError e) {
+      log.warn("Failing to link to native IO methods: {}", e, e);
+      errorText.append("No native IO methods");
+    }
+    return errorText.toString();
+  }
+
+  /**
+   * Strictly verify that windows utils is present.
+   * Checks go as far as opening the file and looking for
+   * the headers. 
+   * @throws IOException on any problem reading the file
+   * @throws FileNotFoundException if the file is not considered valid
+   */
+  public static void maybeVerifyWinUtilsValid() throws
+      IOException,
+      SliderException {
+    String errorText = SliderUtils.checkForRequiredNativeLibraries();
+    if (!errorText.isEmpty()) {
+      throw new BadClusterStateException(errorText);
+    }
+  }
+
+  public static void verifyIsFile(String program, File exe) throws
+      FileNotFoundException {
+    if (!exe.isFile()) {
+      throw new FileNotFoundException(program
+                                      + " at " + exe
+                                      + " is not a file");
+
+    }
+  }
+
+  public static void verifyFileSize(String program,
+      File exe,
+      int minFileSize) throws FileNotFoundException {
+    if (exe.length() < minFileSize) {
+      throw new FileNotFoundException(program
+                                      + " at " + exe
+                                      + " is too short to be an executable");
+    }
+  }
+
+  /**
+   * Look for the windows executable and check it has the right headers.
+   * <code>File.canRead()</code> doesn't work on windows, so the reading
+   * is mandatory.
+   *
+   * @param program program name for errors
+   * @param exe executable
+   * @throws IOException IOE
+   */
+  public static void verifyWindowsExe(String program, File exe)
+      throws IOException {
+    verifyIsFile(program, exe);
+
+    verifyFileSize(program, exe, 0x100);
+
+    // now read two bytes and verify the header.
+    try(FileReader reader = new FileReader(exe)) {
+      int[] header = new int[2];
+      header[0] = reader.read();
+      header[1] = reader.read();
+      if ((header[0] != 'M' || header[1] != 'Z')) {
+        throw new FileNotFoundException(program
+                                        + " at " + exe
+                                        + " is not a windows executable file");
+      }
+    }
+  }
+
+  /**
+   * Verify that a Unix exe works
+   * @param program program name for errors
+   * @param exe executable
+   * @throws IOException IOE
+
+   */
+  public static void verifyUnixExe(String program, File exe)
+      throws IOException {
+    verifyIsFile(program, exe);
+
+    // read flag
+    if (!exe.canRead()) {
+      throw new IOException("Cannot read " + program + " at " + exe);
+    }
+    // exe flag
+    if (!exe.canExecute()) {
+      throw new IOException("Cannot execute " + program + " at " + exe);
+    }
+  }
+
+  /**
+   * Validate an executable
+   * @param program program name for errors
+   * @param exe program to look at
+   * @throws IOException
+   */
+  public static void validateExe(String program, File exe) throws IOException {
+    if (!Shell.WINDOWS) {
+      verifyWindowsExe(program, exe);
+    } else {
+      verifyUnixExe(program, exe);
+    }
+  }
+
+  /**
+   * Write bytes to a file
+   * @param outfile output file
+   * @param data data to write
+   * @param createParent flag to indicate that the parent dir should
+   * be created
+   * @throws IOException on any IO problem
+   */
+  public static void write(File outfile, byte[] data, boolean createParent)
+      throws IOException {
+    File parentDir = outfile.getCanonicalFile().getParentFile();
+    if (parentDir == null) {
+      throw new IOException(outfile.getPath() + " has no parent dir");
+    }
+    if (createParent) {
+      parentDir.mkdirs();
+    }
+    SliderUtils.verifyIsDir(parentDir, log);
+    try(FileOutputStream out = new FileOutputStream(outfile)) {
+      out.write(data);
+    }
+
+  }
+
+  /**
+   * Execute a command for a test operation
+   * @param name name in error
+   * @param status status code expected
+   * @param timeoutMillis timeout in millis for process to finish
+   * @param logger
+   * @param outputString optional string to grep for (must not span a line)
+   * @param commands commands   @return the process
+   * @throws IOException on any failure.
+   */
+  public static ForkedProcessService execCommand(String name,
+      int status,
+      long timeoutMillis,
+      Logger logger,
+      String outputString,
+      String... commands) throws IOException, SliderException {
+    Preconditions.checkArgument(isSet(name), "no name");
+    Preconditions.checkArgument(commands.length > 0, "no commands");
+    Preconditions.checkArgument(isSet(commands[0]), "empty command");
+
+    ForkedProcessService process;
+
+
+    process = new ForkedProcessService(
+        name,
+        new HashMap<String, String>(),
+        Arrays.asList(commands));
+    process.setProcessLog(logger);
+    process.init(new Configuration());
+    String errorText = null;
+    process.start();
+    try {
+      if (!process.waitForServiceToStop(timeoutMillis)) {
+        throw new TimeoutException(
+            "Process did not stop in " + timeoutMillis + "mS");
+      }
+      int exitCode = process.getExitCode();
+      List<String> recentOutput = process.getRecentOutput();
+      if (status != exitCode) {
+        // error condition
+        errorText = "Expected exit code={" + status + "}, "
+                    + "actual exit code={" + exitCode + "}";
+      } else {
+        if (isSet(outputString)) {
+          boolean found = false;
+          for (String line : recentOutput) {
+            if (line.contains(outputString)) {
+              found = true;
+              break;
+            }
+          }
+          if (!found) {
+            errorText = "Did not find \"" + outputString + "\""
+                        + " in output";
+          }
+        }
+      }
+      if (errorText == null) {
+        return process;
+      }
+
+    } catch (TimeoutException e) {
+      errorText = e.toString();
+    }
+    // error text: non null ==> operation failed
+    log.warn(errorText);
+    List<String> recentOutput = process.getRecentOutput();
+    for (String line : recentOutput) {
+      log.info(line);
+    }
+    throw new SliderException(LauncherExitCodes.EXIT_OTHER_FAILURE,
+        "Process %s failed: %s", name, errorText);
+
+  }
+
+
+  /**
+   * Validate the slider client-side execution environment.
+   * This looks for everything felt to be critical for execution, including
+   * native binaries and other essential dependencies.
+   * @param logger logger to log to on normal execution
+   * @throws IOException on IO failures
+   * @throws SliderException on validation failures
+   */
+  public static void validateSliderClientEnvironment(Logger logger) throws
+      IOException,
+      SliderException {
+    maybeVerifyWinUtilsValid();
+  }
+
+  /**
+   * Validate the slider server-side execution environment.
+   * This looks for everything felt to be critical for execution, including
+   * native binaries and other essential dependencies.
+   * @param logger logger to log to on normal execution
+   * @param dependencyChecks flag to indicate checks for agent dependencies
+   * @throws IOException on IO failures
+   * @throws SliderException on validation failures
+   */
+  public static void validateSliderServerEnvironment(Logger logger,
+      boolean dependencyChecks) throws
+      IOException,
+      SliderException {
+    maybeVerifyWinUtilsValid();
+    if (dependencyChecks) {
+      validatePythonEnv(logger);
+      validateOpenSSLEnv(logger);
+    }
+  }
+
+  public static void validateOpenSSLEnv(Logger logger) throws
+      IOException,
+      SliderException {
+    execCommand(OPENSSL, 0, 5000, logger, "OpenSSL", OPENSSL, "version");
+  }
+
+  public static void validatePythonEnv(Logger logger) throws
+      IOException,
+      SliderException {
+    execCommand(PYTHON, 0, 5000, logger, "Python", PYTHON, "-V");
+  }
+
+  /**
+   * return the path to the currently running slider command
+   *
+   * @throws NullPointerException
+   *             - If the pathname argument is null
+   * @throws SecurityException
+   *             - if a security manager exists and its checkPermission method
+   *             doesn't allow getting the ProtectionDomain
+   */
+  public static String getCurrentCommandPath() {
+    File f = new File(Slider.class.getProtectionDomain().getCodeSource()
+                                  .getLocation().getPath());
+    return f.getAbsolutePath();
+  }
+
+  /**
+   * return the HDFS path where the application package has been uploaded
+   * manually or by using slider client (install package command)
+   * 
+   * @param conf configuration
+   * @return
+   */
+  public static String getApplicationDefinitionPath(ConfTreeOperations conf)
+      throws BadConfigException {
+    String appDefPath = conf.getGlobalOptions().getMandatoryOption(
+        AgentKeys.APP_DEF);
+    return appDefPath;
+  }
+
+  /**
+   * return the path to the slider-client.xml used by the current running
+   * slider command
+   *
+   * @throws SecurityException
+   *             - if a security manager exists and its checkPermission method
+   *             denies access to the class loader for the class
+   */
+  public static String getClientConfigPath() {
+    URL path = ConfigHelper.class.getClassLoader().getResource(
+        SliderKeys.SLIDER_CLIENT_XML);
+    Preconditions.checkNotNull(path, "Failed to locate resource " + SliderKeys.SLIDER_CLIENT_XML);
+    return path.toString();
+  }
+
+  /**
+   * validate if slider-client.xml under the path can be opened
+   *
+   * @throws IOException
+   *             : the file can't be found or open
+   */
+  public static void validateClientConfigFile() throws IOException {
+    URL resURL = SliderVersionInfo.class.getClassLoader().getResource(
+        SliderKeys.SLIDER_CLIENT_XML);
+    if (resURL == null) {
+      throw new IOException(
+          "slider-client.xml doesn't exist on the path: "
+          + getClientConfigPath());
+    }
+
+    try {
+      InputStream inStream = resURL.openStream();
+      if (inStream == null) {
+        throw new IOException("slider-client.xml can't be opened");
+      }
+    } catch (IOException e) {
+      throw new IOException("slider-client.xml can't be opened: "
+                            + e.toString());
+    }
+  }
+
+  /**
+   * validate if a file on HDFS can be open
+   *
+   * @throws IOException the file can't be found or opened
+   * @throws URISyntaxException
+   */
+  public static void validateHDFSFile(SliderFileSystem sliderFileSystem,
+      String pathStr)
+      throws IOException, URISyntaxException {
+    try(InputStream inputStream =
+            sliderFileSystem.getFileSystem().open(new Path(new URI(pathStr)))) {
+      if (inputStream == null) {
+        throw new IOException("HDFS file " + pathStr + " can't be opened");
+      }
+    }
+  }
+
+  /**
+   * return the version and path of the JDK invoking the current running
+   * slider command
+   *
+   * @throws SecurityException
+   *             - if a security manager exists and its checkPropertyAccess
+   *             method doesn't allow access to the specified system property.
+   */
+  public static String getJDKInfo() {
+    String version = System.getProperty("java.version");
+    String javaHome = System.getProperty("java.home");
+    return
+        "The version of the JDK invoking the current running slider command: "
+        + version + "; The path to it is: " + javaHome;
+  }
+
+  /**
+   * return a description of whether the current user has created credential
+   * cache files from kerberos servers
+   *
+   * @throws IOException
+   * @throws BadConfigException
+   * @throws SecurityException
+   *             - if a security manager exists and its checkPropertyAccess
+   *             method doesn't allow access to the specified system property.
+   */
+  public static String checkCredentialCacheFile() throws IOException,
+      BadConfigException {
+    String result = null;
+    if (!Shell.WINDOWS) {
+      result = Shell.execCommand("klist");
+    }
+    return result;
+  }
+
+  /**
+   * Compare the times of two applications: most recent app comes first
+   * Specifically: the one whose start time value is greater.
+   */
+  private static class MostRecentlyStartedAppFirst
+      implements Comparator<ApplicationReport>, Serializable {
+    @Override
+    public int compare(ApplicationReport r1, ApplicationReport r2) {
+      long x = r1.getStartTime();
+      long y = r2.getStartTime();
+      return compareTwoLongsReverse(x, y);
+    }
+  }
+  
+  /**
+   * Compare the times of two applications: most recent app comes first.
+   * "Recent"== the app whose start time <i>or finish time</i> is the greatest.
+   */
+  private static class MostRecentlyStartedOrFinishedFirst
+      implements Comparator<ApplicationReport>, Serializable {
+    @Override
+    public int compare(ApplicationReport r1, ApplicationReport r2) {
+      long started1 = r1.getStartTime();
+      long started2 = r2.getStartTime();
+      long finished1 = r1.getFinishTime();
+      long finished2 = r2.getFinishTime();
+      long lastEvent1 = Math.max(started1, finished1);
+      long lastEvent2 = Math.max(started2, finished2);
+      return compareTwoLongsReverse(lastEvent1, lastEvent2);
+    }
+  }
+
+  /**
+   * Compare the times of two applications: most recently finished app comes first
+   * Specifically: the one whose finish time value is greater.
+   */
+  private static class MostRecentAppFinishFirst
+      implements Comparator<ApplicationReport>, Serializable {
+    @Override
+    public int compare(ApplicationReport r1, ApplicationReport r2) {
+      long x = r1.getFinishTime();
+      long y = r2.getFinishTime();
+      return compareTwoLongsReverse(x, y);
+    }
+  }
+
+  /**
+   * Compare two long values for sorting. As the return value for 
+   * comparators must be int, the simple value of <code>x-y</code>
+   * is inapplicable
+   * @param x x value
+   * @param y y value
+   * @return +ve if x is less than y, -ve if y is greater than x; 0 for equality
+   */
+  public static int compareTwoLongsReverse(long x, long y) {
+    return (x < y) ? 1 : ((x == y) ? 0 : -1);
+  }
+
+  public static String getSystemEnv(String property) {
+    return System.getenv(property);
+  }
+
+  public static Map<String, String> getSystemEnv() {
+    return System.getenv();
+  }
+
+  public static String requestToString(AMRMClient.ContainerRequest request) {
+    Preconditions.checkArgument(request != null, "Null request");
+    StringBuilder buffer = new StringBuilder(request.toString());
+    buffer.append("; ");
+    buffer.append("relaxLocality=").append(request.getRelaxLocality()).append("; ");
+    String labels = request.getNodeLabelExpression();
+    if (labels != null) {
+      buffer.append("nodeLabels=").append(labels).append("; ");
+    }
+    List<String> nodes = request.getNodes();
+    if (nodes != null) {
+      buffer.append("Nodes = [ ");
+      int size = nodes.size();
+      for (int i = 0; i < Math.min(NODE_LIST_LIMIT, size); i++) {
+        buffer.append(nodes.get(i)).append(' ');
+      }
+      if (size > NODE_LIST_LIMIT) {
+        buffer.append(String.format("...(total %d entries)", size));
+      }
+      buffer.append("]; ");
+    }
+    List<String> racks = request.getRacks();
+    if (racks != null) {
+      buffer.append("racks = [")
+          .append(join(racks, ", ", false))
+          .append("]; ");
+    }
+    return buffer.toString();
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[72/76] [abbrv] hadoop git commit: YARN-5961. Generate native services protobuf classes during build. Contributed by Billie Rinaldi

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc259d50/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/proto/Messages.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/proto/Messages.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/proto/Messages.java
deleted file mode 100644
index 373d64d..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/api/proto/Messages.java
+++ /dev/null
@@ -1,34473 +0,0 @@
-// Generated by the protocol buffer compiler.  DO NOT EDIT!
-// source: SliderClusterMessages.proto
-
-package org.apache.slider.api.proto;
-
-public final class Messages {
-  private Messages() {}
-  public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
-  }
-  public interface RoleInstanceStateOrBuilder
-      extends com.google.protobuf.MessageOrBuilder {
-
-    // required string name = 1;
-    /**
-     * <code>required string name = 1;</code>
-     */
-    boolean hasName();
-    /**
-     * <code>required string name = 1;</code>
-     */
-    java.lang.String getName();
-    /**
-     * <code>required string name = 1;</code>
-     */
-    com.google.protobuf.ByteString
-        getNameBytes();
-
-    // optional string role = 2;
-    /**
-     * <code>optional string role = 2;</code>
-     */
-    boolean hasRole();
-    /**
-     * <code>optional string role = 2;</code>
-     */
-    java.lang.String getRole();
-    /**
-     * <code>optional string role = 2;</code>
-     */
-    com.google.protobuf.ByteString
-        getRoleBytes();
-
-    // required uint32 state = 4;
-    /**
-     * <code>required uint32 state = 4;</code>
-     */
-    boolean hasState();
-    /**
-     * <code>required uint32 state = 4;</code>
-     */
-    int getState();
-
-    // required uint32 exitCode = 5;
-    /**
-     * <code>required uint32 exitCode = 5;</code>
-     */
-    boolean hasExitCode();
-    /**
-     * <code>required uint32 exitCode = 5;</code>
-     */
-    int getExitCode();
-
-    // optional string command = 6;
-    /**
-     * <code>optional string command = 6;</code>
-     */
-    boolean hasCommand();
-    /**
-     * <code>optional string command = 6;</code>
-     */
-    java.lang.String getCommand();
-    /**
-     * <code>optional string command = 6;</code>
-     */
-    com.google.protobuf.ByteString
-        getCommandBytes();
-
-    // optional string diagnostics = 7;
-    /**
-     * <code>optional string diagnostics = 7;</code>
-     */
-    boolean hasDiagnostics();
-    /**
-     * <code>optional string diagnostics = 7;</code>
-     */
-    java.lang.String getDiagnostics();
-    /**
-     * <code>optional string diagnostics = 7;</code>
-     */
-    com.google.protobuf.ByteString
-        getDiagnosticsBytes();
-
-    // repeated string output = 8;
-    /**
-     * <code>repeated string output = 8;</code>
-     */
-    java.util.List<java.lang.String>
-    getOutputList();
-    /**
-     * <code>repeated string output = 8;</code>
-     */
-    int getOutputCount();
-    /**
-     * <code>repeated string output = 8;</code>
-     */
-    java.lang.String getOutput(int index);
-    /**
-     * <code>repeated string output = 8;</code>
-     */
-    com.google.protobuf.ByteString
-        getOutputBytes(int index);
-
-    // repeated string environment = 9;
-    /**
-     * <code>repeated string environment = 9;</code>
-     */
-    java.util.List<java.lang.String>
-    getEnvironmentList();
-    /**
-     * <code>repeated string environment = 9;</code>
-     */
-    int getEnvironmentCount();
-    /**
-     * <code>repeated string environment = 9;</code>
-     */
-    java.lang.String getEnvironment(int index);
-    /**
-     * <code>repeated string environment = 9;</code>
-     */
-    com.google.protobuf.ByteString
-        getEnvironmentBytes(int index);
-
-    // required uint32 roleId = 10;
-    /**
-     * <code>required uint32 roleId = 10;</code>
-     */
-    boolean hasRoleId();
-    /**
-     * <code>required uint32 roleId = 10;</code>
-     */
-    int getRoleId();
-
-    // required bool released = 11;
-    /**
-     * <code>required bool released = 11;</code>
-     */
-    boolean hasReleased();
-    /**
-     * <code>required bool released = 11;</code>
-     */
-    boolean getReleased();
-
-    // required int64 createTime = 12;
-    /**
-     * <code>required int64 createTime = 12;</code>
-     */
-    boolean hasCreateTime();
-    /**
-     * <code>required int64 createTime = 12;</code>
-     */
-    long getCreateTime();
-
-    // required int64 startTime = 13;
-    /**
-     * <code>required int64 startTime = 13;</code>
-     */
-    boolean hasStartTime();
-    /**
-     * <code>required int64 startTime = 13;</code>
-     */
-    long getStartTime();
-
-    // required string host = 14;
-    /**
-     * <code>required string host = 14;</code>
-     */
-    boolean hasHost();
-    /**
-     * <code>required string host = 14;</code>
-     */
-    java.lang.String getHost();
-    /**
-     * <code>required string host = 14;</code>
-     */
-    com.google.protobuf.ByteString
-        getHostBytes();
-
-    // required string hostURL = 15;
-    /**
-     * <code>required string hostURL = 15;</code>
-     */
-    boolean hasHostURL();
-    /**
-     * <code>required string hostURL = 15;</code>
-     */
-    java.lang.String getHostURL();
-    /**
-     * <code>required string hostURL = 15;</code>
-     */
-    com.google.protobuf.ByteString
-        getHostURLBytes();
-
-    // optional string appVersion = 16;
-    /**
-     * <code>optional string appVersion = 16;</code>
-     */
-    boolean hasAppVersion();
-    /**
-     * <code>optional string appVersion = 16;</code>
-     */
-    java.lang.String getAppVersion();
-    /**
-     * <code>optional string appVersion = 16;</code>
-     */
-    com.google.protobuf.ByteString
-        getAppVersionBytes();
-  }
-  /**
-   * Protobuf type {@code org.apache.slider.api.RoleInstanceState}
-   */
-  public static final class RoleInstanceState extends
-      com.google.protobuf.GeneratedMessage
-      implements RoleInstanceStateOrBuilder {
-    // Use RoleInstanceState.newBuilder() to construct.
-    private RoleInstanceState(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-      super(builder);
-      this.unknownFields = builder.getUnknownFields();
-    }
-    private RoleInstanceState(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
-    private static final RoleInstanceState defaultInstance;
-    public static RoleInstanceState getDefaultInstance() {
-      return defaultInstance;
-    }
-
-    public RoleInstanceState getDefaultInstanceForType() {
-      return defaultInstance;
-    }
-
-    private final com.google.protobuf.UnknownFieldSet unknownFields;
-    @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
-        getUnknownFields() {
-      return this.unknownFields;
-    }
-    private RoleInstanceState(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      initFields();
-      int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
-      try {
-        boolean done = false;
-        while (!done) {
-          int tag = input.readTag();
-          switch (tag) {
-            case 0:
-              done = true;
-              break;
-            default: {
-              if (!parseUnknownField(input, unknownFields,
-                                     extensionRegistry, tag)) {
-                done = true;
-              }
-              break;
-            }
-            case 10: {
-              bitField0_ |= 0x00000001;
-              name_ = input.readBytes();
-              break;
-            }
-            case 18: {
-              bitField0_ |= 0x00000002;
-              role_ = input.readBytes();
-              break;
-            }
-            case 32: {
-              bitField0_ |= 0x00000004;
-              state_ = input.readUInt32();
-              break;
-            }
-            case 40: {
-              bitField0_ |= 0x00000008;
-              exitCode_ = input.readUInt32();
-              break;
-            }
-            case 50: {
-              bitField0_ |= 0x00000010;
-              command_ = input.readBytes();
-              break;
-            }
-            case 58: {
-              bitField0_ |= 0x00000020;
-              diagnostics_ = input.readBytes();
-              break;
-            }
-            case 66: {
-              if (!((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
-                output_ = new com.google.protobuf.LazyStringArrayList();
-                mutable_bitField0_ |= 0x00000040;
-              }
-              output_.add(input.readBytes());
-              break;
-            }
-            case 74: {
-              if (!((mutable_bitField0_ & 0x00000080) == 0x00000080)) {
-                environment_ = new com.google.protobuf.LazyStringArrayList();
-                mutable_bitField0_ |= 0x00000080;
-              }
-              environment_.add(input.readBytes());
-              break;
-            }
-            case 80: {
-              bitField0_ |= 0x00000040;
-              roleId_ = input.readUInt32();
-              break;
-            }
-            case 88: {
-              bitField0_ |= 0x00000080;
-              released_ = input.readBool();
-              break;
-            }
-            case 96: {
-              bitField0_ |= 0x00000100;
-              createTime_ = input.readInt64();
-              break;
-            }
-            case 104: {
-              bitField0_ |= 0x00000200;
-              startTime_ = input.readInt64();
-              break;
-            }
-            case 114: {
-              bitField0_ |= 0x00000400;
-              host_ = input.readBytes();
-              break;
-            }
-            case 122: {
-              bitField0_ |= 0x00000800;
-              hostURL_ = input.readBytes();
-              break;
-            }
-            case 130: {
-              bitField0_ |= 0x00001000;
-              appVersion_ = input.readBytes();
-              break;
-            }
-          }
-        }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
-            e.getMessage()).setUnfinishedMessage(this);
-      } finally {
-        if (((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
-          output_ = new com.google.protobuf.UnmodifiableLazyStringList(output_);
-        }
-        if (((mutable_bitField0_ & 0x00000080) == 0x00000080)) {
-          environment_ = new com.google.protobuf.UnmodifiableLazyStringList(environment_);
-        }
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    public static final com.google.protobuf.Descriptors.Descriptor
-        getDescriptor() {
-      return org.apache.slider.api.proto.Messages.internal_static_org_apache_slider_api_RoleInstanceState_descriptor;
-    }
-
-    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-        internalGetFieldAccessorTable() {
-      return org.apache.slider.api.proto.Messages.internal_static_org_apache_slider_api_RoleInstanceState_fieldAccessorTable
-          .ensureFieldAccessorsInitialized(
-              org.apache.slider.api.proto.Messages.RoleInstanceState.class, org.apache.slider.api.proto.Messages.RoleInstanceState.Builder.class);
-    }
-
-    public static com.google.protobuf.Parser<RoleInstanceState> PARSER =
-        new com.google.protobuf.AbstractParser<RoleInstanceState>() {
-      public RoleInstanceState parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return new RoleInstanceState(input, extensionRegistry);
-      }
-    };
-
-    @java.lang.Override
-    public com.google.protobuf.Parser<RoleInstanceState> getParserForType() {
-      return PARSER;
-    }
-
-    private int bitField0_;
-    // required string name = 1;
-    public static final int NAME_FIELD_NUMBER = 1;
-    private java.lang.Object name_;
-    /**
-     * <code>required string name = 1;</code>
-     */
-    public boolean hasName() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>required string name = 1;</code>
-     */
-    public java.lang.String getName() {
-      java.lang.Object ref = name_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          name_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>required string name = 1;</code>
-     */
-    public com.google.protobuf.ByteString
-        getNameBytes() {
-      java.lang.Object ref = name_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        name_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    // optional string role = 2;
-    public static final int ROLE_FIELD_NUMBER = 2;
-    private java.lang.Object role_;
-    /**
-     * <code>optional string role = 2;</code>
-     */
-    public boolean hasRole() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
-    }
-    /**
-     * <code>optional string role = 2;</code>
-     */
-    public java.lang.String getRole() {
-      java.lang.Object ref = role_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          role_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>optional string role = 2;</code>
-     */
-    public com.google.protobuf.ByteString
-        getRoleBytes() {
-      java.lang.Object ref = role_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        role_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    // required uint32 state = 4;
-    public static final int STATE_FIELD_NUMBER = 4;
-    private int state_;
-    /**
-     * <code>required uint32 state = 4;</code>
-     */
-    public boolean hasState() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
-    }
-    /**
-     * <code>required uint32 state = 4;</code>
-     */
-    public int getState() {
-      return state_;
-    }
-
-    // required uint32 exitCode = 5;
-    public static final int EXITCODE_FIELD_NUMBER = 5;
-    private int exitCode_;
-    /**
-     * <code>required uint32 exitCode = 5;</code>
-     */
-    public boolean hasExitCode() {
-      return ((bitField0_ & 0x00000008) == 0x00000008);
-    }
-    /**
-     * <code>required uint32 exitCode = 5;</code>
-     */
-    public int getExitCode() {
-      return exitCode_;
-    }
-
-    // optional string command = 6;
-    public static final int COMMAND_FIELD_NUMBER = 6;
-    private java.lang.Object command_;
-    /**
-     * <code>optional string command = 6;</code>
-     */
-    public boolean hasCommand() {
-      return ((bitField0_ & 0x00000010) == 0x00000010);
-    }
-    /**
-     * <code>optional string command = 6;</code>
-     */
-    public java.lang.String getCommand() {
-      java.lang.Object ref = command_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          command_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>optional string command = 6;</code>
-     */
-    public com.google.protobuf.ByteString
-        getCommandBytes() {
-      java.lang.Object ref = command_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        command_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    // optional string diagnostics = 7;
-    public static final int DIAGNOSTICS_FIELD_NUMBER = 7;
-    private java.lang.Object diagnostics_;
-    /**
-     * <code>optional string diagnostics = 7;</code>
-     */
-    public boolean hasDiagnostics() {
-      return ((bitField0_ & 0x00000020) == 0x00000020);
-    }
-    /**
-     * <code>optional string diagnostics = 7;</code>
-     */
-    public java.lang.String getDiagnostics() {
-      java.lang.Object ref = diagnostics_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          diagnostics_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>optional string diagnostics = 7;</code>
-     */
-    public com.google.protobuf.ByteString
-        getDiagnosticsBytes() {
-      java.lang.Object ref = diagnostics_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        diagnostics_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    // repeated string output = 8;
-    public static final int OUTPUT_FIELD_NUMBER = 8;
-    private com.google.protobuf.LazyStringList output_;
-    /**
-     * <code>repeated string output = 8;</code>
-     */
-    public java.util.List<java.lang.String>
-        getOutputList() {
-      return output_;
-    }
-    /**
-     * <code>repeated string output = 8;</code>
-     */
-    public int getOutputCount() {
-      return output_.size();
-    }
-    /**
-     * <code>repeated string output = 8;</code>
-     */
-    public java.lang.String getOutput(int index) {
-      return output_.get(index);
-    }
-    /**
-     * <code>repeated string output = 8;</code>
-     */
-    public com.google.protobuf.ByteString
-        getOutputBytes(int index) {
-      return output_.getByteString(index);
-    }
-
-    // repeated string environment = 9;
-    public static final int ENVIRONMENT_FIELD_NUMBER = 9;
-    private com.google.protobuf.LazyStringList environment_;
-    /**
-     * <code>repeated string environment = 9;</code>
-     */
-    public java.util.List<java.lang.String>
-        getEnvironmentList() {
-      return environment_;
-    }
-    /**
-     * <code>repeated string environment = 9;</code>
-     */
-    public int getEnvironmentCount() {
-      return environment_.size();
-    }
-    /**
-     * <code>repeated string environment = 9;</code>
-     */
-    public java.lang.String getEnvironment(int index) {
-      return environment_.get(index);
-    }
-    /**
-     * <code>repeated string environment = 9;</code>
-     */
-    public com.google.protobuf.ByteString
-        getEnvironmentBytes(int index) {
-      return environment_.getByteString(index);
-    }
-
-    // required uint32 roleId = 10;
-    public static final int ROLEID_FIELD_NUMBER = 10;
-    private int roleId_;
-    /**
-     * <code>required uint32 roleId = 10;</code>
-     */
-    public boolean hasRoleId() {
-      return ((bitField0_ & 0x00000040) == 0x00000040);
-    }
-    /**
-     * <code>required uint32 roleId = 10;</code>
-     */
-    public int getRoleId() {
-      return roleId_;
-    }
-
-    // required bool released = 11;
-    public static final int RELEASED_FIELD_NUMBER = 11;
-    private boolean released_;
-    /**
-     * <code>required bool released = 11;</code>
-     */
-    public boolean hasReleased() {
-      return ((bitField0_ & 0x00000080) == 0x00000080);
-    }
-    /**
-     * <code>required bool released = 11;</code>
-     */
-    public boolean getReleased() {
-      return released_;
-    }
-
-    // required int64 createTime = 12;
-    public static final int CREATETIME_FIELD_NUMBER = 12;
-    private long createTime_;
-    /**
-     * <code>required int64 createTime = 12;</code>
-     */
-    public boolean hasCreateTime() {
-      return ((bitField0_ & 0x00000100) == 0x00000100);
-    }
-    /**
-     * <code>required int64 createTime = 12;</code>
-     */
-    public long getCreateTime() {
-      return createTime_;
-    }
-
-    // required int64 startTime = 13;
-    public static final int STARTTIME_FIELD_NUMBER = 13;
-    private long startTime_;
-    /**
-     * <code>required int64 startTime = 13;</code>
-     */
-    public boolean hasStartTime() {
-      return ((bitField0_ & 0x00000200) == 0x00000200);
-    }
-    /**
-     * <code>required int64 startTime = 13;</code>
-     */
-    public long getStartTime() {
-      return startTime_;
-    }
-
-    // required string host = 14;
-    public static final int HOST_FIELD_NUMBER = 14;
-    private java.lang.Object host_;
-    /**
-     * <code>required string host = 14;</code>
-     */
-    public boolean hasHost() {
-      return ((bitField0_ & 0x00000400) == 0x00000400);
-    }
-    /**
-     * <code>required string host = 14;</code>
-     */
-    public java.lang.String getHost() {
-      java.lang.Object ref = host_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          host_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>required string host = 14;</code>
-     */
-    public com.google.protobuf.ByteString
-        getHostBytes() {
-      java.lang.Object ref = host_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        host_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    // required string hostURL = 15;
-    public static final int HOSTURL_FIELD_NUMBER = 15;
-    private java.lang.Object hostURL_;
-    /**
-     * <code>required string hostURL = 15;</code>
-     */
-    public boolean hasHostURL() {
-      return ((bitField0_ & 0x00000800) == 0x00000800);
-    }
-    /**
-     * <code>required string hostURL = 15;</code>
-     */
-    public java.lang.String getHostURL() {
-      java.lang.Object ref = hostURL_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          hostURL_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>required string hostURL = 15;</code>
-     */
-    public com.google.protobuf.ByteString
-        getHostURLBytes() {
-      java.lang.Object ref = hostURL_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        hostURL_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    // optional string appVersion = 16;
-    public static final int APPVERSION_FIELD_NUMBER = 16;
-    private java.lang.Object appVersion_;
-    /**
-     * <code>optional string appVersion = 16;</code>
-     */
-    public boolean hasAppVersion() {
-      return ((bitField0_ & 0x00001000) == 0x00001000);
-    }
-    /**
-     * <code>optional string appVersion = 16;</code>
-     */
-    public java.lang.String getAppVersion() {
-      java.lang.Object ref = appVersion_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          appVersion_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>optional string appVersion = 16;</code>
-     */
-    public com.google.protobuf.ByteString
-        getAppVersionBytes() {
-      java.lang.Object ref = appVersion_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        appVersion_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    private void initFields() {
-      name_ = "";
-      role_ = "";
-      state_ = 0;
-      exitCode_ = 0;
-      command_ = "";
-      diagnostics_ = "";
-      output_ = com.google.protobuf.LazyStringArrayList.EMPTY;
-      environment_ = com.google.protobuf.LazyStringArrayList.EMPTY;
-      roleId_ = 0;
-      released_ = false;
-      createTime_ = 0L;
-      startTime_ = 0L;
-      host_ = "";
-      hostURL_ = "";
-      appVersion_ = "";
-    }
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized != -1) return isInitialized == 1;
-
-      if (!hasName()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasState()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasExitCode()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasRoleId()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasReleased()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasCreateTime()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasStartTime()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasHost()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasHostURL()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      getSerializedSize();
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeBytes(1, getNameBytes());
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeBytes(2, getRoleBytes());
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeUInt32(4, state_);
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeUInt32(5, exitCode_);
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        output.writeBytes(6, getCommandBytes());
-      }
-      if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        output.writeBytes(7, getDiagnosticsBytes());
-      }
-      for (int i = 0; i < output_.size(); i++) {
-        output.writeBytes(8, output_.getByteString(i));
-      }
-      for (int i = 0; i < environment_.size(); i++) {
-        output.writeBytes(9, environment_.getByteString(i));
-      }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        output.writeUInt32(10, roleId_);
-      }
-      if (((bitField0_ & 0x00000080) == 0x00000080)) {
-        output.writeBool(11, released_);
-      }
-      if (((bitField0_ & 0x00000100) == 0x00000100)) {
-        output.writeInt64(12, createTime_);
-      }
-      if (((bitField0_ & 0x00000200) == 0x00000200)) {
-        output.writeInt64(13, startTime_);
-      }
-      if (((bitField0_ & 0x00000400) == 0x00000400)) {
-        output.writeBytes(14, getHostBytes());
-      }
-      if (((bitField0_ & 0x00000800) == 0x00000800)) {
-        output.writeBytes(15, getHostURLBytes());
-      }
-      if (((bitField0_ & 0x00001000) == 0x00001000)) {
-        output.writeBytes(16, getAppVersionBytes());
-      }
-      getUnknownFields().writeTo(output);
-    }
-
-    private int memoizedSerializedSize = -1;
-    public int getSerializedSize() {
-      int size = memoizedSerializedSize;
-      if (size != -1) return size;
-
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(1, getNameBytes());
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(2, getRoleBytes());
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt32Size(4, state_);
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt32Size(5, exitCode_);
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(6, getCommandBytes());
-      }
-      if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(7, getDiagnosticsBytes());
-      }
-      {
-        int dataSize = 0;
-        for (int i = 0; i < output_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
-            .computeBytesSizeNoTag(output_.getByteString(i));
-        }
-        size += dataSize;
-        size += 1 * getOutputList().size();
-      }
-      {
-        int dataSize = 0;
-        for (int i = 0; i < environment_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
-            .computeBytesSizeNoTag(environment_.getByteString(i));
-        }
-        size += dataSize;
-        size += 1 * getEnvironmentList().size();
-      }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt32Size(10, roleId_);
-      }
-      if (((bitField0_ & 0x00000080) == 0x00000080)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBoolSize(11, released_);
-      }
-      if (((bitField0_ & 0x00000100) == 0x00000100)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(12, createTime_);
-      }
-      if (((bitField0_ & 0x00000200) == 0x00000200)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(13, startTime_);
-      }
-      if (((bitField0_ & 0x00000400) == 0x00000400)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(14, getHostBytes());
-      }
-      if (((bitField0_ & 0x00000800) == 0x00000800)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(15, getHostURLBytes());
-      }
-      if (((bitField0_ & 0x00001000) == 0x00001000)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(16, getAppVersionBytes());
-      }
-      size += getUnknownFields().getSerializedSize();
-      memoizedSerializedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    protected java.lang.Object writeReplace()
-        throws java.io.ObjectStreamException {
-      return super.writeReplace();
-    }
-
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof org.apache.slider.api.proto.Messages.RoleInstanceState)) {
-        return super.equals(obj);
-      }
-      org.apache.slider.api.proto.Messages.RoleInstanceState other = (org.apache.slider.api.proto.Messages.RoleInstanceState) obj;
-
-      boolean result = true;
-      result = result && (hasName() == other.hasName());
-      if (hasName()) {
-        result = result && getName()
-            .equals(other.getName());
-      }
-      result = result && (hasRole() == other.hasRole());
-      if (hasRole()) {
-        result = result && getRole()
-            .equals(other.getRole());
-      }
-      result = result && (hasState() == other.hasState());
-      if (hasState()) {
-        result = result && (getState()
-            == other.getState());
-      }
-      result = result && (hasExitCode() == other.hasExitCode());
-      if (hasExitCode()) {
-        result = result && (getExitCode()
-            == other.getExitCode());
-      }
-      result = result && (hasCommand() == other.hasCommand());
-      if (hasCommand()) {
-        result = result && getCommand()
-            .equals(other.getCommand());
-      }
-      result = result && (hasDiagnostics() == other.hasDiagnostics());
-      if (hasDiagnostics()) {
-        result = result && getDiagnostics()
-            .equals(other.getDiagnostics());
-      }
-      result = result && getOutputList()
-          .equals(other.getOutputList());
-      result = result && getEnvironmentList()
-          .equals(other.getEnvironmentList());
-      result = result && (hasRoleId() == other.hasRoleId());
-      if (hasRoleId()) {
-        result = result && (getRoleId()
-            == other.getRoleId());
-      }
-      result = result && (hasReleased() == other.hasReleased());
-      if (hasReleased()) {
-        result = result && (getReleased()
-            == other.getReleased());
-      }
-      result = result && (hasCreateTime() == other.hasCreateTime());
-      if (hasCreateTime()) {
-        result = result && (getCreateTime()
-            == other.getCreateTime());
-      }
-      result = result && (hasStartTime() == other.hasStartTime());
-      if (hasStartTime()) {
-        result = result && (getStartTime()
-            == other.getStartTime());
-      }
-      result = result && (hasHost() == other.hasHost());
-      if (hasHost()) {
-        result = result && getHost()
-            .equals(other.getHost());
-      }
-      result = result && (hasHostURL() == other.hasHostURL());
-      if (hasHostURL()) {
-        result = result && getHostURL()
-            .equals(other.getHostURL());
-      }
-      result = result && (hasAppVersion() == other.hasAppVersion());
-      if (hasAppVersion()) {
-        result = result && getAppVersion()
-            .equals(other.getAppVersion());
-      }
-      result = result &&
-          getUnknownFields().equals(other.getUnknownFields());
-      return result;
-    }
-
-    private int memoizedHashCode = 0;
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
-      }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasName()) {
-        hash = (37 * hash) + NAME_FIELD_NUMBER;
-        hash = (53 * hash) + getName().hashCode();
-      }
-      if (hasRole()) {
-        hash = (37 * hash) + ROLE_FIELD_NUMBER;
-        hash = (53 * hash) + getRole().hashCode();
-      }
-      if (hasState()) {
-        hash = (37 * hash) + STATE_FIELD_NUMBER;
-        hash = (53 * hash) + getState();
-      }
-      if (hasExitCode()) {
-        hash = (37 * hash) + EXITCODE_FIELD_NUMBER;
-        hash = (53 * hash) + getExitCode();
-      }
-      if (hasCommand()) {
-        hash = (37 * hash) + COMMAND_FIELD_NUMBER;
-        hash = (53 * hash) + getCommand().hashCode();
-      }
-      if (hasDiagnostics()) {
-        hash = (37 * hash) + DIAGNOSTICS_FIELD_NUMBER;
-        hash = (53 * hash) + getDiagnostics().hashCode();
-      }
-      if (getOutputCount() > 0) {
-        hash = (37 * hash) + OUTPUT_FIELD_NUMBER;
-        hash = (53 * hash) + getOutputList().hashCode();
-      }
-      if (getEnvironmentCount() > 0) {
-        hash = (37 * hash) + ENVIRONMENT_FIELD_NUMBER;
-        hash = (53 * hash) + getEnvironmentList().hashCode();
-      }
-      if (hasRoleId()) {
-        hash = (37 * hash) + ROLEID_FIELD_NUMBER;
-        hash = (53 * hash) + getRoleId();
-      }
-      if (hasReleased()) {
-        hash = (37 * hash) + RELEASED_FIELD_NUMBER;
-        hash = (53 * hash) + hashBoolean(getReleased());
-      }
-      if (hasCreateTime()) {
-        hash = (37 * hash) + CREATETIME_FIELD_NUMBER;
-        hash = (53 * hash) + hashLong(getCreateTime());
-      }
-      if (hasStartTime()) {
-        hash = (37 * hash) + STARTTIME_FIELD_NUMBER;
-        hash = (53 * hash) + hashLong(getStartTime());
-      }
-      if (hasHost()) {
-        hash = (37 * hash) + HOST_FIELD_NUMBER;
-        hash = (53 * hash) + getHost().hashCode();
-      }
-      if (hasHostURL()) {
-        hash = (37 * hash) + HOSTURL_FIELD_NUMBER;
-        hash = (53 * hash) + getHostURL().hashCode();
-      }
-      if (hasAppVersion()) {
-        hash = (37 * hash) + APPVERSION_FIELD_NUMBER;
-        hash = (53 * hash) + getAppVersion().hashCode();
-      }
-      hash = (29 * hash) + getUnknownFields().hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
-
-    public static org.apache.slider.api.proto.Messages.RoleInstanceState parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.slider.api.proto.Messages.RoleInstanceState parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.slider.api.proto.Messages.RoleInstanceState parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.slider.api.proto.Messages.RoleInstanceState parseFrom(
-        byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.slider.api.proto.Messages.RoleInstanceState parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.apache.slider.api.proto.Messages.RoleInstanceState parseFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-    public static org.apache.slider.api.proto.Messages.RoleInstanceState parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input);
-    }
-    public static org.apache.slider.api.proto.Messages.RoleInstanceState parseDelimitedFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input, extensionRegistry);
-    }
-    public static org.apache.slider.api.proto.Messages.RoleInstanceState parseFrom(
-        com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.apache.slider.api.proto.Messages.RoleInstanceState parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-
-    public static Builder newBuilder() { return Builder.create(); }
-    public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.slider.api.proto.Messages.RoleInstanceState prototype) {
-      return newBuilder().mergeFrom(prototype);
-    }
-    public Builder toBuilder() { return newBuilder(this); }
-
-    @java.lang.Override
-    protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-      Builder builder = new Builder(parent);
-      return builder;
-    }
-    /**
-     * Protobuf type {@code org.apache.slider.api.RoleInstanceState}
-     */
-    public static final class Builder extends
-        com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.slider.api.proto.Messages.RoleInstanceStateOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return org.apache.slider.api.proto.Messages.internal_static_org_apache_slider_api_RoleInstanceState_descriptor;
-      }
-
-      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return org.apache.slider.api.proto.Messages.internal_static_org_apache_slider_api_RoleInstanceState_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                org.apache.slider.api.proto.Messages.RoleInstanceState.class, org.apache.slider.api.proto.Messages.RoleInstanceState.Builder.class);
-      }
-
-      // Construct using org.apache.slider.api.proto.Messages.RoleInstanceState.newBuilder()
-      private Builder() {
-        maybeForceBuilderInitialization();
-      }
-
-      private Builder(
-          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-        super(parent);
-        maybeForceBuilderInitialization();
-      }
-      private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-        }
-      }
-      private static Builder create() {
-        return new Builder();
-      }
-
-      public Builder clear() {
-        super.clear();
-        name_ = "";
-        bitField0_ = (bitField0_ & ~0x00000001);
-        role_ = "";
-        bitField0_ = (bitField0_ & ~0x00000002);
-        state_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000004);
-        exitCode_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000008);
-        command_ = "";
-        bitField0_ = (bitField0_ & ~0x00000010);
-        diagnostics_ = "";
-        bitField0_ = (bitField0_ & ~0x00000020);
-        output_ = com.google.protobuf.LazyStringArrayList.EMPTY;
-        bitField0_ = (bitField0_ & ~0x00000040);
-        environment_ = com.google.protobuf.LazyStringArrayList.EMPTY;
-        bitField0_ = (bitField0_ & ~0x00000080);
-        roleId_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000100);
-        released_ = false;
-        bitField0_ = (bitField0_ & ~0x00000200);
-        createTime_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000400);
-        startTime_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000800);
-        host_ = "";
-        bitField0_ = (bitField0_ & ~0x00001000);
-        hostURL_ = "";
-        bitField0_ = (bitField0_ & ~0x00002000);
-        appVersion_ = "";
-        bitField0_ = (bitField0_ & ~0x00004000);
-        return this;
-      }
-
-      public Builder clone() {
-        return create().mergeFrom(buildPartial());
-      }
-
-      public com.google.protobuf.Descriptors.Descriptor
-          getDescriptorForType() {
-        return org.apache.slider.api.proto.Messages.internal_static_org_apache_slider_api_RoleInstanceState_descriptor;
-      }
-
-      public org.apache.slider.api.proto.Messages.RoleInstanceState getDefaultInstanceForType() {
-        return org.apache.slider.api.proto.Messages.RoleInstanceState.getDefaultInstance();
-      }
-
-      public org.apache.slider.api.proto.Messages.RoleInstanceState build() {
-        org.apache.slider.api.proto.Messages.RoleInstanceState result = buildPartial();
-        if (!result.isInitialized()) {
-          throw newUninitializedMessageException(result);
-        }
-        return result;
-      }
-
-      public org.apache.slider.api.proto.Messages.RoleInstanceState buildPartial() {
-        org.apache.slider.api.proto.Messages.RoleInstanceState result = new org.apache.slider.api.proto.Messages.RoleInstanceState(this);
-        int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        result.name_ = name_;
-        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
-          to_bitField0_ |= 0x00000002;
-        }
-        result.role_ = role_;
-        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
-          to_bitField0_ |= 0x00000004;
-        }
-        result.state_ = state_;
-        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
-          to_bitField0_ |= 0x00000008;
-        }
-        result.exitCode_ = exitCode_;
-        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
-          to_bitField0_ |= 0x00000010;
-        }
-        result.command_ = command_;
-        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
-          to_bitField0_ |= 0x00000020;
-        }
-        result.diagnostics_ = diagnostics_;
-        if (((bitField0_ & 0x00000040) == 0x00000040)) {
-          output_ = new com.google.protobuf.UnmodifiableLazyStringList(
-              output_);
-          bitField0_ = (bitField0_ & ~0x00000040);
-        }
-        result.output_ = output_;
-        if (((bitField0_ & 0x00000080) == 0x00000080)) {
-          environment_ = new com.google.protobuf.UnmodifiableLazyStringList(
-              environment_);
-          bitField0_ = (bitField0_ & ~0x00000080);
-        }
-        result.environment_ = environment_;
-        if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
-          to_bitField0_ |= 0x00000040;
-        }
-        result.roleId_ = roleId_;
-        if (((from_bitField0_ & 0x00000200) == 0x00000200)) {
-          to_bitField0_ |= 0x00000080;
-        }
-        result.released_ = released_;
-        if (((from_bitField0_ & 0x00000400) == 0x00000400)) {
-          to_bitField0_ |= 0x00000100;
-        }
-        result.createTime_ = createTime_;
-        if (((from_bitField0_ & 0x00000800) == 0x00000800)) {
-          to_bitField0_ |= 0x00000200;
-        }
-        result.startTime_ = startTime_;
-        if (((from_bitField0_ & 0x00001000) == 0x00001000)) {
-          to_bitField0_ |= 0x00000400;
-        }
-        result.host_ = host_;
-        if (((from_bitField0_ & 0x00002000) == 0x00002000)) {
-          to_bitField0_ |= 0x00000800;
-        }
-        result.hostURL_ = hostURL_;
-        if (((from_bitField0_ & 0x00004000) == 0x00004000)) {
-          to_bitField0_ |= 0x00001000;
-        }
-        result.appVersion_ = appVersion_;
-        result.bitField0_ = to_bitField0_;
-        onBuilt();
-        return result;
-      }
-
-      public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.slider.api.proto.Messages.RoleInstanceState) {
-          return mergeFrom((org.apache.slider.api.proto.Messages.RoleInstanceState)other);
-        } else {
-          super.mergeFrom(other);
-          return this;
-        }
-      }
-
-      public Builder mergeFrom(org.apache.slider.api.proto.Messages.RoleInstanceState other) {
-        if (other == org.apache.slider.api.proto.Messages.RoleInstanceState.getDefaultInstance()) return this;
-        if (other.hasName()) {
-          bitField0_ |= 0x00000001;
-          name_ = other.name_;
-          onChanged();
-        }
-        if (other.hasRole()) {
-          bitField0_ |= 0x00000002;
-          role_ = other.role_;
-          onChanged();
-        }
-        if (other.hasState()) {
-          setState(other.getState());
-        }
-        if (other.hasExitCode()) {
-          setExitCode(other.getExitCode());
-        }
-        if (other.hasCommand()) {
-          bitField0_ |= 0x00000010;
-          command_ = other.command_;
-          onChanged();
-        }
-        if (other.hasDiagnostics()) {
-          bitField0_ |= 0x00000020;
-          diagnostics_ = other.diagnostics_;
-          onChanged();
-        }
-        if (!other.output_.isEmpty()) {
-          if (output_.isEmpty()) {
-            output_ = other.output_;
-            bitField0_ = (bitField0_ & ~0x00000040);
-          } else {
-            ensureOutputIsMutable();
-            output_.addAll(other.output_);
-          }
-          onChanged();
-        }
-        if (!other.environment_.isEmpty()) {
-          if (environment_.isEmpty()) {
-            environment_ = other.environment_;
-            bitField0_ = (bitField0_ & ~0x00000080);
-          } else {
-            ensureEnvironmentIsMutable();
-            environment_.addAll(other.environment_);
-          }
-          onChanged();
-        }
-        if (other.hasRoleId()) {
-          setRoleId(other.getRoleId());
-        }
-        if (other.hasReleased()) {
-          setReleased(other.getReleased());
-        }
-        if (other.hasCreateTime()) {
-          setCreateTime(other.getCreateTime());
-        }
-        if (other.hasStartTime()) {
-          setStartTime(other.getStartTime());
-        }
-        if (other.hasHost()) {
-          bitField0_ |= 0x00001000;
-          host_ = other.host_;
-          onChanged();
-        }
-        if (other.hasHostURL()) {
-          bitField0_ |= 0x00002000;
-          hostURL_ = other.hostURL_;
-          onChanged();
-        }
-        if (other.hasAppVersion()) {
-          bitField0_ |= 0x00004000;
-          appVersion_ = other.appVersion_;
-          onChanged();
-        }
-        this.mergeUnknownFields(other.getUnknownFields());
-        return this;
-      }
-
-      public final boolean isInitialized() {
-        if (!hasName()) {
-          
-          return false;
-        }
-        if (!hasState()) {
-          
-          return false;
-        }
-        if (!hasExitCode()) {
-          
-          return false;
-        }
-        if (!hasRoleId()) {
-          
-          return false;
-        }
-        if (!hasReleased()) {
-          
-          return false;
-        }
-        if (!hasCreateTime()) {
-          
-          return false;
-        }
-        if (!hasStartTime()) {
-          
-          return false;
-        }
-        if (!hasHost()) {
-          
-          return false;
-        }
-        if (!hasHostURL()) {
-          
-          return false;
-        }
-        return true;
-      }
-
-      public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        org.apache.slider.api.proto.Messages.RoleInstanceState parsedMessage = null;
-        try {
-          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.slider.api.proto.Messages.RoleInstanceState) e.getUnfinishedMessage();
-          throw e;
-        } finally {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
-          }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      // required string name = 1;
-      private java.lang.Object name_ = "";
-      /**
-       * <code>required string name = 1;</code>
-       */
-      public boolean hasName() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <code>required string name = 1;</code>
-       */
-      public java.lang.String getName() {
-        java.lang.Object ref = name_;
-        if (!(ref instanceof java.lang.String)) {
-          java.lang.String s = ((com.google.protobuf.ByteString) ref)
-              .toStringUtf8();
-          name_ = s;
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
-      }
-      /**
-       * <code>required string name = 1;</code>
-       */
-      public com.google.protobuf.ByteString
-          getNameBytes() {
-        java.lang.Object ref = name_;
-        if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          name_ = b;
-          return b;
-        } else {
-          return (com.google.protobuf.ByteString) ref;
-        }
-      }
-      /**
-       * <code>required string name = 1;</code>
-       */
-      public Builder setName(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000001;
-        name_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required string name = 1;</code>
-       */
-      public Builder clearName() {
-        bitField0_ = (bitField0_ & ~0x00000001);
-        name_ = getDefaultInstance().getName();
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required string name = 1;</code>
-       */
-      public Builder setNameBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000001;
-        name_ = value;
-        onChanged();
-        return this;
-      }
-
-      // optional string role = 2;
-      private java.lang.Object role_ = "";
-      /**
-       * <code>optional string role = 2;</code>
-       */
-      public boolean hasRole() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
-      }
-      /**
-       * <code>optional string role = 2;</code>
-       */
-      public java.lang.String getRole() {
-        java.lang.Object ref = role_;
-        if (!(ref instanceof java.lang.String)) {
-          java.lang.String s = ((com.google.protobuf.ByteString) ref)
-              .toStringUtf8();
-          role_ = s;
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
-      }
-      /**
-       * <code>optional string role = 2;</code>
-       */
-      public com.google.protobuf.ByteString
-          getRoleBytes() {
-        java.lang.Object ref = role_;
-        if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          role_ = b;
-          return b;
-        } else {
-          return (com.google.protobuf.ByteString) ref;
-        }
-      }
-      /**
-       * <code>optional string role = 2;</code>
-       */
-      public Builder setRole(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000002;
-        role_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string role = 2;</code>
-       */
-      public Builder clearRole() {
-        bitField0_ = (bitField0_ & ~0x00000002);
-        role_ = getDefaultInstance().getRole();
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string role = 2;</code>
-       */
-      public Builder setRoleBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000002;
-        role_ = value;
-        onChanged();
-        return this;
-      }
-
-      // required uint32 state = 4;
-      private int state_ ;
-      /**
-       * <code>required uint32 state = 4;</code>
-       */
-      public boolean hasState() {
-        return ((bitField0_ & 0x00000004) == 0x00000004);
-      }
-      /**
-       * <code>required uint32 state = 4;</code>
-       */
-      public int getState() {
-        return state_;
-      }
-      /**
-       * <code>required uint32 state = 4;</code>
-       */
-      public Builder setState(int value) {
-        bitField0_ |= 0x00000004;
-        state_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required uint32 state = 4;</code>
-       */
-      public Builder clearState() {
-        bitField0_ = (bitField0_ & ~0x00000004);
-        state_ = 0;
-        onChanged();
-        return this;
-      }
-
-      // required uint32 exitCode = 5;
-      private int exitCode_ ;
-      /**
-       * <code>required uint32 exitCode = 5;</code>
-       */
-      public boolean hasExitCode() {
-        return ((bitField0_ & 0x00000008) == 0x00000008);
-      }
-      /**
-       * <code>required uint32 exitCode = 5;</code>
-       */
-      public int getExitCode() {
-        return exitCode_;
-      }
-      /**
-       * <code>required uint32 exitCode = 5;</code>
-       */
-      public Builder setExitCode(int value) {
-        bitField0_ |= 0x00000008;
-        exitCode_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required uint32 exitCode = 5;</code>
-       */
-      public Builder clearExitCode() {
-        bitField0_ = (bitField0_ & ~0x00000008);
-        exitCode_ = 0;
-        onChanged();
-        return this;
-      }
-
-      // optional string command = 6;
-      private java.lang.Object command_ = "";
-      /**
-       * <code>optional string command = 6;</code>
-       */
-      public boolean hasCommand() {
-        return ((bitField0_ & 0x00000010) == 0x00000010);
-      }
-      /**
-       * <code>optional string command = 6;</code>
-       */
-      public java.lang.String getCommand() {
-        java.lang.Object ref = command_;
-        if (!(ref instanceof java.lang.String)) {
-          java.lang.String s = ((com.google.protobuf.ByteString) ref)
-              .toStringUtf8();
-          command_ = s;
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
-      }
-      /**
-       * <code>optional string command = 6;</code>
-       */
-      public com.google.protobuf.ByteString
-          getCommandBytes() {
-        java.lang.Object ref = command_;
-        if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          command_ = b;
-          return b;
-        } else {
-          return (com.google.protobuf.ByteString) ref;
-        }
-      }
-      /**
-       * <code>optional string command = 6;</code>
-       */
-      public Builder setCommand(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000010;
-        command_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string command = 6;</code>
-       */
-      public Builder clearCommand() {
-        bitField0_ = (bitField0_ & ~0x00000010);
-        command_ = getDefaultInstance().getCommand();
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string command = 6;</code>
-       */
-      public Builder setCommandBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000010;
-        command_ = value;
-        onChanged();
-        return this;
-      }
-
-      // optional string diagnostics = 7;
-      private java.lang.Object diagnostics_ = "";
-      /**
-       * <code>optional string diagnostics = 7;</code>
-       */
-      public boolean hasDiagnostics() {
-        return ((bitField0_ & 0x00000020) == 0x00000020);
-      }
-      /**
-       * <code>optional string diagnostics = 7;</code>
-       */
-      public java.lang.String getDiagnostics() {
-        java.lang.Object ref = diagnostics_;
-        if (!(ref instanceof java.lang.String)) {
-          java.lang.String s = ((com.google.protobuf.ByteString) ref)
-              .toStringUtf8();
-          diagnostics_ = s;
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
-      }
-      /**
-       * <code>optional string diagnostics = 7;</code>
-       */
-      public com.google.protobuf.ByteString
-          getDiagnosticsBytes() {
-        java.lang.Object ref = diagnostics_;
-        if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          diagnostics_ = b;
-          return b;
-        } else {
-          return (com.google.protobuf.ByteString) ref;
-        }
-      }
-      /**
-       * <code>optional string diagnostics = 7;</code>
-       */
-      public Builder setDiagnostics(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000020;
-        diagnostics_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string diagnostics = 7;</code>
-       */
-      public Builder clearDiagnostics() {
-        bitField0_ = (bitField0_ & ~0x00000020);
-        diagnostics_ = getDefaultInstance().getDiagnostics();
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string diagnostics = 7;</code>
-       */
-      public Builder setDiagnosticsBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000020;
-        diagnostics_ = value;
-        onChanged();
-        return this;
-      }
-
-      // repeated string output = 8;
-      private com.google.protobuf.LazyStringList output_ = com.google.protobuf.LazyStringArrayList.EMPTY;
-      private void ensureOutputIsMutable() {
-        if (!((bitField0_ & 0x00000040) == 0x00000040)) {
-          output_ = new com.google.protobuf.LazyStringArrayList(output_);
-          bitField0_ |= 0x00000040;
-         }
-      }
-      /**
-       * <code>repeated string output = 8;</code>
-       */
-      public java.util.List<java.lang.String>
-          getOutputList() {
-        return java.util.Collections.unmodifiableList(output_);
-      }
-      /**
-       * <code>repeated string output = 8;</code>
-       */
-      public int getOutputCount() {
-        return output_.size();
-      }
-      /**
-       * <code>repeated string output = 8;</code>
-       */
-      public java.lang.String getOutput(int index) {
-        return output_.get(index);
-      }
-      /**
-       * <code>repeated string output = 8;</code>
-       */
-      public com.google.protobuf.ByteString
-          getOutputBytes(int index) {
-        return output_.getByteString(index);
-      }
-      /**
-       * <code>repeated string output = 8;</code>
-       */
-      public Builder setOutput(
-          int index, java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  ensureOutputIsMutable();
-        output_.set(index, value);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated string output = 8;</code>
-       */
-      public Builder addOutput(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  ensureOutputIsMutable();
-        output_.add(value);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated string output = 8;</code>
-       */
-      public Builder addAllOutput(
-          java.lang.Iterable<java.lang.String> values) {
-        ensureOutputIsMutable();
-        super.addAll(values, output_);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated string output = 8;</code>
-       */
-      public Builder clearOutput() {
-        output_ = com.google.protobuf.LazyStringArrayList.EMPTY;
-        bitField0_ = (bitField0_ & ~0x00000040);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated string output = 8;</code>
-       */
-      public Builder addOutputBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  ensureOutputIsMutable();
-        output_.add(value);
-        onChanged();
-        return this;
-      }
-
-      // repeated string environment = 9;
-      private com.google.protobuf.LazyStringList environment_ = com.google.protobuf.LazyStringArrayList.EMPTY;
-      private void ensureEnvironmentIsMutable() {
-        if (!((bitField0_ & 0x00000080) == 0x00000080)) {
-          environment_ = new com.google.protobuf.LazyStringArrayList(environment_);
-          bitField0_ |= 0x00000080;
-         }
-      }
-      /**
-       * <code>repeated string environment = 9;</code>
-       */
-      public java.util.List<java.lang.String>
-          getEnvironmentList() {
-        return java.util.Collections.unmodifiableList(environment_);
-      }
-      /**
-       * <code>repeated string environment = 9;</code>
-       */
-      public int getEnvironmentCount() {
-        return environment_.size();
-      }
-      /**
-       * <code>repeated string environment = 9;</code>
-       */
-      public java.lang.String getEnvironment(int index) {
-        return environment_.get(index);
-      }
-      /**
-       * <code>repeated string environment = 9;</code>
-       */
-      public com.google.protobuf.ByteString
-          getEnvironmentBytes(int index) {
-        return environment_.getByteString(index);
-      }
-      /**
-       * <code>repeated string environment = 9;</code>
-       */
-      public Builder setEnvironment(
-          int index, java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  ensureEnvironmentIsMutable();
-        environment_.set(index, value);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated string environment = 9;</code>
-       */
-      public Builder addEnvironment(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  ensureEnvironmentIsMutable();
-        environment_.add(value);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated string environment = 9;</code>
-       */
-      public Builder addAllEnvironment(
-          java.lang.Iterable<java.lang.String> values) {
-        ensureEnvironmentIsMutable();
-        super.addAll(values, environment_);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated string environment = 9;</code>
-       */
-      public Builder clearEnvironment() {
-        environment_ = com.google.protobuf.LazyStringArrayList.EMPTY;
-        bitField0_ = (bitField0_ & ~0x00000080);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated string environment = 9;</code>
-       */
-      public Builder addEnvironmentBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  ensureEnvironmentIsMutable();
-        environment_.add(value);
-        onChanged();
-        return this;
-      }
-
-      // required uint32 roleId = 10;
-      private int roleId_ ;
-      /**
-       * <code>required uint32 roleId = 10;</code>
-       */
-      public boolean hasRoleId() {
-        return ((bitField0_ & 0x00000100) == 0x00000100);
-      }
-      /**
-       * <code>required uint32 roleId = 10;</code>
-       */
-      public int getRoleId() {
-        return roleId_;
-      }
-      /**
-       * <code>required uint32 roleId = 10;</code>
-       */
-      public Builder setRoleId(int value) {
-        bitField0_ |= 0x00000100;
-        roleId_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required uint32 roleId = 10;</code>
-       */
-      public Builder clearRoleId() {
-        bitField0_ = (bitField0_ & ~0x00000100);
-        roleId_ = 0;
-        onChanged();
-        return this;
-      }
-
-      // required bool released = 11;
-      private boolean released_ ;
-      /**
-       * <code>required bool released = 11;</code>
-       */
-      public boolean hasReleased() {
-        return ((bitField0_ & 0x00000200) == 0x00000200);
-      }
-      /**
-       * <code>required bool released = 11;</code>
-       */
-      public boolean getReleased() {
-        return released_;
-      }
-      /**
-       * <code>required bool released = 11;</code>
-       */
-      public Builder setReleased(boolean value) {
-        bitField0_ |= 0x00000200;
-        released_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required bool released = 11;</code>
-       */
-      public Builder clearReleased() {
-        bitField0_ = (bitField0_ & ~0x00000200);
-        released_ = false;
-        onChanged();
-        return this;
-      }
-
-      // required int64 createTime = 12;
-      private long createTime_ ;
-      /**
-       * <code>required int64 createTime = 12;</code>
-       */
-      public boolean hasCreateTime() {
-        return ((bitField0_ & 0x00000400) == 0x00000400);
-      }
-      /**
-       * <code>required int64 createTime = 12;</code>
-       */
-      public long getCreateTime() {
-        return createTime_;
-      }
-      /**
-       * <code>required int64 createTime = 12;</code>
-       */
-      public Builder setCreateTime(long value) {
-        bitField0_ |= 0x00000400;
-        createTime_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required int64 createTime = 12;</code>
-       */
-      public Builder clearCreateTime() {
-        bitField0_ = (bitField0_ & ~0x00000400);
-        createTime_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // required int64 startTime = 13;
-      private long startTime_ ;
-      /**
-       * <code>required int64 startTime = 13;</code>
-       */
-      public boolean hasStartTime() {
-        return ((bitField0_ & 0x00000800) == 0x00000800);
-      }
-      /**
-       * <code>required int64 startTime = 13;</code>
-       */
-      public long getStartTime() {
-        return startTime_;
-      }
-      /**
-       * <code>required int64 startTime = 13;</code>
-       */
-      public Builder setStartTime(long value) {
-        bitField0_ |= 0x00000800;
-        startTime_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required int64 startTime = 13;</code>
-       */
-      public Builder clearStartTime() {
-        bitField0_ = (bitField0_ & ~0x00000800);
-        startTime_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // required string host = 14;
-      private java.lang.Object host_ = "";
-      /**
-       * <code>required string host = 14;</code>
-       */
-      public boolean hasHost() {
-        return ((bitField0_ & 0x00001000) == 0x00001000);
-      }
-      /**
-       * <code>required string host = 14;</code>
-       */
-      public java.lang.String getHost() {
-        java.lang.Object ref = host_;
-        if (!(ref instanceof java.lang.String)) {
-          java.lang.String s = ((com.google.protobuf.ByteString) ref)
-              .toStringUtf8();
-          host_ = s;
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
-      }
-      /**
-       * <code>required string host = 14;</code>
-       */
-      public com.google.protobuf.ByteString
-          getHostBytes() {
-        java.lang.Object ref = host_;
-        if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          host_ = b;
-          return b;
-        } else {
-          return (com.google.protobuf.ByteString) ref;
-        }
-      }
-      /**
-       * <code>required string host = 14;</code>
-       */
-      public Builder setHost(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00001000;
-        host_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required string host = 14;</code>
-       */
-      public Builder clearHost() {
-        bitField0_ = (bitField0_ & ~0x00001000);
-        host_ = getDefaultInstance().getHost();
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required string host = 14;</code>
-       */
-      public Builder setHostBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00001000;
-        host_ = value;
-        onChanged();
-        return this;
-      }
-
-      // required string hostURL = 15;
-      private java.lang.Object hostURL_ = "";
-      /**
-       * <code>required string hostURL = 15;</code>
-       */
-      public boolean hasHostURL() {
-        return ((bitField0_ & 0x00002000) == 0x00002000);
-      }
-      /**
-       * <code>required string hostURL = 15;</code>
-       */
-      public java.lang.String getHostURL() {
-        java.lang.Object ref = hostURL_;
-        if (!(ref instanceof java.lang.String)) {
-          java.lang.String s = ((com.google.protobuf.ByteString) ref)
-              .toStringUtf8();
-          hostURL_ = s;
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
-      }
-      /**
-       * <code>required string hostURL = 15;</code>
-       */
-      public com.google.protobuf.ByteString
-          getHostURLBytes() {
-        java.lang.Object ref = hostURL_;
-        if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          hostURL_ = b;
-          return b;
-        } else {
-          return (com.google.protobuf.ByteString) ref;
-        }
-      }
-      /**
-       * <code>required string hostURL = 15;</code>
-       */
-      public Builder setHostURL(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00002000;
-        hostURL_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required string hostURL = 15;</code>
-       */
-      public Builder clearHostURL() {
-        bitField0_ = (bitField0_ & ~0x00002000);
-        hostURL_ = getDefaultInstance().getHostURL();
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required string hostURL = 15;</code>
-       */
-      public Builder setHostURLBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00002000;
-        hostURL_ = value;
-        onChanged();
-        return this;
-      }
-
-      // optional string appVersion = 16;
-      private java.lang.Object appVersion_ = "";
-      /**
-       * <code>optional string appVersion = 16;</code>
-       */
-      public boolean hasAppVersion() {
-        return ((bitField0_ & 0x00004000) == 0x00004000);
-      }
-      /**
-       * <code>optional string appVersion = 16;</code>
-       */
-      public java.lang.String getAppVersion() {
-        java.lang.Object ref = appVersion_;
-        if (!(ref instanceof java.lang.String)) {
-          java.lang.String s = ((com.google.protobuf.ByteString) ref)
-              .toStringUtf8();
-          appVersion_ = s;
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
-      }
-      /**
-       * <code>optional string appVersion = 16;</code>
-       */
-      public com.google.protobuf.ByteString
-          getAppVersionBytes() {
-        java.lang.Object ref = appVersion_;
-        if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          appVersion_ = b;
-          return b;
-        } else {
-          return (com.google.protobuf.ByteString) ref;
-        }
-      }
-      /**
-       * <code>optional string appVersion = 16;</code>
-       */
-      public Builder setAppVersion(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00004000;
-        appVersion_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string appVersion = 16;</code>
-       */
-      public Builder clearAppVersion() {
-        bitField0_ = (bitField0_ & ~0x00004000);
-        appVersion_ = getDefaultInstance().getAppVersion();
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string appVersion = 16;</code>
-       */
-      public Builder setAppVersionBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00004000;
-        appVersion_ = value;
-        onChanged();
-        return this;
-      }
-
-      // @@protoc_insertion_point(builder_scope:org.apache.slider.api.RoleInstanceState)
-    }
-
-    static {
-      defaultInstance = new RoleInstanceState(true);
-      defaultInstance.initFields();
-    }
-
-    // @@protoc_insertion_point(class_scope:org.apache.slider.api.RoleInstanceState)
-  }
-
-  public interface StopClusterRequestProtoOrBuilder
-      extends com.google.protobuf.MessageOrBuilder {
-
-    // required string message = 1;
-    /**
-     * <code>required string message = 1;</code>
-     *
-     * <pre>
-     **
-     *message to include
-     * </pre>
-     */
-    boolean hasMessage();
-    /**
-     * <code>required string message = 1;</code>
-     *
-     * <pre>
-     **
-     *message to include
-     * </pre>
-     */
-    java.lang.String getMessage();
-    /**
-     * <code>required string message = 1;</code>
-     *
-     * <pre>
-     **
-     *message to include
-     * </pre>
-     */
-    com.google.protobuf.ByteString
-        getMessageBytes();
-  }
-  /**
-   * Protobuf type {@code org.apache.slider.api.StopClusterRequestProto}
-   *
-   * <pre>
-   **
-   * stop the cluster
-   * </pre>
-   */
-  public static final class StopClusterRequestProto extends
-      com.google.protobuf.GeneratedMessage
-      implements StopClusterRequestProtoOrBuilder {
-    // Use StopClusterRequestProto.newBuilder() to construct.
-    private StopClusterRequestProto(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-      super(builder);
-      this.unknownFields = builder.getUnknownFields();
-    }
-    private StopClusterRequestProto(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
-    private static final StopClusterRequestProto defaultInstance;
-    public static StopClusterRequestProto getDefaultInstance() {
-      return defaultInstance;
-    }
-
-    public StopClusterRequestProto getDefaultInstanceForType() {
-      return defaultInstance;
-    }
-
-    private final com.google.protobuf.UnknownFieldSet unknownFields;
-    @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
-        getUnknownFields() {
-      return this.unknownFields;
-    }
-    private StopClusterRequestProto(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      initFields();
-      int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
-      try {
-        boolean done = false;
-        while (!done) {
-          int tag = input.readTag();
-          switch (tag) {
-            case 0:
-              done = true;
-              break;
-            default: {
-              if (!parseUnknownField(input, unknownFields,
-                                     extensionRegistry, tag)) {
-                done = true;
-              }
-              break;
-            }
-            case 10: {
-              bitField0_ |= 0x00000001;
-              message_ = input.readBytes();
-              break;
-            }
-          }
-        }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
-            e.getMessage()).setUnfinishedMessage(this);
-      } finally {
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    public static final com.google.protobuf.Descriptors.Descriptor
-        getDescriptor() {
-      return org.apache.slider.api.proto.Messages.internal_static_org_apache_slider_api_StopClusterRequestProto_descriptor;
-    }
-
-    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-        internalGetFieldAccessorTable() {
-      return org.apache.slider.api.proto.Messages.internal_static_org_apache_slider_api_StopClusterRequestProto_fieldAccessorTable
-          .ensureFieldAccessorsInitialized(
-              org.apache.slider.api.proto.Messages.StopClusterRequestProto.class, org.apache.slider.api.proto.Messages.StopClusterRequestProto.Builder.class);
-    }
-
-    public static com.google.protobuf.Parser<StopClusterRequestProto> PARSER =
-        new com.google.protobuf.AbstractParser<StopClusterRequestProto>() {
-      public StopClusterRequestProto parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return new StopClusterRequestProto(input, extensionRegistry);
-      }
-    };
-
-    @java.lang.Override
-    public com.google.protobuf.Parser<StopClusterRequestProto> getParserForType() {
-      return PARSER;
-    }
-
-    private int bitField0_;
-    // required string message = 1;
-    public static final int MESSAGE_FIELD_NUMBER = 1;
-    private java.lang.Object message_;
-    /**
-     * <code>required string message = 1;</code>
-     *
-     * <pre>
-     **
-     *message to include
-     * </pre>
-     */
-    public boolean hasMessage() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>required string message = 1;</code>
-     *
-     * <pre>
-     **
-     *message to include
-     * </pre>
-     */
-    public java.lang.String getMessage() {
-      java.lang.Object ref = message_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          message_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>required string message = 1;</code>
-     *
-     * <pre>
-     **
-     *message to include
-     * </pre>
-     */
-    public com.google.protobuf.ByteString
-        getMessageBytes() {
-      java.lang.Object ref = message_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        message_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    private void initFields() {
-      message_ = "";
-    }
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized != -1) return isInitialized == 1;
-
-      if (!hasMessage()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      getSerializedSize();
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeBytes(1, getMessageBytes());
-      }
-      getUnknownFields().writeTo(output);
-    }
-
-    private int memoizedSerializedSize = -1;
-    public int getSerializedSize() {
-      int size = memoizedSerializedSize;
-      if (size != -1) return size;
-
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(1, getMessageBytes());
-      }
-      size += getUnknownFields().getSerializedSize();
-      memoizedSerializedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    protected java.lang.Object writeReplace()
-        throws java.io.ObjectStreamException {
-      return super.writeReplace();
-    }
-
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof org.apache.slider.api.proto.Messages.StopClusterRequestProto)) {
-        return super.equals(obj);
-      }
-      org.apache.slider.api.proto.Messages.StopClusterRequestProto other = (org.apache.slider.api.proto.Messages.StopClusterRequestProto) obj;
-
-      boolean result = true;
-      result = result && (hasMessage() == other.hasMessage());
-      if (hasMessage()) {
-        result = result && getMessage()
-            .equals(other.getMessage());
-      }
-      result = result &&
-          getUnknownFields().equals(other.getUnknownFields());
-      return result;
-    }
-
-    private int memoizedHashCode = 0;
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
-      }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasMessage()) {
-        hash = (37 * hash) + MESSAGE_FIELD_NUMBER;
-        hash = (53 * hash) + getMessage().hashCode();
-      }
-      hash = (29 * hash) + getUnknownFields().hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
-
-    public static org.apache.slider.api.proto.Messages.StopClusterRequestProto parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.slider.api.proto.Messages.StopClusterRequestProto parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.slider.api.proto.Messages.StopClusterRequestProto parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.slider.api.proto.Messages.StopClusterRequestProto parseFrom(
-        byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.slider.api.proto.Messages.StopClusterRequestProto parseFrom(java.io.InputSt

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[69/76] [abbrv] hadoop git commit: YARN-5808. Add gc log options to the yarn daemon script when starting services-api. Contributed by Billie Rinaldi

Posted by ji...@apache.org.
YARN-5808. Add gc log options to the yarn daemon script when starting services-api. Contributed by Billie Rinaldi


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

Branch: refs/heads/yarn-native-services
Commit: 5877e0c4e59fe08ffdbed735b8ff248c2468bd16
Parents: 2d644df
Author: Gour Saha <go...@apache.org>
Authored: Thu Nov 10 11:35:02 2016 -0800
Committer: Jian He <ji...@apache.org>
Committed: Wed Dec 7 13:00:06 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/hadoop-yarn/bin/yarn         | 16 ++++++++--------
 hadoop-yarn-project/hadoop-yarn/conf/yarn-env.sh | 12 ++++++++++++
 2 files changed, 20 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877e0c4/hadoop-yarn-project/hadoop-yarn/bin/yarn
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/bin/yarn b/hadoop-yarn-project/hadoop-yarn/bin/yarn
index 26d54b8..2396a7a 100755
--- a/hadoop-yarn-project/hadoop-yarn/bin/yarn
+++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn
@@ -47,7 +47,7 @@ function hadoop_usage
   hadoop_add_subcommand "resourcemanager" "run the ResourceManager"
   hadoop_add_subcommand "rmadmin" "admin tools"
   hadoop_add_subcommand "scmadmin" "SharedCacheManager admin tools"
-  hadoop_add_subcommand "services-api" "run slider services api"
+  hadoop_add_subcommand "servicesapi" "run slider services api"
   hadoop_add_subcommand "sharedcachemanager" "run the SharedCacheManager daemon"
   hadoop_add_subcommand "slider" "run a slider app"
   hadoop_add_subcommand "timelinereader" "run the timeline reader server"
@@ -144,20 +144,20 @@ function yarncmd_case
     scmadmin)
       HADOOP_CLASSNAME='org.apache.hadoop.yarn.client.SCMAdmin'
     ;;
-    services-api)
+    servicesapi)
       HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
       hadoop_add_classpath "${HADOOP_YARN_HOME}/${YARN_LIB_JARS_DIR}/slider"'/*'
       hadoop_add_classpath "${HADOOP_YARN_HOME}/${YARN_LIB_JARS_DIR}/services-api"'/*'
       HADOOP_CLASSNAME='org.apache.hadoop.yarn.services.webapp.ApplicationApiWebApp'
-      hadoop_debug "Append YARN_CLIENT_OPTS onto HADOOP_OPTS"
-      HADOOP_OPTS="${HADOOP_OPTS} ${YARN_CLIENT_OPTS} \
--Dslider.libdir=${HADOOP_YARN_HOME}/${YARN_DIR},\
+      local sld="${HADOOP_YARN_HOME}/${YARN_DIR},\
 ${HADOOP_YARN_HOME}/${YARN_LIB_JARS_DIR},\
 ${HADOOP_YARN_HOME}/${YARN_LIB_JARS_DIR}/slider,\
 ${HADOOP_HDFS_HOME}/${HDFS_DIR},\
 ${HADOOP_HDFS_HOME}/${HDFS_LIB_JARS_DIR},\
 ${HADOOP_COMMON_HOME}/${HADOOP_COMMON_DIR},\
 ${HADOOP_COMMON_HOME}/${HADOOP_COMMON_LIB_JARS_DIR}"
+      hadoop_translate_cygwin_path sld
+      hadoop_add_param HADOOP_OPTS slider.libdir "-Dslider.libdir=${sld}"
     ;;
     sharedcachemanager)
       HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
@@ -166,15 +166,15 @@ ${HADOOP_COMMON_HOME}/${HADOOP_COMMON_LIB_JARS_DIR}"
     slider)
       hadoop_add_classpath "${HADOOP_YARN_HOME}/${YARN_LIB_JARS_DIR}/slider"'/*'
       HADOOP_CLASSNAME='org.apache.slider.Slider'
-      hadoop_debug "Append YARN_CLIENT_OPTS onto HADOOP_OPTS"
-      HADOOP_OPTS="${HADOOP_OPTS} ${YARN_CLIENT_OPTS} \
--Dslider.libdir=${HADOOP_YARN_HOME}/${YARN_DIR},\
+      local sld="${HADOOP_YARN_HOME}/${YARN_DIR},\
 ${HADOOP_YARN_HOME}/${YARN_LIB_JARS_DIR},\
 ${HADOOP_YARN_HOME}/${YARN_LIB_JARS_DIR}/slider,\
 ${HADOOP_HDFS_HOME}/${HDFS_DIR},\
 ${HADOOP_HDFS_HOME}/${HDFS_LIB_JARS_DIR},\
 ${HADOOP_COMMON_HOME}/${HADOOP_COMMON_DIR},\
 ${HADOOP_COMMON_HOME}/${HADOOP_COMMON_LIB_JARS_DIR}"
+      hadoop_translate_cygwin_path sld
+      hadoop_add_param HADOOP_OPTS slider.libdir "-Dslider.libdir=${sld}"
     ;;
     timelinereader)
       HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5877e0c4/hadoop-yarn-project/hadoop-yarn/conf/yarn-env.sh
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/conf/yarn-env.sh b/hadoop-yarn-project/hadoop-yarn/conf/yarn-env.sh
index d003adb..3828897 100644
--- a/hadoop-yarn-project/hadoop-yarn/conf/yarn-env.sh
+++ b/hadoop-yarn-project/hadoop-yarn/conf/yarn-env.sh
@@ -136,3 +136,15 @@
 # See ResourceManager for some examples
 #
 #export YARN_SHAREDCACHEMANAGER_OPTS=
+
+###
+# Services API specific parameters
+###
+# Specify the JVM options to be used when starting the services API.
+#
+# These options will be appended to the options specified as HADOOP_OPTS
+# and therefore may override any similar flags set in HADOOP_OPTS
+#
+# See ResourceManager for some examples
+#
+#export YARN_SERVICESAPI_OPTS="-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -XX:+PrintGCDateStamps -Xloggc:${HADOOP_LOG_DIR}/gc-servicesapi.log-$(date +'%Y%m%d%H%M')"


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org