You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@slider.apache.org by st...@apache.org on 2015/11/20 22:07:56 UTC

[11/17] incubator-slider git commit: SLIDER-994 add "nodemap" command to get the (JSON) nodemap of the YARN cluster

SLIDER-994 add "nodemap" command to get the (JSON) nodemap of the YARN cluster


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

Branch: refs/heads/feature/SLIDER-82-pass-3.1
Commit: eac0de9fcf7260ba8236825986108fc44548788a
Parents: e9c5ebc
Author: Steve Loughran <st...@apache.org>
Authored: Fri Nov 20 17:45:30 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Nov 20 17:45:30 2015 +0000

----------------------------------------------------------------------
 .../slider/api/types/NodeInformation.java       | 23 +++++--
 .../slider/api/types/NodeInformationList.java   | 41 +++++++++++++
 .../org/apache/slider/client/SliderClient.java  | 63 ++++++++++++++++++--
 .../apache/slider/client/SliderClientAPI.java   | 12 ++++
 .../slider/client/SliderYarnClientImpl.java     | 46 ++++++++++++++
 .../slider/common/params/ActionNodesArgs.java   | 20 +++++++
 .../apache/slider/common/tools/SliderUtils.java | 14 +++++
 .../core/launch/JavaCommandLineBuilder.java     |  2 -
 .../slider/core/persist/JsonSerDeser.java       | 45 +++-----------
 .../slider/server/appmaster/state/AppState.java |  7 +--
 .../server/appmaster/state/NodeInstance.java    |  8 +--
 .../slider/client/TestClientBadArgs.groovy      | 16 +++--
 .../client/TestSliderClientMethods.groovy       |  3 +-
 .../client/TestUpgradeCommandOptions.groovy     | 10 +---
 .../slider/providers/agent/TestAgentEcho.groovy | 32 ++++++++++
 15 files changed, 270 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/eac0de9f/slider-core/src/main/java/org/apache/slider/api/types/NodeInformation.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/api/types/NodeInformation.java b/slider-core/src/main/java/org/apache/slider/api/types/NodeInformation.java
index edf7e21..e759bc9 100644
--- a/slider-core/src/main/java/org/apache/slider/api/types/NodeInformation.java
+++ b/slider-core/src/main/java/org/apache/slider/api/types/NodeInformation.java
@@ -33,12 +33,27 @@ import java.util.Map;
 @JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 public class NodeInformation {
 
-  public String healthReport;
   public String hostname;
-  public String httpAddress;
+  public String state;
   public String labels;
-  public long lastUpdated;
   public String rackName;
-  public String state;
+  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/incubator-slider/blob/eac0de9f/slider-core/src/main/java/org/apache/slider/api/types/NodeInformationList.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/api/types/NodeInformationList.java b/slider-core/src/main/java/org/apache/slider/api/types/NodeInformationList.java
new file mode 100644
index 0000000..741523e
--- /dev/null
+++ b/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/incubator-slider/blob/eac0de9f/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
index bfec417..ca9bb12 100644
--- a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
@@ -68,6 +68,7 @@ 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.SliderClusterOperations;
 import org.apache.slider.common.Constants;
@@ -91,6 +92,7 @@ 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;
@@ -133,6 +135,7 @@ 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;
@@ -166,7 +169,6 @@ import org.slf4j.LoggerFactory;
 
 import java.io.BufferedReader;
 import java.io.File;
-import java.io.FilenameFilter;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -333,7 +335,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     if (isUnset(action)) {
       throw new SliderException(EXIT_USAGE, serviceArgs.usage());
     }
-      
+
     int exitCode = EXIT_SUCCESS;
     String clusterName = serviceArgs.getClusterName();
     // actions
@@ -405,11 +407,15 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
       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;
@@ -2156,7 +2162,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
       log.info("Using queue {} for the application instance.", amQueue);
     }
 
-    if (amQueue != null) {
+    if (isSet(amQueue)) {
       amLauncher.setQueue(amQueue);
     }
 
@@ -4225,6 +4231,55 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     throw new UsageException("%s %s", errMsg, CommonArgs.usage(serviceArgs,
         actionName));
   }
