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/09 00:53:31 UTC

[42/50] [abbrv] git commit: SLIDER-365 "slider resolve" command

SLIDER-365 "slider resolve" command


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

Branch: refs/heads/develop
Commit: 032dacf4cf1757302fffc59782572f574d185cdc
Parents: c09205f
Author: Steve Loughran <st...@apache.org>
Authored: Tue Oct 7 17:03:21 2014 -0700
Committer: Steve Loughran <st...@apache.org>
Committed: Tue Oct 7 17:03:21 2014 -0700

----------------------------------------------------------------------
 .../org/apache/slider/client/SliderClient.java  | 301 +++++++------------
 .../apache/slider/client/SliderClientAPI.java   | 249 +++++++++++++++
 .../slider/common/params/ActionResolveArgs.java |   5 -
 .../apache/slider/common/params/ClientArgs.java |   4 +-
 .../apache/slider/common/tools/SliderUtils.java |  30 +-
 .../TestStandaloneYarnRegistryAM.groovy         |  40 ++-
 6 files changed, 418 insertions(+), 211 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/032dacf4/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 185e77d..c5022ee 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
@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.registry.client.exceptions.InvalidRecordException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.alias.CredentialProvider;
 import org.apache.hadoop.security.alias.CredentialProviderFactory;
@@ -140,6 +141,7 @@ import java.io.StringWriter;
 import java.io.Writer;
 import java.net.InetSocketAddress;
 import java.net.URISyntaxException;
+import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -159,7 +161,7 @@ import static org.apache.slider.common.params.SliderActions.*;
  */
 
 public class SliderClient extends AbstractSliderLaunchedService implements RunService,
-    SliderExitCodes, SliderKeys, ErrorStrings {
+    SliderExitCodes, SliderKeys, ErrorStrings, SliderClientAPI {
   private static final Logger log = LoggerFactory.getLogger(SliderClient.class);
 
   private ClientArgs serviceArgs;
@@ -197,6 +199,16 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     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);
@@ -218,9 +230,11 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
       SliderUtils.forceLogin();
       SliderUtils.initProcessSecurity(conf);
     }
-
+    AbstractActionArgs coreAction = serviceArgs.getCoreAction();
+    if (coreAction.getHadoopServicesRequired()) {
+      initHadoopBinding();
+    }
     super.serviceInit(conf);
-    
   }
 
   /**
@@ -308,11 +322,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
 
     // choose the action
     String action = serviceArgs.getAction();
-    
-    AbstractActionArgs coreAction = serviceArgs.getCoreAction();
-    if (coreAction.getHadoopServicesRequired()) {
-      initHadoopBinding();
-    }
+
     int exitCode = EXIT_SUCCESS;
     String clusterName = serviceArgs.getClusterName();
     // actions
@@ -358,9 +368,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
           serviceArgs.getActionStatusArgs());
     } else if (ACTION_UPDATE.equals(action)) {
       exitCode = actionUpdate(clusterName, serviceArgs.getActionUpdateArgs());
-
     } else if (ACTION_VERSION.equals(action)) {
-
       exitCode = actionVersion();
     } else {
       throw new SliderException(EXIT_UNIMPLEMENTED,
@@ -490,11 +498,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     return client;
   }
 
-  /**
-   * 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.
-   */
+  @Override
   public int actionDestroy(String clustername) throws YarnException,
                                                       IOException {
     // verify that a live cluster isn't there
@@ -536,11 +540,9 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     return EXIT_SUCCESS;
   }
   
-  /**
-   * AM to commit an asynchronous suicide
-   */
+  @Override
   public int actionAmSuicide(String clustername,
-                                 ActionAMSuicideArgs args) throws
+      ActionAMSuicideArgs args) throws
                                                               YarnException,
                                                               IOException {
     SliderClusterOperations clusterOperations =
@@ -549,13 +551,8 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     return EXIT_SUCCESS;
   }
 
-  /**
-   * Get the provider for this cluster
-   * @param provider the name of the provider
-   * @return the provider instance
-   * @throws SliderException problems building the provider
-   */
-  private AbstractClientProvider createClientProvider(String provider)
+  @Override
+  public AbstractClientProvider createClientProvider(String provider)
     throws SliderException {
     SliderProviderFactory factory =
       SliderProviderFactory.createSliderProviderFactory(provider);
@@ -622,17 +619,9 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     }
   }
 
