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 2014/10/01 02:48:27 UTC

[05/12] git commit: SLIDER-158. Add a slider diagnostics command (Thomas Liu via smohanty)

SLIDER-158. Add a slider diagnostics command (Thomas Liu via smohanty)


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

Branch: refs/heads/feature/SLIDER-149_Support_a_YARN_service_registry
Commit: f8210a7c00c54054dc690abbe5088cd4ddea4e3e
Parents: ce647d3
Author: Sumit Mohanty <sm...@hortonworks.com>
Authored: Sat Sep 27 00:22:12 2014 -0700
Committer: Sumit Mohanty <sm...@hortonworks.com>
Committed: Sat Sep 27 00:22:26 2014 -0700

----------------------------------------------------------------------
 slider-core/pom.xml                             |   6 +
 .../org/apache/slider/client/SliderClient.java  | 280 ++++++++++++++++++-
 .../common/params/ActionDiagnosticArgs.java     |  66 +++++
 .../apache/slider/common/params/Arguments.java  |   7 +
 .../apache/slider/common/params/ClientArgs.java |  11 +-
 .../slider/common/params/SliderActions.java     |   4 +
 .../apache/slider/common/tools/SliderUtils.java | 108 +++++++
 .../slider/providers/agent/AgentKeys.java       |   5 +
 8 files changed, 485 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f8210a7c/slider-core/pom.xml
----------------------------------------------------------------------
diff --git a/slider-core/pom.xml b/slider-core/pom.xml
index 39c17c6..7af8bd7 100644
--- a/slider-core/pom.xml
+++ b/slider-core/pom.xml
@@ -448,6 +448,12 @@
     </dependency>
 
     <dependency>
+      <groupId>org.codehaus.jettison</groupId>
+      <artifactId>jettison</artifactId>
+      <version>1.1</version>
+    </dependency>
+
+    <dependency>
       <groupId>org.powermock</groupId>
       <artifactId>powermock-core</artifactId>
       <version>1.5</version>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f8210a7c/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 d57f7dc..06c37ba 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
@@ -19,6 +19,7 @@
 package org.apache.slider.client;
 
 import com.google.common.annotations.VisibleForTesting;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -33,6 +34,8 @@ 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.YarnException;
@@ -48,6 +51,7 @@ 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.ActionDiagnosticArgs;
 import org.apache.slider.common.params.ActionInstallPackageArgs;
 import org.apache.slider.common.params.ActionAMSuicideArgs;
 import org.apache.slider.common.params.ActionCreateArgs;
@@ -91,6 +95,7 @@ import org.apache.slider.core.launch.LaunchedApplication;
 import org.apache.slider.core.launch.RunningApplication;
 import org.apache.slider.core.main.RunService;
 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.YARNRegistryClient;
 import org.apache.slider.core.registry.docstore.ConfigFormat;
@@ -115,9 +120,15 @@ 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.codehaus.jackson.map.DeserializationConfig;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.SerializationConfig;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.FileWriter;
@@ -125,6 +136,7 @@ import java.io.IOException;
 import java.io.StringWriter;
 import java.io.Writer;
 import java.net.InetSocketAddress;
+import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -338,6 +350,8 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     } else if (ACTION_VERSION.equals(action)) {
 
       exitCode = actionVersion();
+    } else if (ACTION_DIAGNOSTIC.equals(action)) {
+        exitCode = actionDiagnostic(serviceArgs.getActionDiagnosticArgs());
     } else {
       throw new SliderException(EXIT_UNIMPLEMENTED,
           "Unimplemented: " + action);
@@ -346,7 +360,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     return exitCode;
   }
 