+
+  /**
+   * List the nodes in the cluster, possibly filtering by node state or label.
+   *
+   * @param args argument list
+   * @return a possibly empty list of nodes in the cluster
+   * @throws IOException IO problems
+   * @throws YarnException YARN problems
+   */
+  @Override
+  public NodeInformationList listYarnClusterNodes(ActionNodesArgs args)
+    throws YarnException, IOException {
+    return yarnClient.listNodes(args.label, args.healthy);
+  }
+
+  /**
+   * List the nodes in the cluster, possibly filtering by node state or label.
+   *
+   * @param args argument list
+   * @return a possibly empty list of nodes in the cluster
+   * @throws IOException IO problems
+   * @throws YarnException YARN problems
+   */
+  public NodeInformationList listInstanceNodes(ActionNodesArgs args)
+    throws YarnException, IOException {
+    return yarnClient.listNodes(args.label, args.healthy);
+  }
+
+  /**
+   * List the nodes in the cluster, possibly filtering by node state or label.
+   * Prints them to stdout unless the args names a file instead.
+   * @param args argument list
+   * @throws IOException IO problems
+   * @throws YarnException YARN problems
+   */
+  public int actionNodes(String instance, ActionNodesArgs args) throws YarnException, IOException {
+
+    args.instance = instance;
+    NodeInformationList nodes = listYarnClusterNodes(args);
+    log.debug("Node listing for {} has {} nodes", args, nodes.size());
+    JsonSerDeser<NodeInformationList> serDeser = NodeInformationList.createSerializer();
+    if (args.outputFile != null) {
+      serDeser.save(nodes, args.outputFile);
+    } else {
+      println(serDeser.toJson(nodes));
+    }
+    return 0;
+  }
+
 }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/eac0de9f/slider-core/src/main/java/org/apache/slider/client/SliderClientAPI.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/SliderClientAPI.java b/slider-core/src/main/java/org/apache/slider/client/SliderClientAPI.java