-  /**
-   * 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.
-   */
+  @Override
   public int actionBuild(String clustername,
-                           AbstractClusterBuildingActionArgs buildInfo) throws
+      AbstractClusterBuildingActionArgs buildInfo) throws
                                                YarnException,
                                                IOException {
 
@@ -640,24 +629,17 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     return EXIT_SUCCESS; 
   }
 
-  /**
-   * Upload application package to user home directory
-   *
-   * @param installPkgInfo the arguments needed to build the cluster
-   * @throws YarnException Yarn problems
-   * @throws IOException other problems
-   * @throws BadCommandArgumentsException bad arguments.
-   */
+  @Override
   public int actionInstallPkg(ActionInstallPackageArgs installPkgInfo) throws
       YarnException,
       IOException {
 
     Path srcFile = null;
-    if (null == installPkgInfo.name || installPkgInfo.name.length() == 0) {
+    if (StringUtils.isEmpty(installPkgInfo.name )) {
       throw new BadCommandArgumentsException("A valid application type name is required (e.g. HBASE).");
     }
 
-    if (null == installPkgInfo.packageURI || installPkgInfo.packageURI.length() == 0) {
+    if (StringUtils.isEmpty(installPkgInfo.packageURI)) {
       throw new BadCommandArgumentsException("A valid application package location required.");
     } else {
       File pkgFile = new File(installPkgInfo.packageURI);
@@ -684,15 +666,9 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     return EXIT_SUCCESS;
   }
 
-  /**
-   * 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
-   */
-  public int actionUpdate(String clustername, AbstractClusterBuildingActionArgs buildInfo) throws
+  @Override
+  public int actionUpdate(String clustername,
+      AbstractClusterBuildingActionArgs buildInfo) throws
       YarnException, IOException {
     buildInstanceDefinition(clustername, buildInfo, true, true);
     return EXIT_SUCCESS; 
@@ -1011,10 +987,10 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
 
   /**
    *
-   * @param clustername
-   * @param clusterDirectory
-   * @param instanceDefinition
-   * @param debugAM
+   * @param clustername name of the cluster
+   * @param clusterDirectory cluster dir
+   * @param instanceDefinition the instance definition
+   * @param debugAM enable debug AM options
    * @return the launched application
    * @throws YarnException
    * @throws IOException
@@ -1579,23 +1555,14 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     return launchedApplication.monitorAppToState(desiredState, duration);
   }
 
-  /**
-   * Get the report of a this application
-   * @return the app report or null if it could not be found.
-   * @throws IOException
-   * @throws YarnException
-   */
+  @Override
   public ApplicationReport getApplicationReport() throws
                                                   IOException,
                                                   YarnException {
     return getApplicationReport(applicationId);
   }
 
-  /**
-   * Kill the submitted application via YARN
-   * @throws YarnException
-   * @throws IOException
-   */
+  @Override
   public boolean forceKillApplication(String reason)
     throws YarnException, IOException {
     if (applicationId != null) {
@@ -1616,10 +1583,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     return YarnAppListClient.listInstances();
   }
 
-  /**
-   * Implement the list action: list all nodes
-   * @return exit code of 0 if a list was created
-   */
+  @Override
   @VisibleForTesting
   public int actionList(String clustername) throws IOException, YarnException {
     verifyBindingsDefined();
@@ -1657,10 +1621,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     log.info(SliderUtils.appReportToString(report, "\n"));
   }
 
-  /**
-   * Implement the islive action: probe for a cluster of the given name existing
-   * @return exit code
-   */
+  @Override
   @VisibleForTesting
   public int actionFlex(String name, ActionFlexArgs args) throws YarnException, IOException {
     verifyBindingsDefined();
@@ -1682,11 +1643,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     return flex(name, roleInstances);
   }
 
-  /**
-   * 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
-   */
+  @Override
   @VisibleForTesting
   public int actionExists(String name, boolean checkLive) throws YarnException, IOException {
     verifyBindingsDefined();
@@ -1727,18 +1684,9 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   }
 
 
-  /**
-   * Kill a specific container of the cluster
-   * @param name cluster name
-   * @param args arguments
-   * @return exit code
-   * @throws YarnException
-   * @throws IOException
-   */
+  @Override
   public int actionKillContainer(String name,
-                                 ActionKillContainerArgs args) throws
-                                                               YarnException,
-                                                               IOException {
+      ActionKillContainerArgs args) throws YarnException, IOException {
     String id = args.id;
     if (SliderUtils.isUnset(id)) {
       throw new BadCommandArgumentsException("Missing container id");
@@ -1755,14 +1703,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     return EXIT_SUCCESS;
   }
 
-  /**
-   * Echo operation (not currently wired up to command line)
-   * @param name cluster name
-   * @param args arguments
-   * @return the echoed text
-   * @throws YarnException
-   * @throws IOException
-   */
+  @Override
   public String actionEcho(String name, ActionEchoArgs args) throws
                                                              YarnException,
                                                              IOException {
@@ -1790,15 +1731,13 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
    * @throws YarnException YARN issues
    * @throws IOException IO problems
    */
-  private ApplicationReport findInstance(String appname) throws
-                                                        YarnException,
-                                                        IOException {
+  private ApplicationReport findInstance(String appname)
+      throws YarnException, IOException {
     return YarnAppListClient.findInstance(appname);
   }
   
-  private RunningApplication findApplication(String appname) throws
-                                                                      YarnException,
-                                                                      IOException {
+  private RunningApplication findApplication(String appname)
+      throws YarnException, IOException {
     ApplicationReport applicationReport = findInstance(appname);
     return applicationReport != null ? new RunningApplication(yarnClient, applicationReport): null; 
       
@@ -1829,9 +1768,8 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
    * @throws YarnException
    * @throws IOException
    */
-  private SliderClusterProtocol connect(ApplicationReport app) throws
-                                                              YarnException,
-                                                              IOException {
+  private SliderClusterProtocol connect(ApplicationReport app)
+      throws YarnException, IOException {
 
     try {
       return RpcBinder.getProxy(getConfig(),
@@ -1846,15 +1784,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     }
   }
 
-  /**
-   * Status operation
-   *
-   * @param clustername cluster name
-   * @param statusArgs status arguments
-   * @return 0 -for success, else an exception is thrown
-   * @throws YarnException
-   * @throws IOException
-   */
+  @Override
   @VisibleForTesting
   public int actionStatus(String clustername, ActionStatusArgs statusArgs) throws
                                               YarnException,
@@ -1872,26 +1802,15 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     return EXIT_SUCCESS;
   }
 
-  /**
-   * Version Details
-   * @return exit code
-   */
+  @Override
   public int actionVersion() {
     SliderVersionInfo.loadAndPrintVersionInfo(log);
     return EXIT_SUCCESS;
   }
 
-  /**
-   * 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
-   */
+  @Override
   public int actionFreeze(String clustername,
-                          ActionFreezeArgs freezeArgs) throws
-                                                            YarnException,
-                                                            IOException {
+      ActionFreezeArgs freezeArgs) throws YarnException, IOException {
     verifyBindingsDefined();
     SliderUtils.validateClusterName(clustername);
     int waittime = freezeArgs.getWaittime();
@@ -2004,9 +1923,8 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
 
   @SuppressWarnings(
     {"UseOfSystemOutOrSystemErr", "IOResourceOpenedButNotSafelyClosed"})
-  public int actionGetConf(String clustername, ActionGetConfArgs confArgs) throws
-                                               YarnException,
-                                               IOException {
+  public int actionGetConf(String clustername, ActionGetConfArgs confArgs)
+      throws YarnException, IOException {
     File outfile = null;
     
     if (confArgs.getOutput() != null) {
@@ -2067,9 +1985,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     return EXIT_SUCCESS;
   }
 
-  /**
-   * Restore a cluster
-   */
+  @Override
   public int actionThaw(String clustername, ActionThawArgs thaw) throws YarnException, IOException {
     SliderUtils.validateClusterName(clustername);
     // see if it is actually running and bail out;
@@ -2089,10 +2005,8 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
    * @throws YarnException
    * @throws IOException
    */
-  public int flex(String clustername,
-                  Map<String, Integer> roleInstances) throws
-                                   YarnException,
-                                   IOException {
+  public int flex(String clustername, Map<String, Integer> roleInstances)
+      throws YarnException, IOException {
     verifyBindingsDefined();
     SliderUtils.validateClusterName(clustername);
     Path clusterDirectory = sliderFileSystem.buildClusterDirPath(clustername);
@@ -2382,58 +2296,67 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   }
 
 
-  /**
-   * 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
-   */
-  public int actionResolve(ActionResolveArgs args) throws
-      YarnException,
-      IOException {
-    // as this is also a test entry point, validate
+  @Override
+  public int actionResolve(ActionResolveArgs args)
+      throws YarnException, IOException {
+    // as this is an API entry point, validate
     // the arguments
     args.validate();
     RegistryOperations operations = getRegistryOperations();
     String path = args.path;
     Collection<ServiceRecord> serviceRecords;
+    ServiceRecordMarshal serviceRecordMarshal = new ServiceRecordMarshal();
+    File outputPath = args.out;
     try {
       if (args.list) {
         Map<String, ServiceRecord> recordMap =
             listServiceRecords(operations, path);
         serviceRecords = recordMap.values();
         // list records out
+        StringBuilder builder = new StringBuilder(1024);
+        for (Entry<String, ServiceRecord> recordEntry : recordMap
+            .entrySet()) {
+          ServiceRecord instance = recordEntry.getValue();
+          builder.append("\"").append(recordEntry.getKey()).append("\":\n");
+          builder.append(serviceRecordMarshal.toJson(instance));
+          builder.append("}\n");
+        }
+        String records = builder.toString();
+        if (outputPath == null) {
+          print(records);
+        } else {
+          SliderUtils.write(outputPath, records.getBytes("UTF-8"), false);
+        }
       } else  {
+        // resolve single entry
         ServiceRecord instance = resolve(path);
         serviceRecords = new ArrayList<ServiceRecord>(1);
         serviceRecords.add(instance);
-        // list or save records
+        // write out JSON content
+        if (outputPath != null) {
+          byte[] data = serviceRecordMarshal.toBytes(instance);
+          SliderUtils.write(outputPath, data, false);
+        } else {
+          // print to the console
+          print(serviceRecordMarshal.toJson(instance));
+        }
       }
 //      JDK7
-    } catch (FileNotFoundException e) {
-      log.info("{}", e);
-      log.debug("{}", e, e);
-      return EXIT_NOT_FOUND;
     } catch (PathNotFoundException e) {
-      log.info("{}", e);
-      log.debug("{}", e, e);
+      // no record at this path
+      return EXIT_NOT_FOUND;
+    } catch (NoRecordException e) {
       return EXIT_NOT_FOUND;
+    } catch (InvalidRecordException e) {
+      // it is not a record
+      log.error("{}", e);
+      log.debug("{}", e, e);
+      return EXIT_EXCEPTION_THROWN;
     }
     return EXIT_SUCCESS;
   }
 
-  /**
-   * 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
-   */
+  @Override
   public int actionRegistry(ActionRegistryArgs registryArgs) throws
       YarnException,
       IOException {
@@ -2514,22 +2437,8 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     return serviceRecords;
   }
 
-	/**
-	 * 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
-	 */
-	public int actionDiagnostic(ActionDiagnosticArgs diagnosticArgs) {
+	@Override
+  public int actionDiagnostic(ActionDiagnosticArgs diagnosticArgs) {
 		try {
 			if (diagnosticArgs.client) {
 				actionDiagnosticClient();
@@ -2862,9 +2771,11 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   }
 
   /**
-   * write out the config
-   * @param published
-   * @param registryArgs
+   * write out the config. If a destination is provided and that dir is a
+   * directory, the entry is written to it with the name provided + extension,
+   * else it is printed to standard out.
+   * @param published published config
+   * @param registryArgs registry Arguments
    * @throws BadCommandArgumentsException
    * @throws IOException
    */
@@ -3007,11 +2918,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     return registryOperations;
   }
 
-  /**
-   * Get the registry binding. As this may start the registry, it can take time
-   * and fail
-   * @return the registry 
-   */
+  @Override
   public RegistryOperations getRegistryOperations()
       throws SliderException, IOException {
     return maybeStartYarnRegistry();

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/032dacf4/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
new file mode 100644
index 0000000..cacf962
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderClientAPI.java
@@ -0,0 +1,249 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 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.exceptions.YarnException;
+import org.apache.slider.common.params.AbstractClusterBuildingActionArgs;
+import org.apache.slider.common.params.ActionAMSuicideArgs;
+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.ActionInstallPackageArgs;
+import org.apache.slider.common.params.ActionKillContainerArgs;
+import org.apache.slider.common.params.ActionRegistryArgs;
+import org.apache.slider.common.params.ActionResolveArgs;
+import org.apache.slider.common.params.ActionStatusArgs;
+import org.apache.slider.common.params.ActionThawArgs;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.providers.AbstractClientProvider;
+
+import java.io.IOException;
+
+/**
+ * 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) 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 application package to user home directory
+   *
+   * @param installPkgInfo the arguments needed to build the cluster
+   * @throws YarnException Yarn problems
+   * @throws IOException other problems
+   * @throws BadCommandArgumentsException bad arguments.
+   */
+  int actionInstallPkg(ActionInstallPackageArgs installPkgInfo)
+      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; 
+  /**
+   * 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
+   */
+  @VisibleForTesting
+  int actionList(String clustername) throws IOException, YarnException;
+
+  /**
+   * Implement the islive action: probe for a cluster of the given name existing
+   * @return exit code
+   */
+  @VisibleForTesting
+  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
+   */
+  @VisibleForTesting
+  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
+   */
+  @VisibleForTesting
+  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 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
+   */
+  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;
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/032dacf4/slider-core/src/main/java/org/apache/slider/common/params/ActionResolveArgs.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/ActionResolveArgs.java b/slider-core/src/main/java/org/apache/slider/common/params/ActionResolveArgs.java
index 9b6a5f1..cf5611c 100644
--- a/slider-core/src/main/java/org/apache/slider/common/params/ActionResolveArgs.java
+++ b/slider-core/src/main/java/org/apache/slider/common/params/ActionResolveArgs.java
@@ -83,10 +83,5 @@ public class ActionResolveArgs extends AbstractActionArgs {
  @Parameter(names = {ARG_VERBOSE},
       description = "verbose output")
   public boolean verbose;
-
-  @Parameter(names = {ARG_INTERNAL},
-      description = "fetch internal registry entries")
-  public boolean internal;
-
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/032dacf4/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 0fffffb..16363d2 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
@@ -191,7 +191,7 @@ public class ClientArgs extends CommonArgs {
    * Look at the chosen action and bind it as the core action for the operation.
    * In theory this could be done by introspecting on the list of actions and 
    * choosing it without the switch statement. In practise this switch, while
-   * verbose, is easier to debug.
+   * verbose, is easier to debug. And in JDK7, much simpler.
    * @throws SliderException bad argument or similar
    */
   @Override
@@ -206,7 +206,7 @@ public class ClientArgs extends CommonArgs {
       //its a builder, so set those actions too
       buildingActionArgs = actionCreateArgs;
 
-    }else if (SliderActions.ACTION_FREEZE.equals(action)) {
+    } else if (SliderActions.ACTION_FREEZE.equals(action)) {
       bindCoreAction(actionFreezeArgs);
 
     } else if (SliderActions.ACTION_THAW.equals(action)) {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/032dacf4/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 9c93753..72aa4fe 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
@@ -69,6 +69,7 @@ import org.slf4j.LoggerFactory;
 import java.io.ByteArrayInputStream;
 import java.io.File;
 import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
 import java.io.FileReader;
 import java.io.IOException;
 import java.io.InputStream;
@@ -1664,8 +1665,8 @@ public final class SliderUtils {
 
   /**
    * Validate an executable
-   * @param program
-   * @param exe
+   * @param program program name for errors
+   * @param exe program to look at
    * @throws IOException
    */
   public static void validateExe(String program, File exe) throws IOException {
@@ -1676,6 +1677,29 @@ public final class SliderUtils {
     }
   }
 
+  /**
+   * 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.getParentFile();
+    if (createParent) {
+      parentDir.mkdirs();
+    }
+    SliderUtils.verifyIsDir(parentDir, log);
+    FileOutputStream out = new FileOutputStream(outfile);
+    try {
+      out.write(data);
+    } finally {
+      IOUtils.closeStream(out);
+    }
+
+  }
 
   /**
    * Execute a command for a test operation
@@ -1683,7 +1707,7 @@ public final class SliderUtils {
    * @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 outputString optional string to grep for (must not span a line)
    * @param commands commands   @return the process
    * @throws IOException on any failure.
    */

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/032dacf4/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneYarnRegistryAM.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneYarnRegistryAM.groovy b/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneYarnRegistryAM.groovy
index 1f65d2f..97e995d 100644
--- a/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneYarnRegistryAM.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneYarnRegistryAM.groovy
@@ -29,6 +29,7 @@ import org.apache.hadoop.registry.client.impl.RegistryOperationsClient
 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.slider.common.params.ActionResolveArgs
 import org.apache.slider.core.exceptions.UnknownApplicationInstanceException
 
 import static org.apache.hadoop.registry.client.binding.RegistryUtils.*
@@ -109,9 +110,6 @@ class TestStandaloneYarnRegistryAM extends AgentMiniClusterTestBase {
     ClusterNode master = nodes[0]
     assert master.role == SliderKeys.COMPONENT_AM
 
-
-
-
     String username = client.username
     def yarnRegistryClient = client.yarnAppListClient
     describe("list of all applications")
@@ -174,12 +172,47 @@ class TestStandaloneYarnRegistryAM extends AgentMiniClusterTestBase {
     assert null != serviceRecord.getInternalEndpoint(AGENT_ONEWAY_REST_API)
     assert null != serviceRecord.getInternalEndpoint(AGENT_SECURE_REST_API)
 
+    // use the resolve operation
+    describe "resolve CLI action"
+    ActionResolveArgs resolveArgs = new ActionResolveArgs()
+    resolveArgs.path = recordsPath;
+    resolveArgs.list = true;
+    // to stdout
+    client.actionResolve(resolveArgs)
+    // to a file
+    File destFile = new File("target/resolve.json")
+    destFile.delete()
+    resolveArgs.out = destFile
+    client.actionResolve(resolveArgs)
+    assert destFile.exists()
+    destFile.delete()
+    
+    // look at a single record
+    resolveArgs.out = null;
+    resolveArgs.list = false;
+    resolveArgs.path = recordsPath +"/"+ clustername
+    // to stdout
+    client.actionResolve(resolveArgs)
+    resolveArgs.out = destFile
+    client.actionResolve(resolveArgs)
+    assert destFile.exists()
+    ServiceRecordMarshal serviceRecordMarshal = new ServiceRecordMarshal()
+    def recordFromFile = serviceRecordMarshal.fromFile(destFile)
+    assert recordFromFile[YarnRegistryAttributes.YARN_ID] ==
+           serviceRecord[YarnRegistryAttributes.YARN_ID]
+    assert recordFromFile[YarnRegistryAttributes.YARN_PERSISTENCE] ==
+           serviceRecord[YarnRegistryAttributes.YARN_PERSISTENCE]
+    
+    
+
     // hit the registry web page
     def registryEndpoint = serviceRecord.getExternalEndpoint(
         CustomRegistryConstants.REGISTRY_REST_API)
     assert registryEndpoint != null
     def registryURL = RegistryTypeUtils.retrieveAddressURLs(registryEndpoint)[0]
     
+    
+    // Look at the Registry WADL
     describe("Registry WADL @ $registryURL")
     def publisherEndpoint = serviceRecord.getExternalEndpoint(
         CustomRegistryConstants.PUBLISHER_REST_API)
@@ -393,6 +426,5 @@ class TestStandaloneYarnRegistryAM extends AgentMiniClusterTestBase {
     assert oldInstance.yarnApplicationState >= YarnApplicationState.FINISHED
 
 
-
   }
 }