-  /**
+/**
    * Perform everything needed to init the hadoop binding.
    * This assumes that the service is already  in inited or started state
    * @throws IOException
@@ -2386,6 +2400,268 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     return sids;
   }
 
+	/**
+	 * diagnostic operation
+	 *
+	 * @param clusterName
+	 *            application name
+	 * @param diagosticArgs
+	 *            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
+	 */
+	private int actionDiagnostic(ActionDiagnosticArgs diagnosticArgs) {
+		try {
+			if (diagnosticArgs.client) {
+				actionDiagnosticClient();
+			} else if (SliderUtils.isSet(diagnosticArgs.application)) {
+				actionDiagnosticApplication(diagnosticArgs);
+			} else if (SliderUtils.isSet(diagnosticArgs.slider)) {
+				actionDiagnosticSlider(diagnosticArgs);
+			} else if (diagnosticArgs.yarn) {
+				actionDiagnosticYarn(diagnosticArgs);
+			} else if (diagnosticArgs.credentials) {
+				actionDiagnosticCredentials();
+			} else if (SliderUtils.isSet(diagnosticArgs.all)) {
+				actionDiagnosticAll(diagnosticArgs);
+			} else if (SliderUtils.isSet(diagnosticArgs.level)) {
+				actionDiagnosticIntelligent(diagnosticArgs);
+			} else {
+				// it's an unknown command
+		        log.info(ActionDiagnosticArgs.USAGE);
+		        return EXIT_USAGE;
+			}
+		} catch (Exception e) {
+			log.error(e.toString());
+			return EXIT_FALSE;
+		}
+		return EXIT_SUCCESS;
+	}
+
+	private void actionDiagnosticIntelligent(ActionDiagnosticArgs diagnosticArgs)
+			throws YarnException, IOException, URISyntaxException {
+		// not using member variable clustername because we want to place
+		// application name after --application option and member variable
+		// cluster name has to be put behind action
+		String clusterName = diagnosticArgs.level;
+
+		try {
+			SliderUtils.validateClientConfigFile();
+			log.info("Slider-client.xml is accessible");
+		} catch (IOException e) {
+			// we are catching exceptions here because those are indication of
+			// validation result, and we need to print them here
+			log.error("validation of slider-client.xml fails because: "
+					+ e.toString());
+			return;
+		}
+		SliderClusterOperations clusterOperations = createClusterOperations(clusterName);
+		// cluster not found exceptions will be thrown upstream
+		ClusterDescription clusterDescription = clusterOperations
+				.getClusterDescription();
+		log.info("Slider AppMaster is accessible");
+		
+		if (clusterDescription.state == ClusterDescription.STATE_LIVE) {
+			AggregateConf instanceDefinition = clusterOperations
+					.getInstanceDefinition();
+			String imagePath = instanceDefinition.getInternalOperations().get(
+					InternalKeys.INTERNAL_APPLICATION_IMAGE_PATH);
+			//if null, that means slider uploaded the agent tarball for the user
+			//and we need to use where slider has put
+			if(imagePath == null){
+				ApplicationReport appReport = YARNRegistryClient.findInstance(clusterName);
+				Path path1 = sliderFileSystem.getTempPathForCluster(clusterName);
+				Path subPath = new Path(path1, appReport.getApplicationId().toString() + "/am");
+				imagePath = subPath.toString();
+			}
+			try {
+				SliderUtils.validateHDFSFile(sliderFileSystem, imagePath);
+				log.info("Slider agent tarball is properly installed");
+			} catch (IOException e) {
+				log.error("can not find or open agent tar ball: " + e.toString());
+				return;
+			}
+			String pkgTarballPath = instanceDefinition.getAppConfOperations()
+					.getGlobalOptions().getMandatoryOption(AgentKeys.APP_DEF);
+			try {
+				SliderUtils.validateHDFSFile(sliderFileSystem, pkgTarballPath);
+				log.info("Application tarball is properly installed");
+			} catch (IOException e) {
+				log.error("can not find or open application tar ball: "
+						+ e.toString());
+				return;
+			}
+		}
+	}
+
+	private void actionDiagnosticAll(ActionDiagnosticArgs diagnosticArgs)
+			throws IOException, YarnException {
+		//assign application name from param to each sub diagnostic function
+		diagnosticArgs.application = diagnosticArgs.all;
+		diagnosticArgs.slider = diagnosticArgs.all;
+		actionDiagnosticClient();
+		actionDiagnosticApplication(diagnosticArgs);
+		actionDiagnosticSlider(diagnosticArgs);
+		actionDiagnosticYarn(diagnosticArgs);
+		actionDiagnosticCredentials();
+	}
+
+	private void actionDiagnosticCredentials() throws BadConfigException, IOException
+			 {
+		if (SliderUtils.isHadoopClusterSecure(SliderUtils
+				.loadClientConfigurationResource())) {
+			String credentialCacheFileDescription = null;
+			try {
+				credentialCacheFileDescription = SliderUtils
+						.checkCredentialCacheFile();
+			} catch (BadConfigException e) {
+				log.error("The credential config is not valid: " + e.toString());
+				throw e;
+			} catch (IOException e) {
+				log.error("Unable to read the credential file: " + e.toString());
+				throw e;
+			}
+			log.info("Credential cache file for the current user: "
+					+ credentialCacheFileDescription);
+		} else {
+			log.info("the cluster is not in secure mode");
+		}
+	}
+
+	private void actionDiagnosticYarn(ActionDiagnosticArgs diagnosticArgs) throws IOException, YarnException {
+		JSONObject converter = null;
+		log.info("the node in the YARN cluster has below state: ");
+		List<NodeReport> yarnClusterInfo;
+		try {
+			yarnClusterInfo = yarnClient.getNodeReports(NodeState.RUNNING);
+		} catch (YarnException e1) {
+			log.error("Exception happened when fetching node report from the YARN cluster: " + e1.toString());
+			throw e1;
+		} catch (IOException e1) {
+			log.error("Network problem happened when fetching node report YARN cluster: " + e1.toString());
+			throw e1;
+		}
+		for(NodeReport nodeReport : yarnClusterInfo){
+			log.info(nodeReport.toString());
+		}
+		
+		if (diagnosticArgs.verbose) {
+			Writer configWriter = new StringWriter();
+			try {
+				Configuration.dumpConfiguration(yarnClient.getConfig(), configWriter);
+			} catch (IOException e1) {
+				log.error("Network problem happened when retrieving YARN config from YARN: " + e1.toString());
+				throw e1;
+			}
+			try {
+				converter = new JSONObject(configWriter.toString());
+				log.info("the configuration of the YARN cluster is: "
+						+ converter.toString(2));
+				
+			} catch (JSONException e) {
+				log.error("JSONException happened during parsing response from YARN: " + e.toString());
+			}
+		}
+	}
+
+	private void actionDiagnosticSlider(ActionDiagnosticArgs diagnosticArgs) throws YarnException, IOException
+			{
+		// not using member variable clustername because we want to place
+		// application name after --application option and member variable
+		// cluster name has to be put behind action
+		String clusterName = diagnosticArgs.slider;
+		SliderClusterOperations clusterOperations;
+		AggregateConf instanceDefinition = null;
+		try {
+			clusterOperations = createClusterOperations(clusterName);
+			instanceDefinition = clusterOperations
+					.getInstanceDefinition();
+		} catch (YarnException e) {
+			log.error("Exception happened when retrieving instance definition from YARN: " + e.toString());
+			throw e;
+		} catch (IOException e) {
+			log.error("Network problem happened when retrieving instance definition from YARN: " + e.toString());
+			throw e;
+		}
+		String imagePath = instanceDefinition.getInternalOperations().get(
+				InternalKeys.INTERNAL_APPLICATION_IMAGE_PATH);
+		//if null, it will be uploaded by Slider and thus at slider's path
+		if(imagePath == null){
+			ApplicationReport appReport = YARNRegistryClient.findInstance(clusterName);
+			Path path1 = sliderFileSystem.getTempPathForCluster(clusterName);
+			Path subPath = new Path(path1, appReport.getApplicationId().toString() + "/am");
+			imagePath = subPath.toString();
+		}
+		log.info("The path of slider agent tarball on HDFS is: " + imagePath);
+	}
+
+	private void actionDiagnosticApplication(ActionDiagnosticArgs diagnosticArgs) throws YarnException, IOException
+			{
+		// not using member variable clustername because we want to place
+		// application name after --application option and member variable
+		// cluster name has to be put behind action
+		String clusterName = diagnosticArgs.application;
+		SliderClusterOperations clusterOperations;
+		AggregateConf instanceDefinition = null;
+		try {
+			clusterOperations = createClusterOperations(clusterName);
+			instanceDefinition = clusterOperations
+					.getInstanceDefinition();
+		} catch (YarnException e) {
+			log.error("Exception happened when retrieving instance definition from YARN: " + e.toString());
+			throw e;
+		} catch (IOException e) {
+			log.error("Network problem happened when retrieving instance definition from YARN: " + e.toString());
+			throw e;
+		}
+		String clusterDir = instanceDefinition.getAppConfOperations()
+				.getGlobalOptions().get(AgentKeys.APP_ROOT);
+		String pkgTarball = instanceDefinition.getAppConfOperations()
+				.getGlobalOptions().get(AgentKeys.APP_DEF);
+		String runAsUser = instanceDefinition.getAppConfOperations()
+				.getGlobalOptions().get(AgentKeys.RUNAS_USER);
+
+		log.info("The location of the cluster instance directory in HDFS is: "
+				+ clusterDir);
+		log.info("The name of the application package tarball on HDFS is: "
+				+ pkgTarball);
+		log.info("The runas user of the application in the cluster is: "
+				+ runAsUser);
+
+		if (diagnosticArgs.verbose) {
+			log.info("App config of the application: "
+					+ instanceDefinition.getAppConf().toJson());
+			log.info("Resource config of the application: "
+					+ instanceDefinition.getResources().toJson());
+		}
+	}
+
+	private void actionDiagnosticClient() throws SliderException, IOException {
+		String currentCommandPath = SliderUtils.getCurrentCommandPath();
+		SliderVersionInfo.loadAndPrintVersionInfo(log);
+		String clientConfigPath = SliderUtils.getClientConfigPath();
+		String jdkInfo = SliderUtils.getJDKInfo();
+		log.info("The slider command path: " + currentCommandPath);
+		log.info("The slider-client.xml used by current running command path: "
+				+ clientConfigPath);
+		log.info(jdkInfo);
+
+		try {
+			SliderUtils.validateSliderClientEnvironment(log);
+		} catch (SliderException e) {
+			log.error(e.toString());
+			throw e;
+		} catch (IOException e) {
+			log.error(e.toString());
+			throw e;
+		}
+	}
+
   private void logInstance(ServiceInstanceData instance,
       boolean verbose) {
     if (!verbose) {
@@ -2612,3 +2888,5 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     System.out.append(src);
   }
 }