index d87b121..5c5d96b 100644
--- a/slider-core/src/main/java/org/apache/slider/client/SliderClientAPI.java
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderClientAPI.java
@@ -23,6 +23,7 @@ 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;
@@ -36,6 +37,7 @@ 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;
@@ -342,4 +344,14 @@ public interface SliderClientAPI extends Service {
    */
   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/incubator-slider/blob/eac0de9f/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java b/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
index a315345..85a582b 100644
--- a/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderYarnClientImpl.java
@@ -28,13 +28,18 @@ 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;
@@ -361,4 +366,45 @@ public class SliderYarnClientImpl extends YarnClientImpl {
     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/incubator-slider/blob/eac0de9f/slider-core/src/main/java/org/apache/slider/common/params/ActionNodesArgs.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/ActionNodesArgs.java b/slider-core/src/main/java/org/apache/slider/common/params/ActionNodesArgs.java
index bb214e1..5ddccf6 100644
--- a/slider-core/src/main/java/org/apache/slider/common/params/ActionNodesArgs.java
+++ b/slider-core/src/main/java/org/apache/slider/common/params/ActionNodesArgs.java
@@ -27,6 +27,11 @@ import java.io.File;
             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;
@@ -42,5 +47,20 @@ public class ActionNodesArgs extends AbstractActionArgs {
   @Parameter(names = {ARG_HEALTHY} )
   public boolean healthy;
 
+  @Override
+  public int getMinParams() {
+    return 0;
+  }
 
+  @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/incubator-slider/blob/eac0de9f/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java b/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
index eb7a9d5..712eb75 100644
--- a/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
+++ b/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
@@ -47,6 +47,7 @@ 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;
@@ -1774,6 +1775,19 @@ public final class SliderUtils {
     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

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/eac0de9f/slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java b/slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java
index ccb610a..dcb322e 100644
--- a/slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java
+++ b/slider-core/src/main/java/org/apache/slider/core/launch/JavaCommandLineBuilder.java
@@ -25,8 +25,6 @@ import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.slider.common.tools.SliderUtils;
 import org.apache.slider.core.exceptions.BadConfigException;
 
-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

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/eac0de9f/slider-core/src/main/java/org/apache/slider/core/persist/JsonSerDeser.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/persist/JsonSerDeser.java b/slider-core/src/main/java/org/apache/slider/core/persist/JsonSerDeser.java
index c5908bb..4f60c06 100644
--- a/slider-core/src/main/java/org/apache/slider/core/persist/JsonSerDeser.java
+++ b/slider-core/src/main/java/org/apache/slider/core/persist/JsonSerDeser.java
@@ -91,10 +91,11 @@ public class JsonSerDeser<T> {
    */
   public T fromFile(File jsonFile)
     throws IOException, JsonParseException, JsonMappingException {
+    File absoluteFile = jsonFile.getAbsoluteFile();
     try {
-      return mapper.readValue(jsonFile, classType);
+      return mapper.readValue(absoluteFile, classType);
     } catch (IOException e) {
-      log.error("Exception while parsing json file {}: {}", jsonFile, e);
+      log.error("Exception while parsing json file {}", absoluteFile, e);
       throw e;
     }
   }
@@ -106,7 +107,6 @@ public class JsonSerDeser<T> {
    * @throws IOException IO problems
    * @throws JsonMappingException failure to map from the JSON to this class
    */
-/* JDK7
  public T fromResource(String resource)
     throws IOException, JsonParseException, JsonMappingException {
     try(InputStream resStream = this.getClass().getResourceAsStream(resource)) {
@@ -115,34 +115,9 @@ public class JsonSerDeser<T> {
       }
       return (T) (mapper.readValue(resStream, classType));
     } catch (IOException e) {
-      log.error("Exception while parsing json resource {}: {}", resource, e);
+      log.error("Exception while parsing json resource {}", resource, 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
-   */
-  @SuppressWarnings("IOResourceOpenedButNotSafelyClosed")
-  public synchronized T fromResource(String resource)
-      throws IOException, JsonParseException, JsonMappingException {
-    InputStream resStream = null;
-    try {
-      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;
-    } finally {
-      IOUtils.closeStream(resStream);
-    }
   }
 
   /**
@@ -155,7 +130,7 @@ public class JsonSerDeser<T> {
     try {
       return (T) (mapper.readValue(stream, classType));
     } catch (IOException e) {
-      log.error("Exception while parsing json input stream: {}", e);
+      log.error("Exception while parsing json input stream", e);
       throw e;
     } finally {
       IOUtils.closeStream(stream);
@@ -201,7 +176,7 @@ public class JsonSerDeser<T> {
     FSDataInputStream dataInputStream = fs.open(path);
     int count = dataInputStream.read(b);
     if (count != len) {
-      throw new EOFException("Read finished prematurely");
+      throw new EOFException("Read of " + path +" finished prematurely");
     }
     return fromBytes(b);
   }
@@ -230,9 +205,9 @@ public class JsonSerDeser<T> {
    */
   public void save(T instance, File file) throws
       IOException {
-    writeJsonAsBytes(instance, new FileOutputStream(file));
+    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
@@ -251,7 +226,6 @@ public class JsonSerDeser<T> {
     }
   }
 
-
   /**
    * Convert an object to a JSON string
    * @param instance instance to convert
@@ -265,6 +239,5 @@ public class JsonSerDeser<T> {
     mapper.configure(SerializationConfig.Feature.INDENT_OUTPUT, true);
     return mapper.writeValueAsString(instance);
   }
-  
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/eac0de9f/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
index 49bc225..ee5d43d 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
@@ -688,8 +688,8 @@ public class AppState {
     instanceDefinition.resolve();
 
     // force in the AM desired state values
-    ConfTreeOperations resources =
-        instanceDefinition.getResourceOperations();
+    ConfTreeOperations resources = instanceDefinition.getResourceOperations();
+
     if (resources.getComponent(SliderKeys.COMPONENT_AM) != null) {
       resources.setComponentOpt(
           SliderKeys.COMPONENT_AM, COMPONENT_INSTANCES, "1");
@@ -706,8 +706,7 @@ public class AppState {
                                                    internalsSnapshot.confTree);
     instanceDefinitionSnapshot.setName(instanceDefinition.getName());
 
-    clusterStatusTemplate =
-      ClusterDescriptionOperations.buildFromInstanceDefinition(
+    clusterStatusTemplate = ClusterDescriptionOperations.buildFromInstanceDefinition(
           instanceDefinition);
 
     // Add the -site configuration properties

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/eac0de9f/slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeInstance.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeInstance.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeInstance.java
index 2b8f01c..8d63239 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeInstance.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/NodeInstance.java
@@ -31,7 +31,6 @@ import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
 import java.util.ListIterator;
-import java.util.Set;
 
 /**
  * A node instance -stores information about a node in the cluster.
@@ -103,12 +102,7 @@ public class NodeInstance {
     boolean newUsable = !nodeState.isUnusable();
     boolean nodeNowAvailable = oldStateUnusable && newUsable;
     String labels = this.nodeLabels;
-    Set<String> newlabels = report.getNodeLabels();
-    if (newlabels != null && !newlabels.isEmpty()) {
-      nodeLabels = newlabels.iterator().next().trim();
-    } else {
-      nodeLabels = "";
-    }
+    nodeLabels = SliderUtils.extractNodeLabel(report);
     return nodeNowAvailable
         || newUsable && !this.nodeLabels.equals(labels);
   }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/eac0de9f/slider-core/src/test/groovy/org/apache/slider/client/TestClientBadArgs.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/client/TestClientBadArgs.groovy b/slider-core/src/test/groovy/org/apache/slider/client/TestClientBadArgs.groovy
index 1b074dc..9b6c7dc 100644
--- a/slider-core/src/test/groovy/org/apache/slider/client/TestClientBadArgs.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/client/TestClientBadArgs.groovy
@@ -22,14 +22,11 @@ import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.yarn.conf.YarnConfiguration
-import org.apache.slider.common.params.ActionRegistryArgs
 import org.apache.slider.common.params.Arguments
 import org.apache.slider.common.params.SliderActions
 import org.apache.slider.core.exceptions.BadCommandArgumentsException
 import org.apache.slider.core.exceptions.ErrorStrings
 import org.apache.slider.core.exceptions.UsageException
-import org.apache.slider.core.exceptions.BadConfigException
-import org.apache.slider.core.main.ServiceLauncher
 import org.apache.slider.core.main.ServiceLauncherBaseTest
 import org.junit.Test
 
@@ -39,9 +36,7 @@ import org.junit.Test
 @CompileStatic
 @Slf4j
 class TestClientBadArgs extends ServiceLauncherBaseTest {
-  
-  static String TEST_FILES = "./src/test/resources/org/apache/slider/providers/agent/tests/"
-  
+
   @Test
   public void testNoAction() throws Throwable {
     launchExpectingException(SliderClient,
@@ -244,4 +239,13 @@ class TestClientBadArgs extends ServiceLauncherBaseTest {
       assert exception instanceof BadCommandArgumentsException
       log.info(exception.toString())
     }
+
+  @Test
+  public void testNodesMissingFile() throws Throwable {
+    def exception = launchExpectingException(SliderClient,
+        createTestConfig(),
+        "after parameter --out",
+        [SliderActions.ACTION_NODES, Arguments.ARG_OUTPUT])
+    assert exception instanceof BadCommandArgumentsException
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/eac0de9f/slider-core/src/test/groovy/org/apache/slider/client/TestSliderClientMethods.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/client/TestSliderClientMethods.groovy b/slider-core/src/test/groovy/org/apache/slider/client/TestSliderClientMethods.groovy
index 168415c..e244923 100644
--- a/slider-core/src/test/groovy/org/apache/slider/client/TestSliderClientMethods.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/client/TestSliderClientMethods.groovy
@@ -30,6 +30,7 @@ import org.apache.slider.core.exceptions.SliderException
 import org.apache.slider.core.launch.LaunchedApplication
 import org.apache.slider.core.main.ServiceLauncherBaseTest
 import org.apache.slider.core.persist.LockAcquireFailedException
+import org.apache.slider.server.appmaster.model.mock.MockApplicationId
 import org.easymock.EasyMock
 import org.junit.Assert
 import org.junit.Test
@@ -137,7 +138,7 @@ class TestSliderClientMethods extends ServiceLauncherBaseTest {
         AggregateConf instanceDefinition,
         boolean debugAM)
     throws YarnException, IOException {
-      return new LaunchedApplication(clustername, new SliderYarnClientImpl());
+      return new LaunchedApplication(new MockApplicationId(1), new SliderYarnClientImpl());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/eac0de9f/slider-core/src/test/groovy/org/apache/slider/client/TestUpgradeCommandOptions.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/client/TestUpgradeCommandOptions.groovy b/slider-core/src/test/groovy/org/apache/slider/client/TestUpgradeCommandOptions.groovy
index 8d5aef1..5295f9b 100644
--- a/slider-core/src/test/groovy/org/apache/slider/client/TestUpgradeCommandOptions.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/client/TestUpgradeCommandOptions.groovy
@@ -82,10 +82,8 @@ class TestUpgradeCommandOptions extends AgentMiniClusterTestBase {
               new File(tmpDir, "resources.json").toURI()
           ])
       fail("Upgrade command should have failed")
-    } catch (SliderException e) {
-      log.info(e.toString())
-      assert e instanceof UnknownApplicationInstanceException
-      assert e.getMessage().contains("Unknown application instance")
+    } catch (UnknownApplicationInstanceException e) {
+      assertExceptionDetails(e, SliderExitCodes.EXIT_UNKNOWN_INSTANCE, "Unknown application instance")
     }
   }
 
@@ -301,10 +299,6 @@ class TestUpgradeCommandOptions extends AgentMiniClusterTestBase {
     assert launcher.serviceExitCode == 0
   }
 
-  private File getTempLocation () {
-    return new File(System.getProperty("user.dir") + "/target/_")
-  }
-
   static class TestSliderClient extends SliderClient {
     public TestSliderClient() {
       super()

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/eac0de9f/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentEcho.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentEcho.groovy b/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentEcho.groovy
index 4ceeca0..23a7bbb 100644
--- a/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentEcho.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentEcho.groovy
@@ -22,11 +22,14 @@ import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
 import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.slider.api.ResourceKeys
+import org.apache.slider.api.types.NodeInformationList
 import org.apache.slider.client.SliderClient
 import org.apache.slider.common.SliderExitCodes
 import org.apache.slider.common.SliderXmlConfKeys
+import org.apache.slider.common.params.ActionNodesArgs
 import org.apache.slider.core.exceptions.BadClusterStateException
 import org.apache.slider.core.main.ServiceLauncher
+import org.apache.slider.core.persist.JsonSerDeser
 import org.junit.Before
 import org.junit.Test
 
@@ -124,6 +127,35 @@ class TestAgentEcho extends AgentTestBase {
       assertExceptionDetails(e, SliderExitCodes.EXIT_BAD_STATE, "negative")
     }
 
+
+    runNodemapTests(sliderClient)
+
+  }
+
+  /**
+   * do some nodemap checks, currently cluster-wide
+   * @param sliderClient
+   */
+  protected void runNodemapTests(SliderClient sliderClient) {
+    describe "slider nodes"
+    sliderClient.actionNodes("", new ActionNodesArgs())
+
+    def allNodes = sliderClient.listYarnClusterNodes(new ActionNodesArgs()).collect { it.httpAddress }
+    assert !allNodes.empty
+
+    // healthy only
+    def healthyNodes = sliderClient.listYarnClusterNodes(new ActionNodesArgs(healthy: true))
+    assert healthyNodes.collect { it.httpAddress }.containsAll(allNodes)
+    // look for an unknown label and expect none
+    def gpuNodes = sliderClient.listYarnClusterNodes(new ActionNodesArgs(label: "gpu"))
+    assert gpuNodes.empty
+    File t1 = createTempJsonFile()
+    sliderClient.actionNodes("", new ActionNodesArgs(outputFile: t1))
+    assert t1.exists()
+    JsonSerDeser<NodeInformationList> serDeser = new JsonSerDeser<>(NodeInformationList.class);
+    NodeInformationList loaded = serDeser.fromFile(t1)
+    assert allNodes.containsAll(loaded.collect { it.httpAddress })
+
   }
 
   protected void validatePaths() {