+
+

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f8210a7c/slider-core/src/main/java/org/apache/slider/common/params/ActionDiagnosticArgs.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/ActionDiagnosticArgs.java b/slider-core/src/main/java/org/apache/slider/common/params/ActionDiagnosticArgs.java
new file mode 100644
index 0000000..b2fde07
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/common/params/ActionDiagnosticArgs.java
@@ -0,0 +1,66 @@
+package org.apache.slider.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+
+@Parameters(commandNames = {SliderActions.ACTION_DIAGNOSTIC},
+commandDescription = SliderActions.DESCRIBE_ACTION_DIAGNOSTIC)
+public class ActionDiagnosticArgs extends AbstractActionArgs
+{
+	  public static final String USAGE =
+	      "Usage: " + SliderActions.ACTION_DIAGNOSTIC
+	      + Arguments.ARG_CLIENT + "| "
+	      + Arguments.ARG_SLIDER + " <appname> " + "| "
+	      + Arguments.ARG_APPLICATION + " <appname> " + "| "
+	      + Arguments.ARG_YARN + "| "
+	      + Arguments.ARG_CREDENTIALS + "| "
+	      + Arguments.ARG_ALL + " <appname> " + "| "
+	      + Arguments.ARG_LEVEL + " <appname> "
+	      + " [" + Arguments.ARG_VERBOSE + "] ";
+	  
+	@Override
+	public String getActionName() {
+		return SliderActions.ACTION_DIAGNOSTIC;
+	}
+	
+	  @Parameter(names = {ARG_CLIENT}, 
+	      description = "print configuration of the slider client")
+	  public boolean client = false;
+	
+	  @Parameter(names = {ARG_SLIDER}, 
+	      description = "print configuration of the running slider app master")
+	  public String slider;
+	
+	  @Parameter(names = {ARG_APPLICATION}, 
+	      description = "print configuration of the running application")
+	  public String 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 String all;
+	
+	  @Parameter(names = {ARG_LEVEL}, 
+	      description = "diagnoze the application intelligently")
+	  public String 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/incubator-slider/blob/f8210a7c/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java b/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
index b119245..2b45ce8 100644
--- a/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
+++ b/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
@@ -78,6 +78,13 @@ public interface Arguments {
   String ARG_ZKPORT = "--zkport";
   String ARG_ZKHOSTS = "--zkhosts";
   String ARG_REPLACE_PKG = "--replacepkg";
+  String ARG_CLIENT = "--client";
+  String ARG_SLIDER = "--slider";
+  String ARG_APPLICATION = "--application";
+  String ARG_YARN = "--yarn";
+  String ARG_CREDENTIALS = "--credentials";
+  String ARG_ALL = "--all";
+  String ARG_LEVEL = "--level";
   String ARG_QUEUE = "--queue";
 
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f8210a7c/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java b/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java
index 7173a85..cd981b1 100644
--- a/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java
+++ b/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java
@@ -62,6 +62,7 @@ public class ClientArgs extends CommonArgs {
   private final ActionThawArgs actionThawArgs = new ActionThawArgs();
   private final ActionVersionArgs actionVersionArgs = new ActionVersionArgs();
   private final ActionHelpArgs actionHelpArgs = new ActionHelpArgs();
+  private final ActionDiagnosticArgs actionDiagnosticArgs = new ActionDiagnosticArgs();
 
 
   public ClientArgs(String[] args) {
@@ -92,7 +93,8 @@ public class ClientArgs extends CommonArgs {
       actionThawArgs,
       actionHelpArgs,
       actionVersionArgs,
-      actionInstallPackageArgs
+      actionInstallPackageArgs,
+      actionDiagnosticArgs
               );
   }
 
@@ -112,6 +114,10 @@ public class ClientArgs extends CommonArgs {
     }
   }
 
+  public ActionDiagnosticArgs getActionDiagnosticArgs() {
+	return actionDiagnosticArgs;
+  }
+
   public AbstractClusterBuildingActionArgs getBuildingActionArgs() {
     return buildingActionArgs;
   }
@@ -240,6 +246,9 @@ public class ClientArgs extends CommonArgs {
     } else if (SliderActions.ACTION_VERSION.equals(action)) {
       bindCoreAction(actionVersionArgs);
 
+    } else if (SliderActions.ACTION_DIAGNOSTIC.equals(action)) {
+        bindCoreAction(actionDiagnosticArgs);
+
     } else if (action == null || action.isEmpty()) {
       throw new BadCommandArgumentsException(ErrorStrings.ERROR_NO_ACTION);
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f8210a7c/slider-core/src/main/java/org/apache/slider/common/params/SliderActions.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/SliderActions.java b/slider-core/src/main/java/org/apache/slider/common/params/SliderActions.java
index 964f184..29fd098 100644
--- a/slider-core/src/main/java/org/apache/slider/common/params/SliderActions.java
+++ b/slider-core/src/main/java/org/apache/slider/common/params/SliderActions.java
@@ -44,6 +44,7 @@ public interface SliderActions {
   String ACTION_THAW = "start";
   String ACTION_USAGE = "usage";
   String ACTION_VERSION = "version";
+  String ACTION_DIAGNOSTIC = "diagnostic";
   String ACTION_INSTALL_PACKAGE = "install-package";
   String DESCRIBE_ACTION_AM_SUICIDE =
     "Tell the Slider Application Master to simulate a process failure by terminating itself";
@@ -78,4 +79,7 @@ public interface SliderActions {
   String DESCRIBE_ACTION_VERSION =
                         "Print the Slider version information";
   String DESCRIBE_ACTION_INSTALL_PACKAGE = "Install the application package in the home directory under sub-folder packages";
+  String DESCRIBE_ACTION_DIAGNOSTIC = "Diagnose the configuration of the running slider application and slider client";
+  
 }
+

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f8210a7c/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 263460d..5313ab9 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
@@ -19,6 +19,7 @@
 package org.apache.slider.common.tools;
 
 import com.google.common.base.Preconditions;
+
 import org.apache.commons.compress.archivers.zip.ZipArchiveEntry;
 import org.apache.commons.compress.archivers.zip.ZipArchiveInputStream;
 import org.apache.commons.io.output.ByteArrayOutputStream;
@@ -44,6 +45,7 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 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.common.SliderKeys;
@@ -76,6 +78,8 @@ 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.util.ArrayList;
@@ -1780,4 +1784,108 @@ public final class SliderUtils {
     execCommand(OPENSSL, 0, 5000, logger, "OpenSSL", OPENSSL, "version");
     execCommand(PYTHON, 0, 5000, logger, "Python", PYTHON, "--version");
   }
+
+	/**
+	 * 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 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.CLIENT_RESOURCE);
+		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.CLIENT_RESOURCE);
+		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 open
+	 * @throws URISyntaxException
+	 */
+	public static void validateHDFSFile(SliderFileSystem sliderFileSystem, String pathStr) throws IOException, URISyntaxException{
+	  URI pathURI = new URI(pathStr);
+	  InputStream inputStream = sliderFileSystem.getFileSystem().open(new Path(pathURI));
+	  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;
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f8210a7c/slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java b/slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java
index c271e8e..b30c18c 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java
@@ -60,6 +60,11 @@ public interface AgentKeys {
    * Execution home for the agent.
    */
   String APP_HOME = "app.home";
+  String APP_ROOT = "site.global.app_root";
+  /**
+   * Runas user of the application
+   */
+  String RUNAS_USER = "site.global.app_user";
   /**
    * Name of the service.
    */